diff --git a/LICENSE.txt b/LICENSE.txt index ff818bafa12..ebabcf9862c 100644 --- a/LICENSE.txt +++ b/LICENSE.txt @@ -588,6 +588,8 @@ Mockito 1.8.5 JUL to SLF4J bridge 1.7.25 SLF4J API Module 1.7.25 SLF4J LOG4J-12 Binding 1.7.25 +JCodings 1.0.8 +Joni 2.1.2 -------------------------------------------------------------------------------- The MIT License (MIT) @@ -1874,6 +1876,9 @@ be bound by any additional provisions that may appear in any communication from You. This License may not be modified without the mutual written agreement of the Licensor and You. +The binary distribution of this product bundles these dependencies under the +following license: +jamon-runtime 2.4.1 -------------------------------------------------------------------------------- For: hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs diff --git a/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml b/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml index a3a05e4eeb5..220c1270bd9 100644 --- a/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml +++ b/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml @@ -188,9 +188,38 @@ ${project.build.directory}/site /share/doc/hadoop/${hadoop.component} + + + hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/target/lib + share/hadoop/${hadoop.component}/timelineservice/lib + + + org.apache.hadoop:hadoop-yarn-server-timelineservice + org.apache.hadoop:hadoop-yarn-server-timelineservice-hbase + + + share/hadoop/${hadoop.component}/timelineservice + false + false + + + + + org.apache.hadoop:hadoop-yarn-server-timelineservice-hbase-tests + + + share/hadoop/${hadoop.component}/timelineservice/test + false + false + + + + + org.apache.hadoop:hadoop-yarn-server-timelineservice* + share/hadoop/${hadoop.component} false diff --git a/hadoop-common-project/hadoop-common/src/main/conf/hadoop-policy.xml b/hadoop-common-project/hadoop-common/src/main/conf/hadoop-policy.xml index 2bf5c02e110..d282c5841c1 100644 --- a/hadoop-common-project/hadoop-common/src/main/conf/hadoop-policy.xml +++ b/hadoop-common-project/hadoop-common/src/main/conf/hadoop-policy.xml @@ -223,4 +223,15 @@ group list is separated by a blank. For e.g. "alice,bob users,wheel". A special value of "*" means all users are allowed. + + + security.collector-nodemanager.protocol.acl + * + ACL for CollectorNodemanagerProtocol, used by nodemanager + if timeline service v2 is enabled, for the timeline collector and nodemanager + to communicate with each other. + The ACL is a comma-separated list of user and group names. The user and + group list is separated by a blank. For e.g. "alice,bob users,wheel". + A special value of "*" means all users are allowed. + diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md b/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md index a7ded2400ff..05b18b59298 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md @@ -128,6 +128,7 @@ REST API compatibility corresponds to both the requests (URLs) and responses to * [MR Application Master](../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapredAppMasterRest.html) * [History Server](../../hadoop-mapreduce-client/hadoop-mapreduce-client-hs/HistoryServerRest.html) * [Timeline Server v1 REST API](../../hadoop-yarn/hadoop-yarn-site/TimelineServer.html) +* [Timeline Service v2 REST API](../../hadoop-yarn/hadoop-yarn-site/TimelineServiceV2.html) #### Policy diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java index b2ef0dbd15c..51f35387944 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java @@ -25,10 +25,12 @@ import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.Timer; import java.util.TimerTask; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience.Private; @@ -44,16 +46,17 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.TaskStatus; import org.apache.hadoop.mapreduce.Counter; -import org.apache.hadoop.mapreduce.CounterGroup; import org.apache.hadoop.mapreduce.Counters; import org.apache.hadoop.mapreduce.JobCounter; import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.TaskType; import org.apache.hadoop.mapreduce.TypeConverter; import org.apache.hadoop.mapreduce.util.MRJobConfUtil; +import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils; import org.apache.hadoop.mapreduce.v2.api.records.JobId; import org.apache.hadoop.mapreduce.v2.api.records.JobState; import org.apache.hadoop.mapreduce.v2.app.AppContext; +import org.apache.hadoop.mapreduce.v2.app.MRAppMaster; import org.apache.hadoop.mapreduce.v2.app.job.Job; import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal; import org.apache.hadoop.mapreduce.v2.jobhistory.FileNameIndexUtils; @@ -67,16 +70,15 @@ import org.apache.hadoop.util.StringUtils; 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.TimelinePutResponse; +import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; import org.apache.hadoop.yarn.client.api.TimelineClient; +import org.apache.hadoop.yarn.client.api.TimelineV2Client; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; -import org.codehaus.jackson.JsonNode; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.node.ArrayNode; -import org.codehaus.jackson.node.JsonNodeFactory; -import org.codehaus.jackson.node.ObjectNode; +import org.apache.hadoop.yarn.util.TimelineServiceHelper; import com.google.common.annotations.VisibleForTesting; import com.sun.jersey.api.client.ClientHandlerException; @@ -89,8 +91,6 @@ import com.sun.jersey.api.client.ClientHandlerException; */ public class JobHistoryEventHandler extends AbstractService implements EventHandler { - private static final JsonNodeFactory FACTORY = - new ObjectMapper().getNodeFactory(); private final AppContext context; private final int startCount; @@ -132,10 +132,15 @@ public class JobHistoryEventHandler extends AbstractService // should job completion be force when the AM shuts down? protected volatile boolean forceJobCompletion = false; + @VisibleForTesting protected TimelineClient timelineClient; + @VisibleForTesting + protected TimelineV2Client timelineV2Client; private static String MAPREDUCE_JOB_ENTITY_TYPE = "MAPREDUCE_JOB"; private static String MAPREDUCE_TASK_ENTITY_TYPE = "MAPREDUCE_TASK"; + private static final String MAPREDUCE_TASK_ATTEMPT_ENTITY_TYPE = + "MAPREDUCE_TASK_ATTEMPT"; public JobHistoryEventHandler(AppContext context, int startCount) { super("JobHistoryEventHandler"); @@ -255,19 +260,33 @@ public class JobHistoryEventHandler extends AbstractService MRJobConfig.MR_AM_HISTORY_USE_BATCHED_FLUSH_QUEUE_SIZE_THRESHOLD, MRJobConfig.DEFAULT_MR_AM_HISTORY_USE_BATCHED_FLUSH_QUEUE_SIZE_THRESHOLD); + // TODO replace MR specific configurations on timeline service with getting + // configuration from RM through registerApplicationMaster() in + // ApplicationMasterProtocol with return value for timeline service + // configuration status: off, on_with_v1 or on_with_v2. if (conf.getBoolean(MRJobConfig.MAPREDUCE_JOB_EMIT_TIMELINE_DATA, MRJobConfig.DEFAULT_MAPREDUCE_JOB_EMIT_TIMELINE_DATA)) { - if (conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, - YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED)) { - timelineClient = TimelineClient.createTimelineClient(); - timelineClient.init(conf); - LOG.info("Timeline service is enabled"); - LOG.info("Emitting job history data to the timeline server is enabled"); + LOG.info("Emitting job history data to the timeline service is enabled"); + if (YarnConfiguration.timelineServiceEnabled(conf)) { + boolean timelineServiceV2Enabled = + ((int) YarnConfiguration.getTimelineServiceVersion(conf) == 2); + if(timelineServiceV2Enabled) { + timelineV2Client = + ((MRAppMaster.RunningAppContext)context).getTimelineV2Client(); + timelineV2Client.init(conf); + } else { + timelineClient = + ((MRAppMaster.RunningAppContext) context).getTimelineClient(); + timelineClient.init(conf); + } + LOG.info("Timeline service is enabled; version: " + + YarnConfiguration.getTimelineServiceVersion(conf)); } else { LOG.info("Timeline service is not enabled"); } } else { - LOG.info("Emitting job history data to the timeline server is not enabled"); + LOG.info("Emitting job history data to the timeline server is not " + + "enabled"); } // Flag for setting @@ -310,6 +329,8 @@ public class JobHistoryEventHandler extends AbstractService protected void serviceStart() throws Exception { if (timelineClient != null) { timelineClient.start(); + } else if (timelineV2Client != null) { + timelineV2Client.start(); } eventHandlingThread = new Thread(new Runnable() { @Override @@ -434,6 +455,8 @@ public class JobHistoryEventHandler extends AbstractService } if (timelineClient != null) { timelineClient.stop(); + } else if (timelineV2Client != null) { + timelineV2Client.stop(); } LOG.info("Stopped JobHistoryEventHandler. super.stop()"); super.serviceStop(); @@ -591,7 +614,10 @@ public class JobHistoryEventHandler extends AbstractService } processEventForJobSummary(event.getHistoryEvent(), mi.getJobSummary(), event.getJobID()); - if (timelineClient != null) { + if (timelineV2Client != null) { + processEventForNewTimelineService(historyEvent, event.getJobID(), + event.getTimestamp()); + } else if (timelineClient != null) { processEventForTimelineServer(historyEvent, event.getJobID(), event.getTimestamp()); } @@ -835,11 +861,11 @@ public class JobHistoryEventHandler extends AbstractService tEvent.addEventInfo("FINISHED_MAPS", jfe.getFinishedMaps()); tEvent.addEventInfo("FINISHED_REDUCES", jfe.getFinishedReduces()); tEvent.addEventInfo("MAP_COUNTERS_GROUPS", - countersToJSON(jfe.getMapCounters())); + JobHistoryEventUtils.countersToJSON(jfe.getMapCounters())); tEvent.addEventInfo("REDUCE_COUNTERS_GROUPS", - countersToJSON(jfe.getReduceCounters())); + JobHistoryEventUtils.countersToJSON(jfe.getReduceCounters())); tEvent.addEventInfo("TOTAL_COUNTERS_GROUPS", - countersToJSON(jfe.getTotalCounters())); + JobHistoryEventUtils.countersToJSON(jfe.getTotalCounters())); tEvent.addEventInfo("JOB_STATUS", JobState.SUCCEEDED.toString()); tEntity.addEvent(tEvent); tEntity.setEntityId(jobId.toString()); @@ -865,7 +891,7 @@ public class JobHistoryEventHandler extends AbstractService tfe.getFailedAttemptID() == null ? "" : tfe.getFailedAttemptID().toString()); tEvent.addEventInfo("COUNTERS_GROUPS", - countersToJSON(tfe.getCounters())); + JobHistoryEventUtils.countersToJSON(tfe.getCounters())); tEntity.addEvent(tEvent); tEntity.setEntityId(tfe.getTaskId().toString()); tEntity.setEntityType(MAPREDUCE_TASK_ENTITY_TYPE); @@ -883,7 +909,7 @@ public class JobHistoryEventHandler extends AbstractService TaskFinishedEvent tfe2 = (TaskFinishedEvent) event; tEvent.addEventInfo("TASK_TYPE", tfe2.getTaskType().toString()); tEvent.addEventInfo("COUNTERS_GROUPS", - countersToJSON(tfe2.getCounters())); + JobHistoryEventUtils.countersToJSON(tfe2.getCounters())); tEvent.addEventInfo("FINISH_TIME", tfe2.getFinishTime()); tEvent.addEventInfo("STATUS", TaskStatus.State.SUCCEEDED.toString()); tEvent.addEventInfo("SUCCESSFUL_TASK_ATTEMPT_ID", @@ -905,7 +931,6 @@ public class JobHistoryEventHandler extends AbstractService tEvent.addEventInfo("START_TIME", tase.getStartTime()); tEvent.addEventInfo("HTTP_PORT", tase.getHttpPort()); tEvent.addEventInfo("TRACKER_NAME", tase.getTrackerName()); - tEvent.addEventInfo("TASK_TYPE", tase.getTaskType().toString()); tEvent.addEventInfo("SHUFFLE_PORT", tase.getShufflePort()); tEvent.addEventInfo("CONTAINER_ID", tase.getContainerId() == null ? "" : tase.getContainerId().toString()); @@ -938,7 +963,7 @@ public class JobHistoryEventHandler extends AbstractService tEvent.addEventInfo("SORT_FINISH_TIME", tauce.getFinishTime()); tEvent.addEventInfo("MAP_FINISH_TIME", tauce.getFinishTime()); tEvent.addEventInfo("COUNTERS_GROUPS", - countersToJSON(tauce.getCounters())); + JobHistoryEventUtils.countersToJSON(tauce.getCounters())); tEntity.addEvent(tEvent); tEntity.setEntityId(tauce.getTaskId().toString()); tEntity.setEntityType(MAPREDUCE_TASK_ENTITY_TYPE); @@ -952,7 +977,7 @@ public class JobHistoryEventHandler extends AbstractService tEvent.addEventInfo("STATE", mafe.getState()); tEvent.addEventInfo("MAP_FINISH_TIME", mafe.getMapFinishTime()); tEvent.addEventInfo("COUNTERS_GROUPS", - countersToJSON(mafe.getCounters())); + JobHistoryEventUtils.countersToJSON(mafe.getCounters())); tEvent.addEventInfo("HOSTNAME", mafe.getHostname()); tEvent.addEventInfo("PORT", mafe.getPort()); tEvent.addEventInfo("RACK_NAME", mafe.getRackName()); @@ -974,7 +999,7 @@ public class JobHistoryEventHandler extends AbstractService tEvent.addEventInfo("SHUFFLE_FINISH_TIME", rafe.getShuffleFinishTime()); tEvent.addEventInfo("SORT_FINISH_TIME", rafe.getSortFinishTime()); tEvent.addEventInfo("COUNTERS_GROUPS", - countersToJSON(rafe.getCounters())); + JobHistoryEventUtils.countersToJSON(rafe.getCounters())); tEvent.addEventInfo("HOSTNAME", rafe.getHostname()); tEvent.addEventInfo("PORT", rafe.getPort()); tEvent.addEventInfo("RACK_NAME", rafe.getRackName()); @@ -993,7 +1018,7 @@ public class JobHistoryEventHandler extends AbstractService tEvent.addEventInfo("STATUS", tafe.getTaskStatus()); tEvent.addEventInfo("STATE", tafe.getState()); tEvent.addEventInfo("COUNTERS_GROUPS", - countersToJSON(tafe.getCounters())); + JobHistoryEventUtils.countersToJSON(tafe.getCounters())); tEvent.addEventInfo("HOSTNAME", tafe.getHostname()); tEntity.addEvent(tEvent); tEntity.setEntityId(tafe.getTaskId().toString()); @@ -1043,24 +1068,272 @@ public class JobHistoryEventHandler extends AbstractService } } - @Private - public JsonNode countersToJSON(Counters counters) { - ArrayNode nodes = FACTORY.arrayNode(); - if (counters != null) { - for (CounterGroup counterGroup : counters) { - ObjectNode groupNode = nodes.addObject(); - groupNode.put("NAME", counterGroup.getName()); - groupNode.put("DISPLAY_NAME", counterGroup.getDisplayName()); - ArrayNode countersNode = groupNode.putArray("COUNTERS"); - for (Counter counter : counterGroup) { - ObjectNode counterNode = countersNode.addObject(); - counterNode.put("NAME", counter.getName()); - counterNode.put("DISPLAY_NAME", counter.getDisplayName()); - counterNode.put("VALUE", counter.getValue()); + // create JobEntity from HistoryEvent with adding other info, like: + // jobId, timestamp and entityType. + private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity + createJobEntity(HistoryEvent event, long timestamp, JobId jobId, + String entityType, boolean setCreatedTime) { + + org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity = + createBaseEntity(event, timestamp, entityType, setCreatedTime); + entity.setId(jobId.toString()); + return entity; + } + + private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity + createJobEntity(JobId jobId) { + org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity = + new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity(); + entity.setId(jobId.toString()); + entity.setType(MAPREDUCE_JOB_ENTITY_TYPE); + return entity; + } + + // create ApplicationEntity with job finished Metrics from HistoryEvent + private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity + createAppEntityWithJobMetrics(HistoryEvent event, JobId jobId) { + ApplicationEntity entity = new ApplicationEntity(); + entity.setId(jobId.getAppId().toString()); + entity.setMetrics(event.getTimelineMetrics()); + return entity; + } + + // create BaseEntity from HistoryEvent with adding other info, like: + // timestamp and entityType. + private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity + createBaseEntity(HistoryEvent event, long timestamp, String entityType, + boolean setCreatedTime) { + org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent tEvent = + event.toTimelineEvent(); + tEvent.setTimestamp(timestamp); + + org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity = + new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity(); + entity.addEvent(tEvent); + entity.setType(entityType); + if (setCreatedTime) { + entity.setCreatedTime(timestamp); + } + Set timelineMetrics = event.getTimelineMetrics(); + if (timelineMetrics != null) { + entity.setMetrics(timelineMetrics); + } + return entity; + } + + // create TaskEntity from HistoryEvent with adding other info, like: + // taskId, jobId, timestamp, entityType and relatedJobEntity. + private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity + createTaskEntity(HistoryEvent event, long timestamp, String taskId, + String entityType, String relatedJobEntity, JobId jobId, + boolean setCreatedTime, long taskIdPrefix) { + org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity = + createBaseEntity(event, timestamp, entityType, setCreatedTime); + entity.setId(taskId); + if (event.getEventType() == EventType.TASK_STARTED) { + entity.addInfo("TASK_TYPE", + ((TaskStartedEvent)event).getTaskType().toString()); + } + entity.addIsRelatedToEntity(relatedJobEntity, jobId.toString()); + entity.setIdPrefix(taskIdPrefix); + return entity; + } + + // create TaskAttemptEntity from HistoryEvent with adding other info, like: + // timestamp, taskAttemptId, entityType, relatedTaskEntity and taskId. + private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity + createTaskAttemptEntity(HistoryEvent event, long timestamp, + String taskAttemptId, String entityType, String relatedTaskEntity, + String taskId, boolean setCreatedTime, long taskAttemptIdPrefix) { + org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity = + createBaseEntity(event, timestamp, entityType, setCreatedTime); + entity.setId(taskAttemptId); + entity.addIsRelatedToEntity(relatedTaskEntity, taskId); + entity.setIdPrefix(taskAttemptIdPrefix); + return entity; + } + + private void publishConfigsOnJobSubmittedEvent(JobSubmittedEvent event, + JobId jobId) { + if (event.getJobConf() == null) { + return; + } + // Publish job configurations both as job and app entity. + // Configs are split into multiple entities if they exceed 100kb in size. + org.apache.hadoop.yarn.api.records.timelineservice. + TimelineEntity jobEntityForConfigs = createJobEntity(jobId); + ApplicationEntity appEntityForConfigs = new ApplicationEntity(); + String appId = jobId.getAppId().toString(); + appEntityForConfigs.setId(appId); + try { + int configSize = 0; + for (Map.Entry entry : event.getJobConf()) { + int size = entry.getKey().length() + entry.getValue().length(); + configSize += size; + if (configSize > JobHistoryEventUtils.ATS_CONFIG_PUBLISH_SIZE_BYTES) { + if (jobEntityForConfigs.getConfigs().size() > 0) { + timelineV2Client.putEntities(jobEntityForConfigs); + timelineV2Client.putEntities(appEntityForConfigs); + jobEntityForConfigs = createJobEntity(jobId); + appEntityForConfigs = new ApplicationEntity(); + appEntityForConfigs.setId(appId); + } + configSize = size; } + jobEntityForConfigs.addConfig(entry.getKey(), entry.getValue()); + appEntityForConfigs.addConfig(entry.getKey(), entry.getValue()); + } + if (configSize > 0) { + timelineV2Client.putEntities(jobEntityForConfigs); + timelineV2Client.putEntities(appEntityForConfigs); + } + } catch (IOException | YarnException e) { + LOG.error("Exception while publishing configs on JOB_SUBMITTED Event " + + " for the job : " + jobId, e); + } + } + + private void processEventForNewTimelineService(HistoryEvent event, + JobId jobId, long timestamp) { + org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity tEntity = + null; + String taskId = null; + String taskAttemptId = null; + boolean setCreatedTime = false; + long taskIdPrefix = 0; + long taskAttemptIdPrefix = 0; + + switch (event.getEventType()) { + // Handle job events + case JOB_SUBMITTED: + setCreatedTime = true; + break; + case JOB_STATUS_CHANGED: + case JOB_INFO_CHANGED: + case JOB_INITED: + case JOB_PRIORITY_CHANGED: + case JOB_QUEUE_CHANGED: + case JOB_FAILED: + case JOB_KILLED: + case JOB_ERROR: + case JOB_FINISHED: + case AM_STARTED: + case NORMALIZED_RESOURCE: + break; + // Handle task events + case TASK_STARTED: + setCreatedTime = true; + taskId = ((TaskStartedEvent)event).getTaskId().toString(); + taskIdPrefix = TimelineServiceHelper. + invertLong(((TaskStartedEvent)event).getStartTime()); + break; + case TASK_FAILED: + taskId = ((TaskFailedEvent)event).getTaskId().toString(); + taskIdPrefix = TimelineServiceHelper. + invertLong(((TaskFailedEvent)event).getStartTime()); + break; + case TASK_UPDATED: + taskId = ((TaskUpdatedEvent)event).getTaskId().toString(); + break; + case TASK_FINISHED: + taskId = ((TaskFinishedEvent)event).getTaskId().toString(); + taskIdPrefix = TimelineServiceHelper. + invertLong(((TaskFinishedEvent)event).getStartTime()); + break; + case MAP_ATTEMPT_STARTED: + case REDUCE_ATTEMPT_STARTED: + setCreatedTime = true; + taskId = ((TaskAttemptStartedEvent)event).getTaskId().toString(); + taskAttemptId = ((TaskAttemptStartedEvent)event). + getTaskAttemptId().toString(); + taskAttemptIdPrefix = TimelineServiceHelper. + invertLong(((TaskAttemptStartedEvent)event).getStartTime()); + break; + case CLEANUP_ATTEMPT_STARTED: + case SETUP_ATTEMPT_STARTED: + taskId = ((TaskAttemptStartedEvent)event).getTaskId().toString(); + taskAttemptId = ((TaskAttemptStartedEvent)event). + getTaskAttemptId().toString(); + break; + case MAP_ATTEMPT_FAILED: + case CLEANUP_ATTEMPT_FAILED: + case REDUCE_ATTEMPT_FAILED: + case SETUP_ATTEMPT_FAILED: + case MAP_ATTEMPT_KILLED: + case CLEANUP_ATTEMPT_KILLED: + case REDUCE_ATTEMPT_KILLED: + case SETUP_ATTEMPT_KILLED: + taskId = ((TaskAttemptUnsuccessfulCompletionEvent)event). + getTaskId().toString(); + taskAttemptId = ((TaskAttemptUnsuccessfulCompletionEvent)event). + getTaskAttemptId().toString(); + taskAttemptIdPrefix = TimelineServiceHelper.invertLong( + ((TaskAttemptUnsuccessfulCompletionEvent)event).getStartTime()); + break; + case MAP_ATTEMPT_FINISHED: + taskId = ((MapAttemptFinishedEvent)event).getTaskId().toString(); + taskAttemptId = ((MapAttemptFinishedEvent)event). + getAttemptId().toString(); + taskAttemptIdPrefix = TimelineServiceHelper. + invertLong(((MapAttemptFinishedEvent)event).getStartTime()); + break; + case REDUCE_ATTEMPT_FINISHED: + taskId = ((ReduceAttemptFinishedEvent)event).getTaskId().toString(); + taskAttemptId = ((ReduceAttemptFinishedEvent)event). + getAttemptId().toString(); + taskAttemptIdPrefix = TimelineServiceHelper. + invertLong(((ReduceAttemptFinishedEvent)event).getStartTime()); + break; + case SETUP_ATTEMPT_FINISHED: + case CLEANUP_ATTEMPT_FINISHED: + taskId = ((TaskAttemptFinishedEvent)event).getTaskId().toString(); + taskAttemptId = ((TaskAttemptFinishedEvent)event). + getAttemptId().toString(); + break; + default: + LOG.warn("EventType: " + event.getEventType() + " cannot be recognized" + + " and handled by timeline service."); + return; + } + + org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity + appEntityWithJobMetrics = null; + if (taskId == null) { + // JobEntity + tEntity = createJobEntity(event, timestamp, jobId, + MAPREDUCE_JOB_ENTITY_TYPE, setCreatedTime); + if (event.getEventType() == EventType.JOB_FINISHED + && event.getTimelineMetrics() != null) { + appEntityWithJobMetrics = createAppEntityWithJobMetrics(event, jobId); + } + } else { + if (taskAttemptId == null) { + // TaskEntity + tEntity = createTaskEntity(event, timestamp, taskId, + MAPREDUCE_TASK_ENTITY_TYPE, MAPREDUCE_JOB_ENTITY_TYPE, + jobId, setCreatedTime, taskIdPrefix); + } else { + // TaskAttemptEntity + tEntity = createTaskAttemptEntity(event, timestamp, taskAttemptId, + MAPREDUCE_TASK_ATTEMPT_ENTITY_TYPE, MAPREDUCE_TASK_ENTITY_TYPE, + taskId, setCreatedTime, taskAttemptIdPrefix); } } - return nodes; + try { + if (appEntityWithJobMetrics == null) { + timelineV2Client.putEntitiesAsync(tEntity); + } else { + timelineV2Client.putEntities(tEntity, appEntityWithJobMetrics); + } + } catch (IOException | YarnException e) { + LOG.error("Failed to process Event " + event.getEventType() + + " for the job : " + jobId, e); + return; + } + if (event.getEventType() == EventType.JOB_SUBMITTED) { + // Publish configs after main job submitted event has been posted. + publishConfigsOnJobSubmittedEvent((JobSubmittedEvent)event, jobId); + } } private void setSummarySlotSeconds(JobSummary summary, Counters allCounters) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java index 16fa7942fe0..1f8be12afdb 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java @@ -38,6 +38,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Matcher; import java.util.regex.Pattern; +import javax.crypto.KeyGenerator; + import org.apache.commons.io.IOUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -137,6 +139,8 @@ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment; 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.client.api.TimelineClient; +import org.apache.hadoop.yarn.client.api.TimelineV2Client; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.event.Dispatcher; @@ -150,8 +154,6 @@ import org.apache.hadoop.yarn.util.SystemClock; import com.google.common.annotations.VisibleForTesting; -import javax.crypto.KeyGenerator; - /** * The Map-Reduce Application Master. * The state machine is encapsulated in the implementation of Job interface. @@ -1057,6 +1059,8 @@ public class MRAppMaster extends CompositeService { private final ClusterInfo clusterInfo = new ClusterInfo(); private final ClientToAMTokenSecretManager clientToAMTokenSecretManager; private String historyUrl = null; + private TimelineClient timelineClient = null; + private TimelineV2Client timelineV2Client = null; private final TaskAttemptFinishingMonitor taskAttemptFinishingMonitor; @@ -1066,6 +1070,18 @@ public class MRAppMaster extends CompositeService { this.clientToAMTokenSecretManager = new ClientToAMTokenSecretManager(appAttemptID, null); this.taskAttemptFinishingMonitor = taskAttemptFinishingMonitor; + if (conf.getBoolean(MRJobConfig.MAPREDUCE_JOB_EMIT_TIMELINE_DATA, + MRJobConfig.DEFAULT_MAPREDUCE_JOB_EMIT_TIMELINE_DATA) + && YarnConfiguration.timelineServiceEnabled(conf)) { + + if (YarnConfiguration.timelineServiceV2Enabled(conf)) { + // create new version TimelineClient + timelineV2Client = TimelineV2Client.createTimelineClient( + appAttemptID.getApplicationId()); + } else { + timelineClient = TimelineClient.createTimelineClient(); + } + } } @Override @@ -1165,6 +1181,15 @@ public class MRAppMaster extends CompositeService { public void setHistoryUrl(String historyUrl) { this.historyUrl = historyUrl; } + + public TimelineClient getTimelineClient() { + return timelineClient; + } + + // Get Timeline Collector's address (get sync from RM) + public TimelineV2Client getTimelineV2Client() { + return timelineV2Client; + } } @SuppressWarnings("unchecked") diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java index 81bc3ce8859..49951208ced 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java @@ -1465,7 +1465,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job, job.conf.get(MRJobConfig.WORKFLOW_NAME, ""), job.conf.get(MRJobConfig.WORKFLOW_NODE_NAME, ""), getWorkflowAdjacencies(job.conf), - job.conf.get(MRJobConfig.WORKFLOW_TAGS, "")); + job.conf.get(MRJobConfig.WORKFLOW_TAGS, ""), job.conf); job.eventHandler.handle(new JobHistoryEvent(job.jobId, jse)); //TODO JH Verify jobACLs, UserName via UGI? diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java index d3451248433..270fbe33cfe 100755 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java @@ -1580,7 +1580,7 @@ public abstract class TaskAttemptImpl implements StringUtils.join( LINE_SEPARATOR, taskAttempt.getDiagnostics()), taskAttempt.getCounters(), taskAttempt - .getProgressSplitBlock().burst()); + .getProgressSplitBlock().burst(), taskAttempt.launchTime); return tauce; } @@ -1993,35 +1993,35 @@ public abstract class TaskAttemptImpl implements this.container == null ? -1 : this.container.getNodeId().getPort(); if (attemptId.getTaskId().getTaskType() == TaskType.MAP) { MapAttemptFinishedEvent mfe = - new MapAttemptFinishedEvent(TypeConverter.fromYarn(attemptId), - TypeConverter.fromYarn(attemptId.getTaskId().getTaskType()), - state.toString(), - this.reportedStatus.mapFinishTime, - finishTime, - containerHostName, - containerNodePort, - this.nodeRackName == null ? "UNKNOWN" : this.nodeRackName, - this.reportedStatus.stateString, - getCounters(), - getProgressSplitBlock().burst()); - eventHandler.handle( - new JobHistoryEvent(attemptId.getTaskId().getJobId(), mfe)); + new MapAttemptFinishedEvent(TypeConverter.fromYarn(attemptId), + TypeConverter.fromYarn(attemptId.getTaskId().getTaskType()), + state.toString(), + this.reportedStatus.mapFinishTime, + finishTime, + containerHostName, + containerNodePort, + this.nodeRackName == null ? "UNKNOWN" : this.nodeRackName, + this.reportedStatus.stateString, + getCounters(), + getProgressSplitBlock().burst(), launchTime); + eventHandler.handle( + new JobHistoryEvent(attemptId.getTaskId().getJobId(), mfe)); } else { - ReduceAttemptFinishedEvent rfe = - new ReduceAttemptFinishedEvent(TypeConverter.fromYarn(attemptId), - TypeConverter.fromYarn(attemptId.getTaskId().getTaskType()), - state.toString(), - this.reportedStatus.shuffleFinishTime, - this.reportedStatus.sortFinishTime, - finishTime, - containerHostName, - containerNodePort, - this.nodeRackName == null ? "UNKNOWN" : this.nodeRackName, - this.reportedStatus.stateString, - getCounters(), - getProgressSplitBlock().burst()); - eventHandler.handle( - new JobHistoryEvent(attemptId.getTaskId().getJobId(), rfe)); + ReduceAttemptFinishedEvent rfe = + new ReduceAttemptFinishedEvent(TypeConverter.fromYarn(attemptId), + TypeConverter.fromYarn(attemptId.getTaskId().getTaskType()), + state.toString(), + this.reportedStatus.shuffleFinishTime, + this.reportedStatus.sortFinishTime, + finishTime, + containerHostName, + containerNodePort, + this.nodeRackName == null ? "UNKNOWN" : this.nodeRackName, + this.reportedStatus.stateString, + getCounters(), + getProgressSplitBlock().burst(), launchTime); + eventHandler.handle( + new JobHistoryEvent(attemptId.getTaskId().getJobId(), rfe)); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java index 8a6fa304d4e..228ae24a955 100755 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java @@ -139,6 +139,8 @@ public abstract class TaskImpl implements Task, EventHandler { private final Set inProgressAttempts; private boolean historyTaskStartGenerated = false; + // Launch time reported in history events. + private long launchTime; private static final SingleArcTransition ATTEMPT_KILLED_TRANSITION = new AttemptKilledTransition(); @@ -705,8 +707,9 @@ public abstract class TaskImpl implements Task, EventHandler { } private void sendTaskStartedEvent() { + launchTime = getLaunchTime(); TaskStartedEvent tse = new TaskStartedEvent( - TypeConverter.fromYarn(taskId), getLaunchTime(), + TypeConverter.fromYarn(taskId), launchTime, TypeConverter.fromYarn(taskId.getTaskType()), getSplitsAsString()); eventHandler @@ -714,18 +717,19 @@ public abstract class TaskImpl implements Task, EventHandler { historyTaskStartGenerated = true; } - private static TaskFinishedEvent createTaskFinishedEvent(TaskImpl task, TaskStateInternal taskState) { + private static TaskFinishedEvent createTaskFinishedEvent(TaskImpl task, + TaskStateInternal taskState) { TaskFinishedEvent tfe = new TaskFinishedEvent(TypeConverter.fromYarn(task.taskId), TypeConverter.fromYarn(task.successfulAttempt), task.getFinishTime(task.successfulAttempt), TypeConverter.fromYarn(task.taskId.getTaskType()), - taskState.toString(), - task.getCounters()); + taskState.toString(), task.getCounters(), task.launchTime); return tfe; } - private static TaskFailedEvent createTaskFailedEvent(TaskImpl task, List diag, TaskStateInternal taskState, TaskAttemptId taId) { + private static TaskFailedEvent createTaskFailedEvent(TaskImpl task, + List diag, TaskStateInternal taskState, TaskAttemptId taId) { StringBuilder errorSb = new StringBuilder(); if (diag != null) { for (String d : diag) { @@ -740,7 +744,7 @@ public abstract class TaskImpl implements Task, EventHandler { errorSb.toString(), taskState.toString(), taId == null ? null : TypeConverter.fromYarn(taId), - task.getCounters()); + task.getCounters(), task.launchTime); return taskFailedEvent; } @@ -861,7 +865,8 @@ public abstract class TaskImpl implements Task, EventHandler { TaskFailedEvent tfe = new TaskFailedEvent(taskInfo.getTaskId(), taskInfo.getFinishTime(), taskInfo.getTaskType(), taskInfo.getError(), taskInfo.getTaskStatus(), - taskInfo.getFailedDueToAttemptId(), taskInfo.getCounters()); + taskInfo.getFailedDueToAttemptId(), taskInfo.getCounters(), + launchTime); eventHandler.handle(new JobHistoryEvent(taskId.getJobId(), tfe)); eventHandler.handle( new JobTaskEvent(taskId, getExternalState(taskState))); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java index 3300bd210a2..543b20ec015 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java @@ -48,6 +48,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.JobId; import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId; import org.apache.hadoop.mapreduce.v2.api.records.TaskType; import org.apache.hadoop.mapreduce.v2.app.AppContext; +import org.apache.hadoop.mapreduce.v2.app.MRAppMaster; import org.apache.hadoop.mapreduce.v2.app.client.ClientService; import org.apache.hadoop.mapreduce.v2.app.job.event.JobCounterUpdateEvent; import org.apache.hadoop.mapreduce.v2.app.job.event.JobDiagnosticsUpdateEvent; @@ -837,7 +838,8 @@ public class RMContainerAllocator extends RMContainerRequestor updateAMRMToken(response.getAMRMToken()); } - List finishedContainers = response.getCompletedContainersStatuses(); + List finishedContainers = + response.getCompletedContainersStatuses(); if (newContainers.size() + finishedContainers.size() > 0 || !headRoom.equals(newHeadRoom)) { //something changed @@ -858,7 +860,13 @@ public class RMContainerAllocator extends RMContainerRequestor handleUpdatedNodes(response); handleJobPriorityChange(response); - + // Handle receiving the timeline collector address and token for this app. + MRAppMaster.RunningAppContext appContext = + (MRAppMaster.RunningAppContext)this.getContext(); + if (appContext.getTimelineV2Client() != null) { + appContext.getTimelineV2Client(). + setTimelineCollectorInfo(response.getCollectorInfo()); + } for (ContainerStatus cont : finishedContainers) { processFinishedContainer(cont); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java index 7612ceb0718..e2713191ac7 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java @@ -18,13 +18,15 @@ package org.apache.hadoop.mapreduce.jobhistory; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.util.ArrayList; import java.util.Arrays; - -import static org.junit.Assert.*; +import java.util.Set; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -35,6 +37,8 @@ import org.apache.hadoop.mapreduce.TaskAttemptID; import org.apache.hadoop.mapreduce.TaskID; import org.apache.hadoop.mapreduce.TaskType; import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; import org.junit.Test; public class TestEvents { @@ -54,7 +58,7 @@ public class TestEvents { Counters counters = new Counters(); TaskAttemptFinishedEvent test = new TaskAttemptFinishedEvent(taskAttemptId, TaskType.REDUCE, "TEST", 123L, "RAKNAME", "HOSTNAME", "STATUS", - counters); + counters, 234); assertEquals(test.getAttemptId().toString(), taskAttemptId.toString()); assertEquals(test.getCounters(), counters); @@ -65,7 +69,7 @@ public class TestEvents { assertEquals(test.getTaskId(), tid); assertEquals(test.getTaskStatus(), "TEST"); assertEquals(test.getTaskType(), TaskType.REDUCE); - + assertEquals(234, test.getStartTime()); } /** @@ -405,6 +409,16 @@ public class TestEvents { this.datum = datum; } + @Override + public TimelineEvent toTimelineEvent() { + return null; + } + + @Override + public Set getTimelineMetrics() { + return null; + } + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java index 255eb22d670..51ac2ce1d12 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java @@ -32,8 +32,8 @@ import static org.mockito.Mockito.when; import java.io.File; import java.io.FileOutputStream; -import java.io.InputStream; import java.io.IOException; +import java.io.InputStream; import java.util.HashMap; import org.apache.commons.logging.Log; @@ -56,8 +56,10 @@ import org.apache.hadoop.mapreduce.TaskID; import org.apache.hadoop.mapreduce.TaskType; import org.apache.hadoop.mapreduce.TypeConverter; import org.apache.hadoop.mapreduce.util.MRJobConfUtil; +import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils; import org.apache.hadoop.mapreduce.v2.api.records.JobId; import org.apache.hadoop.mapreduce.v2.app.AppContext; +import org.apache.hadoop.mapreduce.v2.app.MRAppMaster.RunningAppContext; import org.apache.hadoop.mapreduce.v2.app.job.Job; import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal; import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig; @@ -70,6 +72,8 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities; import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity; +import org.apache.hadoop.yarn.client.api.TimelineClient; +import org.apache.hadoop.yarn.client.api.TimelineV2Client; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.server.MiniYARNCluster; @@ -143,7 +147,7 @@ public class TestJobHistoryEventHandler { // First completion event, but min-queue-size for batching flushes is 10 handleEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent( - t.taskID, t.taskAttemptID, 0, TaskType.MAP, "", null))); + t.taskID, t.taskAttemptID, 0, TaskType.MAP, "", null, 0))); verify(mockWriter).flush(); } finally { @@ -179,7 +183,7 @@ public class TestJobHistoryEventHandler { for (int i = 0 ; i < 100 ; i++) { queueEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent( - t.taskID, t.taskAttemptID, 0, TaskType.MAP, "", null))); + t.taskID, t.taskAttemptID, 0, TaskType.MAP, "", null, 0))); } handleNextNEvents(jheh, 9); @@ -224,7 +228,7 @@ public class TestJobHistoryEventHandler { for (int i = 0 ; i < 100 ; i++) { queueEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent( - t.taskID, t.taskAttemptID, 0, TaskType.MAP, "", null))); + t.taskID, t.taskAttemptID, 0, TaskType.MAP, "", null, 0))); } handleNextNEvents(jheh, 9); @@ -267,7 +271,7 @@ public class TestJobHistoryEventHandler { for (int i = 0 ; i < 100 ; i++) { queueEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent( - t.taskID, t.taskAttemptID, 0, TaskType.MAP, "", null))); + t.taskID, t.taskAttemptID, 0, TaskType.MAP, "", null, 0))); } queueEvent(jheh, new JobHistoryEvent(t.jobId, new JobFinishedEvent( TypeConverter.fromYarn(t.jobId), 0, 10, 10, 0, 0, null, null, new Counters()))); @@ -560,7 +564,7 @@ public class TestJobHistoryEventHandler { // stored to the Timeline store @Test (timeout=50000) public void testTimelineEventHandling() throws Exception { - TestParams t = new TestParams(false); + TestParams t = new TestParams(RunningAppContext.class, false); Configuration conf = new YarnConfiguration(); conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); long currentTime = System.currentTimeMillis(); @@ -738,7 +742,7 @@ public class TestJobHistoryEventHandler { group2.addCounter("MARTHA_JONES", "Martha Jones", 3); group2.addCounter("DONNA_NOBLE", "Donna Noble", 2); group2.addCounter("ROSE_TYLER", "Rose Tyler", 1); - JsonNode jsonNode = jheh.countersToJSON(counters); + JsonNode jsonNode = JobHistoryEventUtils.countersToJSON(counters); String jsonStr = new ObjectMapper().writeValueAsString(jsonNode); String expected = "[{\"NAME\":\"COMPANIONS\",\"DISPLAY_NAME\":\"Companions " + "of the Doctor\",\"COUNTERS\":[{\"NAME\":\"AMY_POND\",\"DISPLAY_NAME\"" @@ -761,19 +765,19 @@ public class TestJobHistoryEventHandler { public void testCountersToJSONEmpty() throws Exception { JobHistoryEventHandler jheh = new JobHistoryEventHandler(null, 0); Counters counters = null; - JsonNode jsonNode = jheh.countersToJSON(counters); + JsonNode jsonNode = JobHistoryEventUtils.countersToJSON(counters); String jsonStr = new ObjectMapper().writeValueAsString(jsonNode); String expected = "[]"; Assert.assertEquals(expected, jsonStr); counters = new Counters(); - jsonNode = jheh.countersToJSON(counters); + jsonNode = JobHistoryEventUtils.countersToJSON(counters); jsonStr = new ObjectMapper().writeValueAsString(jsonNode); expected = "[]"; Assert.assertEquals(expected, jsonStr); counters.addGroup("DOCTORS", "Incarnations of the Doctor"); - jsonNode = jheh.countersToJSON(counters); + jsonNode = JobHistoryEventUtils.countersToJSON(counters); jsonStr = new ObjectMapper().writeValueAsString(jsonNode); expected = "[{\"NAME\":\"DOCTORS\",\"DISPLAY_NAME\":\"Incarnations of the " + "Doctor\",\"COUNTERS\":[]}]"; @@ -809,21 +813,33 @@ public class TestJobHistoryEventHandler { } } - private AppContext mockAppContext(ApplicationId appId, boolean isLastAMRetry) { - JobId jobId = TypeConverter.toYarn(TypeConverter.fromYarn(appId)); - AppContext mockContext = mock(AppContext.class); + private Job mockJob() { Job mockJob = mock(Job.class); when(mockJob.getAllCounters()).thenReturn(new Counters()); when(mockJob.getTotalMaps()).thenReturn(10); when(mockJob.getTotalReduces()).thenReturn(10); when(mockJob.getName()).thenReturn("mockjob"); + return mockJob; + } + + private AppContext mockAppContext(Class contextClass, + ApplicationId appId, boolean isLastAMRetry) { + JobId jobId = TypeConverter.toYarn(TypeConverter.fromYarn(appId)); + AppContext mockContext = mock(contextClass); + Job mockJob = mockJob(); when(mockContext.getJob(jobId)).thenReturn(mockJob); when(mockContext.getApplicationID()).thenReturn(appId); when(mockContext.isLastAMRetry()).thenReturn(isLastAMRetry); + if (mockContext instanceof RunningAppContext) { + when(((RunningAppContext)mockContext).getTimelineClient()). + thenReturn(TimelineClient.createTimelineClient()); + when(((RunningAppContext) mockContext).getTimelineV2Client()) + .thenReturn(TimelineV2Client + .createTimelineClient(ApplicationId.newInstance(0, 1))); + } return mockContext; } - private class TestParams { boolean isLastAMRetry; String workDir = setupTestWorkDir(); @@ -838,11 +854,15 @@ public class TestJobHistoryEventHandler { AppContext mockAppContext; public TestParams() { - this(false); + this(AppContext.class, false); } public TestParams(boolean isLastAMRetry) { + this(AppContext.class, isLastAMRetry); + } + public TestParams(Class contextClass, + boolean isLastAMRetry) { this.isLastAMRetry = isLastAMRetry; - mockAppContext = mockAppContext(appId, this.isLastAMRetry); + mockAppContext = mockAppContext(contextClass, appId, this.isLastAMRetry); } } @@ -1022,6 +1042,8 @@ class JHEvenHandlerForTest extends JobHistoryEventHandler { protected void serviceStart() { if (timelineClient != null) { timelineClient.start(); + } else if (timelineV2Client != null) { + timelineV2Client.start(); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java index 3fa004389b0..9549b6880c4 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java @@ -297,7 +297,7 @@ public class TestLocalContainerAllocator { Resources.none(), null, 1, null, Collections.emptyList(), yarnToken, - Collections.emptyList()); + Collections.emptyList(), null); response.setApplicationPriority(Priority.newInstance(0)); return response; } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java index f7f03cfec02..4ee0a148ee9 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java @@ -18,6 +18,7 @@ package org.apache.hadoop.mapreduce.v2.app.rm; +import static org.junit.Assert.assertEquals; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyFloat; import static org.mockito.Matchers.anyInt; @@ -27,6 +28,7 @@ import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.inOrder; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -61,6 +63,7 @@ import org.apache.hadoop.mapreduce.v2.app.AppContext; import org.apache.hadoop.mapreduce.v2.app.ClusterInfo; import org.apache.hadoop.mapreduce.v2.app.MRApp; import org.apache.hadoop.mapreduce.v2.app.MRAppMaster; +import org.apache.hadoop.mapreduce.v2.app.MRAppMaster.RunningAppContext; import org.apache.hadoop.mapreduce.v2.app.client.ClientService; import org.apache.hadoop.mapreduce.v2.app.job.Job; import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal; @@ -97,6 +100,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRespo import org.apache.hadoop.yarn.api.records.ApplicationAccessType; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.CollectorInfo; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerExitStatus; import org.apache.hadoop.yarn.api.records.ContainerId; @@ -108,6 +112,7 @@ import org.apache.hadoop.yarn.api.records.NodeReport; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.client.api.TimelineV2Client; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.event.DrainDispatcher; @@ -119,6 +124,7 @@ import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes; import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; +import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier; import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse; import org.apache.hadoop.yarn.server.api.records.NodeAction; import org.apache.hadoop.yarn.server.resourcemanager.MockNM; @@ -138,6 +144,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretMan import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.Clock; import org.apache.hadoop.yarn.util.ControlledClock; +import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.SystemClock; import org.junit.After; import org.junit.Assert; @@ -746,6 +753,96 @@ public class TestRMContainerAllocator { } } + @Test + public void testUpdateCollectorInfo() throws Exception { + LOG.info("Running testUpdateCollectorInfo"); + Configuration conf = new Configuration(); + conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); + conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f); + ApplicationId appId = ApplicationId.newInstance(1, 1); + ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1); + JobId jobId = MRBuilderUtils.newJobId(appId, 0); + Job mockJob = mock(Job.class); + when(mockJob.getReport()).thenReturn( + MRBuilderUtils.newJobReport(jobId, "job", "user", JobState.RUNNING, 0, + 0, 0, 0, 0, 0, 0, "jobfile", null, false, "")); + String localAddr = "localhost:1234"; + UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); + + // Generate a timeline delegation token. + TimelineDelegationTokenIdentifier ident = + new TimelineDelegationTokenIdentifier(new Text(ugi.getUserName()), + new Text("renewer"), null); + ident.setSequenceNumber(1); + Token collectorToken = + new Token(ident.getBytes(), + new byte[0], TimelineDelegationTokenIdentifier.KIND_NAME, + new Text(localAddr)); + org.apache.hadoop.yarn.api.records.Token token = + org.apache.hadoop.yarn.api.records.Token.newInstance( + collectorToken.getIdentifier(), collectorToken.getKind().toString(), + collectorToken.getPassword(), + collectorToken.getService().toString()); + CollectorInfo collectorInfo = CollectorInfo.newInstance(localAddr, token); + // Mock scheduler to server Allocate request. + final MockSchedulerForTimelineCollector mockScheduler = + new MockSchedulerForTimelineCollector(collectorInfo); + MyContainerAllocator allocator = + new MyContainerAllocator(null, conf, attemptId, mockJob, + SystemClock.getInstance()) { + @Override + protected void register() { + } + + @Override + protected ApplicationMasterProtocol createSchedulerProxy() { + return mockScheduler; + } + }; + // Initially UGI should have no tokens. + ArrayList> tokens = + new ArrayList<>(ugi.getTokens()); + assertEquals(0, tokens.size()); + TimelineV2Client client = spy(TimelineV2Client.createTimelineClient(appId)); + client.init(conf); + when(((RunningAppContext)allocator.getContext()).getTimelineV2Client()). + thenReturn(client); + + // Send allocate request to RM and fetch collector address and token. + allocator.schedule(); + verify(client).setTimelineCollectorInfo(collectorInfo); + // Verify if token has been updated in UGI. + tokens = new ArrayList<>(ugi.getTokens()); + assertEquals(1, tokens.size()); + assertEquals(TimelineDelegationTokenIdentifier.KIND_NAME, + tokens.get(0).getKind()); + assertEquals(collectorToken.decodeIdentifier(), + tokens.get(0).decodeIdentifier()); + + // Generate new collector token, send allocate request to RM and fetch the + // new token. + ident.setSequenceNumber(100); + Token collectorToken1 = + new Token(ident.getBytes(), + new byte[0], TimelineDelegationTokenIdentifier.KIND_NAME, + new Text(localAddr)); + token = org.apache.hadoop.yarn.api.records.Token.newInstance( + collectorToken1.getIdentifier(), collectorToken1.getKind().toString(), + collectorToken1.getPassword(), collectorToken1.getService().toString()); + collectorInfo = CollectorInfo.newInstance(localAddr, token); + mockScheduler.updateCollectorInfo(collectorInfo); + allocator.schedule(); + verify(client).setTimelineCollectorInfo(collectorInfo); + // Verify if new token has been updated in UGI. + tokens = new ArrayList<>(ugi.getTokens()); + assertEquals(1, tokens.size()); + assertEquals(TimelineDelegationTokenIdentifier.KIND_NAME, + tokens.get(0).getKind()); + assertEquals(collectorToken1.decodeIdentifier(), + tokens.get(0).decodeIdentifier()); + allocator.close(); + } + @Test public void testMapReduceScheduling() throws Exception { @@ -1869,7 +1966,7 @@ public class TestRMContainerAllocator { private AllocateResponse allocateResponse; private static AppContext createAppContext( ApplicationAttemptId appAttemptId, Job job) { - AppContext context = mock(AppContext.class); + AppContext context = mock(RunningAppContext.class); ApplicationId appId = appAttemptId.getApplicationId(); when(context.getApplicationID()).thenReturn(appId); when(context.getApplicationAttemptId()).thenReturn(appAttemptId); @@ -3481,6 +3578,46 @@ public class TestRMContainerAllocator { } } + private final static class MockSchedulerForTimelineCollector + implements ApplicationMasterProtocol { + private CollectorInfo collectorInfo; + + private MockSchedulerForTimelineCollector(CollectorInfo info) { + this.collectorInfo = info; + } + + private void updateCollectorInfo(CollectorInfo info) { + collectorInfo = info; + } + + @Override + public RegisterApplicationMasterResponse registerApplicationMaster( + RegisterApplicationMasterRequest request) throws YarnException, + IOException { + return Records.newRecord(RegisterApplicationMasterResponse.class); + } + + @Override + public FinishApplicationMasterResponse finishApplicationMaster( + FinishApplicationMasterRequest request) throws YarnException, + IOException { + return FinishApplicationMasterResponse.newInstance(false); + } + + @Override + public AllocateResponse allocate(AllocateRequest request) + throws YarnException, IOException { + AllocateResponse response = AllocateResponse.newInstance( + request.getResponseId(), Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), + Resource.newInstance(512000, 1024), null, 10, null, + Collections.emptyList()); + response.setCollectorInfo(collectorInfo); + return response; + } + } + public static void main(String[] args) throws Exception { TestRMContainerAllocator t = new TestRMContainerAllocator(); t.testSimple(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java index 266aa94f0cb..a1447c58b46 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java @@ -18,13 +18,17 @@ package org.apache.hadoop.mapreduce.jobhistory; -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.util.ConverterUtils; +import java.util.Set; import org.apache.avro.util.Utf8; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.util.ConverterUtils; /** * Event to record start of a task attempt @@ -166,4 +170,24 @@ public class AMStartedEvent implements HistoryEvent { public EventType getEventType() { return EventType.AM_STARTED; } + + @Override + public TimelineEvent toTimelineEvent() { + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(StringUtils.toUpperCase(getEventType().name())); + tEvent.addInfo("APPLICATION_ATTEMPT_ID", + getAppAttemptId() == null ? "" : getAppAttemptId().toString()); + tEvent.addInfo("CONTAINER_ID", getContainerId() == null ? + "" : getContainerId().toString()); + tEvent.addInfo("NODE_MANAGER_HOST", getNodeManagerHost()); + tEvent.addInfo("NODE_MANAGER_PORT", getNodeManagerPort()); + tEvent.addInfo("NODE_MANAGER_HTTP_PORT", getNodeManagerHttpPort()); + tEvent.addInfo("START_TIME", getStartTime()); + return tEvent; + } + + @Override + public Set getTimelineMetrics() { + return null; + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HistoryEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HistoryEvent.java index a30748cd651..1ba7195ec2e 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HistoryEvent.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HistoryEvent.java @@ -18,8 +18,12 @@ package org.apache.hadoop.mapreduce.jobhistory; +import java.util.Set; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; /** * Interface for event wrapper classes. Implementations each wrap an @@ -37,4 +41,18 @@ public interface HistoryEvent { /** Set the Avro datum wrapped by this. */ void setDatum(Object datum); + + /** + * Map HistoryEvent to TimelineEvent. + * + * @return the timeline event + */ + TimelineEvent toTimelineEvent(); + + /** + * Counters or Metrics if any else return null. + * + * @return the set of timeline metrics + */ + Set getTimelineMetrics(); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobFinishedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobFinishedEvent.java index 0fa5868bcfb..ea21f603412 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobFinishedEvent.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobFinishedEvent.java @@ -18,11 +18,17 @@ package org.apache.hadoop.mapreduce.jobhistory; +import java.util.Set; + import org.apache.avro.util.Utf8; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.mapreduce.Counters; import org.apache.hadoop.mapreduce.JobID; +import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; /** * Event to record successful completion of job @@ -133,4 +139,31 @@ public class JobFinishedEvent implements HistoryEvent { public Counters getReduceCounters() { return reduceCounters; } + + @Override + public TimelineEvent toTimelineEvent() { + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(StringUtils.toUpperCase(getEventType().name())); + tEvent.addInfo("FINISH_TIME", getFinishTime()); + tEvent.addInfo("NUM_MAPS", getFinishedMaps()); + tEvent.addInfo("NUM_REDUCES", getFinishedReduces()); + tEvent.addInfo("FAILED_MAPS", getFailedMaps()); + tEvent.addInfo("FAILED_REDUCES", getFailedReduces()); + tEvent.addInfo("FINISHED_MAPS", getFinishedMaps()); + tEvent.addInfo("FINISHED_REDUCES", getFinishedReduces()); + // TODO replace SUCCEEDED with JobState.SUCCEEDED.toString() + tEvent.addInfo("JOB_STATUS", "SUCCEEDED"); + return tEvent; + } + + @Override + public Set getTimelineMetrics() { + Set jobMetrics = JobHistoryEventUtils. + countersToTimelineMetric(getTotalCounters(), finishTime); + jobMetrics.addAll(JobHistoryEventUtils. + countersToTimelineMetric(getMapCounters(), finishTime, "MAP:")); + jobMetrics.addAll(JobHistoryEventUtils. + countersToTimelineMetric(getReduceCounters(), finishTime, "REDUCE:")); + return jobMetrics; + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInfoChangeEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInfoChangeEvent.java index b45f373cd0f..f5941aa0447 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInfoChangeEvent.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInfoChangeEvent.java @@ -18,13 +18,15 @@ package org.apache.hadoop.mapreduce.jobhistory; -import java.io.IOException; +import java.util.Set; +import org.apache.avro.util.Utf8; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.mapreduce.JobID; - -import org.apache.avro.util.Utf8; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; /** * Event to record changes in the submit and launch time of @@ -65,4 +67,17 @@ public class JobInfoChangeEvent implements HistoryEvent { return EventType.JOB_INFO_CHANGED; } + @Override + public TimelineEvent toTimelineEvent() { + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(StringUtils.toUpperCase(getEventType().name())); + tEvent.addInfo("SUBMIT_TIME", getSubmitTime()); + tEvent.addInfo("LAUNCH_TIME", getLaunchTime()); + return tEvent; + } + + @Override + public Set getTimelineMetrics() { + return null; + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInitedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInitedEvent.java index 5abb40e6bb1..784267f2b0a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInitedEvent.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInitedEvent.java @@ -18,11 +18,15 @@ package org.apache.hadoop.mapreduce.jobhistory; +import java.util.Set; + +import org.apache.avro.util.Utf8; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.mapreduce.JobID; - -import org.apache.avro.util.Utf8; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; /** * Event to record the initialization of a job @@ -73,4 +77,21 @@ public class JobInitedEvent implements HistoryEvent { } /** Get whether the job's map and reduce stages were combined */ public boolean getUberized() { return datum.getUberized(); } + + @Override + public TimelineEvent toTimelineEvent() { + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(StringUtils.toUpperCase(getEventType().name())); + tEvent.addInfo("START_TIME", getLaunchTime()); + tEvent.addInfo("STATUS", getStatus()); + tEvent.addInfo("TOTAL_MAPS", getTotalMaps()); + tEvent.addInfo("TOTAL_REDUCES", getTotalReduces()); + tEvent.addInfo("UBERIZED", getUberized()); + return tEvent; + } + + @Override + public Set getTimelineMetrics() { + return null; + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobPriorityChangeEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobPriorityChangeEvent.java index 6c324628c13..1616dd1962b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobPriorityChangeEvent.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobPriorityChangeEvent.java @@ -18,14 +18,16 @@ package org.apache.hadoop.mapreduce.jobhistory; -import java.io.IOException; +import java.util.Set; +import org.apache.avro.util.Utf8; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.mapred.JobPriority; import org.apache.hadoop.mapreduce.JobID; - -import org.apache.avro.util.Utf8; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; /** * Event to record the change of priority of a job @@ -65,4 +67,17 @@ public class JobPriorityChangeEvent implements HistoryEvent { return EventType.JOB_PRIORITY_CHANGED; } + @Override + public TimelineEvent toTimelineEvent() { + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(StringUtils.toUpperCase(getEventType().name())); + tEvent.addInfo("PRIORITY", getPriority().toString()); + return tEvent; + } + + @Override + public Set getTimelineMetrics() { + return null; + } + } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobQueueChangeEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobQueueChangeEvent.java index 86078e6b12c..66f37812398 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobQueueChangeEvent.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobQueueChangeEvent.java @@ -18,8 +18,13 @@ package org.apache.hadoop.mapreduce.jobhistory; +import java.util.Set; + import org.apache.avro.util.Utf8; import org.apache.hadoop.mapreduce.JobID; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; @SuppressWarnings("deprecation") public class JobQueueChangeEvent implements HistoryEvent { @@ -60,4 +65,17 @@ public class JobQueueChangeEvent implements HistoryEvent { return null; } + @Override + public TimelineEvent toTimelineEvent() { + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(StringUtils.toUpperCase(getEventType().name())); + tEvent.addInfo("QUEUE_NAMES", getJobQueueName()); + return tEvent; + } + + @Override + public Set getTimelineMetrics() { + return null; + } + } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobStatusChangedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobStatusChangedEvent.java index 9e11a55256c..0963b45b492 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobStatusChangedEvent.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobStatusChangedEvent.java @@ -18,13 +18,15 @@ package org.apache.hadoop.mapreduce.jobhistory; -import java.io.IOException; +import java.util.Set; +import org.apache.avro.util.Utf8; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.mapreduce.JobID; - -import org.apache.avro.util.Utf8; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; /** * Event to record the change of status for a job @@ -61,4 +63,17 @@ public class JobStatusChangedEvent implements HistoryEvent { return EventType.JOB_STATUS_CHANGED; } + @Override + public TimelineEvent toTimelineEvent() { + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(StringUtils.toUpperCase(getEventType().name())); + tEvent.addInfo("STATUS", getStatus()); + return tEvent; + } + + @Override + public Set getTimelineMetrics() { + return null; + } + } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java index 845f6f7c253..e394f5bbf6e 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java @@ -21,14 +21,18 @@ package org.apache.hadoop.mapreduce.jobhistory; import java.util.HashMap; import java.util.Map; import java.util.Map.Entry; +import java.util.Set; +import org.apache.avro.util.Utf8; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapreduce.JobACL; import org.apache.hadoop.mapreduce.JobID; import org.apache.hadoop.security.authorize.AccessControlList; - -import org.apache.avro.util.Utf8; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; /** * Event to record the submission of a job @@ -38,6 +42,7 @@ import org.apache.avro.util.Utf8; @InterfaceStability.Unstable public class JobSubmittedEvent implements HistoryEvent { private JobSubmitted datum = new JobSubmitted(); + private JobConf jobConf = null; /** * Create an event to record job submission @@ -80,6 +85,31 @@ public class JobSubmittedEvent implements HistoryEvent { workflowAdjacencies, ""); } + /** + * Create an event to record job submission. + * @param id The job Id of the job + * @param jobName Name of the job + * @param userName Name of the user who submitted the job + * @param submitTime Time of submission + * @param jobConfPath Path of the Job Configuration file + * @param jobACLs The configured acls for the job. + * @param jobQueueName The job-queue to which this job was submitted to + * @param workflowId The Id of the workflow + * @param workflowName The name of the workflow + * @param workflowNodeName The node name of the workflow + * @param workflowAdjacencies The adjacencies of the workflow + * @param workflowTags Comma-separated tags for the workflow + */ + public JobSubmittedEvent(JobID id, String jobName, String userName, + long submitTime, String jobConfPath, + Map jobACLs, String jobQueueName, + String workflowId, String workflowName, String workflowNodeName, + String workflowAdjacencies, String workflowTags) { + this(id, jobName, userName, submitTime, jobConfPath, jobACLs, + jobQueueName, workflowId, workflowName, workflowNodeName, + workflowAdjacencies, workflowTags, null); + } + /** * Create an event to record job submission * @param id The job Id of the job @@ -94,12 +124,13 @@ public class JobSubmittedEvent implements HistoryEvent { * @param workflowNodeName The node name of the workflow * @param workflowAdjacencies The adjacencies of the workflow * @param workflowTags Comma-separated tags for the workflow + * @param conf Job configuration */ public JobSubmittedEvent(JobID id, String jobName, String userName, long submitTime, String jobConfPath, Map jobACLs, String jobQueueName, String workflowId, String workflowName, String workflowNodeName, - String workflowAdjacencies, String workflowTags) { + String workflowAdjacencies, String workflowTags, JobConf conf) { datum.setJobid(new Utf8(id.toString())); datum.setJobName(new Utf8(jobName)); datum.setUserName(new Utf8(userName)); @@ -129,6 +160,7 @@ public class JobSubmittedEvent implements HistoryEvent { if (workflowTags != null) { datum.setWorkflowTags(new Utf8(workflowTags)); } + jobConf = conf; } JobSubmittedEvent() {} @@ -206,4 +238,33 @@ public class JobSubmittedEvent implements HistoryEvent { /** Get the event type */ public EventType getEventType() { return EventType.JOB_SUBMITTED; } + public JobConf getJobConf() { + return jobConf; + } + + @Override + public TimelineEvent toTimelineEvent() { + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(StringUtils.toUpperCase(getEventType().name())); + tEvent.addInfo("SUBMIT_TIME", getSubmitTime()); + tEvent.addInfo("QUEUE_NAME", getJobQueueName()); + tEvent.addInfo("JOB_NAME", getJobName()); + tEvent.addInfo("USER_NAME", getUserName()); + tEvent.addInfo("JOB_CONF_PATH", getJobConfPath()); + tEvent.addInfo("ACLS", getJobAcls()); + tEvent.addInfo("JOB_QUEUE_NAME", getJobQueueName()); + tEvent.addInfo("WORKLFOW_ID", getWorkflowId()); + tEvent.addInfo("WORKFLOW_NAME", getWorkflowName()); + tEvent.addInfo("WORKFLOW_NODE_NAME", getWorkflowNodeName()); + tEvent.addInfo("WORKFLOW_ADJACENCIES", + getWorkflowAdjacencies()); + tEvent.addInfo("WORKFLOW_TAGS", getWorkflowTags()); + + return tEvent; + } + + @Override + public Set getTimelineMetrics() { + return null; + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobUnsuccessfulCompletionEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobUnsuccessfulCompletionEvent.java index 1b7773d6ba0..ce6fa328ef2 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobUnsuccessfulCompletionEvent.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobUnsuccessfulCompletionEvent.java @@ -18,14 +18,18 @@ package org.apache.hadoop.mapreduce.jobhistory; -import com.google.common.base.Joiner; +import java.util.Collections; +import java.util.Set; import org.apache.avro.util.Utf8; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.mapreduce.JobID; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; -import java.util.Collections; +import com.google.common.base.Joiner; /** * Event to record Failed and Killed completion of jobs @@ -119,4 +123,23 @@ public class JobUnsuccessfulCompletionEvent implements HistoryEvent { final CharSequence diagnostics = datum.getDiagnostics(); return diagnostics == null ? NODIAGS : diagnostics.toString(); } + + @Override + public TimelineEvent toTimelineEvent() { + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(StringUtils.toUpperCase(getEventType().name())); + tEvent.addInfo("FINISH_TIME", getFinishTime()); + tEvent.addInfo("NUM_MAPS", getFinishedMaps()); + tEvent.addInfo("NUM_REDUCES", getFinishedReduces()); + tEvent.addInfo("JOB_STATUS", getStatus()); + tEvent.addInfo("DIAGNOSTICS", getDiagnostics()); + tEvent.addInfo("FINISHED_MAPS", getFinishedMaps()); + tEvent.addInfo("FINISHED_REDUCES", getFinishedReduces()); + return tEvent; + } + + @Override + public Set getTimelineMetrics() { + return null; + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/MapAttemptFinishedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/MapAttemptFinishedEvent.java index bc046c7eb4b..2b1357ea859 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/MapAttemptFinishedEvent.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/MapAttemptFinishedEvent.java @@ -18,6 +18,8 @@ package org.apache.hadoop.mapreduce.jobhistory; +import java.util.Set; + import org.apache.avro.util.Utf8; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -26,14 +28,19 @@ import org.apache.hadoop.mapreduce.Counters; import org.apache.hadoop.mapreduce.TaskAttemptID; import org.apache.hadoop.mapreduce.TaskID; import org.apache.hadoop.mapreduce.TaskType; +import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.util.SystemClock; /** - * Event to record successful completion of a map attempt + * Event to record successful completion of a map attempt. * */ @InterfaceAudience.Private @InterfaceStability.Unstable -public class MapAttemptFinishedEvent implements HistoryEvent { +public class MapAttemptFinishedEvent implements HistoryEvent { private MapAttemptFinished datum = null; @@ -52,9 +59,10 @@ public class MapAttemptFinishedEvent implements HistoryEvent { int[] cpuUsages; int[] vMemKbytes; int[] physMemKbytes; + private long startTime; /** - * Create an event for successful completion of map attempts + * Create an event for successful completion of map attempts. * @param id Task Attempt ID * @param taskType Type of the task * @param taskStatus Status of the task @@ -71,12 +79,13 @@ public class MapAttemptFinishedEvent implements HistoryEvent { * virtual memory and physical memory. * * If you have no splits data, code {@code null} for this - * parameter. + * parameter. + * @param startTs Task start time to be used for writing entity to ATSv2. */ - public MapAttemptFinishedEvent - (TaskAttemptID id, TaskType taskType, String taskStatus, - long mapFinishTime, long finishTime, String hostname, int port, - String rackName, String state, Counters counters, int[][] allSplits) { + public MapAttemptFinishedEvent(TaskAttemptID id, TaskType taskType, + String taskStatus, long mapFinishTime, long finishTime, String hostname, + int port, String rackName, String state, Counters counters, + int[][] allSplits, long startTs) { this.attemptId = id; this.taskType = taskType; this.taskStatus = taskStatus; @@ -92,6 +101,16 @@ public class MapAttemptFinishedEvent implements HistoryEvent { this.cpuUsages = ProgressSplitsBlock.arrayGetCPUTime(allSplits); this.vMemKbytes = ProgressSplitsBlock.arrayGetVMemKbytes(allSplits); this.physMemKbytes = ProgressSplitsBlock.arrayGetPhysMemKbytes(allSplits); + this.startTime = startTs; + } + + public MapAttemptFinishedEvent(TaskAttemptID id, TaskType taskType, + String taskStatus, long mapFinishTime, long finishTime, String hostname, + int port, String rackName, String state, Counters counters, + int[][] allSplits) { + this(id, taskType, taskStatus, mapFinishTime, finishTime, hostname, port, + rackName, state, counters, allSplits, + SystemClock.getInstance().getTime()); } /** @@ -111,15 +130,13 @@ public class MapAttemptFinishedEvent implements HistoryEvent { * @param counters Counters for the attempt */ @Deprecated - public MapAttemptFinishedEvent - (TaskAttemptID id, TaskType taskType, String taskStatus, - long mapFinishTime, long finishTime, String hostname, - String state, Counters counters) { + public MapAttemptFinishedEvent(TaskAttemptID id, TaskType taskType, + String taskStatus, long mapFinishTime, long finishTime, String hostname, + String state, Counters counters) { this(id, taskType, taskStatus, mapFinishTime, finishTime, hostname, -1, "", state, counters, null); } - - + MapAttemptFinishedEvent() {} public Object getDatum() { @@ -169,38 +186,56 @@ public class MapAttemptFinishedEvent implements HistoryEvent { this.physMemKbytes = AvroArrayUtils.fromAvro(datum.getPhysMemKbytes()); } - /** Get the task ID */ - public TaskID getTaskId() { return attemptId.getTaskID(); } - /** Get the attempt id */ + /** Gets the task ID. */ + public TaskID getTaskId() { + return attemptId.getTaskID(); + } + /** Gets the attempt id. */ public TaskAttemptID getAttemptId() { return attemptId; } - /** Get the task type */ + /** Gets the task type. */ public TaskType getTaskType() { return taskType; } - /** Get the task status */ + /** Gets the task status. */ public String getTaskStatus() { return taskStatus.toString(); } - /** Get the map phase finish time */ + /** Gets the map phase finish time. */ public long getMapFinishTime() { return mapFinishTime; } - /** Get the attempt finish time */ + /** Gets the attempt finish time. */ public long getFinishTime() { return finishTime; } - /** Get the host name */ + /** + * Gets the task attempt start time. + * @return task attempt start time. + */ + public long getStartTime() { + return startTime; + } + /** Gets the host name. */ public String getHostname() { return hostname.toString(); } - /** Get the tracker rpc port */ + /** Gets the tracker rpc port. */ public int getPort() { return port; } - /** Get the rack name */ + /** Gets the rack name. */ public String getRackName() { return rackName == null ? null : rackName.toString(); } - - /** Get the state string */ - public String getState() { return state.toString(); } - /** Get the counters */ - Counters getCounters() { return counters; } - /** Get the event type */ + /** + * Gets the attempt state string. + * @return map attempt state + */ + public String getState() { + return state.toString(); + } + /** + * Gets the counters. + * @return counters + */ + Counters getCounters() { + return counters; + } + /** Gets the event type. */ public EventType getEventType() { return EventType.MAP_ATTEMPT_FINISHED; } @@ -218,4 +253,28 @@ public class MapAttemptFinishedEvent implements HistoryEvent { return physMemKbytes; } + @Override + public TimelineEvent toTimelineEvent() { + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(StringUtils.toUpperCase(getEventType().name())); + tEvent.addInfo("TASK_TYPE", getTaskType().toString()); + tEvent.addInfo("FINISH_TIME", getFinishTime()); + tEvent.addInfo("STATUS", getTaskStatus()); + tEvent.addInfo("STATE", getState()); + tEvent.addInfo("MAP_FINISH_TIME", getMapFinishTime()); + tEvent.addInfo("HOSTNAME", getHostname()); + tEvent.addInfo("PORT", getPort()); + tEvent.addInfo("RACK_NAME", getRackName()); + tEvent.addInfo("ATTEMPT_ID", getAttemptId() == null ? + "" : getAttemptId().toString()); + return tEvent; + } + + @Override + public Set getTimelineMetrics() { + Set metrics = JobHistoryEventUtils + .countersToTimelineMetric(getCounters(), finishTime); + return metrics; + } + } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/NormalizedResourceEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/NormalizedResourceEvent.java index eead9cf03df..7adae234599 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/NormalizedResourceEvent.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/NormalizedResourceEvent.java @@ -17,9 +17,14 @@ */ package org.apache.hadoop.mapreduce.jobhistory; +import java.util.Set; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.mapreduce.TaskType; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; /** * Event to record the normalized map/reduce requirements. @@ -71,4 +76,18 @@ public class NormalizedResourceEvent implements HistoryEvent { public void setDatum(Object datum) { throw new UnsupportedOperationException("Not a seriable object"); } + + @Override + public TimelineEvent toTimelineEvent() { + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(StringUtils.toUpperCase(getEventType().name())); + tEvent.addInfo("MEMORY", "" + getMemory()); + tEvent.addInfo("TASK_TYPE", getTaskType()); + return tEvent; + } + + @Override + public Set getTimelineMetrics() { + return null; + } } \ No newline at end of file diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/ReduceAttemptFinishedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/ReduceAttemptFinishedEvent.java index 6644a48f1e8..5a16f834acb 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/ReduceAttemptFinishedEvent.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/ReduceAttemptFinishedEvent.java @@ -18,6 +18,8 @@ package org.apache.hadoop.mapreduce.jobhistory; +import java.util.Set; + import org.apache.avro.util.Utf8; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -26,6 +28,11 @@ import org.apache.hadoop.mapreduce.Counters; import org.apache.hadoop.mapreduce.TaskAttemptID; import org.apache.hadoop.mapreduce.TaskID; import org.apache.hadoop.mapreduce.TaskType; +import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.util.SystemClock; /** * Event to record successful completion of a reduce attempt @@ -33,7 +40,7 @@ import org.apache.hadoop.mapreduce.TaskType; */ @InterfaceAudience.Private @InterfaceStability.Unstable -public class ReduceAttemptFinishedEvent implements HistoryEvent { +public class ReduceAttemptFinishedEvent implements HistoryEvent { private ReduceAttemptFinished datum = null; @@ -53,6 +60,7 @@ public class ReduceAttemptFinishedEvent implements HistoryEvent { int[] cpuUsages; int[] vMemKbytes; int[] physMemKbytes; + private long startTime; /** * Create an event to record completion of a reduce attempt @@ -70,13 +78,13 @@ public class ReduceAttemptFinishedEvent implements HistoryEvent { * @param allSplits the "splits", or a pixelated graph of various * measurable worker node state variables against progress. * Currently there are four; wallclock time, CPU time, - * virtual memory and physical memory. + * virtual memory and physical memory. + * @param startTs Task start time to be used for writing entity to ATSv2. */ - public ReduceAttemptFinishedEvent - (TaskAttemptID id, TaskType taskType, String taskStatus, - long shuffleFinishTime, long sortFinishTime, long finishTime, - String hostname, int port, String rackName, String state, - Counters counters, int[][] allSplits) { + public ReduceAttemptFinishedEvent(TaskAttemptID id, TaskType taskType, + String taskStatus, long shuffleFinishTime, long sortFinishTime, + long finishTime, String hostname, int port, String rackName, + String state, Counters counters, int[][] allSplits, long startTs) { this.attemptId = id; this.taskType = taskType; this.taskStatus = taskStatus; @@ -93,6 +101,16 @@ public class ReduceAttemptFinishedEvent implements HistoryEvent { this.cpuUsages = ProgressSplitsBlock.arrayGetCPUTime(allSplits); this.vMemKbytes = ProgressSplitsBlock.arrayGetVMemKbytes(allSplits); this.physMemKbytes = ProgressSplitsBlock.arrayGetPhysMemKbytes(allSplits); + this.startTime = startTs; + } + + public ReduceAttemptFinishedEvent(TaskAttemptID id, TaskType taskType, + String taskStatus, long shuffleFinishTime, long sortFinishTime, + long finishTime, String hostname, int port, String rackName, + String state, Counters counters, int[][] allSplits) { + this(id, taskType, taskStatus, shuffleFinishTime, sortFinishTime, + finishTime, hostname, port, rackName, state, counters, allSplits, + SystemClock.getInstance().getTime()); } /** @@ -112,13 +130,12 @@ public class ReduceAttemptFinishedEvent implements HistoryEvent { * @param state State of the attempt * @param counters Counters for the attempt */ - public ReduceAttemptFinishedEvent - (TaskAttemptID id, TaskType taskType, String taskStatus, - long shuffleFinishTime, long sortFinishTime, long finishTime, - String hostname, String state, Counters counters) { + public ReduceAttemptFinishedEvent(TaskAttemptID id, TaskType taskType, + String taskStatus, long shuffleFinishTime, long sortFinishTime, + long finishTime, String hostname, String state, Counters counters) { this(id, taskType, taskStatus, - shuffleFinishTime, sortFinishTime, finishTime, - hostname, -1, "", state, counters, null); + shuffleFinishTime, sortFinishTime, finishTime, + hostname, -1, "", state, counters, null); } ReduceAttemptFinishedEvent() {} @@ -172,39 +189,55 @@ public class ReduceAttemptFinishedEvent implements HistoryEvent { this.physMemKbytes = AvroArrayUtils.fromAvro(datum.getPhysMemKbytes()); } - /** Get the Task ID */ + /** Gets the Task ID. */ public TaskID getTaskId() { return attemptId.getTaskID(); } - /** Get the attempt id */ + /** Gets the attempt id. */ public TaskAttemptID getAttemptId() { return attemptId; } - /** Get the task type */ + /** Gets the task type. */ public TaskType getTaskType() { return taskType; } - /** Get the task status */ + /** Gets the task status. */ public String getTaskStatus() { return taskStatus.toString(); } - /** Get the finish time of the sort phase */ + /** Gets the finish time of the sort phase. */ public long getSortFinishTime() { return sortFinishTime; } - /** Get the finish time of the shuffle phase */ + /** Gets the finish time of the shuffle phase. */ public long getShuffleFinishTime() { return shuffleFinishTime; } - /** Get the finish time of the attempt */ + /** Gets the finish time of the attempt. */ public long getFinishTime() { return finishTime; } - /** Get the name of the host where the attempt ran */ + /** + * Gets the start time. + * @return task attempt start time. + */ + public long getStartTime() { + return startTime; + } + /** Gets the name of the host where the attempt ran. */ public String getHostname() { return hostname.toString(); } - /** Get the tracker rpc port */ + /** Gets the tracker rpc port. */ public int getPort() { return port; } - /** Get the rack name of the node where the attempt ran */ + /** Gets the rack name of the node where the attempt ran. */ public String getRackName() { return rackName == null ? null : rackName.toString(); } - - /** Get the state string */ - public String getState() { return state.toString(); } - /** Get the counters for the attempt */ - Counters getCounters() { return counters; } - /** Get the event type */ + /** + * Gets the state string. + * @return reduce attempt state + */ + public String getState() { + return state.toString(); + } + /** + * Gets the counters. + * @return counters + */ + Counters getCounters() { + return counters; + } + /** Gets the event type. */ public EventType getEventType() { return EventType.REDUCE_ATTEMPT_FINISHED; } @@ -223,4 +256,29 @@ public class ReduceAttemptFinishedEvent implements HistoryEvent { return physMemKbytes; } + @Override + public TimelineEvent toTimelineEvent() { + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(StringUtils.toUpperCase(getEventType().name())); + tEvent.addInfo("TASK_TYPE", getTaskType().toString()); + tEvent.addInfo("ATTEMPT_ID", getAttemptId() == null ? + "" : getAttemptId().toString()); + tEvent.addInfo("FINISH_TIME", getFinishTime()); + tEvent.addInfo("STATUS", getTaskStatus()); + tEvent.addInfo("STATE", getState()); + tEvent.addInfo("SHUFFLE_FINISH_TIME", getShuffleFinishTime()); + tEvent.addInfo("SORT_FINISH_TIME", getSortFinishTime()); + tEvent.addInfo("HOSTNAME", getHostname()); + tEvent.addInfo("PORT", getPort()); + tEvent.addInfo("RACK_NAME", getRackName()); + return tEvent; + } + + @Override + public Set getTimelineMetrics() { + Set metrics = JobHistoryEventUtils + .countersToTimelineMetric(getCounters(), finishTime); + return metrics; + } + } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptFinishedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptFinishedEvent.java index bb7dbe00fd8..c28c21605df 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptFinishedEvent.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptFinishedEvent.java @@ -18,6 +18,8 @@ package org.apache.hadoop.mapreduce.jobhistory; +import java.util.Set; + import org.apache.avro.util.Utf8; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -25,6 +27,11 @@ import org.apache.hadoop.mapreduce.Counters; import org.apache.hadoop.mapreduce.TaskAttemptID; import org.apache.hadoop.mapreduce.TaskID; import org.apache.hadoop.mapreduce.TaskType; +import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.util.SystemClock; /** * Event to record successful task completion @@ -44,10 +51,11 @@ public class TaskAttemptFinishedEvent implements HistoryEvent { private String hostname; private String state; private Counters counters; + private long startTime; /** - * Create an event to record successful finishes for setup and cleanup - * attempts + * Create an event to record successful finishes for setup and cleanup + * attempts. * @param id Attempt ID * @param taskType Type of task * @param taskStatus Status of task @@ -55,11 +63,12 @@ public class TaskAttemptFinishedEvent implements HistoryEvent { * @param hostname Host where the attempt executed * @param state State string * @param counters Counters for the attempt + * @param startTs Task start time to be used for writing entity to ATSv2. */ public TaskAttemptFinishedEvent(TaskAttemptID id, TaskType taskType, String taskStatus, long finishTime, String rackName, - String hostname, String state, Counters counters) { + String hostname, String state, Counters counters, long startTs) { this.attemptId = id; this.taskType = taskType; this.taskStatus = taskStatus; @@ -68,6 +77,14 @@ public class TaskAttemptFinishedEvent implements HistoryEvent { this.hostname = hostname; this.state = state; this.counters = counters; + this.startTime = startTs; + } + + public TaskAttemptFinishedEvent(TaskAttemptID id, TaskType taskType, + String taskStatus, long finishTime, String rackName, String hostname, + String state, Counters counters) { + this(id, taskType, taskStatus, finishTime, rackName, hostname, state, + counters, SystemClock.getInstance().getTime()); } TaskAttemptFinishedEvent() {} @@ -101,33 +118,43 @@ public class TaskAttemptFinishedEvent implements HistoryEvent { this.counters = EventReader.fromAvro(datum.getCounters()); } - /** Get the task ID */ + /** Gets the task ID. */ public TaskID getTaskId() { return attemptId.getTaskID(); } - /** Get the task attempt id */ + /** Gets the task attempt id. */ public TaskAttemptID getAttemptId() { return attemptId; } - /** Get the task type */ + /** Gets the task type. */ public TaskType getTaskType() { return taskType; } - /** Get the task status */ + /** Gets the task status. */ public String getTaskStatus() { return taskStatus.toString(); } - /** Get the attempt finish time */ + /** Gets the attempt finish time. */ public long getFinishTime() { return finishTime; } - /** Get the host where the attempt executed */ + /** + * Gets the task attempt start time to be used while publishing to ATSv2. + * @return task attempt start time. + */ + public long getStartTime() { + return startTime; + } + /** Gets the host where the attempt executed. */ public String getHostname() { return hostname.toString(); } - /** Get the rackname where the attempt executed */ + /** Gets the rackname where the attempt executed. */ public String getRackName() { return rackName == null ? null : rackName.toString(); } - /** Get the state string */ + /** + * Gets the state string. + * @return task attempt state. + */ public String getState() { return state.toString(); } - /** Get the counters for the attempt */ + /** Gets the counters for the attempt. */ Counters getCounters() { return counters; } - /** Get the event type */ + /** Gets the event type. */ public EventType getEventType() { // Note that the task type can be setup/map/reduce/cleanup but the // attempt-type can only be map/reduce. @@ -136,4 +163,24 @@ public class TaskAttemptFinishedEvent implements HistoryEvent { : EventType.REDUCE_ATTEMPT_FINISHED; } + @Override + public TimelineEvent toTimelineEvent() { + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(StringUtils.toUpperCase(getEventType().name())); + tEvent.addInfo("TASK_TYPE", getTaskType().toString()); + tEvent.addInfo("ATTEMPT_ID", getAttemptId() == null ? + "" : getAttemptId().toString()); + tEvent.addInfo("FINISH_TIME", getFinishTime()); + tEvent.addInfo("STATUS", getTaskStatus()); + tEvent.addInfo("STATE", getState()); + tEvent.addInfo("HOSTNAME", getHostname()); + return tEvent; + } + + @Override + public Set getTimelineMetrics() { + Set metrics = JobHistoryEventUtils + .countersToTimelineMetric(getCounters(), finishTime); + return metrics; + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java index 3073d5b95f1..d09d5ca6e9b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java @@ -18,16 +18,20 @@ package org.apache.hadoop.mapreduce.jobhistory; +import java.util.Set; + +import org.apache.avro.util.Utf8; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.mapreduce.TaskAttemptID; import org.apache.hadoop.mapreduce.TaskID; import org.apache.hadoop.mapreduce.TaskType; +import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; import org.apache.hadoop.yarn.util.ConverterUtils; -import org.apache.avro.util.Utf8; - /** * Event to record start of a task attempt * @@ -133,4 +137,25 @@ public class TaskAttemptStartedEvent implements HistoryEvent { return null; } + @Override + public TimelineEvent toTimelineEvent() { + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(StringUtils.toUpperCase(getEventType().name())); + tEvent.addInfo("TASK_TYPE", getTaskType().toString()); + tEvent.addInfo("TASK_ATTEMPT_ID", + getTaskAttemptId().toString()); + tEvent.addInfo("START_TIME", getStartTime()); + tEvent.addInfo("HTTP_PORT", getHttpPort()); + tEvent.addInfo("TRACKER_NAME", getTrackerName()); + tEvent.addInfo("SHUFFLE_PORT", getShufflePort()); + tEvent.addInfo("CONTAINER_ID", getContainerId() == null ? + "" : getContainerId().toString()); + return tEvent; + } + + @Override + public Set getTimelineMetrics() { + return null; + } + } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java index 9a11e5746ba..9afa09384cc 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java @@ -18,17 +18,22 @@ package org.apache.hadoop.mapreduce.jobhistory; +import java.util.Set; + +import org.apache.avro.util.Utf8; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.mapred.ProgressSplitsBlock; import org.apache.hadoop.mapred.TaskStatus; import org.apache.hadoop.mapreduce.Counters; import org.apache.hadoop.mapreduce.TaskAttemptID; import org.apache.hadoop.mapreduce.TaskID; import org.apache.hadoop.mapreduce.TaskType; - -import org.apache.hadoop.mapred.ProgressSplitsBlock; - -import org.apache.avro.util.Utf8; +import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.util.SystemClock; /** * Event to record unsuccessful (Killed/Failed) completion of task attempts @@ -54,10 +59,11 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent { int[] cpuUsages; int[] vMemKbytes; int[] physMemKbytes; + private long startTime; private static final Counters EMPTY_COUNTERS = new Counters(); /** - * Create an event to record the unsuccessful completion of attempts + * Create an event to record the unsuccessful completion of attempts. * @param id Attempt ID * @param taskType Type of the task * @param status Status of the attempt @@ -71,12 +77,13 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent { * measurable worker node state variables against progress. * Currently there are four; wallclock time, CPU time, * virtual memory and physical memory. + * @param startTs Task start time to be used for writing entity to ATSv2. */ public TaskAttemptUnsuccessfulCompletionEvent (TaskAttemptID id, TaskType taskType, String status, long finishTime, String hostname, int port, String rackName, - String error, Counters counters, int[][] allSplits) { + String error, Counters counters, int[][] allSplits, long startTs) { this.attemptId = id; this.taskType = taskType; this.status = status; @@ -95,6 +102,15 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent { ProgressSplitsBlock.arrayGetVMemKbytes(allSplits); this.physMemKbytes = ProgressSplitsBlock.arrayGetPhysMemKbytes(allSplits); + this.startTime = startTs; + } + + public TaskAttemptUnsuccessfulCompletionEvent(TaskAttemptID id, + TaskType taskType, String status, long finishTime, String hostname, + int port, String rackName, String error, Counters counters, + int[][] allSplits) { + this(id, taskType, status, finishTime, hostname, port, rackName, error, + counters, allSplits, SystemClock.getInstance().getTime()); } /** @@ -186,39 +202,49 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent { AvroArrayUtils.fromAvro(datum.getPhysMemKbytes()); } - /** Get the task id */ + /** Gets the task id. */ public TaskID getTaskId() { return attemptId.getTaskID(); } - /** Get the task type */ + /** Gets the task type. */ public TaskType getTaskType() { return TaskType.valueOf(taskType.toString()); } - /** Get the attempt id */ + /** Gets the attempt id. */ public TaskAttemptID getTaskAttemptId() { return attemptId; } - /** Get the finish time */ + /** Gets the finish time. */ public long getFinishTime() { return finishTime; } - /** Get the name of the host where the attempt executed */ + /** + * Gets the task attempt start time to be used while publishing to ATSv2. + * @return task attempt start time. + */ + public long getStartTime() { + return startTime; + } + /** Gets the name of the host where the attempt executed. */ public String getHostname() { return hostname; } - /** Get the rpc port for the host where the attempt executed */ + /** Gets the rpc port for the host where the attempt executed. */ public int getPort() { return port; } - /** Get the rack name of the node where the attempt ran */ + /** Gets the rack name of the node where the attempt ran. */ public String getRackName() { return rackName == null ? null : rackName.toString(); } - /** Get the error string */ + /** Gets the error string. */ public String getError() { return error.toString(); } - /** Get the task status */ + /** + * Gets the task attempt status. + * @return task attempt status. + */ public String getTaskStatus() { return status.toString(); } - /** Get the counters */ + /** Gets the counters. */ Counters getCounters() { return counters; } - /** Get the event type */ + /** Gets the event type. */ public EventType getEventType() { // Note that the task type can be setup/map/reduce/cleanup but the // attempt-type can only be map/reduce. @@ -248,4 +274,29 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent { return physMemKbytes; } + @Override + public TimelineEvent toTimelineEvent() { + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(StringUtils.toUpperCase(getEventType().name())); + tEvent.addInfo("TASK_TYPE", getTaskType().toString()); + tEvent.addInfo("TASK_ATTEMPT_ID", getTaskAttemptId() == null ? + "" : getTaskAttemptId().toString()); + tEvent.addInfo("FINISH_TIME", getFinishTime()); + tEvent.addInfo("ERROR", getError()); + tEvent.addInfo("STATUS", getTaskStatus()); + tEvent.addInfo("HOSTNAME", getHostname()); + tEvent.addInfo("PORT", getPort()); + tEvent.addInfo("RACK_NAME", getRackName()); + tEvent.addInfo("SHUFFLE_FINISH_TIME", getFinishTime()); + tEvent.addInfo("SORT_FINISH_TIME", getFinishTime()); + tEvent.addInfo("MAP_FINISH_TIME", getFinishTime()); + return tEvent; + } + + @Override + public Set getTimelineMetrics() { + Set metrics = JobHistoryEventUtils + .countersToTimelineMetric(getCounters(), finishTime); + return metrics; + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFailedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFailedEvent.java index 2838f08cd3c..b4d9e410da2 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFailedEvent.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFailedEvent.java @@ -18,14 +18,21 @@ package org.apache.hadoop.mapreduce.jobhistory; +import java.util.Set; + +import org.apache.avro.util.Utf8; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.mapred.TaskStatus; import org.apache.hadoop.mapreduce.Counters; import org.apache.hadoop.mapreduce.TaskAttemptID; import org.apache.hadoop.mapreduce.TaskID; import org.apache.hadoop.mapreduce.TaskType; - -import org.apache.avro.util.Utf8; +import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.util.SystemClock; /** * Event to record the failure of a task @@ -43,11 +50,12 @@ public class TaskFailedEvent implements HistoryEvent { private String status; private String error; private Counters counters; + private long startTime; private static final Counters EMPTY_COUNTERS = new Counters(); /** - * Create an event to record task failure + * Create an event to record task failure. * @param id Task ID * @param finishTime Finish time of the task * @param taskType Type of the task @@ -55,10 +63,11 @@ public class TaskFailedEvent implements HistoryEvent { * @param status Status * @param failedDueToAttempt The attempt id due to which the task failed * @param counters Counters for the task + * @param startTs task start time. */ public TaskFailedEvent(TaskID id, long finishTime, TaskType taskType, String error, String status, - TaskAttemptID failedDueToAttempt, Counters counters) { + TaskAttemptID failedDueToAttempt, Counters counters, long startTs) { this.id = id; this.finishTime = finishTime; this.taskType = taskType; @@ -66,15 +75,23 @@ public class TaskFailedEvent implements HistoryEvent { this.status = status; this.failedDueToAttempt = failedDueToAttempt; this.counters = counters; + this.startTime = startTs; + } + + public TaskFailedEvent(TaskID id, long finishTime, TaskType taskType, + String error, String status, TaskAttemptID failedDueToAttempt, + Counters counters) { + this(id, finishTime, taskType, error, status, failedDueToAttempt, counters, + SystemClock.getInstance().getTime()); } public TaskFailedEvent(TaskID id, long finishTime, - TaskType taskType, String error, String status, - TaskAttemptID failedDueToAttempt) { - this(id, finishTime, taskType, error, status, - failedDueToAttempt, EMPTY_COUNTERS); + TaskType taskType, String error, String status, + TaskAttemptID failedDueToAttempt) { + this(id, finishTime, taskType, error, status, failedDueToAttempt, + EMPTY_COUNTERS); } - + TaskFailedEvent() {} public Object getDatum() { @@ -112,29 +129,58 @@ public class TaskFailedEvent implements HistoryEvent { EventReader.fromAvro(datum.getCounters()); } - /** Get the task id */ + /** Gets the task id. */ public TaskID getTaskId() { return id; } - /** Get the error string */ + /** Gets the error string. */ public String getError() { return error; } - /** Get the finish time of the attempt */ + /** Gets the finish time of the attempt. */ public long getFinishTime() { return finishTime; } - /** Get the task type */ + /** + * Gets the task start time to be reported to ATSv2. + * @return task start time. + */ + public long getStartTime() { + return startTime; + } + /** Gets the task type. */ public TaskType getTaskType() { return taskType; } - /** Get the attempt id due to which the task failed */ + /** Gets the attempt id due to which the task failed. */ public TaskAttemptID getFailedAttemptID() { return failedDueToAttempt; } - /** Get the task status */ + /** + * Gets the task status. + * @return task status + */ public String getTaskStatus() { return status; } - /** Get task counters */ + /** Gets task counters. */ public Counters getCounters() { return counters; } - /** Get the event type */ + /** Gets the event type. */ public EventType getEventType() { return EventType.TASK_FAILED; } + @Override + public TimelineEvent toTimelineEvent() { + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(StringUtils.toUpperCase(getEventType().name())); + tEvent.addInfo("TASK_TYPE", getTaskType().toString()); + tEvent.addInfo("STATUS", TaskStatus.State.FAILED.toString()); + tEvent.addInfo("FINISH_TIME", getFinishTime()); + tEvent.addInfo("ERROR", getError()); + tEvent.addInfo("FAILED_ATTEMPT_ID", + getFailedAttemptID() == null ? "" : getFailedAttemptID().toString()); + return tEvent; + } + + @Override + public Set getTimelineMetrics() { + Set metrics = JobHistoryEventUtils + .countersToTimelineMetric(getCounters(), finishTime); + return metrics; + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFinishedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFinishedEvent.java index d4ec74db0ad..97557c7e0b4 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFinishedEvent.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFinishedEvent.java @@ -18,13 +18,21 @@ package org.apache.hadoop.mapreduce.jobhistory; +import java.util.Set; + import org.apache.avro.util.Utf8; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.mapred.TaskStatus; import org.apache.hadoop.mapreduce.Counters; import org.apache.hadoop.mapreduce.TaskAttemptID; import org.apache.hadoop.mapreduce.TaskID; import org.apache.hadoop.mapreduce.TaskType; +import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.util.SystemClock; /** * Event to record the successful completion of a task @@ -42,27 +50,36 @@ public class TaskFinishedEvent implements HistoryEvent { private TaskType taskType; private String status; private Counters counters; - + private long startTime; + /** - * Create an event to record the successful completion of a task + * Create an event to record the successful completion of a task. * @param id Task ID * @param attemptId Task Attempt ID of the successful attempt for this task * @param finishTime Finish time of the task * @param taskType Type of the task * @param status Status string * @param counters Counters for the task + * @param startTs task start time */ public TaskFinishedEvent(TaskID id, TaskAttemptID attemptId, long finishTime, TaskType taskType, - String status, Counters counters) { + String status, Counters counters, long startTs) { this.taskid = id; this.successfulAttemptId = attemptId; this.finishTime = finishTime; this.taskType = taskType; this.status = status; this.counters = counters; + this.startTime = startTs; } - + + public TaskFinishedEvent(TaskID id, TaskAttemptID attemptId, long finishTime, + TaskType taskType, String status, Counters counters) { + this(id, attemptId, finishTime, taskType, status, counters, + SystemClock.getInstance().getTime()); + } + TaskFinishedEvent() {} public Object getDatum() { @@ -94,26 +111,54 @@ public class TaskFinishedEvent implements HistoryEvent { this.counters = EventReader.fromAvro(datum.getCounters()); } - /** Get task id */ + /** Gets task id. */ public TaskID getTaskId() { return taskid; } - /** Get successful task attempt id */ + /** Gets successful task attempt id. */ public TaskAttemptID getSuccessfulTaskAttemptId() { return successfulAttemptId; } - /** Get the task finish time */ + /** Gets the task finish time. */ public long getFinishTime() { return finishTime; } - /** Get task counters */ + /** + * Gets the task start time to be reported to ATSv2. + * @return task start time + */ + public long getStartTime() { + return startTime; + } + /** Gets task counters. */ public Counters getCounters() { return counters; } - /** Get task type */ + /** Gets task type. */ public TaskType getTaskType() { return taskType; } - /** Get task status */ + /** + * Gets task status. + * @return task status + */ public String getTaskStatus() { return status.toString(); } - /** Get event type */ + /** Gets event type. */ public EventType getEventType() { return EventType.TASK_FINISHED; } - + @Override + public TimelineEvent toTimelineEvent() { + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(StringUtils.toUpperCase(getEventType().name())); + tEvent.addInfo("TASK_TYPE", getTaskType().toString()); + tEvent.addInfo("FINISH_TIME", getFinishTime()); + tEvent.addInfo("STATUS", TaskStatus.State.SUCCEEDED.toString()); + tEvent.addInfo("SUCCESSFUL_TASK_ATTEMPT_ID", + getSuccessfulTaskAttemptId() == null ? "" : + getSuccessfulTaskAttemptId().toString()); + return tEvent; + } + + @Override + public Set getTimelineMetrics() { + Set jobMetrics = JobHistoryEventUtils + .countersToTimelineMetric(getCounters(), finishTime); + return jobMetrics; + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskStartedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskStartedEvent.java index ed53b030289..9d2fc0eee55 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskStartedEvent.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskStartedEvent.java @@ -18,11 +18,16 @@ package org.apache.hadoop.mapreduce.jobhistory; +import java.util.Set; + import org.apache.avro.util.Utf8; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.mapreduce.TaskID; import org.apache.hadoop.mapreduce.TaskType; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; /** * Event to record the start of a task @@ -72,4 +77,19 @@ public class TaskStartedEvent implements HistoryEvent { return EventType.TASK_STARTED; } + @Override + public TimelineEvent toTimelineEvent() { + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(StringUtils.toUpperCase(getEventType().name())); + tEvent.addInfo("TASK_TYPE", getTaskType().toString()); + tEvent.addInfo("START_TIME", getStartTime()); + tEvent.addInfo("SPLIT_LOCATIONS", getSplitLocations()); + return tEvent; + } + + @Override + public Set getTimelineMetrics() { + return null; + } + } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskUpdatedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskUpdatedEvent.java index 58f414330d8..010129d73ce 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskUpdatedEvent.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskUpdatedEvent.java @@ -18,13 +18,15 @@ package org.apache.hadoop.mapreduce.jobhistory; -import java.io.IOException; +import java.util.Set; +import org.apache.avro.util.Utf8; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.mapreduce.TaskID; - -import org.apache.avro.util.Utf8; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; /** * Event to record updates to a task @@ -61,4 +63,17 @@ public class TaskUpdatedEvent implements HistoryEvent { return EventType.TASK_UPDATED; } + @Override + public TimelineEvent toTimelineEvent() { + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(StringUtils.toUpperCase(getEventType().name())); + tEvent.addInfo("FINISH_TIME", getFinishTime()); + return tEvent; + } + + @Override + public Set getTimelineMetrics() { + return null; + } + } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/JobHistoryEventUtils.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/JobHistoryEventUtils.java new file mode 100644 index 00000000000..89abf0e30d1 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/JobHistoryEventUtils.java @@ -0,0 +1,83 @@ +/** +* 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.mapreduce.util; + +import java.util.HashSet; +import java.util.Set; + +import org.apache.hadoop.mapreduce.Counter; +import org.apache.hadoop.mapreduce.CounterGroup; +import org.apache.hadoop.mapreduce.Counters; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.codehaus.jackson.JsonNode; +import org.codehaus.jackson.map.ObjectMapper; +import org.codehaus.jackson.node.ArrayNode; +import org.codehaus.jackson.node.ObjectNode; + +/** + * Class containing utility methods to be used by JobHistoryEventHandler. + */ +public final class JobHistoryEventUtils { + private JobHistoryEventUtils() { + } + + // Number of bytes of config which can be published in one shot to ATSv2. + public static final int ATS_CONFIG_PUBLISH_SIZE_BYTES = 10 * 1024; + + public static JsonNode countersToJSON(Counters counters) { + ObjectMapper mapper = new ObjectMapper(); + ArrayNode nodes = mapper.createArrayNode(); + if (counters != null) { + for (CounterGroup counterGroup : counters) { + ObjectNode groupNode = nodes.addObject(); + groupNode.put("NAME", counterGroup.getName()); + groupNode.put("DISPLAY_NAME", counterGroup.getDisplayName()); + ArrayNode countersNode = groupNode.putArray("COUNTERS"); + for (Counter counter : counterGroup) { + ObjectNode counterNode = countersNode.addObject(); + counterNode.put("NAME", counter.getName()); + counterNode.put("DISPLAY_NAME", counter.getDisplayName()); + counterNode.put("VALUE", counter.getValue()); + } + } + } + return nodes; + } + + public static Set countersToTimelineMetric(Counters counters, + long timestamp) { + return countersToTimelineMetric(counters, timestamp, ""); + } + + public static Set countersToTimelineMetric(Counters counters, + long timestamp, String groupNamePrefix) { + Set entityMetrics = new HashSet(); + for (CounterGroup g : counters) { + String groupName = g.getName(); + for (Counter c : g) { + String name = groupNamePrefix + groupName + ":" + c.getName(); + TimelineMetric metric = new TimelineMetric(); + metric.setId(name); + metric.addValue(timestamp, c.getValue()); + entityMetrics.add(metric); + } + } + return entityMetrics; + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml index 263a3e728fd..0e87d3b9296 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml @@ -99,6 +99,12 @@ test test-jar + + org.apache.hadoop + hadoop-yarn-server-timelineservice + test + test-jar + org.hsqldb hsqldb diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java index 140591c2f84..e61eecdd25f 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java @@ -18,6 +18,21 @@ package org.apache.hadoop.mapred; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.util.EnumSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; + +import org.apache.commons.io.FileUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -26,17 +41,33 @@ import org.apache.hadoop.mapreduce.jobhistory.EventType; import org.apache.hadoop.mapreduce.jobhistory.TestJobHistoryEventHandler; import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster; import org.apache.hadoop.test.GenericTestUtils; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities; import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.MiniYARNCluster; import org.apache.hadoop.yarn.server.timeline.TimelineStore; - +import org.apache.hadoop.yarn.server.timelineservice.collector.PerNodeTimelineCollectorsAuxService; +import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineReaderImpl; +import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter; +import org.apache.hadoop.yarn.util.timeline.TimelineUtils; import org.junit.Assert; import org.junit.Test; +import com.google.common.collect.Sets; + public class TestMRTimelineEventHandling { + private static final String TIMELINE_AUX_SERVICE_NAME = "timeline_collector"; + private static final Log LOG = + LogFactory.getLog(TestMRTimelineEventHandling.class); + @Test public void testTimelineServiceStartInMiniCluster() throws Exception { Configuration conf = new YarnConfiguration(); @@ -50,7 +81,7 @@ public class TestMRTimelineEventHandling { MiniMRYarnCluster cluster = null; try { cluster = new MiniMRYarnCluster( - TestJobHistoryEventHandler.class.getSimpleName(), 1); + TestMRTimelineEventHandling.class.getSimpleName(), 1); cluster.init(conf); cluster.start(); @@ -91,7 +122,7 @@ public class TestMRTimelineEventHandling { MiniMRYarnCluster cluster = null; try { cluster = new MiniMRYarnCluster( - TestJobHistoryEventHandler.class.getSimpleName(), 1); + TestMRTimelineEventHandling.class.getSimpleName(), 1); cluster.init(conf); cluster.start(); conf.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS, @@ -140,6 +171,304 @@ public class TestMRTimelineEventHandling { } } + @SuppressWarnings("deprecation") + @Test + public void testMRNewTimelineServiceEventHandling() throws Exception { + LOG.info("testMRNewTimelineServiceEventHandling start."); + + String testDir = + new File("target", getClass().getSimpleName() + + "-test_dir").getAbsolutePath(); + String storageDir = + testDir + File.separator + "timeline_service_data"; + + Configuration conf = new YarnConfiguration(); + conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); + // enable new timeline service + conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f); + conf.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS, + FileSystemTimelineWriterImpl.class, TimelineWriter.class); + conf.setBoolean(MRJobConfig.MAPREDUCE_JOB_EMIT_TIMELINE_DATA, true); + // set the file system root directory + conf.set(FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_DIR_ROOT, + storageDir); + + // enable aux-service based timeline collectors + conf.set(YarnConfiguration.NM_AUX_SERVICES, TIMELINE_AUX_SERVICE_NAME); + conf.set(YarnConfiguration.NM_AUX_SERVICES + "." + TIMELINE_AUX_SERVICE_NAME + + ".class", PerNodeTimelineCollectorsAuxService.class.getName()); + + conf.setBoolean(YarnConfiguration.SYSTEM_METRICS_PUBLISHER_ENABLED, true); + + MiniMRYarnCluster cluster = null; + try { + cluster = new MiniMRYarnCluster( + TestMRTimelineEventHandling.class.getSimpleName(), 1, true); + cluster.init(conf); + cluster.start(); + LOG.info("A MiniMRYarnCluster get start."); + + Path inDir = new Path(testDir, "input"); + Path outDir = new Path(testDir, "output"); + LOG.info("Run 1st job which should be successful."); + JobConf successConf = new JobConf(conf); + successConf.set("dummy_conf1", + UtilsForTests.createConfigValue(51 * 1024)); + successConf.set("dummy_conf2", + UtilsForTests.createConfigValue(51 * 1024)); + successConf.set("huge_dummy_conf1", + UtilsForTests.createConfigValue(101 * 1024)); + successConf.set("huge_dummy_conf2", + UtilsForTests.createConfigValue(101 * 1024)); + RunningJob job = + UtilsForTests.runJobSucceed(successConf, inDir, outDir); + Assert.assertEquals(JobStatus.SUCCEEDED, + job.getJobStatus().getState().getValue()); + + YarnClient yarnClient = YarnClient.createYarnClient(); + yarnClient.init(new Configuration(cluster.getConfig())); + yarnClient.start(); + EnumSet appStates = + EnumSet.allOf(YarnApplicationState.class); + + ApplicationId firstAppId = null; + List apps = yarnClient.getApplications(appStates); + Assert.assertEquals(apps.size(), 1); + ApplicationReport appReport = apps.get(0); + firstAppId = appReport.getApplicationId(); + UtilsForTests.waitForAppFinished(job, cluster); + checkNewTimelineEvent(firstAppId, appReport, storageDir); + + LOG.info("Run 2nd job which should be failed."); + job = UtilsForTests.runJobFail(new JobConf(conf), inDir, outDir); + Assert.assertEquals(JobStatus.FAILED, + job.getJobStatus().getState().getValue()); + + apps = yarnClient.getApplications(appStates); + Assert.assertEquals(apps.size(), 2); + + appReport = apps.get(0).getApplicationId().equals(firstAppId) ? + apps.get(0) : apps.get(1); + + checkNewTimelineEvent(firstAppId, appReport, storageDir); + + } finally { + if (cluster != null) { + cluster.stop(); + } + // Cleanup test file + File testDirFolder = new File(testDir); + if(testDirFolder.isDirectory()) { + FileUtils.deleteDirectory(testDirFolder); + } + + } + } + + private void checkNewTimelineEvent(ApplicationId appId, + ApplicationReport appReport, String storageDir) throws IOException { + String tmpRoot = storageDir + File.separator + "entities" + File.separator; + + File tmpRootFolder = new File(tmpRoot); + + Assert.assertTrue(tmpRootFolder.isDirectory()); + String basePath = tmpRoot + YarnConfiguration.DEFAULT_RM_CLUSTER_ID + + File.separator + + UserGroupInformation.getCurrentUser().getShortUserName() + + File.separator + appReport.getName() + + File.separator + TimelineUtils.DEFAULT_FLOW_VERSION + + File.separator + appReport.getStartTime() + + File.separator + appId.toString(); + // for this test, we expect MAPREDUCE_JOB and MAPREDUCE_TASK dirs + String outputDirJob = + basePath + File.separator + "MAPREDUCE_JOB" + File.separator; + + File entityFolder = new File(outputDirJob); + Assert.assertTrue("Job output directory: " + outputDirJob + + " does not exist.", + entityFolder.isDirectory()); + + // check for job event file + String jobEventFileName = appId.toString().replaceAll("application", "job") + + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION; + + String jobEventFilePath = outputDirJob + jobEventFileName; + File jobEventFile = new File(jobEventFilePath); + Assert.assertTrue("jobEventFilePath: " + jobEventFilePath + + " does not exist.", + jobEventFile.exists()); + verifyEntity(jobEventFile, EventType.JOB_FINISHED.name(), + true, false, null, false); + Set cfgsToCheck = Sets.newHashSet("dummy_conf1", "dummy_conf2", + "huge_dummy_conf1", "huge_dummy_conf2"); + verifyEntity(jobEventFile, null, false, true, cfgsToCheck, false); + + // for this test, we expect MR job metrics are published in YARN_APPLICATION + String outputAppDir = + basePath + File.separator + "YARN_APPLICATION" + File.separator; + entityFolder = new File(outputAppDir); + Assert.assertTrue( + "Job output directory: " + outputAppDir + + " does not exist.", + entityFolder.isDirectory()); + + // check for job event file + String appEventFileName = appId.toString() + + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION; + + String appEventFilePath = outputAppDir + appEventFileName; + File appEventFile = new File(appEventFilePath); + Assert.assertTrue( + "appEventFilePath: " + appEventFilePath + + " does not exist.", + appEventFile.exists()); + verifyEntity(appEventFile, null, true, false, null, false); + verifyEntity(appEventFile, null, false, true, cfgsToCheck, false); + + // check for task event file + String outputDirTask = + basePath + File.separator + "MAPREDUCE_TASK" + File.separator; + File taskFolder = new File(outputDirTask); + Assert.assertTrue("Task output directory: " + outputDirTask + + " does not exist.", + taskFolder.isDirectory()); + + String taskEventFileName = + appId.toString().replaceAll("application", "task") + + "_m_000000" + + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION; + + String taskEventFilePath = outputDirTask + taskEventFileName; + File taskEventFile = new File(taskEventFilePath); + Assert.assertTrue("taskEventFileName: " + taskEventFilePath + + " does not exist.", + taskEventFile.exists()); + verifyEntity(taskEventFile, EventType.TASK_FINISHED.name(), + true, false, null, true); + + // check for task attempt event file + String outputDirTaskAttempt = + basePath + File.separator + "MAPREDUCE_TASK_ATTEMPT" + File.separator; + File taskAttemptFolder = new File(outputDirTaskAttempt); + Assert.assertTrue("TaskAttempt output directory: " + outputDirTaskAttempt + + " does not exist.", taskAttemptFolder.isDirectory()); + + String taskAttemptEventFileName = appId.toString().replaceAll( + "application", "attempt") + "_m_000000_0" + + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION; + + String taskAttemptEventFilePath = outputDirTaskAttempt + + taskAttemptEventFileName; + File taskAttemptEventFile = new File(taskAttemptEventFilePath); + Assert.assertTrue("taskAttemptEventFileName: " + taskAttemptEventFilePath + + " does not exist.", taskAttemptEventFile.exists()); + verifyEntity(taskAttemptEventFile, EventType.MAP_ATTEMPT_FINISHED.name(), + true, false, null, true); + } + + /** + * Verifies entity by reading the entity file written via FS impl. + * @param entityFile File to be read. + * @param eventId Event to be checked. + * @param chkMetrics If event is not null, this flag determines if metrics + * exist when the event is encountered. If event is null, we merely check + * if metrics exist in the entity file. + * @param chkCfg If event is not null, this flag determines if configs + * exist when the event is encountered. If event is null, we merely check + * if configs exist in the entity file. + * @param cfgsToVerify a set of configs which should exist in the entity file. + * @throws IOException + */ + private void verifyEntity(File entityFile, String eventId, + boolean chkMetrics, boolean chkCfg, Set cfgsToVerify, + boolean checkIdPrefix) throws IOException { + BufferedReader reader = null; + String strLine; + try { + reader = new BufferedReader(new FileReader(entityFile)); + long idPrefix = -1; + while ((strLine = reader.readLine()) != null) { + if (strLine.trim().length() > 0) { + org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity + entity = + FileSystemTimelineReaderImpl.getTimelineRecordFromJSON( + strLine.trim(), + org.apache.hadoop.yarn.api.records.timelineservice. + TimelineEntity.class); + + LOG.info("strLine.trim()= " + strLine.trim()); + if (checkIdPrefix) { + Assert.assertTrue("Entity ID prefix expected to be > 0" , + entity.getIdPrefix() > 0); + if (idPrefix == -1) { + idPrefix = entity.getIdPrefix(); + } else { + Assert.assertEquals("Entity ID prefix should be same across " + + "each publish of same entity", + idPrefix, entity.getIdPrefix()); + } + } + if (eventId == null) { + // Job metrics are published without any events for + // ApplicationEntity. There is also possibility that some other + // ApplicationEntity is published without events, hence loop till + // its found. Same applies to configs. + if (chkMetrics && entity.getMetrics().size() > 0) { + return; + } + if (chkCfg && entity.getConfigs().size() > 0) { + if (cfgsToVerify == null) { + return; + } else { + // Have configs to verify. Keep on removing configs from the set + // of configs to verify as they are found. When the all the + // entities have been looped through, we will check if the set + // is empty or not(indicating if all configs have been found or + // not). + for (Iterator itr = + cfgsToVerify.iterator(); itr.hasNext();) { + String config = itr.next(); + if (entity.getConfigs().containsKey(config)) { + itr.remove(); + } + } + // All the required configs have been verified, so return. + if (cfgsToVerify.isEmpty()) { + return; + } + } + } + } else { + for (TimelineEvent event : entity.getEvents()) { + if (event.getId().equals(eventId)) { + if (chkMetrics) { + assertTrue(entity.getMetrics().size() > 0); + } + if (chkCfg) { + assertTrue(entity.getConfigs().size() > 0); + if (cfgsToVerify != null) { + for (String cfg : cfgsToVerify) { + assertTrue(entity.getConfigs().containsKey(cfg)); + } + } + } + return; + } + } + } + } + } + if (cfgsToVerify != null) { + assertTrue(cfgsToVerify.isEmpty()); + return; + } + fail("Expected event : " + eventId + " not found in the file " + + entityFile); + } finally { + reader.close(); + } + } + @Test public void testMapreduceJobTimelineServiceEnabled() throws Exception { @@ -153,7 +482,7 @@ public class TestMRTimelineEventHandling { try { fs = FileSystem.get(conf); cluster = new MiniMRYarnCluster( - TestJobHistoryEventHandler.class.getSimpleName(), 1); + TestMRTimelineEventHandling.class.getSimpleName(), 1); cluster.init(conf); cluster.start(); conf.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS, diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java index 972391c5e13..ec6ad38f107 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java @@ -31,8 +31,10 @@ import java.util.Enumeration; import java.util.Iterator; import java.util.List; import java.util.Properties; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeoutException; +import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -52,10 +54,13 @@ import org.apache.hadoop.mapred.SortValidator.RecordStatsChecker.NonSplitableSeq import org.apache.hadoop.mapred.lib.IdentityMapper; import org.apache.hadoop.mapred.lib.IdentityReducer; import org.apache.hadoop.mapreduce.Cluster.JobTrackerStatus; -import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig; +import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster; +import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.StringUtils; - -import org.apache.commons.logging.Log; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl; +import com.google.common.base.Supplier; /** * Utilities used in unit test. @@ -150,6 +155,14 @@ public class UtilsForTests { return buf.toString(); } + public static String createConfigValue(int msgSize) { + StringBuilder sb = new StringBuilder(msgSize); + for (int i = 0; i < msgSize; i++) { + sb.append('a'); + } + return sb.toString(); + } + public static String safeGetCanonicalPath(File f) { try { String s = f.getCanonicalPath(); @@ -607,6 +620,29 @@ public class UtilsForTests { return job; } + public static void waitForAppFinished(RunningJob job, + MiniMRYarnCluster cluster) throws IOException { + ApplicationId appId = ApplicationId.newInstance( + Long.parseLong(job.getID().getJtIdentifier()), job.getID().getId()); + ConcurrentMap rmApps = cluster.getResourceManager() + .getRMContext().getRMApps(); + if (!rmApps.containsKey(appId)) { + throw new IOException("Job not found"); + } + final RMApp rmApp = rmApps.get(appId); + try { + GenericTestUtils.waitFor(new Supplier() { + @Override + public Boolean get() { + return RMAppImpl.isAppInFinalState(rmApp); + } + }, 1000, 1000 * 180); + } catch (TimeoutException | InterruptedException e1) { + throw new IOException("Yarn application with " + appId + " didn't finish " + + "did not reach finale State", e1); + } + } + // Run a job that will be succeeded and wait until it completes public static RunningJob runJobSucceed(JobConf conf, Path inDir, Path outDir) throws IOException { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/EntityWriterV2.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/EntityWriterV2.java new file mode 100644 index 00000000000..74d7b9476e6 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/EntityWriterV2.java @@ -0,0 +1,57 @@ +/** + * 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.mapreduce; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorManager; + +/** + * Base mapper for writing entities to the timeline service. Subclasses + * override {@link #writeEntities(Configuration, TimelineCollectorManager, + * org.apache.hadoop.mapreduce.Mapper.Context)} to create and write entities + * to the timeline service. + */ +abstract class EntityWriterV2 + extends org.apache.hadoop.mapreduce.Mapper + { + @Override + public void map(IntWritable key, IntWritable val, Context context) + throws IOException { + + // create the timeline collector manager wired with the writer + Configuration tlConf = new YarnConfiguration(); + TimelineCollectorManager manager = new TimelineCollectorManager("test"); + manager.init(tlConf); + manager.start(); + try { + // invoke the method to have the subclass write entities + writeEntities(tlConf, manager, context); + } finally { + manager.close(); + } + } + + protected abstract void writeEntities(Configuration tlConf, + TimelineCollectorManager manager, Context context) throws IOException; +} \ No newline at end of file diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileParser.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileParser.java index c290cd64f5e..5d9dc0b88eb 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileParser.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileParser.java @@ -28,6 +28,9 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser; import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo; +/** + * Used to parse job history and configuration files. + */ class JobHistoryFileParser { private static final Log LOG = LogFactory.getLog(JobHistoryFileParser.class); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayMapperV1.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayMapperV1.java index 5e106622f53..d553596b2fa 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayMapperV1.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayMapperV1.java @@ -20,36 +20,23 @@ package org.apache.hadoop.mapreduce; import java.io.IOException; import java.util.Collection; -import java.util.HashMap; -import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; -import java.util.regex.Matcher; -import java.util.regex.Pattern; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.Writable; -import org.apache.hadoop.mapreduce.Mapper.Context; -import org.apache.hadoop.mapreduce.TimelineServicePerformance.PerfCounters; -import org.apache.hadoop.mapreduce.JobHistoryFileReplayHelper; import org.apache.hadoop.mapreduce.JobHistoryFileReplayHelper.JobFiles; -import org.apache.hadoop.mapreduce.MRJobConfig; -import org.apache.hadoop.mapreduce.TypeConverter; +import org.apache.hadoop.mapreduce.TimelineServicePerformance.PerfCounters; import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo; import org.apache.hadoop.mapreduce.v2.api.records.JobId; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities; import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity; import org.apache.hadoop.yarn.client.api.TimelineClient; -import org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl; import org.apache.hadoop.yarn.exceptions.YarnException; @@ -66,7 +53,7 @@ class JobHistoryFileReplayMapperV1 extends public void map(IntWritable key, IntWritable val, Context context) throws IOException { // collect the apps it needs to process - TimelineClient tlc = new TimelineClientImpl(); + TimelineClient tlc = TimelineClient.createTimelineClient(); TimelineEntityConverterV1 converter = new TimelineEntityConverterV1(); JobHistoryFileReplayHelper helper = new JobHistoryFileReplayHelper(context); int replayMode = helper.getReplayMode(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayMapperV2.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayMapperV2.java new file mode 100644 index 00000000000..2ec48336e7f --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/JobHistoryFileReplayMapperV2.java @@ -0,0 +1,161 @@ +/** + * 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.mapreduce; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +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.mapred.JobID; +import org.apache.hadoop.mapreduce.JobHistoryFileReplayHelper.JobFiles; +import org.apache.hadoop.mapreduce.TimelineServicePerformance.PerfCounters; +import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo; +import org.apache.hadoop.mapreduce.v2.api.records.JobId; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.server.timelineservice.collector.AppLevelTimelineCollector; +import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext; +import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorManager; + +/** + * Mapper for TimelineServicePerformance that replays job history files to the + * timeline service v.2. + * + */ +class JobHistoryFileReplayMapperV2 extends EntityWriterV2 { + private static final Log LOG = + LogFactory.getLog(JobHistoryFileReplayMapperV2.class); + + @Override + protected void writeEntities(Configuration tlConf, + TimelineCollectorManager manager, Context context) throws IOException { + JobHistoryFileReplayHelper helper = new JobHistoryFileReplayHelper(context); + int replayMode = helper.getReplayMode(); + JobHistoryFileParser parser = helper.getParser(); + TimelineEntityConverterV2 converter = new TimelineEntityConverterV2(); + + // collect the apps it needs to process + Collection jobs = helper.getJobFiles(); + if (jobs.isEmpty()) { + LOG.info(context.getTaskAttemptID().getTaskID() + + " will process no jobs"); + } else { + LOG.info(context.getTaskAttemptID().getTaskID() + " will process " + + jobs.size() + " jobs"); + } + for (JobFiles job: jobs) { + // process each job + String jobIdStr = job.getJobId(); + // skip if either of the file is missing + if (job.getJobConfFilePath() == null || + job.getJobHistoryFilePath() == null) { + LOG.info(jobIdStr + " missing either the job history file or the " + + "configuration file. Skipping."); + continue; + } + LOG.info("processing " + jobIdStr + "..."); + JobId jobId = TypeConverter.toYarn(JobID.forName(jobIdStr)); + ApplicationId appId = jobId.getAppId(); + + // create the app level timeline collector and start it + AppLevelTimelineCollector collector = + new AppLevelTimelineCollector(appId); + manager.putIfAbsent(appId, collector); + try { + // parse the job info and configuration + JobInfo jobInfo = + parser.parseHistoryFile(job.getJobHistoryFilePath()); + Configuration jobConf = + parser.parseConfiguration(job.getJobConfFilePath()); + LOG.info("parsed the job history file and the configuration file " + + "for job " + jobIdStr); + + // set the context + // flow id: job name, flow run id: timestamp, user id + TimelineCollectorContext tlContext = + collector.getTimelineEntityContext(); + tlContext.setFlowName(jobInfo.getJobname()); + tlContext.setFlowRunId(jobInfo.getSubmitTime()); + tlContext.setUserId(jobInfo.getUsername()); + + // create entities from job history and write them + long totalTime = 0; + List entitySet = + converter.createTimelineEntities(jobInfo, jobConf); + LOG.info("converted them into timeline entities for job " + jobIdStr); + // use the current user for this purpose + UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); + long startWrite = System.nanoTime(); + try { + switch (replayMode) { + case JobHistoryFileReplayHelper.WRITE_ALL_AT_ONCE: + writeAllEntities(collector, entitySet, ugi); + break; + case JobHistoryFileReplayHelper.WRITE_PER_ENTITY: + writePerEntity(collector, entitySet, ugi); + break; + default: + break; + } + } catch (Exception e) { + context.getCounter(PerfCounters.TIMELINE_SERVICE_WRITE_FAILURES). + increment(1); + LOG.error("writing to the timeline service failed", e); + } + long endWrite = System.nanoTime(); + totalTime += TimeUnit.NANOSECONDS.toMillis(endWrite-startWrite); + int numEntities = entitySet.size(); + LOG.info("wrote " + numEntities + " entities in " + totalTime + " ms"); + + context.getCounter(PerfCounters.TIMELINE_SERVICE_WRITE_TIME). + increment(totalTime); + context.getCounter(PerfCounters.TIMELINE_SERVICE_WRITE_COUNTER). + increment(numEntities); + } finally { + manager.remove(appId); + context.progress(); // move it along + } + } + } + + private void writeAllEntities(AppLevelTimelineCollector collector, + List entitySet, UserGroupInformation ugi) + throws IOException { + TimelineEntities entities = new TimelineEntities(); + entities.setEntities(entitySet); + collector.putEntities(entities, ugi); + } + + private void writePerEntity(AppLevelTimelineCollector collector, + List entitySet, UserGroupInformation ugi) + throws IOException { + for (TimelineEntity entity : entitySet) { + TimelineEntities entities = new TimelineEntities(); + entities.addEntity(entity); + collector.putEntities(entities, ugi); + LOG.info("wrote entity " + entity.getId()); + } + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterConstants.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterConstants.java new file mode 100644 index 00000000000..2afe52ce15b --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterConstants.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.mapreduce; + +/** + * Constants for simple entity writers. + */ +final class SimpleEntityWriterConstants { + private SimpleEntityWriterConstants() {} + // constants for mtype = 1 + static final String KBS_SENT = "kbs sent"; + static final int KBS_SENT_DEFAULT = 1; + static final String TEST_TIMES = "testtimes"; + static final int TEST_TIMES_DEFAULT = 100; + static final String TIMELINE_SERVICE_PERFORMANCE_RUN_ID = + "timeline.server.performance.run.id"; + + /** + * To ensure that the compression really gets exercised, generate a + * random alphanumeric fixed length payload. + */ + static final char[] ALPHA_NUMS = new char[] {'a', 'b', 'c', 'd', 'e', 'f', + 'g', 'h', 'i', 'j', 'k', 'l', 'm', 'n', 'o', 'p', 'q', 'r', + 's', 't', 'u', 'v', 'w', 'x', 'y', 'z', 'A', 'B', 'C', 'D', + 'E', 'F', 'G', 'H', 'I', 'J', 'K', 'L', 'M', 'N', 'O', 'P', + 'Q', 'R', 'S', 'T', 'U', 'V', 'W', 'X', 'Y', 'Z', '1', '2', + '3', '4', '5', '6', '7', '8', '9', '0', ' '}; +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterV1.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterV1.java index 2c851e95869..cf2da2b52c9 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterV1.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterV1.java @@ -27,52 +27,32 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.Writable; -import org.apache.hadoop.mapreduce.Mapper.Context; import org.apache.hadoop.mapreduce.TimelineServicePerformance.PerfCounters; -import org.apache.hadoop.mapreduce.TaskAttemptID; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities; import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity; import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent; import org.apache.hadoop.yarn.client.api.TimelineClient; -import org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl; -import org.apache.hadoop.yarn.conf.YarnConfiguration; /** * Adds simple entities with random string payload, events, metrics, and * configuration. */ -class SimpleEntityWriterV1 extends - org.apache.hadoop.mapreduce.Mapper { +class SimpleEntityWriterV1 + extends org.apache.hadoop.mapreduce.Mapper + { private static final Log LOG = LogFactory.getLog(SimpleEntityWriterV1.class); - // constants for mtype = 1 - static final String KBS_SENT = "kbs sent"; - static final int KBS_SENT_DEFAULT = 1; - static final String TEST_TIMES = "testtimes"; - static final int TEST_TIMES_DEFAULT = 100; - static final String TIMELINE_SERVICE_PERFORMANCE_RUN_ID = - "timeline.server.performance.run.id"; - /** - * To ensure that the compression really gets exercised, generate a - * random alphanumeric fixed length payload - */ - private static char[] ALPHA_NUMS = new char[] { 'a', 'b', 'c', 'd', 'e', 'f', - 'g', 'h', 'i', 'j', 'k', 'l', 'm', 'n', 'o', 'p', 'q', 'r', - 's', 't', 'u', 'v', 'w', 'x', 'y', 'z', 'A', 'B', 'C', 'D', - 'E', 'F', 'G', 'H', 'I', 'J', 'K', 'L', 'M', 'N', 'O', 'P', - 'Q', 'R', 'S', 'T', 'U', 'V', 'W', 'X', 'Y', 'Z', '1', '2', - '3', '4', '5', '6', '7', '8', '9', '0', ' ' }; - - public void map(IntWritable key, IntWritable val, Context context) throws IOException { - TimelineClient tlc = new TimelineClientImpl(); + public void map(IntWritable key, IntWritable val, Context context) + throws IOException { + TimelineClient tlc = TimelineClient.createTimelineClient(); Configuration conf = context.getConfiguration(); - final int kbs = conf.getInt(KBS_SENT, KBS_SENT_DEFAULT); + final int kbs = conf.getInt(SimpleEntityWriterConstants.KBS_SENT, + SimpleEntityWriterConstants.KBS_SENT_DEFAULT); long totalTime = 0; - final int testtimes = conf.getInt(TEST_TIMES, TEST_TIMES_DEFAULT); + final int testtimes = conf.getInt(SimpleEntityWriterConstants.TEST_TIMES, + SimpleEntityWriterConstants.TEST_TIMES_DEFAULT); final Random rand = new Random(); final TaskAttemptID taskAttemptId = context.getTaskAttemptID(); final char[] payLoad = new char[kbs * 1024]; @@ -81,8 +61,8 @@ class SimpleEntityWriterV1 extends // Generate a fixed length random payload for (int xx = 0; xx < kbs * 1024; xx++) { int alphaNumIdx = - rand.nextInt(ALPHA_NUMS.length); - payLoad[xx] = ALPHA_NUMS[alphaNumIdx]; + rand.nextInt(SimpleEntityWriterConstants.ALPHA_NUMS.length); + payLoad[xx] = SimpleEntityWriterConstants.ALPHA_NUMS[alphaNumIdx]; } String entId = taskAttemptId + "_" + Integer.toString(i); final TimelineEntity entity = new TimelineEntity(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterV2.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterV2.java new file mode 100644 index 00000000000..4612f9cbc17 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/SimpleEntityWriterV2.java @@ -0,0 +1,133 @@ +/** + * 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.mapreduce; + +import java.io.IOException; +import java.util.Random; +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.mapreduce.TimelineServicePerformance.PerfCounters; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.server.timelineservice.collector.AppLevelTimelineCollector; +import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext; +import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorManager; + +/** + * Adds simple entities with random string payload, events, metrics, and + * configuration. + */ +class SimpleEntityWriterV2 extends EntityWriterV2 { + private static final Log LOG = LogFactory.getLog(SimpleEntityWriterV2.class); + + protected void writeEntities(Configuration tlConf, + TimelineCollectorManager manager, Context context) throws IOException { + Configuration conf = context.getConfiguration(); + // simulate the app id with the task id + int taskId = context.getTaskAttemptID().getTaskID().getId(); + long timestamp = conf.getLong( + SimpleEntityWriterConstants.TIMELINE_SERVICE_PERFORMANCE_RUN_ID, 0); + ApplicationId appId = ApplicationId.newInstance(timestamp, taskId); + + // create the app level timeline collector + AppLevelTimelineCollector collector = + new AppLevelTimelineCollector(appId); + manager.putIfAbsent(appId, collector); + + try { + // set the context + // flow id: job name, flow run id: timestamp, user id + TimelineCollectorContext tlContext = + collector.getTimelineEntityContext(); + tlContext.setFlowName(context.getJobName()); + tlContext.setFlowRunId(timestamp); + tlContext.setUserId(context.getUser()); + + final int kbs = conf.getInt(SimpleEntityWriterConstants.KBS_SENT, + SimpleEntityWriterConstants.KBS_SENT_DEFAULT); + + long totalTime = 0; + final int testtimes = conf.getInt(SimpleEntityWriterConstants.TEST_TIMES, + SimpleEntityWriterConstants.TEST_TIMES_DEFAULT); + final Random rand = new Random(); + final TaskAttemptID taskAttemptId = context.getTaskAttemptID(); + final char[] payLoad = new char[kbs * 1024]; + + for (int i = 0; i < testtimes; i++) { + // Generate a fixed length random payload + for (int xx = 0; xx < kbs * 1024; xx++) { + int alphaNumIdx = + rand.nextInt(SimpleEntityWriterConstants.ALPHA_NUMS.length); + payLoad[xx] = SimpleEntityWriterConstants.ALPHA_NUMS[alphaNumIdx]; + } + String entId = taskAttemptId + "_" + Integer.toString(i); + final TimelineEntity entity = new TimelineEntity(); + entity.setId(entId); + entity.setType("FOO_ATTEMPT"); + entity.addInfo("PERF_TEST", payLoad); + // add an event + TimelineEvent event = new TimelineEvent(); + event.setId("foo_event_id"); + event.setTimestamp(System.currentTimeMillis()); + event.addInfo("foo_event", "test"); + entity.addEvent(event); + // add a metric + TimelineMetric metric = new TimelineMetric(); + metric.setId("foo_metric"); + metric.addValue(System.currentTimeMillis(), 123456789L); + entity.addMetric(metric); + // add a config + entity.addConfig("foo", "bar"); + + TimelineEntities entities = new TimelineEntities(); + entities.addEntity(entity); + // use the current user for this purpose + UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); + long startWrite = System.nanoTime(); + try { + collector.putEntities(entities, ugi); + } catch (Exception e) { + context.getCounter(PerfCounters.TIMELINE_SERVICE_WRITE_FAILURES). + increment(1); + LOG.error("writing to the timeline service failed", e); + } + long endWrite = System.nanoTime(); + totalTime += TimeUnit.NANOSECONDS.toMillis(endWrite-startWrite); + } + LOG.info("wrote " + testtimes + " entities (" + kbs*testtimes + + " kB) in " + totalTime + " ms"); + context.getCounter(PerfCounters.TIMELINE_SERVICE_WRITE_TIME). + increment(totalTime); + context.getCounter(PerfCounters.TIMELINE_SERVICE_WRITE_COUNTER). + increment(testtimes); + context.getCounter(PerfCounters.TIMELINE_SERVICE_WRITE_KBS). + increment(kbs*testtimes); + } finally { + // clean up + manager.remove(appId); + } + } +} \ No newline at end of file diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineEntityConverterV1.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineEntityConverterV1.java index 79d123eb534..dcc3ce014b7 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineEntityConverterV1.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineEntityConverterV1.java @@ -25,11 +25,6 @@ import java.util.Set; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.mapreduce.Counter; -import org.apache.hadoop.mapreduce.CounterGroup; -import org.apache.hadoop.mapreduce.Counters; -import org.apache.hadoop.mapreduce.TaskAttemptID; -import org.apache.hadoop.mapreduce.TaskID; import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo; import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo; import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo; @@ -95,9 +90,10 @@ class TimelineEntityConverterV1 { return job; } - private Set createTaskAndTaskAttemptEntities(JobInfo jobInfo) { + private Set + createTaskAndTaskAttemptEntities(JobInfo jobInfo) { Set entities = new HashSet<>(); - Map taskInfoMap = jobInfo.getAllTasks(); + Map taskInfoMap = jobInfo.getAllTasks(); LOG.info("job " + jobInfo.getJobId()+ " has " + taskInfoMap.size() + " tasks"); for (TaskInfo taskInfo: taskInfoMap.values()) { @@ -129,7 +125,7 @@ class TimelineEntityConverterV1 { private Set createTaskAttemptEntities(TaskInfo taskInfo) { Set taskAttempts = new HashSet(); - Map taskAttemptInfoMap = + Map taskAttemptInfoMap = taskInfo.getAllTaskAttempts(); LOG.info("task " + taskInfo.getTaskId() + " has " + taskAttemptInfoMap.size() + " task attempts"); @@ -140,7 +136,8 @@ class TimelineEntityConverterV1 { return taskAttempts; } - private TimelineEntity createTaskAttemptEntity(TaskAttemptInfo taskAttemptInfo) { + private TimelineEntity + createTaskAttemptEntity(TaskAttemptInfo taskAttemptInfo) { TimelineEntity taskAttempt = new TimelineEntity(); taskAttempt.setEntityType(TASK_ATTEMPT); taskAttempt.setEntityId(taskAttemptInfo.getAttemptId().toString()); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineEntityConverterV2.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineEntityConverterV2.java new file mode 100644 index 00000000000..45812fe7dac --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineEntityConverterV2.java @@ -0,0 +1,206 @@ +/** + * 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.mapreduce; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo; +import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo; +import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; + +class TimelineEntityConverterV2 { + private static final Log LOG = + LogFactory.getLog(TimelineEntityConverterV2.class); + + static final String JOB = "MAPREDUCE_JOB"; + static final String TASK = "MAPREDUCE_TASK"; + static final String TASK_ATTEMPT = "MAPREDUCE_TASK_ATTEMPT"; + + /** + * Creates job, task, and task attempt entities based on the job history info + * and configuration. + * + * Note: currently these are plan timeline entities created for mapreduce + * types. These are not meant to be the complete and accurate entity set-up + * for mapreduce jobs. We do not leverage hierarchical timeline entities. If + * we create canonical mapreduce hierarchical timeline entities with proper + * parent-child relationship, we could modify this to use that instead. + * + * Note that we also do not add info to the YARN application entity, which + * would be needed for aggregation. + */ + public List createTimelineEntities(JobInfo jobInfo, + Configuration conf) { + List entities = new ArrayList<>(); + + // create the job entity + TimelineEntity job = createJobEntity(jobInfo, conf); + entities.add(job); + + // create the task and task attempt entities + List tasksAndAttempts = + createTaskAndTaskAttemptEntities(jobInfo); + entities.addAll(tasksAndAttempts); + + return entities; + } + + private TimelineEntity createJobEntity(JobInfo jobInfo, Configuration conf) { + TimelineEntity job = new TimelineEntity(); + job.setType(JOB); + job.setId(jobInfo.getJobId().toString()); + job.setCreatedTime(jobInfo.getSubmitTime()); + + job.addInfo("JOBNAME", jobInfo.getJobname()); + job.addInfo("USERNAME", jobInfo.getUsername()); + job.addInfo("JOB_QUEUE_NAME", jobInfo.getJobQueueName()); + job.addInfo("SUBMIT_TIME", jobInfo.getSubmitTime()); + job.addInfo("LAUNCH_TIME", jobInfo.getLaunchTime()); + job.addInfo("FINISH_TIME", jobInfo.getFinishTime()); + job.addInfo("JOB_STATUS", jobInfo.getJobStatus()); + job.addInfo("PRIORITY", jobInfo.getPriority()); + job.addInfo("TOTAL_MAPS", jobInfo.getTotalMaps()); + job.addInfo("TOTAL_REDUCES", jobInfo.getTotalReduces()); + job.addInfo("UBERIZED", jobInfo.getUberized()); + job.addInfo("ERROR_INFO", jobInfo.getErrorInfo()); + + // add metrics from total counters + // we omit the map counters and reduce counters for now as it's kind of + // awkward to put them (map/reduce/total counters are really a group of + // related counters) + Counters totalCounters = jobInfo.getTotalCounters(); + if (totalCounters != null) { + addMetrics(job, totalCounters); + } + // finally add configuration to the job + addConfiguration(job, conf); + LOG.info("converted job " + jobInfo.getJobId() + " to a timeline entity"); + return job; + } + + private void addConfiguration(TimelineEntity job, Configuration conf) { + for (Map.Entry e: conf) { + job.addConfig(e.getKey(), e.getValue()); + } + } + + private void addMetrics(TimelineEntity entity, Counters counters) { + for (CounterGroup g: counters) { + String groupName = g.getName(); + for (Counter c: g) { + String name = groupName + ":" + c.getName(); + TimelineMetric metric = new TimelineMetric(); + metric.setId(name); + metric.addValue(System.currentTimeMillis(), c.getValue()); + entity.addMetric(metric); + } + } + } + + private List createTaskAndTaskAttemptEntities( + JobInfo jobInfo) { + List entities = new ArrayList<>(); + Map taskInfoMap = jobInfo.getAllTasks(); + LOG.info("job " + jobInfo.getJobId()+ " has " + taskInfoMap.size() + + " tasks"); + for (TaskInfo taskInfo: taskInfoMap.values()) { + TimelineEntity task = createTaskEntity(taskInfo); + entities.add(task); + // add the task attempts from this task + Set taskAttempts = createTaskAttemptEntities(taskInfo); + entities.addAll(taskAttempts); + } + return entities; + } + + private TimelineEntity createTaskEntity(TaskInfo taskInfo) { + TimelineEntity task = new TimelineEntity(); + task.setType(TASK); + task.setId(taskInfo.getTaskId().toString()); + task.setCreatedTime(taskInfo.getStartTime()); + + task.addInfo("START_TIME", taskInfo.getStartTime()); + task.addInfo("FINISH_TIME", taskInfo.getFinishTime()); + task.addInfo("TASK_TYPE", taskInfo.getTaskType()); + task.addInfo("TASK_STATUS", taskInfo.getTaskStatus()); + task.addInfo("ERROR_INFO", taskInfo.getError()); + + // add metrics from counters + Counters counters = taskInfo.getCounters(); + if (counters != null) { + addMetrics(task, counters); + } + LOG.info("converted task " + taskInfo.getTaskId() + + " to a timeline entity"); + return task; + } + + private Set createTaskAttemptEntities(TaskInfo taskInfo) { + Set taskAttempts = new HashSet(); + Map taskAttemptInfoMap = + taskInfo.getAllTaskAttempts(); + LOG.info("task " + taskInfo.getTaskId() + " has " + + taskAttemptInfoMap.size() + " task attempts"); + for (TaskAttemptInfo taskAttemptInfo: taskAttemptInfoMap.values()) { + TimelineEntity taskAttempt = createTaskAttemptEntity(taskAttemptInfo); + taskAttempts.add(taskAttempt); + } + return taskAttempts; + } + + private TimelineEntity createTaskAttemptEntity( + TaskAttemptInfo taskAttemptInfo) { + TimelineEntity taskAttempt = new TimelineEntity(); + taskAttempt.setType(TASK_ATTEMPT); + taskAttempt.setId(taskAttemptInfo.getAttemptId().toString()); + taskAttempt.setCreatedTime(taskAttemptInfo.getStartTime()); + + taskAttempt.addInfo("START_TIME", taskAttemptInfo.getStartTime()); + taskAttempt.addInfo("FINISH_TIME", taskAttemptInfo.getFinishTime()); + taskAttempt.addInfo("MAP_FINISH_TIME", + taskAttemptInfo.getMapFinishTime()); + taskAttempt.addInfo("SHUFFLE_FINISH_TIME", + taskAttemptInfo.getShuffleFinishTime()); + taskAttempt.addInfo("SORT_FINISH_TIME", + taskAttemptInfo.getSortFinishTime()); + taskAttempt.addInfo("TASK_STATUS", taskAttemptInfo.getTaskStatus()); + taskAttempt.addInfo("STATE", taskAttemptInfo.getState()); + taskAttempt.addInfo("ERROR", taskAttemptInfo.getError()); + taskAttempt.addInfo("CONTAINER_ID", + taskAttemptInfo.getContainerId().toString()); + + // add metrics from counters + Counters counters = taskAttemptInfo.getCounters(); + if (counters != null) { + addMetrics(taskAttempt, counters); + } + LOG.info("converted task attempt " + taskAttemptInfo.getAttemptId() + + " to a timeline entity"); + return taskAttempt; + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineServicePerformance.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineServicePerformance.java index 0753d7fa35d..7fa04448407 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineServicePerformance.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TimelineServicePerformance.java @@ -23,8 +23,6 @@ import java.util.Date; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; -import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.SleepJob.SleepInputFormat; import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat; import org.apache.hadoop.util.GenericOptionsParser; @@ -46,15 +44,19 @@ public class TimelineServicePerformance extends Configured implements Tool { System.err.println( "Usage: [-m ] number of mappers (default: " + NUM_MAPS_DEFAULT + ")\n" + - " [-v] timeline service version\n" + - " [-mtype ]\n" + - " 1. simple entity write mapper (default)\n" + + " [-v] timeline service version (default: " + + TIMELINE_SERVICE_VERSION_1 + ")\n" + + " 1. version 1.x\n" + + " 2. version 2.x\n" + + " [-mtype ] (default: " + + SIMPLE_ENTITY_WRITER + ")\n" + + " 1. simple entity write mapper\n" + " 2. jobhistory files replay mapper\n" + " [-s <(KBs)test>] number of KB per put (mtype=1, default: " + - SimpleEntityWriterV1.KBS_SENT_DEFAULT + " KB)\n" + + SimpleEntityWriterConstants.KBS_SENT_DEFAULT + " KB)\n" + " [-t] package sending iterations per mapper (mtype=1, default: " + - SimpleEntityWriterV1.TEST_TIMES_DEFAULT + ")\n" + - " [-d ] root path of job history files (mtype=2)\n" + + SimpleEntityWriterConstants.TEST_TIMES_DEFAULT + ")\n" + + " [-d ] hdfs root path of job history files (mtype=2)\n" + " [-r ] (mtype=2)\n" + " 1. write all entities for a job in one put (default)\n" + " 2. write one entity at a time\n"); @@ -78,8 +80,7 @@ public class TimelineServicePerformance extends Configured implements Tool { try { if ("-v".equals(args[i])) { timeline_service_version = Integer.parseInt(args[++i]); - } - if ("-m".equals(args[i])) { + } else if ("-m".equals(args[i])) { if (Integer.parseInt(args[++i]) > 0) { job.getConfiguration() .setInt(MRJobConfig.NUM_MAPS, Integer.parseInt(args[i])); @@ -88,11 +89,12 @@ public class TimelineServicePerformance extends Configured implements Tool { mapperType = Integer.parseInt(args[++i]); } else if ("-s".equals(args[i])) { if (Integer.parseInt(args[++i]) > 0) { - conf.setInt(SimpleEntityWriterV1.KBS_SENT, Integer.parseInt(args[i])); + conf.setInt(SimpleEntityWriterConstants.KBS_SENT, + Integer.parseInt(args[i])); } } else if ("-t".equals(args[i])) { if (Integer.parseInt(args[++i]) > 0) { - conf.setInt(SimpleEntityWriterV1.TEST_TIMES, + conf.setInt(SimpleEntityWriterConstants.TEST_TIMES, Integer.parseInt(args[i])); } } else if ("-d".equals(args[i])) { @@ -113,28 +115,41 @@ public class TimelineServicePerformance extends Configured implements Tool { } // handle mapper-specific settings - switch (timeline_service_version) { - case TIMELINE_SERVICE_VERSION_1: - default: - switch (mapperType) { - case JOB_HISTORY_FILE_REPLAY_MAPPER: - job.setMapperClass(JobHistoryFileReplayMapperV1.class); - String processingPath = - conf.get(JobHistoryFileReplayHelper.PROCESSING_PATH); - if (processingPath == null || processingPath.isEmpty()) { - System.out.println("processing path is missing while mtype = 2"); - return printUsage() == 0; - } + switch (mapperType) { + case JOB_HISTORY_FILE_REPLAY_MAPPER: + String processingPath = + conf.get(JobHistoryFileReplayHelper.PROCESSING_PATH); + if (processingPath == null || processingPath.isEmpty()) { + System.out.println("processing path is missing while mtype = 2"); + return printUsage() == 0; + } + switch (timeline_service_version) { + case TIMELINE_SERVICE_VERSION_2: + job.setMapperClass(JobHistoryFileReplayMapperV2.class); break; - case SIMPLE_ENTITY_WRITER: + case TIMELINE_SERVICE_VERSION_1: default: - job.setMapperClass(SimpleEntityWriterV1.class); - // use the current timestamp as the "run id" of the test: this will - // be used as simulating the cluster timestamp for apps - conf.setLong(SimpleEntityWriterV1.TIMELINE_SERVICE_PERFORMANCE_RUN_ID, - System.currentTimeMillis()); + job.setMapperClass(JobHistoryFileReplayMapperV1.class); break; } + break; + case SIMPLE_ENTITY_WRITER: + default: + // use the current timestamp as the "run id" of the test: this will + // be used as simulating the cluster timestamp for apps + conf.setLong( + SimpleEntityWriterConstants.TIMELINE_SERVICE_PERFORMANCE_RUN_ID, + System.currentTimeMillis()); + switch (timeline_service_version) { + case TIMELINE_SERVICE_VERSION_2: + job.setMapperClass(SimpleEntityWriterV2.class); + break; + case TIMELINE_SERVICE_VERSION_1: + default: + job.setMapperClass(SimpleEntityWriterV1.class); + break; + } + break; } return true; } @@ -164,25 +179,46 @@ public class TimelineServicePerformance extends Configured implements Tool { Date startTime = new Date(); System.out.println("Job started: " + startTime); int ret = job.waitForCompletion(true) ? 0 : 1; - org.apache.hadoop.mapreduce.Counters counters = job.getCounters(); - long writetime = - counters.findCounter(PerfCounters.TIMELINE_SERVICE_WRITE_TIME).getValue(); - long writecounts = - counters.findCounter(PerfCounters.TIMELINE_SERVICE_WRITE_COUNTER).getValue(); - long writesize = - counters.findCounter(PerfCounters.TIMELINE_SERVICE_WRITE_KBS).getValue(); - double transacrate = writecounts * 1000 / (double)writetime; - double iorate = writesize * 1000 / (double)writetime; - int numMaps = - Integer.parseInt(job.getConfiguration().get(MRJobConfig.NUM_MAPS)); + if (job.isSuccessful()) { + org.apache.hadoop.mapreduce.Counters counters = job.getCounters(); + long writecounts = + counters.findCounter( + PerfCounters.TIMELINE_SERVICE_WRITE_COUNTER).getValue(); + long writefailures = + counters.findCounter( + PerfCounters.TIMELINE_SERVICE_WRITE_FAILURES).getValue(); + if (writefailures > 0 && writefailures == writecounts) { + // see if we have a complete failure to write + System.out.println("Job failed: all writes failed!"); + } else { + long writetime = + counters.findCounter( + PerfCounters.TIMELINE_SERVICE_WRITE_TIME).getValue(); + long writesize = + counters.findCounter( + PerfCounters.TIMELINE_SERVICE_WRITE_KBS).getValue(); + if (writetime == 0L) { + // see if write time is zero (normally shouldn't happen) + System.out.println("Job failed: write time is 0!"); + } else { + double transacrate = writecounts * 1000 / (double)writetime; + double iorate = writesize * 1000 / (double)writetime; + int numMaps = + Integer.parseInt( + job.getConfiguration().get(MRJobConfig.NUM_MAPS)); - System.out.println("TRANSACTION RATE (per mapper): " + transacrate + - " ops/s"); - System.out.println("IO RATE (per mapper): " + iorate + " KB/s"); + System.out.println("TRANSACTION RATE (per mapper): " + transacrate + + " ops/s"); + System.out.println("IO RATE (per mapper): " + iorate + " KB/s"); - System.out.println("TRANSACTION RATE (total): " + transacrate*numMaps + - " ops/s"); - System.out.println("IO RATE (total): " + iorate*numMaps + " KB/s"); + System.out.println("TRANSACTION RATE (total): " + + transacrate*numMaps + " ops/s"); + System.out.println("IO RATE (total): " + iorate*numMaps + " KB/s"); + } + } + } else { + System.out.println("Job failed: " + job.getStatus().getFailureInfo()); + } return ret; } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java index 7465d7cd6d5..d0a7446997c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java @@ -68,6 +68,7 @@ public class MiniMRYarnCluster extends MiniYARNCluster { private static final Log LOG = LogFactory.getLog(MiniMRYarnCluster.class); private JobHistoryServer historyServer; private JobHistoryServerWrapper historyServerWrapper; + private static final String TIMELINE_AUX_SERVICE_NAME = "timeline_collector"; public MiniMRYarnCluster(String testName) { this(testName, 1); @@ -169,8 +170,25 @@ public class MiniMRYarnCluster extends MiniYARNCluster { conf.set(MRConfig.MASTER_ADDRESS, "test"); // The default is local because of // which shuffle doesn't happen //configure the shuffle service in NM - conf.setStrings(YarnConfiguration.NM_AUX_SERVICES, - new String[] { ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID }); + String[] nmAuxServices = conf.getStrings(YarnConfiguration.NM_AUX_SERVICES); + // if need to enable TIMELINE_AUX_SERVICE_NAME + boolean enableTimelineAuxService = false; + if (nmAuxServices != null) { + for (String nmAuxService: nmAuxServices) { + if (nmAuxService.equals(TIMELINE_AUX_SERVICE_NAME)) { + enableTimelineAuxService = true; + break; + } + } + } + if (enableTimelineAuxService) { + conf.setStrings(YarnConfiguration.NM_AUX_SERVICES, + new String[] {ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID, + TIMELINE_AUX_SERVICE_NAME}); + } else { + conf.setStrings(YarnConfiguration.NM_AUX_SERVICES, + new String[] {ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID}); + } conf.setClass(String.format(YarnConfiguration.NM_AUX_SERVICE_FMT, ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID), ShuffleHandler.class, Service.class); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/test/MapredTestDriver.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/test/MapredTestDriver.java index 4643bf3892c..a2a13d5ba91 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/test/MapredTestDriver.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/test/MapredTestDriver.java @@ -18,6 +18,19 @@ package org.apache.hadoop.test; +import org.apache.hadoop.fs.DFSCIOTest; +import org.apache.hadoop.fs.DistributedFSCheck; +import org.apache.hadoop.fs.JHLogAnalyzer; +import org.apache.hadoop.fs.TestDFSIO; +import org.apache.hadoop.fs.TestFileSystem; +import org.apache.hadoop.fs.loadGenerator.DataGenerator; +import org.apache.hadoop.fs.loadGenerator.LoadGenerator; +import org.apache.hadoop.fs.loadGenerator.LoadGeneratorMR; +import org.apache.hadoop.fs.loadGenerator.StructureGenerator; +import org.apache.hadoop.fs.slive.SliveTest; +import org.apache.hadoop.hdfs.NNBench; +import org.apache.hadoop.hdfs.NNBenchWithoutMR; +import org.apache.hadoop.io.FileBench; import org.apache.hadoop.io.TestSequenceFile; import org.apache.hadoop.mapred.BigMapOutput; import org.apache.hadoop.mapred.GenericMRLoadGenerator; @@ -28,28 +41,14 @@ import org.apache.hadoop.mapred.TestMapRed; import org.apache.hadoop.mapred.TestSequenceFileInputFormat; import org.apache.hadoop.mapred.TestTextInputFormat; import org.apache.hadoop.mapred.ThreadedMapBenchmark; -import org.apache.hadoop.mapreduce.TimelineServicePerformance; import org.apache.hadoop.mapreduce.FailJob; import org.apache.hadoop.mapreduce.GrowingSleepJob; import org.apache.hadoop.mapreduce.LargeSorter; import org.apache.hadoop.mapreduce.MiniHadoopClusterManager; import org.apache.hadoop.mapreduce.SleepJob; +import org.apache.hadoop.mapreduce.TimelineServicePerformance; import org.apache.hadoop.util.ProgramDriver; -import org.apache.hadoop.hdfs.NNBench; -import org.apache.hadoop.hdfs.NNBenchWithoutMR; -import org.apache.hadoop.fs.TestFileSystem; -import org.apache.hadoop.fs.TestDFSIO; -import org.apache.hadoop.fs.DFSCIOTest; -import org.apache.hadoop.fs.DistributedFSCheck; -import org.apache.hadoop.io.FileBench; -import org.apache.hadoop.fs.JHLogAnalyzer; -import org.apache.hadoop.fs.loadGenerator.DataGenerator; -import org.apache.hadoop.fs.loadGenerator.LoadGenerator; -import org.apache.hadoop.fs.loadGenerator.LoadGeneratorMR; -import org.apache.hadoop.fs.loadGenerator.StructureGenerator; -import org.apache.hadoop.fs.slive.SliveTest; - /** * Driver for Map-reduce tests. * @@ -95,7 +94,8 @@ public class MapredTestDriver { pgd.addClass("gsleep", GrowingSleepJob.class, "A sleep job whose mappers create 1MB buffer for every record."); pgd.addClass("timelineperformance", TimelineServicePerformance.class, - "A job that launches mappers to test timlineserver performance."); + "A job that launches mappers to test timline service " + + "performance."); pgd.addClass("nnbench", NNBench.class, "A benchmark that stresses the namenode w/ MR."); pgd.addClass("nnbenchWithoutMR", NNBenchWithoutMR.class, diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index e41c4c1bc20..1beaef9349e 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -48,6 +48,11 @@ 2.11.0 + 0.8.2.1 + + 1.2.6 + 2.5.1 + ${project.version} 1.0.13 @@ -319,6 +324,31 @@ test-jar + + org.apache.hadoop + hadoop-yarn-server-applicationhistoryservice + ${project.version} + + + + org.apache.hadoop + hadoop-yarn-server-timelineservice + ${project.version} + + + + org.apache.hadoop + hadoop-yarn-server-timelineservice + ${project.version} + test-jar + + + + org.apache.hadoop + hadoop-yarn-server-timelineservice-hbase + ${project.version} + + org.apache.hadoop hadoop-yarn-applications-distributedshell @@ -489,6 +519,11 @@ commons-compress 1.4.1 + + org.apache.commons + commons-csv + 1.0 + xmlenc xmlenc @@ -1110,6 +1145,73 @@ mssql-jdbc ${mssql.version} + + org.apache.hbase + hbase-common + ${hbase.version} + + + jdk.tools + jdk.tools + + + + + org.apache.hbase + hbase-common + ${hbase.version} + test + tests + + + org.apache.hbase + hbase-client + ${hbase.version} + + + + jdk.tools + jdk.tools + + + + + org.apache.hbase + hbase-server + ${hbase.version} + + + org.apache.hbase + hbase-server + ${hbase.version} + test + tests + + + org.apache.hbase + hbase-testing-util + ${hbase.version} + test + true + + + org.jruby + jruby-complete + + + org.apache.hadoop + hadoop-hdfs + + + org.apache.hadoop + hadoop-minicluster + + + jdk.tools + jdk.tools + + + diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml index 993f42a11a6..e20fdecd9c5 100644 --- a/hadoop-project/src/site/site.xml +++ b/hadoop-project/src/site/site.xml @@ -127,6 +127,7 @@ + @@ -147,6 +148,7 @@ + diff --git a/hadoop-yarn-project/hadoop-yarn/bin/yarn b/hadoop-yarn-project/hadoop-yarn/bin/yarn index dfcb797d616..892d15f2f45 100644 --- a/hadoop-yarn-project/hadoop-yarn/bin/yarn +++ b/hadoop-yarn-project/hadoop-yarn/bin/yarn @@ -69,6 +69,7 @@ function print_usage(){ echo " Use -remove-application-from-state-store for " echo " removing application from RMStateStore." echo " nodemanager run a nodemanager on each slave" + echo " timelinereader run the timeline reader server" echo " timelineserver run the timeline server" echo " rmadmin admin tools" echo " router run the Router daemon" @@ -246,6 +247,8 @@ elif [ "$COMMAND" = "queue" ] ; then YARN_OPTS="$YARN_OPTS $YARN_CLIENT_OPTS" elif [ "$COMMAND" = "resourcemanager" ] ; then CLASSPATH=${CLASSPATH}:$YARN_CONF_DIR/rm-config/log4j.properties + CLASSPATH=${CLASSPATH}:"$HADOOP_YARN_HOME/$YARN_DIR/timelineservice/*" + CLASSPATH=${CLASSPATH}:"$HADOOP_YARN_HOME/$YARN_DIR/timelineservice/lib/*" CLASS='org.apache.hadoop.yarn.server.resourcemanager.ResourceManager' YARN_OPTS="$YARN_OPTS $YARN_RESOURCEMANAGER_OPTS" if [ "$YARN_RESOURCEMANAGER_HEAPSIZE" != "" ]; then @@ -260,6 +263,10 @@ elif [ "$COMMAND" = "historyserver" ] ; then if [ "$YARN_HISTORYSERVER_HEAPSIZE" != "" ]; then JAVA_HEAP_MAX="-Xmx""$YARN_HISTORYSERVER_HEAPSIZE""m" fi +elif [ "$COMMAND" = "timelinereader" ] ; then + CLASSPATH=${CLASSPATH}:"$HADOOP_YARN_HOME/$YARN_DIR/timelineservice/*" + CLASSPATH=${CLASSPATH}:"$HADOOP_YARN_HOME/$YARN_DIR/timelineservice/lib/*" + CLASS='org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderServer' elif [ "$COMMAND" = "timelineserver" ] ; then CLASSPATH=${CLASSPATH}:$YARN_CONF_DIR/timelineserver-config/log4j.properties CLASS='org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryServer' @@ -276,6 +283,8 @@ elif [ "$COMMAND" = "sharedcachemanager" ] ; then fi elif [ "$COMMAND" = "nodemanager" ] ; then CLASSPATH=${CLASSPATH}:$YARN_CONF_DIR/nm-config/log4j.properties + CLASSPATH=${CLASSPATH}:"$HADOOP_YARN_HOME/$YARN_DIR/timelineservice/*" + CLASSPATH=${CLASSPATH}:"$HADOOP_YARN_HOME/$YARN_DIR/timelineservice/lib/*" CLASS='org.apache.hadoop.yarn.server.nodemanager.NodeManager' YARN_OPTS="$YARN_OPTS -server $YARN_NODEMANAGER_OPTS" if [ "$YARN_NODEMANAGER_HEAPSIZE" != "" ]; then diff --git a/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd b/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd index 3a650cdb5ed..708a7c55976 100644 --- a/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd +++ b/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd @@ -155,7 +155,7 @@ if "%1" == "--loglevel" ( set yarncommands=resourcemanager nodemanager proxyserver rmadmin version jar ^ application applicationattempt cluster container node queue logs daemonlog historyserver ^ - timelineserver router classpath + timelineserver router timelinereader classpath for %%i in ( %yarncommands% ) do ( if %yarn-command% == %%i set yarncommand=true ) @@ -219,6 +219,8 @@ goto :eof :resourcemanager set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR%\rm-config\log4j.properties + set CLASSPATH=%CLASSPATH%;%HADOOP_YARN_HOME%\%YARN_DIR%\timelineservice\* + set CLASSPATH=%CLASSPATH%;%HADOOP_YARN_HOME%\%YARN_DIR%\timelineservice\lib\* set CLASS=org.apache.hadoop.yarn.server.resourcemanager.ResourceManager set YARN_OPTS=%YARN_OPTS% %YARN_RESOURCEMANAGER_OPTS% if defined YARN_RESOURCEMANAGER_HEAPSIZE ( @@ -252,8 +254,17 @@ goto :eof set YARN_OPTS=%YARN_OPTS% %HADOOP_ROUTER_OPTS% goto :eof +:timelinereader + set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR%\timelineserver-config\log4j.properties + set CLASSPATH=%CLASSPATH%;%HADOOP_YARN_HOME%\%YARN_DIR%\timelineservice\* + set CLASSPATH=%CLASSPATH%;%HADOOP_YARN_HOME%\%YARN_DIR%\timelineservice\lib\* + set CLASS=org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderServer + goto :eof + :nodemanager set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR%\nm-config\log4j.properties + set CLASSPATH=%CLASSPATH%;%HADOOP_YARN_HOME%\%YARN_DIR%\timelineservice\* + set CLASSPATH=%CLASSPATH%;%HADOOP_YARN_HOME%\%YARN_DIR%\timelineservice\lib\* set CLASS=org.apache.hadoop.yarn.server.nodemanager.NodeManager set YARN_OPTS=%YARN_OPTS% -server %HADOOP_NODEMANAGER_OPTS% if defined YARN_NODEMANAGER_HEAPSIZE ( @@ -328,6 +339,7 @@ goto :eof @echo nodemanager run a nodemanager on each slave @echo router run the Router daemon @echo timelineserver run the timeline server + @echo timelinereader run the timeline reader server @echo rmadmin admin tools @echo version print the version @echo jar ^ run a jar file diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml index 2664cd523d7..6825a36ebdd 100644 --- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml +++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml @@ -118,6 +118,23 @@ + + + + + + + + + + + + + + + + + @@ -517,6 +534,16 @@ + + + + + + + + + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml index 33eca121f8f..4a5bd58f8c9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml @@ -47,6 +47,10 @@ commons-logging commons-logging + + javax.xml.bind + jaxb-api + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java index 64573c0d6bc..b2b40a0fbce 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java @@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceStability.Evolving; import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; +import org.apache.hadoop.yarn.api.records.CollectorInfo; import org.apache.hadoop.yarn.api.records.AMCommand; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerResourceDecrease; @@ -96,7 +97,8 @@ public abstract class AllocateResponse { /** * Use {@link AllocateResponse#newInstance(int, List, List, List, Resource, - * AMCommand, int, PreemptionMessage, List, Token, List)} instead + * AMCommand, int, PreemptionMessage, List, Token, List, CollectorInfo)} + * instead. * @param responseId responseId * @param completedContainers completedContainers * @param allocatedContainers allocatedContainers @@ -117,10 +119,14 @@ public abstract class AllocateResponse { Resource availResources, AMCommand command, int numClusterNodes, PreemptionMessage preempt, List nmTokens, List increasedContainers, - List decreasedContainers) { - return newInstance(responseId, completedContainers, allocatedContainers, - updatedNodes, availResources, command, numClusterNodes, preempt, - nmTokens); + List decreasedContainers, + CollectorInfo collectorInfo) { + return AllocateResponse.newBuilder().responseId(responseId) + .completedContainersStatuses(completedContainers) + .allocatedContainers(allocatedContainers) + .updatedNodes(updatedNodes).availableResources(availResources) + .amCommand(command).nmTokens(nmTokens).collectorInfo(collectorInfo) + .build(); } @Public @@ -147,14 +153,15 @@ public abstract class AllocateResponse { List allocatedContainers, List updatedNodes, Resource availResources, AMCommand command, int numClusterNodes, PreemptionMessage preempt, List nmTokens, Token amRMToken, - List updatedContainers) { + List updatedContainers, CollectorInfo collectorInfo) { return AllocateResponse.newBuilder().numClusterNodes(numClusterNodes) .responseId(responseId) .completedContainersStatuses(completedContainers) .allocatedContainers(allocatedContainers).updatedNodes(updatedNodes) .availableResources(availResources).amCommand(command) .preemptionMessage(preempt).nmTokens(nmTokens) - .updatedContainers(updatedContainers).amRmToken(amRMToken).build(); + .updatedContainers(updatedContainers).amRmToken(amRMToken) + .collectorInfo(collectorInfo).build(); } /** @@ -345,6 +352,20 @@ public abstract class AllocateResponse { @Unstable public abstract void setApplicationPriority(Priority priority); + /** + * The data associated with the collector that belongs to this app. Contains + * address and token alongwith identification information. + * + * @return The data of collector that belong to this attempt + */ + @Public + @Unstable + public abstract CollectorInfo getCollectorInfo(); + + @Private + @Unstable + public abstract void setCollectorInfo(CollectorInfo info); + /** * Get the list of container update errors to inform the * Application Master about the container updates that could not be @@ -543,6 +564,50 @@ public abstract class AllocateResponse { return this; } + /** + * Set the applicationPriority of the response. + * @see AllocateResponse#setApplicationPriority(Priority) + * @param applicationPriority + * applicationPriority of the response + * @return {@link AllocateResponseBuilder} + */ + @Private + @Unstable + public AllocateResponseBuilder applicationPriority( + Priority applicationPriority) { + allocateResponse.setApplicationPriority(applicationPriority); + return this; + } + + /** + * Set the collectorInfo of the response. + * @see AllocateResponse#setCollectorInfo(CollectorInfo) + * @param collectorInfo collectorInfo of the response which + * contains collector address, RM id, version and collector token. + * @return {@link AllocateResponseBuilder} + */ + @Private + @Unstable + public AllocateResponseBuilder collectorInfo( + CollectorInfo collectorInfo) { + allocateResponse.setCollectorInfo(collectorInfo); + return this; + } + + /** + * Set the updateErrors of the response. + * @see AllocateResponse#setUpdateErrors(List) + * @param updateErrors updateErrors of the response + * @return {@link AllocateResponseBuilder} + */ + @Private + @Unstable + public AllocateResponseBuilder updateErrors( + List updateErrors) { + allocateResponse.setUpdateErrors(updateErrors); + return this; + } + /** * Return generated {@link AllocateResponse} object. * @return {@link AllocateResponse} @@ -567,4 +632,5 @@ public abstract class AllocateResponse { */ @Deprecated public abstract List getDecreasedContainers(); + } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/CollectorInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/CollectorInfo.java new file mode 100644 index 00000000000..30450d655f8 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/CollectorInfo.java @@ -0,0 +1,59 @@ +/** + * 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.records; + +import org.apache.hadoop.classification.InterfaceStability.Evolving; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.yarn.util.Records; + +/** + * Collector info containing collector address and collector token passed from + * RM to AM in Allocate Response. + */ +@Public +@Evolving +public abstract class CollectorInfo { + + protected static final long DEFAULT_TIMESTAMP_VALUE = -1; + + public static CollectorInfo newInstance(String collectorAddr) { + return newInstance(collectorAddr, null); + } + + public static CollectorInfo newInstance(String collectorAddr, Token token) { + CollectorInfo amCollectorInfo = + Records.newRecord(CollectorInfo.class); + amCollectorInfo.setCollectorAddr(collectorAddr); + amCollectorInfo.setCollectorToken(token); + return amCollectorInfo; + } + + public abstract String getCollectorAddr(); + + public abstract void setCollectorAddr(String addr); + + /** + * Get delegation token for app collector which AM will use to publish + * entities. + * @return the delegation token for app collector. + */ + public abstract Token getCollectorToken(); + + public abstract void setCollectorToken(Token token); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEntity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEntity.java index a43259b5ff1..e695050e2b9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEntity.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEntity.java @@ -34,6 +34,7 @@ import javax.xml.bind.annotation.XmlRootElement; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Evolving; +import org.apache.hadoop.yarn.util.TimelineServiceHelper; /** *

@@ -231,11 +232,8 @@ public class TimelineEntity implements Comparable { */ public void setRelatedEntities( Map> relatedEntities) { - if (relatedEntities != null && !(relatedEntities instanceof HashMap)) { - this.relatedEntities = new HashMap>(relatedEntities); - } else { - this.relatedEntities = (HashMap>) relatedEntities; - } + this.relatedEntities = TimelineServiceHelper.mapCastToHashMap( + relatedEntities); } /** @@ -297,11 +295,8 @@ public class TimelineEntity implements Comparable { * a map of primary filters */ public void setPrimaryFilters(Map> primaryFilters) { - if (primaryFilters != null && !(primaryFilters instanceof HashMap)) { - this.primaryFilters = new HashMap>(primaryFilters); - } else { - this.primaryFilters = (HashMap>) primaryFilters; - } + this.primaryFilters = + TimelineServiceHelper.mapCastToHashMap(primaryFilters); } /** @@ -350,11 +345,7 @@ public class TimelineEntity implements Comparable { * a map of other information */ public void setOtherInfo(Map otherInfo) { - if (otherInfo != null && !(otherInfo instanceof HashMap)) { - this.otherInfo = new HashMap(otherInfo); - } else { - this.otherInfo = (HashMap) otherInfo; - } + this.otherInfo = TimelineServiceHelper.mapCastToHashMap(otherInfo); } /** diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEvent.java index 73b2e729c1e..d5611f8da99 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEvent.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEvent.java @@ -29,6 +29,7 @@ import javax.xml.bind.annotation.XmlRootElement; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Evolving; +import org.apache.hadoop.yarn.util.TimelineServiceHelper; /** * The class that contains the information of an event that is related to some @@ -135,11 +136,8 @@ public class TimelineEvent implements Comparable { * a map of of the information of the event */ public void setEventInfo(Map eventInfo) { - if (eventInfo != null && !(eventInfo instanceof HashMap)) { - this.eventInfo = new HashMap(eventInfo); - } else { - this.eventInfo = (HashMap) eventInfo; - } + this.eventInfo = TimelineServiceHelper.mapCastToHashMap( + eventInfo); } @Override diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/ApplicationAttemptEntity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/ApplicationAttemptEntity.java new file mode 100644 index 00000000000..053d84edb55 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/ApplicationAttemptEntity.java @@ -0,0 +1,41 @@ +/* + * 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.records.timelineservice; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * This entity represents an application attempt. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public class ApplicationAttemptEntity extends HierarchicalTimelineEntity { + public ApplicationAttemptEntity() { + super(TimelineEntityType.YARN_APPLICATION_ATTEMPT.toString()); + } + + public ApplicationAttemptEntity(TimelineEntity entity) { + super(entity); + if (!entity.getType().equals( + TimelineEntityType.YARN_APPLICATION_ATTEMPT.toString())) { + throw new IllegalArgumentException("Incompatible entity type: " + + getId()); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/ApplicationEntity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/ApplicationEntity.java new file mode 100644 index 00000000000..20226aa8e9c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/ApplicationEntity.java @@ -0,0 +1,80 @@ +/* + * 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.records.timelineservice; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * This entity represents an application. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public class ApplicationEntity extends HierarchicalTimelineEntity { + public static final String QUEUE_INFO_KEY = + TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "QUEUE"; + + public ApplicationEntity() { + super(TimelineEntityType.YARN_APPLICATION.toString()); + } + + public ApplicationEntity(TimelineEntity entity) { + super(entity); + if (!entity.getType().equals( + TimelineEntityType.YARN_APPLICATION.toString())) { + throw new IllegalArgumentException("Incompatible entity type: " + + getId()); + } + } + + public String getQueue() { + return getInfo().get(QUEUE_INFO_KEY).toString(); + } + + public void setQueue(String queue) { + addInfo(QUEUE_INFO_KEY, queue); + } + + /** + * Checks if the input TimelineEntity object is an ApplicationEntity. + * + * @param te TimelineEntity object. + * @return true if input is an ApplicationEntity, false otherwise + */ + public static boolean isApplicationEntity(TimelineEntity te) { + return (te == null ? false + : te.getType().equals(TimelineEntityType.YARN_APPLICATION.toString())); + } + + /** + * @param te TimelineEntity object. + * @param eventId event with this id needs to be fetched + * @return TimelineEvent if TimelineEntity contains the desired event. + */ + public static TimelineEvent getApplicationEvent(TimelineEntity te, + String eventId) { + if (isApplicationEntity(te)) { + for (TimelineEvent event : te.getEvents()) { + if (event.getId().equals(eventId)) { + return event; + } + } + } + return null; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/ClusterEntity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/ClusterEntity.java new file mode 100644 index 00000000000..1f96505f98c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/ClusterEntity.java @@ -0,0 +1,40 @@ +/* + * 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.records.timelineservice; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * This entity represents a YARN cluster. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public class ClusterEntity extends HierarchicalTimelineEntity { + public ClusterEntity() { + super(TimelineEntityType.YARN_CLUSTER.toString()); + } + + public ClusterEntity(TimelineEntity entity) { + super(entity); + if (!entity.getType().equals(TimelineEntityType.YARN_CLUSTER.toString())) { + throw new IllegalArgumentException("Incompatible entity type: " + + getId()); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/ContainerEntity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/ContainerEntity.java new file mode 100644 index 00000000000..f61920f4950 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/ContainerEntity.java @@ -0,0 +1,41 @@ +/* + * 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.records.timelineservice; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * This entity represents a container belonging to an application attempt. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public class ContainerEntity extends HierarchicalTimelineEntity { + public ContainerEntity() { + super(TimelineEntityType.YARN_CONTAINER.toString()); + } + + public ContainerEntity(TimelineEntity entity) { + super(entity); + if (!entity.getType().equals( + TimelineEntityType.YARN_CONTAINER.toString())) { + throw new IllegalArgumentException("Incompatible entity type: " + + getId()); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowActivityEntity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowActivityEntity.java new file mode 100644 index 00000000000..cf19328f675 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowActivityEntity.java @@ -0,0 +1,191 @@ +/* + * 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.records.timelineservice; + +import java.util.Collection; +import java.util.Date; +import java.util.NavigableSet; +import java.util.TreeSet; + +import javax.xml.bind.annotation.XmlElement; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; + +/** + * Entity that represents a record for flow activity. It's essentially a + * container entity for flow runs with limited information. + */ +@Public +@Unstable +public class FlowActivityEntity extends TimelineEntity { + public static final String CLUSTER_INFO_KEY = + TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "CLUSTER"; + public static final String DATE_INFO_KEY = + TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "DATE"; + public static final String USER_INFO_KEY = + TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "USER"; + public static final String FLOW_NAME_INFO_KEY = + TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "FLOW_NAME"; + + private final NavigableSet flowRuns = new TreeSet<>(); + + public FlowActivityEntity() { + super(TimelineEntityType.YARN_FLOW_ACTIVITY.toString()); + // set config to null + setConfigs(null); + } + + public FlowActivityEntity(String cluster, long time, String user, + String flowName) { + this(); + setCluster(cluster); + setDate(time); + setUser(user); + setFlowName(flowName); + } + + public FlowActivityEntity(TimelineEntity entity) { + super(entity); + if (!TimelineEntityType.YARN_FLOW_ACTIVITY.matches(entity.getType())) { + throw new IllegalArgumentException("Incompatible entity type: " + + getId()); + } + // set config to null + setConfigs(null); + } + + @XmlElement(name = "id") + @Override + public String getId() { + // flow activity: cluster/day/user@flow_name + String id = super.getId(); + if (id == null) { + StringBuilder sb = new StringBuilder(); + sb.append(getCluster()); + sb.append('/'); + sb.append(getDate().getTime()); + sb.append('/'); + sb.append(getUser()); + sb.append('@'); + sb.append(getFlowName()); + id = sb.toString(); + setId(id); + } + return id; + } + + @Override + public int compareTo(TimelineEntity entity) { + int comparison = getType().compareTo(entity.getType()); + if (comparison == 0) { + // order by cluster, date (descending), user, and flow name + FlowActivityEntity other = (FlowActivityEntity)entity; + int clusterComparison = getCluster().compareTo(other.getCluster()); + if (clusterComparison != 0) { + return clusterComparison; + } + int dateComparisonDescending = + (int)(other.getDate().getTime() - getDate().getTime()); // descending + if (dateComparisonDescending != 0) { + return dateComparisonDescending; // descending + } + int userComparison = getUser().compareTo(other.getUser()); + if (userComparison != 0) { + return userComparison; + } + return getFlowName().compareTo(other.getFlowName()); + } else { + return comparison; + } + } + + /** + * Reuse the base class equals method. + */ + @Override + public boolean equals(Object obj) { + return super.equals(obj); + } + + /** + * Reuse the base class hashCode method. + */ + @Override + public int hashCode() { + return super.hashCode(); + } + + public String getCluster() { + return (String)getInfo().get(CLUSTER_INFO_KEY); + } + + public void setCluster(String cluster) { + addInfo(CLUSTER_INFO_KEY, cluster); + } + + public Date getDate() { + Object date = getInfo().get(DATE_INFO_KEY); + if (date != null) { + if (date instanceof Long) { + return new Date((Long)date); + } else if (date instanceof Date) { + return (Date)date; + } + } + return null; + } + + public void setDate(long time) { + Date date = new Date(time); + addInfo(DATE_INFO_KEY, date); + } + + public String getUser() { + return (String)getInfo().get(USER_INFO_KEY); + } + + public void setUser(String user) { + addInfo(USER_INFO_KEY, user); + } + + public String getFlowName() { + return (String)getInfo().get(FLOW_NAME_INFO_KEY); + } + + public void setFlowName(String flowName) { + addInfo(FLOW_NAME_INFO_KEY, flowName); + } + + public void addFlowRun(FlowRunEntity run) { + flowRuns.add(run); + } + + public void addFlowRuns(Collection runs) { + flowRuns.addAll(runs); + } + + @XmlElement(name = "flowruns") + public NavigableSet getFlowRuns() { + return flowRuns; + } + + public int getNumberOfRuns() { + return flowRuns.size(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowRunEntity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowRunEntity.java new file mode 100644 index 00000000000..5c666cdeb7b --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowRunEntity.java @@ -0,0 +1,126 @@ +/* + * 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.records.timelineservice; + +import javax.xml.bind.annotation.XmlElement; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * This entity represents a flow run. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public class FlowRunEntity extends HierarchicalTimelineEntity { + public static final String USER_INFO_KEY = + TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "USER"; + public static final String FLOW_NAME_INFO_KEY = + TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "FLOW_NAME"; + public static final String FLOW_VERSION_INFO_KEY = + TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "FLOW_VERSION"; + public static final String FLOW_RUN_ID_INFO_KEY = + TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "FLOW_RUN_ID"; + public static final String FLOW_RUN_END_TIME = + TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "FLOW_RUN_END_TIME"; + + public FlowRunEntity() { + super(TimelineEntityType.YARN_FLOW_RUN.toString()); + // set config to null + setConfigs(null); + } + + public FlowRunEntity(TimelineEntity entity) { + super(entity); + if (!entity.getType().equals( + TimelineEntityType.YARN_FLOW_RUN.toString())) { + throw new IllegalArgumentException("Incompatible entity type: " + + getId()); + } + // set config to null + setConfigs(null); + } + + @XmlElement(name = "id") + @Override + public String getId() { + //Flow id schema: user@flow_name(or id)/run_id + String id = super.getId(); + if (id == null) { + StringBuilder sb = new StringBuilder(); + sb.append(getInfo().get(USER_INFO_KEY).toString()); + sb.append('@'); + sb.append(getInfo().get(FLOW_NAME_INFO_KEY).toString()); + sb.append('/'); + sb.append(getInfo().get(FLOW_RUN_ID_INFO_KEY).toString()); + id = sb.toString(); + setId(id); + } + return id; + } + + public String getUser() { + return (String)getInfo().get(USER_INFO_KEY); + } + + public void setUser(String user) { + addInfo(USER_INFO_KEY, user); + } + + public String getName() { + return (String)getInfo().get(FLOW_NAME_INFO_KEY); + } + + public void setName(String name) { + addInfo(FLOW_NAME_INFO_KEY, name); + } + + public String getVersion() { + return (String)getInfo().get(FLOW_VERSION_INFO_KEY); + } + + public void setVersion(String version) { + addInfo(FLOW_VERSION_INFO_KEY, version); + } + + public long getRunId() { + Object runId = getInfo().get(FLOW_RUN_ID_INFO_KEY); + return runId == null ? 0L : ((Number) runId).longValue(); + } + + public void setRunId(long runId) { + addInfo(FLOW_RUN_ID_INFO_KEY, runId); + } + + public long getStartTime() { + return getCreatedTime(); + } + + public void setStartTime(long startTime) { + setCreatedTime(startTime); + } + + public long getMaxEndTime() { + Object time = getInfo().get(FLOW_RUN_END_TIME); + return time == null ? 0L : ((Number) time).longValue(); + } + + public void setMaxEndTime(long endTime) { + addInfo(FLOW_RUN_END_TIME, endTime); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/HierarchicalTimelineEntity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/HierarchicalTimelineEntity.java new file mode 100644 index 00000000000..4744e39c1a1 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/HierarchicalTimelineEntity.java @@ -0,0 +1,133 @@ +/* + * 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.records.timelineservice; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; + +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + +/** + * This class extends timeline entity and defines parent-child relationships + * with other entities. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public abstract class HierarchicalTimelineEntity extends TimelineEntity { + public static final String PARENT_INFO_KEY = + TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "PARENT_ENTITY"; + public static final String CHILDREN_INFO_KEY = + TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "CHILDREN_ENTITY"; + + HierarchicalTimelineEntity(TimelineEntity entity) { + super(entity); + } + + HierarchicalTimelineEntity(String type) { + super(type); + } + + public Identifier getParent() { + Object obj = getInfo().get(PARENT_INFO_KEY); + if (obj != null) { + if (obj instanceof Identifier) { + return (Identifier) obj; + } else { + throw new YarnRuntimeException( + "Parent info is invalid identifier object"); + } + } + return null; + } + + public void setParent(Identifier parent) { + validateParent(parent.getType()); + addInfo(PARENT_INFO_KEY, parent); + } + + public void setParent(String type, String id) { + setParent(new Identifier(type, id)); + } + + @SuppressWarnings("unchecked") + public Set getChildren() { + Object identifiers = getInfo().get(CHILDREN_INFO_KEY); + if (identifiers == null) { + return new HashSet<>(); + } + TimelineEntityType thisType = TimelineEntityType.valueOf(getType()); + if (identifiers instanceof Set) { + for (Object identifier : (Set) identifiers) { + if (!(identifier instanceof Identifier)) { + throw new YarnRuntimeException( + "Children info contains invalid identifier object"); + } else { + validateChild((Identifier) identifier, thisType); + } + } + } else { + throw new YarnRuntimeException( + "Children info is invalid identifier set"); + } + Set children = (Set) identifiers; + return children; + } + + public void setChildren(Set children) { + addInfo(CHILDREN_INFO_KEY, children); + } + + public void addChildren(Set children) { + TimelineEntityType thisType = TimelineEntityType.valueOf(getType()); + for (Identifier child : children) { + validateChild(child, thisType); + } + Set existingChildren = getChildren(); + existingChildren.addAll(children); + setChildren(existingChildren); + } + + public void addChild(Identifier child) { + addChildren(Collections.singleton(child)); + } + + public void addChild(String type, String id) { + addChild(new Identifier(type, id)); + } + + private void validateParent(String type) { + TimelineEntityType parentType = TimelineEntityType.valueOf(type); + TimelineEntityType thisType = TimelineEntityType.valueOf(getType()); + if (!thisType.isParent(parentType)) { + throw new IllegalArgumentException( + type + " is not the acceptable parent of " + this.getType()); + } + } + + private void validateChild(Identifier child, TimelineEntityType thisType) { + TimelineEntityType childType = TimelineEntityType.valueOf(child.getType()); + if (!thisType.isChild(childType)) { + throw new IllegalArgumentException( + child.getType() + " is not the acceptable child of " + + this.getType()); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/QueueEntity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/QueueEntity.java new file mode 100644 index 00000000000..b654450fc2c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/QueueEntity.java @@ -0,0 +1,40 @@ +/* + * 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.records.timelineservice; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * This entity represents a queue. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public class QueueEntity extends HierarchicalTimelineEntity { + public QueueEntity() { + super(TimelineEntityType.YARN_QUEUE.toString()); + } + + public QueueEntity(TimelineEntity entity) { + super(entity); + if (!entity.getType().equals(TimelineEntityType.YARN_QUEUE.toString())) { + throw new IllegalArgumentException("Incompatible entity type: " + + getId()); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntities.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntities.java new file mode 100644 index 00000000000..63989e682eb --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntities.java @@ -0,0 +1,62 @@ +/* + * 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.records.timelineservice; + +import java.util.ArrayList; +import java.util.List; + +import javax.xml.bind.annotation.XmlAccessType; +import javax.xml.bind.annotation.XmlAccessorType; +import javax.xml.bind.annotation.XmlElement; +import javax.xml.bind.annotation.XmlRootElement; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * This class hosts a set of timeline entities. + */ +@XmlRootElement(name = "entities") +@XmlAccessorType(XmlAccessType.NONE) +@InterfaceAudience.Public +@InterfaceStability.Unstable +public class TimelineEntities { + + private List entities = new ArrayList<>(); + + public TimelineEntities() { + + } + + @XmlElement(name = "entities") + public List getEntities() { + return entities; + } + + public void setEntities(List timelineEntities) { + this.entities = timelineEntities; + } + + public void addEntities(List timelineEntities) { + this.entities.addAll(timelineEntities); + } + + public void addEntity(TimelineEntity entity) { + entities.add(entity); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntity.java new file mode 100644 index 00000000000..845e2cc547b --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntity.java @@ -0,0 +1,610 @@ +/* + * 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.records.timelineservice; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.NavigableSet; +import java.util.Set; +import java.util.TreeSet; + +import javax.xml.bind.annotation.XmlAccessType; +import javax.xml.bind.annotation.XmlAccessorType; +import javax.xml.bind.annotation.XmlElement; +import javax.xml.bind.annotation.XmlRootElement; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.yarn.util.TimelineServiceHelper; +import org.codehaus.jackson.annotate.JsonSetter; + +/** + * The basic timeline entity data structure for timeline service v2. Timeline + * entity objects are not thread safe and should not be accessed concurrently. + * All collection members will be initialized into empty collections. Two + * timeline entities are equal iff. their type and id are identical. + * + * All non-primitive type, non-collection members will be initialized into null. + * User should set the type and id of a timeline entity to make it valid (can be + * checked by using the {@link #isValid()} method). Callers to the getters + * should perform null checks for non-primitive type, non-collection members. + * + * Callers are recommended not to alter the returned collection objects from the + * getters. + */ +@XmlRootElement(name = "entity") +@XmlAccessorType(XmlAccessType.NONE) +@InterfaceAudience.Public +@InterfaceStability.Unstable +public class TimelineEntity implements Comparable { + protected final static String SYSTEM_INFO_KEY_PREFIX = "SYSTEM_INFO_"; + public final static long DEFAULT_ENTITY_PREFIX = 0L; + + /** + * Identifier of timeline entity(entity id + entity type). + */ + @XmlRootElement(name = "identifier") + @XmlAccessorType(XmlAccessType.NONE) + public static class Identifier { + private String type; + private String id; + + public Identifier(String type, String id) { + this.type = type; + this.id = id; + } + + public Identifier() { + + } + + @XmlElement(name = "type") + public String getType() { + return type; + } + + public void setType(String entityType) { + this.type = entityType; + } + + @XmlElement(name = "id") + public String getId() { + return id; + } + + public void setId(String entityId) { + this.id = entityId; + } + + @Override + public String toString() { + return "TimelineEntity[" + + "type='" + type + '\'' + + ", id='" + id + '\'' + "]"; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((id == null) ? 0 : id.hashCode()); + result = + prime * result + ((type == null) ? 0 : type.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (!(obj instanceof Identifier)) { + return false; + } + Identifier other = (Identifier) obj; + if (id == null) { + if (other.getId() != null) { + return false; + } + } else if (!id.equals(other.getId())) { + return false; + } + if (type == null) { + if (other.getType() != null) { + return false; + } + } else if (!type.equals(other.getType())) { + return false; + } + return true; + } + } + + private TimelineEntity real; + private Identifier identifier; + private HashMap info = new HashMap<>(); + private HashMap configs = new HashMap<>(); + private Set metrics = new HashSet<>(); + // events should be sorted by timestamp in descending order + private NavigableSet events = new TreeSet<>(); + private HashMap> isRelatedToEntities = new HashMap<>(); + private HashMap> relatesToEntities = new HashMap<>(); + private Long createdTime; + private long idPrefix; + + public TimelineEntity() { + identifier = new Identifier(); + } + + /** + *

+ * The constuctor is used to construct a proxy {@link TimelineEntity} or its + * subclass object from the real entity object that carries information. + *

+ * + *

+ * It is usually used in the case where we want to recover class polymorphism + * after deserializing the entity from its JSON form. + *

+ * @param entity the real entity that carries information + */ + public TimelineEntity(TimelineEntity entity) { + real = entity.getReal(); + } + + protected TimelineEntity(String type) { + this(); + identifier.type = type; + } + + @XmlElement(name = "type") + public String getType() { + if (real == null) { + return identifier.type; + } else { + return real.getType(); + } + } + + public void setType(String type) { + if (real == null) { + identifier.type = type; + } else { + real.setType(type); + } + } + + @XmlElement(name = "id") + public String getId() { + if (real == null) { + return identifier.id; + } else { + return real.getId(); + } + } + + public void setId(String id) { + if (real == null) { + identifier.id = id; + } else { + real.setId(id); + } + } + + public Identifier getIdentifier() { + if (real == null) { + return identifier; + } else { + return real.getIdentifier(); + } + } + + public void setIdentifier(Identifier entityIdentifier) { + if (real == null) { + this.identifier = entityIdentifier; + } else { + real.setIdentifier(entityIdentifier); + } + } + + // required by JAXB + @InterfaceAudience.Private + @XmlElement(name = "info") + public HashMap getInfoJAXB() { + if (real == null) { + return info; + } else { + return real.getInfoJAXB(); + } + } + + public Map getInfo() { + if (real == null) { + return info; + } else { + return real.getInfo(); + } + } + + public void setInfo(Map entityInfos) { + if (real == null) { + this.info = TimelineServiceHelper.mapCastToHashMap(entityInfos); + } else { + real.setInfo(entityInfos); + } + } + + public void addInfo(Map entityInfos) { + if (real == null) { + this.info.putAll(entityInfos); + } else { + real.addInfo(entityInfos); + } + } + + public void addInfo(String key, Object value) { + if (real == null) { + info.put(key, value); + } else { + real.addInfo(key, value); + } + } + + // required by JAXB + @InterfaceAudience.Private + @XmlElement(name = "configs") + public HashMap getConfigsJAXB() { + if (real == null) { + return configs; + } else { + return real.getConfigsJAXB(); + } + } + + public Map getConfigs() { + if (real == null) { + return configs; + } else { + return real.getConfigs(); + } + } + + public void setConfigs(Map entityConfigs) { + if (real == null) { + this.configs = TimelineServiceHelper.mapCastToHashMap(entityConfigs); + } else { + real.setConfigs(entityConfigs); + } + } + + public void addConfigs(Map entityConfigs) { + if (real == null) { + this.configs.putAll(entityConfigs); + } else { + real.addConfigs(entityConfigs); + } + } + + public void addConfig(String key, String value) { + if (real == null) { + configs.put(key, value); + } else { + real.addConfig(key, value); + } + } + + @XmlElement(name = "metrics") + public Set getMetrics() { + if (real == null) { + return metrics; + } else { + return real.getMetrics(); + } + } + + public void setMetrics(Set entityMetrics) { + if (real == null) { + this.metrics = entityMetrics; + } else { + real.setMetrics(entityMetrics); + } + } + + public void addMetrics(Set entityMetrics) { + if (real == null) { + this.metrics.addAll(entityMetrics); + } else { + real.addMetrics(entityMetrics); + } + } + + public void addMetric(TimelineMetric metric) { + if (real == null) { + metrics.add(metric); + } else { + real.addMetric(metric); + } + } + + @XmlElement(name = "events") + public NavigableSet getEvents() { + if (real == null) { + return events; + } else { + return real.getEvents(); + } + } + + public void setEvents(NavigableSet entityEvents) { + if (real == null) { + this.events = entityEvents; + } else { + real.setEvents(entityEvents); + } + } + + public void addEvents(Set entityEvents) { + if (real == null) { + this.events.addAll(entityEvents); + } else { + real.addEvents(entityEvents); + } + } + + public void addEvent(TimelineEvent event) { + if (real == null) { + events.add(event); + } else { + real.addEvent(event); + } + } + + public Map> getIsRelatedToEntities() { + if (real == null) { + return isRelatedToEntities; + } else { + return real.getIsRelatedToEntities(); + } + } + + // required by JAXB + @InterfaceAudience.Private + @XmlElement(name = "isrelatedto") + public HashMap> getIsRelatedToEntitiesJAXB() { + if (real == null) { + return isRelatedToEntities; + } else { + return real.getIsRelatedToEntitiesJAXB(); + } + } + + @JsonSetter("isrelatedto") + public void setIsRelatedToEntities( + Map> isRelatedTo) { + if (real == null) { + this.isRelatedToEntities = + TimelineServiceHelper.mapCastToHashMap(isRelatedTo); + } else { + real.setIsRelatedToEntities(isRelatedTo); + } + } + + public void addIsRelatedToEntities( + Map> isRelatedTo) { + if (real == null) { + for (Map.Entry> entry : isRelatedTo.entrySet()) { + Set ids = this.isRelatedToEntities.get(entry.getKey()); + if (ids == null) { + ids = new HashSet<>(); + this.isRelatedToEntities.put(entry.getKey(), ids); + } + ids.addAll(entry.getValue()); + } + } else { + real.addIsRelatedToEntities(isRelatedTo); + } + } + + public void addIsRelatedToEntity(String type, String id) { + if (real == null) { + Set ids = isRelatedToEntities.get(type); + if (ids == null) { + ids = new HashSet<>(); + isRelatedToEntities.put(type, ids); + } + ids.add(id); + } else { + real.addIsRelatedToEntity(type, id); + } + } + + // required by JAXB + @InterfaceAudience.Private + @XmlElement(name = "relatesto") + public HashMap> getRelatesToEntitiesJAXB() { + if (real == null) { + return relatesToEntities; + } else { + return real.getRelatesToEntitiesJAXB(); + } + } + + public Map> getRelatesToEntities() { + if (real == null) { + return relatesToEntities; + } else { + return real.getRelatesToEntities(); + } + } + + public void addRelatesToEntities(Map> relatesTo) { + if (real == null) { + for (Map.Entry> entry : relatesTo.entrySet()) { + Set ids = this.relatesToEntities.get(entry.getKey()); + if (ids == null) { + ids = new HashSet<>(); + this.relatesToEntities.put(entry.getKey(), ids); + } + ids.addAll(entry.getValue()); + } + } else { + real.addRelatesToEntities(relatesTo); + } + } + + public void addRelatesToEntity(String type, String id) { + if (real == null) { + Set ids = relatesToEntities.get(type); + if (ids == null) { + ids = new HashSet<>(); + relatesToEntities.put(type, ids); + } + ids.add(id); + } else { + real.addRelatesToEntity(type, id); + } + } + + @JsonSetter("relatesto") + public void setRelatesToEntities(Map> relatesTo) { + if (real == null) { + this.relatesToEntities = + TimelineServiceHelper.mapCastToHashMap(relatesTo); + } else { + real.setRelatesToEntities(relatesTo); + } + } + + @XmlElement(name = "createdtime") + public Long getCreatedTime() { + if (real == null) { + return createdTime; + } else { + return real.getCreatedTime(); + } + } + + @JsonSetter("createdtime") + public void setCreatedTime(Long createdTs) { + if (real == null) { + this.createdTime = createdTs; + } else { + real.setCreatedTime(createdTs); + } + } + + /** + * Set UID in info which will be then used for query by UI. + * @param uidKey key for UID in info. + * @param uId UID to be set for the key. + */ + public void setUID(String uidKey, String uId) { + if (real == null) { + info.put(uidKey, uId); + } else { + real.addInfo(uidKey, uId); + } + } + + public boolean isValid() { + return (getId() != null && getType() != null); + } + + // When get hashCode for a timeline entity, or check if two timeline entities + // are equal, we only compare their identifiers (id and type) + @Override + public int hashCode() { + return getIdentifier().hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (!(obj instanceof TimelineEntity)) { + return false; + } + TimelineEntity other = (TimelineEntity) obj; + return getIdentifier().equals(other.getIdentifier()); + } + + @Override + public int compareTo(TimelineEntity other) { + int comparison = getType().compareTo(other.getType()); + if (comparison == 0) { + if (getIdPrefix() > other.getIdPrefix()) { + // Descending order by entity id prefix + return -1; + } else if (getIdPrefix() < other.getIdPrefix()) { + return 1; + } else { + return getId().compareTo(other.getId()); + } + } else { + return comparison; + } + } + + protected TimelineEntity getReal() { + return real == null ? this : real; + } + + public String toString() { + if (real == null) { + return identifier.toString(); + } else { + return real.toString(); + } + } + + @XmlElement(name = "idprefix") + public long getIdPrefix() { + if (real == null) { + return idPrefix; + } else { + return real.getIdPrefix(); + } + } + + /** + * Sets idPrefix for an entity. + *

+ * Note: Entities will be stored in the order of idPrefix specified. + * If users decide to set idPrefix for an entity, they MUST provide + * the same prefix for every update of this entity. + *

+ * Example:
+   * TimelineEntity entity = new TimelineEntity();
+   * entity.setIdPrefix(value);
+   * 
+ * Users can use {@link TimelineServiceHelper#invertLong(long)} to invert + * the prefix if necessary. + * + * @param entityIdPrefix prefix for an entity. + */ + @JsonSetter("idprefix") + public void setIdPrefix(long entityIdPrefix) { + if (real == null) { + this.idPrefix = entityIdPrefix; + } else { + real.setIdPrefix(entityIdPrefix); + } + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntityType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntityType.java new file mode 100644 index 00000000000..8fcc2ae08d4 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntityType.java @@ -0,0 +1,101 @@ +/* + * 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.records.timelineservice; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Defines type of entity. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public enum TimelineEntityType { + YARN_CLUSTER, + YARN_FLOW_RUN, + YARN_APPLICATION, + YARN_APPLICATION_ATTEMPT, + YARN_CONTAINER, + YARN_USER, + YARN_QUEUE, + YARN_FLOW_ACTIVITY; + + /** + * Whether the input type can be a parent of this entity. + * + * @param type entity type. + * @return true, if this entity type is parent of passed entity type, false + * otherwise. + */ + public boolean isParent(TimelineEntityType type) { + switch (this) { + case YARN_CLUSTER: + return false; + case YARN_FLOW_RUN: + return YARN_FLOW_RUN == type || YARN_CLUSTER == type; + case YARN_APPLICATION: + return YARN_FLOW_RUN == type || YARN_CLUSTER == type; + case YARN_APPLICATION_ATTEMPT: + return YARN_APPLICATION == type; + case YARN_CONTAINER: + return YARN_APPLICATION_ATTEMPT == type; + case YARN_QUEUE: + return YARN_QUEUE == type; + default: + return false; + } + } + + /** + * Whether the input type can be a child of this entity. + * + * @param type entity type. + * @return true, if this entity type is child of passed entity type, false + * otherwise. + */ + public boolean isChild(TimelineEntityType type) { + switch (this) { + case YARN_CLUSTER: + return YARN_FLOW_RUN == type || YARN_APPLICATION == type; + case YARN_FLOW_RUN: + return YARN_FLOW_RUN == type || YARN_APPLICATION == type; + case YARN_APPLICATION: + return YARN_APPLICATION_ATTEMPT == type; + case YARN_APPLICATION_ATTEMPT: + return YARN_CONTAINER == type; + case YARN_CONTAINER: + return false; + case YARN_QUEUE: + return YARN_QUEUE == type; + default: + return false; + } + } + + /** + * Whether the type of this entity matches the type indicated by the input + * argument. + * + * @param typeString entity type represented as a string. + * @return true, if string representation of this entity type matches the + * entity type passed. + */ + public boolean matches(String typeString) { + return toString().equals(typeString); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEvent.java new file mode 100644 index 00000000000..87fc291fc1d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEvent.java @@ -0,0 +1,133 @@ +/* + * 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.records.timelineservice; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.yarn.util.TimelineServiceHelper; + +import javax.xml.bind.annotation.XmlAccessType; +import javax.xml.bind.annotation.XmlAccessorType; +import javax.xml.bind.annotation.XmlElement; +import javax.xml.bind.annotation.XmlRootElement; +import java.util.HashMap; +import java.util.Map; + +/** + * This class contains the information of an event that belongs to an entity. + * Users are free to define what the event means, such as starting an + * application, container being allocated, etc. + */ +@XmlRootElement(name = "event") +@XmlAccessorType(XmlAccessType.NONE) +@InterfaceAudience.Public +@InterfaceStability.Unstable +public class TimelineEvent implements Comparable { + public static final long INVALID_TIMESTAMP = 0L; + + private String id; + private HashMap info = new HashMap<>(); + private long timestamp; + + public TimelineEvent() { + + } + + @XmlElement(name = "id") + public String getId() { + return id; + } + + public void setId(String eventId) { + this.id = eventId; + } + + // required by JAXB + @InterfaceAudience.Private + @XmlElement(name = "info") + public HashMap getInfoJAXB() { + return info; + } + + public Map getInfo() { + return info; + } + + public void setInfo(Map infos) { + this.info = TimelineServiceHelper.mapCastToHashMap(infos); + } + + public void addInfo(Map infos) { + this.info.putAll(infos); + } + + public void addInfo(String key, Object value) { + info.put(key, value); + } + + @XmlElement(name = "timestamp") + public long getTimestamp() { + return timestamp; + } + + public void setTimestamp(long ts) { + this.timestamp = ts; + } + + public boolean isValid() { + return (id != null && timestamp != INVALID_TIMESTAMP); + } + + @Override + public int hashCode() { + int result = (int) (timestamp ^ (timestamp >>> 32)); + result = 31 * result + id.hashCode(); + return result; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof TimelineEvent)) { + return false; + } + + TimelineEvent event = (TimelineEvent) o; + + if (timestamp != event.timestamp) { + return false; + } + if (!id.equals(event.id)) { + return false; + } + return true; + } + + @Override + public int compareTo(TimelineEvent other) { + if (timestamp > other.timestamp) { + return -1; + } else if (timestamp < other.timestamp) { + return 1; + } else { + return id.compareTo(other.id); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetric.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetric.java new file mode 100644 index 00000000000..5c908d69a6a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetric.java @@ -0,0 +1,289 @@ +/* + * 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.records.timelineservice; + +import java.util.Collections; +import java.util.Map; +import java.util.TreeMap; + +import javax.xml.bind.annotation.XmlAccessType; +import javax.xml.bind.annotation.XmlAccessorType; +import javax.xml.bind.annotation.XmlElement; +import javax.xml.bind.annotation.XmlRootElement; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; + +/** + * This class contains the information of a metric that is related to some + * entity. Metric can either be a time series or single value. + */ +@XmlRootElement(name = "metric") +@XmlAccessorType(XmlAccessType.NONE) +@InterfaceAudience.Public +@InterfaceStability.Unstable +public class TimelineMetric { + + /** + * Type of metric. + */ + public static enum Type { + SINGLE_VALUE, + TIME_SERIES + } + + private Type type; + private String id; + // By default, not to do any aggregation operations. This field will NOT be + // persisted (like a "transient" member). + private TimelineMetricOperation realtimeAggregationOp + = TimelineMetricOperation.NOP; + + private TreeMap values + = new TreeMap<>(Collections.reverseOrder()); + + public TimelineMetric() { + this(Type.SINGLE_VALUE); + } + + public TimelineMetric(Type type) { + this.type = type; + } + + + @XmlElement(name = "type") + public Type getType() { + return type; + } + + public void setType(Type metricType) { + this.type = metricType; + } + + @XmlElement(name = "id") + public String getId() { + return id; + } + + public void setId(String metricId) { + this.id = metricId; + } + + /** + * Get the real time aggregation operation of this metric. + * + * @return Real time aggregation operation + */ + // required by JAXB + @XmlElement(name = "aggregationOp") + public TimelineMetricOperation getRealtimeAggregationOp() { + return realtimeAggregationOp; + } + + /** + * Set the real time aggregation operation of this metric. + * + * @param op A timeline metric operation that the metric should perform on + * real time aggregations + */ + public void setRealtimeAggregationOp( + final TimelineMetricOperation op) { + this.realtimeAggregationOp = op; + } + + // required by JAXB + @InterfaceAudience.Private + @XmlElement(name = "values") + public TreeMap getValuesJAXB() { + return values; + } + + public Map getValues() { + return values; + } + + public void setValues(Map vals) { + if (type == Type.SINGLE_VALUE) { + overwrite(vals); + } else { + if (vals != null) { + this.values = new TreeMap<>(Collections.reverseOrder()); + this.values.putAll(vals); + } else { + this.values = null; + } + } + } + + public void addValues(Map vals) { + if (type == Type.SINGLE_VALUE) { + overwrite(vals); + } else { + this.values.putAll(vals); + } + } + + public void addValue(long timestamp, Number value) { + if (type == Type.SINGLE_VALUE) { + values.clear(); + } + values.put(timestamp, value); + } + + private void overwrite(Map vals) { + if (vals.size() > 1) { + throw new IllegalArgumentException( + "Values cannot contain more than one point in " + + Type.SINGLE_VALUE + " mode"); + } + this.values.clear(); + this.values.putAll(vals); + } + + public boolean isValid() { + return (id != null); + } + + @Override + public int hashCode() { + int result = id.hashCode(); + result = 31 * result + type.hashCode(); + return result; + } + + // Only check if type and id are equal + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof TimelineMetric)) { + return false; + } + + TimelineMetric m = (TimelineMetric) o; + + if (!id.equals(m.id)) { + return false; + } + if (type != m.type) { + return false; + } + return true; + } + + @Override + public String toString() { + return "{id: " + id + ", type: " + type + + ", realtimeAggregationOp: " + + realtimeAggregationOp + "; " + values.toString() + + "}"; + } + + /** + * Get the latest timeline metric as single value type. + * + * @param metric Incoming timeline metric + * @return The latest metric in the incoming metric + */ + public static TimelineMetric getLatestSingleValueMetric( + TimelineMetric metric) { + if (metric.getType() == Type.SINGLE_VALUE) { + return metric; + } else { + TimelineMetric singleValueMetric = new TimelineMetric(Type.SINGLE_VALUE); + Long firstKey = metric.values.firstKey(); + if (firstKey != null) { + Number firstValue = metric.values.get(firstKey); + singleValueMetric.addValue(firstKey, firstValue); + } + return singleValueMetric; + } + } + + /** + * Get single data timestamp of the metric. + * + * @return the single data timestamp + */ + public long getSingleDataTimestamp() { + if (this.type == Type.SINGLE_VALUE) { + if (values.size() == 0) { + throw new YarnRuntimeException("Values for this timeline metric is " + + "empty."); + } else { + return values.firstKey(); + } + } else { + throw new YarnRuntimeException("Type for this timeline metric is not " + + "SINGLE_VALUE."); + } + } + + /** + * Get single data value of the metric. + * + * @return the single data value + */ + public Number getSingleDataValue() { + if (this.type == Type.SINGLE_VALUE) { + if (values.size() == 0) { + return null; + } else { + return values.get(values.firstKey()); + } + } else { + throw new YarnRuntimeException("Type for this timeline metric is not " + + "SINGLE_VALUE."); + } + } + + /** + * Aggregate an incoming metric to the base aggregated metric with the given + * operation state in a stateless fashion. The assumption here is + * baseAggregatedMetric and latestMetric should be single value data if not + * null. + * + * @param incomingMetric Incoming timeline metric to aggregate + * @param baseAggregatedMetric Base timeline metric + * @return Result metric after aggregation + */ + public static TimelineMetric aggregateTo(TimelineMetric incomingMetric, + TimelineMetric baseAggregatedMetric) { + return aggregateTo(incomingMetric, baseAggregatedMetric, null); + } + + /** + * Aggregate an incoming metric to the base aggregated metric with the given + * operation state. The assumption here is baseAggregatedMetric and + * latestMetric should be single value data if not null. + * + * @param incomingMetric Incoming timeline metric to aggregate + * @param baseAggregatedMetric Base timeline metric + * @param state Operation state + * @return Result metric after aggregation + */ + public static TimelineMetric aggregateTo(TimelineMetric incomingMetric, + TimelineMetric baseAggregatedMetric, Map state) { + TimelineMetricOperation operation + = incomingMetric.getRealtimeAggregationOp(); + return operation.aggregate(incomingMetric, baseAggregatedMetric, state); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetricCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetricCalculator.java new file mode 100644 index 00000000000..4c9045ff044 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetricCalculator.java @@ -0,0 +1,115 @@ +/* + * 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.records.timelineservice; + +import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; + +/** + * A calculator for timeline metrics. + */ +public final class TimelineMetricCalculator { + + private TimelineMetricCalculator() { + // do nothing. + } + + /** + * Compare two not-null numbers. + * @param n1 Number n1 + * @param n2 Number n2 + * @return 0 if n1 equals n2, a negative int if n1 is less than n2, a + * positive int otherwise. + */ + public static int compare(Number n1, Number n2) { + if (n1 == null || n2 == null) { + throw new YarnRuntimeException( + "Number to be compared shouldn't be null."); + } + + if (n1 instanceof Integer || n1 instanceof Long) { + if (n1.longValue() == n2.longValue()) { + return 0; + } else { + return (n1.longValue() < n2.longValue()) ? -1 : 1; + } + } + + if (n1 instanceof Float || n1 instanceof Double) { + if (n1.doubleValue() == n2.doubleValue()) { + return 0; + } else { + return (n1.doubleValue() < n2.doubleValue()) ? -1 : 1; + } + } + + // TODO throw warnings/exceptions for other types of number. + throw new YarnRuntimeException("Unsupported types for number comparison: " + + n1.getClass().getName() + ", " + n2.getClass().getName()); + } + + /** + * Subtract operation between two Numbers. + * @param n1 Number n1 + * @param n2 Number n2 + * @return Number represent to (n1 - n2). + */ + public static Number sub(Number n1, Number n2) { + if (n1 == null) { + throw new YarnRuntimeException( + "Number to be subtracted shouldn't be null."); + } else if (n2 == null) { + return n1; + } + + if (n1 instanceof Integer || n1 instanceof Long) { + return n1.longValue() - n2.longValue(); + } + + if (n1 instanceof Float || n1 instanceof Double) { + return n1.doubleValue() - n2.doubleValue(); + } + + // TODO throw warnings/exceptions for other types of number. + return null; + } + + /** + * Sum up two Numbers. + * @param n1 Number n1 + * @param n2 Number n2 + * @return Number represent to (n1 + n2). + */ + public static Number sum(Number n1, Number n2) { + if (n1 == null) { + return n2; + } else if (n2 == null) { + return n1; + } + + if (n1 instanceof Integer || n1 instanceof Long) { + return n1.longValue() + n2.longValue(); + } + + if (n1 instanceof Float || n1 instanceof Double) { + return n1.doubleValue() + n2.doubleValue(); + } + + // TODO throw warnings/exceptions for other types of number. + return null; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetricOperation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetricOperation.java new file mode 100644 index 00000000000..58e5c3811cf --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetricOperation.java @@ -0,0 +1,167 @@ +/* + * 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.records.timelineservice; + +import java.util.Map; + +/** + * Aggregation operations. + */ +public enum TimelineMetricOperation { + NOP("NOP") { + /** + * Do nothing on the base metric. + * + * @param incoming Metric a + * @param base Metric b + * @param state Operation state (not used) + * @return Metric b + */ + @Override + public TimelineMetric exec(TimelineMetric incoming, + TimelineMetric base, Map state) { + return base; + } + }, + MAX("MAX") { + /** + * Keep the greater value of incoming and base. Stateless operation. + * + * @param incoming Metric a + * @param base Metric b + * @param state Operation state (not used) + * @return the greater value of a and b + */ + @Override + public TimelineMetric exec(TimelineMetric incoming, + TimelineMetric base, Map state) { + if (base == null) { + return incoming; + } + Number incomingValue = incoming.getSingleDataValue(); + Number aggregateValue = base.getSingleDataValue(); + if (aggregateValue == null) { + aggregateValue = Long.MIN_VALUE; + } + if (TimelineMetricCalculator.compare(incomingValue, aggregateValue) > 0) { + base.addValue(incoming.getSingleDataTimestamp(), incomingValue); + } + return base; + } + }, + REPLACE("REPLACE") { + /** + * Replace the base metric with the incoming value. Stateless operation. + * + * @param incoming Metric a + * @param base Metric b + * @param state Operation state (not used) + * @return Metric a + */ + @Override + public TimelineMetric exec(TimelineMetric incoming, + TimelineMetric base, + Map state) { + return incoming; + } + }, + SUM("SUM") { + /** + * Return the sum of the incoming metric and the base metric if the + * operation is stateless. For stateful operations, also subtract the + * value of the timeline metric mapped to the PREV_METRIC_STATE_KEY + * in the state object. + * + * @param incoming Metric a + * @param base Metric b + * @param state Operation state (PREV_METRIC_STATE_KEY's value as Metric p) + * @return A metric with value a + b - p + */ + @Override + public TimelineMetric exec(TimelineMetric incoming, TimelineMetric base, + Map state) { + if (base == null) { + return incoming; + } + Number incomingValue = incoming.getSingleDataValue(); + Number aggregateValue = base.getSingleDataValue(); + Number result + = TimelineMetricCalculator.sum(incomingValue, aggregateValue); + + // If there are previous value in the state, we will take it off from the + // sum + if (state != null) { + Object prevMetric = state.get(PREV_METRIC_STATE_KEY); + if (prevMetric instanceof TimelineMetric) { + result = TimelineMetricCalculator.sub(result, + ((TimelineMetric) prevMetric).getSingleDataValue()); + } + } + base.addValue(incoming.getSingleDataTimestamp(), result); + return base; + } + }, + AVG("AVERAGE") { + /** + * Return the average value of the incoming metric and the base metric, + * with a given state. Not supported yet. + * + * @param incoming Metric a + * @param base Metric b + * @param state Operation state + * @return Not finished yet + */ + @Override + public TimelineMetric exec(TimelineMetric incoming, TimelineMetric base, + Map state) { + // Not supported yet + throw new UnsupportedOperationException( + "Unsupported aggregation operation: AVERAGE"); + } + }; + + public static final String PREV_METRIC_STATE_KEY = "PREV_METRIC"; + + /** + * Perform the aggregation operation. + * + * @param incoming Incoming metric + * @param aggregate Base aggregation metric + * @param state Operation state + * @return Result metric for this aggregation operation + */ + public TimelineMetric aggregate(TimelineMetric incoming, + TimelineMetric aggregate, Map state) { + return exec(incoming, aggregate, state); + } + + private final String opName; + + TimelineMetricOperation(String opString) { + opName = opString; + } + + @Override + public String toString() { + return this.opName; + } + + abstract TimelineMetric exec(TimelineMetric incoming, TimelineMetric base, + Map state); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineWriteResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineWriteResponse.java new file mode 100644 index 00000000000..eda1ee2ef7e --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineWriteResponse.java @@ -0,0 +1,167 @@ +/** + * 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.records.timelineservice; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; + +import javax.xml.bind.annotation.XmlAccessType; +import javax.xml.bind.annotation.XmlAccessorType; +import javax.xml.bind.annotation.XmlElement; +import javax.xml.bind.annotation.XmlRootElement; +import java.util.ArrayList; +import java.util.List; + +/** + * A class that holds a list of put errors. This is the response returned when a + * list of {@link TimelineEntity} objects is added to the timeline. If there are + * errors in storing individual entity objects, they will be indicated in the + * list of errors. + */ +@XmlRootElement(name = "response") +@XmlAccessorType(XmlAccessType.NONE) +@Public +@Unstable +public class TimelineWriteResponse { + + private List errors = new ArrayList(); + + public TimelineWriteResponse() { + + } + + /** + * Get a list of {@link TimelineWriteError} instances. + * + * @return a list of {@link TimelineWriteError} instances + */ + @XmlElement(name = "errors") + public List getErrors() { + return errors; + } + + /** + * Add a single {@link TimelineWriteError} instance into the existing list. + * + * @param error + * a single {@link TimelineWriteError} instance + */ + public void addError(TimelineWriteError error) { + errors.add(error); + } + + /** + * Add a list of {@link TimelineWriteError} instances into the existing list. + * + * @param writeErrors + * a list of {@link TimelineWriteError} instances + */ + public void addErrors(List writeErrors) { + this.errors.addAll(writeErrors); + } + + /** + * Set the list to the given list of {@link TimelineWriteError} instances. + * + * @param writeErrors + * a list of {@link TimelineWriteError} instances + */ + public void setErrors(List writeErrors) { + this.errors.clear(); + this.errors.addAll(writeErrors); + } + + /** + * A class that holds the error code for one entity. + */ + @XmlRootElement(name = "error") + @XmlAccessorType(XmlAccessType.NONE) + @Public + @Unstable + public static class TimelineWriteError { + + /** + * Error code returned if an IOException is encountered when storing an + * entity. + */ + public static final int IO_EXCEPTION = 1; + + private String entityId; + private String entityType; + private int errorCode; + + /** + * Get the entity Id. + * + * @return the entity Id + */ + @XmlElement(name = "entity") + public String getEntityId() { + return entityId; + } + + /** + * Set the entity Id. + * + * @param id the entity Id. + */ + public void setEntityId(String id) { + this.entityId = id; + } + + /** + * Get the entity type. + * + * @return the entity type + */ + @XmlElement(name = "entitytype") + public String getEntityType() { + return entityType; + } + + /** + * Set the entity type. + * + * @param type the entity type. + */ + public void setEntityType(String type) { + this.entityType = type; + } + + /** + * Get the error code. + * + * @return an error code + */ + @XmlElement(name = "errorcode") + public int getErrorCode() { + return errorCode; + } + + /** + * Set the error code to the given error code. + * + * @param code an error code. + */ + public void setErrorCode(int code) { + this.errorCode = code; + } + + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/UserEntity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/UserEntity.java new file mode 100644 index 00000000000..ced57c67cd6 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/UserEntity.java @@ -0,0 +1,40 @@ +/* + * 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.records.timelineservice; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * This entity represents a user. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public class UserEntity extends TimelineEntity { + public UserEntity() { + super(TimelineEntityType.YARN_USER.toString()); + } + + public UserEntity(TimelineEntity entity) { + super(entity); + if (!entity.getType().equals(TimelineEntityType.YARN_USER.toString())) { + throw new IllegalArgumentException("Incompatible entity type: " + + getId()); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/package-info.java new file mode 100644 index 00000000000..43805c8c98a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/package-info.java @@ -0,0 +1,26 @@ +/* + * 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.records.timelineservice contains classes + * which define the data model for ATSv2. + */ +@InterfaceAudience.Public +package org.apache.hadoop.yarn.api.records.timelineservice; + +import org.apache.hadoop.classification.InterfaceAudience; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java index 817ada1c57e..fb9fb534fff 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java @@ -96,6 +96,10 @@ public class YarnConfiguration extends Configuration { new DeprecationDelta(RM_ZK_RETRY_INTERVAL_MS, CommonConfigurationKeys.ZK_RETRY_INTERVAL_MS), }); + Configuration.addDeprecations(new DeprecationDelta[] { + new DeprecationDelta(RM_SYSTEM_METRICS_PUBLISHER_ENABLED, + SYSTEM_METRICS_PUBLISHER_ENABLED) + }); } //Configurations @@ -146,6 +150,7 @@ public class YarnConfiguration extends Configuration { public static final String RM_PREFIX = "yarn.resourcemanager."; public static final String RM_CLUSTER_ID = RM_PREFIX + "cluster-id"; + public static final String DEFAULT_RM_CLUSTER_ID = "yarn_cluster"; public static final String RM_HOSTNAME = RM_PREFIX + "hostname"; @@ -500,16 +505,37 @@ public class YarnConfiguration extends Configuration { /** * The setting that controls whether yarn system metrics is published on the - * timeline server or not by RM. + * timeline server or not by RM. This configuration setting is for ATS V1. + * This is now deprecated in favor of SYSTEM_METRICS_PUBLISHER_ENABLED. */ - public static final String RM_SYSTEM_METRICS_PUBLISHER_ENABLED = - RM_PREFIX + "system-metrics-publisher.enabled"; - public static final boolean DEFAULT_RM_SYSTEM_METRICS_PUBLISHER_ENABLED = false; + public static final String RM_SYSTEM_METRICS_PUBLISHER_ENABLED = RM_PREFIX + + "system-metrics-publisher.enabled"; + public static final boolean DEFAULT_RM_SYSTEM_METRICS_PUBLISHER_ENABLED = + false; + + /** + * The setting that controls whether yarn system metrics is published on the + * timeline server or not by RM and NM. This configuration setting is for + * ATS v2. + */ + public static final String SYSTEM_METRICS_PUBLISHER_ENABLED = YARN_PREFIX + + "system-metrics-publisher.enabled"; + public static final boolean DEFAULT_SYSTEM_METRICS_PUBLISHER_ENABLED = false; + + /** + * The setting that controls whether yarn container events are published to + * the timeline service or not by RM. This configuration setting is for ATS + * V2 + */ + public static final String RM_PUBLISH_CONTAINER_EVENTS_ENABLED = YARN_PREFIX + + "rm.system-metrics-publisher.emit-container-events"; + public static final boolean DEFAULT_RM_PUBLISH_CONTAINER_EVENTS_ENABLED = + false; public static final String RM_SYSTEM_METRICS_PUBLISHER_DISPATCHER_POOL_SIZE = RM_PREFIX + "system-metrics-publisher.dispatcher.pool-size"; - public static final int DEFAULT_RM_SYSTEM_METRICS_PUBLISHER_DISPATCHER_POOL_SIZE = - 10; + public static final int + DEFAULT_RM_SYSTEM_METRICS_PUBLISHER_DISPATCHER_POOL_SIZE = 10; //RM delegation token related keys public static final String RM_DELEGATION_KEY_UPDATE_INTERVAL_KEY = @@ -956,6 +982,11 @@ public class YarnConfiguration extends Configuration { NM_PREFIX + "container-manager.thread-count"; public static final int DEFAULT_NM_CONTAINER_MGR_THREAD_COUNT = 20; + /** Number of threads container manager uses.*/ + public static final String NM_COLLECTOR_SERVICE_THREAD_COUNT = + NM_PREFIX + "collector-service.thread-count"; + public static final int DEFAULT_NM_COLLECTOR_SERVICE_THREAD_COUNT = 5; + /** Number of threads used in cleanup.*/ public static final String NM_DELETE_THREAD_COUNT = NM_PREFIX + "delete.thread-count"; @@ -983,6 +1014,13 @@ public class YarnConfiguration extends Configuration { public static final String DEFAULT_NM_LOCALIZER_ADDRESS = "0.0.0.0:" + DEFAULT_NM_LOCALIZER_PORT; + /** Address where the collector service IPC is.*/ + public static final String NM_COLLECTOR_SERVICE_ADDRESS = + NM_PREFIX + "collector-service.address"; + public static final int DEFAULT_NM_COLLECTOR_SERVICE_PORT = 8048; + public static final String DEFAULT_NM_COLLECTOR_SERVICE_ADDRESS = + "0.0.0.0:" + DEFAULT_NM_LOCALIZER_PORT; + /** Interval in between cache cleanups.*/ public static final String NM_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS = NM_PREFIX + "localizer.cache.cleanup.interval-ms"; @@ -1831,6 +1869,10 @@ public class YarnConfiguration extends Configuration { YARN_SECURITY_SERVICE_AUTHORIZATION_APPLICATIONHISTORY_PROTOCOL = "security.applicationhistory.protocol.acl"; + public static final String + YARN_SECURITY_SERVICE_AUTHORIZATION_COLLECTOR_NODEMANAGER_PROTOCOL = + "security.collector-nodemanager.protocol.acl"; + /** No. of milliseconds to wait between sending a SIGTERM and SIGKILL * to a running container */ public static final String NM_SLEEP_DELAY_BEFORE_SIGKILL_MS = @@ -2011,7 +2053,7 @@ public class YarnConfiguration extends Configuration { public static final String TIMELINE_SERVICE_UI_WEB_PATH_PREFIX = TIMELINE_SERVICE_PREFIX + "ui-web-path."; - /** Timeline client settings */ + /** Timeline client settings. */ public static final String TIMELINE_SERVICE_CLIENT_PREFIX = TIMELINE_SERVICE_PREFIX + "client."; @@ -2150,6 +2192,138 @@ public class YarnConfiguration extends Configuration { = TIMELINE_SERVICE_PREFIX + "entity-file.fs-support-append"; + /** + * Settings for timeline service v2.0. + */ + public static final String TIMELINE_SERVICE_WRITER_CLASS = + TIMELINE_SERVICE_PREFIX + "writer.class"; + + public static final String DEFAULT_TIMELINE_SERVICE_WRITER_CLASS = + "org.apache.hadoop.yarn.server.timelineservice" + + ".storage.HBaseTimelineWriterImpl"; + + public static final String TIMELINE_SERVICE_READER_CLASS = + TIMELINE_SERVICE_PREFIX + "reader.class"; + + public static final String DEFAULT_TIMELINE_SERVICE_READER_CLASS = + "org.apache.hadoop.yarn.server.timelineservice" + + ".storage.HBaseTimelineReaderImpl"; + + + /** + * default schema prefix for hbase tables. + */ + public static final String DEFAULT_TIMELINE_SERVICE_HBASE_SCHEMA_PREFIX = + "prod."; + + /** + * config param name to override schema prefix. + */ + public static final String TIMELINE_SERVICE_HBASE_SCHEMA_PREFIX_NAME = + TIMELINE_SERVICE_PREFIX + "hbase-schema.prefix"; + + /** The setting that controls how often the timeline collector flushes the + * timeline writer. + */ + public static final String TIMELINE_SERVICE_WRITER_FLUSH_INTERVAL_SECONDS = + TIMELINE_SERVICE_PREFIX + "writer.flush-interval-seconds"; + + public static final int + DEFAULT_TIMELINE_SERVICE_WRITER_FLUSH_INTERVAL_SECONDS = 60; + + /** + * The name for setting that controls how long the final value of + * a metric of a completed app is retained before merging + * into the flow sum. + */ + public static final String APP_FINAL_VALUE_RETENTION_THRESHOLD = + TIMELINE_SERVICE_PREFIX + + "hbase.coprocessor.app-final-value-retention-milliseconds"; + + /** + * The name of the setting for the location of the coprocessor + * jar on hdfs. + */ + public static final String FLOW_RUN_COPROCESSOR_JAR_HDFS_LOCATION = + TIMELINE_SERVICE_PREFIX + + "hbase.coprocessor.jar.hdfs.location"; + + /** default hdfs location for flowrun coprocessor jar. */ + public static final String DEFAULT_HDFS_LOCATION_FLOW_RUN_COPROCESSOR_JAR = + "/hbase/coprocessor/hadoop-yarn-server-timelineservice.jar"; + + /** + * The name for setting that points to an optional HBase configuration + * (hbase-site.xml file) with settings that will override the ones found on + * the classpath. + */ + public static final String TIMELINE_SERVICE_HBASE_CONFIGURATION_FILE = + TIMELINE_SERVICE_PREFIX + + "hbase.configuration.file"; + + /** + * The name for setting that enables or disables authentication checks + * for reading timeline service v2 data. + */ + public static final String TIMELINE_SERVICE_READ_AUTH_ENABLED = + TIMELINE_SERVICE_PREFIX + "read.authentication.enabled"; + + /** + * The default setting for authentication checks for reading timeline + * service v2 data. + */ + public static final Boolean DEFAULT_TIMELINE_SERVICE_READ_AUTH_ENABLED = + false; + + /** + * The name for setting that lists the users and groups who are allowed + * to read timeline service v2 data. It is a comma separated list of + * user, followed by space, then comma separated list of groups. + * It will allow this list of users and groups to read the data + * and reject everyone else. + */ + public static final String TIMELINE_SERVICE_READ_ALLOWED_USERS = + TIMELINE_SERVICE_PREFIX + "read.allowed.users"; + + /** + * The default value for list of the users who are allowed to read + * timeline service v2 data. + */ + public static final String DEFAULT_TIMELINE_SERVICE_READ_ALLOWED_USERS = + ""; + + /** + * The setting that controls how long the final value of a metric of a + * completed app is retained before merging into the flow sum. Up to this time + * after an application is completed out-of-order values that arrive can be + * recognized and discarded at the cost of increased storage. + */ + public static final long DEFAULT_APP_FINAL_VALUE_RETENTION_THRESHOLD = 3 * 24 + * 60 * 60 * 1000L; + + public static final String ATS_APP_COLLECTOR_LINGER_PERIOD_IN_MS = + TIMELINE_SERVICE_PREFIX + "app-collector.linger-period.ms"; + + public static final int DEFAULT_ATS_APP_COLLECTOR_LINGER_PERIOD_IN_MS = 1000; + + public static final String NUMBER_OF_ASYNC_ENTITIES_TO_MERGE = + TIMELINE_SERVICE_PREFIX + + "timeline-client.number-of-async-entities-to-merge"; + + public static final int DEFAULT_NUMBER_OF_ASYNC_ENTITIES_TO_MERGE = 10; + + /** default version for any flow. */ + public static final String DEFAULT_FLOW_VERSION = "1"; + + /** + * The time period for which timeline v2 client will wait for draining + * leftover entities after stop. + */ + public static final String TIMELINE_V2_CLIENT_DRAIN_TIME_MILLIS = + TIMELINE_SERVICE_CLIENT_PREFIX + "drain-entities.timeout.ms"; + public static final long DEFAULT_TIMELINE_V2_CLIENT_DRAIN_TIME_MILLIS + = 2000L; + // mark app-history related configs @Private as application history is going // to be integrated into the timeline service @Private @@ -2208,6 +2382,7 @@ public class YarnConfiguration extends Configuration { /** The listening endpoint for the timeline service application.*/ public static final String TIMELINE_SERVICE_BIND_HOST = TIMELINE_SERVICE_PREFIX + "bind-host"; + public static final String DEFAULT_TIMELINE_SERVICE_BIND_HOST = "0.0.0.0"; /** The number of threads to handle client RPC API requests. */ public static final String TIMELINE_SERVICE_HANDLER_THREAD_COUNT = @@ -2407,6 +2582,16 @@ public class YarnConfiguration extends Configuration { public static final long DEFAULT_TIMELINE_DELEGATION_TOKEN_MAX_LIFETIME = 7*24*60*60*1000; // 7 days + // Timeline service v2 offlien aggregation related keys + public static final String TIMELINE_OFFLINE_AGGREGATION_PREFIX = + YarnConfiguration.TIMELINE_SERVICE_PREFIX + "aggregation.offline."; + public static final String PHOENIX_OFFLINE_STORAGE_CONN_STR + = TIMELINE_OFFLINE_AGGREGATION_PREFIX + + "phoenix.connectionString"; + + public static final String PHOENIX_OFFLINE_STORAGE_CONN_STR_DEFAULT + = "jdbc:phoenix:localhost:2181:/hbase"; + // /////////////////////////////// // Shared Cache Configs // /////////////////////////////// @@ -3148,6 +3333,53 @@ public class YarnConfiguration extends Configuration { YarnConfiguration.DEFAULT_OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED); } + // helper methods for timeline service configuration + /** + * Returns whether the timeline service is enabled via configuration. + * + * @param conf the configuration + * @return whether the timeline service is enabled. + */ + public static boolean timelineServiceEnabled(Configuration conf) { + return conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED); + } + + /** + * Returns the timeline service version. It does not check whether the + * timeline service itself is enabled. + * + * @param conf the configuration + * @return the timeline service version as a float. + */ + public static float getTimelineServiceVersion(Configuration conf) { + return conf.getFloat(TIMELINE_SERVICE_VERSION, + DEFAULT_TIMELINE_SERVICE_VERSION); + } + + /** + * Returns whether the timeline service v.2 is enabled via configuration. + * + * @param conf the configuration + * @return whether the timeline service v.2 is enabled. V.2 refers to a + * version greater than equal to 2 but smaller than 3. + */ + public static boolean timelineServiceV2Enabled(Configuration conf) { + return timelineServiceEnabled(conf) && + (int)getTimelineServiceVersion(conf) == 2; + } + + /** + * Returns whether the system publisher is enabled. + * + * @param conf the configuration + * @return whether the system publisher is enabled. + */ + public static boolean systemMetricsPublisherEnabled(Configuration conf) { + return conf.getBoolean(YarnConfiguration.SYSTEM_METRICS_PUBLISHER_ENABLED, + YarnConfiguration.DEFAULT_SYSTEM_METRICS_PUBLISHER_ENABLED); + } + /* For debugging. mp configurations to system output as XML format. */ public static void main(String[] args) throws Exception { new YarnConfiguration(new Configuration()).writeXml(System.out); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/TimelineServiceHelper.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/TimelineServiceHelper.java new file mode 100644 index 00000000000..65ed18a7a1e --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/TimelineServiceHelper.java @@ -0,0 +1,57 @@ +/* + * 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.util; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate; + +/** + * Helper class for Timeline service. + */ +@LimitedPrivate({ "MapReduce", "YARN" }) +public final class TimelineServiceHelper { + + private TimelineServiceHelper() { + // Utility classes should not have a public or default constructor. + } + + /** + * Cast map to HashMap for generic type. + * @param originalMap the map need to be casted + * @param key type + * @param value type + * @return casted HashMap object + */ + public static HashMap mapCastToHashMap( + Map originalMap) { + return originalMap == null ? null : originalMap instanceof HashMap ? + (HashMap) originalMap : new HashMap(originalMap); + } + + /** + * Inverts the given key. + * @param key value to be inverted . + * @return inverted long + */ + public static long invertLong(long key) { + return Long.MAX_VALUE - key; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto index a3ad1e32c53..07b8335d0aa 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto @@ -667,3 +667,8 @@ message ContainerResourceDecreaseProto { optional ContainerIdProto container_id = 1; optional ResourceProto capability = 2; } + +message CollectorInfoProto { + optional string collector_addr = 1; + optional hadoop.common.TokenProto collector_token = 2; +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto index b87c6f0592a..8301971f6d2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto @@ -115,6 +115,7 @@ message AllocateResponseProto { repeated ContainerResourceDecreaseProto decreased_containers = 11; optional hadoop.common.TokenProto am_rm_token = 12; optional PriorityProto application_priority = 13; + optional CollectorInfoProto collector_info = 14; repeated UpdateContainerErrorProto update_errors = 15; repeated UpdatedContainerProto updated_containers = 16; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestApplicationEntity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestApplicationEntity.java new file mode 100644 index 00000000000..c3f277718e7 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestApplicationEntity.java @@ -0,0 +1,71 @@ +/** + * 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.records.timelineservice; + +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertEquals; +import org.junit.Test; + +/** + * Various tests for the ApplicationEntity class. + * + */ +public class TestApplicationEntity { + + @Test + public void testIsApplicationEntity() { + TimelineEntity te = new TimelineEntity(); + te.setType(TimelineEntityType.YARN_APPLICATION.toString()); + assertTrue(ApplicationEntity.isApplicationEntity(te)); + + te = null; + assertEquals(false, ApplicationEntity.isApplicationEntity(te)); + + te = new TimelineEntity(); + te.setType(TimelineEntityType.YARN_CLUSTER.toString()); + assertEquals(false, ApplicationEntity.isApplicationEntity(te)); + } + + @Test + public void testGetApplicationEvent() { + TimelineEntity te = null; + TimelineEvent tEvent = ApplicationEntity.getApplicationEvent(te, + "no event"); + assertEquals(null, tEvent); + + te = new TimelineEntity(); + te.setType(TimelineEntityType.YARN_APPLICATION.toString()); + TimelineEvent event = new TimelineEvent(); + event.setId("start_event"); + event.setTimestamp(System.currentTimeMillis()); + te.addEvent(event); + tEvent = ApplicationEntity.getApplicationEvent(te, "start_event"); + assertEquals(event, tEvent); + + te = new TimelineEntity(); + te.setType(TimelineEntityType.YARN_CLUSTER.toString()); + event = new TimelineEvent(); + event.setId("start_event_cluster"); + event.setTimestamp(System.currentTimeMillis()); + te.addEvent(event); + tEvent = ApplicationEntity.getApplicationEvent(te, "start_event_cluster"); + assertEquals(null, tEvent); + + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestTimelineMetric.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestTimelineMetric.java new file mode 100644 index 00000000000..a6a3fdaf2a0 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestTimelineMetric.java @@ -0,0 +1,92 @@ +/** + * 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.records.timelineservice; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric.Type; + +import org.junit.Test; + +/** + * Tests {@link TimelineMetric} operations such as aggregation. + */ +public class TestTimelineMetric { + + @Test + public void testTimelineMetricAggregation() { + long ts = System.currentTimeMillis(); + // single_value metric add against null metric + TimelineMetric m1 = getSingleValueMetric("MEGA_BYTES_MILLIS", + TimelineMetricOperation.SUM, ts, 10000L); + TimelineMetric aggregatedMetric = TimelineMetric.aggregateTo(m1, null); + assertEquals(10000L, aggregatedMetric.getSingleDataValue()); + + TimelineMetric m2 = getSingleValueMetric("MEGA_BYTES_MILLIS", + TimelineMetricOperation.SUM, ts, 20000L); + aggregatedMetric = TimelineMetric.aggregateTo(m2, aggregatedMetric); + assertEquals(30000L, aggregatedMetric.getSingleDataValue()); + + // stateful sum test + Map state = new HashMap<>(); + state.put(TimelineMetricOperation.PREV_METRIC_STATE_KEY, m2); + TimelineMetric m2New = getSingleValueMetric("MEGA_BYTES_MILLIS", + TimelineMetricOperation.SUM, ts, 10000L); + aggregatedMetric = TimelineMetric.aggregateTo(m2New, aggregatedMetric, + state); + assertEquals(20000L, aggregatedMetric.getSingleDataValue()); + + // single_value metric max against single_value metric + TimelineMetric m3 = getSingleValueMetric("TRANSFER_RATE", + TimelineMetricOperation.MAX, ts, 150L); + TimelineMetric aggregatedMax = TimelineMetric.aggregateTo(m3, null); + assertEquals(150L, aggregatedMax.getSingleDataValue()); + + TimelineMetric m4 = getSingleValueMetric("TRANSFER_RATE", + TimelineMetricOperation.MAX, ts, 170L); + aggregatedMax = TimelineMetric.aggregateTo(m4, aggregatedMax); + assertEquals(170L, aggregatedMax.getSingleDataValue()); + + // single_value metric avg against single_value metric + TimelineMetric m5 = getSingleValueMetric("TRANSFER_RATE", + TimelineMetricOperation.AVG, ts, 150L); + try { + TimelineMetric.aggregateTo(m5, null); + fail("Taking average among metrics is not supported! "); + } catch (UnsupportedOperationException e) { + // Expected + } + + } + + private static TimelineMetric getSingleValueMetric(String id, + TimelineMetricOperation op, long timestamp, long value) { + TimelineMetric m = new TimelineMetric(); + m.setId(id); + m.setType(Type.SINGLE_VALUE); + m.setRealtimeAggregationOp(op); + Map metricValues = new HashMap(); + metricValues.put(timestamp, value); + m.setValues(metricValues); + return m; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java index 6cb92f03829..1d3111ce8b0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java @@ -66,6 +66,8 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase { configurationPropsToSkipCompare .add(YarnConfiguration .YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCETRACKER_PROTOCOL); + configurationPropsToSkipCompare.add(YarnConfiguration + .YARN_SECURITY_SERVICE_AUTHORIZATION_COLLECTOR_NODEMANAGER_PROTOCOL); configurationPropsToSkipCompare.add(YarnConfiguration.CURATOR_LEADER_ELECTOR); configurationPropsToSkipCompare .add(YarnConfiguration.RM_RESERVATION_SYSTEM_MAX_PERIODICITY); @@ -125,6 +127,9 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase { // Ignore all YARN Application Timeline Service (version 1) properties configurationPrefixToSkipCompare.add("yarn.timeline-service."); + // skip deprecated RM_SYSTEM_METRICS_PUBLISHER_ENABLED + configurationPropsToSkipCompare + .add(YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_ENABLED); // skip deprecated ZooKeeper settings configurationPropsToSkipCompare.add(YarnConfiguration.RM_ZK_ADDRESS); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml index 2c1e36dc1d5..9dde71d98c2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml @@ -70,6 +70,16 @@ commons-io commons-io
+ + org.apache.hadoop + hadoop-yarn-server-applicationhistoryservice + + + org.apache.hadoop + hadoop-yarn-server-timelineservice + test-jar + test + org.apache.hadoop diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java index 5bde6a13cc0..3de87c96aca 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java @@ -97,13 +97,15 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent; import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse; import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest; import org.apache.hadoop.yarn.client.api.TimelineClient; +import org.apache.hadoop.yarn.client.api.TimelineV2Client; import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; import org.apache.hadoop.yarn.client.api.async.NMClientAsync; import org.apache.hadoop.yarn.client.api.async.impl.NMClientAsyncImpl; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; -import org.apache.hadoop.yarn.util.ConverterUtils; +import org.apache.hadoop.yarn.util.SystemClock; +import org.apache.hadoop.yarn.util.TimelineServiceHelper; import org.apache.hadoop.yarn.util.timeline.TimelineUtils; import org.apache.log4j.LogManager; @@ -220,6 +222,10 @@ public class ApplicationMaster { // Tracking url to which app master publishes info for clients to monitor private String appMasterTrackingUrl = ""; + private boolean timelineServiceV2Enabled = false; + + private boolean timelineServiceV1Enabled = false; + // App Master configuration // No. of containers to run shell command on @VisibleForTesting @@ -292,6 +298,10 @@ public class ApplicationMaster { // Timeline Client @VisibleForTesting TimelineClient timelineClient; + + // Timeline v2 Client + private TimelineV2Client timelineV2Client; + static final String CONTAINER_ENTITY_GROUP_ID = "CONTAINERS"; static final String APPID_TIMELINE_FILTER_NAME = "appId"; static final String USER_TIMELINE_FILTER_NAME = "user"; @@ -305,6 +315,17 @@ public class ApplicationMaster { protected final Set launchedContainers = Collections.newSetFromMap(new ConcurrentHashMap()); + /** + * Container start times used to set id prefix while publishing entity + * to ATSv2. + */ + private final ConcurrentMap containerStartTimes = + new ConcurrentHashMap(); + + private ConcurrentMap getContainerStartTimes() { + return containerStartTimes; + } + /** * @param args Command line args */ @@ -553,6 +574,17 @@ public class ApplicationMaster { cliParser.getOptionValue("container_max_retries", "0")); containrRetryInterval = Integer.parseInt(cliParser.getOptionValue( "container_retry_interval", "0")); + + if (YarnConfiguration.timelineServiceEnabled(conf)) { + timelineServiceV2Enabled = + ((int) YarnConfiguration.getTimelineServiceVersion(conf) == 2); + timelineServiceV1Enabled = !timelineServiceV2Enabled; + } else { + timelineClient = null; + timelineV2Client = null; + LOG.warn("Timeline service is not enabled"); + } + return true; } @@ -600,7 +632,6 @@ public class ApplicationMaster { UserGroupInformation.createRemoteUser(appSubmitterUserName); appSubmitterUgi.addCredentials(credentials); - AMRMClientAsync.AbstractCallbackHandler allocListener = new RMCallbackHandler(); amRMClient = AMRMClientAsync.createAMRMClientAsync(1000, allocListener); @@ -613,7 +644,15 @@ public class ApplicationMaster { nmClientAsync.start(); startTimelineClient(conf); - if(timelineClient != null) { + if (timelineServiceV2Enabled) { + // need to bind timelineClient + amRMClient.registerTimelineV2Client(timelineV2Client); + } + + if (timelineServiceV2Enabled) { + publishApplicationAttemptEventOnTimelineServiceV2( + DSEvent.DS_APP_ATTEMPT_START); + } else if (timelineServiceV1Enabled) { publishApplicationAttemptEvent(timelineClient, appAttemptID.toString(), DSEvent.DS_APP_ATTEMPT_START, domainId, appSubmitterUgi); } @@ -685,14 +724,23 @@ public class ApplicationMaster { appSubmitterUgi.doAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - if (conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, - YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED)) { + if (YarnConfiguration.timelineServiceEnabled(conf)) { // Creating the Timeline Client - timelineClient = TimelineClient.createTimelineClient(); - timelineClient.init(conf); - timelineClient.start(); + if (timelineServiceV2Enabled) { + timelineV2Client = TimelineV2Client.createTimelineClient( + appAttemptID.getApplicationId()); + timelineV2Client.init(conf); + timelineV2Client.start(); + LOG.info("Timeline service V2 client is enabled"); + } else { + timelineClient = TimelineClient.createTimelineClient(); + timelineClient.init(conf); + timelineClient.start(); + LOG.info("Timeline service V1 client is enabled"); + } } else { timelineClient = null; + timelineV2Client = null; LOG.warn("Timeline service is not enabled"); } return null; @@ -718,7 +766,10 @@ public class ApplicationMaster { } catch (InterruptedException ex) {} } - if(timelineClient != null) { + if (timelineServiceV2Enabled) { + publishApplicationAttemptEventOnTimelineServiceV2( + DSEvent.DS_APP_ATTEMPT_END); + } else if (timelineServiceV1Enabled) { publishApplicationAttemptEvent(timelineClient, appAttemptID.toString(), DSEvent.DS_APP_ATTEMPT_END, domainId, appSubmitterUgi); } @@ -769,8 +820,10 @@ public class ApplicationMaster { amRMClient.stop(); // Stop Timeline Client - if(timelineClient != null) { + if(timelineServiceV1Enabled) { timelineClient.stop(); + } else if (timelineServiceV2Enabled) { + timelineV2Client.stop(); } return success; @@ -825,9 +878,19 @@ public class ApplicationMaster { LOG.info("Container completed successfully." + ", containerId=" + containerStatus.getContainerId()); } - if(timelineClient != null) { - publishContainerEndEvent( - timelineClient, containerStatus, domainId, appSubmitterUgi); + if (timelineServiceV2Enabled) { + Long containerStartTime = + containerStartTimes.get(containerStatus.getContainerId()); + if (containerStartTime == null) { + containerStartTime = SystemClock.getInstance().getTime(); + containerStartTimes.put(containerStatus.getContainerId(), + containerStartTime); + } + publishContainerEndEventOnTimelineServiceV2(containerStatus, + containerStartTime); + } else if (timelineServiceV1Enabled) { + publishContainerEndEvent(timelineClient, containerStatus, domainId, + appSubmitterUgi); } } @@ -948,9 +1011,15 @@ public class ApplicationMaster { } Container container = containers.get(containerId); if (container != null) { - applicationMaster.nmClientAsync.getContainerStatusAsync(containerId, container.getNodeId()); + applicationMaster.nmClientAsync.getContainerStatusAsync( + containerId, container.getNodeId()); } - if(applicationMaster.timelineClient != null) { + if (applicationMaster.timelineServiceV2Enabled) { + long startTime = SystemClock.getInstance().getTime(); + applicationMaster.getContainerStartTimes().put(containerId, startTime); + applicationMaster.publishContainerStartEventOnTimelineServiceV2( + container, startTime); + } else if (applicationMaster.timelineServiceV1Enabled) { applicationMaster.publishContainerStartEvent( applicationMaster.timelineClient, container, applicationMaster.domainId, applicationMaster.appSubmitterUgi); @@ -1272,7 +1341,7 @@ public class ApplicationMaster { LOG.error("App Attempt " + (appEvent.equals(DSEvent.DS_APP_ATTEMPT_START) ? "start" : "end") + " event could not be published for " - + appAttemptId.toString(), e); + + appAttemptID, e); } } @@ -1319,4 +1388,112 @@ public class ApplicationMaster { shellId); return new Thread(runnableLaunchContainer); } + + private void publishContainerStartEventOnTimelineServiceV2( + Container container, long startTime) { + final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity + entity = + new org.apache.hadoop.yarn.api.records.timelineservice. + TimelineEntity(); + entity.setId(container.getId().toString()); + entity.setType(DSEntity.DS_CONTAINER.toString()); + entity.setCreatedTime(startTime); + entity.addInfo("user", appSubmitterUgi.getShortUserName()); + + org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent event = + new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent(); + event.setTimestamp(startTime); + event.setId(DSEvent.DS_CONTAINER_START.toString()); + event.addInfo("Node", container.getNodeId().toString()); + event.addInfo("Resources", container.getResource().toString()); + entity.addEvent(event); + entity.setIdPrefix(TimelineServiceHelper.invertLong(startTime)); + + try { + appSubmitterUgi.doAs(new PrivilegedExceptionAction() { + @Override + public TimelinePutResponse run() throws Exception { + timelineV2Client.putEntities(entity); + return null; + } + }); + } catch (Exception e) { + LOG.error("Container start event could not be published for " + + container.getId().toString(), + e instanceof UndeclaredThrowableException ? e.getCause() : e); + } + } + + private void publishContainerEndEventOnTimelineServiceV2( + final ContainerStatus container, long containerStartTime) { + final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity + entity = + new org.apache.hadoop.yarn.api.records.timelineservice. + TimelineEntity(); + entity.setId(container.getContainerId().toString()); + entity.setType(DSEntity.DS_CONTAINER.toString()); + //entity.setDomainId(domainId); + entity.addInfo("user", appSubmitterUgi.getShortUserName()); + org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent event = + new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent(); + event.setTimestamp(System.currentTimeMillis()); + event.setId(DSEvent.DS_CONTAINER_END.toString()); + event.addInfo("State", container.getState().name()); + event.addInfo("Exit Status", container.getExitStatus()); + entity.addEvent(event); + entity.setIdPrefix(TimelineServiceHelper.invertLong(containerStartTime)); + + try { + appSubmitterUgi.doAs(new PrivilegedExceptionAction() { + @Override + public TimelinePutResponse run() throws Exception { + timelineV2Client.putEntities(entity); + return null; + } + }); + } catch (Exception e) { + LOG.error("Container end event could not be published for " + + container.getContainerId().toString(), + e instanceof UndeclaredThrowableException ? e.getCause() : e); + } + } + + private void publishApplicationAttemptEventOnTimelineServiceV2( + DSEvent appEvent) { + final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity + entity = + new org.apache.hadoop.yarn.api.records.timelineservice. + TimelineEntity(); + entity.setId(appAttemptID.toString()); + entity.setType(DSEntity.DS_APP_ATTEMPT.toString()); + long ts = System.currentTimeMillis(); + if (appEvent == DSEvent.DS_APP_ATTEMPT_START) { + entity.setCreatedTime(ts); + } + entity.addInfo("user", appSubmitterUgi.getShortUserName()); + org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent event = + new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent(); + event.setId(appEvent.toString()); + event.setTimestamp(ts); + entity.addEvent(event); + entity.setIdPrefix( + TimelineServiceHelper.invertLong(appAttemptID.getAttemptId())); + + try { + appSubmitterUgi.doAs(new PrivilegedExceptionAction() { + @Override + public TimelinePutResponse run() throws Exception { + timelineV2Client.putEntitiesAsync(entity); + return null; + } + }); + } catch (Exception e) { + LOG.error("App Attempt " + + (appEvent.equals(DSEvent.DS_APP_ATTEMPT_START) ? "start" : "end") + + " event could not be published for " + + appAttemptID, + e instanceof UndeclaredThrowableException ? e.getCause() : e); + } + } + } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java index c8c9303a0e6..eedb5016e4f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java @@ -22,8 +22,10 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.Vector; import org.apache.commons.cli.CommandLine; @@ -187,6 +189,10 @@ public class Client { // Timeline domain writer access control private String modifyACLs = null; + private String flowName = null; + private String flowVersion = null; + private long flowRunId = 0L; + // Command line options private Options opts; @@ -258,7 +264,8 @@ public class Client { opts.addOption("shell_args", true, "Command line args for the shell script." + "Multiple args can be separated by empty space."); opts.getOption("shell_args").setArgs(Option.UNLIMITED_VALUES); - opts.addOption("shell_env", true, "Environment for shell script. Specified as env_key=env_val pairs"); + opts.addOption("shell_env", true, + "Environment for shell script. Specified as env_key=env_val pairs"); opts.addOption("shell_cmd_priority", true, "Priority for the shell command containers"); opts.addOption("container_memory", true, "Amount of memory in MB to be requested to run the shell command"); opts.addOption("container_vcores", true, "Amount of virtual cores to be requested to run the shell command"); @@ -284,6 +291,12 @@ public class Client { + "modify the timeline entities in the given domain"); opts.addOption("create", false, "Flag to indicate whether to create the " + "domain specified with -domain."); + opts.addOption("flow_name", true, "Flow name which the distributed shell " + + "app belongs to"); + opts.addOption("flow_version", true, "Flow version which the distributed " + + "shell app belongs to"); + opts.addOption("flow_run_id", true, "Flow run ID which the distributed " + + "shell app belongs to"); opts.addOption("help", false, "Print usage"); opts.addOption("node_label_expression", true, "Node label expression to determine the nodes" @@ -463,6 +476,20 @@ public class Client { + cliParser.getOptionValue("container_retry_interval")); } + if (cliParser.hasOption("flow_name")) { + flowName = cliParser.getOptionValue("flow_name"); + } + if (cliParser.hasOption("flow_version")) { + flowVersion = cliParser.getOptionValue("flow_version"); + } + if (cliParser.hasOption("flow_run_id")) { + try { + flowRunId = Long.parseLong(cliParser.getOptionValue("flow_run_id")); + } catch (NumberFormatException e) { + throw new IllegalArgumentException( + "Flow run is not a valid long value", e); + } + } return true; } @@ -554,6 +581,18 @@ public class Client { .setAttemptFailuresValidityInterval(attemptFailuresValidityInterval); } + Set tags = new HashSet(); + if (flowName != null) { + tags.add(TimelineUtils.generateFlowNameTag(flowName)); + } + if (flowVersion != null) { + tags.add(TimelineUtils.generateFlowVersionTag(flowVersion)); + } + if (flowRunId != 0) { + tags.add(TimelineUtils.generateFlowRunIdTag(flowRunId)); + } + appContext.setApplicationTags(tags); + // set local resources for the application master // local files or archives as needed // In this scenario, the jar file for the application master is part of the local resources @@ -667,7 +706,7 @@ public class Client { for (Map.Entry entry : shellEnv.entrySet()) { vargs.add("--shell_env " + entry.getKey() + "=" + entry.getValue()); - } + } if (debugFlag) { vargs.add("--debug"); } @@ -683,7 +722,7 @@ public class Client { command.append(str).append(" "); } - LOG.info("Completed setting up app master command " + command.toString()); + LOG.info("Completed setting up app master command " + command.toString()); List commands = new ArrayList(); commands.add(command.toString()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java index 4ab963785ab..47485aefc77 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java @@ -38,6 +38,7 @@ import java.util.Arrays; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.commons.io.FileUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -55,38 +56,47 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.JarFinder; import org.apache.hadoop.util.Shell; 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.ContainerState; import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain; import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities; -import org.apache.hadoop.yarn.applications.distributedshell.ApplicationMaster; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; +import org.apache.hadoop.yarn.applications.distributedshell.ApplicationMaster.DSEvent; +import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.client.api.impl.DirectTimelineWriter; +import org.apache.hadoop.yarn.client.api.impl.TestTimelineClient; import org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl; import org.apache.hadoop.yarn.client.api.impl.TimelineWriter; -import org.apache.hadoop.yarn.client.api.impl.TestTimelineClient; -import org.apache.hadoop.yarn.client.api.TimelineClient; -import org.apache.hadoop.yarn.client.api.YarnClient; - import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.MiniYARNCluster; +import org.apache.hadoop.yarn.server.metrics.AppAttemptMetricsConstants; +import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants; +import org.apache.hadoop.yarn.server.metrics.ContainerMetricsConstants; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; -import org.apache.hadoop.yarn.server.timeline.PluginStoreTestUtils; import org.apache.hadoop.yarn.server.timeline.NameValuePair; +import org.apache.hadoop.yarn.server.timeline.PluginStoreTestUtils; import org.apache.hadoop.yarn.server.timeline.TimelineVersion; import org.apache.hadoop.yarn.server.timeline.TimelineVersionWatcher; +import org.apache.hadoop.yarn.server.timelineservice.collector.PerNodeTimelineCollectorsAuxService; +import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineReaderImpl; +import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl; import org.apache.hadoop.yarn.server.utils.BuilderUtils; -import org.apache.hadoop.yarn.util.ConverterUtils; +import org.apache.hadoop.yarn.util.LinuxResourceCalculatorPlugin; +import org.apache.hadoop.yarn.util.ProcfsBasedProcessTree; +import org.apache.hadoop.yarn.util.timeline.TimelineUtils; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.Test; +import org.junit.rules.TemporaryFolder; import org.junit.rules.Timeout; -import com.sun.jersey.api.client.ClientHandlerException; - public class TestDistributedShell { private static final Log LOG = @@ -97,8 +107,11 @@ public class TestDistributedShell { private FileSystem fs = null; private TimelineWriter spyTimelineWriter; protected YarnConfiguration conf = null; + // location of the filesystem timeline writer for timeline service v.2 + private String timelineV2StorageDir = null; private static final int NUM_NMS = 1; private static final float DEFAULT_TIMELINE_VERSION = 1.0f; + private static final String TIMELINE_AUX_SERVICE_NAME = "timeline_collector"; protected final static String APPMASTER_JAR = JarFinder.getJar(ApplicationMaster.class); @@ -108,6 +121,8 @@ public class TestDistributedShell { = new TimelineVersionWatcher(); @Rule public Timeout globalTimeout = new Timeout(90000); + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); @Before public void setup() throws Exception { @@ -120,17 +135,36 @@ public class TestDistributedShell { private void setupInternal(int numNodeManager, float timelineVersion) throws Exception { - LOG.info("Starting up YARN cluster"); - + conf = new YarnConfiguration(); conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 128); + // reduce the teardown waiting time + conf.setLong(YarnConfiguration.DISPATCHER_DRAIN_EVENTS_TIMEOUT, 1000); conf.set("yarn.log.dir", "target"); conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); + // mark if we need to launch the v1 timeline server + // disable aux-service based timeline aggregators + conf.set(YarnConfiguration.NM_AUX_SERVICES, ""); + conf.setBoolean(YarnConfiguration.SYSTEM_METRICS_PUBLISHER_ENABLED, true); + + conf.set(YarnConfiguration.NM_VMEM_PMEM_RATIO, "8"); conf.set(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class.getName()); conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true); conf.set("mapreduce.jobhistory.address", "0.0.0.0:" + ServerSocketUtil.getPort(10021, 10)); + // Enable ContainersMonitorImpl + conf.set(YarnConfiguration.NM_CONTAINER_MON_RESOURCE_CALCULATOR, + LinuxResourceCalculatorPlugin.class.getName()); + conf.set(YarnConfiguration.NM_CONTAINER_MON_PROCESS_TREE, + ProcfsBasedProcessTree.class.getName()); + conf.setBoolean(YarnConfiguration.NM_PMEM_CHECK_ENABLED, true); + conf.setBoolean(YarnConfiguration.NM_VMEM_CHECK_ENABLED, true); + conf.setBoolean( + YarnConfiguration.YARN_MINICLUSTER_CONTROL_RESOURCE_MONITORING, + true); + conf.setBoolean(YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_ENABLED, + true); // ATS version specific settings if (timelineVersion == 1.0f) { @@ -148,6 +182,23 @@ public class TestDistributedShell { PluginStoreTestUtils.prepareConfiguration(conf, hdfsCluster); conf.set(YarnConfiguration.TIMELINE_SERVICE_ENTITY_GROUP_PLUGIN_CLASSES, DistributedShellTimelinePlugin.class.getName()); + } else if (timelineVersion == 2.0f) { + // set version to 2 + conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f); + // disable v1 timeline server since we no longer have a server here + // enable aux-service based timeline aggregators + conf.set(YarnConfiguration.NM_AUX_SERVICES, TIMELINE_AUX_SERVICE_NAME); + conf.set(YarnConfiguration.NM_AUX_SERVICES + "." + + TIMELINE_AUX_SERVICE_NAME + ".class", + PerNodeTimelineCollectorsAuxService.class.getName()); + conf.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS, + FileSystemTimelineWriterImpl.class, + org.apache.hadoop.yarn.server.timelineservice.storage. + TimelineWriter.class); + timelineV2StorageDir = tmpFolder.newFolder().getAbsolutePath(); + // set the file system timeline writer storage directory + conf.set(FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_DIR_ROOT, + timelineV2StorageDir); } else { Assert.fail("Wrong timeline version number: " + timelineVersion); } @@ -218,7 +269,7 @@ public class TestDistributedShell { new Path(conf.get(YarnConfiguration.TIMELINE_SERVICE_LEVELDB_PATH)), true); } - + @Test public void testDSShellWithDomain() throws Exception { testDSShell(true); @@ -241,7 +292,30 @@ public class TestDistributedShell { testDSShell(true); } + @Test + @TimelineVersion(2.0f) + public void testDSShellWithoutDomainV2() throws Exception { + testDSShell(false); + } + public void testDSShell(boolean haveDomain) throws Exception { + testDSShell(haveDomain, true); + } + + @Test + @TimelineVersion(2.0f) + public void testDSShellWithoutDomainV2DefaultFlow() throws Exception { + testDSShell(false, true); + } + + @Test + @TimelineVersion(2.0f) + public void testDSShellWithoutDomainV2CustomizedFlow() throws Exception { + testDSShell(false, false); + } + + public void testDSShell(boolean haveDomain, boolean defaultFlow) + throws Exception { String[] args = { "--jar", APPMASTER_JAR, @@ -268,9 +342,23 @@ public class TestDistributedShell { "writer_user writer_group", "--create" }; - List argsList = new ArrayList(Arrays.asList(args)); - argsList.addAll(Arrays.asList(domainArgs)); - args = argsList.toArray(new String[argsList.size()]); + args = mergeArgs(args, domainArgs); + } + boolean isTestingTimelineV2 = false; + if (timelineVersionWatcher.getTimelineVersion() == 2.0f) { + isTestingTimelineV2 = true; + if (!defaultFlow) { + String[] flowArgs = { + "--flow_name", + "test_flow_name", + "--flow_version", + "test_flow_version", + "--flow_run_id", + "12345678" + }; + args = mergeArgs(args, flowArgs); + } + LOG.info("Setup: Using timeline v2!"); } LOG.info("Initializing DS Client"); @@ -297,13 +385,16 @@ public class TestDistributedShell { boolean verified = false; String errorMessage = ""; + ApplicationId appId = null; + ApplicationReport appReport = null; while(!verified) { List apps = yarnClient.getApplications(); if (apps.size() == 0 ) { Thread.sleep(10); continue; } - ApplicationReport appReport = apps.get(0); + appReport = apps.get(0); + appId = appReport.getApplicationId(); if(appReport.getHost().equals("N/A")) { Thread.sleep(10); continue; @@ -315,13 +406,16 @@ public class TestDistributedShell { if (checkHostname(appReport.getHost()) && appReport.getRpcPort() == -1) { verified = true; } - if (appReport.getYarnApplicationState() == YarnApplicationState.FINISHED) { + + if (appReport.getYarnApplicationState() == YarnApplicationState.FINISHED + && appReport.getFinalApplicationStatus() != + FinalApplicationStatus.UNDEFINED) { break; } } Assert.assertTrue(errorMessage, verified); t.join(); - LOG.info("Client run completed. Result=" + result); + LOG.info("Client run completed for testDSShell. Result=" + result); Assert.assertTrue(result.get()); if (timelineVersionWatcher.getTimelineVersion() == 1.5f) { @@ -342,6 +436,15 @@ public class TestDistributedShell { } } + TimelineDomain domain = null; + if (!isTestingTimelineV2) { + checkTimelineV1(haveDomain); + } else { + checkTimelineV2(haveDomain, appId, defaultFlow, appReport); + } + } + + private void checkTimelineV1(boolean haveDomain) throws Exception { TimelineDomain domain = null; if (haveDomain) { domain = yarnCluster.getApplicationHistoryServer() @@ -393,6 +496,201 @@ public class TestDistributedShell { } } + private void checkTimelineV2(boolean haveDomain, ApplicationId appId, + boolean defaultFlow, ApplicationReport appReport) throws Exception { + LOG.info("Started checkTimelineV2 "); + // For PoC check using the file-based timeline writer (YARN-3264) + String tmpRoot = timelineV2StorageDir + File.separator + "entities" + + File.separator; + + File tmpRootFolder = new File(tmpRoot); + try { + Assert.assertTrue(tmpRootFolder.isDirectory()); + String basePath = tmpRoot + + YarnConfiguration.DEFAULT_RM_CLUSTER_ID + File.separator + + UserGroupInformation.getCurrentUser().getShortUserName() + + (defaultFlow ? + File.separator + appReport.getName() + File.separator + + TimelineUtils.DEFAULT_FLOW_VERSION + File.separator + + appReport.getStartTime() + File.separator : + File.separator + "test_flow_name" + File.separator + + "test_flow_version" + File.separator + "12345678" + + File.separator) + + appId.toString(); + LOG.info("basePath: " + basePath); + // for this test, we expect DS_APP_ATTEMPT AND DS_CONTAINER dirs + + // Verify DS_APP_ATTEMPT entities posted by the client + // there will be at least one attempt, look for that file + String appTimestampFileName = + "appattempt_" + appId.getClusterTimestamp() + "_000" + appId.getId() + + "_000001" + + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION; + File dsAppAttemptEntityFile = verifyEntityTypeFileExists(basePath, + "DS_APP_ATTEMPT", appTimestampFileName); + // Check if required events are published and same idprefix is sent for + // on each publish. + verifyEntityForTimelineV2(dsAppAttemptEntityFile, + DSEvent.DS_APP_ATTEMPT_START.toString(), 1, 1, 0, true); + // to avoid race condition of testcase, atleast check 40 times with sleep + // of 50ms + verifyEntityForTimelineV2(dsAppAttemptEntityFile, + DSEvent.DS_APP_ATTEMPT_END.toString(), 1, 40, 50, true); + + // Verify DS_CONTAINER entities posted by the client. + String containerTimestampFileName = + "container_" + appId.getClusterTimestamp() + "_000" + appId.getId() + + "_01_000002.thist"; + File dsContainerEntityFile = verifyEntityTypeFileExists(basePath, + "DS_CONTAINER", containerTimestampFileName); + // Check if required events are published and same idprefix is sent for + // on each publish. + verifyEntityForTimelineV2(dsContainerEntityFile, + DSEvent.DS_CONTAINER_START.toString(), 1, 1, 0, true); + // to avoid race condition of testcase, atleast check 40 times with sleep + // of 50ms + verifyEntityForTimelineV2(dsContainerEntityFile, + DSEvent.DS_CONTAINER_END.toString(), 1, 40, 50, true); + + // Verify NM posting container metrics info. + String containerMetricsTimestampFileName = + "container_" + appId.getClusterTimestamp() + "_000" + appId.getId() + + "_01_000001" + + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION; + File containerEntityFile = verifyEntityTypeFileExists(basePath, + TimelineEntityType.YARN_CONTAINER.toString(), + containerMetricsTimestampFileName); + verifyEntityForTimelineV2(containerEntityFile, + ContainerMetricsConstants.CREATED_EVENT_TYPE, 1, 1, 0, true); + + // to avoid race condition of testcase, atleast check 40 times with sleep + // of 50ms + verifyEntityForTimelineV2(containerEntityFile, + ContainerMetricsConstants.FINISHED_EVENT_TYPE, 1, 40, 50, true); + + // Verify RM posting Application life cycle Events are getting published + String appMetricsTimestampFileName = + "application_" + appId.getClusterTimestamp() + "_000" + appId.getId() + + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION; + File appEntityFile = + verifyEntityTypeFileExists(basePath, + TimelineEntityType.YARN_APPLICATION.toString(), + appMetricsTimestampFileName); + // No need to check idprefix for app. + verifyEntityForTimelineV2(appEntityFile, + ApplicationMetricsConstants.CREATED_EVENT_TYPE, 1, 1, 0, false); + + // to avoid race condition of testcase, atleast check 40 times with sleep + // of 50ms + verifyEntityForTimelineV2(appEntityFile, + ApplicationMetricsConstants.FINISHED_EVENT_TYPE, 1, 40, 50, false); + + // Verify RM posting AppAttempt life cycle Events are getting published + String appAttemptMetricsTimestampFileName = + "appattempt_" + appId.getClusterTimestamp() + "_000" + appId.getId() + + "_000001" + + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION; + File appAttemptEntityFile = + verifyEntityTypeFileExists(basePath, + TimelineEntityType.YARN_APPLICATION_ATTEMPT.toString(), + appAttemptMetricsTimestampFileName); + verifyEntityForTimelineV2(appAttemptEntityFile, + AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE, 1, 1, 0, true); + verifyEntityForTimelineV2(appAttemptEntityFile, + AppAttemptMetricsConstants.FINISHED_EVENT_TYPE, 1, 1, 0, true); + } finally { + FileUtils.deleteDirectory(tmpRootFolder.getParentFile()); + } + } + + private File verifyEntityTypeFileExists(String basePath, String entityType, + String entityfileName) { + String outputDirPathForEntity = + basePath + File.separator + entityType + File.separator; + File outputDirForEntity = new File(outputDirPathForEntity); + Assert.assertTrue(outputDirForEntity.isDirectory()); + + String entityFilePath = outputDirPathForEntity + entityfileName; + + File entityFile = new File(entityFilePath); + Assert.assertTrue(entityFile.exists()); + return entityFile; + } + + /** + * Checks the events and idprefix published for an entity. + * + * @param entityFile Entity file. + * @param expectedEvent Expected event Id. + * @param numOfExpectedEvent Number of expected occurences of expected event + * id. + * @param checkTimes Number of times to check. + * @param sleepTime Sleep time for each iteration. + * @param checkIdPrefix Whether to check idprefix. + * @throws IOException if entity file reading fails. + * @throws InterruptedException if sleep is interrupted. + */ + private void verifyEntityForTimelineV2(File entityFile, String expectedEvent, + long numOfExpectedEvent, int checkTimes, long sleepTime, + boolean checkIdPrefix) throws IOException, InterruptedException { + long actualCount = 0; + for (int i = 0; i < checkTimes; i++) { + BufferedReader reader = null; + String strLine = null; + actualCount = 0; + try { + reader = new BufferedReader(new FileReader(entityFile)); + long idPrefix = -1; + while ((strLine = reader.readLine()) != null) { + String entityLine = strLine.trim(); + if (entityLine.isEmpty()) { + continue; + } + if (entityLine.contains(expectedEvent)) { + actualCount++; + } + if (checkIdPrefix) { + TimelineEntity entity = FileSystemTimelineReaderImpl. + getTimelineRecordFromJSON(entityLine, TimelineEntity.class); + Assert.assertTrue("Entity ID prefix expected to be > 0" , + entity.getIdPrefix() > 0); + if (idPrefix == -1) { + idPrefix = entity.getIdPrefix(); + } else { + Assert.assertEquals("Entity ID prefix should be same across " + + "each publish of same entity", + idPrefix, entity.getIdPrefix()); + } + } + } + } finally { + reader.close(); + } + if (numOfExpectedEvent == actualCount) { + break; + } + if (sleepTime > 0 && i < checkTimes - 1) { + Thread.sleep(sleepTime); + } + } + Assert.assertEquals("Unexpected number of " + expectedEvent + + " event published.", numOfExpectedEvent, actualCount); + } + + /** + * Utility function to merge two String arrays to form a new String array for + * our argumemts. + * + * @param args + * @param newArgs + * @return a String array consists of {args, newArgs} + */ + private String[] mergeArgs(String[] args, String[] newArgs) { + List argsList = new ArrayList(Arrays.asList(args)); + argsList.addAll(Arrays.asList(newArgs)); + return argsList.toArray(new String[argsList.size()]); + } + /* * NetUtils.getHostname() returns a string in the form "hostname/ip". * Sometimes the hostname we get is the FQDN and sometimes the short name. In @@ -1052,4 +1350,3 @@ public class TestDistributedShell { return numOfWords; } } - diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShellWithNodeLabels.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShellWithNodeLabels.java index b40548fb023..143e5c4e472 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShellWithNodeLabels.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShellWithNodeLabels.java @@ -41,7 +41,7 @@ public class TestDistributedShellWithNodeLabels { static final int NUM_NMS = 2; TestDistributedShell distShellTest; - + @Before public void setup() throws Exception { distShellTest = new TestDistributedShell(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java index 1c3ac1c7bff..b8eb58cdc0f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java @@ -54,6 +54,8 @@ public abstract class AMRMClient extends AbstractService { private static final Log LOG = LogFactory.getLog(AMRMClient.class); + private TimelineV2Client timelineV2Client; + /** * Create a new instance of AMRMClient. * For usage: @@ -680,11 +682,29 @@ public abstract class AMRMClient extends return nmTokenCache; } + /** + * Register TimelineV2Client to AMRMClient. Writer's address for the timeline + * V2 client will be updated dynamically if registered. + * + * @param client the timeline v2 client to register + */ + public void registerTimelineV2Client(TimelineV2Client client) { + timelineV2Client = client; + } + + /** + * Get registered timeline v2 client. + * @return the registered timeline v2 client + */ + public TimelineV2Client getRegisteredTimelineV2Client() { + return this.timelineV2Client; + } + /** * Wait for check to return true for each 1000 ms. * See also {@link #waitFor(com.google.common.base.Supplier, int)} * and {@link #waitFor(com.google.common.base.Supplier, int, int)} - * @param check + * @param check the condition for which it should wait */ public void waitFor(Supplier check) throws InterruptedException { waitFor(check, 1000); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java index 4963881b2b3..1ecfe1f588c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java @@ -18,8 +18,6 @@ package org.apache.hadoop.yarn.client.api.async; -import com.google.common.base.Preconditions; -import com.google.common.base.Supplier; import java.io.IOException; import java.util.Collection; import java.util.List; @@ -29,8 +27,8 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; 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.classification.InterfaceStability.Stable; +import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.service.AbstractService; import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; import org.apache.hadoop.yarn.api.records.Container; @@ -46,12 +44,15 @@ import org.apache.hadoop.yarn.api.records.UpdateContainerRequest; import org.apache.hadoop.yarn.api.records.UpdatedContainer; import org.apache.hadoop.yarn.client.api.AMRMClient; import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest; +import org.apache.hadoop.yarn.client.api.TimelineV2Client; import org.apache.hadoop.yarn.client.api.async.impl.AMRMClientAsyncImpl; import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl; import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.util.resource.Resources; import com.google.common.annotations.VisibleForTesting; -import org.apache.hadoop.yarn.util.resource.Resources; +import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; /** * AMRMClientAsync handles communication with the ResourceManager @@ -342,6 +343,25 @@ extends AbstractService { */ public abstract int getClusterNodeCount(); + /** + * Register TimelineClient to AMRMClient. + * @param timelineClient + * @throws YarnException when this method is invoked even when ATS V2 is not + * configured. + */ + public void registerTimelineV2Client(TimelineV2Client timelineClient) + throws YarnException { + client.registerTimelineV2Client(timelineClient); + } + + /** + * Get registered timeline client. + * @return the registered timeline client + */ + public TimelineV2Client getRegisteredTimelineV2Client() { + return client.getRegisteredTimelineV2Client(); + } + /** * Update application's blacklist with addition or removal resources. * @@ -357,7 +377,7 @@ extends AbstractService { * Wait for check to return true for each 1000 ms. * See also {@link #waitFor(com.google.common.base.Supplier, int)} * and {@link #waitFor(com.google.common.base.Supplier, int, int)} - * @param check + * @param check the condition for which it should wait */ public void waitFor(Supplier check) throws InterruptedException { waitFor(check, 1000); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java index 089884a12f6..05132113e14 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java @@ -43,6 +43,7 @@ import org.apache.hadoop.yarn.api.records.UpdateContainerRequest; import org.apache.hadoop.yarn.api.records.UpdatedContainer; import org.apache.hadoop.yarn.client.api.AMRMClient; import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest; +import org.apache.hadoop.yarn.client.api.TimelineV2Client; import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl; import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException; @@ -68,6 +69,8 @@ extends AMRMClientAsync { private volatile boolean keepRunning; private volatile float progress; + private volatile Throwable savedException; + /** * * @param intervalMs heartbeat interval in milliseconds between AM and RM @@ -318,6 +321,19 @@ extends AMRMClientAsync { } AllocateResponse response = (AllocateResponse) object; + + String collectorAddress = null; + if (response.getCollectorInfo() != null) { + collectorAddress = response.getCollectorInfo().getCollectorAddr(); + } + + TimelineV2Client timelineClient = + client.getRegisteredTimelineV2Client(); + if (timelineClient != null && response.getCollectorInfo() != null) { + timelineClient. + setTimelineCollectorInfo(response.getCollectorInfo()); + } + List updatedNodes = response.getUpdatedNodes(); if (!updatedNodes.isEmpty()) { handler.onNodesUpdated(updatedNodes); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java index 1d920fcaa64..83210bd28a5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java @@ -141,7 +141,7 @@ public class YarnClientImpl extends YarnClient { Text timelineService; @VisibleForTesting String timelineDTRenewer; - protected boolean timelineServiceEnabled; + private boolean timelineV1ServiceEnabled; protected boolean timelineServiceBestEffort; private static final String ROOT = "root"; @@ -167,9 +167,14 @@ public class YarnClientImpl extends YarnClient { YarnConfiguration.DEFAULT_YARN_CLIENT_APPLICATION_CLIENT_PROTOCOL_POLL_INTERVAL_MS); } + float timelineServiceVersion = + conf.getFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_VERSION); if (conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, - YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED)) { - timelineServiceEnabled = true; + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED) + && ((Float.compare(timelineServiceVersion, 1.0f) == 0) + || (Float.compare(timelineServiceVersion, 1.5f) == 0))) { + timelineV1ServiceEnabled = true; timelineDTRenewer = getTimelineDelegationTokenRenewer(conf); timelineService = TimelineUtils.buildTimelineTokenService(conf); } @@ -178,7 +183,7 @@ public class YarnClientImpl extends YarnClient { // TimelineServer which means we are able to get history information // for applications/applicationAttempts/containers by using ahsClient // when the TimelineServer is running. - if (timelineServiceEnabled || conf.getBoolean( + if (timelineV1ServiceEnabled || conf.getBoolean( YarnConfiguration.APPLICATION_HISTORY_ENABLED, YarnConfiguration.DEFAULT_APPLICATION_HISTORY_ENABLED)) { historyServiceEnabled = true; @@ -257,7 +262,7 @@ public class YarnClientImpl extends YarnClient { // Automatically add the timeline DT into the CLC // Only when the security and the timeline service are both enabled - if (isSecurityEnabled() && timelineServiceEnabled) { + if (isSecurityEnabled() && timelineV1ServiceEnabled) { addTimelineDelegationToken(appContext.getAMContainerSpec()); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ApplicationMasterServiceProtoTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ApplicationMasterServiceProtoTestBase.java new file mode 100644 index 00000000000..45210188b51 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ApplicationMasterServiceProtoTestBase.java @@ -0,0 +1,72 @@ +/** +* 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.client; + +import java.io.IOException; + +import org.apache.hadoop.ipc.RPC; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; +import org.junit.After; + +/** + * Test Base for Application Master Service Protocol. + */ +public abstract class ApplicationMasterServiceProtoTestBase + extends ProtocolHATestBase { + + private ApplicationMasterProtocol amClient; + private ApplicationAttemptId attemptId; + + protected void startupHAAndSetupClient() throws Exception { + attemptId = this.cluster.createFakeApplicationAttemptId(); + + Token appToken = + this.cluster.getResourceManager().getRMContext() + .getAMRMTokenSecretManager().createAndGetAMRMToken(attemptId); + appToken.setService(ClientRMProxy.getAMRMTokenService(this.conf)); + UserGroupInformation.setLoginUser(UserGroupInformation + .createRemoteUser(UserGroupInformation.getCurrentUser().getUserName())); + UserGroupInformation.getCurrentUser().addToken(appToken); + syncToken(appToken); + amClient = ClientRMProxy + .createRMProxy(this.conf, ApplicationMasterProtocol.class); + } + + @After + public void shutDown() { + if(this.amClient != null) { + RPC.stopProxy(this.amClient); + } + } + + protected ApplicationMasterProtocol getAMClient() { + return amClient; + } + + private void syncToken(Token token) throws IOException { + for (int i = 0; i < this.cluster.getNumOfResourceManager(); i++) { + this.cluster.getResourceManager(i).getRMContext() + .getAMRMTokenSecretManager().addPersistedPassword(token); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java index a8e91323a38..efb198731b0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java @@ -25,6 +25,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse; import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest; import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; +import org.apache.hadoop.yarn.api.records.CollectorInfo; import org.apache.hadoop.yarn.api.records.ApplicationAccessType; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -98,6 +99,7 @@ import org.apache.hadoop.yarn.api.records.QueueState; import org.apache.hadoop.yarn.api.records.QueueUserACLInfo; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Token; +import org.apache.hadoop.yarn.api.records.UpdatedContainer; import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState; import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.api.records.YarnClusterMetrics; @@ -804,11 +806,21 @@ public abstract class ProtocolHATestBase extends ClientBaseWithFixes { } public AllocateResponse createFakeAllocateResponse() { - return AllocateResponse.newInstance(-1, - new ArrayList(), - new ArrayList(), new ArrayList(), - Resource.newInstance(1024, 2), null, 1, - null, new ArrayList()); + if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) { + return AllocateResponse.newInstance(-1, + new ArrayList(), new ArrayList(), + new ArrayList(), Resource.newInstance(1024, 2), null, 1, + null, new ArrayList(), null, + new ArrayList(), + CollectorInfo.newInstance("host:port", Token.newInstance( + new byte[] {0}, "TIMELINE", new byte[] {0}, "rm"))); + } else { + return AllocateResponse.newInstance(-1, + new ArrayList(), + new ArrayList(), new ArrayList(), + Resource.newInstance(1024, 2), null, 1, + null, new ArrayList()); + } } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestApplicationMasterServiceProtocolForTimelineV2.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestApplicationMasterServiceProtocolForTimelineV2.java new file mode 100644 index 00000000000..be8c3023d08 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestApplicationMasterServiceProtocolForTimelineV2.java @@ -0,0 +1,71 @@ +/** +* 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.client; + +import java.io.IOException; +import java.util.ArrayList; + +import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; +import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest; +import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.server.resourcemanager.HATestUtil; +import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +/** + * Tests Application Master Protocol with timeline service v2 enabled. + */ +public class TestApplicationMasterServiceProtocolForTimelineV2 + extends ApplicationMasterServiceProtoTestBase { + + @Before + public void initialize() throws Exception { + HATestUtil.setRpcAddressForRM(RM1_NODE_ID, RM1_PORT_BASE + 200, conf); + HATestUtil.setRpcAddressForRM(RM2_NODE_ID, RM2_PORT_BASE + 200, conf); + conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); + conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f); + conf.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS, + FileSystemTimelineWriterImpl.class, TimelineWriter.class); + startHACluster(0, false, false, true); + super.startupHAAndSetupClient(); + } + + @Test(timeout = 15000) + public void testAllocateForTimelineV2OnHA() + throws YarnException, IOException { + AllocateRequest request = AllocateRequest.newInstance(0, 50f, + new ArrayList(), + new ArrayList(), + ResourceBlacklistRequest.newInstance(new ArrayList(), + new ArrayList())); + AllocateResponse response = getAMClient().allocate(request); + Assert.assertEquals(response, this.cluster.createFakeAllocateResponse()); + Assert.assertNotNull(response.getCollectorInfo()); + Assert.assertEquals("host:port", + response.getCollectorInfo().getCollectorAddr()); + Assert.assertNotNull(response.getCollectorInfo().getCollectorToken()); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestApplicationMasterServiceProtocolOnHA.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestApplicationMasterServiceProtocolOnHA.java index ad86fb377b8..c2f39a1d4ff 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestApplicationMasterServiceProtocolOnHA.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestApplicationMasterServiceProtocolOnHA.java @@ -23,10 +23,6 @@ import java.util.ArrayList; import org.junit.Assert; -import org.apache.hadoop.ipc.RPC; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest; @@ -34,45 +30,20 @@ import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRespons import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest; import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest; import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; -import org.junit.After; import org.junit.Before; import org.junit.Test; public class TestApplicationMasterServiceProtocolOnHA - extends ProtocolHATestBase { - private ApplicationMasterProtocol amClient; - private ApplicationAttemptId attemptId ; - + extends ApplicationMasterServiceProtoTestBase { @Before public void initialize() throws Exception { startHACluster(0, false, false, true); - attemptId = this.cluster.createFakeApplicationAttemptId(); - - Token appToken = - this.cluster.getResourceManager().getRMContext() - .getAMRMTokenSecretManager().createAndGetAMRMToken(attemptId); - appToken.setService(ClientRMProxy.getAMRMTokenService(this.conf)); - UserGroupInformation.setLoginUser(UserGroupInformation - .createRemoteUser(UserGroupInformation.getCurrentUser().getUserName())); - UserGroupInformation.getCurrentUser().addToken(appToken); - syncToken(appToken); - - amClient = ClientRMProxy - .createRMProxy(this.conf, ApplicationMasterProtocol.class); - } - - @After - public void shutDown() { - if(this.amClient != null) { - RPC.stopProxy(this.amClient); - } + super.startupHAAndSetupClient(); } @Test(timeout = 15000) @@ -81,7 +52,7 @@ public class TestApplicationMasterServiceProtocolOnHA RegisterApplicationMasterRequest request = RegisterApplicationMasterRequest.newInstance("localhost", 0, ""); RegisterApplicationMasterResponse response = - amClient.registerApplicationMaster(request); + getAMClient().registerApplicationMaster(request); Assert.assertEquals(response, this.cluster.createFakeRegisterApplicationMasterResponse()); } @@ -93,7 +64,7 @@ public class TestApplicationMasterServiceProtocolOnHA FinishApplicationMasterRequest.newInstance( FinalApplicationStatus.SUCCEEDED, "", ""); FinishApplicationMasterResponse response = - amClient.finishApplicationMaster(request); + getAMClient().finishApplicationMaster(request); Assert.assertEquals(response, this.cluster.createFakeFinishApplicationMasterResponse()); } @@ -105,14 +76,7 @@ public class TestApplicationMasterServiceProtocolOnHA new ArrayList(), ResourceBlacklistRequest.newInstance(new ArrayList(), new ArrayList())); - AllocateResponse response = amClient.allocate(request); + AllocateResponse response = getAMClient().allocate(request); Assert.assertEquals(response, this.cluster.createFakeAllocateResponse()); } - - private void syncToken(Token token) throws IOException { - for (int i = 0; i < this.cluster.getNumOfResourceManager(); i++) { - this.cluster.getResourceManager(i).getRMContext() - .getAMRMTokenSecretManager().addPersistedPassword(token); - } - } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestAMRMClientAsync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestAMRMClientAsync.java index ba383409752..9c644127470 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestAMRMClientAsync.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestAMRMClientAsync.java @@ -426,8 +426,8 @@ public class TestAMRMClientAsync { } AllocateResponse response = AllocateResponse.newInstance(0, completed, allocated, - new ArrayList(), null, null, 1, null, nmTokens, - updatedContainers); + new ArrayList(), null, null, 1, null, nmTokens, null, + updatedContainers, null); return response; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml index c7ca779f67e..3de3f4f909d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml @@ -215,6 +215,7 @@ src/main/resources/webapps/yarn/.keep src/main/resources/webapps/applicationhistory/.keep src/main/resources/webapps/sharedcache/.keep + src/main/resources/webapps/timeline/.keep src/main/resources/webapps/cluster/.keep src/main/resources/webapps/test/.keep src/main/resources/webapps/proxy/.keep diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java index 3f0c6b43e35..06cc45b8fcd 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java @@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.security.proto.SecurityProtos.TokenProto; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; +import org.apache.hadoop.yarn.api.records.CollectorInfo; import org.apache.hadoop.yarn.api.records.AMCommand; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerResourceDecrease; @@ -40,6 +41,7 @@ import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Token; import org.apache.hadoop.yarn.api.records.UpdateContainerError; import org.apache.hadoop.yarn.api.records.UpdatedContainer; +import org.apache.hadoop.yarn.api.records.impl.pb.CollectorInfoPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.NMTokenPBImpl; @@ -50,6 +52,7 @@ import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils; import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.UpdatedContainerPBImpl; +import org.apache.hadoop.yarn.proto.YarnProtos.CollectorInfoProto; import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto; import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto; import org.apache.hadoop.yarn.proto.YarnProtos.NodeReportProto; @@ -82,6 +85,7 @@ public class AllocateResponsePBImpl extends AllocateResponse { private PreemptionMessage preempt; private Token amrmToken = null; private Priority appPriority = null; + private CollectorInfo collectorInfo = null; public AllocateResponsePBImpl() { builder = AllocateResponseProto.newBuilder(); @@ -164,6 +168,9 @@ public class AllocateResponsePBImpl extends AllocateResponse { if (this.amrmToken != null) { builder.setAmRmToken(convertToProtoFormat(this.amrmToken)); } + if (this.collectorInfo != null) { + builder.setCollectorInfo(convertToProtoFormat(this.collectorInfo)); + } if (this.appPriority != null) { builder.setApplicationPriority(convertToProtoFormat(this.appPriority)); } @@ -408,6 +415,29 @@ public class AllocateResponsePBImpl extends AllocateResponse { this.amrmToken = amRMToken; } + + @Override + public synchronized CollectorInfo getCollectorInfo() { + AllocateResponseProtoOrBuilder p = viaProto ? proto : builder; + if (this.collectorInfo != null) { + return this.collectorInfo; + } + if (!p.hasCollectorInfo()) { + return null; + } + this.collectorInfo = convertFromProtoFormat(p.getCollectorInfo()); + return this.collectorInfo; + } + + @Override + public synchronized void setCollectorInfo(CollectorInfo info) { + maybeInitBuilder(); + if (info == null) { + builder.clearCollectorInfo(); + } + this.collectorInfo = info; + } + @Override public synchronized Priority getApplicationPriority() { AllocateResponseProtoOrBuilder p = viaProto ? proto : builder; @@ -713,6 +743,16 @@ public class AllocateResponsePBImpl extends AllocateResponse { return ((NodeReportPBImpl)t).getProto(); } + private synchronized CollectorInfoPBImpl convertFromProtoFormat( + CollectorInfoProto p) { + return new CollectorInfoPBImpl(p); + } + + private synchronized CollectorInfoProto convertToProtoFormat( + CollectorInfo t) { + return ((CollectorInfoPBImpl)t).getProto(); + } + private synchronized ContainerPBImpl convertFromProtoFormat( ContainerProto p) { return new ContainerPBImpl(p); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/CollectorInfoPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/CollectorInfoPBImpl.java new file mode 100644 index 00000000000..5835d1a2b00 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/CollectorInfoPBImpl.java @@ -0,0 +1,152 @@ +/** +* 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.records.impl.pb; + +import org.apache.hadoop.security.proto.SecurityProtos.TokenProto; +import org.apache.hadoop.yarn.api.records.CollectorInfo; +import org.apache.hadoop.yarn.api.records.Token; +import org.apache.hadoop.yarn.proto.YarnProtos.CollectorInfoProto; +import org.apache.hadoop.yarn.proto.YarnProtos.CollectorInfoProtoOrBuilder; + +import com.google.protobuf.TextFormat; + +/** + * Protocol record implementation of {@link CollectorInfo}. + */ +public class CollectorInfoPBImpl extends CollectorInfo { + + private CollectorInfoProto proto = CollectorInfoProto.getDefaultInstance(); + + private CollectorInfoProto.Builder builder = null; + private boolean viaProto = false; + + private String collectorAddr = null; + private Token collectorToken = null; + + + public CollectorInfoPBImpl() { + builder = CollectorInfoProto.newBuilder(); + } + + public CollectorInfoPBImpl(CollectorInfoProto proto) { + this.proto = proto; + viaProto = true; + } + + public CollectorInfoProto getProto() { + mergeLocalToProto(); + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + private void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = CollectorInfoProto.newBuilder(proto); + } + viaProto = false; + } + + private void mergeLocalToProto() { + if (viaProto) { + maybeInitBuilder(); + } + mergeLocalToBuilder(); + proto = builder.build(); + viaProto = true; + } + + @Override + public boolean equals(Object other) { + if (other == null) { + return false; + } + if (other.getClass().isAssignableFrom(this.getClass())) { + return this.getProto().equals(this.getClass().cast(other).getProto()); + } + return false; + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } + + @Override + public String getCollectorAddr() { + CollectorInfoProtoOrBuilder p = viaProto ? proto : builder; + if (this.collectorAddr == null && p.hasCollectorAddr()) { + this.collectorAddr = p.getCollectorAddr(); + } + return this.collectorAddr; + } + + @Override + public void setCollectorAddr(String addr) { + maybeInitBuilder(); + if (collectorAddr == null) { + builder.clearCollectorAddr(); + } + this.collectorAddr = addr; + } + + @Override + public Token getCollectorToken() { + CollectorInfoProtoOrBuilder p = viaProto ? proto : builder; + if (this.collectorToken != null) { + return this.collectorToken; + } + if (!p.hasCollectorToken()) { + return null; + } + this.collectorToken = convertFromProtoFormat(p.getCollectorToken()); + return this.collectorToken; + } + + @Override + public void setCollectorToken(Token token) { + maybeInitBuilder(); + if (token == null) { + builder.clearCollectorToken(); + } + this.collectorToken = token; + } + + private TokenPBImpl convertFromProtoFormat(TokenProto p) { + return new TokenPBImpl(p); + } + + private TokenProto convertToProtoFormat(Token t) { + return ((TokenPBImpl) t).getProto(); + } + + private void mergeLocalToBuilder() { + if (this.collectorAddr != null) { + builder.setCollectorAddr(this.collectorAddr); + } + if (this.collectorToken != null) { + builder.setCollectorToken(convertToProtoFormat(this.collectorToken)); + } + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java index 09298b5991f..4835239a920 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java @@ -21,15 +21,14 @@ package org.apache.hadoop.yarn.client.api; import java.io.Flushable; import java.io.IOException; -import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Evolving; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.service.CompositeService; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity; import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain; +import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity; import org.apache.hadoop.yarn.api.records.timeline.TimelineEntityGroupId; import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse; import org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl; @@ -38,21 +37,24 @@ import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier; /** * A client library that can be used to post some information in terms of a - * number of conceptual entities. + * number of conceptual entities. This client library needs to be used along + * with Timeline V.1.x server versions. + * Refer {@link TimelineV2Client} for ATS V2 interface. */ @Public @Evolving -public abstract class TimelineClient extends AbstractService implements +public abstract class TimelineClient extends CompositeService implements Flushable { /** - * Create a timeline client. The current UGI when the user initialize the - * client will be used to do the put and the delegation token operations. The - * current user may use {@link UserGroupInformation#doAs} another user to - * construct and initialize a timeline client if the following operations are - * supposed to be conducted by that user. + * Creates an instance of the timeline v.1.x client. + * The current UGI when the user initialize the client will be used to do the + * put and the delegation token operations. The current user may use + * {@link UserGroupInformation#doAs} another user to construct and initialize + * a timeline client if the following operations are supposed to be conducted + * by that user. * - * @return a timeline client + * @return the created timeline client instance */ @Public public static TimelineClient createTimelineClient() { @@ -60,7 +62,6 @@ public abstract class TimelineClient extends AbstractService implements return client; } - @Private protected TimelineClient(String name) { super(name); } @@ -75,8 +76,8 @@ public abstract class TimelineClient extends AbstractService implements * @param entities * the collection of {@link TimelineEntity} * @return the error information if the sent entities are not correctly stored - * @throws IOException - * @throws YarnException + * @throws IOException if there are I/O errors + * @throws YarnException if entities are incomplete/invalid */ @Public public abstract TimelinePutResponse putEntities( @@ -96,8 +97,8 @@ public abstract class TimelineClient extends AbstractService implements * @param entities * the collection of {@link TimelineEntity} * @return the error information if the sent entities are not correctly stored - * @throws IOException - * @throws YarnException + * @throws IOException if there are I/O errors + * @throws YarnException if entities are incomplete/invalid */ @Public public abstract TimelinePutResponse putEntities( diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineV2Client.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineV2Client.java new file mode 100644 index 00000000000..423c059319c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineV2Client.java @@ -0,0 +1,96 @@ +/** + * 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.client.api; + +import java.io.IOException; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.service.CompositeService; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.CollectorInfo; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.client.api.impl.TimelineV2ClientImpl; +import org.apache.hadoop.yarn.exceptions.YarnException; + +/** + * A client library that can be used to post some information in terms of a + * number of conceptual entities. This client library needs to be used along + * with time line v.2 server version. + * Refer {@link TimelineClient} for ATS V1 interface. + */ +public abstract class TimelineV2Client extends CompositeService { + /** + * Creates an instance of the timeline v.2 client. + * + * @param appId the application id with which the timeline client is + * associated + * @return the created timeline client instance + */ + @Public + public static TimelineV2Client createTimelineClient(ApplicationId appId) { + TimelineV2Client client = new TimelineV2ClientImpl(appId); + return client; + } + + protected TimelineV2Client(String name) { + super(name); + } + + /** + *

+ * Send the information of a number of conceptual entities to the timeline + * service v.2 collector. It is a blocking API. The method will not return + * until all the put entities have been persisted. + *

+ * + * @param entities the collection of {@link TimelineEntity} + * @throws IOException if there are I/O errors + * @throws YarnException if entities are incomplete/invalid + */ + @Public + public abstract void putEntities(TimelineEntity... entities) + throws IOException, YarnException; + + /** + *

+ * Send the information of a number of conceptual entities to the timeline + * service v.2 collector. It is an asynchronous API. The method will return + * once all the entities are received. + *

+ * + * @param entities the collection of {@link TimelineEntity} + * @throws IOException if there are I/O errors + * @throws YarnException if entities are incomplete/invalid + */ + @Public + public abstract void putEntitiesAsync(TimelineEntity... entities) + throws IOException, YarnException; + + /** + *

+ * Update collector info received in AllocateResponse which contains the + * timeline service address where the request will be sent to and the timeline + * delegation token which will be used to send the request. + *

+ * + * @param collectorInfo Collector info which contains the timeline service + * address and timeline delegation token. + */ + public abstract void setTimelineCollectorInfo(CollectorInfo collectorInfo); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java index 969f78adf0e..48d720a28c9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java @@ -20,21 +20,10 @@ package org.apache.hadoop.yarn.client.api.impl; import java.io.File; import java.io.IOException; -import java.lang.reflect.UndeclaredThrowableException; -import java.net.ConnectException; -import java.net.HttpURLConnection; import java.net.InetSocketAddress; -import java.net.SocketException; -import java.net.SocketTimeoutException; import java.net.URI; -import java.net.URL; -import java.net.URLConnection; -import java.security.GeneralSecurityException; import java.security.PrivilegedExceptionAction; -import javax.net.ssl.HostnameVerifier; -import javax.net.ssl.HttpsURLConnection; -import javax.net.ssl.SSLSocketFactory; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.GnuParser; import org.apache.commons.cli.HelpFormatter; @@ -44,16 +33,10 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Evolving; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.SecurityUtil; -import org.apache.hadoop.security.authentication.client.AuthenticationException; -import org.apache.hadoop.security.authentication.client.ConnectionConfigurator; -import org.apache.hadoop.security.ssl.SSLFactory; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL; -import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator; -import org.apache.hadoop.security.token.delegation.web.KerberosDelegationTokenAuthenticator; -import org.apache.hadoop.security.token.delegation.web.PseudoDelegationTokenAuthenticator; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain; import org.apache.hadoop.yarn.api.records.timeline.TimelineDomains; @@ -67,28 +50,17 @@ import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier; import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider; import org.codehaus.jackson.map.ObjectMapper; + import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Joiner; -import com.google.common.base.Preconditions; import com.sun.jersey.api.client.Client; -import com.sun.jersey.api.client.ClientHandlerException; -import com.sun.jersey.api.client.ClientRequest; -import com.sun.jersey.api.client.ClientResponse; -import com.sun.jersey.api.client.config.ClientConfig; -import com.sun.jersey.api.client.config.DefaultClientConfig; -import com.sun.jersey.api.client.filter.ClientFilter; -import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory; -import com.sun.jersey.client.urlconnection.URLConnectionClientHandler; @Private @Evolving public class TimelineClientImpl extends TimelineClient { private static final Log LOG = LogFactory.getLog(TimelineClientImpl.class); - private static final String RESOURCE_URI_STR = "/ws/v1/timeline/"; private static final ObjectMapper MAPPER = new ObjectMapper(); - private static final Joiner JOINER = Joiner.on(""); - public final static int DEFAULT_SOCKET_TIMEOUT = 1 * 60 * 1000; // 1 minute + private static final String RESOURCE_URI_STR_V1 = "/ws/v1/timeline/"; private static Options opts; private static final String ENTITY_DATA_TYPE = "entity"; @@ -103,166 +75,38 @@ public class TimelineClientImpl extends TimelineClient { opts.addOption("help", false, "Print usage"); } - private Client client; - private ConnectionConfigurator connConfigurator; - private DelegationTokenAuthenticator authenticator; - private DelegationTokenAuthenticatedURL.Token token; - private URI resURI; - private UserGroupInformation authUgi; - private String doAsUser; - private Configuration configuration; + @VisibleForTesting + protected DelegationTokenAuthenticatedURL.Token token; + @VisibleForTesting + protected UserGroupInformation authUgi; + @VisibleForTesting + protected String doAsUser; + private float timelineServiceVersion; private TimelineWriter timelineWriter; - private SSLFactory sslFactory; + + private String timelineServiceAddress; @Private @VisibleForTesting - TimelineClientConnectionRetry connectionRetry; - - // Abstract class for an operation that should be retried by timeline client - @Private - @VisibleForTesting - public static abstract class TimelineClientRetryOp { - // The operation that should be retried - public abstract Object run() throws IOException; - // The method to indicate if we should retry given the incoming exception - public abstract boolean shouldRetryOn(Exception e); - } - - // Class to handle retry - // Outside this class, only visible to tests - @Private - @VisibleForTesting - static class TimelineClientConnectionRetry { - - // maxRetries < 0 means keep trying - @Private - @VisibleForTesting - public int maxRetries; - - @Private - @VisibleForTesting - public long retryInterval; - - // Indicates if retries happened last time. Only tests should read it. - // In unit tests, retryOn() calls should _not_ be concurrent. - private boolean retried = false; - - @Private - @VisibleForTesting - boolean getRetired() { - return retried; - } - - // Constructor with default retry settings - public TimelineClientConnectionRetry(Configuration conf) { - Preconditions.checkArgument(conf.getInt( - YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES, - YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_MAX_RETRIES) >= -1, - "%s property value should be greater than or equal to -1", - YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES); - Preconditions - .checkArgument( - conf.getLong( - YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS, - YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS) > 0, - "%s property value should be greater than zero", - YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS); - maxRetries = conf.getInt( - YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES, - YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_MAX_RETRIES); - retryInterval = conf.getLong( - YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS, - YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS); - } - - public Object retryOn(TimelineClientRetryOp op) - throws RuntimeException, IOException { - int leftRetries = maxRetries; - retried = false; - - // keep trying - while (true) { - try { - // try perform the op, if fail, keep retrying - return op.run(); - } catch (IOException | RuntimeException e) { - // break if there's no retries left - if (leftRetries == 0) { - break; - } - if (op.shouldRetryOn(e)) { - logException(e, leftRetries); - } else { - throw e; - } - } - if (leftRetries > 0) { - leftRetries--; - } - retried = true; - try { - // sleep for the given time interval - Thread.sleep(retryInterval); - } catch (InterruptedException ie) { - LOG.warn("Client retry sleep interrupted! "); - } - } - throw new RuntimeException("Failed to connect to timeline server. " - + "Connection retries limit exceeded. " - + "The posted timeline event may be missing"); - }; - - private void logException(Exception e, int leftRetries) { - if (leftRetries > 0) { - LOG.info("Exception caught by TimelineClientConnectionRetry," - + " will try " + leftRetries + " more time(s).\nMessage: " - + e.getMessage()); - } else { - // note that maxRetries may be -1 at the very beginning - LOG.info("ConnectionException caught by TimelineClientConnectionRetry," - + " will keep retrying.\nMessage: " - + e.getMessage()); - } - } - } - - private class TimelineJerseyRetryFilter extends ClientFilter { - @Override - public ClientResponse handle(final ClientRequest cr) - throws ClientHandlerException { - // Set up the retry operation - TimelineClientRetryOp jerseyRetryOp = new TimelineClientRetryOp() { - @Override - public Object run() { - // Try pass the request, if fail, keep retrying - return getNext().handle(cr); - } - - @Override - public boolean shouldRetryOn(Exception e) { - // Only retry on connection exceptions - return (e instanceof ClientHandlerException) - && (e.getCause() instanceof ConnectException || - e.getCause() instanceof SocketTimeoutException || - e.getCause() instanceof SocketException); - } - }; - try { - return (ClientResponse) connectionRetry.retryOn(jerseyRetryOp); - } catch (IOException e) { - throw new ClientHandlerException("Jersey retry failed!\nMessage: " - + e.getMessage()); - } - } - } + TimelineConnector connector; public TimelineClientImpl() { super(TimelineClientImpl.class.getName()); } protected void serviceInit(Configuration conf) throws Exception { - this.configuration = conf; + timelineServiceVersion = + conf.getFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_VERSION); + LOG.info("Timeline service address: " + getTimelineServiceAddress()); + if (!YarnConfiguration.timelineServiceEnabled(conf) + || !((Float.compare(this.timelineServiceVersion, 1.0f) == 0) + || (Float.compare(this.timelineServiceVersion, 1.5f) == 0))) { + throw new IOException("Timeline V1 client is not properly configured. " + + "Either timeline service is not enabled or version is not set to" + + " 1.x"); + } UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); UserGroupInformation realUgi = ugi.getRealUser(); if (realUgi != null) { @@ -272,46 +116,34 @@ public class TimelineClientImpl extends TimelineClient { authUgi = ugi; doAsUser = null; } - ClientConfig cc = new DefaultClientConfig(); - cc.getClasses().add(YarnJacksonJaxbJsonProvider.class); - connConfigurator = initConnConfigurator(conf); - if (UserGroupInformation.isSecurityEnabled()) { - authenticator = new KerberosDelegationTokenAuthenticator(); - } else { - authenticator = new PseudoDelegationTokenAuthenticator(); - } - authenticator.setConnectionConfigurator(connConfigurator); token = new DelegationTokenAuthenticatedURL.Token(); - - connectionRetry = new TimelineClientConnectionRetry(conf); - client = new Client(new URLConnectionClientHandler( - new TimelineURLConnectionFactory()), cc); - TimelineJerseyRetryFilter retryFilter = new TimelineJerseyRetryFilter(); - client.addFilter(retryFilter); + connector = createTimelineConnector(); if (YarnConfiguration.useHttps(conf)) { - resURI = URI - .create(JOINER.join("https://", conf.get( - YarnConfiguration.TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS, - YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS), - RESOURCE_URI_STR)); + timelineServiceAddress = + conf.get(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS, + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS); } else { - resURI = URI.create(JOINER.join("http://", conf.get( - YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS, - YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS), - RESOURCE_URI_STR)); + timelineServiceAddress = + conf.get(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS, + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS); } - LOG.info("Timeline service address: " + resURI); - timelineServiceVersion = - conf.getFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, - YarnConfiguration.DEFAULT_TIMELINE_SERVICE_VERSION); super.serviceInit(conf); } + @VisibleForTesting + protected TimelineConnector createTimelineConnector() { + TimelineConnector newConnector = + new TimelineConnector(true, authUgi, doAsUser, token); + addIfService(newConnector); + return newConnector; + } + @Override protected void serviceStart() throws Exception { - timelineWriter = createTimelineWriter( - configuration, authUgi, client, resURI); + timelineWriter = createTimelineWriter(getConfig(), authUgi, + connector.getClient(), TimelineConnector.constructResURI(getConfig(), + timelineServiceAddress, RESOURCE_URI_STR_V1)); } protected TimelineWriter createTimelineWriter(Configuration conf, @@ -330,9 +162,6 @@ public class TimelineClientImpl extends TimelineClient { if (this.timelineWriter != null) { this.timelineWriter.close(); } - if (this.sslFactory != null) { - this.sslFactory.destroy(); - } super.serviceStop(); } @@ -344,36 +173,44 @@ public class TimelineClientImpl extends TimelineClient { } @Override - public TimelinePutResponse putEntities( - TimelineEntity... entities) throws IOException, YarnException { + public TimelinePutResponse putEntities(TimelineEntity... entities) + throws IOException, YarnException { return timelineWriter.putEntities(entities); } - @Override public void putDomain(TimelineDomain domain) throws IOException, YarnException { timelineWriter.putDomain(domain); } + private String getTimelineServiceAddress() { + return this.timelineServiceAddress; + } + @SuppressWarnings("unchecked") @Override public Token getDelegationToken( final String renewer) throws IOException, YarnException { - PrivilegedExceptionAction> getDTAction = + PrivilegedExceptionAction> + getDTAction = new PrivilegedExceptionAction>() { @Override public Token run() throws Exception { DelegationTokenAuthenticatedURL authUrl = - new DelegationTokenAuthenticatedURL(authenticator, - connConfigurator); + connector.getDelegationTokenAuthenticatedURL(); + // TODO we should add retry logic here if timelineServiceAddress is + // not available immediately. return (Token) authUrl.getDelegationToken( - resURI.toURL(), token, renewer, doAsUser); + TimelineConnector.constructResURI(getConfig(), + getTimelineServiceAddress(), RESOURCE_URI_STR_V1).toURL(), + token, renewer, doAsUser); } }; - return (Token) operateDelegationToken(getDTAction); + return (Token) connector + .operateDelegationToken(getDTAction); } @SuppressWarnings("unchecked") @@ -393,31 +230,32 @@ public class TimelineClientImpl extends TimelineClient { @Override public Long run() throws Exception { // If the timeline DT to renew is different than cached, replace it. - // Token to set every time for retry, because when exception happens, - // DelegationTokenAuthenticatedURL will reset it to null; + // Token to set every time for retry, because when exception + // happens, DelegationTokenAuthenticatedURL will reset it to null; if (!timelineDT.equals(token.getDelegationToken())) { token.setDelegationToken((Token) timelineDT); } DelegationTokenAuthenticatedURL authUrl = - new DelegationTokenAuthenticatedURL(authenticator, - connConfigurator); + connector.getDelegationTokenAuthenticatedURL(); // If the token service address is not available, fall back to use // the configured service address. - final URI serviceURI = isTokenServiceAddrEmpty ? resURI + final URI serviceURI = isTokenServiceAddrEmpty + ? TimelineConnector.constructResURI(getConfig(), + getTimelineServiceAddress(), RESOURCE_URI_STR_V1) : new URI(scheme, null, address.getHostName(), - address.getPort(), RESOURCE_URI_STR, null, null); + address.getPort(), RESOURCE_URI_STR_V1, null, null); return authUrl .renewDelegationToken(serviceURI.toURL(), token, doAsUser); } }; - return (Long) operateDelegationToken(renewDTAction); + return (Long) connector.operateDelegationToken(renewDTAction); } @SuppressWarnings("unchecked") @Override public void cancelDelegationToken( final Token timelineDT) - throws IOException, YarnException { + throws IOException, YarnException { final boolean isTokenServiceAddrEmpty = timelineDT.getService().toString().isEmpty(); final String scheme = isTokenServiceAddrEmpty ? null @@ -429,112 +267,37 @@ public class TimelineClientImpl extends TimelineClient { @Override public Void run() throws Exception { - // If the timeline DT to cancel is different than cached, replace it. - // Token to set every time for retry, because when exception happens, - // DelegationTokenAuthenticatedURL will reset it to null; + // If the timeline DT to cancel is different than cached, replace + // it. + // Token to set every time for retry, because when exception + // happens, DelegationTokenAuthenticatedURL will reset it to null; if (!timelineDT.equals(token.getDelegationToken())) { token.setDelegationToken((Token) timelineDT); } DelegationTokenAuthenticatedURL authUrl = - new DelegationTokenAuthenticatedURL(authenticator, - connConfigurator); + connector.getDelegationTokenAuthenticatedURL(); // If the token service address is not available, fall back to use // the configured service address. - final URI serviceURI = isTokenServiceAddrEmpty ? resURI + final URI serviceURI = isTokenServiceAddrEmpty + ? TimelineConnector.constructResURI(getConfig(), + getTimelineServiceAddress(), RESOURCE_URI_STR_V1) : new URI(scheme, null, address.getHostName(), - address.getPort(), RESOURCE_URI_STR, null, null); + address.getPort(), RESOURCE_URI_STR_V1, null, null); authUrl.cancelDelegationToken(serviceURI.toURL(), token, doAsUser); return null; } }; - operateDelegationToken(cancelDTAction); + connector.operateDelegationToken(cancelDTAction); } @Override public String toString() { - return super.toString() + " with timeline server " + resURI + return super.toString() + " with timeline server " + + TimelineConnector.constructResURI(getConfig(), + getTimelineServiceAddress(), RESOURCE_URI_STR_V1) + " and writer " + timelineWriter; } - private Object operateDelegationToken( - final PrivilegedExceptionAction action) - throws IOException, YarnException { - // Set up the retry operation - TimelineClientRetryOp tokenRetryOp = - createTimelineClientRetryOpForOperateDelegationToken(action); - - return connectionRetry.retryOn(tokenRetryOp); - } - - private class TimelineURLConnectionFactory - implements HttpURLConnectionFactory { - - @Override - public HttpURLConnection getHttpURLConnection(final URL url) throws IOException { - authUgi.checkTGTAndReloginFromKeytab(); - try { - return new DelegationTokenAuthenticatedURL( - authenticator, connConfigurator).openConnection(url, token, - doAsUser); - } catch (UndeclaredThrowableException e) { - throw new IOException(e.getCause()); - } catch (AuthenticationException ae) { - throw new IOException(ae); - } - } - - } - - private ConnectionConfigurator initConnConfigurator(Configuration conf) { - try { - return initSslConnConfigurator(DEFAULT_SOCKET_TIMEOUT, conf); - } catch (Exception e) { - LOG.debug("Cannot load customized ssl related configuration. " + - "Fallback to system-generic settings.", e); - return DEFAULT_TIMEOUT_CONN_CONFIGURATOR; - } - } - - private static final ConnectionConfigurator DEFAULT_TIMEOUT_CONN_CONFIGURATOR = - new ConnectionConfigurator() { - @Override - public HttpURLConnection configure(HttpURLConnection conn) - throws IOException { - setTimeouts(conn, DEFAULT_SOCKET_TIMEOUT); - return conn; - } - }; - - private ConnectionConfigurator initSslConnConfigurator(final int timeout, - Configuration conf) throws IOException, GeneralSecurityException { - final SSLSocketFactory sf; - final HostnameVerifier hv; - - sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf); - sslFactory.init(); - sf = sslFactory.createSSLSocketFactory(); - hv = sslFactory.getHostnameVerifier(); - - return new ConnectionConfigurator() { - @Override - public HttpURLConnection configure(HttpURLConnection conn) - throws IOException { - if (conn instanceof HttpsURLConnection) { - HttpsURLConnection c = (HttpsURLConnection) conn; - c.setSSLSocketFactory(sf); - c.setHostnameVerifier(hv); - } - setTimeouts(conn, timeout); - return conn; - } - }; - } - - private static void setTimeouts(URLConnection connection, int socketTimeout) { - connection.setConnectTimeout(socketTimeout); - connection.setReadTimeout(socketTimeout); - } - public static void main(String[] argv) throws Exception { CommandLine cliParser = new GnuParser().parse(opts, argv); if (cliParser.hasOption("put")) { @@ -670,50 +433,4 @@ public class TimelineClientImpl extends TimelineClient { public void setTimelineWriter(TimelineWriter writer) { this.timelineWriter = writer; } - - @Private - @VisibleForTesting - public TimelineClientRetryOp - createTimelineClientRetryOpForOperateDelegationToken( - final PrivilegedExceptionAction action) throws IOException { - return new TimelineClientRetryOpForOperateDelegationToken( - this.authUgi, action); - } - - @Private - @VisibleForTesting - public class TimelineClientRetryOpForOperateDelegationToken - extends TimelineClientRetryOp { - - private final UserGroupInformation authUgi; - private final PrivilegedExceptionAction action; - - public TimelineClientRetryOpForOperateDelegationToken( - UserGroupInformation authUgi, PrivilegedExceptionAction action) { - this.authUgi = authUgi; - this.action = action; - } - - @Override - public Object run() throws IOException { - // Try pass the request, if fail, keep retrying - authUgi.checkTGTAndReloginFromKeytab(); - try { - return authUgi.doAs(action); - } catch (UndeclaredThrowableException e) { - throw new IOException(e.getCause()); - } catch (InterruptedException e) { - throw new IOException(e); - } - } - - @Override - public boolean shouldRetryOn(Exception e) { - // retry on connection exceptions - // and SocketTimeoutException - return (e instanceof ConnectException - || e instanceof SocketTimeoutException); - } - } - } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineConnector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineConnector.java new file mode 100644 index 00000000000..188371e5e3a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineConnector.java @@ -0,0 +1,447 @@ +/** + * 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.client.api.impl; + +import java.io.IOException; +import java.lang.reflect.UndeclaredThrowableException; +import java.net.ConnectException; +import java.net.HttpURLConnection; +import java.net.SocketException; +import java.net.SocketTimeoutException; +import java.net.URI; +import java.net.URL; +import java.net.URLConnection; +import java.security.GeneralSecurityException; +import java.security.PrivilegedExceptionAction; + +import javax.net.ssl.HostnameVerifier; +import javax.net.ssl.HttpsURLConnection; +import javax.net.ssl.SSLSocketFactory; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.authentication.client.AuthenticationException; +import org.apache.hadoop.security.authentication.client.ConnectionConfigurator; +import org.apache.hadoop.security.ssl.SSLFactory; +import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL; +import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL.Token; +import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator; +import org.apache.hadoop.security.token.delegation.web.KerberosDelegationTokenAuthenticator; +import org.apache.hadoop.security.token.delegation.web.PseudoDelegationTokenAuthenticator; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import com.sun.jersey.api.client.Client; +import com.sun.jersey.api.client.ClientHandlerException; +import com.sun.jersey.api.client.ClientRequest; +import com.sun.jersey.api.client.ClientResponse; +import com.sun.jersey.api.client.config.ClientConfig; +import com.sun.jersey.api.client.config.DefaultClientConfig; +import com.sun.jersey.api.client.filter.ClientFilter; +import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory; +import com.sun.jersey.client.urlconnection.URLConnectionClientHandler; + +/** + * Utility Connector class which is used by timeline clients to securely get + * connected to the timeline server. + * + */ +public class TimelineConnector extends AbstractService { + + private static final Joiner JOINER = Joiner.on(""); + private static final Log LOG = LogFactory.getLog(TimelineConnector.class); + public final static int DEFAULT_SOCKET_TIMEOUT = 1 * 60 * 1000; // 1 minute + + private SSLFactory sslFactory; + private Client client; + private ConnectionConfigurator connConfigurator; + private DelegationTokenAuthenticator authenticator; + private DelegationTokenAuthenticatedURL.Token token; + private UserGroupInformation authUgi; + private String doAsUser; + @VisibleForTesting + TimelineClientConnectionRetry connectionRetry; + private boolean requireConnectionRetry; + + public TimelineConnector(boolean requireConnectionRetry, + UserGroupInformation authUgi, String doAsUser, + DelegationTokenAuthenticatedURL.Token token) { + super("TimelineConnector"); + this.requireConnectionRetry = requireConnectionRetry; + this.authUgi = authUgi; + this.doAsUser = doAsUser; + this.token = token; + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + super.serviceInit(conf); + ClientConfig cc = new DefaultClientConfig(); + cc.getClasses().add(YarnJacksonJaxbJsonProvider.class); + + if (YarnConfiguration.useHttps(conf)) { + // If https is chosen, configures SSL client. + sslFactory = getSSLFactory(conf); + connConfigurator = getConnConfigurator(sslFactory); + } else { + connConfigurator = DEFAULT_TIMEOUT_CONN_CONFIGURATOR; + } + + if (UserGroupInformation.isSecurityEnabled()) { + authenticator = new KerberosDelegationTokenAuthenticator(); + } else { + authenticator = new PseudoDelegationTokenAuthenticator(); + } + authenticator.setConnectionConfigurator(connConfigurator); + + connectionRetry = new TimelineClientConnectionRetry(conf); + client = + new Client( + new URLConnectionClientHandler(new TimelineURLConnectionFactory( + authUgi, authenticator, connConfigurator, token, doAsUser)), + cc); + if (requireConnectionRetry) { + TimelineJerseyRetryFilter retryFilter = + new TimelineJerseyRetryFilter(connectionRetry); + client.addFilter(retryFilter); + } + } + + private static final ConnectionConfigurator + DEFAULT_TIMEOUT_CONN_CONFIGURATOR = new ConnectionConfigurator() { + @Override + public HttpURLConnection configure(HttpURLConnection conn) + throws IOException { + setTimeouts(conn, DEFAULT_SOCKET_TIMEOUT); + return conn; + } + }; + + private ConnectionConfigurator getConnConfigurator(SSLFactory sslFactoryObj) { + try { + return initSslConnConfigurator(DEFAULT_SOCKET_TIMEOUT, sslFactoryObj); + } catch (Exception e) { + LOG.debug("Cannot load customized ssl related configuration. " + + "Fallback to system-generic settings.", e); + return DEFAULT_TIMEOUT_CONN_CONFIGURATOR; + } + } + + private static ConnectionConfigurator initSslConnConfigurator( + final int timeout, SSLFactory sslFactory) + throws IOException, GeneralSecurityException { + final SSLSocketFactory sf; + final HostnameVerifier hv; + + sf = sslFactory.createSSLSocketFactory(); + hv = sslFactory.getHostnameVerifier(); + + return new ConnectionConfigurator() { + @Override + public HttpURLConnection configure(HttpURLConnection conn) + throws IOException { + if (conn instanceof HttpsURLConnection) { + HttpsURLConnection c = (HttpsURLConnection) conn; + c.setSSLSocketFactory(sf); + c.setHostnameVerifier(hv); + } + setTimeouts(conn, timeout); + return conn; + } + }; + } + + protected SSLFactory getSSLFactory(Configuration conf) + throws GeneralSecurityException, IOException { + SSLFactory newSSLFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf); + newSSLFactory.init(); + return newSSLFactory; + } + + private static void setTimeouts(URLConnection connection, int socketTimeout) { + connection.setConnectTimeout(socketTimeout); + connection.setReadTimeout(socketTimeout); + } + + public static URI constructResURI(Configuration conf, String address, + String uri) { + return URI.create( + JOINER.join(YarnConfiguration.useHttps(conf) ? "https://" : "http://", + address, uri)); + } + + DelegationTokenAuthenticatedURL getDelegationTokenAuthenticatedURL() { + return new DelegationTokenAuthenticatedURL(authenticator, connConfigurator); + } + + protected void serviceStop() { + if (this.sslFactory != null) { + this.sslFactory.destroy(); + } + } + + public Client getClient() { + return client; + } + + public Object operateDelegationToken( + final PrivilegedExceptionAction action) + throws IOException, YarnException { + // Set up the retry operation + TimelineClientRetryOp tokenRetryOp = + createRetryOpForOperateDelegationToken(action); + + return connectionRetry.retryOn(tokenRetryOp); + } + + @Private + @VisibleForTesting + TimelineClientRetryOp createRetryOpForOperateDelegationToken( + final PrivilegedExceptionAction action) throws IOException { + return new TimelineClientRetryOpForOperateDelegationToken(this.authUgi, + action); + } + + /** + * Abstract class for an operation that should be retried by timeline client. + */ + @Private + @VisibleForTesting + public static abstract class TimelineClientRetryOp { + // The operation that should be retried + public abstract Object run() throws IOException; + + // The method to indicate if we should retry given the incoming exception + public abstract boolean shouldRetryOn(Exception e); + } + + private static class TimelineURLConnectionFactory + implements HttpURLConnectionFactory { + private DelegationTokenAuthenticator authenticator; + private UserGroupInformation authUgi; + private ConnectionConfigurator connConfigurator; + private Token token; + private String doAsUser; + + public TimelineURLConnectionFactory(UserGroupInformation authUgi, + DelegationTokenAuthenticator authenticator, + ConnectionConfigurator connConfigurator, + DelegationTokenAuthenticatedURL.Token token, String doAsUser) { + this.authUgi = authUgi; + this.authenticator = authenticator; + this.connConfigurator = connConfigurator; + this.token = token; + this.doAsUser = doAsUser; + } + + @Override + public HttpURLConnection getHttpURLConnection(final URL url) + throws IOException { + authUgi.checkTGTAndReloginFromKeytab(); + try { + return new DelegationTokenAuthenticatedURL(authenticator, + connConfigurator).openConnection(url, token, doAsUser); + } catch (UndeclaredThrowableException e) { + throw new IOException(e.getCause()); + } catch (AuthenticationException ae) { + throw new IOException(ae); + } + } + + } + + // Class to handle retry + // Outside this class, only visible to tests + @Private + @VisibleForTesting + static class TimelineClientConnectionRetry { + + // maxRetries < 0 means keep trying + @Private + @VisibleForTesting + public int maxRetries; + + @Private + @VisibleForTesting + public long retryInterval; + + // Indicates if retries happened last time. Only tests should read it. + // In unit tests, retryOn() calls should _not_ be concurrent. + private boolean retried = false; + + @Private + @VisibleForTesting + boolean getRetired() { + return retried; + } + + // Constructor with default retry settings + public TimelineClientConnectionRetry(Configuration conf) { + Preconditions.checkArgument( + conf.getInt(YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES, + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_MAX_RETRIES) + >= -1, + "%s property value should be greater than or equal to -1", + YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES); + Preconditions.checkArgument( + conf.getLong( + YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS, + YarnConfiguration. + DEFAULT_TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS) > 0, + "%s property value should be greater than zero", + YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS); + maxRetries = + conf.getInt(YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES, + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_MAX_RETRIES); + retryInterval = conf.getLong( + YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS, + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS); + } + + public Object retryOn(TimelineClientRetryOp op) + throws RuntimeException, IOException { + int leftRetries = maxRetries; + retried = false; + + // keep trying + while (true) { + try { + // try perform the op, if fail, keep retrying + return op.run(); + } catch (IOException | RuntimeException e) { + // break if there's no retries left + if (leftRetries == 0) { + break; + } + if (op.shouldRetryOn(e)) { + logException(e, leftRetries); + } else { + throw e; + } + } + if (leftRetries > 0) { + leftRetries--; + } + retried = true; + try { + // sleep for the given time interval + Thread.sleep(retryInterval); + } catch (InterruptedException ie) { + LOG.warn("Client retry sleep interrupted! "); + } + } + throw new RuntimeException("Failed to connect to timeline server. " + + "Connection retries limit exceeded. " + + "The posted timeline event may be missing"); + }; + + private void logException(Exception e, int leftRetries) { + if (leftRetries > 0) { + LOG.info( + "Exception caught by TimelineClientConnectionRetry," + " will try " + + leftRetries + " more time(s).\nMessage: " + e.getMessage()); + } else { + // note that maxRetries may be -1 at the very beginning + LOG.info("ConnectionException caught by TimelineClientConnectionRetry," + + " will keep retrying.\nMessage: " + e.getMessage()); + } + } + } + + private static class TimelineJerseyRetryFilter extends ClientFilter { + private TimelineClientConnectionRetry connectionRetry; + + public TimelineJerseyRetryFilter( + TimelineClientConnectionRetry connectionRetry) { + this.connectionRetry = connectionRetry; + } + + @Override + public ClientResponse handle(final ClientRequest cr) + throws ClientHandlerException { + // Set up the retry operation + TimelineClientRetryOp jerseyRetryOp = new TimelineClientRetryOp() { + @Override + public Object run() { + // Try pass the request, if fail, keep retrying + return getNext().handle(cr); + } + + @Override + public boolean shouldRetryOn(Exception e) { + // Only retry on connection exceptions + return (e instanceof ClientHandlerException) + && (e.getCause() instanceof ConnectException + || e.getCause() instanceof SocketTimeoutException + || e.getCause() instanceof SocketException); + } + }; + try { + return (ClientResponse) connectionRetry.retryOn(jerseyRetryOp); + } catch (IOException e) { + throw new ClientHandlerException( + "Jersey retry failed!\nMessage: " + e.getMessage()); + } + } + } + + @Private + @VisibleForTesting + public static class TimelineClientRetryOpForOperateDelegationToken + extends TimelineClientRetryOp { + + private final UserGroupInformation authUgi; + private final PrivilegedExceptionAction action; + + public TimelineClientRetryOpForOperateDelegationToken( + UserGroupInformation authUgi, PrivilegedExceptionAction action) { + this.authUgi = authUgi; + this.action = action; + } + + @Override + public Object run() throws IOException { + // Try pass the request, if fail, keep retrying + authUgi.checkTGTAndReloginFromKeytab(); + try { + return authUgi.doAs(action); + } catch (UndeclaredThrowableException e) { + throw new IOException(e.getCause()); + } catch (InterruptedException e) { + throw new IOException(e); + } + } + + @Override + public boolean shouldRetryOn(Exception e) { + // retry on connection exceptions + // and SocketTimeoutException + return (e instanceof ConnectException + || e instanceof SocketTimeoutException); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java new file mode 100644 index 00000000000..bb505955a50 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java @@ -0,0 +1,554 @@ +/** + * 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.client.api.impl; + +import java.io.IOException; +import java.io.InterruptedIOException; +import java.lang.reflect.UndeclaredThrowableException; +import java.net.InetSocketAddress; +import java.net.URI; +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.ws.rs.core.MediaType; +import javax.ws.rs.core.MultivaluedMap; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.CollectorInfo; +import org.apache.hadoop.yarn.api.records.Token; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.client.api.TimelineV2Client; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier; + +import com.google.common.annotations.VisibleForTesting; +import com.sun.jersey.api.client.ClientResponse; +import com.sun.jersey.core.util.MultivaluedMapImpl; + +/** + * Implementation of timeline v2 client interface. + * + */ +public class TimelineV2ClientImpl extends TimelineV2Client { + private static final Log LOG = LogFactory.getLog(TimelineV2ClientImpl.class); + + private static final String RESOURCE_URI_STR_V2 = "/ws/v2/timeline/"; + + private TimelineEntityDispatcher entityDispatcher; + private volatile String timelineServiceAddress; + @VisibleForTesting + volatile Token currentTimelineToken = null; + + // Retry parameters for identifying new timeline service + // TODO consider to merge with connection retry + private int maxServiceRetries; + private long serviceRetryInterval; + + private TimelineConnector connector; + + private ApplicationId contextAppId; + + private UserGroupInformation authUgi; + + public TimelineV2ClientImpl(ApplicationId appId) { + super(TimelineV2ClientImpl.class.getName()); + this.contextAppId = appId; + } + + public ApplicationId getContextAppId() { + return contextAppId; + } + + protected void serviceInit(Configuration conf) throws Exception { + if (!YarnConfiguration.timelineServiceEnabled(conf) + || (int) YarnConfiguration.getTimelineServiceVersion(conf) != 2) { + throw new IOException("Timeline V2 client is not properly configured. " + + "Either timeline service is not enabled or version is not set to" + + " 2"); + } + UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); + UserGroupInformation realUgi = ugi.getRealUser(); + String doAsUser = null; + if (realUgi != null) { + authUgi = realUgi; + doAsUser = ugi.getShortUserName(); + } else { + authUgi = ugi; + doAsUser = null; + } + // TODO need to add/cleanup filter retry later for ATSV2. similar to V1 + DelegationTokenAuthenticatedURL.Token token = + new DelegationTokenAuthenticatedURL.Token(); + connector = new TimelineConnector(false, authUgi, doAsUser, token); + addIfService(connector); + + // new version need to auto discovery (with retry till ATS v2 address is + // got). + maxServiceRetries = + conf.getInt(YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES, + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_MAX_RETRIES); + serviceRetryInterval = conf.getLong( + YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS, + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS); + entityDispatcher = new TimelineEntityDispatcher(conf); + super.serviceInit(conf); + } + + @Override + protected void serviceStart() throws Exception { + super.serviceStart(); + entityDispatcher.start(); + } + + @Override + protected void serviceStop() throws Exception { + entityDispatcher.stop(); + super.serviceStop(); + } + + @Override + public void putEntities(TimelineEntity... entities) + throws IOException, YarnException { + entityDispatcher.dispatchEntities(true, entities); + } + + @Override + public void putEntitiesAsync(TimelineEntity... entities) + throws IOException, YarnException { + entityDispatcher.dispatchEntities(false, entities); + } + + @Override + public void setTimelineCollectorInfo(CollectorInfo collectorInfo) { + if (collectorInfo == null) { + LOG.warn("Not setting collector info as it is null."); + return; + } + // First update the token so that it is available when collector address is + // used. + if (collectorInfo.getCollectorToken() != null) { + // Use collector address to update token service if its not available. + setTimelineDelegationToken( + collectorInfo.getCollectorToken(), collectorInfo.getCollectorAddr()); + } + // Update timeline service address. + if (collectorInfo.getCollectorAddr() != null && + !collectorInfo.getCollectorAddr().isEmpty() && + !collectorInfo.getCollectorAddr().equals(timelineServiceAddress)) { + this.timelineServiceAddress = collectorInfo.getCollectorAddr(); + LOG.info("Updated timeline service address to " + timelineServiceAddress); + } + } + + private void setTimelineDelegationToken(Token delegationToken, + String collectorAddr) { + // Checks below are to ensure that an invalid token is not updated in UGI. + // This is required because timeline token is set via a public API. + if (!delegationToken.getKind().equals( + TimelineDelegationTokenIdentifier.KIND_NAME.toString())) { + LOG.warn("Timeline token to be updated should be of kind " + + TimelineDelegationTokenIdentifier.KIND_NAME); + return; + } + if (collectorAddr == null || collectorAddr.isEmpty()) { + collectorAddr = timelineServiceAddress; + } + // Token need not be updated if either address or token service does not + // exist. + String service = delegationToken.getService(); + if ((service == null || service.isEmpty()) && + (collectorAddr == null || collectorAddr.isEmpty())) { + LOG.warn("Timeline token does not have service and timeline service " + + "address is not yet set. Not updating the token"); + return; + } + // No need to update a duplicate token. + if (currentTimelineToken != null && + currentTimelineToken.equals(delegationToken)) { + return; + } + currentTimelineToken = delegationToken; + // Convert the token, sanitize the token service and add it to UGI. + org.apache.hadoop.security.token. + Token timelineToken = + new org.apache.hadoop.security.token. + Token( + delegationToken.getIdentifier().array(), + delegationToken.getPassword().array(), + new Text(delegationToken.getKind()), + service == null ? new Text() : new Text(service)); + // Prefer timeline service address over service coming in the token for + // updating the token service. + InetSocketAddress serviceAddr = + (collectorAddr != null && !collectorAddr.isEmpty()) ? + NetUtils.createSocketAddr(collectorAddr) : + SecurityUtil.getTokenServiceAddr(timelineToken); + SecurityUtil.setTokenService(timelineToken, serviceAddr); + authUgi.addToken(timelineToken); + LOG.info("Updated timeline delegation token " + timelineToken); + } + + @Private + protected void putObjects(String path, MultivaluedMap params, + Object obj) throws IOException, YarnException { + + int retries = verifyRestEndPointAvailable(); + + // timelineServiceAddress could be stale, add retry logic here. + boolean needRetry = true; + while (needRetry) { + try { + URI uri = TimelineConnector.constructResURI(getConfig(), + timelineServiceAddress, RESOURCE_URI_STR_V2); + putObjects(uri, path, params, obj); + needRetry = false; + } catch (IOException e) { + // handle exception for timelineServiceAddress being updated. + checkRetryWithSleep(retries, e); + retries--; + } + } + } + + /** + * Check if reaching to maximum of retries. + * + * @param retries + * @param e + */ + private void checkRetryWithSleep(int retries, IOException e) + throws YarnException, IOException { + if (retries > 0) { + try { + Thread.sleep(this.serviceRetryInterval); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + throw new YarnException("Interrupted while retrying to connect to ATS"); + } + } else { + StringBuilder msg = + new StringBuilder("TimelineClient has reached to max retry times : "); + msg.append(this.maxServiceRetries); + msg.append(" for service address: "); + msg.append(timelineServiceAddress); + LOG.error(msg.toString()); + throw new IOException(msg.toString(), e); + } + } + + private ClientResponse doPutObjects(URI base, String path, + MultivaluedMap params, Object obj) { + return connector.getClient().resource(base).path(path).queryParams(params) + .accept(MediaType.APPLICATION_JSON).type(MediaType.APPLICATION_JSON) + .put(ClientResponse.class, obj); + } + + protected void putObjects(final URI base, final String path, + final MultivaluedMap params, final Object obj) + throws IOException, YarnException { + ClientResponse resp = null; + try { + resp = authUgi.doAs(new PrivilegedExceptionAction() { + @Override + public ClientResponse run() throws Exception { + return doPutObjects(base, path, params, obj); + } + }); + } catch (UndeclaredThrowableException ue) { + Throwable cause = ue.getCause(); + if (cause instanceof IOException) { + throw (IOException)cause; + } else { + throw new IOException(cause); + } + } catch (InterruptedException ie) { + throw (IOException) new InterruptedIOException().initCause(ie); + } + if (resp == null || + resp.getClientResponseStatus() != ClientResponse.Status.OK) { + String msg = + "Response from the timeline server is " + ((resp == null) ? "null" + : "not successful," + " HTTP error code: " + resp.getStatus() + + ", Server response:\n" + resp.getEntity(String.class)); + LOG.error(msg); + throw new YarnException(msg); + } + } + + 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 Auxiliary Service is configured in all the NMs"; + LOG.error(errMessage); + throw new YarnException(errMessage); + } + return retries; + } + + /** + * Poll TimelineServiceAddress for maximum of retries times if it is null. + * + * @param retries + * @return the left retry times + * @throws IOException + */ + private int pollTimelineServiceAddress(int retries) throws YarnException { + while (timelineServiceAddress == null && retries > 0) { + try { + Thread.sleep(this.serviceRetryInterval); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new YarnException("Interrupted while trying to connect ATS"); + } + retries--; + } + return retries; + } + + private final class EntitiesHolder extends FutureTask { + private final TimelineEntities entities; + private final boolean isSync; + + EntitiesHolder(final TimelineEntities entities, final boolean isSync) { + super(new Callable() { + // publishEntities() + public Void call() throws Exception { + MultivaluedMap params = new MultivaluedMapImpl(); + params.add("appid", getContextAppId().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 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 final long drainTimeoutPeriod; + + private int numberOfAsyncsToMerge; + private final BlockingQueue timelineEntityQueue; + private ExecutorService executor; + + TimelineEntityDispatcher(Configuration conf) { + timelineEntityQueue = new LinkedBlockingQueue(); + numberOfAsyncsToMerge = + conf.getInt(YarnConfiguration.NUMBER_OF_ASYNC_ENTITIES_TO_MERGE, + YarnConfiguration.DEFAULT_NUMBER_OF_ASYNC_ENTITIES_TO_MERGE); + drainTimeoutPeriod = conf.getLong( + YarnConfiguration.TIMELINE_V2_CLIENT_DRAIN_TIME_MILLIS, + YarnConfiguration.DEFAULT_TIMELINE_V2_CLIENT_DRAIN_TIME_MILLIS); + } + + 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() + drainTimeoutPeriod; + 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, + 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 + TimelineEntities entities = new TimelineEntities(); + for (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(drainTimeoutPeriod, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + e.printStackTrace(); + } + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtils.java index a794e9714c1..b618ac16c43 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtils.java @@ -27,6 +27,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.util.VersionInfo; +import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.timeline.TimelineAbout; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.util.YarnVersionInfo; @@ -43,6 +44,13 @@ import org.codehaus.jackson.map.ObjectMapper; @Evolving public class TimelineUtils { + public static final String FLOW_NAME_TAG_PREFIX = "TIMELINE_FLOW_NAME_TAG"; + public static final String FLOW_VERSION_TAG_PREFIX = + "TIMELINE_FLOW_VERSION_TAG"; + public static final String FLOW_RUN_ID_TAG_PREFIX = + "TIMELINE_FLOW_RUN_ID_TAG"; + public final static String DEFAULT_FLOW_VERSION = "1"; + private static ObjectMapper mapper; static { @@ -154,4 +162,45 @@ public class TimelineUtils { getTimelineTokenServiceAddress(conf); return SecurityUtil.buildTokenService(timelineServiceAddr); } + + public static String generateDefaultFlowName(String appName, + ApplicationId appId) { + return (appName != null && + !appName.equals(YarnConfiguration.DEFAULT_APPLICATION_NAME)) ? + appName : + "flow_" + appId.getClusterTimestamp() + "_" + appId.getId(); + } + + /** + * Generate flow name tag. + * + * @param flowName flow name that identifies a distinct flow application which + * can be run repeatedly over time + * @return flow name tag. + */ + public static String generateFlowNameTag(String flowName) { + return FLOW_NAME_TAG_PREFIX + ":" + flowName; + } + + /** + * Generate flow version tag. + * + * @param flowVersion flow version that keeps track of the changes made to the + * flow + * @return flow version tag. + */ + public static String generateFlowVersionTag(String flowVersion) { + return FLOW_VERSION_TAG_PREFIX + ":" + flowVersion; + } + + /** + * Generate flow run ID tag. + * + * @param flowRunId flow run ID that identifies one instance (or specific + * execution) of that flow + * @return flow run id tag. + */ + public static String generateFlowRunIdTag(long flowRunId) { + return FLOW_RUN_ID_TAG_PREFIX + ":" + flowRunId; + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java index d62a8101cbf..52206ded48d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java @@ -252,7 +252,7 @@ public class WebAppUtils { return getResolvedAddress(address); } - private static String getResolvedAddress(InetSocketAddress address) { + public static String getResolvedAddress(InetSocketAddress address) { address = NetUtils.getConnectAddress(address); StringBuilder sb = new StringBuilder(); InetAddress resolved = address.getAddress(); @@ -314,6 +314,10 @@ public class WebAppUtils { } public static String getAHSWebAppURLWithoutScheme(Configuration conf) { + return getTimelineReaderWebAppURL(conf); + } + + public static String getTimelineReaderWebAppURL(Configuration conf) { if (YarnConfiguration.useHttps(conf)) { return conf.get(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS, YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS); @@ -322,7 +326,7 @@ public class WebAppUtils { YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS); } } - + /** * if url has scheme then it will be returned as it is else it will return * url with scheme. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/timeline/.keep b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/timeline/.keep new file mode 100644 index 00000000000..e69de29bb2d diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml index 97de74946ed..14acebde773 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml @@ -790,11 +790,29 @@ The setting that controls whether yarn system metrics is - published on the timeline server or not by RM. + published to the Timeline server (version one) or not, by RM. + This configuration is now deprecated in favor of + yarn.system-metrics-publisher.enabled. yarn.resourcemanager.system-metrics-publisher.enabled false + + The setting that controls whether yarn system metrics is + published on the Timeline service or not by RM And NM. + yarn.system-metrics-publisher.enabled + false + + + + The setting that controls whether yarn container events are + published to the timeline service or not by RM. This configuration setting + is for ATS V2. + yarn.rm.system-metrics-publisher.emit-container-events + false + + + Number of worker threads that send the yarn system metrics data. @@ -978,6 +996,12 @@ 20 + + Number of threads collector service uses. + yarn.nodemanager.collector-service.thread-count + 5 + + Number of threads used in cleanup. yarn.nodemanager.delete.thread-count @@ -1053,6 +1077,13 @@ ${yarn.nodemanager.hostname}:8040 + + + Address where the collector service IPC is. + yarn.nodemanager.collector-service.address + ${yarn.nodemanager.hostname}:8048 + + Interval in between cache cleanups. yarn.nodemanager.localizer.cache.cleanup.interval-ms @@ -1937,7 +1968,7 @@ Indicate what is the current version of the running timeline service. For example, if "yarn.timeline-service.version" is 1.5, and "yarn.timeline-service.enabled" is true, it means the cluster will and - should bring up the timeline service v.1.5. + should bring up the timeline service v.1.5 (and nothing else). On the client side, if the client uses the same version of timeline service, it should succeed. If the client chooses to use a smaller version in spite of this, then depending on how robust the compatibility story is between versions, @@ -2121,6 +2152,15 @@ 1000 + + + The time period for which timeline v2 client will wait for draining + leftover entities after stop. + + yarn.timeline-service.client.drain-entities.timeout.ms + 2000 + + Enable timeline server to recover state after starting. If true, then yarn.timeline-service.state-store-class must be specified. @@ -2261,6 +2301,23 @@ 300 + + + yarn.timeline-service.writer.class + + Storage implementation ATS v2 will use for the TimelineWriter service. + + org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl + + + + yarn.timeline-service.reader.class + + Storage implementation ATS v2 will use for the TimelineReader service. + + org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineReaderImpl + + yarn.timeline-service.client.internal-timers-ttl-secs @@ -2271,6 +2328,74 @@ 420 + + The setting that controls how often the timeline collector + flushes the timeline writer. + yarn.timeline-service.writer.flush-interval-seconds + 60 + + + + Time period till which the application collector will be alive + in NM, after the application master container finishes. + yarn.timeline-service.app-collector.linger-period.ms + 1000 + + + + Time line V2 client tries to merge these many number of + async entities (if available) and then call the REST ATS V2 API to submit. + + yarn.timeline-service.timeline-client.number-of-async-entities-to-merge + 10 + + + + + The setting that controls how long the final value + of a metric of a completed app is retained before merging into + the flow sum. Up to this time after an application is completed + out-of-order values that arrive can be recognized and discarded at the + cost of increased storage. + + yarn.timeline-service.hbase.coprocessor.app-final-value-retention-milliseconds + + 259200000 + + + + + The default hdfs location for flowrun coprocessor jar. + + yarn.timeline-service.hbase.coprocessor.jar.hdfs.location + + /hbase/coprocessor/hadoop-yarn-server-timelineservice.jar + + + + + The value of this parameter sets the prefix for all tables that are part of + timeline service in the hbase storage schema. It can be set to "dev." + or "staging." if it is to be used for development or staging instances. + This way the data in production tables stays in a separate set of tables + prefixed by "prod.". + + yarn.timeline-service.hbase-schema.prefix + prod. + + + + Optional URL to an hbase-site.xml configuration file to be + used to connect to the timeline-service hbase cluster. If empty or not + specified, then the HBase configuration will be loaded from the classpath. + When specified the values in the specified configuration file will override + those from the ones that are present on the classpath. + + yarn.timeline-service.hbase.configuration.file + + + + @@ -3094,6 +3219,17 @@ 64 + + + Flag to enable cross-origin (CORS) support for timeline service v1.x or + Timeline Reader in timeline service v2. For timeline service v2, also add + org.apache.hadoop.security.HttpCrossOriginFilterInitializer to the + configuration hadoop.http.filter.initializers in core-site.xml. + + yarn.timeline-service.http-cross-origin.enabled + false + + Comma-separated list of PlacementRules to determine how applications diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java index 88bd104b090..dfe75349748 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java @@ -29,6 +29,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.ipc.Server; import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.protocolrecords.CommitResponse; @@ -118,7 +119,7 @@ public class TestContainerLaunchRPC { resource, System.currentTimeMillis() + 10000, 42, 42, Priority.newInstance(0), 0); Token containerToken = - TestRPC.newContainerToken(nodeId, "password".getBytes(), + newContainerToken(nodeId, "password".getBytes(), containerTokenIdentifier); StartContainerRequest scRequest = @@ -144,6 +145,19 @@ public class TestContainerLaunchRPC { Assert.fail("timeout exception should have occurred!"); } + public static Token newContainerToken(NodeId nodeId, byte[] password, + ContainerTokenIdentifier tokenIdentifier) { + // RPC layer client expects ip:port as service for tokens + InetSocketAddress addr = + NetUtils.createSocketAddrForHost(nodeId.getHost(), nodeId.getPort()); + // NOTE: use SecurityUtil.setTokenService if this becomes a "real" token + Token containerToken = + Token.newInstance(tokenIdentifier.getBytes(), + ContainerTokenIdentifier.KIND.toString(), password, SecurityUtil + .buildTokenService(addr).toString()); + return containerToken; + } + public class DummyContainerManager implements ContainerManagementProtocol { private ContainerStatus status = null; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.java index fb7d388aa9b..6e9728475e4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.java @@ -23,6 +23,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.ipc.Server; import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.protocolrecords.CommitResponse; @@ -107,7 +108,7 @@ public class TestContainerResourceIncreaseRPC { resource, System.currentTimeMillis() + 10000, 42, 42, Priority.newInstance(0), 0); Token containerToken = - TestRPC.newContainerToken(nodeId, "password".getBytes(), + newContainerToken(nodeId, "password".getBytes(), containerTokenIdentifier); // Construct container resource increase request, List increaseTokens = new ArrayList<>(); @@ -130,6 +131,19 @@ public class TestContainerResourceIncreaseRPC { Assert.fail("timeout exception should have occurred!"); } + public static Token newContainerToken(NodeId nodeId, byte[] password, + ContainerTokenIdentifier tokenIdentifier) { + // RPC layer client expects ip:port as service for tokens + InetSocketAddress addr = + NetUtils.createSocketAddrForHost(nodeId.getHost(), nodeId.getPort()); + // NOTE: use SecurityUtil.setTokenService if this becomes a "real" token + Token containerToken = + Token.newInstance(tokenIdentifier.getBytes(), + ContainerTokenIdentifier.KIND.toString(), password, SecurityUtil + .buildTokenService(addr).toString()); + return containerToken; + } + public class DummyContainerManager implements ContainerManagementProtocol { @Override diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java index d76fc6f5c2b..1d3bf034c32 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java @@ -102,6 +102,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersRequestP import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersResponsePBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationRequestPBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationResponsePBImpl; +import org.apache.hadoop.yarn.api.records.CollectorInfo; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport; import org.apache.hadoop.yarn.api.records.ApplicationId; @@ -409,6 +410,7 @@ public class TestPBImplRecords extends BasePBImplRecordsTest { generateByNewInstance(ApplicationTimeout.class); generateByNewInstance(ContainerResourceIncreaseRequest.class); generateByNewInstance(QueueConfigurations.class); + generateByNewInstance(CollectorInfo.class); } @Test diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestTimelineServiceRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestTimelineServiceRecords.java new file mode 100644 index 00000000000..04bf4294058 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestTimelineServiceRecords.java @@ -0,0 +1,314 @@ +/* + * 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.records.timelineservice; + + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +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.exceptions.YarnRuntimeException; +import org.apache.hadoop.yarn.util.timeline.TimelineUtils; +import org.junit.Test; +import org.junit.Assert; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; + +/** + * Tests timeline service API records. + */ +public class TestTimelineServiceRecords { + private static final Log LOG = + LogFactory.getLog(TestTimelineServiceRecords.class); + + @Test + public void testTimelineEntities() throws Exception { + TimelineEntity entity = new TimelineEntity(); + entity.setType("test type 1"); + entity.setId("test id 1"); + entity.addInfo("test info key 1", "test info value 1"); + entity.addInfo("test info key 2", + Arrays.asList("test info value 2", "test info value 3")); + entity.addInfo("test info key 3", true); + Assert.assertTrue( + entity.getInfo().get("test info key 3") instanceof Boolean); + entity.addConfig("test config key 1", "test config value 1"); + entity.addConfig("test config key 2", "test config value 2"); + + TimelineMetric metric1 = + new TimelineMetric(TimelineMetric.Type.TIME_SERIES); + metric1.setId("test metric id 1"); + metric1.addValue(1L, 1.0F); + metric1.addValue(3L, 3.0D); + metric1.addValue(2L, 2); + Assert.assertEquals(TimelineMetric.Type.TIME_SERIES, metric1.getType()); + Iterator> itr = + metric1.getValues().entrySet().iterator(); + Map.Entry entry = itr.next(); + Assert.assertEquals(new Long(3L), entry.getKey()); + Assert.assertEquals(3.0D, entry.getValue()); + entry = itr.next(); + Assert.assertEquals(new Long(2L), entry.getKey()); + Assert.assertEquals(2, entry.getValue()); + entry = itr.next(); + Assert.assertEquals(new Long(1L), entry.getKey()); + Assert.assertEquals(1.0F, entry.getValue()); + Assert.assertFalse(itr.hasNext()); + entity.addMetric(metric1); + + TimelineMetric metric2 = + new TimelineMetric(TimelineMetric.Type.SINGLE_VALUE); + metric2.setId("test metric id 1"); + metric2.addValue(3L, (short) 3); + Assert.assertEquals(TimelineMetric.Type.SINGLE_VALUE, metric2.getType()); + Assert.assertTrue( + metric2.getValues().values().iterator().next() instanceof Short); + Map points = new HashMap<>(); + points.put(4L, 4.0D); + points.put(5L, 5.0D); + try { + metric2.setValues(points); + Assert.fail(); + } catch (IllegalArgumentException e) { + Assert.assertTrue(e.getMessage().contains( + "Values cannot contain more than one point in")); + } + try { + metric2.addValues(points); + Assert.fail(); + } catch (IllegalArgumentException e) { + Assert.assertTrue(e.getMessage().contains( + "Values cannot contain more than one point in")); + } + entity.addMetric(metric2); + + TimelineMetric metric3 = + new TimelineMetric(TimelineMetric.Type.SINGLE_VALUE); + metric3.setId("test metric id 1"); + metric3.addValue(4L, (short) 4); + Assert.assertEquals("metric3 should equal to metric2! ", metric3, metric2); + Assert.assertNotEquals("metric1 should not equal to metric2! ", + metric1, metric2); + + TimelineEvent event1 = new TimelineEvent(); + event1.setId("test event id 1"); + event1.addInfo("test info key 1", "test info value 1"); + event1.addInfo("test info key 2", + Arrays.asList("test info value 2", "test info value 3")); + event1.addInfo("test info key 3", true); + Assert.assertTrue( + event1.getInfo().get("test info key 3") instanceof Boolean); + event1.setTimestamp(1L); + entity.addEvent(event1); + + TimelineEvent event2 = new TimelineEvent(); + event2.setId("test event id 2"); + event2.addInfo("test info key 1", "test info value 1"); + event2.addInfo("test info key 2", + Arrays.asList("test info value 2", "test info value 3")); + event2.addInfo("test info key 3", true); + Assert.assertTrue( + event2.getInfo().get("test info key 3") instanceof Boolean); + event2.setTimestamp(2L); + entity.addEvent(event2); + + Assert.assertFalse("event1 should not equal to event2! ", + event1.equals(event2)); + TimelineEvent event3 = new TimelineEvent(); + event3.setId("test event id 1"); + event3.setTimestamp(1L); + Assert.assertEquals("event1 should equal to event3! ", event3, event1); + Assert.assertNotEquals("event1 should not equal to event2! ", + event1, event2); + + entity.setCreatedTime(0L); + entity.addRelatesToEntity("test type 2", "test id 2"); + entity.addRelatesToEntity("test type 3", "test id 3"); + entity.addIsRelatedToEntity("test type 4", "test id 4"); + entity.addIsRelatedToEntity("test type 5", "test id 5"); + LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(entity, true)); + + TimelineEntities entities = new TimelineEntities(); + TimelineEntity entity1 = new TimelineEntity(); + entities.addEntity(entity1); + TimelineEntity entity2 = new TimelineEntity(); + entities.addEntity(entity2); + LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(entities, true)); + + Assert.assertFalse("entity 1 should not be valid without type and id", + entity1.isValid()); + entity1.setId("test id 2"); + entity1.setType("test type 2"); + entity2.setId("test id 1"); + entity2.setType("test type 1"); + + Assert.assertEquals("Timeline entity should equal to entity2! ", + entity, entity2); + Assert.assertNotEquals("entity1 should not equal to entity! ", + entity1, entity); + Assert.assertEquals("entity should be less than entity1! ", + entity1.compareTo(entity), 1); + Assert.assertEquals("entity's hash code should be -28727840 but not " + + entity.hashCode(), entity.hashCode(), -28727840); + } + + @Test + public void testFirstClassCitizenEntities() throws Exception { + UserEntity user = new UserEntity(); + user.setId("test user id"); + + QueueEntity queue = new QueueEntity(); + queue.setId("test queue id"); + + + ClusterEntity cluster = new ClusterEntity(); + cluster.setId("test cluster id"); + + FlowRunEntity flow1 = new FlowRunEntity(); + //flow1.setId("test flow id 1"); + flow1.setUser(user.getId()); + flow1.setName("test flow name 1"); + flow1.setVersion("test flow version 1"); + flow1.setRunId(1L); + + FlowRunEntity flow2 = new FlowRunEntity(); + //flow2.setId("test flow run id 2"); + flow2.setUser(user.getId()); + flow2.setName("test flow name 2"); + flow2.setVersion("test flow version 2"); + flow2.setRunId(2L); + + ApplicationEntity app1 = new ApplicationEntity(); + app1.setId(ApplicationId.newInstance(0, 1).toString()); + app1.setQueue(queue.getId()); + + ApplicationEntity app2 = new ApplicationEntity(); + app2.setId(ApplicationId.newInstance(0, 2).toString()); + app2.setQueue(queue.getId()); + + ApplicationAttemptEntity appAttempt = new ApplicationAttemptEntity(); + appAttempt.setId(ApplicationAttemptId.newInstance( + ApplicationId.newInstance(0, 1), 1).toString()); + + ContainerEntity container = new ContainerEntity(); + container.setId(ContainerId.newContainerId( + ApplicationAttemptId.newInstance( + ApplicationId.newInstance(0, 1), 1), 1).toString()); + + cluster.addChild(TimelineEntityType.YARN_FLOW_RUN.toString(), + flow1.getId()); + flow1 + .setParent(TimelineEntityType.YARN_CLUSTER.toString(), cluster.getId()); + flow1.addChild(TimelineEntityType.YARN_FLOW_RUN.toString(), flow2.getId()); + flow2.setParent(TimelineEntityType.YARN_FLOW_RUN.toString(), flow1.getId()); + flow2.addChild(TimelineEntityType.YARN_APPLICATION.toString(), + app1.getId()); + flow2.addChild(TimelineEntityType.YARN_APPLICATION.toString(), + app2.getId()); + app1.setParent(TimelineEntityType.YARN_FLOW_RUN.toString(), flow2.getId()); + app1.addChild(TimelineEntityType.YARN_APPLICATION_ATTEMPT.toString(), + appAttempt.getId()); + appAttempt + .setParent(TimelineEntityType.YARN_APPLICATION.toString(), + app1.getId()); + app2.setParent(TimelineEntityType.YARN_FLOW_RUN.toString(), flow2.getId()); + appAttempt.addChild(TimelineEntityType.YARN_CONTAINER.toString(), + container.getId()); + container.setParent(TimelineEntityType.YARN_APPLICATION_ATTEMPT.toString(), + appAttempt.getId()); + + LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(cluster, true)); + LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(flow1, true)); + LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(flow2, true)); + LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(app1, true)); + LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(app2, true)); + LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(appAttempt, true)); + LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(container, true)); + + + // Check parent/children APIs + Assert.assertNotNull(app1.getParent()); + Assert.assertEquals(flow2.getType(), app1.getParent().getType()); + Assert.assertEquals(flow2.getId(), app1.getParent().getId()); + app1.addInfo(ApplicationEntity.PARENT_INFO_KEY, "invalid parent object"); + try { + app1.getParent(); + Assert.fail(); + } catch (Exception e) { + Assert.assertTrue(e instanceof YarnRuntimeException); + Assert.assertTrue(e.getMessage().contains( + "Parent info is invalid identifier object")); + } + + Assert.assertNotNull(app1.getChildren()); + Assert.assertEquals(1, app1.getChildren().size()); + Assert.assertEquals( + appAttempt.getType(), app1.getChildren().iterator().next().getType()); + Assert.assertEquals( + appAttempt.getId(), app1.getChildren().iterator().next().getId()); + app1.addInfo(ApplicationEntity.CHILDREN_INFO_KEY, + Collections.singletonList("invalid children set")); + try { + app1.getChildren(); + Assert.fail(); + } catch (Exception e) { + Assert.assertTrue(e instanceof YarnRuntimeException); + Assert.assertTrue(e.getMessage().contains( + "Children info is invalid identifier set")); + } + app1.addInfo(ApplicationEntity.CHILDREN_INFO_KEY, + Collections.singleton("invalid child object")); + try { + app1.getChildren(); + Assert.fail(); + } catch (Exception e) { + Assert.assertTrue(e instanceof YarnRuntimeException); + Assert.assertTrue(e.getMessage().contains( + "Children info contains invalid identifier object")); + } + } + + @Test + public void testUser() throws Exception { + UserEntity user = new UserEntity(); + user.setId("test user id"); + user.addInfo("test info key 1", "test info value 1"); + user.addInfo("test info key 2", "test info value 2"); + LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(user, true)); + } + + @Test + public void testQueue() throws Exception { + QueueEntity queue = new QueueEntity(); + queue.setId("test queue id"); + queue.addInfo("test info key 1", "test info value 1"); + queue.addInfo("test info key 2", "test info value 2"); + queue.setParent(TimelineEntityType.YARN_QUEUE.toString(), + "test parent queue id"); + queue.addChild(TimelineEntityType.YARN_QUEUE.toString(), + "test child queue id 1"); + queue.addChild(TimelineEntityType.YARN_QUEUE.toString(), + "test child queue id 2"); + LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(queue, true)); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java index a1d44494026..787607496be 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java @@ -34,6 +34,7 @@ import java.security.PrivilegedExceptionAction; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.http.HttpConfig.Policy; import org.apache.hadoop.io.Text; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.ssl.KeyStoreTestUtil; @@ -215,11 +216,11 @@ public class TestTimelineClient { + "Timeline server should be off to run this test. "); } catch (RuntimeException ce) { Assert.assertTrue( - "Handler exception for reason other than retry: " + ce.getMessage(), - ce.getMessage().contains("Connection retries limit exceeded")); + "Handler exception for reason other than retry: " + ce.getMessage(), + ce.getMessage().contains("Connection retries limit exceeded")); // we would expect this exception here, check if the client has retried - Assert.assertTrue("Retry filter didn't perform any retries! ", client - .connectionRetry.getRetired()); + Assert.assertTrue("Retry filter didn't perform any retries! ", + client.connector.connectionRetry.getRetired()); } } @@ -318,7 +319,7 @@ public class TestTimelineClient { .getMessage().contains("Connection retries limit exceeded")); // we would expect this exception here, check if the client has retried Assert.assertTrue("Retry filter didn't perform any retries! ", - client.connectionRetry.getRetired()); + client.connector.connectionRetry.getRetired()); } public static ClientResponse mockEntityClientResponse( @@ -419,17 +420,25 @@ public class TestTimelineClient { private TimelineClientImpl createTimelineClientFakeTimelineClientRetryOp( YarnConfiguration conf) { TimelineClientImpl client = new TimelineClientImpl() { - @Override - public TimelineClientRetryOp - createTimelineClientRetryOpForOperateDelegationToken( - final PrivilegedExceptionAction action) throws IOException { - TimelineClientRetryOpForOperateDelegationToken op = - spy(new TimelineClientRetryOpForOperateDelegationToken( - UserGroupInformation.getCurrentUser(), action)); - doThrow(new SocketTimeoutException("Test socketTimeoutException")) - .when(op).run(); - return op; + protected TimelineConnector createTimelineConnector() { + TimelineConnector connector = + new TimelineConnector(true, authUgi, doAsUser, token) { + @Override + public TimelineClientRetryOp + createRetryOpForOperateDelegationToken( + final PrivilegedExceptionAction action) + throws IOException { + TimelineClientRetryOpForOperateDelegationToken op = + spy(new TimelineClientRetryOpForOperateDelegationToken( + UserGroupInformation.getCurrentUser(), action)); + doThrow(new SocketTimeoutException( + "Test socketTimeoutException")).when(op).run(); + return op; + } + }; + addIfService(connector); + return connector; } }; client.init(conf); @@ -442,6 +451,7 @@ public class TestTimelineClient { YarnConfiguration conf = new YarnConfiguration(); conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); conf.setInt(YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES, 0); + conf.set(YarnConfiguration.YARN_HTTP_POLICY_KEY, Policy.HTTPS_ONLY.name()); File testDir = TestGenericTestUtils.getTestDir(); String sslConfDir = diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientV2Impl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientV2Impl.java new file mode 100644 index 00000000000..b1e490de9d0 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientV2Impl.java @@ -0,0 +1,435 @@ +/** + * 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.client.api.impl; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +import java.io.IOException; +import java.net.URI; +import java.util.ArrayList; +import java.util.List; + +import javax.ws.rs.core.MultivaluedMap; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.CollectorInfo; +import org.apache.hadoop.yarn.api.records.Token; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestName; + +/** + * Tests {@link TimelineV2ClientImpl}. + */ +public class TestTimelineClientV2Impl { + private static final Log LOG = + LogFactory.getLog(TestTimelineClientV2Impl.class); + private TestV2TimelineClient client; + private static final long TIME_TO_SLEEP = 150L; + private static final String EXCEPTION_MSG = "Exception in the content"; + + @Before + public void setup() { + conf = new YarnConfiguration(); + conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); + conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f); + conf.setInt(YarnConfiguration.NUMBER_OF_ASYNC_ENTITIES_TO_MERGE, 3); + if (!currTestName.getMethodName() + .contains("testRetryOnConnectionFailure")) { + client = createTimelineClient(conf); + } + } + + @Rule + public TestName currTestName = new TestName(); + private YarnConfiguration conf; + + private TestV2TimelineClient createTimelineClient(YarnConfiguration config) { + ApplicationId id = ApplicationId.newInstance(0, 0); + TestV2TimelineClient tc = new TestV2TimelineClient(id); + tc.init(config); + tc.start(); + return tc; + } + + private class TestV2TimelineClientForExceptionHandling + extends TimelineV2ClientImpl { + public TestV2TimelineClientForExceptionHandling(ApplicationId id) { + super(id); + } + + private boolean throwYarnException; + + public void setThrowYarnException(boolean throwYarnException) { + this.throwYarnException = throwYarnException; + } + + public boolean isThrowYarnException() { + return throwYarnException; + } + + @Override + protected void putObjects(URI base, String path, + MultivaluedMap params, Object obj) + throws IOException, YarnException { + if (throwYarnException) { + throw new YarnException(EXCEPTION_MSG); + } else { + throw new IOException( + "Failed to get the response from the timeline server."); + } + } + } + + private class TestV2TimelineClient + extends TestV2TimelineClientForExceptionHandling { + private boolean sleepBeforeReturn; + + private List publishedEntities; + + public TimelineEntities getPublishedEntities(int putIndex) { + Assert.assertTrue("Not So many entities Published", + putIndex < publishedEntities.size()); + return publishedEntities.get(putIndex); + } + + public void setSleepBeforeReturn(boolean sleepBeforeReturn) { + this.sleepBeforeReturn = sleepBeforeReturn; + } + + public int getNumOfTimelineEntitiesPublished() { + return publishedEntities.size(); + } + + public TestV2TimelineClient(ApplicationId id) { + super(id); + publishedEntities = new ArrayList(); + } + + protected void putObjects(String path, + MultivaluedMap params, Object obj) + throws IOException, YarnException { + if (isThrowYarnException()) { + throw new YarnException("ActualException"); + } + publishedEntities.add((TimelineEntities) obj); + if (sleepBeforeReturn) { + try { + Thread.sleep(TIME_TO_SLEEP); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } + } + + @Test + public void testExceptionMultipleRetry() { + TestV2TimelineClientForExceptionHandling c = + new TestV2TimelineClientForExceptionHandling( + ApplicationId.newInstance(0, 0)); + int maxRetries = 2; + conf.setInt(YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES, + maxRetries); + c.init(conf); + c.start(); + c.setTimelineCollectorInfo(CollectorInfo.newInstance("localhost:12345")); + try { + c.putEntities(new TimelineEntity()); + } catch (IOException e) { + Assert.fail("YARN exception is expected"); + } catch (YarnException e) { + Throwable cause = e.getCause(); + Assert.assertTrue("IOException is expected", + cause instanceof IOException); + Assert.assertTrue("YARN exception is expected", + cause.getMessage().contains( + "TimelineClient has reached to max retry times : " + maxRetries)); + } + + c.setThrowYarnException(true); + try { + c.putEntities(new TimelineEntity()); + } catch (IOException e) { + Assert.fail("YARN exception is expected"); + } catch (YarnException e) { + Throwable cause = e.getCause(); + Assert.assertTrue("YARN exception is expected", + cause instanceof YarnException); + Assert.assertTrue("YARN exception is expected", + cause.getMessage().contains(EXCEPTION_MSG)); + } + c.stop(); + } + + @Test + public void testPostEntities() throws Exception { + try { + client.putEntities(generateEntity("1")); + } catch (YarnException e) { + Assert.fail("Exception is not expected"); + } + } + + @Test + public void testASyncCallMerge() throws Exception { + client.setSleepBeforeReturn(true); + try { + client.putEntitiesAsync(generateEntity("1")); + Thread.sleep(TIME_TO_SLEEP / 2); + // by the time first put response comes push 2 entities in the queue + client.putEntitiesAsync(generateEntity("2")); + client.putEntitiesAsync(generateEntity("3")); + } catch (YarnException e) { + Assert.fail("Exception is not expected"); + } + for (int i = 0; i < 4; i++) { + if (client.getNumOfTimelineEntitiesPublished() == 2) { + break; + } + Thread.sleep(TIME_TO_SLEEP); + } + Assert.assertEquals("two merged TimelineEntities needs to be published", 2, + client.getNumOfTimelineEntitiesPublished()); + TimelineEntities secondPublishedEntities = client.getPublishedEntities(1); + Assert.assertEquals( + "Merged TimelineEntities Object needs to 2 TimelineEntity Object", 2, + secondPublishedEntities.getEntities().size()); + Assert.assertEquals("Order of Async Events Needs to be FIFO", "2", + secondPublishedEntities.getEntities().get(0).getId()); + Assert.assertEquals("Order of Async Events Needs to be FIFO", "3", + secondPublishedEntities.getEntities().get(1).getId()); + } + + @Test + public void testSyncCall() throws Exception { + try { + // sync entity should not be be merged with Async + client.putEntities(generateEntity("1")); + client.putEntitiesAsync(generateEntity("2")); + client.putEntitiesAsync(generateEntity("3")); + // except for the sync call above 2 should be merged + client.putEntities(generateEntity("4")); + } catch (YarnException e) { + Assert.fail("Exception is not expected"); + } + for (int i = 0; i < 4; i++) { + if (client.getNumOfTimelineEntitiesPublished() == 3) { + break; + } + Thread.sleep(TIME_TO_SLEEP); + } + printReceivedEntities(); + Assert.assertEquals("TimelineEntities not published as desired", 3, + client.getNumOfTimelineEntitiesPublished()); + TimelineEntities firstPublishedEntities = client.getPublishedEntities(0); + Assert.assertEquals("sync entities should not be merged with async", 1, + firstPublishedEntities.getEntities().size()); + + // test before pushing the sync entities asyncs are merged and pushed + TimelineEntities secondPublishedEntities = client.getPublishedEntities(1); + Assert.assertEquals( + "async entities should be merged before publishing sync", 2, + secondPublishedEntities.getEntities().size()); + Assert.assertEquals("Order of Async Events Needs to be FIFO", "2", + secondPublishedEntities.getEntities().get(0).getId()); + Assert.assertEquals("Order of Async Events Needs to be FIFO", "3", + secondPublishedEntities.getEntities().get(1).getId()); + + // test the last entity published is sync put + TimelineEntities thirdPublishedEntities = client.getPublishedEntities(2); + Assert.assertEquals("sync entities had to be published at the last", 1, + thirdPublishedEntities.getEntities().size()); + Assert.assertEquals("Expected last sync Event is not proper", "4", + thirdPublishedEntities.getEntities().get(0).getId()); + } + + @Test + public void testExceptionCalls() throws Exception { + client.setThrowYarnException(true); + try { + client.putEntitiesAsync(generateEntity("1")); + } catch (YarnException e) { + Assert.fail("Async calls are not expected to throw exception"); + } + + try { + client.putEntities(generateEntity("2")); + Assert.fail("Sync calls are expected to throw exception"); + } catch (YarnException e) { + Assert.assertEquals("Same exception needs to be thrown", + "ActualException", e.getCause().getMessage()); + } + } + + @Test + public void testConfigurableNumberOfMerges() throws Exception { + client.setSleepBeforeReturn(true); + try { + // At max 3 entities need to be merged + client.putEntitiesAsync(generateEntity("1")); + client.putEntitiesAsync(generateEntity("2")); + client.putEntitiesAsync(generateEntity("3")); + client.putEntitiesAsync(generateEntity("4")); + client.putEntities(generateEntity("5")); + client.putEntitiesAsync(generateEntity("6")); + client.putEntitiesAsync(generateEntity("7")); + client.putEntitiesAsync(generateEntity("8")); + client.putEntitiesAsync(generateEntity("9")); + client.putEntitiesAsync(generateEntity("10")); + } catch (YarnException e) { + Assert.fail("No exception expected"); + } + // not having the same logic here as it doesn't depend on how many times + // events are published. + Thread.sleep(2 * TIME_TO_SLEEP); + printReceivedEntities(); + for (TimelineEntities publishedEntities : client.publishedEntities) { + Assert.assertTrue( + "Number of entities should not be greater than 3 for each publish," + + " but was " + publishedEntities.getEntities().size(), + publishedEntities.getEntities().size() <= 3); + } + } + + @Test + public void testSetTimelineToken() throws Exception { + UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); + assertEquals(0, ugi.getTokens().size()); + assertNull("Timeline token in v2 client should not be set", + client.currentTimelineToken); + + Token token = Token.newInstance( + new byte[0], "kind", new byte[0], "service"); + client.setTimelineCollectorInfo(CollectorInfo.newInstance(null, token)); + assertNull("Timeline token in v2 client should not be set as token kind " + + "is unexepcted.", client.currentTimelineToken); + assertEquals(0, ugi.getTokens().size()); + + token = Token.newInstance(new byte[0], TimelineDelegationTokenIdentifier. + KIND_NAME.toString(), new byte[0], null); + client.setTimelineCollectorInfo(CollectorInfo.newInstance(null, token)); + assertNull("Timeline token in v2 client should not be set as serice is " + + "not set.", client.currentTimelineToken); + assertEquals(0, ugi.getTokens().size()); + + TimelineDelegationTokenIdentifier ident = + new TimelineDelegationTokenIdentifier(new Text(ugi.getUserName()), + new Text("renewer"), null); + ident.setSequenceNumber(1); + token = Token.newInstance(ident.getBytes(), + TimelineDelegationTokenIdentifier.KIND_NAME.toString(), new byte[0], + "localhost:1234"); + client.setTimelineCollectorInfo(CollectorInfo.newInstance(null, token)); + assertEquals(1, ugi.getTokens().size()); + assertNotNull("Timeline token should be set in v2 client.", + client.currentTimelineToken); + assertEquals(token, client.currentTimelineToken); + + ident.setSequenceNumber(20); + Token newToken = Token.newInstance(ident.getBytes(), + TimelineDelegationTokenIdentifier.KIND_NAME.toString(), new byte[0], + "localhost:1234"); + client.setTimelineCollectorInfo(CollectorInfo.newInstance(null, newToken)); + assertEquals(1, ugi.getTokens().size()); + assertNotEquals(token, client.currentTimelineToken); + assertEquals(newToken, client.currentTimelineToken); + } + + @Test + public void testAfterStop() throws Exception { + client.setSleepBeforeReturn(true); + try { + // At max 3 entities need to be merged + client.putEntities(generateEntity("1")); + for (int i = 2; i < 20; i++) { + client.putEntitiesAsync(generateEntity("" + i)); + } + client.stop(); + try { + client.putEntitiesAsync(generateEntity("50")); + Assert.fail("Exception expected"); + } catch (YarnException e) { + // expected + } + } catch (YarnException e) { + Assert.fail("No exception expected"); + } + // not having the same logic here as it doesn't depend on how many times + // events are published. + for (int i = 0; i < 5; i++) { + TimelineEntities publishedEntities = + client.publishedEntities.get(client.publishedEntities.size() - 1); + TimelineEntity timelineEntity = publishedEntities.getEntities() + .get(publishedEntities.getEntities().size() - 1); + if (!timelineEntity.getId().equals("19")) { + Thread.sleep(2 * TIME_TO_SLEEP); + } + } + printReceivedEntities(); + TimelineEntities publishedEntities = + client.publishedEntities.get(client.publishedEntities.size() - 1); + TimelineEntity timelineEntity = publishedEntities.getEntities() + .get(publishedEntities.getEntities().size() - 1); + Assert.assertEquals("", "19", timelineEntity.getId()); + } + + private void printReceivedEntities() { + for (int i = 0; i < client.getNumOfTimelineEntitiesPublished(); i++) { + TimelineEntities publishedEntities = client.getPublishedEntities(i); + StringBuilder entitiesPerPublish = new StringBuilder(); + for (TimelineEntity entity : publishedEntities.getEntities()) { + entitiesPerPublish.append(entity.getId()); + entitiesPerPublish.append(","); + } + LOG.info("Entities Published @ index " + i + " : " + + entitiesPerPublish.toString()); + } + } + + private static TimelineEntity generateEntity(String id) { + TimelineEntity entity = new TimelineEntity(); + entity.setId(id); + entity.setType("testEntity"); + entity.setCreatedTime(System.currentTimeMillis()); + return entity; + } + + @After + public void tearDown() { + if (client != null) { + client.stop(); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestTimelineServiceHelper.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestTimelineServiceHelper.java new file mode 100644 index 00000000000..7dffc20a2f0 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestTimelineServiceHelper.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.util; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; + +import org.junit.Assert; +import org.junit.Test; + +/** + * Tests utility methods in {@link TimelineServiceHelper}. + */ +public class TestTimelineServiceHelper { + + @Test + public void testMapCastToHashMap() { + + // Test null map be casted to null + Map nullMap = null; + Assert.assertNull(TimelineServiceHelper.mapCastToHashMap(nullMap)); + + // Test empty hashmap be casted to a empty hashmap + Map emptyHashMap = new HashMap(); + Assert.assertEquals( + TimelineServiceHelper.mapCastToHashMap(emptyHashMap).size(), 0); + + // Test empty non-hashmap be casted to a empty hashmap + Map emptyTreeMap = new TreeMap(); + Assert.assertEquals( + TimelineServiceHelper.mapCastToHashMap(emptyTreeMap).size(), 0); + + // Test non-empty hashmap be casted to hashmap correctly + Map firstHashMap = new HashMap(); + String key = "KEY"; + String value = "VALUE"; + firstHashMap.put(key, value); + Assert.assertEquals( + TimelineServiceHelper.mapCastToHashMap(firstHashMap), firstHashMap); + + // Test non-empty non-hashmap is casted correctly. + Map firstTreeMap = new TreeMap(); + firstTreeMap.put(key, value); + HashMap alternateHashMap = + TimelineServiceHelper.mapCastToHashMap(firstTreeMap); + Assert.assertEquals(firstTreeMap.size(), alternateHashMap.size()); + Assert.assertEquals(alternateHashMap.get(key), value); + + // Test complicated hashmap be casted correctly + Map> complicatedHashMap = + new HashMap>(); + Set hashSet = new HashSet(); + hashSet.add(value); + complicatedHashMap.put(key, hashSet); + Assert.assertEquals( + TimelineServiceHelper.mapCastToHashMap(complicatedHashMap), + complicatedHashMap); + + // Test complicated non-hashmap get casted correctly + Map> complicatedTreeMap = + new TreeMap>(); + complicatedTreeMap.put(key, hashSet); + Assert.assertEquals( + TimelineServiceHelper.mapCastToHashMap(complicatedTreeMap).get(key), + hashSet); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java index e2dc898097c..b4c91f9c979 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java @@ -478,21 +478,21 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService if (eventInfo == null) { continue; } - if (eventInfo.containsKey(AppAttemptMetricsConstants.HOST_EVENT_INFO)) { + if (eventInfo.containsKey(AppAttemptMetricsConstants.HOST_INFO)) { host = - eventInfo.get(AppAttemptMetricsConstants.HOST_EVENT_INFO) + eventInfo.get(AppAttemptMetricsConstants.HOST_INFO) .toString(); } if (eventInfo - .containsKey(AppAttemptMetricsConstants.RPC_PORT_EVENT_INFO)) { + .containsKey(AppAttemptMetricsConstants.RPC_PORT_INFO)) { rpcPort = (Integer) eventInfo.get( - AppAttemptMetricsConstants.RPC_PORT_EVENT_INFO); + AppAttemptMetricsConstants.RPC_PORT_INFO); } if (eventInfo - .containsKey(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO)) { + .containsKey(AppAttemptMetricsConstants.MASTER_CONTAINER_INFO)) { amContainerId = ContainerId.fromString(eventInfo.get( - AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO) + AppAttemptMetricsConstants.MASTER_CONTAINER_INFO) .toString()); } } else if (event.getEventType().equals( @@ -502,39 +502,40 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService continue; } if (eventInfo - .containsKey(AppAttemptMetricsConstants.TRACKING_URL_EVENT_INFO)) { + .containsKey(AppAttemptMetricsConstants.TRACKING_URL_INFO)) { trackingUrl = eventInfo.get( - AppAttemptMetricsConstants.TRACKING_URL_EVENT_INFO) + AppAttemptMetricsConstants.TRACKING_URL_INFO) .toString(); } if (eventInfo - .containsKey(AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_EVENT_INFO)) { + .containsKey( + AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_INFO)) { originalTrackingUrl = eventInfo .get( - AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_EVENT_INFO) + AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_INFO) .toString(); } if (eventInfo - .containsKey(AppAttemptMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO)) { + .containsKey(AppAttemptMetricsConstants.DIAGNOSTICS_INFO)) { diagnosticsInfo = eventInfo.get( - AppAttemptMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO) + AppAttemptMetricsConstants.DIAGNOSTICS_INFO) .toString(); } if (eventInfo - .containsKey(AppAttemptMetricsConstants.STATE_EVENT_INFO)) { + .containsKey(AppAttemptMetricsConstants.STATE_INFO)) { state = YarnApplicationAttemptState.valueOf(eventInfo.get( - AppAttemptMetricsConstants.STATE_EVENT_INFO) + AppAttemptMetricsConstants.STATE_INFO) .toString()); } if (eventInfo - .containsKey(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO)) { + .containsKey(AppAttemptMetricsConstants.MASTER_CONTAINER_INFO)) { amContainerId = ContainerId.fromString(eventInfo.get( - AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO) + AppAttemptMetricsConstants.MASTER_CONTAINER_INFO) .toString()); } } @@ -562,37 +563,37 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService Map entityInfo = entity.getOtherInfo(); if (entityInfo != null) { if (entityInfo - .containsKey(ContainerMetricsConstants.ALLOCATED_MEMORY_ENTITY_INFO)) { + .containsKey(ContainerMetricsConstants.ALLOCATED_MEMORY_INFO)) { allocatedMem = (Integer) entityInfo.get( - ContainerMetricsConstants.ALLOCATED_MEMORY_ENTITY_INFO); + ContainerMetricsConstants.ALLOCATED_MEMORY_INFO); } if (entityInfo - .containsKey(ContainerMetricsConstants.ALLOCATED_VCORE_ENTITY_INFO)) { + .containsKey(ContainerMetricsConstants.ALLOCATED_VCORE_INFO)) { allocatedVcore = (Integer) entityInfo.get( - ContainerMetricsConstants.ALLOCATED_VCORE_ENTITY_INFO); + ContainerMetricsConstants.ALLOCATED_VCORE_INFO); } if (entityInfo - .containsKey(ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO)) { + .containsKey(ContainerMetricsConstants.ALLOCATED_HOST_INFO)) { allocatedHost = entityInfo - .get(ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO) + .get(ContainerMetricsConstants.ALLOCATED_HOST_INFO) .toString(); } if (entityInfo - .containsKey(ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO)) { + .containsKey(ContainerMetricsConstants.ALLOCATED_PORT_INFO)) { allocatedPort = (Integer) entityInfo.get( - ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO); + ContainerMetricsConstants.ALLOCATED_PORT_INFO); } if (entityInfo - .containsKey(ContainerMetricsConstants.ALLOCATED_PRIORITY_ENTITY_INFO)) { + .containsKey(ContainerMetricsConstants.ALLOCATED_PRIORITY_INFO)) { allocatedPriority = (Integer) entityInfo.get( - ContainerMetricsConstants.ALLOCATED_PRIORITY_ENTITY_INFO); + ContainerMetricsConstants.ALLOCATED_PRIORITY_INFO); } if (entityInfo.containsKey( - ContainerMetricsConstants.ALLOCATED_HOST_HTTP_ADDRESS_ENTITY_INFO)) { + ContainerMetricsConstants.ALLOCATED_HOST_HTTP_ADDRESS_INFO)) { nodeHttpAddress = (String) entityInfo - .get(ContainerMetricsConstants.ALLOCATED_HOST_HTTP_ADDRESS_ENTITY_INFO); + .get(ContainerMetricsConstants.ALLOCATED_HOST_HTTP_ADDRESS_INFO); } } List events = entity.getEvents(); @@ -609,22 +610,22 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService continue; } if (eventInfo - .containsKey(ContainerMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO)) { + .containsKey(ContainerMetricsConstants.DIAGNOSTICS_INFO)) { diagnosticsInfo = eventInfo.get( - ContainerMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO) + ContainerMetricsConstants.DIAGNOSTICS_INFO) .toString(); } if (eventInfo - .containsKey(ContainerMetricsConstants.EXIT_STATUS_EVENT_INFO)) { + .containsKey(ContainerMetricsConstants.EXIT_STATUS_INFO)) { exitStatus = (Integer) eventInfo.get( - ContainerMetricsConstants.EXIT_STATUS_EVENT_INFO); + ContainerMetricsConstants.EXIT_STATUS_INFO); } if (eventInfo - .containsKey(ContainerMetricsConstants.STATE_EVENT_INFO)) { + .containsKey(ContainerMetricsConstants.STATE_INFO)) { state = ContainerState.valueOf(eventInfo.get( - ContainerMetricsConstants.STATE_EVENT_INFO).toString()); + ContainerMetricsConstants.STATE_INFO).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/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 1c10117e75c..9a2046be533 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 @@ -20,14 +20,14 @@ package org.apache.hadoop.yarn.server.applicationhistoryservice; import java.io.IOException; import java.net.InetSocketAddress; -import java.util.ArrayList; +import java.util.LinkedHashSet; +import java.util.Set; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.http.HttpServer2; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.metrics2.source.JvmMetrics; -import org.apache.hadoop.security.AuthenticationFilterInitializer; import org.apache.hadoop.security.HttpCrossOriginFilterInitializer; import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.service.CompositeService; @@ -47,10 +47,9 @@ import org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore; import org.apache.hadoop.yarn.server.timeline.TimelineDataManager; import org.apache.hadoop.yarn.server.timeline.TimelineStore; import org.apache.hadoop.yarn.server.timeline.security.TimelineACLsManager; -import org.apache.hadoop.yarn.server.timeline.security.TimelineAuthenticationFilter; -import org.apache.hadoop.yarn.server.timeline.security.TimelineAuthenticationFilterInitializer; -import org.apache.hadoop.yarn.server.timeline.security.TimelineDelegationTokenSecretManagerService; +import org.apache.hadoop.yarn.server.timeline.security.TimelineV1DelegationTokenSecretManagerService; import org.apache.hadoop.yarn.server.timeline.webapp.CrossOriginFilterInitializer; +import org.apache.hadoop.yarn.server.util.timeline.TimelineServerUtils; import org.apache.hadoop.yarn.webapp.WebApp; import org.apache.hadoop.yarn.webapp.WebApps; import org.apache.hadoop.yarn.webapp.util.WebAppUtils; @@ -75,7 +74,7 @@ public class ApplicationHistoryServer extends CompositeService { private ApplicationACLsManager aclsManager; private ApplicationHistoryManager historyManager; private TimelineStore timelineStore; - private TimelineDelegationTokenSecretManagerService secretManagerService; + private TimelineV1DelegationTokenSecretManagerService secretManagerService; private TimelineDataManager timelineDataManager; private WebApp webApp; private JvmPauseMonitor pauseMonitor; @@ -223,9 +222,9 @@ public class ApplicationHistoryServer extends CompositeService { TimelineStore.class), conf); } - private TimelineDelegationTokenSecretManagerService + private TimelineV1DelegationTokenSecretManagerService createTimelineDelegationTokenSecretManagerService(Configuration conf) { - return new TimelineDelegationTokenSecretManagerService(); + return new TimelineV1DelegationTokenSecretManagerService(); } private TimelineDataManager createTimelineDataManager(Configuration conf) { @@ -237,63 +236,33 @@ public class ApplicationHistoryServer extends CompositeService { @SuppressWarnings("unchecked") private void startWebApp() { Configuration conf = getConfig(); - TimelineAuthenticationFilter.setTimelineDelegationTokenSecretManager( - secretManagerService.getTimelineDelegationTokenSecretManager()); // Always load pseudo authentication filter to parse "user.name" in an URL // to identify a HTTP request's user in insecure mode. // When Kerberos authentication type is set (i.e., secure mode is turned on), // the customized filter will be loaded by the timeline server to do Kerberos // + DT authentication. - String initializers = conf.get("hadoop.http.filter.initializers"); - boolean modifiedInitializers = false; - - initializers = - initializers == null || initializers.length() == 0 ? "" : initializers; - + String initializers = conf.get("hadoop.http.filter.initializers", ""); + Set defaultInitializers = new LinkedHashSet(); + // Add CORS filter if (!initializers.contains(CrossOriginFilterInitializer.class.getName())) { - if(conf.getBoolean(YarnConfiguration - .TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED, YarnConfiguration - .TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED_DEFAULT)) { - if (initializers.contains(HttpCrossOriginFilterInitializer.class.getName())) { - initializers = - initializers.replaceAll(HttpCrossOriginFilterInitializer.class.getName(), + if(conf.getBoolean(YarnConfiguration. + TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED, + YarnConfiguration. + TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED_DEFAULT)) { + if (initializers.contains( + HttpCrossOriginFilterInitializer.class.getName())) { + initializers = initializers.replaceAll( + HttpCrossOriginFilterInitializer.class.getName(), CrossOriginFilterInitializer.class.getName()); + } else { + defaultInitializers.add(CrossOriginFilterInitializer.class.getName()); } - else { - if (initializers.length() != 0) { - initializers += ","; - } - initializers += CrossOriginFilterInitializer.class.getName(); - } - modifiedInitializers = true; } } - - if (!initializers.contains(TimelineAuthenticationFilterInitializer.class - .getName())) { - if (initializers.length() != 0) { - initializers += ","; - } - initializers += TimelineAuthenticationFilterInitializer.class.getName(); - modifiedInitializers = true; - } - - String[] parts = initializers.split(","); - ArrayList target = new ArrayList(); - for (String filterInitializer : parts) { - filterInitializer = filterInitializer.trim(); - if (filterInitializer.equals(AuthenticationFilterInitializer.class - .getName())) { - modifiedInitializers = true; - continue; - } - target.add(filterInitializer); - } - String actualInitializers = - org.apache.commons.lang.StringUtils.join(target, ","); - if (modifiedInitializers) { - conf.set("hadoop.http.filter.initializers", actualInitializers); - } + TimelineServerUtils.addTimelineAuthFilter( + initializers, defaultInitializers, secretManagerService); + TimelineServerUtils.setTimelineFilters( + conf, initializers, defaultInitializers); String bindAddress = WebAppUtils.getWebAppBindURL(conf, YarnConfiguration.TIMELINE_SERVICE_BIND_HOST, WebAppUtils.getAHSWebAppURLWithoutScheme(conf)); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelegationTokenSecretManagerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineV1DelegationTokenSecretManagerService.java similarity index 79% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelegationTokenSecretManagerService.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineV1DelegationTokenSecretManagerService.java index 0c6892a19d3..85d8ccab3c3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelegationTokenSecretManagerService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineV1DelegationTokenSecretManagerService.java @@ -26,7 +26,6 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager; import org.apache.hadoop.security.token.delegation.DelegationKey; -import org.apache.hadoop.service.AbstractService; import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier; @@ -37,18 +36,16 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * The service wrapper of {@link TimelineDelegationTokenSecretManager} + * The service wrapper of {@link TimelineV1DelegationTokenSecretManager}. */ @Private @Unstable -public class TimelineDelegationTokenSecretManagerService extends - AbstractService { - - private TimelineDelegationTokenSecretManager secretManager = null; +public class TimelineV1DelegationTokenSecretManagerService extends + TimelineDelgationTokenSecretManagerService { private TimelineStateStore stateStore = null; - public TimelineDelegationTokenSecretManagerService() { - super(TimelineDelegationTokenSecretManagerService.class.getName()); + public TimelineV1DelegationTokenSecretManagerService() { + super(TimelineV1DelegationTokenSecretManagerService.class.getName()); } @Override @@ -58,19 +55,7 @@ public class TimelineDelegationTokenSecretManagerService extends stateStore = createStateStore(conf); stateStore.init(conf); } - - long secretKeyInterval = - conf.getLong(YarnConfiguration.TIMELINE_DELEGATION_KEY_UPDATE_INTERVAL, - YarnConfiguration.DEFAULT_TIMELINE_DELEGATION_KEY_UPDATE_INTERVAL); - long tokenMaxLifetime = - conf.getLong(YarnConfiguration.TIMELINE_DELEGATION_TOKEN_MAX_LIFETIME, - YarnConfiguration.DEFAULT_TIMELINE_DELEGATION_TOKEN_MAX_LIFETIME); - long tokenRenewInterval = - conf.getLong(YarnConfiguration.TIMELINE_DELEGATION_TOKEN_RENEW_INTERVAL, - YarnConfiguration.DEFAULT_TIMELINE_DELEGATION_TOKEN_RENEW_INTERVAL); - secretManager = new TimelineDelegationTokenSecretManager(secretKeyInterval, - tokenMaxLifetime, tokenRenewInterval, 3600000, stateStore); - super.init(conf); + super.serviceInit(conf); } @Override @@ -78,10 +63,9 @@ public class TimelineDelegationTokenSecretManagerService extends if (stateStore != null) { stateStore.start(); TimelineServiceState state = stateStore.loadState(); - secretManager.recover(state); + ((TimelineV1DelegationTokenSecretManager) + getTimelineDelegationTokenSecretManager()).recover(state); } - - secretManager.startThreads(); super.serviceStart(); } @@ -90,9 +74,18 @@ public class TimelineDelegationTokenSecretManagerService extends if (stateStore != null) { stateStore.stop(); } + super.serviceStop(); + } - secretManager.stopThreads(); - super.stop(); + @Override + protected AbstractDelegationTokenSecretManager + + createTimelineDelegationTokenSecretManager(long secretKeyInterval, + long tokenMaxLifetime, long tokenRenewInterval, + long tokenRemovalScanInterval) { + return new TimelineV1DelegationTokenSecretManager(secretKeyInterval, + tokenMaxLifetime, tokenRenewInterval, tokenRemovalScanInterval, + stateStore); } protected TimelineStateStore createStateStore( @@ -104,27 +97,20 @@ public class TimelineDelegationTokenSecretManagerService extends } /** - * Ge the instance of {link #TimelineDelegationTokenSecretManager} - * - * @return the instance of {link #TimelineDelegationTokenSecretManager} + * Delegation token secret manager for ATSv1 and ATSv1.5. */ - public TimelineDelegationTokenSecretManager - getTimelineDelegationTokenSecretManager() { - return secretManager; - } - @Private @Unstable - public static class TimelineDelegationTokenSecretManager extends + public static class TimelineV1DelegationTokenSecretManager extends AbstractDelegationTokenSecretManager { public static final Logger LOG = - LoggerFactory.getLogger(TimelineDelegationTokenSecretManager.class); + LoggerFactory.getLogger(TimelineV1DelegationTokenSecretManager.class); private TimelineStateStore stateStore; /** - * Create a timeline secret manager + * Create a timeline v1 secret manager. * @param delegationKeyUpdateInterval the number of milliseconds for rolling * new secret keys. * @param delegationTokenMaxLifetime the maximum lifetime of the delegation @@ -135,7 +121,7 @@ public class TimelineDelegationTokenSecretManagerService extends * scanned for expired tokens in milliseconds * @param stateStore timeline service state store */ - public TimelineDelegationTokenSecretManager( + public TimelineV1DelegationTokenSecretManager( long delegationKeyUpdateInterval, long delegationTokenMaxLifetime, long delegationTokenRenewInterval, @@ -236,5 +222,4 @@ public class TimelineDelegationTokenSecretManagerService extends } } } - } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerOnTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerOnTimelineStore.java index 99bbe328f7d..ecaaf1e878d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerOnTimelineStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerOnTimelineStore.java @@ -628,13 +628,13 @@ public class TestApplicationHistoryManagerOnTimelineStore { tEvent.setEventType(AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE); tEvent.setTimestamp(Integer.MAX_VALUE + 1L); Map eventInfo = new HashMap(); - eventInfo.put(AppAttemptMetricsConstants.TRACKING_URL_EVENT_INFO, + eventInfo.put(AppAttemptMetricsConstants.TRACKING_URL_INFO, "test tracking url"); - eventInfo.put(AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_EVENT_INFO, + eventInfo.put(AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_INFO, "test original tracking url"); - eventInfo.put(AppAttemptMetricsConstants.HOST_EVENT_INFO, "test host"); - eventInfo.put(AppAttemptMetricsConstants.RPC_PORT_EVENT_INFO, 100); - eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO, + eventInfo.put(AppAttemptMetricsConstants.HOST_INFO, "test host"); + eventInfo.put(AppAttemptMetricsConstants.RPC_PORT_INFO, 100); + eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_INFO, ContainerId.newContainerId(appAttemptId, 1)); tEvent.setEventInfo(eventInfo); entity.addEvent(tEvent); @@ -642,15 +642,15 @@ public class TestApplicationHistoryManagerOnTimelineStore { tEvent.setEventType(AppAttemptMetricsConstants.FINISHED_EVENT_TYPE); tEvent.setTimestamp(Integer.MAX_VALUE + 2L); eventInfo = new HashMap(); - eventInfo.put(AppAttemptMetricsConstants.TRACKING_URL_EVENT_INFO, + eventInfo.put(AppAttemptMetricsConstants.TRACKING_URL_INFO, "test tracking url"); - eventInfo.put(AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_EVENT_INFO, + eventInfo.put(AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_INFO, "test original tracking url"); - eventInfo.put(AppAttemptMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO, + eventInfo.put(AppAttemptMetricsConstants.DIAGNOSTICS_INFO, "test diagnostics info"); - eventInfo.put(AppAttemptMetricsConstants.FINAL_STATUS_EVENT_INFO, + eventInfo.put(AppAttemptMetricsConstants.FINAL_STATUS_INFO, FinalApplicationStatus.UNDEFINED.toString()); - eventInfo.put(AppAttemptMetricsConstants.STATE_EVENT_INFO, + eventInfo.put(AppAttemptMetricsConstants.STATE_INFO, YarnApplicationAttemptState.FINISHED.toString()); tEvent.setEventInfo(eventInfo); entity.addEvent(tEvent); @@ -667,15 +667,15 @@ public class TestApplicationHistoryManagerOnTimelineStore { entity.addPrimaryFilter( TimelineStore.SystemFilter.ENTITY_OWNER.toString(), "yarn"); Map entityInfo = new HashMap(); - entityInfo.put(ContainerMetricsConstants.ALLOCATED_MEMORY_ENTITY_INFO, -1); - entityInfo.put(ContainerMetricsConstants.ALLOCATED_VCORE_ENTITY_INFO, -1); - entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO, + entityInfo.put(ContainerMetricsConstants.ALLOCATED_MEMORY_INFO, -1); + entityInfo.put(ContainerMetricsConstants.ALLOCATED_VCORE_INFO, -1); + entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_INFO, "test host"); - entityInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO, 100); + entityInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_INFO, 100); entityInfo - .put(ContainerMetricsConstants.ALLOCATED_PRIORITY_ENTITY_INFO, -1); + .put(ContainerMetricsConstants.ALLOCATED_PRIORITY_INFO, -1); entityInfo.put(ContainerMetricsConstants - .ALLOCATED_HOST_HTTP_ADDRESS_ENTITY_INFO, "http://test:1234"); + .ALLOCATED_HOST_HTTP_ADDRESS_INFO, "http://test:1234"); entity.setOtherInfo(entityInfo); TimelineEvent tEvent = new TimelineEvent(); tEvent.setEventType(ContainerMetricsConstants.CREATED_EVENT_TYPE); @@ -686,10 +686,10 @@ public class TestApplicationHistoryManagerOnTimelineStore { tEvent.setEventType(ContainerMetricsConstants.FINISHED_EVENT_TYPE); tEvent.setTimestamp(Integer.MAX_VALUE + 2L); Map eventInfo = new HashMap(); - eventInfo.put(ContainerMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO, + eventInfo.put(ContainerMetricsConstants.DIAGNOSTICS_INFO, "test diagnostics info"); - eventInfo.put(ContainerMetricsConstants.EXIT_STATUS_EVENT_INFO, -1); - eventInfo.put(ContainerMetricsConstants.STATE_EVENT_INFO, + eventInfo.put(ContainerMetricsConstants.EXIT_STATUS_INFO, -1); + eventInfo.put(ContainerMetricsConstants.STATE_INFO, ContainerState.COMPLETE.toString()); tEvent.setEventInfo(eventInfo); entity.addEvent(tEvent); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilterForV1.java similarity index 85% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilter.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilterForV1.java index 063f5121e50..d918e8ddde6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilterForV1.java @@ -55,27 +55,31 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +/** + * Test cases for authentication via TimelineAuthenticationFilter while + * publishing entities for ATSv1. + */ @RunWith(Parameterized.class) -public class TestTimelineAuthenticationFilter { +public class TestTimelineAuthenticationFilterForV1 { private static final String FOO_USER = "foo"; private static final String BAR_USER = "bar"; private static final String HTTP_USER = "HTTP"; - private static final File testRootDir = new File( + private static final File TEST_ROOT_DIR = new File( System.getProperty("test.build.dir", "target/test-dir"), - TestTimelineAuthenticationFilter.class.getName() + "-root"); + TestTimelineAuthenticationFilterForV1.class.getName() + "-root"); private static File httpSpnegoKeytabFile = new File( KerberosTestUtils.getKeytabFile()); private static String httpSpnegoPrincipal = KerberosTestUtils.getServerPrincipal(); private static final String BASEDIR = System.getProperty("test.build.dir", "target/test-dir") + "/" - + TestTimelineAuthenticationFilter.class.getSimpleName(); + + TestTimelineAuthenticationFilterForV1.class.getSimpleName(); @Parameterized.Parameters public static Collection withSsl() { - return Arrays.asList(new Object[][] { { false }, { true } }); + return Arrays.asList(new Object[][] {{false}, {true}}); } private static MiniKdc testMiniKDC; @@ -85,14 +89,14 @@ public class TestTimelineAuthenticationFilter { private static Configuration conf; private static boolean withSsl; - public TestTimelineAuthenticationFilter(boolean withSsl) { - TestTimelineAuthenticationFilter.withSsl = withSsl; + public TestTimelineAuthenticationFilterForV1(boolean withSsl) { + TestTimelineAuthenticationFilterForV1.withSsl = withSsl; } @BeforeClass public static void setup() { try { - testMiniKDC = new MiniKdc(MiniKdc.createConf(), testRootDir); + testMiniKDC = new MiniKdc(MiniKdc.createConf(), TEST_ROOT_DIR); testMiniKDC.start(); testMiniKDC.createPrincipal( httpSpnegoKeytabFile, HTTP_USER + "/localhost"); @@ -111,11 +115,11 @@ public class TestTimelineAuthenticationFilter { KerberosAuthenticationHandler.KEYTAB, httpSpnegoKeytabFile.getAbsolutePath()); conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, - "kerberos"); + "kerberos"); conf.set(YarnConfiguration.TIMELINE_SERVICE_PRINCIPAL, - httpSpnegoPrincipal); + httpSpnegoPrincipal); conf.set(YarnConfiguration.TIMELINE_SERVICE_KEYTAB, - httpSpnegoKeytabFile.getAbsolutePath()); + httpSpnegoKeytabFile.getAbsolutePath()); conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); conf.setClass(YarnConfiguration.TIMELINE_SERVICE_STORE, MemoryTimelineStore.class, TimelineStore.class); @@ -136,8 +140,8 @@ public class TestTimelineAuthenticationFilter { FileUtil.fullyDelete(base); base.mkdirs(); keystoresDir = new File(BASEDIR).getAbsolutePath(); - sslConfDir = - KeyStoreTestUtil.getClasspathDir(TestTimelineAuthenticationFilter.class); + sslConfDir = KeyStoreTestUtil.getClasspathDir( + TestTimelineAuthenticationFilterForV1.class); KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, conf, false); } @@ -145,6 +149,7 @@ public class TestTimelineAuthenticationFilter { testTimelineServer.init(conf); testTimelineServer.start(); } catch (Exception e) { + e.printStackTrace(); assertTrue("Couldn't setup TimelineServer", false); } } @@ -181,14 +186,14 @@ public class TestTimelineAuthenticationFilter { TimelineClient client = createTimelineClientForUGI(); TimelineEntity entityToStore = new TimelineEntity(); entityToStore.setEntityType( - TestTimelineAuthenticationFilter.class.getName()); + TestTimelineAuthenticationFilterForV1.class.getName()); entityToStore.setEntityId("entity1"); entityToStore.setStartTime(0L); TimelinePutResponse putResponse = client.putEntities(entityToStore); Assert.assertEquals(0, putResponse.getErrors().size()); TimelineEntity entityToRead = - testTimelineServer.getTimelineStore().getEntity( - "entity1", TestTimelineAuthenticationFilter.class.getName(), null); + testTimelineServer.getTimelineStore().getEntity("entity1", + TestTimelineAuthenticationFilterForV1.class.getName(), null); Assert.assertNotNull(entityToRead); return null; } @@ -202,13 +207,14 @@ public class TestTimelineAuthenticationFilter { public Void call() throws Exception { TimelineClient client = createTimelineClientForUGI(); TimelineDomain domainToStore = new TimelineDomain(); - domainToStore.setId(TestTimelineAuthenticationFilter.class.getName()); + domainToStore.setId( + TestTimelineAuthenticationFilterForV1.class.getName()); domainToStore.setReaders("*"); domainToStore.setWriters("*"); client.putDomain(domainToStore); TimelineDomain domainToRead = testTimelineServer.getTimelineStore().getDomain( - TestTimelineAuthenticationFilter.class.getName()); + TestTimelineAuthenticationFilterForV1.class.getName()); Assert.assertNotNull(domainToRead); return null; } @@ -218,22 +224,24 @@ public class TestTimelineAuthenticationFilter { @Test public void testDelegationTokenOperations() throws Exception { TimelineClient httpUserClient = - KerberosTestUtils.doAs(HTTP_USER + "/localhost", new Callable() { - @Override - public TimelineClient call() throws Exception { - return createTimelineClientForUGI(); - } - }); + KerberosTestUtils.doAs(HTTP_USER + "/localhost", + new Callable() { + @Override + public TimelineClient call() throws Exception { + return createTimelineClientForUGI(); + } + }); UserGroupInformation httpUser = - KerberosTestUtils.doAs(HTTP_USER + "/localhost", new Callable() { - @Override - public UserGroupInformation call() throws Exception { - return UserGroupInformation.getCurrentUser(); - } - }); + KerberosTestUtils.doAs(HTTP_USER + "/localhost", + new Callable() { + @Override + public UserGroupInformation call() throws Exception { + return UserGroupInformation.getCurrentUser(); + } + }); // Let HTTP user to get the delegation for itself Token token = - httpUserClient.getDelegationToken(httpUser.getShortUserName()); + httpUserClient.getDelegationToken(httpUser.getShortUserName()); Assert.assertNotNull(token); TimelineDelegationTokenIdentifier tDT = token.decodeIdentifier(); Assert.assertNotNull(tDT); @@ -317,7 +325,8 @@ public class TestTimelineAuthenticationFilter { barUserClient.getDelegationToken(httpUser.getShortUserName()); Assert.fail(); } catch (Exception e) { - Assert.assertTrue(e.getCause() instanceof AuthorizationException || e.getCause() instanceof AuthenticationException); + Assert.assertTrue(e.getCause() instanceof AuthorizationException || + e.getCause() instanceof AuthenticationException); } } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml index f484e35573a..f891e9263c4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml @@ -190,6 +190,7 @@ yarn_server_federation_protos.proto ResourceTracker.proto SCMUploader.proto + collectornodemanager_protocol.proto diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/CollectorNodemanagerProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/CollectorNodemanagerProtocol.java new file mode 100644 index 00000000000..64eea63160d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/CollectorNodemanagerProtocol.java @@ -0,0 +1,73 @@ +/** + * 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 org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoResponse; + +/** + *

The protocol between an TimelineCollectorManager and a + * NodeManager to report a new application collector get launched. + *

+ * + */ +@Private +public interface CollectorNodemanagerProtocol { + + /** + * + *

+ * The TimelineCollectorManager provides a list of mapping + * between application and collector's address in + * {@link ReportNewCollectorInfoRequest} to a NodeManager to + * register collector's info, include: applicationId and REST URI to + * access collector. NodeManager will add them into registered collectors + * and register them into ResourceManager afterwards. + *

+ * + * @param request the request of registering a new collector or a list of + * collectors + * @return the response for registering the new collector + * @throws YarnException if the request is invalid + * @throws IOException if there are I/O errors + */ + ReportNewCollectorInfoResponse reportNewCollectorInfo( + ReportNewCollectorInfoRequest request) + throws YarnException, IOException; + + /** + *

+ * The collector needs to get the context information including user, flow + * and flow run ID to associate with every incoming put-entity requests. + *

+ * @param request the request of getting the aggregator context information of + * the given application + * @return the response for registering the new collector + * @throws YarnException if the request is invalid + * @throws IOException if there are I/O errors + */ + GetTimelineCollectorContextResponse getTimelineCollectorContext( + GetTimelineCollectorContextRequest request) + 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/api/CollectorNodemanagerProtocolPB.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/CollectorNodemanagerProtocolPB.java new file mode 100644 index 00000000000..9529ddd3f31 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/CollectorNodemanagerProtocolPB.java @@ -0,0 +1,37 @@ +/** + * 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 org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.ipc.ProtocolInfo; +import org.apache.hadoop.yarn.proto.CollectorNodemanagerProtocol.CollectorNodemanagerProtocolService; + +/** + * Collector nodemanager protocol service implementation. + */ +@Private +@Unstable +@ProtocolInfo( + protocolName = + "org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocolPB", + protocolVersion = 1) +public interface CollectorNodemanagerProtocolPB extends + CollectorNodemanagerProtocolService.BlockingInterface { + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/CollectorNodemanagerProtocolPBClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/CollectorNodemanagerProtocolPBClientImpl.java new file mode 100644 index 00000000000..a4f18d276a4 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/CollectorNodemanagerProtocolPBClientImpl.java @@ -0,0 +1,117 @@ +/** + * 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.impl.pb.client; + +import java.io.Closeable; +import java.io.IOException; +import java.net.InetSocketAddress; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.ipc.ProtobufRpcEngine; +import org.apache.hadoop.ipc.RPC; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.ipc.RPCUtil; +import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.GetTimelineCollectorContextRequestProto; +import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewCollectorInfoRequestProto; +import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocol; +import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocolPB; +import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.GetTimelineCollectorContextRequestPBImpl; +import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.GetTimelineCollectorContextResponsePBImpl; +import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReportNewCollectorInfoRequestPBImpl; +import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReportNewCollectorInfoResponsePBImpl; + +import com.google.protobuf.ServiceException; + +/** + * Client implementation of {@link CollectorNodemanagerProtocol}. + */ +public class CollectorNodemanagerProtocolPBClientImpl implements + CollectorNodemanagerProtocol, Closeable { + + // Not a documented config. Only used for tests internally + static final String NM_COMMAND_TIMEOUT = YarnConfiguration.YARN_PREFIX + + "rpc.nm-command-timeout"; + + /** + * Maximum of 1 minute timeout for a Node to react to the command. + */ + static final int DEFAULT_COMMAND_TIMEOUT = 60000; + + private CollectorNodemanagerProtocolPB proxy; + + @Private + public CollectorNodemanagerProtocolPBClientImpl(long clientVersion, + InetSocketAddress addr, Configuration conf) throws IOException { + RPC.setProtocolEngine(conf, CollectorNodemanagerProtocolPB.class, + ProtobufRpcEngine.class); + UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); + + int expireIntvl = conf.getInt(NM_COMMAND_TIMEOUT, DEFAULT_COMMAND_TIMEOUT); + proxy = + (CollectorNodemanagerProtocolPB) RPC.getProxy( + CollectorNodemanagerProtocolPB.class, + clientVersion, addr, ugi, conf, + NetUtils.getDefaultSocketFactory(conf), expireIntvl); + } + + @Override + public ReportNewCollectorInfoResponse reportNewCollectorInfo( + ReportNewCollectorInfoRequest request) throws YarnException, IOException { + + ReportNewCollectorInfoRequestProto requestProto = + ((ReportNewCollectorInfoRequestPBImpl) request).getProto(); + try { + return new ReportNewCollectorInfoResponsePBImpl( + proxy.reportNewCollectorInfo(null, requestProto)); + } catch (ServiceException e) { + RPCUtil.unwrapAndThrowException(e); + return null; + } + } + + @Override + public GetTimelineCollectorContextResponse getTimelineCollectorContext( + GetTimelineCollectorContextRequest request) + throws YarnException, IOException { + GetTimelineCollectorContextRequestProto requestProto = + ((GetTimelineCollectorContextRequestPBImpl) request).getProto(); + try { + return new GetTimelineCollectorContextResponsePBImpl( + proxy.getTimelineCollectorContext(null, requestProto)); + } catch (ServiceException e) { + RPCUtil.unwrapAndThrowException(e); + return null; + } + } + + @Override + public void close() { + if (this.proxy != null) { + RPC.stopProxy(this.proxy); + } + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/package-info.java new file mode 100644 index 00000000000..fad9a71f2ab --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/package-info.java @@ -0,0 +1,23 @@ +/* + * 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. + */ + +/** + * Contains protocol client implementations used to communicate with different + * YARN server components. + */ +package org.apache.hadoop.yarn.server.api.impl.pb.client; \ 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/api/impl/pb/service/CollectorNodemanagerProtocolPBServiceImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/CollectorNodemanagerProtocolPBServiceImpl.java new file mode 100644 index 00000000000..cbe8137e68a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/CollectorNodemanagerProtocolPBServiceImpl.java @@ -0,0 +1,85 @@ +/** + * 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.impl.pb.service; + +import java.io.IOException; + +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.GetTimelineCollectorContextRequestProto; +import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.GetTimelineCollectorContextResponseProto; +import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewCollectorInfoRequestProto; +import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewCollectorInfoResponseProto; +import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocol; +import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocolPB; +import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.GetTimelineCollectorContextRequestPBImpl; +import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.GetTimelineCollectorContextResponsePBImpl; +import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReportNewCollectorInfoRequestPBImpl; +import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReportNewCollectorInfoResponsePBImpl; + +import com.google.protobuf.RpcController; +import com.google.protobuf.ServiceException; + +/** + * Service implementation of CollectorNodemanagerProtocol. + */ +public class CollectorNodemanagerProtocolPBServiceImpl implements + CollectorNodemanagerProtocolPB { + + private CollectorNodemanagerProtocol real; + + public CollectorNodemanagerProtocolPBServiceImpl( + CollectorNodemanagerProtocol impl) { + this.real = impl; + } + + @Override + public ReportNewCollectorInfoResponseProto reportNewCollectorInfo( + RpcController arg0, ReportNewCollectorInfoRequestProto proto) + throws ServiceException { + ReportNewCollectorInfoRequestPBImpl request = + new ReportNewCollectorInfoRequestPBImpl(proto); + try { + ReportNewCollectorInfoResponse response = + real.reportNewCollectorInfo(request); + return ((ReportNewCollectorInfoResponsePBImpl)response).getProto(); + } catch (YarnException e) { + throw new ServiceException(e); + } catch (IOException e) { + throw new ServiceException(e); + } + } + + @Override + public GetTimelineCollectorContextResponseProto getTimelineCollectorContext( + RpcController controller, + GetTimelineCollectorContextRequestProto proto) throws ServiceException { + GetTimelineCollectorContextRequestPBImpl request = + new GetTimelineCollectorContextRequestPBImpl(proto); + try { + GetTimelineCollectorContextResponse response = + real.getTimelineCollectorContext(request); + return ((GetTimelineCollectorContextResponsePBImpl)response).getProto(); + } catch (YarnException e) { + throw new ServiceException(e); + } catch (IOException e) { + throw new ServiceException(e); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/package-info.java new file mode 100644 index 00000000000..52f8df433db --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/package-info.java @@ -0,0 +1,23 @@ +/* + * 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. + */ + +/** + * Contains interfaces which define the protocols to communicate with different + * YARN server components. + */ +package org.apache.hadoop.yarn.server.api; \ 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/metrics/ApplicaitonStateUpdatedEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/GetTimelineCollectorContextRequest.java similarity index 52% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicaitonStateUpdatedEvent.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/GetTimelineCollectorContextRequest.java index 599e8d6710f..321c41dd42a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicaitonStateUpdatedEvent.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/GetTimelineCollectorContextRequest.java @@ -15,33 +15,28 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +package org.apache.hadoop.yarn.server.api.protocolrecords; -package org.apache.hadoop.yarn.server.resourcemanager.metrics; import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.util.Records; /** - * When the state of this application has been changed, RM would sent - * this event to inform Timeline Server for keeping the Application state - * consistent. + * GetTimelineCollectorContextRequest invoked by collector to get the context + * information from Nodemanager. The context information includes user, flow and + * flow run ID and is associated with every incoming put-entity request. */ -public class ApplicaitonStateUpdatedEvent extends SystemMetricsEvent{ - private ApplicationId appId; - private YarnApplicationState appState; +public abstract class GetTimelineCollectorContextRequest { - public ApplicaitonStateUpdatedEvent(ApplicationId appliocationId, - YarnApplicationState state, long updatedTime) { - super(SystemMetricsEventType.APP_STATE_UPDATED, updatedTime); - this.appId = appliocationId; - this.appState = state; + public static GetTimelineCollectorContextRequest newInstance( + ApplicationId appId) { + GetTimelineCollectorContextRequest request = + Records.newRecord(GetTimelineCollectorContextRequest.class); + request.setApplicationId(appId); + return request; } - public ApplicationId getApplicationId() { - return appId; - } + public abstract ApplicationId getApplicationId(); - public YarnApplicationState getAppState() { - return appState; - } + public abstract void setApplicationId(ApplicationId appId); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/GetTimelineCollectorContextResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/GetTimelineCollectorContextResponse.java new file mode 100644 index 00000000000..8a424d92f45 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/GetTimelineCollectorContextResponse.java @@ -0,0 +1,57 @@ +/** + * 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.protocolrecords; + + +import org.apache.hadoop.yarn.util.Records; + +/** + * GetTimelineCollectorContextResponse sent in response of + * GetTimelineCollectorContextRequest invoked by collector. The response + * includes user, flow name, flow version and flow run ID. This is then + * associated by collector with every incoming put-entity request. + */ +public abstract class GetTimelineCollectorContextResponse { + + public static GetTimelineCollectorContextResponse newInstance( + String userId, String flowName, String flowVersion, long flowRunId) { + GetTimelineCollectorContextResponse response = + Records.newRecord(GetTimelineCollectorContextResponse.class); + response.setUserId(userId); + response.setFlowName(flowName); + response.setFlowVersion(flowVersion); + response.setFlowRunId(flowRunId); + return response; + } + + public abstract String getUserId(); + + public abstract void setUserId(String userId); + + public abstract String getFlowName(); + + public abstract void setFlowName(String flowName); + + public abstract String getFlowVersion(); + + public abstract void setFlowVersion(String flowVersion); + + public abstract long getFlowRunId(); + + public abstract void setFlowRunId(long flowRunId); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatRequest.java index 84ca8a497f4..f238f79f172 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatRequest.java @@ -19,9 +19,12 @@ package org.apache.hadoop.yarn.server.api.protocolrecords; import java.util.List; +import java.util.Map; import java.util.Set; +import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.NodeLabel; +import org.apache.hadoop.yarn.server.api.records.AppCollectorData; import org.apache.hadoop.yarn.server.api.records.MasterKey; import org.apache.hadoop.yarn.server.api.records.NodeStatus; import org.apache.hadoop.yarn.util.Records; @@ -42,6 +45,22 @@ public abstract class NodeHeartbeatRequest { return nodeHeartbeatRequest; } + public static NodeHeartbeatRequest newInstance(NodeStatus nodeStatus, + MasterKey lastKnownContainerTokenMasterKey, + MasterKey lastKnownNMTokenMasterKey, Set nodeLabels, + Map registeringCollectors) { + NodeHeartbeatRequest nodeHeartbeatRequest = + Records.newRecord(NodeHeartbeatRequest.class); + nodeHeartbeatRequest.setNodeStatus(nodeStatus); + nodeHeartbeatRequest + .setLastKnownContainerTokenMasterKey(lastKnownContainerTokenMasterKey); + nodeHeartbeatRequest + .setLastKnownNMTokenMasterKey(lastKnownNMTokenMasterKey); + nodeHeartbeatRequest.setNodeLabels(nodeLabels); + nodeHeartbeatRequest.setRegisteringCollectors(registeringCollectors); + return nodeHeartbeatRequest; + } + public abstract NodeStatus getNodeStatus(); public abstract void setNodeStatus(NodeStatus status); @@ -59,4 +78,11 @@ public abstract class NodeHeartbeatRequest { public abstract void setLogAggregationReportsForApps( List logAggregationReportsForApps); + + // This tells RM registered collectors' address info on this node + public abstract Map + getRegisteringCollectors(); + + public abstract void setRegisteringCollectors(Map appCollectorsMap); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java index 8c04ac4e5cb..d7cdc25cd81 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java @@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.api.records.AppCollectorData; import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit; import org.apache.hadoop.yarn.server.api.records.MasterKey; import org.apache.hadoop.yarn.server.api.records.NodeAction; @@ -41,6 +42,11 @@ public interface NodeHeartbeatResponse { List getApplicationsToCleanup(); + // This tells NM the collectors' address info of related apps + Map getAppCollectors(); + void setAppCollectors( + Map appCollectorsMap); + void setResponseId(int responseId); void setNodeAction(NodeAction action); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/ReportNewCollectorInfoRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/ReportNewCollectorInfoRequest.java new file mode 100644 index 00000000000..2634f2416f5 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/ReportNewCollectorInfoRequest.java @@ -0,0 +1,60 @@ +/** + * 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.protocolrecords; + +import java.util.List; +import java.util.Arrays; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.Token; +import org.apache.hadoop.yarn.server.api.records.AppCollectorData; +import org.apache.hadoop.yarn.util.Records; + +/** + * Request used to report new collector info from timeline collector manager + * whenever a collector is launched. + * The request contains a list of collector data, with each collector data + * object represented by {@link AppCollectorData}. + */ +@Private +public abstract class ReportNewCollectorInfoRequest { + + public static ReportNewCollectorInfoRequest newInstance( + List appCollectorsList) { + ReportNewCollectorInfoRequest request = + Records.newRecord(ReportNewCollectorInfoRequest.class); + request.setAppCollectorsList(appCollectorsList); + return request; + } + + public static ReportNewCollectorInfoRequest newInstance( + ApplicationId id, String collectorAddr, Token token) { + ReportNewCollectorInfoRequest request = + Records.newRecord(ReportNewCollectorInfoRequest.class); + request.setAppCollectorsList( + Arrays.asList(AppCollectorData.newInstance(id, collectorAddr, token))); + return request; + } + + public abstract List getAppCollectorsList(); + + public abstract void setAppCollectorsList( + List appCollectorsList); + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/ReportNewCollectorInfoResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/ReportNewCollectorInfoResponse.java new file mode 100644 index 00000000000..aa95122ac7d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/ReportNewCollectorInfoResponse.java @@ -0,0 +1,35 @@ +/** + * 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.protocolrecords; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.yarn.util.Records; + +/** + * Response to {@link ReportNewCollectorInfoRequest}. + */ +public abstract class ReportNewCollectorInfoResponse { + + @Private + public static ReportNewCollectorInfoResponse newInstance() { + ReportNewCollectorInfoResponse response = + Records.newRecord(ReportNewCollectorInfoResponse.class); + return response; + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/GetTimelineCollectorContextRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/GetTimelineCollectorContextRequestPBImpl.java new file mode 100644 index 00000000000..4703362481c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/GetTimelineCollectorContextRequestPBImpl.java @@ -0,0 +1,136 @@ +/** + * 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.protocolrecords.impl.pb; + + +import com.google.protobuf.TextFormat; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl; +import org.apache.hadoop.yarn.proto.YarnProtos; +import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.GetTimelineCollectorContextRequestProtoOrBuilder; +import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.GetTimelineCollectorContextRequestProto; +import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextRequest; + +/** + * Protocol buffer based implementation of + * {@link GetTimelineCollectorContextRequest}. + */ +public class GetTimelineCollectorContextRequestPBImpl extends + GetTimelineCollectorContextRequest { + + private GetTimelineCollectorContextRequestProto + proto = GetTimelineCollectorContextRequestProto.getDefaultInstance(); + private GetTimelineCollectorContextRequestProto.Builder builder = null; + private boolean viaProto = false; + + private ApplicationId appId = null; + + public GetTimelineCollectorContextRequestPBImpl() { + builder = GetTimelineCollectorContextRequestProto.newBuilder(); + } + + public GetTimelineCollectorContextRequestPBImpl( + GetTimelineCollectorContextRequestProto proto) { + this.proto = proto; + viaProto = true; + } + + public GetTimelineCollectorContextRequestProto getProto() { + mergeLocalToProto(); + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other == null) { + return false; + } + if (other.getClass().isAssignableFrom(this.getClass())) { + return this.getProto().equals(this.getClass().cast(other).getProto()); + } + return false; + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } + + private void mergeLocalToBuilder() { + if (appId != null) { + builder.setAppId(convertToProtoFormat(this.appId)); + } + } + + private void mergeLocalToProto() { + if (viaProto) { + maybeInitBuilder(); + } + mergeLocalToBuilder(); + proto = builder.build(); + viaProto = true; + } + + private void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = GetTimelineCollectorContextRequestProto.newBuilder(proto); + } + viaProto = false; + } + + @Override + public ApplicationId getApplicationId() { + if (this.appId != null) { + return this.appId; + } + + GetTimelineCollectorContextRequestProtoOrBuilder p = + viaProto ? proto : builder; + if (!p.hasAppId()) { + return null; + } + + this.appId = convertFromProtoFormat(p.getAppId()); + return this.appId; + } + + @Override + public void setApplicationId(ApplicationId id) { + maybeInitBuilder(); + if (id == null) { + builder.clearAppId(); + } + this.appId = id; + } + + private ApplicationIdPBImpl convertFromProtoFormat( + YarnProtos.ApplicationIdProto p) { + return new ApplicationIdPBImpl(p); + } + + private YarnProtos.ApplicationIdProto convertToProtoFormat(ApplicationId t) { + return ((ApplicationIdPBImpl)t).getProto(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/GetTimelineCollectorContextResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/GetTimelineCollectorContextResponsePBImpl.java new file mode 100644 index 00000000000..6ac1f051846 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/GetTimelineCollectorContextResponsePBImpl.java @@ -0,0 +1,163 @@ +/** + * 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.protocolrecords.impl.pb; + + +import com.google.protobuf.TextFormat; +import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.GetTimelineCollectorContextResponseProtoOrBuilder; +import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.GetTimelineCollectorContextResponseProto; +import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextResponse; + +/** + * Protocol buffer based implementation of + * {@link GetTimelineCollectorContextResponse}. + */ +public class GetTimelineCollectorContextResponsePBImpl extends + GetTimelineCollectorContextResponse { + + private GetTimelineCollectorContextResponseProto proto = + GetTimelineCollectorContextResponseProto.getDefaultInstance(); + private GetTimelineCollectorContextResponseProto.Builder builder = null; + private boolean viaProto = false; + + public GetTimelineCollectorContextResponsePBImpl() { + builder = GetTimelineCollectorContextResponseProto.newBuilder(); + } + + public GetTimelineCollectorContextResponsePBImpl( + GetTimelineCollectorContextResponseProto proto) { + this.proto = proto; + viaProto = true; + } + + public GetTimelineCollectorContextResponseProto getProto() { + mergeLocalToProto(); + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other == null) { + return false; + } + if (other.getClass().isAssignableFrom(this.getClass())) { + return this.getProto().equals(this.getClass().cast(other).getProto()); + } + return false; + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } + + private void mergeLocalToProto() { + if (viaProto) { + maybeInitBuilder(); + } + proto = builder.build(); + viaProto = true; + } + + private void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = GetTimelineCollectorContextResponseProto.newBuilder(proto); + } + viaProto = false; + } + + @Override + public String getUserId() { + GetTimelineCollectorContextResponseProtoOrBuilder p = + viaProto ? proto : builder; + if (!p.hasUserId()) { + return null; + } + return p.getUserId(); + } + + @Override + public void setUserId(String userId) { + maybeInitBuilder(); + if (userId == null) { + builder.clearUserId(); + return; + } + builder.setUserId(userId); + } + + @Override + public String getFlowName() { + GetTimelineCollectorContextResponseProtoOrBuilder p = + viaProto ? proto : builder; + if (!p.hasFlowName()) { + return null; + } + return p.getFlowName(); + } + + @Override + public void setFlowName(String flowName) { + maybeInitBuilder(); + if (flowName == null) { + builder.clearFlowName(); + return; + } + builder.setFlowName(flowName); + } + + @Override + public String getFlowVersion() { + GetTimelineCollectorContextResponseProtoOrBuilder p = + viaProto ? proto : builder; + if (!p.hasFlowVersion()) { + return null; + } + return p.getFlowVersion(); + } + + @Override + public void setFlowVersion(String flowVersion) { + maybeInitBuilder(); + if (flowVersion == null) { + builder.clearFlowVersion(); + return; + } + builder.setFlowVersion(flowVersion); + } + + @Override + public long getFlowRunId() { + GetTimelineCollectorContextResponseProtoOrBuilder p = + viaProto ? proto : builder; + return p.getFlowRunId(); + } + + @Override + public void setFlowRunId(long flowRunId) { + maybeInitBuilder(); + builder.setFlowRunId(flowRunId); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java index 0a9895e6c3a..1ffd223f8a6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java @@ -19,16 +19,26 @@ package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb; import java.util.ArrayList; +import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Set; +import org.apache.hadoop.security.proto.SecurityProtos.TokenProto; +import org.apache.hadoop.yarn.server.api.records.AppCollectorData; +import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.NodeLabel; +import org.apache.hadoop.yarn.api.records.Token; +import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.NodeLabelPBImpl; +import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl; +import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto; import org.apache.hadoop.yarn.proto.YarnProtos.NodeLabelProto; import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto; import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeStatusProto; +import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorDataProto; import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.LogAggregationReportProto; import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatRequestProto; import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatRequestProtoOrBuilder; @@ -52,6 +62,8 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest { private Set labels = null; private List logAggregationReportsForApps = null; + private Map registeringCollectors = null; + public NodeHeartbeatRequestPBImpl() { builder = NodeHeartbeatRequestProto.newBuilder(); } @@ -106,6 +118,9 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest { if (this.logAggregationReportsForApps != null) { addLogAggregationStatusForAppsToProto(); } + if (this.registeringCollectors != null) { + addRegisteringCollectorsToProto(); + } } private void addLogAggregationStatusForAppsToProto() { @@ -147,6 +162,26 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest { return ((LogAggregationReportPBImpl) value).getProto(); } + private void addRegisteringCollectorsToProto() { + maybeInitBuilder(); + builder.clearRegisteringCollectors(); + for (Map.Entry entry : + registeringCollectors.entrySet()) { + AppCollectorData data = entry.getValue(); + AppCollectorDataProto.Builder appCollectorDataBuilder = + AppCollectorDataProto.newBuilder() + .setAppId(convertToProtoFormat(entry.getKey())) + .setAppCollectorAddr(data.getCollectorAddr()) + .setRmIdentifier(data.getRMIdentifier()) + .setVersion(data.getVersion()); + if (data.getCollectorToken() != null) { + appCollectorDataBuilder.setAppCollectorToken( + convertToProtoFormat(data.getCollectorToken())); + } + builder.addRegisteringCollectors(appCollectorDataBuilder); + } + } + private void mergeLocalToProto() { if (viaProto) maybeInitBuilder(); @@ -228,6 +263,45 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest { this.lastKnownNMTokenMasterKey = masterKey; } + @Override + public Map getRegisteringCollectors() { + if (this.registeringCollectors != null) { + return this.registeringCollectors; + } + initRegisteredCollectors(); + return registeringCollectors; + } + + private void initRegisteredCollectors() { + NodeHeartbeatRequestProtoOrBuilder p = viaProto ? proto : builder; + List list = p.getRegisteringCollectorsList(); + if (!list.isEmpty()) { + this.registeringCollectors = new HashMap<>(); + for (AppCollectorDataProto c : list) { + ApplicationId appId = convertFromProtoFormat(c.getAppId()); + Token collectorToken = null; + if (c.hasAppCollectorToken()){ + collectorToken = convertFromProtoFormat(c.getAppCollectorToken()); + } + AppCollectorData data = AppCollectorData.newInstance(appId, + c.getAppCollectorAddr(), c.getRmIdentifier(), c.getVersion(), + collectorToken); + this.registeringCollectors.put(appId, data); + } + } + } + + @Override + public void setRegisteringCollectors( + Map registeredCollectors) { + if (registeredCollectors == null || registeredCollectors.isEmpty()) { + return; + } + maybeInitBuilder(); + this.registeringCollectors = new HashMap<>(); + this.registeringCollectors.putAll(registeredCollectors); + } + private NodeStatusPBImpl convertFromProtoFormat(NodeStatusProto p) { return new NodeStatusPBImpl(p); } @@ -236,6 +310,14 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest { return ((NodeStatusPBImpl)t).getProto(); } + private ApplicationIdPBImpl convertFromProtoFormat(ApplicationIdProto p) { + return new ApplicationIdPBImpl(p); + } + + private ApplicationIdProto convertToProtoFormat(ApplicationId t) { + return ((ApplicationIdPBImpl) t).getProto(); + } + private MasterKeyPBImpl convertFromProtoFormat(MasterKeyProto p) { return new MasterKeyPBImpl(p); } @@ -244,6 +326,14 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest { return ((MasterKeyPBImpl)t).getProto(); } + private TokenPBImpl convertFromProtoFormat(TokenProto p) { + return new TokenPBImpl(p); + } + + private TokenProto convertToProtoFormat(Token t) { + return ((TokenPBImpl) t).getProto(); + } + @Override public Set getNodeLabels() { initNodeLabels(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java index 7ad6b7226fa..72f721db71f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java @@ -26,22 +26,27 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import org.apache.hadoop.security.proto.SecurityProtos.TokenProto; import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SignalContainerRequestPBImpl; +import org.apache.hadoop.yarn.server.api.records.AppCollectorData; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.Token; import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.ProtoBase; import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils; import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl; +import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl; import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto; import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto; import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto; import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto; +import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorDataProto; import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ContainerQueuingLimitProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.SignalContainerRequestProto; import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto; @@ -68,6 +73,7 @@ public class NodeHeartbeatResponsePBImpl extends private List applicationsToCleanup = null; private Map systemCredentials = null; private Resource resource = null; + private Map appCollectorsMap = null; private MasterKey containerTokenMasterKey = null; private MasterKey nmTokenMasterKey = null; @@ -130,6 +136,9 @@ public class NodeHeartbeatResponsePBImpl extends if (this.resource != null) { builder.setResource(convertToProtoFormat(this.resource)); } + if (this.appCollectorsMap != null) { + addAppCollectorsMapToProto(); + } } private void addSystemCredentialsToProto() { @@ -143,6 +152,26 @@ public class NodeHeartbeatResponsePBImpl extends } } + private void addAppCollectorsMapToProto() { + maybeInitBuilder(); + builder.clearAppCollectors(); + for (Map.Entry entry + : appCollectorsMap.entrySet()) { + AppCollectorData data = entry.getValue(); + AppCollectorDataProto.Builder appCollectorDataBuilder = + AppCollectorDataProto.newBuilder() + .setAppId(convertToProtoFormat(entry.getKey())) + .setAppCollectorAddr(data.getCollectorAddr()) + .setRmIdentifier(data.getRMIdentifier()) + .setVersion(data.getVersion()); + if (data.getCollectorToken() != null) { + appCollectorDataBuilder.setAppCollectorToken( + convertToProtoFormat(data.getCollectorToken())); + } + builder.addAppCollectors(appCollectorDataBuilder); + } + } + private void mergeLocalToProto() { if (viaProto) maybeInitBuilder(); @@ -616,6 +645,15 @@ public class NodeHeartbeatResponsePBImpl extends return systemCredentials; } + @Override + public Map getAppCollectors() { + if (this.appCollectorsMap != null) { + return this.appCollectorsMap; + } + initAppCollectorsMap(); + return appCollectorsMap; + } + private void initSystemCredentials() { NodeHeartbeatResponseProtoOrBuilder p = viaProto ? proto : builder; List list = p.getSystemCredentialsForAppsList(); @@ -627,6 +665,25 @@ public class NodeHeartbeatResponsePBImpl extends } } + private void initAppCollectorsMap() { + NodeHeartbeatResponseProtoOrBuilder p = viaProto ? proto : builder; + List list = p.getAppCollectorsList(); + if (!list.isEmpty()) { + this.appCollectorsMap = new HashMap<>(); + for (AppCollectorDataProto c : list) { + ApplicationId appId = convertFromProtoFormat(c.getAppId()); + Token collectorToken = null; + if (c.hasAppCollectorToken()){ + collectorToken = convertFromProtoFormat(c.getAppCollectorToken()); + } + AppCollectorData data = AppCollectorData.newInstance(appId, + c.getAppCollectorAddr(), c.getRmIdentifier(), c.getVersion(), + collectorToken); + this.appCollectorsMap.put(appId, data); + } + } + } + @Override public void setSystemCredentialsForApps( Map systemCredentials) { @@ -638,6 +695,17 @@ public class NodeHeartbeatResponsePBImpl extends this.systemCredentials.putAll(systemCredentials); } + @Override + public void setAppCollectors( + Map appCollectors) { + if (appCollectors == null || appCollectors.isEmpty()) { + return; + } + maybeInitBuilder(); + this.appCollectorsMap = new HashMap<>(); + this.appCollectorsMap.putAll(appCollectors); + } + @Override public long getNextHeartBeatInterval() { NodeHeartbeatResponseProtoOrBuilder p = viaProto ? proto : builder; @@ -790,5 +858,13 @@ public class NodeHeartbeatResponsePBImpl extends SignalContainerRequest t) { return ((SignalContainerRequestPBImpl)t).getProto(); } + + private TokenProto convertToProtoFormat(Token t) { + return ((TokenPBImpl) t).getProto(); + } + + private TokenPBImpl convertFromProtoFormat(TokenProto p) { + return new TokenPBImpl(p); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/ReportNewCollectorInfoRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/ReportNewCollectorInfoRequestPBImpl.java new file mode 100644 index 00000000000..e915beb81fb --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/ReportNewCollectorInfoRequestPBImpl.java @@ -0,0 +1,148 @@ +/** + * 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.protocolrecords.impl.pb; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.yarn.server.api.records.AppCollectorData; +import org.apache.hadoop.yarn.server.api.records.impl.pb.AppCollectorDataPBImpl; +import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorDataProto; +import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewCollectorInfoRequestProto; +import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewCollectorInfoRequestProtoOrBuilder; +import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoRequest; + +/** + * Protocol buffer based implementation of + * {@link ReportNewCollectorInfoRequest}. + */ +public class ReportNewCollectorInfoRequestPBImpl extends + ReportNewCollectorInfoRequest { + + private ReportNewCollectorInfoRequestProto proto = + ReportNewCollectorInfoRequestProto.getDefaultInstance(); + + private ReportNewCollectorInfoRequestProto.Builder builder = null; + private boolean viaProto = false; + + private List collectorsList = null; + + public ReportNewCollectorInfoRequestPBImpl() { + builder = ReportNewCollectorInfoRequestProto.newBuilder(); + } + + public ReportNewCollectorInfoRequestPBImpl( + ReportNewCollectorInfoRequestProto proto) { + this.proto = proto; + viaProto = true; + } + + public ReportNewCollectorInfoRequestProto getProto() { + mergeLocalToProto(); + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other == null) { + return false; + } + if (other.getClass().isAssignableFrom(this.getClass())) { + return this.getProto().equals(this.getClass().cast(other).getProto()); + } + return false; + } + + private void mergeLocalToProto() { + if (viaProto) { + maybeInitBuilder(); + } + mergeLocalToBuilder(); + proto = builder.build(); + viaProto = true; + } + + private void mergeLocalToBuilder() { + if (collectorsList != null) { + addLocalCollectorsToProto(); + } + } + + private void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = ReportNewCollectorInfoRequestProto.newBuilder(proto); + } + viaProto = false; + } + + private void addLocalCollectorsToProto() { + maybeInitBuilder(); + builder.clearAppCollectors(); + List protoList = + new ArrayList(); + for (AppCollectorData m : this.collectorsList) { + protoList.add(convertToProtoFormat(m)); + } + builder.addAllAppCollectors(protoList); + } + + private void initLocalCollectorsList() { + ReportNewCollectorInfoRequestProtoOrBuilder p = viaProto ? proto : builder; + List list = + p.getAppCollectorsList(); + this.collectorsList = new ArrayList(); + for (AppCollectorDataProto m : list) { + this.collectorsList.add(convertFromProtoFormat(m)); + } + } + + @Override + public List getAppCollectorsList() { + if (this.collectorsList == null) { + initLocalCollectorsList(); + } + return this.collectorsList; + } + + @Override + public void setAppCollectorsList(List appCollectorsList) { + maybeInitBuilder(); + if (appCollectorsList == null) { + builder.clearAppCollectors(); + } + this.collectorsList = appCollectorsList; + } + + private AppCollectorDataPBImpl convertFromProtoFormat( + AppCollectorDataProto p) { + return new AppCollectorDataPBImpl(p); + } + + private AppCollectorDataProto convertToProtoFormat( + AppCollectorData m) { + return ((AppCollectorDataPBImpl) m).getProto(); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/ReportNewCollectorInfoResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/ReportNewCollectorInfoResponsePBImpl.java new file mode 100644 index 00000000000..782d445d223 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/ReportNewCollectorInfoResponsePBImpl.java @@ -0,0 +1,80 @@ +/** + * 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.protocolrecords.impl.pb; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewCollectorInfoResponseProto; +import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoResponse; + +import com.google.protobuf.TextFormat; + +/** + * Protocol buffer based implementation of + * {@link ReportNewCollectorInfoResponse}. + */ +@Private +@Unstable +public class ReportNewCollectorInfoResponsePBImpl extends + ReportNewCollectorInfoResponse { + + private ReportNewCollectorInfoResponseProto proto = + ReportNewCollectorInfoResponseProto.getDefaultInstance(); + + private ReportNewCollectorInfoResponseProto.Builder builder = null; + + private boolean viaProto = false; + + public ReportNewCollectorInfoResponsePBImpl() { + builder = ReportNewCollectorInfoResponseProto.newBuilder(); + } + + public ReportNewCollectorInfoResponsePBImpl( + ReportNewCollectorInfoResponseProto proto) { + this.proto = proto; + viaProto = true; + } + + public ReportNewCollectorInfoResponseProto getProto() { + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other == null) { + return false; + } + if (other.getClass().isAssignableFrom(this.getClass())) { + return this.getProto().equals(this.getClass().cast(other).getProto()); + } + return false; + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/package-info.java new file mode 100644 index 00000000000..843f24b112a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * Contains protocol record implementations for YARN server protocols. + */ +package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb; \ 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/api/records/AppCollectorData.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/AppCollectorData.java new file mode 100644 index 00000000000..2e1dc9a0314 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/AppCollectorData.java @@ -0,0 +1,128 @@ +/** + * 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.records; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.Token; +import org.apache.hadoop.yarn.util.Records; + +/** + * Collector data associated with an app collector. Includes app Id, collector + * address, RM identifier, version and collector token. + */ +@Private +@InterfaceStability.Unstable +public abstract class AppCollectorData { + + protected static final long DEFAULT_TIMESTAMP_VALUE = -1; + + public static AppCollectorData newInstance( + ApplicationId id, String collectorAddr, long rmIdentifier, long version, + Token token) { + AppCollectorData appCollectorData = + Records.newRecord(AppCollectorData.class); + appCollectorData.setApplicationId(id); + appCollectorData.setCollectorAddr(collectorAddr); + appCollectorData.setRMIdentifier(rmIdentifier); + appCollectorData.setVersion(version); + appCollectorData.setCollectorToken(token); + return appCollectorData; + } + + public static AppCollectorData newInstance( + ApplicationId id, String collectorAddr, long rmIdentifier, long version) { + return newInstance(id, collectorAddr, rmIdentifier, version, null); + } + + public static AppCollectorData newInstance(ApplicationId id, + String collectorAddr, Token token) { + return newInstance(id, collectorAddr, DEFAULT_TIMESTAMP_VALUE, + DEFAULT_TIMESTAMP_VALUE, token); + } + + public static AppCollectorData newInstance(ApplicationId id, + String collectorAddr) { + return newInstance(id, collectorAddr, null); + } + + /** + * Returns if a collector data item happens before another one. Null data + * items happens before any other non-null items. Non-null data items A + * happens before another non-null item B when A's rmIdentifier is less than + * B's rmIdentifier. Or A's version is less than B's if they have the same + * rmIdentifier. + * + * @param dataA first collector data item. + * @param dataB second collector data item. + * @return true if dataA happens before dataB. + */ + public static boolean happensBefore(AppCollectorData dataA, + AppCollectorData dataB) { + if (dataA == null && dataB == null) { + return false; + } else if (dataA == null || dataB == null) { + return dataA == null; + } + + return + (dataA.getRMIdentifier() < dataB.getRMIdentifier()) + || ((dataA.getRMIdentifier() == dataB.getRMIdentifier()) + && (dataA.getVersion() < dataB.getVersion())); + } + + /** + * Returns if the collector data has been stamped by the RM with a RM cluster + * timestamp and a version number. + * + * @return true if RM has already assigned a timestamp for this collector. + * Otherwise, it means the RM has not recognized the existence of this + * collector. + */ + public boolean isStamped() { + return (getRMIdentifier() != DEFAULT_TIMESTAMP_VALUE) + || (getVersion() != DEFAULT_TIMESTAMP_VALUE); + } + + public abstract ApplicationId getApplicationId(); + + public abstract void setApplicationId(ApplicationId id); + + public abstract String getCollectorAddr(); + + public abstract void setCollectorAddr(String addr); + + public abstract long getRMIdentifier(); + + public abstract void setRMIdentifier(long rmId); + + public abstract long getVersion(); + + public abstract void setVersion(long version); + + /** + * Get delegation token for app collector which AM will use to publish + * entities. + * @return the delegation token for app collector. + */ + public abstract Token getCollectorToken(); + + public abstract void setCollectorToken(Token token); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/AppCollectorDataPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/AppCollectorDataPBImpl.java new file mode 100644 index 00000000000..f87f7f66c1f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/AppCollectorDataPBImpl.java @@ -0,0 +1,230 @@ +/** +* 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.records.impl.pb; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.server.api.records.AppCollectorData; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.Token; +import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl; +import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl; +import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto; +import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorDataProto; +import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorDataProtoOrBuilder; + +import com.google.protobuf.TextFormat; + +/** + * Protocol Buffer based implementation of {@link AppCollectorData}. + */ +@Private +@Unstable +public class AppCollectorDataPBImpl extends AppCollectorData { + + private AppCollectorDataProto proto = + AppCollectorDataProto.getDefaultInstance(); + + private AppCollectorDataProto.Builder builder = null; + private boolean viaProto = false; + + private ApplicationId appId = null; + private String collectorAddr = null; + private Long rmIdentifier = null; + private Long version = null; + private Token collectorToken = null; + + public AppCollectorDataPBImpl() { + builder = AppCollectorDataProto.newBuilder(); + } + + public AppCollectorDataPBImpl(AppCollectorDataProto proto) { + this.proto = proto; + viaProto = true; + } + + public AppCollectorDataProto getProto() { + mergeLocalToProto(); + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other == null) { + return false; + } + if (other.getClass().isAssignableFrom(this.getClass())) { + return this.getProto().equals(this.getClass().cast(other).getProto()); + } + return false; + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } + + @Override + public ApplicationId getApplicationId() { + AppCollectorDataProtoOrBuilder p = viaProto ? proto : builder; + if (this.appId == null && p.hasAppId()) { + this.appId = convertFromProtoFormat(p.getAppId()); + } + return this.appId; + } + + @Override + public String getCollectorAddr() { + AppCollectorDataProtoOrBuilder p = viaProto ? proto : builder; + if (this.collectorAddr == null + && p.hasAppCollectorAddr()) { + this.collectorAddr = p.getAppCollectorAddr(); + } + return this.collectorAddr; + } + + @Override + public void setApplicationId(ApplicationId id) { + maybeInitBuilder(); + if (id == null) { + builder.clearAppId(); + } + this.appId = id; + } + + @Override + public void setCollectorAddr(String collectorAddr) { + maybeInitBuilder(); + if (collectorAddr == null) { + builder.clearAppCollectorAddr(); + } + this.collectorAddr = collectorAddr; + } + + @Override + public long getRMIdentifier() { + AppCollectorDataProtoOrBuilder p = viaProto ? proto : builder; + if (this.rmIdentifier == null && p.hasRmIdentifier()) { + this.rmIdentifier = p.getRmIdentifier(); + } + if (this.rmIdentifier != null) { + return this.rmIdentifier; + } else { + return AppCollectorData.DEFAULT_TIMESTAMP_VALUE; + } + } + + @Override + public void setRMIdentifier(long rmId) { + maybeInitBuilder(); + this.rmIdentifier = rmId; + builder.setRmIdentifier(rmId); + } + + @Override + public long getVersion() { + AppCollectorDataProtoOrBuilder p = viaProto ? proto : builder; + if (this.version == null && p.hasRmIdentifier()) { + this.version = p.getRmIdentifier(); + } + if (this.version != null) { + return this.version; + } else { + return AppCollectorData.DEFAULT_TIMESTAMP_VALUE; + } + } + + @Override + public void setVersion(long version) { + maybeInitBuilder(); + this.version = version; + builder.setVersion(version); + } + + @Override + public Token getCollectorToken() { + AppCollectorDataProtoOrBuilder p = viaProto ? proto : builder; + if (this.collectorToken != null) { + return this.collectorToken; + } + if (!p.hasAppCollectorToken()) { + return null; + } + this.collectorToken = new TokenPBImpl(p.getAppCollectorToken()); + return this.collectorToken; + } + + @Override + public void setCollectorToken(Token token) { + maybeInitBuilder(); + if (token == null) { + builder.clearAppCollectorToken(); + } + this.collectorToken = token; + } + + private ApplicationIdPBImpl convertFromProtoFormat(ApplicationIdProto p) { + return new ApplicationIdPBImpl(p); + } + + private ApplicationIdProto convertToProtoFormat(ApplicationId t) { + return ((ApplicationIdPBImpl) t).getProto(); + } + + private void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = AppCollectorDataProto.newBuilder(proto); + } + viaProto = false; + } + + private void mergeLocalToProto() { + if (viaProto) { + maybeInitBuilder(); + } + mergeLocalToBuilder(); + proto = builder.build(); + viaProto = true; + } + + private void mergeLocalToBuilder() { + if (this.appId != null) { + builder.setAppId(convertToProtoFormat(this.appId)); + } + if (this.collectorAddr != null) { + builder.setAppCollectorAddr(this.collectorAddr); + } + if (this.rmIdentifier != null) { + builder.setRmIdentifier(this.rmIdentifier); + } + if (this.version != null) { + builder.setVersion(this.version); + } + if (this.collectorToken != null) { + builder.setAppCollectorToken( + ((TokenPBImpl)this.collectorToken).getProto()); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/package-info.java new file mode 100644 index 00000000000..4ce3896fbb9 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/package-info.java @@ -0,0 +1,19 @@ +/* + * 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. + */ +/** Server records PB implementations. */ +package org.apache.hadoop.yarn.server.api.records.impl.pb; \ 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/metrics/AppAttemptMetricsConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/AppAttemptMetricsConstants.java index a7809cfe2c1..2f61f43a255 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/AppAttemptMetricsConstants.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/AppAttemptMetricsConstants.java @@ -37,28 +37,28 @@ public class AppAttemptMetricsConstants { public static final String PARENT_PRIMARY_FILTER = "YARN_APPLICATION_ATTEMPT_PARENT"; - public static final String TRACKING_URL_EVENT_INFO = + public static final String TRACKING_URL_INFO = "YARN_APPLICATION_ATTEMPT_TRACKING_URL"; - public static final String ORIGINAL_TRACKING_URL_EVENT_INFO = + public static final String ORIGINAL_TRACKING_URL_INFO = "YARN_APPLICATION_ATTEMPT_ORIGINAL_TRACKING_URL"; - public static final String HOST_EVENT_INFO = + public static final String HOST_INFO = "YARN_APPLICATION_ATTEMPT_HOST"; - public static final String RPC_PORT_EVENT_INFO = + public static final String RPC_PORT_INFO = "YARN_APPLICATION_ATTEMPT_RPC_PORT"; - public static final String MASTER_CONTAINER_EVENT_INFO = + public static final String MASTER_CONTAINER_INFO = "YARN_APPLICATION_ATTEMPT_MASTER_CONTAINER"; - public static final String DIAGNOSTICS_INFO_EVENT_INFO = + public static final String DIAGNOSTICS_INFO = "YARN_APPLICATION_ATTEMPT_DIAGNOSTICS_INFO"; - public static final String FINAL_STATUS_EVENT_INFO = + public static final String FINAL_STATUS_INFO = "YARN_APPLICATION_ATTEMPT_FINAL_STATUS"; - public static final String STATE_EVENT_INFO = + public static final String STATE_INFO = "YARN_APPLICATION_ATTEMPT_STATE"; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ApplicationMetricsConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ApplicationMetricsConstants.java index 2eac0147432..4cec409bb41 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ApplicationMetricsConstants.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ApplicationMetricsConstants.java @@ -71,10 +71,22 @@ public class ApplicationMetricsConstants { "YARN_APPLICATION_STATE"; public static final String APP_CPU_METRICS = - "YARN_APPLICATION_CPU_METRIC"; + "YARN_APPLICATION_CPU"; public static final String APP_MEM_METRICS = - "YARN_APPLICATION_MEM_METRIC"; + "YARN_APPLICATION_MEMORY"; + + public static final String APP_RESOURCE_PREEMPTED_CPU = + "YARN_APPLICATION_RESOURCE_PREEMPTED_CPU"; + + public static final String APP_RESOURCE_PREEMPTED_MEM = + "YARN_APPLICATION_RESOURCE_PREEMPTED_MEMORY"; + + public static final String APP_NON_AM_CONTAINER_PREEMPTED = + "YARN_APPLICATION_NON_AM_CONTAINER_PREEMPTED"; + + public static final String APP_AM_CONTAINER_PREEMPTED = + "YARN_APPLICATION_AM_CONTAINER_PREEMPTED"; public static final String APP_CPU_PREEMPT_METRICS = "YARN_APPLICATION_CPU_PREEMPT_METRIC"; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ContainerMetricsConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ContainerMetricsConstants.java index 0d5540df6bc..9cf2b0abac5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ContainerMetricsConstants.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ContainerMetricsConstants.java @@ -27,36 +27,57 @@ public class ContainerMetricsConstants { public static final String ENTITY_TYPE = "YARN_CONTAINER"; + // Event of this type will be emitted by NM. public static final String CREATED_EVENT_TYPE = "YARN_CONTAINER_CREATED"; + // Event of this type will be emitted by RM. + public static final String CREATED_IN_RM_EVENT_TYPE = + "YARN_RM_CONTAINER_CREATED"; + + // Event of this type will be emitted by NM. public static final String FINISHED_EVENT_TYPE = "YARN_CONTAINER_FINISHED"; + // Event of this type will be emitted by RM. + public static final String FINISHED_IN_RM_EVENT_TYPE = + "YARN_RM_CONTAINER_FINISHED"; + + public static final String CONTAINER_FINISHED_TIME = + "YARN_CONTAINER_FINISHED_TIME"; + public static final String PARENT_PRIMARIY_FILTER = "YARN_CONTAINER_PARENT"; - public static final String ALLOCATED_MEMORY_ENTITY_INFO = + public static final String ALLOCATED_MEMORY_INFO = "YARN_CONTAINER_ALLOCATED_MEMORY"; - public static final String ALLOCATED_VCORE_ENTITY_INFO = + public static final String ALLOCATED_VCORE_INFO = "YARN_CONTAINER_ALLOCATED_VCORE"; - public static final String ALLOCATED_HOST_ENTITY_INFO = + public static final String ALLOCATED_HOST_INFO = "YARN_CONTAINER_ALLOCATED_HOST"; - public static final String ALLOCATED_PORT_ENTITY_INFO = + public static final String ALLOCATED_PORT_INFO = "YARN_CONTAINER_ALLOCATED_PORT"; - public static final String ALLOCATED_PRIORITY_ENTITY_INFO = + public static final String ALLOCATED_PRIORITY_INFO = "YARN_CONTAINER_ALLOCATED_PRIORITY"; - public static final String DIAGNOSTICS_INFO_EVENT_INFO = + public static final String DIAGNOSTICS_INFO = "YARN_CONTAINER_DIAGNOSTICS_INFO"; - public static final String EXIT_STATUS_EVENT_INFO = + public static final String EXIT_STATUS_INFO = "YARN_CONTAINER_EXIT_STATUS"; - public static final String STATE_EVENT_INFO = + public static final String STATE_INFO = "YARN_CONTAINER_STATE"; - public static final String ALLOCATED_HOST_HTTP_ADDRESS_ENTITY_INFO = + public static final String ALLOCATED_HOST_HTTP_ADDRESS_INFO = "YARN_CONTAINER_ALLOCATED_HOST_HTTP_ADDRESS"; + + // Event of this type will be emitted by NM. + public static final String LOCALIZATION_START_EVENT_TYPE = + "YARN_NM_CONTAINER_LOCALIZATION_STARTED"; + + // Event of this type will be emitted by NM. + public static final String LOCALIZATION_FINISHED_EVENT_TYPE = + "YARN_NM_CONTAINER_LOCALIZATION_FINISHED"; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilter.java similarity index 69% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilter.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilter.java index ad8dc2c28dc..f6d1863cfc3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilter.java @@ -23,27 +23,33 @@ import javax.servlet.ServletException; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager; import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationFilter; -import org.apache.hadoop.yarn.server.timeline.security.TimelineDelegationTokenSecretManagerService.TimelineDelegationTokenSecretManager; +import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier; +/** + * Timeline authentication filter provides delegation token support for ATSv1 + * and ATSv2. + */ @Private @Unstable public class TimelineAuthenticationFilter extends DelegationTokenAuthenticationFilter { - private static TimelineDelegationTokenSecretManager secretManager; + private static AbstractDelegationTokenSecretManager + secretManager; @Override public void init(FilterConfig filterConfig) throws ServletException { filterConfig.getServletContext().setAttribute( - DelegationTokenAuthenticationFilter.DELEGATION_TOKEN_SECRET_MANAGER_ATTR, - secretManager); + DelegationTokenAuthenticationFilter. + DELEGATION_TOKEN_SECRET_MANAGER_ATTR, secretManager); super.init(filterConfig); } public static void setTimelineDelegationTokenSecretManager( - TimelineDelegationTokenSecretManager secretManager) { - TimelineAuthenticationFilter.secretManager = secretManager; + AbstractDelegationTokenSecretManager + secretMgr) { + TimelineAuthenticationFilter.secretManager = secretMgr; } - } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java similarity index 81% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java index 4e7c29ad585..eb557a97eca 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java @@ -51,30 +51,19 @@ import java.util.Map; public class TimelineAuthenticationFilterInitializer extends FilterInitializer { /** - * The configuration prefix of timeline HTTP authentication + * The configuration prefix of timeline HTTP authentication. */ - public static final String PREFIX = "yarn.timeline-service.http-authentication."; + public static final String PREFIX = + "yarn.timeline-service.http-authentication."; @VisibleForTesting Map filterConfig; - /** - * Initializes {@link TimelineAuthenticationFilter} - *

- * Propagates to {@link TimelineAuthenticationFilter} configuration all YARN - * configuration properties prefixed with {@value #PREFIX} - * - * @param container - * The filter container - * @param conf - * Configuration for run-time parameters - */ - @Override - public void initFilter(FilterContainer container, Configuration conf) { + protected void setAuthFilterConfig(Configuration conf) { filterConfig = new HashMap(); // setting the cookie path to root '/' so it is used for all resources. - filterConfig.put(TimelineAuthenticationFilter.COOKIE_PATH, "/"); + filterConfig.put(AuthenticationFilter.COOKIE_PATH, "/"); for (Map.Entry entry : conf) { String name = entry.getKey(); @@ -95,6 +84,41 @@ public class TimelineAuthenticationFilterInitializer extends FilterInitializer { } } + // Resolve _HOST into bind address + String bindAddress = conf.get(HttpServer2.BIND_ADDRESS); + String principal = + filterConfig.get(KerberosAuthenticationHandler.PRINCIPAL); + if (principal != null) { + try { + principal = SecurityUtil.getServerPrincipal(principal, bindAddress); + } catch (IOException ex) { + throw new RuntimeException("Could not resolve Kerberos principal " + + "name: " + ex.toString(), ex); + } + filterConfig.put(KerberosAuthenticationHandler.PRINCIPAL, + principal); + } + } + + protected Map getFilterConfig() { + return filterConfig; + } + + /** + * Initializes {@link TimelineAuthenticationFilter}. + *

+ * Propagates to {@link TimelineAuthenticationFilter} configuration all YARN + * configuration properties prefixed with {@value #PREFIX} + * + * @param container + * The filter container + * @param conf + * Configuration for run-time parameters + */ + @Override + public void initFilter(FilterContainer container, Configuration conf) { + setAuthFilterConfig(conf); + String authType = filterConfig.get(AuthenticationFilter.AUTH_TYPE); if (authType.equals(PseudoAuthenticationHandler.TYPE)) { filterConfig.put(AuthenticationFilter.AUTH_TYPE, @@ -102,23 +126,7 @@ public class TimelineAuthenticationFilterInitializer extends FilterInitializer { } else if (authType.equals(KerberosAuthenticationHandler.TYPE)) { filterConfig.put(AuthenticationFilter.AUTH_TYPE, KerberosDelegationTokenAuthenticationHandler.class.getName()); - - // Resolve _HOST into bind address - String bindAddress = conf.get(HttpServer2.BIND_ADDRESS); - String principal = - filterConfig.get(KerberosAuthenticationHandler.PRINCIPAL); - if (principal != null) { - try { - principal = SecurityUtil.getServerPrincipal(principal, bindAddress); - } catch (IOException ex) { - throw new RuntimeException( - "Could not resolve Kerberos principal name: " + ex.toString(), ex); - } - filterConfig.put(KerberosAuthenticationHandler.PRINCIPAL, - principal); - } } - filterConfig.put(DelegationTokenAuthenticationHandler.TOKEN_KIND, TimelineDelegationTokenIdentifier.KIND_NAME.toString()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelgationTokenSecretManagerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelgationTokenSecretManagerService.java new file mode 100644 index 00000000000..2e95af2cdd0 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelgationTokenSecretManagerService.java @@ -0,0 +1,83 @@ +/** + * 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.timeline.security; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier; + +/** + * Abstract implementation of delegation token manager service for different + * versions of timeline service. + */ +public abstract class TimelineDelgationTokenSecretManagerService extends + AbstractService { + + public TimelineDelgationTokenSecretManagerService(String name) { + super(name); + } + + private static long delegationTokenRemovalScanInterval = 3600000L; + + private AbstractDelegationTokenSecretManager + secretManager = null; + + @Override + protected void serviceInit(Configuration conf) throws Exception { + long secretKeyInterval = + conf.getLong(YarnConfiguration.TIMELINE_DELEGATION_KEY_UPDATE_INTERVAL, + YarnConfiguration.DEFAULT_TIMELINE_DELEGATION_KEY_UPDATE_INTERVAL); + long tokenMaxLifetime = + conf.getLong(YarnConfiguration.TIMELINE_DELEGATION_TOKEN_MAX_LIFETIME, + YarnConfiguration.DEFAULT_TIMELINE_DELEGATION_TOKEN_MAX_LIFETIME); + long tokenRenewInterval = + conf.getLong(YarnConfiguration.TIMELINE_DELEGATION_TOKEN_RENEW_INTERVAL, + YarnConfiguration.DEFAULT_TIMELINE_DELEGATION_TOKEN_RENEW_INTERVAL); + secretManager = createTimelineDelegationTokenSecretManager( + secretKeyInterval, tokenMaxLifetime, tokenRenewInterval, + delegationTokenRemovalScanInterval); + super.init(conf); + } + + protected abstract + AbstractDelegationTokenSecretManager + createTimelineDelegationTokenSecretManager(long secretKeyInterval, + long tokenMaxLifetime, long tokenRenewInterval, + long tokenRemovalScanInterval); + + @Override + protected void serviceStart() throws Exception { + secretManager.startThreads(); + super.serviceStart(); + } + + @Override + protected void serviceStop() throws Exception { + secretManager.stopThreads(); + super.stop(); + } + + public AbstractDelegationTokenSecretManager + + getTimelineDelegationTokenSecretManager() { + return secretManager; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/package-info.java new file mode 100644 index 00000000000..14a52e342b3 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/package-info.java @@ -0,0 +1,26 @@ +/** + * 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.server.timeline.security contains classes related + * to timeline authentication filters and abstract delegation token service for + * ATSv1 and ATSv2. + */ +@InterfaceAudience.Private +package org.apache.hadoop.yarn.server.timeline.security; +import org.apache.hadoop.classification.InterfaceAudience; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/util/timeline/TimelineServerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/util/timeline/TimelineServerUtils.java new file mode 100644 index 00000000000..78bf20f8ccd --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/util/timeline/TimelineServerUtils.java @@ -0,0 +1,92 @@ +/** + * 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.util.timeline; + +import java.util.LinkedHashSet; +import java.util.Set; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.AuthenticationFilterInitializer; +import org.apache.hadoop.yarn.server.timeline.security.TimelineAuthenticationFilter; +import org.apache.hadoop.yarn.server.timeline.security.TimelineAuthenticationFilterInitializer; +import org.apache.hadoop.yarn.server.timeline.security.TimelineDelgationTokenSecretManagerService; + +/** + * Set of utility methods to be used across timeline reader and collector. + */ +public final class TimelineServerUtils { + private static final Log LOG = LogFactory.getLog(TimelineServerUtils.class); + + private TimelineServerUtils() { + } + + /** + * Sets filter initializers configuration based on existing configuration and + * default filters added by timeline service(such as timeline auth filter and + * CORS filter). + * @param conf Configuration object. + * @param configuredInitializers Comma separated list of filter initializers. + * @param defaultInitializers Set of initializers added by default by timeline + * service. + */ + public static void setTimelineFilters(Configuration conf, + String configuredInitializers, Set defaultInitializers) { + String[] parts = configuredInitializers.split(","); + Set target = new LinkedHashSet(); + for (String filterInitializer : parts) { + filterInitializer = filterInitializer.trim(); + if (filterInitializer.equals( + AuthenticationFilterInitializer.class.getName()) || + filterInitializer.isEmpty()) { + continue; + } + target.add(filterInitializer); + } + target.addAll(defaultInitializers); + String actualInitializers = + org.apache.commons.lang.StringUtils.join(target, ","); + LOG.info("Filter initializers set for timeline service: " + + actualInitializers); + conf.set("hadoop.http.filter.initializers", actualInitializers); + } + + /** + * Adds timeline authentication filter to the set of default filter + * initializers and assigns the delegation token manager service to it. + * @param initializers Comma separated list of filter initializers. + * @param defaultInitializers Set of initializers added by default by timeline + * service. + * @param delegationTokenMgrService Delegation token manager service. + * This will be used by timeline authentication filter to assign + * delegation tokens. + */ + public static void addTimelineAuthFilter(String initializers, + Set defaultInitializers, + TimelineDelgationTokenSecretManagerService delegationTokenMgrService) { + TimelineAuthenticationFilter.setTimelineDelegationTokenSecretManager( + delegationTokenMgrService.getTimelineDelegationTokenSecretManager()); + if (!initializers.contains( + TimelineAuthenticationFilterInitializer.class.getName())) { + defaultInitializers.add( + TimelineAuthenticationFilterInitializer.class.getName()); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/util/timeline/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/util/timeline/package-info.java new file mode 100644 index 00000000000..75c69738c50 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/util/timeline/package-info.java @@ -0,0 +1,25 @@ +/** + * 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.server.util.timeline contains utility classes used + * by ATSv1 and ATSv2 on the server side. + */ +@InterfaceAudience.Private +package org.apache.hadoop.yarn.server.util.timeline; +import org.apache.hadoop.classification.InterfaceAudience; \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/collectornodemanager_protocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/collectornodemanager_protocol.proto new file mode 100644 index 00000000000..866527410c9 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/collectornodemanager_protocol.proto @@ -0,0 +1,30 @@ +/** + * 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. + */ + +option java_package = "org.apache.hadoop.yarn.proto"; +option java_outer_classname = "CollectorNodemanagerProtocol"; +option java_generic_services = true; +option java_generate_equals_and_hash = true; +package hadoop.yarn; + +import "yarn_server_common_service_protos.proto"; + +service CollectorNodemanagerProtocolService { + rpc reportNewCollectorInfo (ReportNewCollectorInfoRequestProto) returns (ReportNewCollectorInfoResponseProto); + rpc getTimelineCollectorContext (GetTimelineCollectorContextRequestProto) returns (GetTimelineCollectorContextResponseProto); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto index 8118936fdae..3072867efc3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto @@ -22,6 +22,7 @@ option java_generic_services = true; option java_generate_equals_and_hash = true; package hadoop.yarn; +import "Security.proto"; import "yarn_protos.proto"; import "yarn_server_common_protos.proto"; import "yarn_service_protos.proto"; @@ -91,6 +92,7 @@ message NodeHeartbeatRequestProto { optional MasterKeyProto last_known_nm_token_master_key = 3; optional NodeLabelsProto nodeLabels = 4; repeated LogAggregationReportProto log_aggregation_reports_for_apps = 5; + repeated AppCollectorDataProto registering_collectors = 6; } message LogAggregationReportProto { @@ -118,6 +120,7 @@ message NodeHeartbeatResponseProto { optional ContainerQueuingLimitProto container_queuing_limit = 15; // to be used in place of containers_to_decrease repeated ContainerProto containers_to_update = 17; + repeated AppCollectorDataProto app_collectors = 18; } message ContainerQueuingLimitProto { @@ -130,6 +133,38 @@ message SystemCredentialsForAppsProto { optional bytes credentialsForApp = 2; } +//////////////////////////////////////////////////////////////////////// +////// From collector_nodemanager_protocol //////////////////////////// +//////////////////////////////////////////////////////////////////////// +message AppCollectorDataProto { + optional ApplicationIdProto app_id = 1; + optional string app_collector_addr = 2; + optional int64 rm_identifier = 3 [default = -1]; + optional int64 version = 4 [default = -1]; + optional hadoop.common.TokenProto app_collector_token = 5; +} + +////////////////////////////////////////////////////// +/////// collector_nodemanager_protocol ////////////// +////////////////////////////////////////////////////// +message ReportNewCollectorInfoRequestProto { + repeated AppCollectorDataProto app_collectors = 1; +} + +message ReportNewCollectorInfoResponseProto { +} + +message GetTimelineCollectorContextRequestProto { + optional ApplicationIdProto appId = 1; +} + +message GetTimelineCollectorContextResponseProto { + optional string user_id = 1; + optional string flow_name = 2; + optional string flow_version = 3; + optional int64 flow_run_id = 4; +} + message NMContainerStatusProto { optional ContainerIdProto container_id = 1; optional ContainerStateProto container_state = 2; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java similarity index 58% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java index f7dbd3a557d..5fbf4128c14 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java @@ -30,6 +30,7 @@ import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.Server; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.util.Time; import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB; @@ -72,16 +73,50 @@ import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC; import org.apache.hadoop.yarn.ipc.RPCUtil; import org.apache.hadoop.yarn.ipc.YarnRPC; import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; +import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier; +import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocol; +import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoResponse; +import org.apache.hadoop.yarn.server.api.records.AppCollectorData; import org.apache.hadoop.yarn.util.Records; import org.junit.Assert; import org.junit.Test; +/** + * Tests RPC communication for different YARN server protocols. + */ public class TestRPC { private static final String EXCEPTION_MSG = "test error"; private static final String EXCEPTION_CAUSE = "exception cause"; - private static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); - + private static final RecordFactory RECORD_FACTORY = + RecordFactoryProvider.getRecordFactory(null); + + public static final String ILLEGAL_NUMBER_MESSAGE = + "collectors' number in ReportNewCollectorInfoRequest is not ONE."; + + public static final String DEFAULT_COLLECTOR_ADDR = "localhost:0"; + private static final Token DEFAULT_COLLECTOR_TOKEN; + static { + TimelineDelegationTokenIdentifier identifier = + new TimelineDelegationTokenIdentifier(); + identifier.setOwner(new Text("user")); + identifier.setRenewer(new Text("user")); + identifier.setRealUser(new Text("user")); + long now = Time.now(); + identifier.setIssueDate(now); + identifier.setMaxDate(now + 1000L); + identifier.setMasterKeyId(500); + identifier.setSequenceNumber(5); + DEFAULT_COLLECTOR_TOKEN = Token.newInstance(identifier.getBytes(), + identifier.getKind().toString(), identifier.getBytes(), "localhost:0"); + } + + public static final ApplicationId DEFAULT_APP_ID = + ApplicationId.newInstance(0, 0); + @Test public void testUnknownCall() { Configuration conf = new Configuration(); @@ -96,7 +131,8 @@ public class TestRPC { // Any unrelated protocol would do ApplicationClientProtocol proxy = (ApplicationClientProtocol) rpc.getProxy( - ApplicationClientProtocol.class, NetUtils.getConnectAddress(server), conf); + ApplicationClientProtocol.class, NetUtils.getConnectAddress(server), + conf); try { proxy.getNewApplication(Records @@ -106,32 +142,133 @@ public class TestRPC { Assert.assertTrue(e.getMessage().matches( "Unknown method getNewApplication called on.*" + "org.apache.hadoop.yarn.proto.ApplicationClientProtocol" - + "\\$ApplicationClientProtocolService\\$BlockingInterface protocol.")); + + "\\$ApplicationClientProtocolService\\$BlockingInterface " + + "protocol.")); + } catch (Exception e) { + e.printStackTrace(); + } finally { + server.stop(); + } + } + + @Test + public void testRPCOnCollectorNodeManagerProtocol() throws IOException { + Configuration conf = new Configuration(); + conf.set(YarnConfiguration.IPC_RPC_IMPL, HadoopYarnProtoRPC.class + .getName()); + YarnRPC rpc = YarnRPC.create(conf); + String bindAddr = "localhost:0"; + InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr); + Server server = rpc.getServer(CollectorNodemanagerProtocol.class, + new DummyNMCollectorService(), addr, conf, null, 1); + server.start(); + + // Test unrelated protocol wouldn't get response + ApplicationClientProtocol unknownProxy = + (ApplicationClientProtocol) rpc.getProxy( + ApplicationClientProtocol.class, NetUtils.getConnectAddress(server), + conf); + + try { + unknownProxy.getNewApplication(Records + .newRecord(GetNewApplicationRequest.class)); + Assert.fail("Excepted RPC call to fail with unknown method."); + } catch (YarnException e) { + Assert.assertTrue(e.getMessage().matches( + "Unknown method getNewApplication called on.*" + + "org.apache.hadoop.yarn.proto.ApplicationClientProtocol" + + "\\$ApplicationClientProtocolService\\$BlockingInterface " + + "protocol.")); } catch (Exception e) { e.printStackTrace(); } + + // Test CollectorNodemanagerProtocol get proper response + CollectorNodemanagerProtocol proxy = + (CollectorNodemanagerProtocol)rpc.getProxy( + CollectorNodemanagerProtocol.class, NetUtils.getConnectAddress(server), + conf); + // Verify request with DEFAULT_APP_ID and DEFAULT_COLLECTOR_ADDR get + // normally response. + try { + ReportNewCollectorInfoRequest request = + ReportNewCollectorInfoRequest.newInstance( + DEFAULT_APP_ID, DEFAULT_COLLECTOR_ADDR, null); + proxy.reportNewCollectorInfo(request); + } catch (YarnException e) { + Assert.fail("RPC call failured is not expected here."); + } + + try { + ReportNewCollectorInfoRequest request = + ReportNewCollectorInfoRequest.newInstance( + DEFAULT_APP_ID, DEFAULT_COLLECTOR_ADDR, DEFAULT_COLLECTOR_TOKEN); + proxy.reportNewCollectorInfo(request); + } catch (YarnException e) { + Assert.fail("RPC call failured is not expected here."); + } + + // Verify empty request get YarnException back (by design in + // DummyNMCollectorService) + try { + proxy.reportNewCollectorInfo(Records + .newRecord(ReportNewCollectorInfoRequest.class)); + Assert.fail("Excepted RPC call to fail with YarnException."); + } catch (YarnException e) { + Assert.assertTrue(e.getMessage().contains(ILLEGAL_NUMBER_MESSAGE)); + } + + // Verify request with a valid app ID + try { + GetTimelineCollectorContextRequest request = + GetTimelineCollectorContextRequest.newInstance( + ApplicationId.newInstance(0, 1)); + GetTimelineCollectorContextResponse response = + proxy.getTimelineCollectorContext(request); + Assert.assertEquals("test_user_id", response.getUserId()); + Assert.assertEquals("test_flow_name", response.getFlowName()); + Assert.assertEquals("test_flow_version", response.getFlowVersion()); + Assert.assertEquals(12345678L, response.getFlowRunId()); + } catch (YarnException | IOException e) { + Assert.fail("RPC call failured is not expected here."); + } + + // Verify request with an invalid app ID + try { + GetTimelineCollectorContextRequest request = + GetTimelineCollectorContextRequest.newInstance( + ApplicationId.newInstance(0, 2)); + proxy.getTimelineCollectorContext(request); + Assert.fail("RPC call failured is expected here."); + } catch (YarnException | IOException e) { + Assert.assertTrue(e instanceof YarnException); + Assert.assertTrue(e.getMessage().contains( + "The application is not found.")); + } + server.stop(); } @Test public void testHadoopProtoRPC() throws Exception { test(HadoopYarnProtoRPC.class.getName()); } - + private void test(String rpcClass) throws Exception { Configuration conf = new Configuration(); conf.set(YarnConfiguration.IPC_RPC_IMPL, rpcClass); YarnRPC rpc = YarnRPC.create(conf); String bindAddr = "localhost:0"; InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr); - Server server = rpc.getServer(ContainerManagementProtocol.class, + Server server = rpc.getServer(ContainerManagementProtocol.class, new DummyContainerManager(), addr, conf, null, 1); server.start(); - RPC.setProtocolEngine(conf, ContainerManagementProtocolPB.class, ProtobufRpcEngine.class); - ContainerManagementProtocol proxy = (ContainerManagementProtocol) - rpc.getProxy(ContainerManagementProtocol.class, + RPC.setProtocolEngine(conf, ContainerManagementProtocolPB.class, + ProtobufRpcEngine.class); + ContainerManagementProtocol proxy = (ContainerManagementProtocol) + rpc.getProxy(ContainerManagementProtocol.class, NetUtils.getConnectAddress(server), conf); - ContainerLaunchContext containerLaunchContext = - recordFactory.newRecordInstance(ContainerLaunchContext.class); + ContainerLaunchContext containerLaunchContext = + RECORD_FACTORY.newRecordInstance(ContainerLaunchContext.class); ApplicationId applicationId = ApplicationId.newInstance(0, 0); ApplicationAttemptId applicationAttemptId = @@ -168,25 +305,26 @@ public class TestRPC { boolean exception = false; try { StopContainersRequest stopRequest = - recordFactory.newRecordInstance(StopContainersRequest.class); + RECORD_FACTORY.newRecordInstance(StopContainersRequest.class); stopRequest.setContainerIds(containerIds); proxy.stopContainers(stopRequest); - } catch (YarnException e) { + } catch (YarnException e) { exception = true; Assert.assertTrue(e.getMessage().contains(EXCEPTION_MSG)); Assert.assertTrue(e.getMessage().contains(EXCEPTION_CAUSE)); System.out.println("Test Exception is " + e.getMessage()); } catch (Exception ex) { ex.printStackTrace(); + } finally { + server.stop(); } Assert.assertTrue(exception); - - server.stop(); Assert.assertNotNull(statuses.get(0)); Assert.assertEquals(ContainerState.RUNNING, statuses.get(0).getState()); } - public class DummyContainerManager implements ContainerManagementProtocol { + private static class DummyContainerManager + implements ContainerManagementProtocol { private List statuses = new ArrayList(); @@ -194,8 +332,8 @@ public class TestRPC { public GetContainerStatusesResponse getContainerStatuses( GetContainerStatusesRequest request) throws YarnException { - GetContainerStatusesResponse response = - recordFactory.newRecordInstance(GetContainerStatusesResponse.class); + GetContainerStatusesResponse response = + RECORD_FACTORY.newRecordInstance(GetContainerStatusesResponse.class); response.setContainerStatuses(statuses); return response; } @@ -204,8 +342,9 @@ public class TestRPC { public StartContainersResponse startContainers( StartContainersRequest requests) throws YarnException { StartContainersResponse response = - recordFactory.newRecordInstance(StartContainersResponse.class); - for (StartContainerRequest request : requests.getStartContainerRequests()) { + RECORD_FACTORY.newRecordInstance(StartContainersResponse.class); + for (StartContainerRequest request : + requests.getStartContainerRequests()) { Token containerToken = request.getContainerToken(); ContainerTokenIdentifier tokenId = null; @@ -215,7 +354,7 @@ public class TestRPC { throw RPCUtil.getRemoteException(e); } ContainerStatus status = - recordFactory.newRecordInstance(ContainerStatus.class); + RECORD_FACTORY.newRecordInstance(ContainerStatus.class); status.setState(ContainerState.RUNNING); status.setContainerId(tokenId.getContainerID()); status.setExitStatus(0); @@ -226,9 +365,9 @@ public class TestRPC { } @Override - public StopContainersResponse stopContainers(StopContainersRequest request) + public StopContainersResponse stopContainers(StopContainersRequest request) throws YarnException { - Exception e = new Exception(EXCEPTION_MSG, + Exception e = new Exception(EXCEPTION_MSG, new Exception(EXCEPTION_CAUSE)); throw new YarnException(e); } @@ -236,7 +375,8 @@ public class TestRPC { @Override @Deprecated public IncreaseContainersResourceResponse increaseContainersResource( - IncreaseContainersResourceRequest request) throws YarnException, IOException { + IncreaseContainersResourceRequest request) + throws YarnException, IOException { return null; } @@ -309,4 +449,47 @@ public class TestRPC { .buildTokenService(addr).toString()); return containerToken; } + + // A dummy implementation for CollectorNodemanagerProtocol for test purpose, + // it only can accept one appID, collectorAddr pair or throw exceptions + private static class DummyNMCollectorService + implements CollectorNodemanagerProtocol { + + @Override + public ReportNewCollectorInfoResponse reportNewCollectorInfo( + ReportNewCollectorInfoRequest request) + throws YarnException, IOException { + List appCollectors = request.getAppCollectorsList(); + if (appCollectors.size() == 1) { + // check default appID and collectorAddr + AppCollectorData appCollector = appCollectors.get(0); + Assert.assertEquals(appCollector.getApplicationId(), + DEFAULT_APP_ID); + Assert.assertEquals(appCollector.getCollectorAddr(), + DEFAULT_COLLECTOR_ADDR); + Assert.assertTrue(appCollector.getCollectorToken() == null || + appCollector.getCollectorToken().equals(DEFAULT_COLLECTOR_TOKEN)); + } else { + throw new YarnException(ILLEGAL_NUMBER_MESSAGE); + } + + ReportNewCollectorInfoResponse response = + RECORD_FACTORY.newRecordInstance( + ReportNewCollectorInfoResponse.class); + return response; + } + + @Override + public GetTimelineCollectorContextResponse getTimelineCollectorContext( + GetTimelineCollectorContextRequest request) + throws YarnException, IOException { + if (request.getApplicationId().getId() == 1) { + return GetTimelineCollectorContextResponse.newInstance( + "test_user_id", "test_flow_name", "test_flow_version", 12345678L); + } else { + throw new YarnException("The application is not found."); + } + } + } + } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java index 49ee25d23ca..8b1d0bb49e5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java @@ -25,7 +25,9 @@ import static org.junit.Assert.assertTrue; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.HashSet; +import java.util.Map; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; @@ -35,6 +37,7 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeLabel; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.Token; import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl; @@ -46,6 +49,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.NodeHeartbeatRe import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RegisterNodeManagerRequestPBImpl; import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RegisterNodeManagerResponsePBImpl; import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.UnRegisterNodeManagerRequestPBImpl; +import org.apache.hadoop.yarn.server.api.records.AppCollectorData; import org.apache.hadoop.yarn.server.api.records.MasterKey; import org.apache.hadoop.yarn.server.api.records.NodeAction; import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus; @@ -107,11 +111,14 @@ public class TestYarnServerApiClasses { original.setLastKnownNMTokenMasterKey(getMasterKey()); original.setNodeStatus(getNodeStatus()); original.setNodeLabels(getValidNodeLabels()); + Map collectors = getCollectors(false); + original.setRegisteringCollectors(collectors); NodeHeartbeatRequestPBImpl copy = new NodeHeartbeatRequestPBImpl( original.getProto()); assertEquals(1, copy.getLastKnownContainerTokenMasterKey().getKeyId()); assertEquals(1, copy.getLastKnownNMTokenMasterKey().getKeyId()); assertEquals("localhost", copy.getNodeStatus().getNodeId().getHost()); + assertEquals(collectors, copy.getRegisteringCollectors()); // check labels are coming with valid values Assert.assertTrue(original.getNodeLabels() .containsAll(copy.getNodeLabels())); @@ -123,6 +130,16 @@ public class TestYarnServerApiClasses { Assert.assertEquals(0, copy.getNodeLabels().size()); } + @Test + public void testNodeHBRequestPBImplWithNullCollectorToken() { + NodeHeartbeatRequestPBImpl original = new NodeHeartbeatRequestPBImpl(); + Map collectors = getCollectors(true); + original.setRegisteringCollectors(collectors); + NodeHeartbeatRequestPBImpl copy = new NodeHeartbeatRequestPBImpl( + original.getProto()); + assertEquals(collectors, copy.getRegisteringCollectors()); + } + /** * Test NodeHeartbeatRequestPBImpl. */ @@ -148,6 +165,8 @@ public class TestYarnServerApiClasses { original.setNextHeartBeatInterval(1000); original.setNodeAction(NodeAction.NORMAL); original.setResponseId(100); + Map collectors = getCollectors(false); + original.setAppCollectors(collectors); NodeHeartbeatResponsePBImpl copy = new NodeHeartbeatResponsePBImpl( original.getProto()); @@ -157,6 +176,7 @@ public class TestYarnServerApiClasses { assertEquals(1, copy.getContainerTokenMasterKey().getKeyId()); assertEquals(1, copy.getNMTokenMasterKey().getKeyId()); assertEquals("testDiagnosticMessage", copy.getDiagnosticsMessage()); + assertEquals(collectors, copy.getAppCollectors()); assertEquals(false, copy.getAreNodeLabelsAcceptedByRM()); } @@ -169,6 +189,16 @@ public class TestYarnServerApiClasses { assertTrue(copy.getAreNodeLabelsAcceptedByRM()); } + @Test + public void testNodeHBResponsePBImplWithNullCollectorToken() { + NodeHeartbeatResponsePBImpl original = new NodeHeartbeatResponsePBImpl(); + Map collectors = getCollectors(true); + original.setAppCollectors(collectors); + NodeHeartbeatResponsePBImpl copy = new NodeHeartbeatResponsePBImpl( + original.getProto()); + assertEquals(collectors, copy.getAppCollectors()); + } + @Test public void testNodeHeartbeatResponsePBImplWithDecreasedContainers() { NodeHeartbeatResponsePBImpl original = new NodeHeartbeatResponsePBImpl(); @@ -339,6 +369,21 @@ public class TestYarnServerApiClasses { return nodeLabels; } + private Map getCollectors( + boolean hasNullCollectorToken) { + ApplicationId appID = ApplicationId.newInstance(1L, 1); + String collectorAddr = "localhost:0"; + AppCollectorData data = AppCollectorData.newInstance(appID, collectorAddr); + if (!hasNullCollectorToken) { + data.setCollectorToken( + Token.newInstance(new byte[0], "kind", new byte[0], "s")); + } + Map collectorMap = + new HashMap<>(); + collectorMap.put(appID, data); + return collectorMap; + } + private ContainerStatus getContainerStatus(int applicationId, int containerID, int appAttemptId) { ContainerStatus status = recordFactory diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java index e33d7e19774..c49d6e890db 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java @@ -418,10 +418,11 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol, // Always issue a new AMRMToken as if RM rolled master key Token newAMRMToken = Token.newInstance(new byte[0], "", new byte[0], ""); - return AllocateResponse.newInstance(0, new ArrayList(), - containerList, new ArrayList(), null, AMCommand.AM_RESYNC, - 1, null, new ArrayList(), newAMRMToken, - new ArrayList()); + return AllocateResponse.newInstance(0, + new ArrayList(), containerList, + new ArrayList(), null, AMCommand.AM_RESYNC, 1, null, + new ArrayList(), newAMRMToken, + new ArrayList(), null); } @Override diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilterInitializer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilterInitializer.java similarity index 97% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilterInitializer.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilterInitializer.java index 430911eaa34..44f63ead34a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilterInitializer.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilterInitializer.java @@ -27,7 +27,9 @@ import static org.apache.hadoop.yarn.server.timeline.security.TimelineAuthentica import org.junit.Test; import org.mockito.Mockito; - +/** + * Tests {@link TimelineAuthenticationFilterInitializer}. + */ public class TestTimelineAuthenticationFilterInitializer { @Test diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java index b1107599bbf..33cefea91f5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java @@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport; +import org.apache.hadoop.yarn.server.api.records.AppCollectorData; import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus; import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManager; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application; @@ -37,6 +38,7 @@ import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService; import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerAllocator; import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager; import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM; +import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; /** @@ -63,6 +65,20 @@ public interface Context { Map getSystemCredentialsForApps(); + /** + * Get the list of collectors that are registering with the RM from this node. + * @return registering collectors, or null if the timeline service v.2 is not + * enabled + */ + ConcurrentMap getRegisteringCollectors(); + + /** + * Get the list of collectors registered with the RM and known by this node. + * @return known collectors, or null if the timeline service v.2 is not + * enabled. + */ + ConcurrentMap getKnownCollectors(); + ConcurrentMap getContainers(); ConcurrentMap @@ -102,4 +118,8 @@ public interface Context { ContainerExecutor getContainerExecutor(); ContainerStateTransitionListener getContainerStateTransitionListener(); + + void setNMTimelinePublisher(NMTimelinePublisher nmMetricsPublisher); + + NMTimelinePublisher getNMTimelinePublisher(); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java index 03e219b54ba..c23c071920a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java @@ -63,10 +63,13 @@ import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport; +import org.apache.hadoop.yarn.server.api.records.AppCollectorData; import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus; import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManager; +import org.apache.hadoop.yarn.server.nodemanager.collectormanager.NMCollectorService; import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics; import org.apache.hadoop.yarn.server.nodemanager.nodelabels.ConfigurationNodeLabelsProvider; @@ -78,6 +81,7 @@ import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService; import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerAllocator; import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager; import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM; +import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher; import org.apache.hadoop.yarn.server.nodemanager.webapp.WebServer; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; @@ -121,6 +125,8 @@ public class NodeManager extends CompositeService private Context context; private AsyncDispatcher dispatcher; private ContainerManagerImpl containerManager; + // the NM collector service is set only if the timeline service v.2 is enabled + private NMCollectorService nmCollectorService; private NodeStatusUpdater nodeStatusUpdater; private NodeResourceMonitor nodeResourceMonitor; private static CompositeServiceShutdownHook nodeManagerShutdownHook; @@ -211,6 +217,10 @@ public class NodeManager extends CompositeService metrics, dirsHandler); } + protected NMCollectorService createNMCollectorService(Context ctxt) { + return new NMCollectorService(ctxt); + } + protected WebServer createWebServer(Context nmContext, ResourceView resourceView, ApplicationACLsManager aclsManager, LocalDirsHandlerService dirsHandler) { @@ -420,6 +430,11 @@ public class NodeManager extends CompositeService DefaultMetricsSystem.initialize("NodeManager"); + if (YarnConfiguration.timelineServiceV2Enabled(conf)) { + this.nmCollectorService = createNMCollectorService(context); + addService(nmCollectorService); + } + // StatusUpdater should be added last so that it get started last // so that we make sure everything is up before registering with RM. addService(nodeStatusUpdater); @@ -485,8 +500,14 @@ public class NodeManager extends CompositeService if (!rmWorkPreservingRestartEnabled) { LOG.info("Cleaning up running containers on resync"); containerManager.cleanupContainersOnNMResync(); + // Clear all known collectors for resync. + if (context.getKnownCollectors() != null) { + context.getKnownCollectors().clear(); + } } else { LOG.info("Preserving containers on resync"); + // Re-register known timeline collectors. + reregisterCollectors(); } ((NodeStatusUpdaterImpl) nodeStatusUpdater) .rebootNodeStatusUpdaterAndRegisterWithRM(); @@ -498,6 +519,38 @@ public class NodeManager extends CompositeService }.start(); } + /** + * Reregisters all collectors known by this node to the RM. This method is + * called when the RM needs to resync with the node. + */ + protected void reregisterCollectors() { + Map knownCollectors + = context.getKnownCollectors(); + if (knownCollectors == null) { + return; + } + ConcurrentMap registeringCollectors + = context.getRegisteringCollectors(); + for (Map.Entry entry + : knownCollectors.entrySet()) { + Application app = context.getApplications().get(entry.getKey()); + if ((app != null) + && !ApplicationState.FINISHED.equals(app.getApplicationState())) { + registeringCollectors.putIfAbsent(entry.getKey(), entry.getValue()); + AppCollectorData data = entry.getValue(); + if (LOG.isDebugEnabled()) { + LOG.debug(entry.getKey() + " : " + data.getCollectorAddr() + "@<" + + data.getRMIdentifier() + ", " + data.getVersion() + ">"); + } + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Remove collector data for done app " + entry.getKey()); + } + } + } + knownCollectors.clear(); + } + public static class NMContext implements Context { private NodeId nodeId = null; @@ -513,6 +566,11 @@ public class NodeManager extends CompositeService protected final ConcurrentMap containers = new ConcurrentSkipListMap(); + private ConcurrentMap + registeringCollectors; + + private ConcurrentMap knownCollectors; + protected final ConcurrentMap increasedContainers = new ConcurrentHashMap<>(); @@ -539,11 +597,17 @@ public class NodeManager extends CompositeService private ContainerStateTransitionListener containerStateTransitionListener; + private NMTimelinePublisher nmTimelinePublisher; + public NMContext(NMContainerTokenSecretManager containerTokenSecretManager, NMTokenSecretManagerInNM nmTokenSecretManager, LocalDirsHandlerService dirsHandler, ApplicationACLsManager aclsManager, NMStateStoreService stateStore, boolean isDistSchedulingEnabled, Configuration conf) { + if (YarnConfiguration.timelineServiceV2Enabled(conf)) { + this.registeringCollectors = new ConcurrentHashMap<>(); + this.knownCollectors = new ConcurrentHashMap<>(); + } this.containerTokenSecretManager = containerTokenSecretManager; this.nmTokenSecretManager = nmTokenSecretManager; this.dirsHandler = dirsHandler; @@ -715,6 +779,26 @@ public class NodeManager extends CompositeService ContainerStateTransitionListener transitionListener) { this.containerStateTransitionListener = transitionListener; } + + public ConcurrentMap + getRegisteringCollectors() { + return this.registeringCollectors; + } + + @Override + public ConcurrentMap getKnownCollectors() { + return this.knownCollectors; + } + + @Override + public void setNMTimelinePublisher(NMTimelinePublisher nmMetricsPublisher) { + this.nmTimelinePublisher = nmMetricsPublisher; + } + + @Override + public NMTimelinePublisher getNMTimelinePublisher() { + return nmTimelinePublisher; + } } /** @@ -791,9 +875,22 @@ public class NodeManager extends CompositeService return this.context; } + /** + * Returns the NM collector service. It should be used only for testing + * purposes. + * + * @return the NM collector service, or null if the timeline service v.2 is + * not enabled + */ + @VisibleForTesting + NMCollectorService getNMCollectorService() { + return this.nmCollectorService; + } + public static void main(String[] args) throws IOException { Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler()); StringUtils.startupShutdownMessage(NodeManager.class, args, LOG); + @SuppressWarnings("resource") NodeManager nodeManager = new NodeManager(); Configuration conf = new YarnConfiguration(); new GenericOptionsParser(conf, args); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java index a0d91eec2a6..65bde635a73 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java @@ -71,7 +71,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.UnRegisterNodeManagerRequest; - +import org.apache.hadoop.yarn.server.api.records.AppCollectorData; import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit; import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus; import org.apache.hadoop.yarn.server.api.records.MasterKey; @@ -79,11 +79,13 @@ import org.apache.hadoop.yarn.server.api.records.NodeAction; import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus; import org.apache.hadoop.yarn.server.api.records.NodeStatus; import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics; import org.apache.hadoop.yarn.server.nodemanager.nodelabels.NodeLabelsProvider; +import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher; import org.apache.hadoop.yarn.server.nodemanager.util.NodeManagerHardwareUtils; import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin; @@ -778,7 +780,9 @@ public class NodeStatusUpdaterImpl extends AbstractService implements .getContainerTokenSecretManager().getCurrentKey(), NodeStatusUpdaterImpl.this.context .getNMTokenSecretManager().getCurrentKey(), - nodeLabelsForHeartbeat); + nodeLabelsForHeartbeat, + NodeStatusUpdaterImpl.this.context + .getRegisteringCollectors()); if (logAggregationEnabled) { // pull log aggregation status for application running in this NM @@ -868,6 +872,10 @@ public class NodeStatusUpdaterImpl extends AbstractService implements newResource.toString()); } } + if (YarnConfiguration.timelineServiceV2Enabled(context.getConf())) { + updateTimelineCollectorData(response); + } + } catch (ConnectException e) { //catch and throw the exception if tried MAX wait time to connect RM dispatcher.getEventHandler().handle( @@ -895,6 +903,51 @@ public class NodeStatusUpdaterImpl extends AbstractService implements } } + private void updateTimelineCollectorData( + NodeHeartbeatResponse response) { + Map incomingCollectorsMap = + response.getAppCollectors(); + if (incomingCollectorsMap == null) { + if (LOG.isDebugEnabled()) { + LOG.debug("No collectors to update RM"); + } + return; + } + Map knownCollectors + = context.getKnownCollectors(); + for (Map.Entry entry + : incomingCollectorsMap.entrySet()) { + ApplicationId appId = entry.getKey(); + AppCollectorData collectorData = entry.getValue(); + // Only handle applications running on local node. + Application application = context.getApplications().get(appId); + if (application != null) { + // Update collector data if the newly received data happens after + // the known data (updates the known data). + AppCollectorData existingData = knownCollectors.get(appId); + if (AppCollectorData.happensBefore(existingData, collectorData)) { + if (LOG.isDebugEnabled()) { + LOG.debug("Sync a new collector address: " + + collectorData.getCollectorAddr() + + " for application: " + appId + " from RM."); + } + + // Update information for clients. + NMTimelinePublisher nmTimelinePublisher + = context.getNMTimelinePublisher(); + if (nmTimelinePublisher != null) { + nmTimelinePublisher.setTimelineServiceAddress( + application.getAppId(), collectorData.getCollectorAddr()); + } + // Update information for the node manager itself. + knownCollectors.put(appId, collectorData); + } + } + // Remove the registering collector data + context.getRegisteringCollectors().remove(entry.getKey()); + } + } + private void updateMasterKeys(NodeHeartbeatResponse response) { // See if the master-key has rolled over MasterKey updatedMasterKey = response.getContainerTokenMasterKey(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/collectormanager/NMCollectorService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/collectormanager/NMCollectorService.java new file mode 100644 index 00000000000..6e44b5bdc37 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/collectormanager/NMCollectorService.java @@ -0,0 +1,158 @@ +/** + * 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.nodemanager.collectormanager; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.ipc.Server; +import org.apache.hadoop.service.CompositeService; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.ipc.YarnRPC; +import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocol; +import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoResponse; +import org.apache.hadoop.yarn.server.api.records.AppCollectorData; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application; +import org.apache.hadoop.yarn.server.nodemanager.security.authorize.NMPolicyProvider; +import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher; + +/** + * Service that handles collector information. It is used only if the timeline + * service v.2 is enabled. + */ +public class NMCollectorService extends CompositeService implements + CollectorNodemanagerProtocol { + + private static final Log LOG = LogFactory.getLog(NMCollectorService.class); + + private final Context context; + + private Server server; + + public NMCollectorService(Context context) { + super(NMCollectorService.class.getName()); + this.context = context; + } + + @Override + protected void serviceStart() throws Exception { + Configuration conf = getConfig(); + + InetSocketAddress collectorServerAddress = conf.getSocketAddr( + YarnConfiguration.NM_BIND_HOST, + YarnConfiguration.NM_COLLECTOR_SERVICE_ADDRESS, + YarnConfiguration.DEFAULT_NM_COLLECTOR_SERVICE_ADDRESS, + YarnConfiguration.DEFAULT_NM_COLLECTOR_SERVICE_PORT); + + Configuration serverConf = new Configuration(conf); + + YarnRPC rpc = YarnRPC.create(conf); + + // Kerberos based authentication to be used for CollectorNodemanager + // protocol if security is enabled. + server = + rpc.getServer(CollectorNodemanagerProtocol.class, this, + collectorServerAddress, serverConf, null, + conf.getInt(YarnConfiguration.NM_COLLECTOR_SERVICE_THREAD_COUNT, + YarnConfiguration.DEFAULT_NM_COLLECTOR_SERVICE_THREAD_COUNT)); + + if (conf.getBoolean( + CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, false)) { + server.refreshServiceAcl(conf, new NMPolicyProvider()); + } + + server.start(); + collectorServerAddress = conf.updateConnectAddr( + YarnConfiguration.NM_BIND_HOST, + YarnConfiguration.NM_COLLECTOR_SERVICE_ADDRESS, + YarnConfiguration.DEFAULT_NM_COLLECTOR_SERVICE_ADDRESS, + server.getListenerAddress()); + // start remaining services + super.serviceStart(); + LOG.info("NMCollectorService started at " + collectorServerAddress); + } + + @Override + public void serviceStop() throws Exception { + if (server != null) { + server.stop(); + } + // TODO may cleanup app collectors running on this NM in future. + super.serviceStop(); + } + + @Override + public ReportNewCollectorInfoResponse reportNewCollectorInfo( + ReportNewCollectorInfoRequest request) throws YarnException, IOException { + List newCollectorsList = request.getAppCollectorsList(); + if (newCollectorsList != null && !newCollectorsList.isEmpty()) { + Map newCollectorsMap = + new HashMap<>(); + for (AppCollectorData collector : newCollectorsList) { + ApplicationId appId = collector.getApplicationId(); + newCollectorsMap.put(appId, collector); + // set registered collector address to TimelineClient. + // TODO: Do we need to do this after we received confirmation from + // the RM? + NMTimelinePublisher nmTimelinePublisher = + context.getNMTimelinePublisher(); + if (nmTimelinePublisher != null) { + nmTimelinePublisher.setTimelineServiceAddress(appId, + collector.getCollectorAddr()); + } + } + Map registeringCollectors + = context.getRegisteringCollectors(); + if (registeringCollectors != null) { + registeringCollectors.putAll(newCollectorsMap); + } else { + LOG.warn("collectors are added when the registered collectors are " + + "initialized"); + } + } + + return ReportNewCollectorInfoResponse.newInstance(); + } + + @Override + public GetTimelineCollectorContextResponse getTimelineCollectorContext( + GetTimelineCollectorContextRequest request) + throws YarnException, IOException { + Application app = context.getApplications().get(request.getApplicationId()); + if (app == null) { + throw new YarnException("Application " + request.getApplicationId() + + " doesn't exist on NM."); + } + return GetTimelineCollectorContextResponse.newInstance( + app.getUser(), app.getFlowName(), app.getFlowVersion(), + app.getFlowRunId()); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/collectormanager/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/collectormanager/package-info.java new file mode 100644 index 00000000000..7bf597b14f0 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/collectormanager/package-info.java @@ -0,0 +1,28 @@ +/* + * 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.nodemanager.collectormanager contains + * classes for handling timeline collector information. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.nodemanager.collectormanager; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java index 2efc932353e..5e0f2936f83 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java @@ -244,7 +244,8 @@ public class AuxServices extends AbstractService for (AuxiliaryService serv : serviceMap.values()) { try { serv.initializeContainer(new ContainerInitializationContext( - event.getUser(), event.getContainer().getContainerId(), + event.getContainer().getUser(), + event.getContainer().getContainerId(), event.getContainer().getResource(), event.getContainer() .getContainerTokenIdentifier().getContainerType())); } catch (Throwable th) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java index caaa2703daf..ffd1a33cbd9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java @@ -90,6 +90,7 @@ import org.apache.hadoop.yarn.ipc.RPCUtil; import org.apache.hadoop.yarn.ipc.YarnRPC; import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationACLMapProto; import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.ContainerManagerApplicationProto; +import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.FlowContextProto; import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; import org.apache.hadoop.yarn.security.NMTokenIdentifier; import org.apache.hadoop.yarn.server.api.ContainerType; @@ -115,6 +116,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Ap import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationFinishEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationImpl; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationImpl.FlowContext; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationInitEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent; @@ -151,8 +153,12 @@ import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.Re import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredContainerStatus; import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredContainerType; import org.apache.hadoop.yarn.server.nodemanager.security.authorize.NMPolicyProvider; +import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher; import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.server.utils.YarnServerSecurityUtils; +import org.apache.hadoop.yarn.util.SystemClock; +import org.apache.hadoop.yarn.util.resource.Resources; +import org.apache.hadoop.yarn.util.timeline.TimelineUtils; import java.io.DataInputStream; import java.io.IOException; @@ -174,8 +180,6 @@ import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock; import static org.apache.hadoop.service.Service.STATE.STARTED; -import org.apache.hadoop.yarn.util.resource.Resources; - public class ContainerManagerImpl extends CompositeService implements ContainerManager { @@ -217,6 +221,9 @@ public class ContainerManagerImpl extends CompositeService implements private long waitForContainersOnShutdownMillis; + // NM metrics publisher is set only if the timeline service v.2 is enabled + private NMTimelinePublisher nmMetricsPublisher; + public ContainerManagerImpl(Context context, ContainerExecutor exec, DeletionService deletionContext, NodeStatusUpdater nodeStatusUpdater, NodeManagerMetrics metrics, LocalDirsHandlerService dirsHandler) { @@ -246,6 +253,15 @@ public class ContainerManagerImpl extends CompositeService implements auxiliaryServices.registerServiceListener(this); addService(auxiliaryServices); + // initialize the metrics publisher if the timeline service v.2 is enabled + // and the system publisher is enabled + Configuration conf = context.getConf(); + if (YarnConfiguration.timelineServiceV2Enabled(conf) && + YarnConfiguration.systemMetricsPublisherEnabled(conf)) { + LOG.info("YARN system metrics publishing service is enabled"); + nmMetricsPublisher = createNMTimelinePublisher(context); + context.setNMTimelinePublisher(nmMetricsPublisher); + } this.containersMonitor = createContainersMonitor(exec); addService(this.containersMonitor); @@ -253,7 +269,9 @@ public class ContainerManagerImpl extends CompositeService implements new ContainerEventDispatcher()); dispatcher.register(ApplicationEventType.class, createApplicationEventDispatcher()); - dispatcher.register(LocalizationEventType.class, rsrcLocalizationSrvc); + dispatcher.register(LocalizationEventType.class, + new LocalizationEventHandlerWrapper(rsrcLocalizationSrvc, + nmMetricsPublisher)); dispatcher.register(AuxServicesEventType.class, auxiliaryServices); dispatcher.register(ContainersMonitorEventType.class, containersMonitor); dispatcher.register(ContainersLauncherEventType.class, containersLauncher); @@ -383,13 +401,35 @@ public class ContainerManagerImpl extends CompositeService implements new LogAggregationContextPBImpl(p.getLogAggregationContext()); } + FlowContext fc = null; + if (p.getFlowContext() != null) { + FlowContextProto fcp = p.getFlowContext(); + fc = new FlowContext(fcp.getFlowName(), fcp.getFlowVersion(), + fcp.getFlowRunId()); + if (LOG.isDebugEnabled()) { + LOG.debug( + "Recovering Flow context: " + fc + " for an application " + appId); + } + } else { + // in upgrade situations, where there is no prior existing flow context, + // default would be used. + fc = new FlowContext(TimelineUtils.generateDefaultFlowName(null, appId), + YarnConfiguration.DEFAULT_FLOW_VERSION, appId.getClusterTimestamp()); + if (LOG.isDebugEnabled()) { + LOG.debug( + "No prior existing flow context found. Using default Flow context: " + + fc + " for an application " + appId); + } + } + LOG.info("Recovering application " + appId); - ApplicationImpl app = new ApplicationImpl(dispatcher, p.getUser(), appId, - creds, context, p.getAppLogAggregationInitedTime()); + ApplicationImpl app = new ApplicationImpl(dispatcher, p.getUser(), fc, + appId, creds, context, p.getAppLogAggregationInitedTime()); context.getApplications().put(appId, app); app.handle(new ApplicationInitEvent(appId, acls, logAggregationContext)); } + @SuppressWarnings("unchecked") private void recoverContainer(RecoveredContainerState rcs) throws IOException { StartContainerRequest req = rcs.getStartRequest(); @@ -512,6 +552,14 @@ public class ContainerManagerImpl extends CompositeService implements return new SharedCacheUploadService(); } + @VisibleForTesting + protected NMTimelinePublisher createNMTimelinePublisher(Context ctxt) { + NMTimelinePublisher nmTimelinePublisherLocal = + new NMTimelinePublisher(ctxt); + addIfService(nmTimelinePublisherLocal); + return nmTimelinePublisherLocal; + } + protected ContainersLauncher createContainersLauncher(Context context, ContainerExecutor exec) { return new ContainersLauncher(context, this.dispatcher, exec, dirsHandler, this); @@ -954,7 +1002,7 @@ public class ContainerManagerImpl extends CompositeService implements private ContainerManagerApplicationProto buildAppProto(ApplicationId appId, String user, Credentials credentials, Map appAcls, - LogAggregationContext logAggregationContext) { + LogAggregationContext logAggregationContext, FlowContext flowContext) { ContainerManagerApplicationProto.Builder builder = ContainerManagerApplicationProto.newBuilder(); @@ -989,6 +1037,16 @@ public class ContainerManagerImpl extends CompositeService implements } } + builder.clearFlowContext(); + if (flowContext != null && flowContext.getFlowName() != null + && flowContext.getFlowVersion() != null) { + FlowContextProto fcp = + FlowContextProto.newBuilder().setFlowName(flowContext.getFlowName()) + .setFlowVersion(flowContext.getFlowVersion()) + .setFlowRunId(flowContext.getFlowRunId()).build(); + builder.setFlowContext(fcp); + } + return builder.build(); } @@ -1023,10 +1081,11 @@ public class ContainerManagerImpl extends CompositeService implements Credentials credentials = YarnServerSecurityUtils.parseCredentials(launchContext); + long containerStartTime = SystemClock.getInstance().getTime(); Container container = new ContainerImpl(getConfig(), this.dispatcher, launchContext, credentials, metrics, containerTokenIdentifier, - context); + context, containerStartTime); ApplicationId applicationID = containerId.getApplicationAttemptId().getApplicationId(); if (context.getContainers().putIfAbsent(containerId, container) != null) { @@ -1040,27 +1099,50 @@ public class ContainerManagerImpl extends CompositeService implements this.readLock.lock(); try { if (!isServiceStopped()) { - // Create the application - Application application = new ApplicationImpl(dispatcher, user, - applicationID, credentials, context); - if (null == context.getApplications().putIfAbsent(applicationID, - application)) { - LOG.info("Creating a new application reference for app " - + applicationID); - LogAggregationContext logAggregationContext = - containerTokenIdentifier.getLogAggregationContext(); - Map appAcls = - container.getLaunchContext().getApplicationACLs(); - context.getNMStateStore().storeApplication(applicationID, - buildAppProto(applicationID, user, credentials, appAcls, - logAggregationContext)); - dispatcher.getEventHandler().handle( - new ApplicationInitEvent(applicationID, appAcls, - logAggregationContext)); + if (!context.getApplications().containsKey(applicationID)) { + // Create the application + // populate the flow context from the launch context if the timeline + // service v.2 is enabled + FlowContext flowContext = null; + if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) { + String flowName = launchContext.getEnvironment() + .get(TimelineUtils.FLOW_NAME_TAG_PREFIX); + String flowVersion = launchContext.getEnvironment() + .get(TimelineUtils.FLOW_VERSION_TAG_PREFIX); + String flowRunIdStr = launchContext.getEnvironment() + .get(TimelineUtils.FLOW_RUN_ID_TAG_PREFIX); + long flowRunId = 0L; + if (flowRunIdStr != null && !flowRunIdStr.isEmpty()) { + flowRunId = Long.parseLong(flowRunIdStr); + } + flowContext = new FlowContext(flowName, flowVersion, flowRunId); + if (LOG.isDebugEnabled()) { + LOG.debug("Flow context: " + flowContext + + " created for an application " + applicationID); + } + } + + Application application = + new ApplicationImpl(dispatcher, user, flowContext, + applicationID, credentials, context); + if (context.getApplications().putIfAbsent(applicationID, + application) == null) { + LOG.info("Creating a new application reference for app " + + applicationID); + LogAggregationContext logAggregationContext = + containerTokenIdentifier.getLogAggregationContext(); + Map appAcls = + container.getLaunchContext().getApplicationACLs(); + context.getNMStateStore().storeApplication(applicationID, + buildAppProto(applicationID, user, credentials, appAcls, + logAggregationContext, flowContext)); + dispatcher.getEventHandler().handle(new ApplicationInitEvent( + applicationID, appAcls, logAggregationContext)); + } } this.context.getNMStateStore().storeContainer(containerId, - containerTokenIdentifier.getVersion(), request); + containerTokenIdentifier.getVersion(), containerStartTime, request); dispatcher.getEventHandler().handle( new ApplicationContainerInitEvent(container)); @@ -1401,6 +1483,9 @@ public class ContainerManagerImpl extends CompositeService implements Container c = containers.get(event.getContainerID()); if (c != null) { c.handle(event); + if (nmMetricsPublisher != null) { + nmMetricsPublisher.publishContainerEvent(event); + } } else { LOG.warn("Event " + event + " sent to absent container " + event.getContainerID()); @@ -1409,7 +1494,6 @@ public class ContainerManagerImpl extends CompositeService implements } class ApplicationEventDispatcher implements EventHandler { - @Override public void handle(ApplicationEvent event) { Application app = @@ -1417,6 +1501,9 @@ public class ContainerManagerImpl extends CompositeService implements event.getApplicationID()); if (app != null) { app.handle(event); + if (nmMetricsPublisher != null) { + nmMetricsPublisher.publishApplicationEvent(event); + } } else { LOG.warn("Event " + event + " sent to absent application " + event.getApplicationID()); @@ -1424,6 +1511,27 @@ public class ContainerManagerImpl extends CompositeService implements } } + private static final class LocalizationEventHandlerWrapper implements + EventHandler { + + private EventHandler origLocalizationEventHandler; + private NMTimelinePublisher timelinePublisher; + + LocalizationEventHandlerWrapper(EventHandler handler, + NMTimelinePublisher publisher) { + this.origLocalizationEventHandler = handler; + this.timelinePublisher = publisher; + } + + @Override + public void handle(LocalizationEvent event) { + origLocalizationEventHandler.handle(event); + if (timelinePublisher != null) { + timelinePublisher.publishLocalizationEvent(event); + } + } + } + @SuppressWarnings("unchecked") @Override public void handle(ContainerManagerEvent event) { @@ -1628,6 +1736,7 @@ public class ContainerManagerImpl extends CompositeService implements * @param reInitLaunchContext Target Launch Context. * @throws YarnException Yarn Exception. */ + @SuppressWarnings("unchecked") public void reInitializeContainer(ContainerId containerId, ContainerLaunchContext reInitLaunchContext, boolean autoCommit) throws YarnException { @@ -1655,6 +1764,7 @@ public class ContainerManagerImpl extends CompositeService implements * @return Rollback Response. * @throws YarnException Yarn Exception. */ + @SuppressWarnings("unchecked") @Override public RollbackResponse rollbackLastReInitialization(ContainerId containerId) throws YarnException { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/Application.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/Application.java index b1571e9a9bd..aee0862ae81 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/Application.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/Application.java @@ -35,4 +35,9 @@ public interface Application extends EventHandler { ApplicationState getApplicationState(); + String getFlowName(); + + String getFlowVersion(); + + long getFlowRunId(); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationContainerFinishedEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationContainerFinishedEvent.java index 6b8007f3fe7..09c946b6829 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationContainerFinishedEvent.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationContainerFinishedEvent.java @@ -19,18 +19,31 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.application; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerStatus; public class ApplicationContainerFinishedEvent extends ApplicationEvent { - private ContainerId containerID; + private ContainerStatus containerStatus; + // Required by NMTimelinePublisher. + private long containerStartTime; - public ApplicationContainerFinishedEvent( - ContainerId containerID) { - super(containerID.getApplicationAttemptId().getApplicationId(), + public ApplicationContainerFinishedEvent(ContainerStatus containerStatus, + long containerStartTs) { + super(containerStatus.getContainerId().getApplicationAttemptId(). + getApplicationId(), ApplicationEventType.APPLICATION_CONTAINER_FINISHED); - this.containerID = containerID; + this.containerStatus = containerStatus; + this.containerStartTime = containerStartTs; } public ContainerId getContainerID() { - return this.containerID; + return containerStatus.getContainerId(); + } + + public ContainerStatus getContainerStatus() { + return containerStatus; + } + + public long getContainerStartTime() { + return containerStartTime; } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java index c61a09d515f..39be7a790c4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java @@ -30,6 +30,7 @@ import org.slf4j.LoggerFactory; import com.google.protobuf.ByteString; import org.apache.hadoop.io.DataOutputBuffer; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.Credentials; import org.apache.hadoop.yarn.api.records.ApplicationAccessType; import org.apache.hadoop.yarn.api.records.ApplicationId; @@ -39,9 +40,12 @@ import org.apache.hadoop.yarn.api.records.LogAggregationContext; import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.LogAggregationContextPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.proto.YarnProtos; import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.ContainerManagerApplicationProto; +import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.FlowContextProto; +import org.apache.hadoop.yarn.server.api.records.AppCollectorData; import org.apache.hadoop.yarn.server.nodemanager.Context; import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEventType; @@ -55,6 +59,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerAppFinishedEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerAppStartedEvent; import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService; +import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; import org.apache.hadoop.yarn.state.InvalidStateTransitionException; import org.apache.hadoop.yarn.state.MultipleArcTransition; @@ -71,6 +76,8 @@ public class ApplicationImpl implements Application { final Dispatcher dispatcher; final String user; + // flow context is set only if the timeline service v.2 is enabled + private FlowContext flowContext; final ApplicationId appId; final Credentials credentials; Map applicationACLs; @@ -97,13 +104,28 @@ public class ApplicationImpl implements Application { private final NMStateStoreService appStateStore; public ApplicationImpl(Dispatcher dispatcher, String user, - ApplicationId appId, Credentials credentials, + ApplicationId appId, Credentials credentials, Context context) { + this(dispatcher, user, null, appId, credentials, context, -1L); + } + + public ApplicationImpl(Dispatcher dispatcher, String user, + FlowContext flowContext, ApplicationId appId, Credentials credentials, Context context, long recoveredLogInitedTime) { this.dispatcher = dispatcher; this.user = user; this.appId = appId; this.credentials = credentials; this.aclsManager = context.getApplicationACLsManager(); + Configuration conf = context.getConf(); + if (YarnConfiguration.timelineServiceV2Enabled(conf)) { + if (flowContext == null) { + throw new IllegalArgumentException("flow context cannot be null"); + } + this.flowContext = flowContext; + if (YarnConfiguration.systemMetricsPublisherEnabled(conf)) { + context.getNMTimelinePublisher().createTimelineClient(appId); + } + } this.context = context; this.appStateStore = context.getNMStateStore(); ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); @@ -114,8 +136,46 @@ public class ApplicationImpl implements Application { } public ApplicationImpl(Dispatcher dispatcher, String user, - ApplicationId appId, Credentials credentials, Context context) { - this(dispatcher, user, appId, credentials, context, -1); + FlowContext flowContext, ApplicationId appId, + Credentials credentials, Context context) { + this(dispatcher, user, flowContext, appId, credentials, + context, -1); + } + + /** + * Data object that encapsulates the flow context for the application purpose. + */ + public static class FlowContext { + private final String flowName; + private final String flowVersion; + private final long flowRunId; + + public FlowContext(String flowName, String flowVersion, long flowRunId) { + this.flowName = flowName; + this.flowVersion = flowVersion; + this.flowRunId = flowRunId; + } + + public String getFlowName() { + return flowName; + } + + public String getFlowVersion() { + return flowVersion; + } + + public long getFlowRunId() { + return flowRunId; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("{"); + sb.append("Flow Name=").append(getFlowName()); + sb.append(" Flow Versioin=").append(getFlowVersion()); + sb.append(" Flow Run Id=").append(getFlowRunId()).append(" }"); + return sb.toString(); + } } @Override @@ -335,6 +395,16 @@ public class ApplicationImpl implements Application { builder.setAppLogAggregationInitedTime(app.applicationLogInitedTimestamp); + builder.clearFlowContext(); + if (app.flowContext != null && app.flowContext.getFlowName() != null + && app.flowContext.getFlowVersion() != null) { + FlowContextProto fcp = FlowContextProto.newBuilder() + .setFlowName(app.flowContext.getFlowName()) + .setFlowVersion(app.flowContext.getFlowVersion()) + .setFlowRunId(app.flowContext.getFlowRunId()).build(); + builder.setFlowContext(fcp); + } + return builder.build(); } @@ -489,6 +559,29 @@ public class ApplicationImpl implements Application { @SuppressWarnings("unchecked") static class AppCompletelyDoneTransition implements SingleArcTransition { + + private void updateCollectorStatus(ApplicationImpl app) { + // Remove collectors info for finished apps. + // TODO check we remove related collectors info in failure cases + // (YARN-3038) + Map registeringCollectors + = app.context.getRegisteringCollectors(); + if (registeringCollectors != null) { + registeringCollectors.remove(app.getAppId()); + } + Map knownCollectors = + app.context.getKnownCollectors(); + if (knownCollectors != null) { + knownCollectors.remove(app.getAppId()); + } + // stop timelineClient when application get finished. + NMTimelinePublisher nmTimelinePublisher = + app.context.getNMTimelinePublisher(); + if (nmTimelinePublisher != null) { + nmTimelinePublisher.stopTimelineClient(app.getAppId()); + } + } + @Override public void transition(ApplicationImpl app, ApplicationEvent event) { @@ -497,6 +590,7 @@ public class ApplicationImpl implements Application { new LogHandlerAppFinishedEvent(app.appId)); app.context.getNMTokenSecretManager().appFinished(app.getAppId()); + updateCollectorStatus(app); } } @@ -557,4 +651,19 @@ public class ApplicationImpl implements Application { this.readLock.unlock(); } } + + @Override + public String getFlowName() { + return flowContext == null ? null : flowContext.getFlowName(); + } + + @Override + public String getFlowVersion() { + return flowContext == null ? null : flowContext.getFlowVersion(); + } + + @Override + public long getFlowRunId() { + return flowContext == null ? 0L : flowContext.getFlowRunId(); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java index 14aab88880b..b9d1e31c7e2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java @@ -23,6 +23,7 @@ import org.apache.hadoop.security.Credentials; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; @@ -36,6 +37,8 @@ public interface Container extends EventHandler { ContainerId getContainerId(); + long getContainerStartTime(); + Resource getResource(); ContainerTokenIdentifier getContainerTokenIdentifier(); @@ -93,4 +96,6 @@ public interface Container extends EventHandler { boolean isRecovering(); void sendPauseEvent(String description); + + Priority getPriority(); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java index 382b73b1042..53cdbdb9d1b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java @@ -51,6 +51,7 @@ import org.apache.hadoop.yarn.api.records.ContainerRetryPolicy; import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; +import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.Dispatcher; @@ -84,6 +85,7 @@ import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics; import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService; import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredContainerState; import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredContainerStatus; +import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher; import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.state.InvalidStateTransitionException; import org.apache.hadoop.yarn.state.MultipleArcTransition; @@ -91,7 +93,6 @@ import org.apache.hadoop.yarn.state.SingleArcTransition; import org.apache.hadoop.yarn.state.StateMachine; import org.apache.hadoop.yarn.state.StateMachineFactory; import org.apache.hadoop.yarn.util.Clock; -import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.util.SystemClock; import org.apache.hadoop.yarn.util.resource.Resources; @@ -175,11 +176,11 @@ public class ContainerImpl implements Container { /** The NM-wide configuration - not specific to this container */ private final Configuration daemonConf; + private final long startTime; private static final Logger LOG = LoggerFactory.getLogger(ContainerImpl.class); - // whether container has been recovered after a restart private RecoveredContainerStatus recoveredStatus = RecoveredContainerStatus.REQUESTED; @@ -192,6 +193,16 @@ public class ContainerImpl implements Container { ContainerLaunchContext launchContext, Credentials creds, NodeManagerMetrics metrics, ContainerTokenIdentifier containerTokenIdentifier, Context context) { + this(conf, dispatcher, launchContext, creds, metrics, + containerTokenIdentifier, context, SystemClock.getInstance().getTime()); + } + + public ContainerImpl(Configuration conf, Dispatcher dispatcher, + ContainerLaunchContext launchContext, Credentials creds, + NodeManagerMetrics metrics, + ContainerTokenIdentifier containerTokenIdentifier, Context context, + long startTs) { + this.startTime = startTs; this.daemonConf = conf; this.dispatcher = dispatcher; this.stateStore = context.getNMStateStore(); @@ -265,7 +276,7 @@ public class ContainerImpl implements Container { ContainerTokenIdentifier containerTokenIdentifier, Context context, RecoveredContainerState rcs) { this(conf, dispatcher, launchContext, creds, metrics, - containerTokenIdentifier, context); + containerTokenIdentifier, context, rcs.getStartTime()); this.recoveredStatus = rcs.getStatus(); this.exitCode = rcs.getExitCode(); this.recoveredAsKilled = rcs.getKilled(); @@ -750,6 +761,10 @@ public class ContainerImpl implements Container { } } + public NMTimelinePublisher getNMTimelinePublisher() { + return context.getNMTimelinePublisher(); + } + @Override public String getUser() { this.readLock.lock(); @@ -841,6 +856,11 @@ public class ContainerImpl implements Container { return this.containerId; } + @Override + public long getContainerStartTime() { + return this.startTime; + } + @Override public Resource getResource() { return Resources.clone( @@ -904,14 +924,13 @@ public class ContainerImpl implements Container { @SuppressWarnings("rawtypes") EventHandler eventHandler = dispatcher.getEventHandler(); - ContainerStatus containerStatus = cloneAndGetContainerStatus(); - eventHandler.handle( - new ApplicationContainerFinishedEvent( - containerStatus.getContainerId())); - // Tell the scheduler the container is Done eventHandler.handle(new ContainerSchedulerEvent(this, ContainerSchedulerEventType.CONTAINER_COMPLETED)); + ContainerStatus containerStatus = cloneAndGetContainerStatus(); + eventHandler.handle( + new ApplicationContainerFinishedEvent(containerStatus, startTime)); + // Remove the container from the resource-monitor eventHandler.handle(new ContainerStopMonitoringEvent(containerId)); // Tell the logService too @@ -1841,7 +1860,8 @@ public class ContainerImpl implements Container { container.containerMetrics.finished(); } container.sendFinishedEvents(); - //if the current state is NEW it means the CONTAINER_INIT was never + + // if the current state is NEW it means the CONTAINER_INIT was never // sent for the event, thus no need to send the CONTAINER_STOP if (container.getCurrentState() != org.apache.hadoop.yarn.api.records.ContainerState.NEW) { @@ -2137,4 +2157,8 @@ public class ContainerImpl implements Container { getContainerState() == ContainerState.NEW); return isRecovering; } + + public Priority getPriority() { + return containerTokenIdentifier.getPriority(); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java index ed83c93c4e0..a7ea8cc19b8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java @@ -22,6 +22,11 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.AbstractService; import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix; @@ -35,15 +40,14 @@ import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; import org.apache.hadoop.yarn.server.nodemanager.Context; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerKillEvent; +import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher; import org.apache.hadoop.yarn.server.nodemanager.util.NodeManagerHardwareUtils; import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin; import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree; import java.util.Arrays; -import java.util.Map; -import java.util.Map.Entry; -import java.util.concurrent.ConcurrentHashMap; public class ContainersMonitorImpl extends AbstractService implements ContainersMonitor { @@ -81,6 +85,14 @@ public class ContainersMonitorImpl extends AbstractService implements private static final long UNKNOWN_MEMORY_LIMIT = -1L; private int nodeCpuPercentageForYARN; + /** + * Identifies the type of container metric to be published. + */ + @Private + public static enum ContainerMetric { + CPU, MEMORY + } + private ResourceUtilization containersUtilization; private volatile boolean stopped = false; @@ -421,7 +433,9 @@ public class ContainersMonitorImpl extends AbstractService implements } ResourceCalculatorProcessTree pt = - ResourceCalculatorProcessTree.getResourceCalculatorProcessTree(pId, processTreeClass, conf); + ResourceCalculatorProcessTree. + getResourceCalculatorProcessTree( + pId, processTreeClass, conf); ptInfo.setPid(pId); ptInfo.setProcessTree(pt); @@ -458,6 +472,7 @@ public class ContainersMonitorImpl extends AbstractService implements pTree.updateProcessTree(); // update process-tree long currentVmemUsage = pTree.getVirtualMemorySize(); long currentPmemUsage = pTree.getRssMemorySize(); + // if machine has 6 cores and 3 are used, // cpuUsagePercentPerCore should be 300% and // cpuUsageTotalCoresPercentage should be 50% @@ -564,10 +579,19 @@ public class ContainersMonitorImpl extends AbstractService implements trackingContainers.remove(containerId); LOG.info("Removed ProcessTree with root " + pId); } + + ContainerImpl container = + (ContainerImpl) context.getContainers().get(containerId); + NMTimelinePublisher nmMetricsPublisher = + container.getNMTimelinePublisher(); + if (nmMetricsPublisher != null) { + nmMetricsPublisher.reportContainerResourceUsage(container, + currentPmemUsage, cpuUsagePercentPerCore); + } } catch (Exception e) { // Log the exception and proceed to the next container. - LOG.warn("Uncaught exception in ContainerMemoryManager " - + "while managing memory of " + containerId, e); + LOG.warn("Uncaught exception in ContainersMonitorImpl " + + "while monitoring resource of " + containerId, e); } } if (LOG.isDebugEnabled()) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java index 219aeed91e6..c59a84cb48c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java @@ -114,6 +114,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService { "ContainerManager/containers/"; private static final String CONTAINER_REQUEST_KEY_SUFFIX = "/request"; private static final String CONTAINER_VERSION_KEY_SUFFIX = "/version"; + private static final String CONTAINER_START_TIME_KEY_SUFFIX = "/starttime"; private static final String CONTAINER_DIAGS_KEY_SUFFIX = "/diagnostics"; private static final String CONTAINER_LAUNCHED_KEY_SUFFIX = "/launched"; private static final String CONTAINER_QUEUED_KEY_SUFFIX = "/queued"; @@ -260,6 +261,8 @@ public class NMLeveldbStateStoreService extends NMStateStoreService { StartContainerRequestProto.parseFrom(entry.getValue())); } else if (suffix.equals(CONTAINER_VERSION_KEY_SUFFIX)) { rcs.version = Integer.parseInt(asString(entry.getValue())); + } else if (suffix.equals(CONTAINER_START_TIME_KEY_SUFFIX)) { + rcs.setStartTime(Long.parseLong(asString(entry.getValue()))); } else if (suffix.equals(CONTAINER_DIAGS_KEY_SUFFIX)) { rcs.diagnostics = asString(entry.getValue()); } else if (suffix.equals(CONTAINER_QUEUED_KEY_SUFFIX)) { @@ -314,21 +317,23 @@ public class NMLeveldbStateStoreService extends NMStateStoreService { @Override public void storeContainer(ContainerId containerId, int containerVersion, - StartContainerRequest startRequest) throws IOException { + long startTime, StartContainerRequest startRequest) throws IOException { String idStr = containerId.toString(); if (LOG.isDebugEnabled()) { LOG.debug("storeContainer: containerId= " + idStr + ", startRequest= " + startRequest); } - String keyRequest = CONTAINERS_KEY_PREFIX + idStr - + CONTAINER_REQUEST_KEY_SUFFIX; + String keyRequest = getContainerKey(idStr, CONTAINER_REQUEST_KEY_SUFFIX); String keyVersion = getContainerVersionKey(idStr); + String keyStartTime = + getContainerKey(idStr, CONTAINER_START_TIME_KEY_SUFFIX); try { WriteBatch batch = db.createWriteBatch(); try { batch.put(bytes(keyRequest), - ((StartContainerRequestPBImpl) startRequest) - .getProto().toByteArray()); + ((StartContainerRequestPBImpl) startRequest).getProto(). + toByteArray()); + batch.put(bytes(keyStartTime), bytes(Long.toString(startTime))); if (containerVersion != 0) { batch.put(bytes(keyVersion), bytes(Integer.toString(containerVersion))); @@ -344,7 +349,11 @@ public class NMLeveldbStateStoreService extends NMStateStoreService { @VisibleForTesting String getContainerVersionKey(String containerId) { - return CONTAINERS_KEY_PREFIX + containerId + CONTAINER_VERSION_KEY_SUFFIX; + return getContainerKey(containerId, CONTAINER_VERSION_KEY_SUFFIX); + } + + private String getContainerKey(String containerId, String suffix) { + return CONTAINERS_KEY_PREFIX + containerId + suffix; } @Override diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java index 11a8c91e08f..81b8c79e94a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java @@ -71,7 +71,7 @@ public class NMNullStateStoreService extends NMStateStoreService { @Override public void storeContainer(ContainerId containerId, int version, - StartContainerRequest startRequest) throws IOException { + long startTime, StartContainerRequest startRequest) throws IOException { } @Override diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java index c45bc01eef9..bd73b486e69 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java @@ -89,6 +89,7 @@ public abstract class NMStateStoreService extends AbstractService { int version; private RecoveredContainerType recoveryType = RecoveredContainerType.RECOVER; + private long startTime; public RecoveredContainerStatus getStatus() { return status; @@ -110,6 +111,14 @@ public abstract class NMStateStoreService extends AbstractService { return version; } + public long getStartTime() { + return startTime; + } + + public void setStartTime(long ts) { + startTime = ts; + } + public StartContainerRequest getStartRequest() { return startRequest; } @@ -147,6 +156,7 @@ public abstract class NMStateStoreService extends AbstractService { return new StringBuffer("Status: ").append(getStatus()) .append(", Exit code: ").append(exitCode) .append(", Version: ").append(version) + .append(", Start Time: ").append(startTime) .append(", Killed: ").append(getKilled()) .append(", Diagnostics: ").append(getDiagnostics()) .append(", Capability: ").append(getCapability()) @@ -367,11 +377,12 @@ public abstract class NMStateStoreService extends AbstractService { * Record a container start request * @param containerId the container ID * @param containerVersion the container Version + * @param startTime container start time * @param startRequest the container start request * @throws IOException */ public abstract void storeContainer(ContainerId containerId, - int containerVersion, StartContainerRequest startRequest) + int containerVersion, long startTime, StartContainerRequest startRequest) throws IOException; /** diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/authorize/NMPolicyProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/authorize/NMPolicyProvider.java index 89e3d78935a..1c831d75260 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/authorize/NMPolicyProvider.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/authorize/NMPolicyProvider.java @@ -23,6 +23,7 @@ import org.apache.hadoop.security.authorize.PolicyProvider; import org.apache.hadoop.security.authorize.Service; import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB; import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocolPB; import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocolPB; /** @@ -32,18 +33,22 @@ import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocolPB; @InterfaceStability.Unstable public class NMPolicyProvider extends PolicyProvider { - private static final Service[] nodeManagerServices = + private static final Service[] NODE_MANAGER_SERVICES = new Service[] { - new Service( - YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_CONTAINER_MANAGEMENT_PROTOCOL, - ContainerManagementProtocolPB.class), - new Service(YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCE_LOCALIZER, - LocalizationProtocolPB.class) - }; + new Service(YarnConfiguration. + YARN_SECURITY_SERVICE_AUTHORIZATION_CONTAINER_MANAGEMENT_PROTOCOL, + ContainerManagementProtocolPB.class), + new Service(YarnConfiguration. + YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCE_LOCALIZER, + LocalizationProtocolPB.class), + new Service(YarnConfiguration. + YARN_SECURITY_SERVICE_AUTHORIZATION_COLLECTOR_NODEMANAGER_PROTOCOL, + CollectorNodemanagerProtocolPB.class) + }; @Override public Service[] getServices() { - return nodeManagerServices; + return NODE_MANAGER_SERVICES; } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelineEvent.java similarity index 71% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsEvent.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelineEvent.java index 18473968f45..f275b3722c9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsEvent.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelineEvent.java @@ -16,18 +16,20 @@ * limitations under the License. */ -package org.apache.hadoop.yarn.server.resourcemanager.metrics; +package org.apache.hadoop.yarn.server.nodemanager.timelineservice; import org.apache.hadoop.yarn.event.AbstractEvent; -public class SystemMetricsEvent extends AbstractEvent { - - public SystemMetricsEvent(SystemMetricsEventType type) { +/** + * Event posted to NMTimelinePublisher which in turn publishes it to + * timelineservice v2. + */ +public class NMTimelineEvent extends AbstractEvent { + public NMTimelineEvent(NMTimelineEventType type) { super(type); } - public SystemMetricsEvent(SystemMetricsEventType type, long timestamp) { + public NMTimelineEvent(NMTimelineEventType type, long timestamp) { super(type, timestamp); } - -} +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelineEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelineEventType.java new file mode 100644 index 00000000000..b4ae45ad3f9 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelineEventType.java @@ -0,0 +1,27 @@ +/** + * 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.nodemanager.timelineservice; + +/** + * Type of {@link NMTimelineEvent}. + */ +public enum NMTimelineEventType { + // Publish the NM Timeline entity + TIMELINE_ENTITY_PUBLISH, +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java new file mode 100644 index 00000000000..515ff8dfadb --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java @@ -0,0 +1,447 @@ +/** + * 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.nodemanager.timelineservice; + +import java.io.IOException; +import java.security.PrivilegedExceptionAction; +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; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.service.CompositeService; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.CollectorInfo; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerState; +import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.NodeId; +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; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity.Identifier; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetricOperation; +import org.apache.hadoop.yarn.client.api.TimelineV2Client; +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.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent; +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.monitor.ContainersMonitorImpl.ContainerMetric; +import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree; +import org.apache.hadoop.yarn.util.TimelineServiceHelper; +import org.apache.hadoop.yarn.util.timeline.TimelineUtils; + +import com.google.common.annotations.VisibleForTesting; + +/** + * Metrics publisher service that publishes data to the timeline service v.2. It + * is used only if the timeline service v.2 is enabled and the system publishing + * of events and metrics is enabled. + */ +public class NMTimelinePublisher extends CompositeService { + + private static final Log LOG = LogFactory.getLog(NMTimelinePublisher.class); + + private Dispatcher dispatcher; + + private Context context; + + private NodeId nodeId; + + private String httpAddress; + + private UserGroupInformation nmLoginUGI; + + private final Map appToClientMap; + + public NMTimelinePublisher(Context context) { + super(NMTimelinePublisher.class.getName()); + this.context = context; + appToClientMap = new ConcurrentHashMap<>(); + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + dispatcher = new AsyncDispatcher(); + dispatcher.register(NMTimelineEventType.class, + new ForwardingEventHandler()); + addIfService(dispatcher); + this.nmLoginUGI = UserGroupInformation.isSecurityEnabled() ? + UserGroupInformation.getLoginUser() : + UserGroupInformation.getCurrentUser(); + super.serviceInit(conf); + } + + @Override + protected void serviceStart() throws Exception { + super.serviceStart(); + // context will be updated after containerManagerImpl is started + // hence NMMetricsPublisher is added subservice of containerManagerImpl + this.nodeId = context.getNodeId(); + } + + @Override + protected void serviceStop() throws Exception { + for(ApplicationId app : appToClientMap.keySet()) { + stopTimelineClient(app); + } + super.serviceStop(); + } + + @VisibleForTesting + Map getAppToClientMap() { + return appToClientMap; + } + + protected void handleNMTimelineEvent(NMTimelineEvent event) { + switch (event.getType()) { + case TIMELINE_ENTITY_PUBLISH: + putEntity(((TimelinePublishEvent) event).getTimelineEntityToPublish(), + ((TimelinePublishEvent) event).getApplicationId()); + break; + default: + LOG.error("Unknown NMTimelineEvent type: " + event.getType()); + } + } + + public void reportContainerResourceUsage(Container container, Long pmemUsage, + Float cpuUsagePercentPerCore) { + if (pmemUsage != ResourceCalculatorProcessTree.UNAVAILABLE || + cpuUsagePercentPerCore != ResourceCalculatorProcessTree.UNAVAILABLE) { + ContainerEntity entity = + createContainerEntity(container.getContainerId()); + long currentTimeMillis = System.currentTimeMillis(); + if (pmemUsage != ResourceCalculatorProcessTree.UNAVAILABLE) { + TimelineMetric memoryMetric = new TimelineMetric(); + memoryMetric.setId(ContainerMetric.MEMORY.toString()); + memoryMetric.setRealtimeAggregationOp(TimelineMetricOperation.SUM); + memoryMetric.addValue(currentTimeMillis, pmemUsage); + entity.addMetric(memoryMetric); + } + if (cpuUsagePercentPerCore != ResourceCalculatorProcessTree.UNAVAILABLE) { + TimelineMetric cpuMetric = new TimelineMetric(); + cpuMetric.setId(ContainerMetric.CPU.toString()); + // TODO: support average + cpuMetric.setRealtimeAggregationOp(TimelineMetricOperation.SUM); + cpuMetric.addValue(currentTimeMillis, + Math.round(cpuUsagePercentPerCore)); + entity.addMetric(cpuMetric); + } + entity.setIdPrefix(TimelineServiceHelper. + invertLong(container.getContainerStartTime())); + ApplicationId appId = container.getContainerId().getApplicationAttemptId() + .getApplicationId(); + try { + // no need to put it as part of publisher as timeline client already has + // Queuing concept + TimelineV2Client 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); + } + } + } + + @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 entityInfo = new HashMap(); + entityInfo.put(ContainerMetricsConstants.ALLOCATED_MEMORY_INFO, + resource.getMemory()); + entityInfo.put(ContainerMetricsConstants.ALLOCATED_VCORE_INFO, + resource.getVirtualCores()); + entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_INFO, + nodeId.getHost()); + entityInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_INFO, + nodeId.getPort()); + entityInfo.put(ContainerMetricsConstants.ALLOCATED_PRIORITY_INFO, + container.getPriority().toString()); + entityInfo.put( + ContainerMetricsConstants.ALLOCATED_HOST_HTTP_ADDRESS_INFO, + httpAddress); + entity.setInfo(entityInfo); + + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(ContainerMetricsConstants.CREATED_EVENT_TYPE); + tEvent.setTimestamp(event.getTimestamp()); + + long containerStartTime = container.getContainerStartTime(); + entity.addEvent(tEvent); + entity.setCreatedTime(containerStartTime); + entity.setIdPrefix(TimelineServiceHelper.invertLong(containerStartTime)); + dispatcher.getEventHandler().handle(new TimelinePublishEvent(entity, + containerId.getApplicationAttemptId().getApplicationId())); + } + + @SuppressWarnings("unchecked") + private void publishContainerFinishedEvent(ContainerStatus containerStatus, + long containerFinishTime, long containerStartTime) { + ContainerId containerId = containerStatus.getContainerId(); + TimelineEntity entity = createContainerEntity(containerId); + + Map entityInfo = new HashMap(); + entityInfo.put(ContainerMetricsConstants.DIAGNOSTICS_INFO, + containerStatus.getDiagnostics()); + entityInfo.put(ContainerMetricsConstants.EXIT_STATUS_INFO, + containerStatus.getExitStatus()); + entityInfo.put(ContainerMetricsConstants.STATE_INFO, + ContainerState.COMPLETE.toString()); + entityInfo.put(ContainerMetricsConstants.CONTAINER_FINISHED_TIME, + containerFinishTime); + entity.setInfo(entityInfo); + + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(ContainerMetricsConstants.FINISHED_EVENT_TYPE); + tEvent.setTimestamp(containerFinishTime); + entity.addEvent(tEvent); + entity.setIdPrefix(TimelineServiceHelper.invertLong(containerStartTime)); + + 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); + entity.setIdPrefix(TimelineServiceHelper. + invertLong(container.getContainerStartTime())); + + ApplicationId appId = + container.getContainerId().getApplicationAttemptId().getApplicationId(); + try { + // no need to put it as part of publisher as timeline client already has + // Queuing concept + TimelineV2Client 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( + ContainerId containerId) { + ContainerEntity entity = new ContainerEntity(); + entity.setId(containerId.toString()); + Identifier parentIdentifier = new Identifier(); + parentIdentifier + .setType(TimelineEntityType.YARN_APPLICATION_ATTEMPT.name()); + parentIdentifier.setId(containerId.getApplicationAttemptId().toString()); + entity.setParent(parentIdentifier); + return entity; + } + + private void putEntity(TimelineEntity entity, ApplicationId appId) { + try { + if (LOG.isDebugEnabled()) { + LOG.debug("Publishing the entity " + entity + ", JSON-style content: " + + TimelineUtils.dumpTimelineRecordtoJSON(entity)); + } + TimelineV2Client 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); + } + } + + public void publishApplicationEvent(ApplicationEvent event) { + // publish only when the desired event is received + switch (event.getType()) { + case INIT_APPLICATION: + case FINISH_APPLICATION: + case APPLICATION_LOG_HANDLING_FAILED: + // 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(), evnt.getContainerStartTime()); + break; + + default: + if (LOG.isDebugEnabled()) { + LOG.debug(event.getType() + " is not a desired ApplicationEvent which" + + " needs to be published by NMTimelinePublisher"); + } + break; + } + } + + public void publishContainerEvent(ContainerEvent event) { + // publish only when the desired event is received + if (this.httpAddress == null) { + // update httpAddress for first time. When this service started, + // web server will not be started. + this.httpAddress = nodeId.getHost() + ":" + context.getHttpPort(); + } + switch (event.getType()) { + case INIT_CONTAINER: + publishContainerCreatedEvent(event); + break; + + default: + if (LOG.isDebugEnabled()) { + LOG.debug(event.getType() + + " is not a desired ContainerEvent which needs to be published by" + + " NMTimelinePublisher"); + } + break; + } + } + + 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 LOCALIZE_CONTAINER_RESOURCES: + publishContainerLocalizationEvent((ContainerLocalizationEvent) event, + ContainerMetricsConstants.LOCALIZATION_START_EVENT_TYPE); + break; + default: + if (LOG.isDebugEnabled()) { + LOG.debug(event.getType() + + " is not a desired LocalizationEvent which needs to be published" + + " by NMTimelinePublisher"); + } + break; + } + } + + /** + * EventHandler implementation which forward events to NMMetricsPublisher. + * Making use of it, NMMetricsPublisher can avoid to have a public handle + * method. + */ + private final class ForwardingEventHandler implements + EventHandler { + + @Override + public void handle(NMTimelineEvent event) { + handleNMTimelineEvent(event); + } + } + + private static class TimelinePublishEvent extends NMTimelineEvent { + private ApplicationId appId; + private TimelineEntity entityToPublish; + + public TimelinePublishEvent(TimelineEntity entity, ApplicationId appId) { + super(NMTimelineEventType.TIMELINE_ENTITY_PUBLISH, System + .currentTimeMillis()); + this.appId = appId; + this.entityToPublish = entity; + } + + public ApplicationId getApplicationId() { + return appId; + } + + public TimelineEntity getTimelineEntityToPublish() { + return entityToPublish; + } + } + + public void createTimelineClient(final ApplicationId appId) { + if (!appToClientMap.containsKey(appId)) { + try { + TimelineV2Client timelineClient = + nmLoginUGI.doAs(new PrivilegedExceptionAction() { + @Override + public TimelineV2Client run() throws Exception { + TimelineV2Client timelineClient = + TimelineV2Client.createTimelineClient(appId); + timelineClient.init(getConfig()); + timelineClient.start(); + return timelineClient; + } + }); + appToClientMap.put(appId, timelineClient); + } catch (IOException | InterruptedException | RuntimeException | + Error e) { + LOG.warn("Unable to create timeline client for app " + appId, e); + } + } + } + + public void stopTimelineClient(ApplicationId appId) { + TimelineV2Client client = appToClientMap.remove(appId); + if (client != null) { + client.stop(); + } + } + + public void setTimelineServiceAddress(ApplicationId appId, + String collectorAddr) { + TimelineV2Client client = appToClientMap.get(appId); + if (client != null) { + client.setTimelineCollectorInfo(CollectorInfo.newInstance(collectorAddr)); + } + } + + private TimelineV2Client getTimelineClient(ApplicationId appId) { + return appToClientMap.get(appId); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/package-info.java new file mode 100644 index 00000000000..66233fd655c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/package-info.java @@ -0,0 +1,29 @@ +/* + * 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.nodemanager.timelineservice contains + * classes related to publishing container events and other NM lifecycle events + * to ATSv2. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.nodemanager.timelineservice; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/proto/yarn_server_nodemanager_recovery.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/proto/yarn_server_nodemanager_recovery.proto index ecb9835d80d..7212953268d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/proto/yarn_server_nodemanager_recovery.proto +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/proto/yarn_server_nodemanager_recovery.proto @@ -31,6 +31,7 @@ message ContainerManagerApplicationProto { repeated ApplicationACLMapProto acls = 4; optional LogAggregationContextProto log_aggregation_context = 5; optional int64 appLogAggregationInitedTime = 6 [ default = -1 ]; + optional FlowContextProto flowContext = 7; } message DeletionServiceDeleteTaskProto { @@ -53,3 +54,9 @@ message LogDeleterProto { optional string user = 1; optional int64 deletionTime = 2; } + +message FlowContextProto { + optional string flowName = 1; + optional string flowVersion = 2; + optional int64 flowRunId = 3; +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java index 58fc9e786fd..055dab44897 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java @@ -1097,6 +1097,7 @@ public class TestNodeStatusUpdater { BuilderUtils.newContainerToken(containerId, 0, "host", 1234, "user", BuilderUtils.newResource(1024, 1), 0, 123, "password".getBytes(), 0); + Container completedContainer = new ContainerImpl(conf, null, null, null, null, BuilderUtils.newContainerTokenIdentifier(containerToken), diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java index 52065d4f699..3afbce76d56 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java @@ -61,6 +61,7 @@ import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport; +import org.apache.hadoop.yarn.server.api.records.AppCollectorData; import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus; import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; import org.apache.hadoop.yarn.server.nodemanager.Context; @@ -76,6 +77,7 @@ import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService; import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerAllocator; import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager; import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM; +import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; import org.apache.hadoop.yarn.util.Records; import org.junit.After; @@ -659,6 +661,17 @@ public abstract class BaseAMRMProxyTest { return null; } + @Override + public ConcurrentMap + getRegisteringCollectors() { + return null; + } + + @Override + public ConcurrentMap getKnownCollectors() { + return null; + } + @Override public ConcurrentMap getContainers() { return null; @@ -752,5 +765,13 @@ public abstract class BaseAMRMProxyTest { getContainerStateTransitionListener() { return null; } + + public void setNMTimelinePublisher(NMTimelinePublisher nmMetricsPublisher) { + } + + @Override + public NMTimelinePublisher getNMTimelinePublisher() { + return null; + } } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java index 55f362428b7..d17a3fa15c0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java @@ -22,7 +22,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.isA; +import static org.mockito.Matchers.isA; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; @@ -105,7 +105,9 @@ import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreServic import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService; import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager; import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM; +import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; +import org.apache.hadoop.yarn.util.timeline.TimelineUtils; import org.junit.Before; import org.junit.Test; @@ -135,6 +137,11 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest { conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath()); conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogDir.getAbsolutePath()); conf.setLong(YarnConfiguration.NM_LOG_RETAIN_SECONDS, 1); + + // enable atsv2 by default in test + conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); + conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f); + // Default delSrvc delSrvc = createDeletionService(); delSrvc.init(conf); @@ -143,6 +150,7 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest { nodeHealthChecker = new NodeHealthCheckerService( NodeManager.getNodeHealthScriptRunner(conf), dirsHandler); nodeHealthChecker.init(conf); + } @Test @@ -160,6 +168,7 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest { cm.start(); // add an application by starting a container + String appName = "app_name1"; String appUser = "app_user1"; String modUser = "modify_user1"; String viewUser = "view_user1"; @@ -169,7 +178,8 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest { ApplicationAttemptId.newInstance(appId, 1); ContainerId cid = ContainerId.newContainerId(attemptId, 1); Map localResources = Collections.emptyMap(); - Map containerEnv = Collections.emptyMap(); + Map containerEnv = new HashMap<>(); + setFlowContext(containerEnv, appName, appId); List containerCmds = Collections.emptyList(); Map serviceData = Collections.emptyMap(); Credentials containerCreds = new Credentials(); @@ -317,7 +327,8 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest { ApplicationAttemptId.newInstance(appId, 1); ContainerId cid = ContainerId.newContainerId(attemptId, 1); Map localResources = Collections.emptyMap(); - Map containerEnv = Collections.emptyMap(); + Map containerEnv = new HashMap<>(); + setFlowContext(containerEnv, "app_name1", appId); List containerCmds = Collections.emptyList(); Map serviceData = Collections.emptyMap(); @@ -399,7 +410,8 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest { ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1); ContainerId cid = ContainerId.newContainerId(attemptId, 1); - Map containerEnv = Collections.emptyMap(); + Map containerEnv = new HashMap<>(); + setFlowContext(containerEnv, "app_name1", appId); Map serviceData = Collections.emptyMap(); Credentials containerCreds = new Credentials(); DataOutputBuffer dob = new DataOutputBuffer(); @@ -476,7 +488,8 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest { ApplicationAttemptId.newInstance(appId, 1); ContainerId cid = ContainerId.newContainerId(attemptId, 1); Map localResources = Collections.emptyMap(); - Map containerEnv = Collections.emptyMap(); + Map containerEnv = new HashMap<>(); + setFlowContext(containerEnv, "app_name1", appId); List containerCmds = Collections.emptyList(); Map serviceData = Collections.emptyMap(); Credentials containerCreds = new Credentials(); @@ -750,6 +763,32 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest { Context context, ContainerExecutor exec) { return launcher; } + + @Override + public NMTimelinePublisher + createNMTimelinePublisher(Context context) { + return null; + } }; } + + private void setFlowContext(Map containerEnv, String appName, + ApplicationId appId) { + if (YarnConfiguration.timelineServiceV2Enabled(conf)) { + setFlowTags(containerEnv, TimelineUtils.FLOW_NAME_TAG_PREFIX, + TimelineUtils.generateDefaultFlowName(appName, appId)); + setFlowTags(containerEnv, TimelineUtils.FLOW_VERSION_TAG_PREFIX, + TimelineUtils.DEFAULT_FLOW_VERSION); + setFlowTags(containerEnv, TimelineUtils.FLOW_RUN_ID_TAG_PREFIX, + String.valueOf(System.currentTimeMillis())); + } + } + + private static void setFlowTags(Map environment, + String tagPrefix, String value) { + if (!value.isEmpty()) { + environment.put(tagPrefix, value); + } + } + } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/TestApplication.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/TestApplication.java index 157ba97cfe3..65558e937cb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/TestApplication.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/TestApplication.java @@ -34,7 +34,9 @@ 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.ContainerLaunchContext; +import org.apache.hadoop.yarn.api.records.ContainerState; import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.event.DrainDispatcher; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.ContainerManagerApplicationProto; @@ -539,7 +541,8 @@ public class TestApplication { new ApplicationACLsManager(conf)); when(context.getNMTokenSecretManager()).thenReturn(nmTokenSecretMgr); when(context.getNMStateStore()).thenReturn(stateStoreService); - + when(context.getConf()).thenReturn(conf); + // Setting master key MasterKey masterKey = new MasterKeyPBImpl(); masterKey.setKeyId(123); @@ -550,7 +553,8 @@ public class TestApplication { this.user = user; this.appId = BuilderUtils.newApplicationId(timestamp, id); - app = new ApplicationImpl(dispatcher, this.user, appId, null, context); + app = new ApplicationImpl( + dispatcher, this.user, appId, null, context); containers = new ArrayList(); for (int i = 0; i < numContainers; i++) { Container container = createMockedContainer(this.appId, i); @@ -597,7 +601,7 @@ public class TestApplication { public void containerFinished(int containerNum) { app.handle(new ApplicationContainerFinishedEvent(containers.get( - containerNum).getContainerId())); + containerNum).cloneAndGetContainerStatus(), 0)); drainDispatcherEvents(); } @@ -641,6 +645,9 @@ public class TestApplication { when(c.getLaunchContext()).thenReturn(launchContext); when(launchContext.getApplicationACLs()).thenReturn( new HashMap()); + when(c.cloneAndGetContainerStatus()).thenReturn( + BuilderUtils.newContainerStatus(cId, + ContainerState.NEW, "", 0, Resource.newInstance(1024, 1))); return c; } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java index 6ca8aa09710..9f8c0cb24fb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java @@ -127,10 +127,12 @@ public class NMMemoryStateStoreService extends NMStateStoreService { @Override public synchronized void storeContainer(ContainerId containerId, - int version, StartContainerRequest startRequest) throws IOException { + int version, long startTime, StartContainerRequest startRequest) + throws IOException { RecoveredContainerState rcs = new RecoveredContainerState(); rcs.startRequest = startRequest; rcs.version = version; + rcs.setStartTime(startTime); containerStates.put(containerId, rcs); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java index b8a55c07aee..3ab02ebe246 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java @@ -234,7 +234,8 @@ public class TestNMLeveldbStateStoreService { StartContainerRequest containerReq = createContainerRequest(containerId); // store a container and verify recovered - stateStore.storeContainer(containerId, 0, containerReq); + long containerStartTime = System.currentTimeMillis(); + stateStore.storeContainer(containerId, 0, containerStartTime, containerReq); // verify the container version key is not stored for new containers DB db = stateStore.getDB(); @@ -246,6 +247,7 @@ public class TestNMLeveldbStateStoreService { assertEquals(1, recoveredContainers.size()); RecoveredContainerState rcs = recoveredContainers.get(0); assertEquals(0, rcs.getVersion()); + assertEquals(containerStartTime, rcs.getStartTime()); assertEquals(RecoveredContainerStatus.REQUESTED, rcs.getStatus()); assertEquals(ContainerExitStatus.INVALID, rcs.getExitCode()); assertEquals(false, rcs.getKilled()); @@ -1021,7 +1023,7 @@ public class TestNMLeveldbStateStoreService { StartContainerRequest containerReq = StartContainerRequest.newInstance(clc, containerToken); - stateStore.storeContainer(containerId, 0, containerReq); + stateStore.storeContainer(containerId, 0, 0, containerReq); // add a invalid key byte[] invalidKey = ("ContainerManager/containers/" diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/TestNMTimelinePublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/TestNMTimelinePublisher.java new file mode 100644 index 00000000000..50f9e8ba122 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/TestNMTimelinePublisher.java @@ -0,0 +1,175 @@ +/** +* 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.nodemanager.timelineservice; + +/** + * Tests publishing of entities from NM to ATSv2. + */ +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.util.Iterator; +import java.util.Map.Entry; + +import org.apache.hadoop.conf.Configuration; +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.NodeId; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.client.api.impl.TimelineV2ClientImpl; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree; +import org.junit.Assert; +import org.junit.Test; + +/** + * Tests {@link NMTimelinePublisher}. + */ +public class TestNMTimelinePublisher { + private static final String MEMORY_ID = "MEMORY"; + private static final String CPU_ID = "CPU"; + + @Test + public void testContainerResourceUsage() { + Context context = mock(Context.class); + @SuppressWarnings("unchecked") + final DummyTimelineClient timelineClient = new DummyTimelineClient(null); + when(context.getNodeId()).thenReturn(NodeId.newInstance("localhost", 0)); + when(context.getHttpPort()).thenReturn(0); + + Configuration conf = new Configuration(); + conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); + conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f); + + NMTimelinePublisher publisher = new NMTimelinePublisher(context) { + public void createTimelineClient(ApplicationId appId) { + if (!getAppToClientMap().containsKey(appId)) { + timelineClient.init(getConfig()); + timelineClient.start(); + getAppToClientMap().put(appId, timelineClient); + } + } + }; + publisher.init(conf); + publisher.start(); + ApplicationId appId = ApplicationId.newInstance(0, 1); + publisher.createTimelineClient(appId); + Container aContainer = mock(Container.class); + when(aContainer.getContainerId()).thenReturn(ContainerId.newContainerId( + ApplicationAttemptId.newInstance(appId, 1), + 0L)); + publisher.reportContainerResourceUsage(aContainer, 1024L, 8F); + verifyPublishedResourceUsageMetrics(timelineClient, 1024L, 8); + timelineClient.reset(); + + publisher.reportContainerResourceUsage(aContainer, 1024L, 0.8F); + verifyPublishedResourceUsageMetrics(timelineClient, 1024L, 1); + timelineClient.reset(); + + publisher.reportContainerResourceUsage(aContainer, 1024L, 0.49F); + verifyPublishedResourceUsageMetrics(timelineClient, 1024L, 0); + timelineClient.reset(); + + publisher.reportContainerResourceUsage(aContainer, 1024L, + (float) ResourceCalculatorProcessTree.UNAVAILABLE); + verifyPublishedResourceUsageMetrics(timelineClient, 1024L, + ResourceCalculatorProcessTree.UNAVAILABLE); + publisher.stop(); + } + + private void verifyPublishedResourceUsageMetrics( + DummyTimelineClient timelineClient, long memoryUsage, int cpuUsage) { + TimelineEntity[] entities = null; + for (int i = 0; i < 10; i++) { + entities = timelineClient.getLastPublishedEntities(); + if (entities != null) { + break; + } + try { + Thread.sleep(150L); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + int numberOfResourceMetrics = 0; + numberOfResourceMetrics += + (memoryUsage == ResourceCalculatorProcessTree.UNAVAILABLE) ? 0 : 1; + numberOfResourceMetrics += + (cpuUsage == ResourceCalculatorProcessTree.UNAVAILABLE) ? 0 : 1; + assertNotNull("entities are expected to be published", entities); + assertEquals("Expected number of metrics notpublished", + numberOfResourceMetrics, entities[0].getMetrics().size()); + Iterator metrics = entities[0].getMetrics().iterator(); + while (metrics.hasNext()) { + TimelineMetric metric = metrics.next(); + Iterator> entrySet; + switch (metric.getId()) { + case CPU_ID: + if (cpuUsage == ResourceCalculatorProcessTree.UNAVAILABLE) { + Assert.fail("Not Expecting CPU Metric to be published"); + } + entrySet = metric.getValues().entrySet().iterator(); + assertEquals("CPU usage metric not matching", cpuUsage, + entrySet.next().getValue()); + break; + case MEMORY_ID: + if (memoryUsage == ResourceCalculatorProcessTree.UNAVAILABLE) { + Assert.fail("Not Expecting Memory Metric to be published"); + } + entrySet = metric.getValues().entrySet().iterator(); + assertEquals("Memory usage metric not matching", memoryUsage, + entrySet.next().getValue()); + break; + default: + Assert.fail("Invalid Resource Usage metric"); + break; + } + } + } + + protected static class DummyTimelineClient extends TimelineV2ClientImpl { + public DummyTimelineClient(ApplicationId appId) { + super(appId); + } + + private TimelineEntity[] lastPublishedEntities; + + @Override + public void putEntitiesAsync(TimelineEntity... entities) + throws IOException, YarnException { + this.lastPublishedEntities = entities; + } + + public TimelineEntity[] getLastPublishedEntities() { + return lastPublishedEntities; + } + + public void reset() { + lastPublishedEntities = null; + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockApp.java index 4e130109f4f..8feca21ccce 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockApp.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockApp.java @@ -39,6 +39,9 @@ public class MockApp implements Application { Map containers = new HashMap(); ApplicationState appState; Application app; + private String flowName; + private String flowVersion; + private long flowRunId; public MockApp(int uniqId) { this("mockUser", 1234, uniqId); @@ -55,6 +58,14 @@ public class MockApp implements Application { appState = ApplicationState.NEW; } + public MockApp(String user, long clusterTimeStamp, int uniqId, + String flowName, String flowVersion, long flowRunId) { + this(user, clusterTimeStamp, uniqId); + this.flowName = flowName; + this.flowVersion = flowVersion; + this.flowRunId = flowRunId; + } + public void setState(ApplicationState state) { this.appState = state; } @@ -77,4 +88,15 @@ public class MockApp implements Application { public void handle(ApplicationEvent event) {} + public String getFlowName() { + return flowName; + } + + public String getFlowVersion() { + return flowVersion; + } + + public long getFlowRunId() { + return flowRunId; + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java index 07df56ceb7a..b9c6fffd6b2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java @@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.factories.RecordFactory; @@ -207,7 +208,10 @@ public class MockContainer implements Container { @Override public void commitUpgrade() { + } + public Priority getPriority() { + return Priority.UNDEFINED; } @Override @@ -232,6 +236,10 @@ public class MockContainer implements Container { @Override public void sendPauseEvent(String description) { + } + @Override + public long getContainerStartTime() { + return 0; } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestContainerLogsPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestContainerLogsPage.java index 33a821e1915..d8b40e71079 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestContainerLogsPage.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestContainerLogsPage.java @@ -21,8 +21,8 @@ package org.apache.hadoop.yarn.server.nodemanager.webapp; import static org.junit.Assume.assumeTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.when; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; import java.io.BufferedOutputStream; import java.io.File; @@ -47,10 +47,9 @@ import org.apache.hadoop.util.NodeHealthScriptRunner; 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.impl.pb.ContainerIdPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl; +import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.exceptions.YarnException; @@ -63,6 +62,7 @@ import org.apache.hadoop.yarn.server.nodemanager.NodeManager; import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState; import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch; import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreService; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml index 75ba54a6085..85df2c0d784 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml @@ -162,6 +162,12 @@ log4j log4j + + org.apache.hadoop + hadoop-yarn-server-timelineservice + test + test-jar + @@ -174,6 +180,11 @@ hadoop-yarn-server-applicationhistoryservice ${project.version} + + org.apache.hadoop + hadoop-yarn-server-timelineservice + provided + org.apache.hadoop hadoop-yarn-server-web-proxy diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java index 99fa7514bb2..90c42be8d44 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java @@ -45,8 +45,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse; import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest; import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; - - import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; @@ -62,6 +60,7 @@ import org.apache.hadoop.yarn.ipc.YarnRPC; import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl; @@ -257,6 +256,11 @@ public class ApplicationMasterService extends AbstractService implements RMApp rmApp = rmContext.getRMApps().get(applicationAttemptId.getApplicationId()); + + // Remove collector address when app get finished. + if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) { + ((RMAppImpl) rmApp).removeCollectorData(); + } // checking whether the app exits in RMStateStore at first not to throw // ApplicationDoesNotExistInCacheException before and after // RM work-preserving restart. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java index 6ba6b91ba96..26471308f30 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java @@ -173,6 +173,7 @@ import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.Clock; import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.UTCClock; +import org.apache.hadoop.yarn.util.timeline.TimelineUtils; import com.google.common.annotations.VisibleForTesting; @@ -560,6 +561,29 @@ public class ClientRMService extends AbstractService implements throw RPCUtil.getRemoteException(ie); } + if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) { + // Sanity check for flow run + String value = null; + try { + for (String tag : submissionContext.getApplicationTags()) { + if (tag.startsWith(TimelineUtils.FLOW_RUN_ID_TAG_PREFIX + ":") || + tag.startsWith( + TimelineUtils.FLOW_RUN_ID_TAG_PREFIX.toLowerCase() + ":")) { + value = tag.substring(TimelineUtils.FLOW_RUN_ID_TAG_PREFIX.length() + + 1); + Long.valueOf(value); + } + } + } catch (NumberFormatException e) { + LOG.warn("Invalid to flow run: " + value + + ". Flow run should be a long integer", e); + RMAuditLogger.logFailure(user, AuditConstants.SUBMIT_APP_REQUEST, + e.getMessage(), "ClientRMService", + "Exception in submitting application", applicationId); + throw RPCUtil.getRemoteException(e); + } + } + // Check whether app has already been put into rmContext, // If it is, simply return the response if (rmContext.getRMApps().get(applicationId) != null) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DefaultAMSProcessor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DefaultAMSProcessor.java index ef32699189f..d5444b48091 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DefaultAMSProcessor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DefaultAMSProcessor.java @@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterReque import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; +import org.apache.hadoop.yarn.api.records.CollectorInfo; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerUpdateType; @@ -295,6 +296,15 @@ final class DefaultAMSProcessor implements ApplicationMasterServiceProcessor { response.setNumClusterNodes(getScheduler().getNumClusterNodes()); + // add collector address for this application + if (YarnConfiguration.timelineServiceV2Enabled( + getRmContext().getYarnConfiguration())) { + CollectorInfo collectorInfo = app.getCollectorInfo(); + if (collectorInfo != null) { + response.setCollectorInfo(collectorInfo); + } + } + // add preemption to the allocateResponse message (if any) response.setPreemptionMessage(generatePreemptionMessage(allocation)); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java index d5eaca6059c..9dc59451499 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java @@ -30,8 +30,8 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.Dispatcher; -import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMDelegatedNodeLabelsUpdater; +import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager; import org.apache.hadoop.yarn.server.resourcemanager.recovery.NullRMStateStore; import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore; @@ -53,9 +53,9 @@ import org.apache.hadoop.yarn.util.Clock; import org.apache.hadoop.yarn.util.SystemClock; /** - * The RMActiveServiceContext is the class that maintains all the - * RMActiveService contexts.This is expected to be used only by ResourceManager - * and RMContext. + * The RMActiveServiceContext is the class that maintains Active service + * context. Services that need to run only on the Active RM. This is expected to + * be used only by RMContext. */ @Private @Unstable @@ -94,6 +94,7 @@ public class RMActiveServiceContext { private NodesListManager nodesListManager; private ResourceTrackerService resourceTrackerService; private ApplicationMasterService applicationMasterService; + private RMNodeLabelsManager nodeLabelManager; private RMDelegatedNodeLabelsUpdater rmDelegatedNodeLabelsUpdater; private long epoch; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java index aa3caf6a9c6..e365112b429 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java @@ -294,8 +294,10 @@ public class RMAppManager implements EventHandler, String user) throws YarnException { ApplicationId applicationId = submissionContext.getApplicationId(); - RMAppImpl application = - createAndPopulateNewRMApp(submissionContext, submitTime, user, false); + // Passing start time as -1. It will be eventually set in RMAppImpl + // constructor. + RMAppImpl application = createAndPopulateNewRMApp( + submissionContext, submitTime, user, false, -1); try { if (UserGroupInformation.isSecurityEnabled()) { this.rmContext.getDelegationTokenRenewer() @@ -332,14 +334,15 @@ public class RMAppManager implements EventHandler, // create and recover app. RMAppImpl application = createAndPopulateNewRMApp(appContext, appState.getSubmitTime(), - appState.getUser(), true); + appState.getUser(), true, appState.getStartTime()); application.handle(new RMAppRecoverEvent(appId, rmState)); } private RMAppImpl createAndPopulateNewRMApp( ApplicationSubmissionContext submissionContext, long submitTime, - String user, boolean isRecovery) throws YarnException { + String user, boolean isRecovery, long startTime) throws YarnException { + // Do queue mapping if (!isRecovery) { // Do queue mapping if (rmContext.getQueuePlacementManager() != null) { @@ -392,12 +395,13 @@ public class RMAppManager implements EventHandler, } // Create RMApp - RMAppImpl application = new RMAppImpl(applicationId, rmContext, this.conf, - submissionContext.getApplicationName(), user, - submissionContext.getQueue(), submissionContext, this.scheduler, - this.masterService, submitTime, submissionContext.getApplicationType(), - submissionContext.getApplicationTags(), amReqs); - + RMAppImpl application = + new RMAppImpl(applicationId, rmContext, this.conf, + submissionContext.getApplicationName(), user, + submissionContext.getQueue(), + submissionContext, this.scheduler, this.masterService, + submitTime, submissionContext.getApplicationType(), + submissionContext.getApplicationTags(), amReqs, startTime); // Concurrent app submissions with same applicationId will fail here // Concurrent app submissions with different applicationIds will not // influence each other @@ -408,6 +412,11 @@ public class RMAppManager implements EventHandler, LOG.warn(message); throw new YarnException(message); } + + if (YarnConfiguration.timelineServiceV2Enabled(conf)) { + // Start timeline collector for the submitted app + application.startTimelineCollector(); + } // Inform the ACLs Manager this.applicationACLsManager.addApplication(applicationId, submissionContext.getAMContainerSpec().getApplicationACLs()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java index 02c06ca37e0..b255a304eae 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java @@ -49,6 +49,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRen import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM; import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager; import org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager; +import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager; /** * Context of the ResourceManager. @@ -115,6 +116,11 @@ public interface RMContext extends ApplicationMasterServiceContext { SystemMetricsPublisher getSystemMetricsPublisher(); + void setRMTimelineCollectorManager( + RMTimelineCollectorManager timelineCollectorManager); + + RMTimelineCollectorManager getRMTimelineCollectorManager(); + ConfigurationProvider getConfigurationProvider(); boolean isWorkPreservingRecoveryEnabled(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java index e04ff1ad1a7..d7c624d4863 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java @@ -26,8 +26,8 @@ import java.util.concurrent.ConcurrentMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience.Private; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState; import org.apache.hadoop.yarn.LocalConfigurationProvider; import org.apache.hadoop.yarn.api.records.ApplicationId; @@ -56,6 +56,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManag import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager; import org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager; import org.apache.hadoop.yarn.server.webproxy.ProxyUriUtils; +import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager; import org.apache.hadoop.yarn.util.Clock; import org.apache.hadoop.yarn.webapp.util.WebAppUtils; @@ -246,6 +247,17 @@ public class RMContextImpl implements RMContext { serviceContext.setSystemMetricsPublisher(metricsPublisher); } + @Override + public RMTimelineCollectorManager getRMTimelineCollectorManager() { + return serviceContext.getRMTimelineCollectorManager(); + } + + @Override + public void setRMTimelineCollectorManager( + RMTimelineCollectorManager timelineCollectorManager) { + serviceContext.setRMTimelineCollectorManager(timelineCollectorManager); + } + @Override public ConfigurationProvider getConfigurationProvider() { return serviceContext.getConfigurationProvider(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServiceContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServiceContext.java index fe34d630b7b..45c61667cf9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServiceContext.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServiceContext.java @@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.conf.ConfigurationProvider; import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter; import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher; +import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager; /** * RMServiceContext class maintains "Always On" services. Services that need to @@ -52,6 +53,7 @@ public class RMServiceContext { private EmbeddedElector elector; private final Object haServiceStateLock = new Object(); private ResourceManager resourceManager; + private RMTimelineCollectorManager timelineCollectorManager; public ResourceManager getResourceManager() { return resourceManager; @@ -140,6 +142,15 @@ public class RMServiceContext { this.yarnConfiguration = yarnConfiguration; } + public RMTimelineCollectorManager getRMTimelineCollectorManager() { + return timelineCollectorManager; + } + + public void setRMTimelineCollectorManager( + RMTimelineCollectorManager collectorManager) { + this.timelineCollectorManager = collectorManager; + } + public String getHAZookeeperConnectionState() { if (elector == null) { return "Could not find leader elector. Verify both HA and automatic " diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java index a573e44114b..a345b97d97d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java @@ -18,7 +18,16 @@ package org.apache.hadoop.yarn.server.resourcemanager; -import com.google.common.annotations.VisibleForTesting; +import java.io.IOException; +import java.io.InputStream; +import java.io.PrintStream; +import java.net.InetSocketAddress; +import java.nio.charset.Charset; +import java.security.PrivilegedExceptionAction; +import java.security.SecureRandom; +import java.util.ArrayList; +import java.util.List; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.curator.framework.AuthInfo; @@ -61,7 +70,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWri import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType; import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher; import org.apache.hadoop.yarn.server.resourcemanager.federation.FederationStateStoreService; +import org.apache.hadoop.yarn.server.resourcemanager.metrics.NoOpSystemMetricPublisher; import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher; +import org.apache.hadoop.yarn.server.resourcemanager.metrics.TimelineServiceV1Publisher; +import org.apache.hadoop.yarn.server.resourcemanager.metrics.TimelineServiceV2Publisher; import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy; import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitor; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMDelegatedNodeLabelsUpdater; @@ -91,6 +103,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEv import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType; import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer; import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager; +import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager; import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebApp; import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebAppUtil; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; @@ -104,16 +117,8 @@ import org.apache.hadoop.yarn.webapp.WebApps.Builder; import org.apache.hadoop.yarn.webapp.util.WebAppUtils; import org.apache.zookeeper.server.auth.DigestAuthenticationProvider; -import java.io.IOException; -import java.io.InputStream; -import java.io.PrintStream; -import java.net.InetSocketAddress; -import java.nio.charset.Charset; -import java.security.PrivilegedExceptionAction; -import java.security.SecureRandom; -import java.util.ArrayList; -import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; +import com.google.common.annotations.VisibleForTesting; /** * The ResourceManager is the main class that is a set of components. @@ -292,8 +297,18 @@ public class ResourceManager extends CompositeService implements Recoverable { addService(rmApplicationHistoryWriter); rmContext.setRMApplicationHistoryWriter(rmApplicationHistoryWriter); - SystemMetricsPublisher systemMetricsPublisher = createSystemMetricsPublisher(); - addService(systemMetricsPublisher); + // initialize the RM timeline collector first so that the system metrics + // publisher can bind to it + if (YarnConfiguration.timelineServiceV2Enabled(this.conf)) { + RMTimelineCollectorManager timelineCollectorManager = + createRMTimelineCollectorManager(); + addService(timelineCollectorManager); + rmContext.setRMTimelineCollectorManager(timelineCollectorManager); + } + + SystemMetricsPublisher systemMetricsPublisher = + createSystemMetricsPublisher(); + addIfService(systemMetricsPublisher); rmContext.setSystemMetricsPublisher(systemMetricsPublisher); super.serviceInit(this.conf); @@ -466,8 +481,30 @@ public class ResourceManager extends CompositeService implements Recoverable { return new FederationStateStoreService(rmContext); } + private RMTimelineCollectorManager createRMTimelineCollectorManager() { + return new RMTimelineCollectorManager(rmContext); + } + protected SystemMetricsPublisher createSystemMetricsPublisher() { - return new SystemMetricsPublisher(); + SystemMetricsPublisher publisher; + if (YarnConfiguration.timelineServiceEnabled(conf) && + YarnConfiguration.systemMetricsPublisherEnabled(conf)) { + if (YarnConfiguration.timelineServiceV2Enabled(conf)) { + // we're dealing with the v.2.x publisher + LOG.info("system metrics publisher with the timeline service V2 is " + + "configured"); + publisher = new TimelineServiceV2Publisher(rmContext); + } else { + // we're dealing with the v.1.x publisher + LOG.info("system metrics publisher with the timeline service V1 is " + + "configured"); + publisher = new TimelineServiceV1Publisher(); + } + } else { + LOG.info("TimelineServicePublisher is not configured"); + publisher = new NoOpSystemMetricPublisher(); + } + return publisher; } // sanity check for configurations diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java index eb84033836f..e62da0088b6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java @@ -23,9 +23,11 @@ import java.net.InetSocketAddress; import java.nio.ByteBuffer; import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock; import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock; @@ -62,12 +64,14 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequ import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.UnRegisterNodeManagerRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.UnRegisterNodeManagerResponse; +import org.apache.hadoop.yarn.server.api.records.AppCollectorData; import org.apache.hadoop.yarn.server.api.records.MasterKey; import org.apache.hadoop.yarn.server.api.records.NodeAction; import org.apache.hadoop.yarn.server.api.records.NodeStatus; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NodeLabelsUtils; import org.apache.hadoop.yarn.server.resourcemanager.resource.DynamicResourceConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; @@ -117,6 +121,8 @@ public class ResourceTrackerService extends AbstractService implements private boolean isDelegatedCentralizedNodeLabelsConf; private DynamicResourceConfiguration drConf; + private final AtomicLong timelineCollectorVersion = new AtomicLong(0); + public ResourceTrackerService(RMContext rmContext, NodesListManager nodesListManager, NMLivelinessMonitor nmLivelinessMonitor, @@ -534,6 +540,13 @@ public class ResourceTrackerService extends AbstractService implements NodeAction.SHUTDOWN, message); } + boolean timelineV2Enabled = + YarnConfiguration.timelineServiceV2Enabled(getConfig()); + if (timelineV2Enabled) { + // Check & update collectors info from request. + updateAppCollectorsMap(request); + } + // Heartbeat response NodeHeartbeatResponse nodeHeartBeatResponse = YarnServerBuilderUtils .newNodeHeartbeatResponse(lastNodeHeartbeatResponse. @@ -551,6 +564,12 @@ public class ResourceTrackerService extends AbstractService implements nodeHeartBeatResponse.setSystemCredentialsForApps(systemCredentials); } + if (timelineV2Enabled) { + // Return collectors' map that NM needs to know + setAppCollectorsMapToResponse(rmNode.getRunningApps(), + nodeHeartBeatResponse); + } + // 4. Send status to RMNode, saving the latest response. RMNodeStatusEvent nodeStatusEvent = new RMNodeStatusEvent(nodeId, remoteNodeStatus, nodeHeartBeatResponse); @@ -594,6 +613,75 @@ public class ResourceTrackerService extends AbstractService implements return nodeHeartBeatResponse; } + private void setAppCollectorsMapToResponse( + List runningApps, NodeHeartbeatResponse response) { + Map liveAppCollectorsMap = new + HashMap<>(); + Map rmApps = rmContext.getRMApps(); + // Set collectors for all running apps on this node. + for (ApplicationId appId : runningApps) { + RMApp app = rmApps.get(appId); + if (app != null) { + AppCollectorData appCollectorData = rmApps.get(appId) + .getCollectorData(); + if (appCollectorData != null) { + liveAppCollectorsMap.put(appId, appCollectorData); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Collector for applicaton: " + appId + + " hasn't registered yet!"); + } + } + } + } + response.setAppCollectors(liveAppCollectorsMap); + } + + private void updateAppCollectorsMap(NodeHeartbeatRequest request) { + Map registeringCollectorsMap = + request.getRegisteringCollectors(); + if (registeringCollectorsMap != null + && !registeringCollectorsMap.isEmpty()) { + Map rmApps = rmContext.getRMApps(); + for (Map.Entry entry: + registeringCollectorsMap.entrySet()) { + ApplicationId appId = entry.getKey(); + AppCollectorData collectorData = entry.getValue(); + if (collectorData != null) { + if (!collectorData.isStamped()) { + // Stamp the collector if we have not done so + collectorData.setRMIdentifier( + ResourceManager.getClusterTimeStamp()); + collectorData.setVersion( + timelineCollectorVersion.getAndIncrement()); + } + RMApp rmApp = rmApps.get(appId); + if (rmApp == null) { + LOG.warn("Cannot update collector info because application ID: " + + appId + " is not found in RMContext!"); + } else { + synchronized (rmApp) { + AppCollectorData previousCollectorData = rmApp.getCollectorData(); + if (AppCollectorData.happensBefore(previousCollectorData, + collectorData)) { + // Sending collector update event. + // Note: RM has to store the newly received collector data + // synchronously. Otherwise, the RM may send out stale collector + // data before this update is done, and the RM then crashes, the + // newly updated collector data will get lost. + LOG.info("Update collector information for application " + appId + + " with new address: " + collectorData.getCollectorAddr() + + " timestamp: " + collectorData.getRMIdentifier() + + ", " + collectorData.getVersion()); + ((RMAppImpl) rmApp).setCollectorData(collectorData); + } + } + } + } + } + } + } + /** * Check if node in decommissioning state. * @param nodeId diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java index 8b44bd9b170..e0754a0481e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java @@ -34,8 +34,8 @@ import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.SecretManager.InvalidToken; +import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.api.ContainerManagementProtocol; @@ -51,16 +51,19 @@ import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.client.NMProxy; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.ipc.YarnRPC; import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl; import org.apache.hadoop.yarn.util.ConverterUtils; +import org.apache.hadoop.yarn.util.timeline.TimelineUtils; import com.google.common.annotations.VisibleForTesting; @@ -188,6 +191,8 @@ public class AMLauncher implements Runnable { // Finalize the container setupTokens(container, containerID); + // set the flow context optionally for timeline service v.2 + setFlowContext(container); return container; } @@ -232,6 +237,58 @@ public class AMLauncher implements Runnable { container.setTokens(ByteBuffer.wrap(dob.getData(), 0, dob.getLength())); } + private void setFlowContext(ContainerLaunchContext container) { + if (YarnConfiguration.timelineServiceV2Enabled(conf)) { + Map environment = container.getEnvironment(); + ApplicationId applicationId = + application.getAppAttemptId().getApplicationId(); + RMApp app = rmContext.getRMApps().get(applicationId); + + // initialize the flow in the environment with default values for those + // that do not specify the flow tags + // flow name: app name (or app id if app name is missing), + // flow version: "1", flow run id: start time + setFlowTags(environment, TimelineUtils.FLOW_NAME_TAG_PREFIX, + TimelineUtils.generateDefaultFlowName(app.getName(), applicationId)); + setFlowTags(environment, TimelineUtils.FLOW_VERSION_TAG_PREFIX, + TimelineUtils.DEFAULT_FLOW_VERSION); + setFlowTags(environment, TimelineUtils.FLOW_RUN_ID_TAG_PREFIX, + String.valueOf(app.getStartTime())); + + // Set flow context info: the flow context is received via the application + // tags + for (String tag : app.getApplicationTags()) { + String[] parts = tag.split(":", 2); + if (parts.length != 2 || parts[1].isEmpty()) { + continue; + } + switch (parts[0].toUpperCase()) { + case TimelineUtils.FLOW_NAME_TAG_PREFIX: + setFlowTags(environment, TimelineUtils.FLOW_NAME_TAG_PREFIX, + parts[1]); + break; + case TimelineUtils.FLOW_VERSION_TAG_PREFIX: + setFlowTags(environment, TimelineUtils.FLOW_VERSION_TAG_PREFIX, + parts[1]); + break; + case TimelineUtils.FLOW_RUN_ID_TAG_PREFIX: + setFlowTags(environment, TimelineUtils.FLOW_RUN_ID_TAG_PREFIX, + parts[1]); + break; + default: + break; + } + } + } + } + + private static void setFlowTags( + Map environment, String tagPrefix, String value) { + if (!value.isEmpty()) { + environment.put(tagPrefix, value); + } + } + @VisibleForTesting protected Token createAndSetAMRMToken() { Token amrmToken = diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AbstractSystemMetricsPublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AbstractSystemMetricsPublisher.java new file mode 100644 index 00000000000..d4a4fc3ca78 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AbstractSystemMetricsPublisher.java @@ -0,0 +1,178 @@ +/** + * 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.metrics; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.CompositeService; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.AbstractEvent; +import org.apache.hadoop.yarn.event.AsyncDispatcher; +import org.apache.hadoop.yarn.event.Dispatcher; +import org.apache.hadoop.yarn.event.Event; +import org.apache.hadoop.yarn.event.EventHandler; + +/** + * Abstract implementation of SystemMetricsPublisher which is then extended by + * metrics publisher implementations depending on timeline service version. + */ +public abstract class AbstractSystemMetricsPublisher extends CompositeService + implements SystemMetricsPublisher { + private MultiThreadedDispatcher dispatcher; + + protected Dispatcher getDispatcher() { + return dispatcher; + } + + public AbstractSystemMetricsPublisher(String name) { + super(name); + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + dispatcher = + new MultiThreadedDispatcher(getConfig().getInt( + YarnConfiguration. + RM_SYSTEM_METRICS_PUBLISHER_DISPATCHER_POOL_SIZE, + YarnConfiguration. + DEFAULT_RM_SYSTEM_METRICS_PUBLISHER_DISPATCHER_POOL_SIZE)); + dispatcher.setDrainEventsOnStop(); + addIfService(dispatcher); + super.serviceInit(conf); + } + + /** + * Dispatches ATS related events using multiple threads. + */ + @SuppressWarnings({ "rawtypes", "unchecked" }) + public static class MultiThreadedDispatcher extends CompositeService + implements Dispatcher { + + private List dispatchers = + new ArrayList(); + + public MultiThreadedDispatcher(int num) { + super(MultiThreadedDispatcher.class.getName()); + for (int i = 0; i < num; ++i) { + AsyncDispatcher dispatcher = createDispatcher(); + dispatchers.add(dispatcher); + addIfService(dispatcher); + } + } + + @Override + public EventHandler getEventHandler() { + return new CompositEventHandler(); + } + + @Override + public void register(Class eventType, + EventHandler handler) { + for (AsyncDispatcher dispatcher : dispatchers) { + dispatcher.register(eventType, handler); + } + } + + public void setDrainEventsOnStop() { + for (AsyncDispatcher dispatcher : dispatchers) { + dispatcher.setDrainEventsOnStop(); + } + } + + private class CompositEventHandler implements EventHandler { + + @Override + public void handle(Event event) { + // Use hashCode (of ApplicationId) to dispatch the event to the child + // dispatcher, such that all the writing events of one application will + // be handled by one thread, the scheduled order of the these events + // will be preserved + int index = (event.hashCode() & Integer.MAX_VALUE) % dispatchers.size(); + dispatchers.get(index).getEventHandler().handle(event); + } + } + + protected AsyncDispatcher createDispatcher() { + return new AsyncDispatcher(); + } + } + + /** + * EventType which is used while publishing the events. + */ + protected static enum SystemMetricsEventType { + PUBLISH_ENTITY, PUBLISH_APPLICATION_FINISHED_ENTITY + } + + /** + * TimelinePublishEvent's hash code should be based on application's id this + * will ensure all the events related to a particular app goes to particular + * thread of MultiThreaded dispatcher. + */ + protected static abstract class TimelinePublishEvent + extends AbstractEvent { + + private ApplicationId appId; + + public TimelinePublishEvent(SystemMetricsEventType type, + ApplicationId appId) { + super(type); + this.appId = appId; + } + + public ApplicationId getApplicationId() { + return appId; + } + + @Override + public int hashCode() { + return appId.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (!(obj instanceof TimelinePublishEvent)) { + return false; + } + TimelinePublishEvent other = (TimelinePublishEvent) obj; + if (appId == null) { + if (other.appId != null) { + return false; + } + } else if (getType() == null) { + if (other.getType() != null) { + return false; + } + } else { + if (!appId.equals(other.appId) || !getType().equals(other.getType())) { + return false; + } + } + return true; + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AppAttemptFinishedEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AppAttemptFinishedEvent.java deleted file mode 100644 index fc1d10fee0e..00000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AppAttemptFinishedEvent.java +++ /dev/null @@ -1,90 +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.metrics; - -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; -import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState; - -public class AppAttemptFinishedEvent extends - SystemMetricsEvent { - - private ApplicationAttemptId appAttemptId; - private String trackingUrl; - private String originalTrackingUrl; - private String diagnosticsInfo; - private FinalApplicationStatus appStatus; - private YarnApplicationAttemptState state; - private ContainerId masterContainerId; - - public AppAttemptFinishedEvent( - ApplicationAttemptId appAttemptId, - String trackingUrl, - String originalTrackingUrl, - String diagnosticsInfo, - FinalApplicationStatus appStatus, - YarnApplicationAttemptState state, - long finishedTime, - ContainerId masterContainerId) { - super(SystemMetricsEventType.APP_ATTEMPT_FINISHED, finishedTime); - this.appAttemptId = appAttemptId; - // This is the tracking URL after the application attempt is finished - this.trackingUrl = trackingUrl; - this.originalTrackingUrl = originalTrackingUrl; - this.diagnosticsInfo = diagnosticsInfo; - this.appStatus = appStatus; - this.state = state; - this.masterContainerId = masterContainerId; - } - - @Override - public int hashCode() { - return appAttemptId.getApplicationId().hashCode(); - } - - public ApplicationAttemptId getApplicationAttemptId() { - return appAttemptId; - } - - public String getTrackingUrl() { - return trackingUrl; - } - - public String getOriginalTrackingURL() { - return originalTrackingUrl; - } - - public String getDiagnosticsInfo() { - return diagnosticsInfo; - } - - public FinalApplicationStatus getFinalApplicationStatus() { - return appStatus; - } - - public YarnApplicationAttemptState getYarnApplicationAttemptState() { - return state; - } - - public ContainerId getMasterContainerId() { - return masterContainerId; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AppAttemptRegisteredEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AppAttemptRegisteredEvent.java deleted file mode 100644 index 1d0f16de438..00000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AppAttemptRegisteredEvent.java +++ /dev/null @@ -1,81 +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.metrics; - -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ContainerId; - -public class AppAttemptRegisteredEvent extends - SystemMetricsEvent { - - private ApplicationAttemptId appAttemptId; - private String host; - private int rpcPort; - private String trackingUrl; - private String originalTrackingUrl; - private ContainerId masterContainerId; - - public AppAttemptRegisteredEvent( - ApplicationAttemptId appAttemptId, - String host, - int rpcPort, - String trackingUrl, - String originalTrackingUrl, - ContainerId masterContainerId, - long registeredTime) { - super(SystemMetricsEventType.APP_ATTEMPT_REGISTERED, registeredTime); - this.appAttemptId = appAttemptId; - this.host = host; - this.rpcPort = rpcPort; - // This is the tracking URL after the application attempt is registered - this.trackingUrl = trackingUrl; - this.originalTrackingUrl = originalTrackingUrl; - this.masterContainerId = masterContainerId; - } - - @Override - public int hashCode() { - return appAttemptId.getApplicationId().hashCode(); - } - - public ApplicationAttemptId getApplicationAttemptId() { - return appAttemptId; - } - - public String getHost() { - return host; - } - - public int getRpcPort() { - return rpcPort; - } - - public String getTrackingUrl() { - return trackingUrl; - } - - public String getOriginalTrackingURL() { - return originalTrackingUrl; - } - - public ContainerId getMasterContainerId() { - return masterContainerId; - } - -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationCreatedEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationCreatedEvent.java deleted file mode 100644 index 1adf89f5af1..00000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationCreatedEvent.java +++ /dev/null @@ -1,132 +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.metrics; - -import java.util.Set; - -import org.apache.hadoop.ipc.CallerContext; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; -import org.apache.hadoop.yarn.api.records.Priority; - -public class ApplicationCreatedEvent extends - SystemMetricsEvent { - - private ApplicationId appId; - private String name; - private String type; - private String user; - private String queue; - private long submittedTime; - private Set appTags; - private boolean unmanagedApplication; - private Priority applicationPriority; - private String appNodeLabelsExpression; - private String amNodeLabelsExpression; - private final CallerContext callerContext; - private ContainerLaunchContext amContainerSpec; - - - public ApplicationCreatedEvent(ApplicationId appId, - String name, - String type, - String user, - String queue, - long submittedTime, - long createdTime, - Set appTags, - boolean unmanagedApplication, - Priority applicationPriority, - String appNodeLabelsExpression, - String amNodeLabelsExpression, - CallerContext callerContext, - ContainerLaunchContext amContainerSpec) { - super(SystemMetricsEventType.APP_CREATED, createdTime); - this.appId = appId; - this.name = name; - this.type = type; - this.user = user; - this.queue = queue; - this.submittedTime = submittedTime; - this.appTags = appTags; - this.unmanagedApplication = unmanagedApplication; - this.applicationPriority = applicationPriority; - this.appNodeLabelsExpression = appNodeLabelsExpression; - this.amNodeLabelsExpression = amNodeLabelsExpression; - this.callerContext = callerContext; - this.amContainerSpec = amContainerSpec; - } - - @Override - public int hashCode() { - return appId.hashCode(); - } - - public ApplicationId getApplicationId() { - return appId; - } - - public String getApplicationName() { - return name; - } - - public String getApplicationType() { - return type; - } - - public String getUser() { - return user; - } - - public String getQueue() { - return queue; - } - - public long getSubmittedTime() { - return submittedTime; - } - - public Set getAppTags() { - return appTags; - } - - public boolean isUnmanagedApp() { - return unmanagedApplication; - } - - public Priority getApplicationPriority() { - return applicationPriority; - } - - public String getAppNodeLabelsExpression() { - return appNodeLabelsExpression; - } - - public String getAmNodeLabelsExpression() { - return amNodeLabelsExpression; - } - - public CallerContext getCallerContext() { - return callerContext; - } - - public ContainerLaunchContext getAmContainerSpec() { - return amContainerSpec; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationFinishedEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationFinishedEvent.java deleted file mode 100644 index 8d75f92e426..00000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationFinishedEvent.java +++ /dev/null @@ -1,82 +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.metrics; - -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -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.rmapp.RMAppMetrics; - -public class ApplicationFinishedEvent extends - SystemMetricsEvent { - - private ApplicationId appId;; - private String diagnosticsInfo; - private FinalApplicationStatus appStatus; - private YarnApplicationState state; - private ApplicationAttemptId latestAppAttemptId; - private RMAppMetrics appMetrics; - - public ApplicationFinishedEvent( - ApplicationId appId, - String diagnosticsInfo, - FinalApplicationStatus appStatus, - YarnApplicationState state, - ApplicationAttemptId latestAppAttemptId, - long finishedTime, - RMAppMetrics appMetrics) { - super(SystemMetricsEventType.APP_FINISHED, finishedTime); - this.appId = appId; - this.diagnosticsInfo = diagnosticsInfo; - this.appStatus = appStatus; - this.latestAppAttemptId = latestAppAttemptId; - this.state = state; - this.appMetrics=appMetrics; - } - - @Override - public int hashCode() { - return appId.hashCode(); - } - - public ApplicationId getApplicationId() { - return appId; - } - - public String getDiagnosticsInfo() { - return diagnosticsInfo; - } - - public FinalApplicationStatus getFinalApplicationStatus() { - return appStatus; - } - - public YarnApplicationState getYarnApplicationState() { - return state; - } - - public ApplicationAttemptId getLatestApplicationAttemptId() { - return latestAppAttemptId; - } - - public RMAppMetrics getAppMetrics() { - return appMetrics; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ContainerCreatedEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ContainerCreatedEvent.java deleted file mode 100644 index 05b67811138..00000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ContainerCreatedEvent.java +++ /dev/null @@ -1,73 +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.metrics; - -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.api.records.Priority; -import org.apache.hadoop.yarn.api.records.Resource; - -public class ContainerCreatedEvent extends SystemMetricsEvent { - - private ContainerId containerId; - private Resource allocatedResource; - private NodeId allocatedNode; - private Priority allocatedPriority; - private String nodeHttpAddress; - - public ContainerCreatedEvent( - ContainerId containerId, - Resource allocatedResource, - NodeId allocatedNode, - Priority allocatedPriority, - long createdTime, - String nodeHttpAddress) { - super(SystemMetricsEventType.CONTAINER_CREATED, createdTime); - this.containerId = containerId; - this.allocatedResource = allocatedResource; - this.allocatedNode = allocatedNode; - this.allocatedPriority = allocatedPriority; - this.nodeHttpAddress = nodeHttpAddress; - } - - @Override - public int hashCode() { - return containerId.getApplicationAttemptId().getApplicationId().hashCode(); - } - - public ContainerId getContainerId() { - return containerId; - } - - public Resource getAllocatedResource() { - return allocatedResource; - } - - public NodeId getAllocatedNode() { - return allocatedNode; - } - - public Priority getAllocatedPriority() { - return allocatedPriority; - } - - public String getNodeHttpAddress() { - return nodeHttpAddress; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ContainerFinishedEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ContainerFinishedEvent.java deleted file mode 100644 index ca4d3117aa5..00000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ContainerFinishedEvent.java +++ /dev/null @@ -1,72 +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.metrics; - -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.ContainerState; -import org.apache.hadoop.yarn.api.records.NodeId; - -public class ContainerFinishedEvent extends SystemMetricsEvent { - - private ContainerId containerId; - private String diagnosticsInfo; - private int containerExitStatus; - private ContainerState state; - private NodeId allocatedNode; - - public ContainerFinishedEvent( - ContainerId containerId, - String diagnosticsInfo, - int containerExitStatus, - ContainerState state, - long finishedTime, - NodeId allocatedNode) { - super(SystemMetricsEventType.CONTAINER_FINISHED, finishedTime); - this.containerId = containerId; - this.diagnosticsInfo = diagnosticsInfo; - this.containerExitStatus = containerExitStatus; - this.allocatedNode = allocatedNode; - this.state = state; - } - - @Override - public int hashCode() { - return containerId.getApplicationAttemptId().getApplicationId().hashCode(); - } - - public ContainerId getContainerId() { - return containerId; - } - - public String getDiagnosticsInfo() { - return diagnosticsInfo; - } - - public int getContainerExitStatus() { - return containerExitStatus; - } - - public ContainerState getContainerState() { - return state; - } - - public NodeId getAllocatedNode() { - return allocatedNode; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/NoOpSystemMetricPublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/NoOpSystemMetricPublisher.java new file mode 100644 index 00000000000..e0bf3c7c5c5 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/NoOpSystemMetricPublisher.java @@ -0,0 +1,70 @@ +/** + * 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.metrics; + +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState; +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.rmcontainer.RMContainer; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +/** + * This class does nothing when any of the methods are invoked on + * SystemMetricsPublisher. + */ +public class NoOpSystemMetricPublisher implements SystemMetricsPublisher{ + + @Override + public void appCreated(RMApp app, long createdTime) { + } + + @Override + public void appFinished(RMApp app, RMAppState state, long finishedTime) { + } + + @Override + public void appACLsUpdated(RMApp app, String appViewACLs, long updatedTime) { + } + + @Override + public void appAttemptRegistered(RMAppAttempt appAttempt, + long registeredTime) { + } + + @Override + public void appAttemptFinished(RMAppAttempt appAttempt, + RMAppAttemptState appAttemtpState, RMApp app, long finishedTime) { + } + + @Override + public void containerCreated(RMContainer container, long createdTime) { + } + + @Override + public void containerFinished(RMContainer container, long finishedTime) { + } + + @Override + public void appUpdated(RMApp app, long currentTimeMillis) { + } + + @Override + public void appStateUpdated(RMApp app, YarnApplicationState appState, + long updatedTime) { + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsPublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsPublisher.java index 3382509d615..edfb9be6bef 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsPublisher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsPublisher.java @@ -18,625 +18,37 @@ package org.apache.hadoop.yarn.server.resourcemanager.metrics; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.classification.InterfaceAudience.Private; -import org.apache.hadoop.classification.InterfaceStability.Unstable; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.service.CompositeService; -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ApplicationId; -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.YarnApplicationState; -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.TimelinePutResponse; -import org.apache.hadoop.yarn.client.api.TimelineClient; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.event.AsyncDispatcher; -import org.apache.hadoop.yarn.event.Dispatcher; -import org.apache.hadoop.yarn.event.Event; -import org.apache.hadoop.yarn.event.EventHandler; -import org.apache.hadoop.yarn.server.metrics.AppAttemptMetricsConstants; -import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants; -import org.apache.hadoop.yarn.server.metrics.ContainerMetricsConstants; -import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils; 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.resourcemanager.rmapp.attempt.RMAppAttempt; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; -import org.apache.hadoop.yarn.util.timeline.TimelineUtils; - -import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; /** - * The class that helps RM publish metrics to the timeline server. RM will - * always invoke the methods of this class regardless the service is enabled or - * not. If it is disabled, publishing requests will be ignored silently. + * Interface used to publish app/container events to timelineservice. */ -@Private -@Unstable -public class SystemMetricsPublisher extends CompositeService { - private static final Log LOG = LogFactory - .getLog(SystemMetricsPublisher.class); +public interface SystemMetricsPublisher { - private Dispatcher dispatcher; - private TimelineClient client; - private boolean publishSystemMetrics; - public SystemMetricsPublisher() { - super(SystemMetricsPublisher.class.getName()); - } + void appCreated(RMApp app, long createdTime); - @Override - protected void serviceInit(Configuration conf) throws Exception { - publishSystemMetrics = - conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, - YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED) && - conf.getBoolean(YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_ENABLED, - YarnConfiguration.DEFAULT_RM_SYSTEM_METRICS_PUBLISHER_ENABLED); + void appACLsUpdated(RMApp app, String appViewACLs, long updatedTime); - if (publishSystemMetrics) { - client = TimelineClient.createTimelineClient(); - addIfService(client); + void appUpdated(RMApp app, long updatedTime); - dispatcher = createDispatcher(conf); - dispatcher.register(SystemMetricsEventType.class, - new ForwardingEventHandler()); - addIfService(dispatcher); - LOG.info("YARN system metrics publishing service is enabled"); - } else { - LOG.info("YARN system metrics publishing service is not enabled"); - } - super.serviceInit(conf); - } + void appStateUpdated(RMApp app, YarnApplicationState appState, + long updatedTime); - @SuppressWarnings("unchecked") - public void appCreated(RMApp app, long createdTime) { - if (publishSystemMetrics) { - ApplicationSubmissionContext appSubmissionContext = - app.getApplicationSubmissionContext(); - dispatcher.getEventHandler().handle( - new ApplicationCreatedEvent( - app.getApplicationId(), - app.getName(), - app.getApplicationType(), - app.getUser(), - app.getQueue(), - app.getSubmitTime(), - createdTime, app.getApplicationTags(), - appSubmissionContext.getUnmanagedAM(), - appSubmissionContext.getPriority(), - app.getAppNodeLabelExpression(), - app.getAmNodeLabelExpression(), - app.getCallerContext(), - appSubmissionContext.getAMContainerSpec())); - } - } + void appFinished(RMApp app, RMAppState state, long finishedTime); - @SuppressWarnings("unchecked") - public void appUpdated(RMApp app, long updatedTime) { - if (publishSystemMetrics) { - dispatcher.getEventHandler() - .handle(new ApplicationUpdatedEvent(app.getApplicationId(), - app.getQueue(), updatedTime, app.getApplicationPriority())); - } - } + void appAttemptRegistered(RMAppAttempt appAttempt, long registeredTime); - @SuppressWarnings("unchecked") - public void appFinished(RMApp app, RMAppState state, long finishedTime) { - if (publishSystemMetrics) { - dispatcher.getEventHandler().handle( - new ApplicationFinishedEvent( - app.getApplicationId(), - app.getDiagnostics().toString(), - app.getFinalApplicationStatus(), - RMServerUtils.createApplicationState(state), - app.getCurrentAppAttempt() == null ? - null : app.getCurrentAppAttempt().getAppAttemptId(), - finishedTime, - app.getRMAppMetrics())); - } - } + void appAttemptFinished(RMAppAttempt appAttempt, + RMAppAttemptState appAttemtpState, RMApp app, long finishedTime); - @SuppressWarnings("unchecked") - public void appACLsUpdated(RMApp app, String appViewACLs, - long updatedTime) { - if (publishSystemMetrics) { - dispatcher.getEventHandler().handle( - new ApplicationACLsUpdatedEvent( - app.getApplicationId(), - appViewACLs == null ? "" : appViewACLs, - updatedTime)); - } - } - - @SuppressWarnings("unchecked") - public void appStateUpdated(RMApp app, YarnApplicationState appState, - long updatedTime) { - if (publishSystemMetrics) { - dispatcher.getEventHandler().handle( - new ApplicaitonStateUpdatedEvent( - app.getApplicationId(), - appState, - updatedTime)); - } - } - - @SuppressWarnings("unchecked") - public void appAttemptRegistered(RMAppAttempt appAttempt, - long registeredTime) { - if (publishSystemMetrics) { - ContainerId container = (appAttempt.getMasterContainer() == null) ? null - : appAttempt.getMasterContainer().getId(); - dispatcher.getEventHandler().handle( - new AppAttemptRegisteredEvent( - appAttempt.getAppAttemptId(), - appAttempt.getHost(), - appAttempt.getRpcPort(), - appAttempt.getTrackingUrl(), - appAttempt.getOriginalTrackingUrl(), - container, - registeredTime)); - } - } - - @SuppressWarnings("unchecked") - public void appAttemptFinished(RMAppAttempt appAttempt, - RMAppAttemptState appAttemtpState, RMApp app, long finishedTime) { - if (publishSystemMetrics) { - ContainerId container = (appAttempt.getMasterContainer() == null) ? null - : appAttempt.getMasterContainer().getId(); - dispatcher.getEventHandler().handle( - new AppAttemptFinishedEvent( - appAttempt.getAppAttemptId(), - appAttempt.getTrackingUrl(), - appAttempt.getOriginalTrackingUrl(), - appAttempt.getDiagnostics(), - // app will get the final status from app attempt, or create one - // based on app state if it doesn't exist - app.getFinalApplicationStatus(), - RMServerUtils.createApplicationAttemptState(appAttemtpState), - finishedTime, - container)); - } - } - - @SuppressWarnings("unchecked") - public void containerCreated(RMContainer container, long createdTime) { - if (publishSystemMetrics) { - dispatcher.getEventHandler().handle( - new ContainerCreatedEvent( - container.getContainerId(), - container.getAllocatedResource(), - container.getAllocatedNode(), - container.getAllocatedPriority(), - createdTime, container.getNodeHttpAddress())); - } - } - - @SuppressWarnings("unchecked") - public void containerFinished(RMContainer container, long finishedTime) { - if (publishSystemMetrics) { - dispatcher.getEventHandler().handle( - new ContainerFinishedEvent( - container.getContainerId(), - container.getDiagnosticsInfo(), - container.getContainerExitStatus(), - container.getContainerState(), - finishedTime, container.getAllocatedNode())); - } - } - - protected Dispatcher createDispatcher(Configuration conf) { - MultiThreadedDispatcher dispatcher = - new MultiThreadedDispatcher( - conf.getInt( - YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_DISPATCHER_POOL_SIZE, - YarnConfiguration.DEFAULT_RM_SYSTEM_METRICS_PUBLISHER_DISPATCHER_POOL_SIZE)); - dispatcher.setDrainEventsOnStop(); - return dispatcher; - } - - protected void handleSystemMetricsEvent( - SystemMetricsEvent event) { - switch (event.getType()) { - case APP_CREATED: - publishApplicationCreatedEvent((ApplicationCreatedEvent) event); - break; - case APP_FINISHED: - publishApplicationFinishedEvent((ApplicationFinishedEvent) event); - break; - case APP_ACLS_UPDATED: - publishApplicationACLsUpdatedEvent((ApplicationACLsUpdatedEvent) event); - break; - case APP_UPDATED: - publishApplicationUpdatedEvent((ApplicationUpdatedEvent) event); - break; - case APP_STATE_UPDATED: - publishApplicationStateUpdatedEvent( - (ApplicaitonStateUpdatedEvent)event); - break; - case APP_ATTEMPT_REGISTERED: - publishAppAttemptRegisteredEvent((AppAttemptRegisteredEvent) event); - break; - case APP_ATTEMPT_FINISHED: - publishAppAttemptFinishedEvent((AppAttemptFinishedEvent) event); - break; - case CONTAINER_CREATED: - publishContainerCreatedEvent((ContainerCreatedEvent) event); - break; - case CONTAINER_FINISHED: - publishContainerFinishedEvent((ContainerFinishedEvent) event); - break; - default: - LOG.error("Unknown SystemMetricsEvent type: " + event.getType()); - } - } - - private void publishApplicationCreatedEvent(ApplicationCreatedEvent event) { - TimelineEntity entity = - createApplicationEntity(event.getApplicationId()); - Map entityInfo = new HashMap(); - entityInfo.put(ApplicationMetricsConstants.NAME_ENTITY_INFO, - event.getApplicationName()); - entityInfo.put(ApplicationMetricsConstants.TYPE_ENTITY_INFO, - event.getApplicationType()); - entityInfo.put(ApplicationMetricsConstants.USER_ENTITY_INFO, - event.getUser()); - entityInfo.put(ApplicationMetricsConstants.QUEUE_ENTITY_INFO, - event.getQueue()); - entityInfo.put(ApplicationMetricsConstants.SUBMITTED_TIME_ENTITY_INFO, - event.getSubmittedTime()); - entityInfo.put(ApplicationMetricsConstants.APP_TAGS_INFO, - event.getAppTags()); - entityInfo.put( - ApplicationMetricsConstants.UNMANAGED_APPLICATION_ENTITY_INFO, - event.isUnmanagedApp()); - entityInfo.put(ApplicationMetricsConstants.APPLICATION_PRIORITY_INFO, - event.getApplicationPriority().getPriority()); - entityInfo.put(ApplicationMetricsConstants.APP_NODE_LABEL_EXPRESSION, - event.getAppNodeLabelsExpression()); - entityInfo.put(ApplicationMetricsConstants.AM_NODE_LABEL_EXPRESSION, - event.getAmNodeLabelsExpression()); - if (event.getCallerContext() != null) { - if (event.getCallerContext().getContext() != null) { - entityInfo.put(ApplicationMetricsConstants.YARN_APP_CALLER_CONTEXT, - event.getCallerContext().getContext()); - } - if (event.getCallerContext().getSignature() != null) { - entityInfo.put(ApplicationMetricsConstants.YARN_APP_CALLER_SIGNATURE, - event.getCallerContext().getSignature()); - } - } - - ContainerLaunchContext amContainerSpec = event.getAmContainerSpec(); - entityInfo.put(ApplicationMetricsConstants.AM_CONTAINER_LAUNCH_COMMAND, - amContainerSpec.getCommands()); - - entity.setOtherInfo(entityInfo); - TimelineEvent tEvent = new TimelineEvent(); - tEvent.setEventType( - ApplicationMetricsConstants.CREATED_EVENT_TYPE); - tEvent.setTimestamp(event.getTimestamp()); - entity.addEvent(tEvent); - putEntity(entity); - } - - private void publishApplicationFinishedEvent(ApplicationFinishedEvent event) { - TimelineEntity entity = - createApplicationEntity(event.getApplicationId()); - TimelineEvent tEvent = new TimelineEvent(); - tEvent.setEventType( - ApplicationMetricsConstants.FINISHED_EVENT_TYPE); - tEvent.setTimestamp(event.getTimestamp()); - Map eventInfo = new HashMap(); - eventInfo.put(ApplicationMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO, - event.getDiagnosticsInfo()); - eventInfo.put(ApplicationMetricsConstants.FINAL_STATUS_EVENT_INFO, - event.getFinalApplicationStatus().toString()); - eventInfo.put(ApplicationMetricsConstants.STATE_EVENT_INFO, - event.getYarnApplicationState().toString()); - if (event.getLatestApplicationAttemptId() != null) { - eventInfo.put(ApplicationMetricsConstants.LATEST_APP_ATTEMPT_EVENT_INFO, - event.getLatestApplicationAttemptId().toString()); - } - RMAppMetrics appMetrics = event.getAppMetrics(); - entity.addOtherInfo(ApplicationMetricsConstants.APP_CPU_METRICS, - appMetrics.getVcoreSeconds()); - entity.addOtherInfo(ApplicationMetricsConstants.APP_MEM_METRICS, - appMetrics.getMemorySeconds()); - entity.addOtherInfo(ApplicationMetricsConstants.APP_MEM_PREEMPT_METRICS, - appMetrics.getPreemptedMemorySeconds()); - entity.addOtherInfo(ApplicationMetricsConstants.APP_CPU_PREEMPT_METRICS, - appMetrics.getPreemptedVcoreSeconds()); - - tEvent.setEventInfo(eventInfo); - entity.addEvent(tEvent); - putEntity(entity); - } - - private void publishApplicationUpdatedEvent(ApplicationUpdatedEvent event) { - TimelineEntity entity = createApplicationEntity(event.getApplicationId()); - Map eventInfo = new HashMap(); - eventInfo.put(ApplicationMetricsConstants.QUEUE_ENTITY_INFO, - event.getQueue()); - eventInfo.put(ApplicationMetricsConstants.APPLICATION_PRIORITY_INFO, event - .getApplicationPriority().getPriority()); - TimelineEvent tEvent = new TimelineEvent(); - tEvent.setEventType(ApplicationMetricsConstants.UPDATED_EVENT_TYPE); - tEvent.setTimestamp(event.getTimestamp()); - tEvent.setEventInfo(eventInfo); - entity.addEvent(tEvent); - putEntity(entity); - } - - private void publishApplicationStateUpdatedEvent( - ApplicaitonStateUpdatedEvent event) { - TimelineEntity entity = createApplicationEntity(event.getApplicationId()); - Map eventInfo = new HashMap(); - eventInfo.put(ApplicationMetricsConstants.STATE_EVENT_INFO, - event.getAppState()); - TimelineEvent tEvent = new TimelineEvent(); - tEvent.setEventType(ApplicationMetricsConstants.STATE_UPDATED_EVENT_TYPE); - tEvent.setTimestamp(event.getTimestamp()); - tEvent.setEventInfo(eventInfo); - entity.addEvent(tEvent); - putEntity(entity); - } - - private void publishApplicationACLsUpdatedEvent( - ApplicationACLsUpdatedEvent event) { - TimelineEntity entity = - createApplicationEntity(event.getApplicationId()); - TimelineEvent tEvent = new TimelineEvent(); - Map entityInfo = new HashMap(); - entityInfo.put(ApplicationMetricsConstants.APP_VIEW_ACLS_ENTITY_INFO, - event.getViewAppACLs()); - entity.setOtherInfo(entityInfo); - tEvent.setEventType( - ApplicationMetricsConstants.ACLS_UPDATED_EVENT_TYPE); - tEvent.setTimestamp(event.getTimestamp()); - entity.addEvent(tEvent); - putEntity(entity); - } - - private static TimelineEntity createApplicationEntity( - ApplicationId applicationId) { - TimelineEntity entity = new TimelineEntity(); - entity.setEntityType(ApplicationMetricsConstants.ENTITY_TYPE); - entity.setEntityId(applicationId.toString()); - return entity; - } - - private void - publishAppAttemptRegisteredEvent(AppAttemptRegisteredEvent event) { - TimelineEntity entity = - createAppAttemptEntity(event.getApplicationAttemptId()); - TimelineEvent tEvent = new TimelineEvent(); - tEvent.setEventType( - AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE); - tEvent.setTimestamp(event.getTimestamp()); - Map eventInfo = new HashMap(); - eventInfo.put( - AppAttemptMetricsConstants.TRACKING_URL_EVENT_INFO, - event.getTrackingUrl()); - eventInfo.put( - AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_EVENT_INFO, - event.getOriginalTrackingURL()); - eventInfo.put(AppAttemptMetricsConstants.HOST_EVENT_INFO, - event.getHost()); - eventInfo.put(AppAttemptMetricsConstants.RPC_PORT_EVENT_INFO, - event.getRpcPort()); - if (event.getMasterContainerId() != null) { - eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO, - event.getMasterContainerId().toString()); - } - tEvent.setEventInfo(eventInfo); - entity.addEvent(tEvent); - putEntity(entity); - } - - private void publishAppAttemptFinishedEvent(AppAttemptFinishedEvent event) { - TimelineEntity entity = - createAppAttemptEntity(event.getApplicationAttemptId()); - TimelineEvent tEvent = new TimelineEvent(); - tEvent.setEventType(AppAttemptMetricsConstants.FINISHED_EVENT_TYPE); - tEvent.setTimestamp(event.getTimestamp()); - Map eventInfo = new HashMap(); - eventInfo.put( - AppAttemptMetricsConstants.TRACKING_URL_EVENT_INFO, - event.getTrackingUrl()); - eventInfo.put( - AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_EVENT_INFO, - event.getOriginalTrackingURL()); - eventInfo.put(AppAttemptMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO, - event.getDiagnosticsInfo()); - eventInfo.put(AppAttemptMetricsConstants.FINAL_STATUS_EVENT_INFO, - event.getFinalApplicationStatus().toString()); - eventInfo.put(AppAttemptMetricsConstants.STATE_EVENT_INFO, - event.getYarnApplicationAttemptState().toString()); - if (event.getMasterContainerId() != null) { - eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO, - event.getMasterContainerId().toString()); - } - tEvent.setEventInfo(eventInfo); - entity.addEvent(tEvent); - putEntity(entity); - } - - private static TimelineEntity createAppAttemptEntity( - ApplicationAttemptId appAttemptId) { - TimelineEntity entity = new TimelineEntity(); - entity.setEntityType( - AppAttemptMetricsConstants.ENTITY_TYPE); - entity.setEntityId(appAttemptId.toString()); - entity.addPrimaryFilter(AppAttemptMetricsConstants.PARENT_PRIMARY_FILTER, - appAttemptId.getApplicationId().toString()); - return entity; - } - - private void publishContainerCreatedEvent(ContainerCreatedEvent event) { - TimelineEntity entity = createContainerEntity(event.getContainerId()); - Map entityInfo = new HashMap(); - entityInfo.put(ContainerMetricsConstants.ALLOCATED_MEMORY_ENTITY_INFO, - event.getAllocatedResource().getMemorySize()); - entityInfo.put(ContainerMetricsConstants.ALLOCATED_VCORE_ENTITY_INFO, - event.getAllocatedResource().getVirtualCores()); - entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO, - event.getAllocatedNode().getHost()); - entityInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO, - event.getAllocatedNode().getPort()); - entityInfo.put(ContainerMetricsConstants.ALLOCATED_PRIORITY_ENTITY_INFO, - event.getAllocatedPriority().getPriority()); - entityInfo.put( - ContainerMetricsConstants.ALLOCATED_HOST_HTTP_ADDRESS_ENTITY_INFO, - event.getNodeHttpAddress()); - entity.setOtherInfo(entityInfo); - TimelineEvent tEvent = new TimelineEvent(); - tEvent.setEventType(ContainerMetricsConstants.CREATED_EVENT_TYPE); - tEvent.setTimestamp(event.getTimestamp()); - entity.addEvent(tEvent); - putEntity(entity); - } - - private void publishContainerFinishedEvent(ContainerFinishedEvent event) { - TimelineEntity entity = createContainerEntity(event.getContainerId()); - TimelineEvent tEvent = new TimelineEvent(); - tEvent.setEventType(ContainerMetricsConstants.FINISHED_EVENT_TYPE); - tEvent.setTimestamp(event.getTimestamp()); - Map eventInfo = new HashMap(); - eventInfo.put(ContainerMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO, - event.getDiagnosticsInfo()); - eventInfo.put(ContainerMetricsConstants.EXIT_STATUS_EVENT_INFO, - event.getContainerExitStatus()); - eventInfo.put(ContainerMetricsConstants.STATE_EVENT_INFO, - event.getContainerState().toString()); - Map entityInfo = new HashMap(); - entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO, - event.getAllocatedNode().getHost()); - entityInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO, - event.getAllocatedNode().getPort()); - entity.setOtherInfo(entityInfo); - tEvent.setEventInfo(eventInfo); - entity.addEvent(tEvent); - putEntity(entity); - } - - private static TimelineEntity createContainerEntity( - ContainerId containerId) { - TimelineEntity entity = new TimelineEntity(); - entity.setEntityType( - ContainerMetricsConstants.ENTITY_TYPE); - entity.setEntityId(containerId.toString()); - entity.addPrimaryFilter(ContainerMetricsConstants.PARENT_PRIMARIY_FILTER, - containerId.getApplicationAttemptId().toString()); - return entity; - } - - @Private - @VisibleForTesting - public void putEntity(TimelineEntity entity) { - try { - if (LOG.isDebugEnabled()) { - LOG.debug("Publishing the entity " + entity.getEntityId() + - ", JSON-style content: " + TimelineUtils.dumpTimelineRecordtoJSON(entity)); - } - TimelinePutResponse response = client.putEntities(entity); - List errors = response.getErrors(); - if (errors.size() == 0) { - LOG.debug("Timeline entities are successfully put"); - } else { - for (TimelinePutResponse.TimelinePutError error : errors) { - LOG.error( - "Error when publishing entity [" + error.getEntityType() + "," - + error.getEntityId() + "], server side error code: " - + error.getErrorCode()); - } - } - } catch (Exception e) { - LOG.error("Error when publishing entity [" + entity.getEntityType() + "," - + entity.getEntityId() + "]", e); - } - } - - /** - * EventHandler implementation which forward events to SystemMetricsPublisher. - * Making use of it, SystemMetricsPublisher can avoid to have a public handle - * method. - */ - private final class ForwardingEventHandler implements - EventHandler { - - @Override - public void handle(SystemMetricsEvent event) { - handleSystemMetricsEvent(event); - } - - } - - @SuppressWarnings({ "rawtypes", "unchecked" }) - protected static class MultiThreadedDispatcher extends CompositeService - implements Dispatcher { - - private List dispatchers = - new ArrayList(); - - public MultiThreadedDispatcher(int num) { - super(MultiThreadedDispatcher.class.getName()); - for (int i = 0; i < num; ++i) { - AsyncDispatcher dispatcher = createDispatcher(); - dispatchers.add(dispatcher); - addIfService(dispatcher); - } - } - - @Override - public EventHandler getEventHandler() { - return new CompositEventHandler(); - } - - @Override - public void register(Class eventType, EventHandler handler) { - for (AsyncDispatcher dispatcher : dispatchers) { - dispatcher.register(eventType, handler); - } - } - - public void setDrainEventsOnStop() { - for (AsyncDispatcher dispatcher : dispatchers) { - dispatcher.setDrainEventsOnStop(); - } - } - - private class CompositEventHandler implements EventHandler { - - @Override - public void handle(Event event) { - // Use hashCode (of ApplicationId) to dispatch the event to the child - // dispatcher, such that all the writing events of one application will - // be handled by one thread, the scheduled order of the these events - // will be preserved - int index = (event.hashCode() & Integer.MAX_VALUE) % dispatchers.size(); - dispatchers.get(index).getEventHandler().handle(event); - } - - } - - protected AsyncDispatcher createDispatcher() { - return new AsyncDispatcher("RM Timeline dispatcher"); - } - - } + void containerCreated(RMContainer container, long createdTime); + void containerFinished(RMContainer container, long finishedTime); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV1Publisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV1Publisher.java new file mode 100644 index 00000000000..4c371a7f86d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV1Publisher.java @@ -0,0 +1,396 @@ +/** + * 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.metrics; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.api.records.ContainerId; +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.TimelineEvent; +import org.apache.hadoop.yarn.client.api.TimelineClient; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.metrics.AppAttemptMetricsConstants; +import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants; +import org.apache.hadoop.yarn.server.metrics.ContainerMetricsConstants; +import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils; +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.resourcemanager.rmapp.attempt.RMAppAttempt; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; +import org.apache.hadoop.yarn.util.timeline.TimelineUtils; + +/** + * This class is responsible for posting application, appattempt & Container + * lifecycle related events to timeline service v1. + */ +public class TimelineServiceV1Publisher extends AbstractSystemMetricsPublisher { + + private static final Log LOG = + LogFactory.getLog(TimelineServiceV1Publisher.class); + + public TimelineServiceV1Publisher() { + super("TimelineserviceV1Publisher"); + } + + private TimelineClient client; + + @Override + protected void serviceInit(Configuration conf) throws Exception { + client = TimelineClient.createTimelineClient(); + addIfService(client); + super.serviceInit(conf); + getDispatcher().register(SystemMetricsEventType.class, + new TimelineV1EventHandler()); + } + + @SuppressWarnings("unchecked") + @Override + public void appCreated(RMApp app, long createdTime) { + TimelineEntity entity = createApplicationEntity(app.getApplicationId()); + Map entityInfo = new HashMap(); + entityInfo.put(ApplicationMetricsConstants.NAME_ENTITY_INFO, app.getName()); + entityInfo.put(ApplicationMetricsConstants.TYPE_ENTITY_INFO, + app.getApplicationType()); + entityInfo.put(ApplicationMetricsConstants.USER_ENTITY_INFO, app.getUser()); + entityInfo.put(ApplicationMetricsConstants.QUEUE_ENTITY_INFO, + app.getQueue()); + entityInfo.put(ApplicationMetricsConstants.SUBMITTED_TIME_ENTITY_INFO, + app.getSubmitTime()); + entityInfo.put(ApplicationMetricsConstants.APP_TAGS_INFO, + app.getApplicationTags()); + entityInfo.put( + ApplicationMetricsConstants.UNMANAGED_APPLICATION_ENTITY_INFO, + app.getApplicationSubmissionContext().getUnmanagedAM()); + entityInfo.put(ApplicationMetricsConstants.APPLICATION_PRIORITY_INFO, + app.getApplicationPriority().getPriority()); + entityInfo.put(ApplicationMetricsConstants.AM_NODE_LABEL_EXPRESSION, + app.getAmNodeLabelExpression()); + entityInfo.put(ApplicationMetricsConstants.APP_NODE_LABEL_EXPRESSION, + app.getAppNodeLabelExpression()); + if (app.getCallerContext() != null) { + if (app.getCallerContext().getContext() != null) { + entityInfo.put(ApplicationMetricsConstants.YARN_APP_CALLER_CONTEXT, + app.getCallerContext().getContext()); + } + if (app.getCallerContext().getSignature() != null) { + entityInfo.put(ApplicationMetricsConstants.YARN_APP_CALLER_SIGNATURE, + app.getCallerContext().getSignature()); + } + } + + ContainerLaunchContext amContainerSpec = + app.getApplicationSubmissionContext().getAMContainerSpec(); + entityInfo.put(ApplicationMetricsConstants.AM_CONTAINER_LAUNCH_COMMAND, + amContainerSpec.getCommands()); + + entity.setOtherInfo(entityInfo); + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setEventType(ApplicationMetricsConstants.CREATED_EVENT_TYPE); + tEvent.setTimestamp(createdTime); + + entity.addEvent(tEvent); + getDispatcher().getEventHandler().handle(new TimelineV1PublishEvent( + SystemMetricsEventType.PUBLISH_ENTITY, entity, app.getApplicationId())); + } + + @Override + public void appFinished(RMApp app, RMAppState state, long finishedTime) { + TimelineEntity entity = createApplicationEntity(app.getApplicationId()); + + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setEventType(ApplicationMetricsConstants.FINISHED_EVENT_TYPE); + tEvent.setTimestamp(finishedTime); + Map eventInfo = new HashMap(); + eventInfo.put(ApplicationMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO, + app.getDiagnostics().toString()); + eventInfo.put(ApplicationMetricsConstants.FINAL_STATUS_EVENT_INFO, + app.getFinalApplicationStatus().toString()); + eventInfo.put(ApplicationMetricsConstants.STATE_EVENT_INFO, + RMServerUtils.createApplicationState(state).toString()); + String latestApplicationAttemptId = app.getCurrentAppAttempt() == null + ? null : app.getCurrentAppAttempt().getAppAttemptId().toString(); + if (latestApplicationAttemptId != null) { + eventInfo.put(ApplicationMetricsConstants.LATEST_APP_ATTEMPT_EVENT_INFO, + latestApplicationAttemptId); + } + RMAppMetrics appMetrics = app.getRMAppMetrics(); + entity.addOtherInfo(ApplicationMetricsConstants.APP_CPU_METRICS, + appMetrics.getVcoreSeconds()); + entity.addOtherInfo(ApplicationMetricsConstants.APP_MEM_METRICS, + appMetrics.getMemorySeconds()); + entity.addOtherInfo(ApplicationMetricsConstants.APP_MEM_PREEMPT_METRICS, + appMetrics.getPreemptedMemorySeconds()); + entity.addOtherInfo(ApplicationMetricsConstants.APP_CPU_PREEMPT_METRICS, + appMetrics.getPreemptedVcoreSeconds()); + tEvent.setEventInfo(eventInfo); + + entity.addEvent(tEvent); + // sync sending of finish event to avoid possibility of saving application + // finished state in RMStateStore save without publishing in ATS. + putEntity(entity); // sync event so that ATS update is done without fail. + } + + @SuppressWarnings("unchecked") + @Override + public void appUpdated(RMApp app, long updatedTime) { + TimelineEntity entity = createApplicationEntity(app.getApplicationId()); + Map eventInfo = new HashMap(); + eventInfo.put(ApplicationMetricsConstants.QUEUE_ENTITY_INFO, + app.getQueue()); + eventInfo.put(ApplicationMetricsConstants.APPLICATION_PRIORITY_INFO, + app.getApplicationSubmissionContext().getPriority().getPriority()); + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setEventType(ApplicationMetricsConstants.UPDATED_EVENT_TYPE); + tEvent.setTimestamp(updatedTime); + tEvent.setEventInfo(eventInfo); + entity.addEvent(tEvent); + getDispatcher().getEventHandler().handle(new TimelineV1PublishEvent( + SystemMetricsEventType.PUBLISH_ENTITY, entity, app.getApplicationId())); + } + + @SuppressWarnings("unchecked") + @Override + public void appStateUpdated(RMApp app, YarnApplicationState appState, + long updatedTime) { + TimelineEntity entity = createApplicationEntity(app.getApplicationId()); + Map eventInfo = new HashMap(); + eventInfo.put(ApplicationMetricsConstants.STATE_EVENT_INFO, + appState); + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setEventType(ApplicationMetricsConstants.STATE_UPDATED_EVENT_TYPE); + tEvent.setTimestamp(updatedTime); + tEvent.setEventInfo(eventInfo); + entity.addEvent(tEvent); + getDispatcher().getEventHandler().handle(new TimelineV1PublishEvent( + SystemMetricsEventType.PUBLISH_ENTITY, entity, app.getApplicationId())); + } + + @SuppressWarnings("unchecked") + @Override + public void appACLsUpdated(RMApp app, String appViewACLs, long updatedTime) { + TimelineEntity entity = createApplicationEntity(app.getApplicationId()); + TimelineEvent tEvent = new TimelineEvent(); + Map entityInfo = new HashMap(); + entityInfo.put(ApplicationMetricsConstants.APP_VIEW_ACLS_ENTITY_INFO, + (appViewACLs == null) ? "" : appViewACLs); + entity.setOtherInfo(entityInfo); + tEvent.setEventType(ApplicationMetricsConstants.ACLS_UPDATED_EVENT_TYPE); + tEvent.setTimestamp(updatedTime); + + entity.addEvent(tEvent); + getDispatcher().getEventHandler().handle(new TimelineV1PublishEvent( + SystemMetricsEventType.PUBLISH_ENTITY, entity, app.getApplicationId())); + } + + @SuppressWarnings("unchecked") + @Override + public void appAttemptRegistered(RMAppAttempt appAttempt, + long registeredTime) { + TimelineEntity entity = + createAppAttemptEntity(appAttempt.getAppAttemptId()); + + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setEventType(AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE); + tEvent.setTimestamp(registeredTime); + Map eventInfo = new HashMap(); + eventInfo.put(AppAttemptMetricsConstants.TRACKING_URL_INFO, + appAttempt.getTrackingUrl()); + eventInfo.put(AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_INFO, + appAttempt.getOriginalTrackingUrl()); + eventInfo.put(AppAttemptMetricsConstants.HOST_INFO, + appAttempt.getHost()); + eventInfo.put(AppAttemptMetricsConstants.RPC_PORT_INFO, + appAttempt.getRpcPort()); + if (appAttempt.getMasterContainer() != null) { + eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_INFO, + appAttempt.getMasterContainer().getId().toString()); + } + tEvent.setEventInfo(eventInfo); + entity.addEvent(tEvent); + getDispatcher().getEventHandler().handle( + new TimelineV1PublishEvent(SystemMetricsEventType.PUBLISH_ENTITY, + entity, appAttempt.getAppAttemptId().getApplicationId())); + + } + + @SuppressWarnings("unchecked") + @Override + public void appAttemptFinished(RMAppAttempt appAttempt, + RMAppAttemptState appAttemtpState, RMApp app, long finishedTime) { + TimelineEntity entity = + createAppAttemptEntity(appAttempt.getAppAttemptId()); + + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setEventType(AppAttemptMetricsConstants.FINISHED_EVENT_TYPE); + tEvent.setTimestamp(finishedTime); + Map eventInfo = new HashMap(); + eventInfo.put(AppAttemptMetricsConstants.TRACKING_URL_INFO, + appAttempt.getTrackingUrl()); + eventInfo.put(AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_INFO, + appAttempt.getOriginalTrackingUrl()); + eventInfo.put(AppAttemptMetricsConstants.DIAGNOSTICS_INFO, + appAttempt.getDiagnostics()); + eventInfo.put(AppAttemptMetricsConstants.FINAL_STATUS_INFO, + app.getFinalApplicationStatus().toString()); + eventInfo.put(AppAttemptMetricsConstants.STATE_INFO, RMServerUtils + .createApplicationAttemptState(appAttemtpState).toString()); + if (appAttempt.getMasterContainer() != null) { + eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_INFO, + appAttempt.getMasterContainer().getId().toString()); + } + tEvent.setEventInfo(eventInfo); + + entity.addEvent(tEvent); + getDispatcher().getEventHandler().handle( + new TimelineV1PublishEvent(SystemMetricsEventType.PUBLISH_ENTITY, + entity, appAttempt.getAppAttemptId().getApplicationId())); + } + + @SuppressWarnings("unchecked") + @Override + public void containerCreated(RMContainer container, long createdTime) { + TimelineEntity entity = createContainerEntity(container.getContainerId()); + Map entityInfo = new HashMap(); + entityInfo.put(ContainerMetricsConstants.ALLOCATED_MEMORY_INFO, + container.getAllocatedResource().getMemorySize()); + entityInfo.put(ContainerMetricsConstants.ALLOCATED_VCORE_INFO, + container.getAllocatedResource().getVirtualCores()); + entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_INFO, + container.getAllocatedNode().getHost()); + entityInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_INFO, + container.getAllocatedNode().getPort()); + entityInfo.put(ContainerMetricsConstants.ALLOCATED_PRIORITY_INFO, + container.getAllocatedPriority().getPriority()); + entityInfo.put( + ContainerMetricsConstants.ALLOCATED_HOST_HTTP_ADDRESS_INFO, + container.getNodeHttpAddress()); + entity.setOtherInfo(entityInfo); + + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setEventType(ContainerMetricsConstants.CREATED_EVENT_TYPE); + tEvent.setTimestamp(createdTime); + + entity.addEvent(tEvent); + getDispatcher().getEventHandler().handle(new TimelineV1PublishEvent( + SystemMetricsEventType.PUBLISH_ENTITY, entity, container + .getContainerId().getApplicationAttemptId().getApplicationId())); + } + + @SuppressWarnings("unchecked") + @Override + public void containerFinished(RMContainer container, long finishedTime) { + TimelineEntity entity = createContainerEntity(container.getContainerId()); + + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setEventType(ContainerMetricsConstants.FINISHED_EVENT_TYPE); + tEvent.setTimestamp(finishedTime); + Map eventInfo = new HashMap(); + eventInfo.put(ContainerMetricsConstants.DIAGNOSTICS_INFO, + container.getDiagnosticsInfo()); + eventInfo.put(ContainerMetricsConstants.EXIT_STATUS_INFO, + container.getContainerExitStatus()); + eventInfo.put(ContainerMetricsConstants.STATE_INFO, + container.getContainerState().toString()); + Map entityInfo = new HashMap(); + entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_INFO, + container.getAllocatedNode().getHost()); + entityInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_INFO, + container.getAllocatedNode().getPort()); + entity.setOtherInfo(entityInfo); + tEvent.setEventInfo(eventInfo); + + entity.addEvent(tEvent); + getDispatcher().getEventHandler().handle(new TimelineV1PublishEvent( + SystemMetricsEventType.PUBLISH_ENTITY, entity, container + .getContainerId().getApplicationAttemptId().getApplicationId())); + } + + private static TimelineEntity createApplicationEntity( + ApplicationId applicationId) { + TimelineEntity entity = new TimelineEntity(); + entity.setEntityType(ApplicationMetricsConstants.ENTITY_TYPE); + entity.setEntityId(applicationId.toString()); + return entity; + } + + private static TimelineEntity createAppAttemptEntity( + ApplicationAttemptId appAttemptId) { + TimelineEntity entity = new TimelineEntity(); + entity.setEntityType(AppAttemptMetricsConstants.ENTITY_TYPE); + entity.setEntityId(appAttemptId.toString()); + entity.addPrimaryFilter(AppAttemptMetricsConstants.PARENT_PRIMARY_FILTER, + appAttemptId.getApplicationId().toString()); + return entity; + } + + private static TimelineEntity createContainerEntity(ContainerId containerId) { + TimelineEntity entity = new TimelineEntity(); + entity.setEntityType(ContainerMetricsConstants.ENTITY_TYPE); + entity.setEntityId(containerId.toString()); + entity.addPrimaryFilter(ContainerMetricsConstants.PARENT_PRIMARIY_FILTER, + containerId.getApplicationAttemptId().toString()); + return entity; + } + + private void putEntity(TimelineEntity entity) { + 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 TimelineV1PublishEvent extends TimelinePublishEvent { + private TimelineEntity entity; + + public TimelineV1PublishEvent(SystemMetricsEventType type, + TimelineEntity entity, ApplicationId appId) { + super(type, appId); + this.entity = entity; + } + + public TimelineEntity getEntity() { + return entity; + } + } + + private class TimelineV1EventHandler + implements EventHandler { + @Override + public void handle(TimelineV1PublishEvent event) { + putEntity(event.getEntity()); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java new file mode 100644 index 00000000000..f3b495b4763 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java @@ -0,0 +1,503 @@ +/** + * 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.metrics; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; +import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationAttemptEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.ContainerEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity.Identifier; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.metrics.AppAttemptMetricsConstants; +import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants; +import org.apache.hadoop.yarn.server.metrics.ContainerMetricsConstants; +import org.apache.hadoop.yarn.server.resourcemanager.RMContext; +import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState; +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.rmcontainer.RMContainer; +import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager; +import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollector; +import org.apache.hadoop.yarn.util.TimelineServiceHelper; +import org.apache.hadoop.yarn.util.timeline.TimelineUtils; + +import com.google.common.annotations.VisibleForTesting; + +/** + * This class is responsible for posting application, appattempt & Container + * lifecycle related events to timeline service v2. + */ +@Private +@Unstable +public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher { + private static final Log LOG = + LogFactory.getLog(TimelineServiceV2Publisher.class); + private RMTimelineCollectorManager rmTimelineCollectorManager; + private boolean publishContainerEvents; + + public TimelineServiceV2Publisher(RMContext rmContext) { + super("TimelineserviceV2Publisher"); + rmTimelineCollectorManager = rmContext.getRMTimelineCollectorManager(); + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + super.serviceInit(conf); + getDispatcher().register(SystemMetricsEventType.class, + new TimelineV2EventHandler()); + publishContainerEvents = getConfig().getBoolean( + YarnConfiguration.RM_PUBLISH_CONTAINER_EVENTS_ENABLED, + YarnConfiguration.DEFAULT_RM_PUBLISH_CONTAINER_EVENTS_ENABLED); + } + + @VisibleForTesting + boolean isPublishContainerEvents() { + return publishContainerEvents; + } + + @SuppressWarnings("unchecked") + @Override + public void appCreated(RMApp app, long createdTime) { + ApplicationEntity entity = createApplicationEntity(app.getApplicationId()); + entity.setQueue(app.getQueue()); + entity.setCreatedTime(createdTime); + + Map entityInfo = new HashMap(); + entityInfo.put(ApplicationMetricsConstants.NAME_ENTITY_INFO, app.getName()); + entityInfo.put(ApplicationMetricsConstants.TYPE_ENTITY_INFO, + app.getApplicationType()); + entityInfo.put(ApplicationMetricsConstants.USER_ENTITY_INFO, app.getUser()); + entityInfo.put(ApplicationMetricsConstants.QUEUE_ENTITY_INFO, + app.getQueue()); + entityInfo.put(ApplicationMetricsConstants.SUBMITTED_TIME_ENTITY_INFO, + app.getSubmitTime()); + entityInfo.put(ApplicationMetricsConstants.APP_TAGS_INFO, + app.getApplicationTags()); + entityInfo.put( + ApplicationMetricsConstants.UNMANAGED_APPLICATION_ENTITY_INFO, + app.getApplicationSubmissionContext().getUnmanagedAM()); + entityInfo.put(ApplicationMetricsConstants.APPLICATION_PRIORITY_INFO, + app.getApplicationPriority().getPriority()); + entity.getConfigs().put( + ApplicationMetricsConstants.AM_NODE_LABEL_EXPRESSION, + app.getAmNodeLabelExpression()); + entity.getConfigs().put( + ApplicationMetricsConstants.APP_NODE_LABEL_EXPRESSION, + app.getAppNodeLabelExpression()); + if (app.getCallerContext() != null) { + if (app.getCallerContext().getContext() != null) { + entityInfo.put(ApplicationMetricsConstants.YARN_APP_CALLER_CONTEXT, + app.getCallerContext().getContext()); + } + if (app.getCallerContext().getSignature() != null) { + entityInfo.put(ApplicationMetricsConstants.YARN_APP_CALLER_SIGNATURE, + app.getCallerContext().getSignature()); + } + } + ContainerLaunchContext amContainerSpec = + app.getApplicationSubmissionContext().getAMContainerSpec(); + entityInfo.put(ApplicationMetricsConstants.AM_CONTAINER_LAUNCH_COMMAND, + amContainerSpec.getCommands()); + + entity.setInfo(entityInfo); + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE); + tEvent.setTimestamp(createdTime); + entity.addEvent(tEvent); + + getDispatcher().getEventHandler().handle(new TimelineV2PublishEvent( + SystemMetricsEventType.PUBLISH_ENTITY, entity, app.getApplicationId())); + } + + @SuppressWarnings("unchecked") + @Override + public void appFinished(RMApp app, RMAppState state, long finishedTime) { + ApplicationEntity entity = createApplicationEntity(app.getApplicationId()); + + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE); + tEvent.setTimestamp(finishedTime); + entity.addEvent(tEvent); + + Map entityInfo = new HashMap(); + entityInfo.put(ApplicationMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO, + app.getDiagnostics().toString()); + entityInfo.put(ApplicationMetricsConstants.FINAL_STATUS_EVENT_INFO, + app.getFinalApplicationStatus().toString()); + entityInfo.put(ApplicationMetricsConstants.STATE_EVENT_INFO, + RMServerUtils.createApplicationState(state).toString()); + ApplicationAttemptId appAttemptId = app.getCurrentAppAttempt() == null + ? null : app.getCurrentAppAttempt().getAppAttemptId(); + if (appAttemptId != null) { + entityInfo.put(ApplicationMetricsConstants.LATEST_APP_ATTEMPT_EVENT_INFO, + appAttemptId.toString()); + } + entity.setInfo(entityInfo); + + RMAppMetrics appMetrics = app.getRMAppMetrics(); + Set entityMetrics = + getTimelinelineAppMetrics(appMetrics, finishedTime); + entity.setMetrics(entityMetrics); + + getDispatcher().getEventHandler().handle( + new ApplicationFinishPublishEvent(SystemMetricsEventType. + PUBLISH_APPLICATION_FINISHED_ENTITY, entity, app)); + } + + private Set getTimelinelineAppMetrics( + RMAppMetrics appMetrics, long timestamp) { + Set entityMetrics = new HashSet(); + + entityMetrics.add(getTimelineMetric( + ApplicationMetricsConstants.APP_CPU_METRICS, timestamp, + appMetrics.getVcoreSeconds())); + entityMetrics.add(getTimelineMetric( + ApplicationMetricsConstants.APP_MEM_METRICS, timestamp, + appMetrics.getMemorySeconds())); + entityMetrics.add(getTimelineMetric( + ApplicationMetricsConstants.APP_MEM_PREEMPT_METRICS, timestamp, + appMetrics.getPreemptedMemorySeconds())); + entityMetrics.add(getTimelineMetric( + ApplicationMetricsConstants.APP_CPU_PREEMPT_METRICS, timestamp, + appMetrics.getPreemptedVcoreSeconds())); + entityMetrics.add(getTimelineMetric( + ApplicationMetricsConstants.APP_RESOURCE_PREEMPTED_CPU, timestamp, + appMetrics.getResourcePreempted().getVirtualCores())); + entityMetrics.add(getTimelineMetric( + ApplicationMetricsConstants.APP_RESOURCE_PREEMPTED_MEM, timestamp, + appMetrics.getResourcePreempted().getMemorySize())); + entityMetrics.add(getTimelineMetric( + ApplicationMetricsConstants.APP_NON_AM_CONTAINER_PREEMPTED, timestamp, + appMetrics.getNumNonAMContainersPreempted())); + entityMetrics.add(getTimelineMetric( + ApplicationMetricsConstants.APP_AM_CONTAINER_PREEMPTED, timestamp, + appMetrics.getNumAMContainersPreempted())); + + return entityMetrics; + } + + private TimelineMetric getTimelineMetric(String name, long timestamp, + Number value) { + TimelineMetric metric = new TimelineMetric(); + metric.setId(name); + metric.addValue(timestamp, value); + return metric; + } + + @SuppressWarnings("unchecked") + @Override + public void appStateUpdated(RMApp app, YarnApplicationState appState, + long updatedTime) { + ApplicationEntity entity = + createApplicationEntity(app.getApplicationId()); + Map eventInfo = new HashMap(); + eventInfo.put(ApplicationMetricsConstants.STATE_EVENT_INFO, + appState); + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(ApplicationMetricsConstants.STATE_UPDATED_EVENT_TYPE); + tEvent.setTimestamp(updatedTime); + tEvent.setInfo(eventInfo); + entity.addEvent(tEvent); + + // publish in entity info also to query using filters + Map entityInfo = new HashMap(); + entityInfo.put(ApplicationMetricsConstants.STATE_EVENT_INFO, appState); + entity.setInfo(entityInfo); + + getDispatcher().getEventHandler().handle(new TimelineV2PublishEvent( + SystemMetricsEventType.PUBLISH_ENTITY, entity, app.getApplicationId())); + } + + @SuppressWarnings("unchecked") + @Override + public void appACLsUpdated(RMApp app, String appViewACLs, long updatedTime) { + ApplicationEntity entity = createApplicationEntity(app.getApplicationId()); + TimelineEvent tEvent = new TimelineEvent(); + Map entityInfo = new HashMap(); + entityInfo.put(ApplicationMetricsConstants.APP_VIEW_ACLS_ENTITY_INFO, + (appViewACLs == null) ? "" : appViewACLs); + entity.setInfo(entityInfo); + tEvent.setId(ApplicationMetricsConstants.ACLS_UPDATED_EVENT_TYPE); + tEvent.setTimestamp(updatedTime); + entity.addEvent(tEvent); + + getDispatcher().getEventHandler().handle(new TimelineV2PublishEvent( + SystemMetricsEventType.PUBLISH_ENTITY, entity, app.getApplicationId())); + } + + @SuppressWarnings("unchecked") + @Override + public void appUpdated(RMApp app, long currentTimeMillis) { + ApplicationEntity entity = createApplicationEntity(app.getApplicationId()); + Map eventInfo = new HashMap(); + eventInfo.put(ApplicationMetricsConstants.QUEUE_ENTITY_INFO, + app.getQueue()); + eventInfo.put(ApplicationMetricsConstants.APPLICATION_PRIORITY_INFO, + app.getApplicationSubmissionContext().getPriority().getPriority()); + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(ApplicationMetricsConstants.UPDATED_EVENT_TYPE); + tEvent.setTimestamp(currentTimeMillis); + tEvent.setInfo(eventInfo); + entity.addEvent(tEvent); + getDispatcher().getEventHandler().handle(new TimelineV2PublishEvent( + SystemMetricsEventType.PUBLISH_ENTITY, entity, app.getApplicationId())); + } + + private static ApplicationEntity createApplicationEntity( + ApplicationId applicationId) { + ApplicationEntity entity = new ApplicationEntity(); + entity.setId(applicationId.toString()); + return entity; + } + + @SuppressWarnings("unchecked") + @Override + public void appAttemptRegistered(RMAppAttempt appAttempt, + long registeredTime) { + ApplicationAttemptId attemptId = appAttempt.getAppAttemptId(); + TimelineEntity entity = createAppAttemptEntity(attemptId); + entity.setCreatedTime(registeredTime); + + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE); + tEvent.setTimestamp(registeredTime); + entity.addEvent(tEvent); + + Map entityInfo = new HashMap(); + entityInfo.put(AppAttemptMetricsConstants.TRACKING_URL_INFO, + appAttempt.getTrackingUrl()); + entityInfo.put(AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_INFO, + appAttempt.getOriginalTrackingUrl()); + entityInfo.put(AppAttemptMetricsConstants.HOST_INFO, + appAttempt.getHost()); + entityInfo.put(AppAttemptMetricsConstants.RPC_PORT_INFO, + appAttempt.getRpcPort()); + if (appAttempt.getMasterContainer() != null) { + entityInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_INFO, + appAttempt.getMasterContainer().getId().toString()); + } + entity.setInfo(entityInfo); + entity.setIdPrefix( + TimelineServiceHelper.invertLong(attemptId.getAttemptId())); + + getDispatcher().getEventHandler().handle( + new TimelineV2PublishEvent(SystemMetricsEventType.PUBLISH_ENTITY, + entity, appAttempt.getAppAttemptId().getApplicationId())); + } + + @SuppressWarnings("unchecked") + @Override + public void appAttemptFinished(RMAppAttempt appAttempt, + RMAppAttemptState appAttemtpState, RMApp app, long finishedTime) { + ApplicationAttemptId attemptId = appAttempt.getAppAttemptId(); + ApplicationAttemptEntity entity = + createAppAttemptEntity(appAttempt.getAppAttemptId()); + + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(AppAttemptMetricsConstants.FINISHED_EVENT_TYPE); + tEvent.setTimestamp(finishedTime); + entity.addEvent(tEvent); + + Map entityInfo = new HashMap(); + entityInfo.put(AppAttemptMetricsConstants.DIAGNOSTICS_INFO, + appAttempt.getDiagnostics()); + // app will get the final status from app attempt, or create one + // based on app state if it doesn't exist + entityInfo.put(AppAttemptMetricsConstants.FINAL_STATUS_INFO, + app.getFinalApplicationStatus().toString()); + entityInfo.put(AppAttemptMetricsConstants.STATE_INFO, RMServerUtils + .createApplicationAttemptState(appAttemtpState).toString()); + entity.setInfo(entityInfo); + entity.setIdPrefix( + TimelineServiceHelper.invertLong(attemptId.getAttemptId())); + + getDispatcher().getEventHandler().handle( + new TimelineV2PublishEvent(SystemMetricsEventType.PUBLISH_ENTITY, + entity, appAttempt.getAppAttemptId().getApplicationId())); + } + + private static ApplicationAttemptEntity createAppAttemptEntity( + ApplicationAttemptId appAttemptId) { + ApplicationAttemptEntity entity = new ApplicationAttemptEntity(); + entity.setId(appAttemptId.toString()); + entity.setParent(new Identifier(TimelineEntityType.YARN_APPLICATION.name(), + appAttemptId.getApplicationId().toString())); + return entity; + } + + @SuppressWarnings("unchecked") + @Override + public void containerCreated(RMContainer container, long createdTime) { + if (publishContainerEvents) { + TimelineEntity entity = createContainerEntity(container.getContainerId()); + entity.setCreatedTime(createdTime); + + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(ContainerMetricsConstants.CREATED_IN_RM_EVENT_TYPE); + tEvent.setTimestamp(createdTime); + entity.addEvent(tEvent); + + // updated as event info instead of entity info, as entity info is updated + // by NM + Map entityInfo = new HashMap(); + entityInfo.put(ContainerMetricsConstants.ALLOCATED_MEMORY_INFO, + container.getAllocatedResource().getMemorySize()); + entityInfo.put(ContainerMetricsConstants.ALLOCATED_VCORE_INFO, + container.getAllocatedResource().getVirtualCores()); + entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_INFO, + container.getAllocatedNode().getHost()); + entityInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_INFO, + container.getAllocatedNode().getPort()); + entityInfo.put(ContainerMetricsConstants.ALLOCATED_PRIORITY_INFO, + container.getAllocatedPriority().getPriority()); + entityInfo.put( + ContainerMetricsConstants.ALLOCATED_HOST_HTTP_ADDRESS_INFO, + container.getNodeHttpAddress()); + entity.setInfo(entityInfo); + + getDispatcher().getEventHandler().handle(new TimelineV2PublishEvent( + SystemMetricsEventType.PUBLISH_ENTITY, entity, container + .getContainerId().getApplicationAttemptId().getApplicationId())); + } + } + + @SuppressWarnings("unchecked") + @Override + public void containerFinished(RMContainer container, long finishedTime) { + if (publishContainerEvents) { + TimelineEntity entity = createContainerEntity(container.getContainerId()); + + TimelineEvent tEvent = new TimelineEvent(); + tEvent.setId(ContainerMetricsConstants.FINISHED_IN_RM_EVENT_TYPE); + tEvent.setTimestamp(finishedTime); + entity.addEvent(tEvent); + + Map entityInfo = new HashMap(); + entityInfo.put(ContainerMetricsConstants.DIAGNOSTICS_INFO, + container.getDiagnosticsInfo()); + entityInfo.put(ContainerMetricsConstants.EXIT_STATUS_INFO, + container.getContainerExitStatus()); + entityInfo.put(ContainerMetricsConstants.STATE_INFO, + container.getContainerState().toString()); + entityInfo.put(ContainerMetricsConstants.CONTAINER_FINISHED_TIME, + finishedTime); + entity.setInfo(entityInfo); + + getDispatcher().getEventHandler().handle(new TimelineV2PublishEvent( + SystemMetricsEventType.PUBLISH_ENTITY, entity, container + .getContainerId().getApplicationAttemptId().getApplicationId())); + } + } + + private static ContainerEntity createContainerEntity( + ContainerId containerId) { + ContainerEntity entity = new ContainerEntity(); + entity.setId(containerId.toString()); + entity.setParent(new Identifier(TimelineEntityType.YARN_APPLICATION_ATTEMPT + .name(), containerId.getApplicationAttemptId().toString())); + return entity; + } + + private void putEntity(TimelineEntity entity, ApplicationId appId) { + try { + if (LOG.isDebugEnabled()) { + LOG.debug("Publishing the entity " + entity + ", JSON-style content: " + + TimelineUtils.dumpTimelineRecordtoJSON(entity)); + } + TimelineCollector timelineCollector = + rmTimelineCollectorManager.get(appId); + TimelineEntities entities = new TimelineEntities(); + entities.addEntity(entity); + timelineCollector.putEntities(entities, + UserGroupInformation.getCurrentUser()); + } catch (Exception e) { + LOG.error("Error when publishing entity " + entity, e); + } + } + + private class ApplicationFinishPublishEvent extends TimelineV2PublishEvent { + private RMApp app; + + public ApplicationFinishPublishEvent(SystemMetricsEventType type, + TimelineEntity entity, RMApp app) { + super(type, entity, app.getApplicationId()); + this.app = app; + } + + public RMAppImpl getRMAppImpl() { + return (RMAppImpl) app; + } + } + + private class TimelineV2EventHandler + implements EventHandler { + @Override + public void handle(TimelineV2PublishEvent event) { + switch (event.getType()) { + case PUBLISH_APPLICATION_FINISHED_ENTITY: + putEntity(event.getEntity(), event.getApplicationId()); + ((ApplicationFinishPublishEvent) event).getRMAppImpl() + .stopTimelineCollector(); + break; + default: + putEntity(event.getEntity(), event.getApplicationId()); + break; + } + } + } + + private class TimelineV2PublishEvent extends TimelinePublishEvent { + private TimelineEntity entity; + + public TimelineV2PublishEvent(SystemMetricsEventType type, + TimelineEntity entity, ApplicationId appId) { + super(type, appId); + this.entity = entity; + } + + public TimelineEntity getEntity() { + return entity; + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/package-info.java similarity index 73% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsEventType.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/package-info.java index fcda4b4e47e..a8a38040860 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsEventType.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/package-info.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -16,22 +16,13 @@ * limitations under the License. */ +/** + * Package org.apache.hadoop.yarn.server.resourcemanager.metrics contains + * classes related to publishing app/container events to ATS. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable package org.apache.hadoop.yarn.server.resourcemanager.metrics; - -public enum SystemMetricsEventType { - // app events - APP_CREATED, - APP_FINISHED, - APP_ACLS_UPDATED, - APP_UPDATED, - APP_STATE_UPDATED, - - // app attempt events - APP_ATTEMPT_REGISTERED, - APP_ATTEMPT_FINISHED, - - // container events - CONTAINER_CREATED, - CONTAINER_FINISHED -} +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java index a42b3bfc0c8..93c41b6747c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java @@ -23,6 +23,8 @@ import java.util.List; import java.util.Map; import java.util.Set; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.ipc.CallerContext; import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; @@ -30,6 +32,7 @@ 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.ApplicationTimeoutType; +import org.apache.hadoop.yarn.api.records.CollectorInfo; import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; import org.apache.hadoop.yarn.api.records.LogAggregationStatus; import org.apache.hadoop.yarn.api.records.NodeId; @@ -39,6 +42,7 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport; +import org.apache.hadoop.yarn.server.api.records.AppCollectorData; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; @@ -179,6 +183,28 @@ public interface RMApp extends EventHandler { */ String getTrackingUrl(); + /** + * The timeline collector information for the application. It should be used + * only if the timeline service v.2 is enabled. + * + * @return the data for the application's collector, including collector + * address, RM ID, version and collector token. Return null if the timeline + * service v.2 is not enabled. + */ + @InterfaceAudience.Private + @InterfaceStability.Unstable + AppCollectorData getCollectorData(); + + /** + * The timeline collector information to be sent to AM. It should be used + * only if the timeline service v.2 is enabled. + * + * @return collector info, including collector address and collector token. + * Return null if the timeline service v.2 is not enabled. + */ + @InterfaceAudience.Private + @InterfaceStability.Unstable + CollectorInfo getCollectorInfo(); /** * The original tracking url for the application master. * @return the original tracking url for the application master. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java index c29f24410ac..7526ea3c611 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java @@ -53,6 +53,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport; import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; import org.apache.hadoop.yarn.api.records.ApplicationTimeout; import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType; +import org.apache.hadoop.yarn.api.records.CollectorInfo; import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; import org.apache.hadoop.yarn.api.records.LogAggregationStatus; import org.apache.hadoop.yarn.api.records.NodeId; @@ -70,6 +71,7 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier; import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport; +import org.apache.hadoop.yarn.server.api.records.AppCollectorData; import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService; import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEvent; import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEventType; @@ -97,6 +99,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanAppEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent; +import org.apache.hadoop.yarn.server.timelineservice.collector.AppLevelTimelineCollector; import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.state.InvalidStateTransitionException; import org.apache.hadoop.yarn.state.MultipleArcTransition; @@ -160,6 +163,8 @@ public class RMAppImpl implements RMApp, Recoverable { private long storedFinishTime = 0; private int firstAttemptIdInStateStore = 1; private int nextAttemptId = 1; + private AppCollectorData collectorData; + private CollectorInfo collectorInfo; // This field isn't protected by readlock now. private volatile RMAppAttempt currentAttempt; private String queue; @@ -405,6 +410,17 @@ public class RMAppImpl implements RMApp, Recoverable { ApplicationMasterService masterService, long submitTime, String applicationType, Set applicationTags, List amReqs) { + this(applicationId, rmContext, config, name, user, queue, submissionContext, + scheduler, masterService, submitTime, applicationType, applicationTags, + amReqs, -1); + } + + public RMAppImpl(ApplicationId applicationId, RMContext rmContext, + Configuration config, String name, String user, String queue, + ApplicationSubmissionContext submissionContext, YarnScheduler scheduler, + ApplicationMasterService masterService, long submitTime, + String applicationType, Set applicationTags, + List amReqs, long startTime) { this.systemClock = SystemClock.getInstance(); @@ -420,7 +436,11 @@ public class RMAppImpl implements RMApp, Recoverable { this.scheduler = scheduler; this.masterService = masterService; this.submitTime = submitTime; - this.startTime = this.systemClock.getTime(); + if (startTime <= 0) { + this.startTime = this.systemClock.getTime(); + } else { + this.startTime = startTime; + } this.applicationType = applicationType; this.applicationTags = applicationTags; this.amReqs = amReqs; @@ -502,6 +522,25 @@ public class RMAppImpl implements RMApp, Recoverable { } } + /** + * Starts the application level timeline collector for this app. This should + * be used only if the timeline service v.2 is enabled. + */ + public void startTimelineCollector() { + AppLevelTimelineCollector collector = + new AppLevelTimelineCollector(applicationId, user); + rmContext.getRMTimelineCollectorManager().putIfAbsent( + applicationId, collector); + } + + /** + * Stops the application level timeline collector for this app. This should be + * used only if the timeline service v.2 is enabled. + */ + public void stopTimelineCollector() { + rmContext.getRMTimelineCollectorManager().remove(applicationId); + } + @Override public ApplicationId getApplicationId() { return this.applicationId; @@ -570,6 +609,25 @@ public class RMAppImpl implements RMApp, Recoverable { this.queue = queue; } + @Override + public AppCollectorData getCollectorData() { + return this.collectorData; + } + + public void setCollectorData(AppCollectorData incomingData) { + this.collectorData = incomingData; + this.collectorInfo = CollectorInfo.newInstance( + incomingData.getCollectorAddr(), incomingData.getCollectorToken()); + } + + public CollectorInfo getCollectorInfo() { + return this.collectorInfo; + } + + public void removeCollectorData() { + this.collectorData = null; + } + @Override public String getName() { return this.name; @@ -865,11 +923,12 @@ public class RMAppImpl implements RMApp, Recoverable { this.firstAttemptIdInStateStore = appState.getFirstAttemptId(); this.nextAttemptId = firstAttemptIdInStateStore; } + //TODO recover collector address. + //this.collectorAddr = appState.getCollectorAddr(); // send the ATS create Event during RM recovery. // NOTE: it could be duplicated with events sent before RM get restarted. sendATSCreateEvent(); - RMAppAttemptImpl preAttempt = null; for (ApplicationAttemptId attemptId : new TreeSet<>(appState.attempts.keySet())) { @@ -935,7 +994,6 @@ public class RMAppImpl implements RMApp, Recoverable { SingleArcTransition { public void transition(RMAppImpl app, RMAppEvent event) { }; - } private static final class RMAppNodeUpdateTransition extends RMAppTransition { @@ -1880,7 +1938,7 @@ public class RMAppImpl implements RMApp, Recoverable { } return amNodeLabelExpression; } - + @Override public CallerContext getCallerContext() { return callerContext; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/timelineservice/RMTimelineCollectorManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/timelineservice/RMTimelineCollectorManager.java new file mode 100644 index 00000000000..64c374910ff --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/timelineservice/RMTimelineCollectorManager.java @@ -0,0 +1,104 @@ +/** + * 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.timelineservice; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; +import org.apache.hadoop.yarn.server.resourcemanager.RMContext; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollector; +import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext; +import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorManager; +import org.apache.hadoop.yarn.util.timeline.TimelineUtils; + +/** + * This class extends TimelineCollectorManager to provide RM specific + * implementations. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public class RMTimelineCollectorManager extends TimelineCollectorManager { + private static final Log LOG = + LogFactory.getLog(RMTimelineCollectorManager.class); + + private RMContext rmContext; + + public RMTimelineCollectorManager(RMContext rmContext) { + super(RMTimelineCollectorManager.class.getName()); + this.rmContext = rmContext; + } + + @Override + protected void doPostPut(ApplicationId appId, TimelineCollector collector) { + RMApp app = rmContext.getRMApps().get(appId); + if (app == null) { + throw new YarnRuntimeException( + "Unable to get the timeline collector context info for a " + + "non-existing app " + appId); + } + String userId = app.getUser(); + TimelineCollectorContext context = collector.getTimelineEntityContext(); + if (userId != null && !userId.isEmpty()) { + context.setUserId(userId); + } + + // initialize the flow in the environment with default values for those + // that do not specify the flow tags + // flow name: app name (or app id if app name is missing), + // flow version: "1", flow run id: start time + context.setFlowName(TimelineUtils.generateDefaultFlowName( + app.getName(), appId)); + context.setFlowVersion(TimelineUtils.DEFAULT_FLOW_VERSION); + context.setFlowRunId(app.getStartTime()); + + // the flow context is received via the application tags + for (String tag : app.getApplicationTags()) { + String[] parts = tag.split(":", 2); + if (parts.length != 2 || parts[1].isEmpty()) { + continue; + } + switch (parts[0].toUpperCase()) { + case TimelineUtils.FLOW_NAME_TAG_PREFIX: + if (LOG.isDebugEnabled()) { + LOG.debug("Setting the flow name: " + parts[1]); + } + context.setFlowName(parts[1]); + break; + case TimelineUtils.FLOW_VERSION_TAG_PREFIX: + if (LOG.isDebugEnabled()) { + LOG.debug("Setting the flow version: " + parts[1]); + } + context.setFlowVersion(parts[1]); + break; + case TimelineUtils.FLOW_RUN_ID_TAG_PREFIX: + if (LOG.isDebugEnabled()) { + LOG.debug("Setting the flow run id: " + parts[1]); + } + context.setFlowRunId(Long.parseLong(parts[1])); + break; + default: + break; + } + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/timelineservice/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/timelineservice/package-info.java new file mode 100644 index 00000000000..c4700117793 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/timelineservice/package-info.java @@ -0,0 +1,28 @@ +/* + * 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.timelineservice + * contains classes related to handling of app level collectors. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.resourcemanager.timelineservice; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java index 87e9ae91fc8..0db1092de99 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java @@ -24,6 +24,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; @@ -39,6 +40,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse; +import org.apache.hadoop.yarn.server.api.records.AppCollectorData; import org.apache.hadoop.yarn.server.api.records.MasterKey; import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus; import org.apache.hadoop.yarn.server.api.records.NodeStatus; @@ -60,6 +62,8 @@ public class MockNM { private String version; private Map containerStats = new HashMap(); + private Map registeringCollectors + = new ConcurrentHashMap<>(); public MockNM(String nodeIdStr, int memory, ResourceTrackerService resourceTracker) { // scale vcores based on the requested memory @@ -117,6 +121,15 @@ public class MockNM { true, ++responseId); } + public void addRegisteringCollector(ApplicationId appId, + AppCollectorData data) { + this.registeringCollectors.put(appId, data); + } + + public Map getRegisteringCollectors() { + return this.registeringCollectors; + } + public RegisterNodeManagerResponse registerNode() throws Exception { return registerNode(null, null); } @@ -229,6 +242,9 @@ public class MockNM { req.setNodeStatus(status); req.setLastKnownContainerTokenMasterKey(this.currentContainerTokenMasterKey); req.setLastKnownNMTokenMasterKey(this.currentNMTokenMasterKey); + + req.setRegisteringCollectors(this.registeringCollectors); + NodeHeartbeatResponse heartbeatResponse = resourceTracker.nodeHeartbeat(req); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java index ea3117f51f1..009eb2c56e1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java @@ -84,6 +84,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM; +import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; @@ -206,9 +207,11 @@ public class TestAppManager{ public int getCompletedAppsListSize() { return super.getCompletedAppsListSize(); } + public int getCompletedAppsInStateStore() { return this.completedAppsInStateStore; } + public void submitApplication( ApplicationSubmissionContext submissionContext, String user) throws YarnException, IOException { @@ -239,6 +242,8 @@ public class TestAppManager{ long now = System.currentTimeMillis(); rmContext = mockRMContext(1, now - 10); + rmContext + .setRMTimelineCollectorManager(mock(RMTimelineCollectorManager.class)); ResourceScheduler scheduler = mockResourceScheduler(); ((RMContextImpl)rmContext).setScheduler(scheduler); Configuration conf = new Configuration(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java index df5fae37c2b..6946f3cd2e7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java @@ -25,6 +25,7 @@ import static org.mockito.Matchers.anyBoolean; import static org.mockito.Matchers.anyListOf; import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -146,6 +147,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedule import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager; import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey; +import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.Clock; @@ -661,6 +663,9 @@ public class TestClientRMService { new EventHandler() { public void handle(Event event) {} }); + doReturn(mock(RMTimelineCollectorManager.class)).when(rmContext) + .getRMTimelineCollectorManager(); + ApplicationId appId1 = getApplicationId(100); ApplicationACLsManager mockAclsManager = mock(ApplicationACLsManager.class); @@ -676,6 +681,7 @@ public class TestClientRMService { ClientRMService rmService = new ClientRMService(rmContext, yarnScheduler, appManager, mockAclsManager, mockQueueACLsManager, null); + rmService.init(new Configuration()); // without name and queue @@ -762,6 +768,9 @@ public class TestClientRMService { mockRMContext(yarnScheduler, rmContext); RMStateStore stateStore = mock(RMStateStore.class); when(rmContext.getStateStore()).thenReturn(stateStore); + doReturn(mock(RMTimelineCollectorManager.class)).when(rmContext) + .getRMTimelineCollectorManager(); + RMAppManager appManager = new RMAppManager(rmContext, yarnScheduler, null, mock(ApplicationACLsManager.class), new Configuration()); when(rmContext.getDispatcher().getEventHandler()).thenReturn( @@ -778,6 +787,7 @@ public class TestClientRMService { ClientRMService rmService = new ClientRMService(rmContext, yarnScheduler, appManager, mockAclsManager, mockQueueACLsManager, null); + rmService.init(new Configuration()); // Initialize appnames and queues String[] queues = {QUEUE_1, QUEUE_2}; @@ -941,10 +951,13 @@ public class TestClientRMService { }; when(rmContext.getDispatcher().getEventHandler()).thenReturn(eventHandler); - + doReturn(mock(RMTimelineCollectorManager.class)).when(rmContext) + .getRMTimelineCollectorManager(); + final ClientRMService rmService = new ClientRMService(rmContext, yarnScheduler, appManager, null, null, null); + rmService.init(new Configuration()); // submit an app and wait for it to block while in app submission Thread t = new Thread() { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHATimelineCollectors.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHATimelineCollectors.java new file mode 100644 index 00000000000..a0cd0486d3a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHATimelineCollectors.java @@ -0,0 +1,122 @@ +/* + * 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; + +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.api.records.AppCollectorData; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +/** + * Test if the new active RM could recover collector status on a state + * transition. + */ +public class TestRMHATimelineCollectors extends RMHATestBase { + + @Before + @Override + public void setup() throws Exception { + super.setup(); + confForRM1.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); + confForRM2.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); + confForRM1.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS, + FileSystemTimelineWriterImpl.class, TimelineWriter.class); + confForRM1.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f); + confForRM2.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f); + confForRM2.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS, + FileSystemTimelineWriterImpl.class, TimelineWriter.class); + } + + @Test + public void testRebuildCollectorDataOnFailover() throws Exception { + startRMs(); + MockNM nm1 + = new MockNM("127.0.0.1:1234", 15120, rm2.getResourceTrackerService()); + MockNM nm2 + = new MockNM("127.0.0.1:5678", 15121, rm2.getResourceTrackerService()); + RMApp app1 = rm1.submitApp(1024); + String collectorAddr1 = "1.2.3.4:5"; + AppCollectorData data1 = AppCollectorData.newInstance( + app1.getApplicationId(), collectorAddr1); + nm1.addRegisteringCollector(app1.getApplicationId(), data1); + + String collectorAddr2 = "5.4.3.2:1"; + RMApp app2 = rm1.submitApp(1024); + AppCollectorData data2 = AppCollectorData.newInstance( + app2.getApplicationId(), collectorAddr2, rm1.getStartTime(), 1); + nm1.addRegisteringCollector(app2.getApplicationId(), data2); + + explicitFailover(); + + List runningApps = new ArrayList<>(); + runningApps.add(app1.getApplicationId()); + runningApps.add(app2.getApplicationId()); + nm1.registerNode(runningApps); + nm2.registerNode(runningApps); + + String collectorAddr12 = "1.2.3.4:56"; + AppCollectorData data12 = AppCollectorData.newInstance( + app1.getApplicationId(), collectorAddr12, rm1.getStartTime(), 0); + nm2.addRegisteringCollector(app1.getApplicationId(), data12); + + String collectorAddr22 = "5.4.3.2:10"; + AppCollectorData data22 = AppCollectorData.newInstance( + app2.getApplicationId(), collectorAddr22, rm1.getStartTime(), 2); + nm2.addRegisteringCollector(app2.getApplicationId(), data22); + + Map results1 + = nm1.nodeHeartbeat(true).getAppCollectors(); + assertEquals(collectorAddr1, + results1.get(app1.getApplicationId()).getCollectorAddr()); + assertEquals(collectorAddr2, + results1.get(app2.getApplicationId()).getCollectorAddr()); + + Map results2 + = nm2.nodeHeartbeat(true).getAppCollectors(); + // addr of app1 should be collectorAddr1 since it's registering (no time + // stamp). + assertEquals(collectorAddr1, + results2.get(app1.getApplicationId()).getCollectorAddr()); + // addr of app2 should be collectorAddr22 since its version number is + // greater. + assertEquals(collectorAddr22, + results2.get(app2.getApplicationId()).getCollectorAddr()); + + // Now nm1 should get updated collector list + nm1.getRegisteringCollectors().clear(); + Map results12 + = nm1.nodeHeartbeat(true).getAppCollectors(); + assertEquals(collectorAddr1, + results12.get(app1.getApplicationId()).getCollectorAddr()); + assertEquals(collectorAddr22, + results12.get(app2.getApplicationId()).getCollectorAddr()); + + + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java index e99bcde848c..aa866216fe9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java @@ -118,6 +118,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtil import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; +import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext; +import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter; import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.log4j.Level; @@ -153,6 +156,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase { conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true); conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, false); conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName()); + conf.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS, + FileSystemTimelineWriterImpl.class, TimelineWriter.class); rmAddr = new InetSocketAddress("localhost", 8032); Assert.assertTrue(YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS > 1); } @@ -1135,6 +1140,68 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase { Assert.assertNull(rmAppState.get(app2.getApplicationId()).getState()); } + @Test (timeout = 60000) + public void testRMRestartTimelineCollectorContext() throws Exception { + conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); + conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f); + MemoryRMStateStore memStore = new MemoryRMStateStore(); + memStore.init(conf); + RMState rmState = memStore.getState(); + Map rmAppState = + rmState.getApplicationState(); + MockRM rm1 = null; + MockRM rm2 = null; + try { + rm1 = createMockRM(conf, memStore); + rm1.start(); + MockNM nm1 = + new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService()); + nm1.registerNode(); + + // submit an app. + RMApp app = rm1.submitApp(200, "name", "user", + new HashMap(), false, "default", -1, + null); + // Check if app info has been saved. + ApplicationStateData appState = rmAppState.get(app.getApplicationId()); + Assert.assertNotNull(appState); + Assert.assertEquals(0, appState.getAttemptCount()); + Assert.assertEquals(appState.getApplicationSubmissionContext() + .getApplicationId(), app.getApplicationSubmissionContext() + .getApplicationId()); + + // Allocate the AM + nm1.nodeHeartbeat(true); + RMAppAttempt attempt = app.getCurrentAppAttempt(); + ApplicationAttemptId attemptId1 = attempt.getAppAttemptId(); + rm1.waitForState(attemptId1, RMAppAttemptState.ALLOCATED); + + ApplicationId appId = app.getApplicationId(); + TimelineCollectorContext contextBeforeRestart = + rm1.getRMContext().getRMTimelineCollectorManager().get(appId). + getTimelineEntityContext(); + + // Restart RM. + rm2 = createMockRM(conf, memStore); + rm2.start(); + Assert.assertEquals(1, rm2.getRMContext().getRMApps().size()); + rm2.waitForState(app.getApplicationId(), RMAppState.ACCEPTED); + TimelineCollectorContext contextAfterRestart = + rm2.getRMContext().getRMTimelineCollectorManager().get(appId). + getTimelineEntityContext(); + Assert.assertEquals("Collector contexts for an app should be same " + + "across restarts", contextBeforeRestart, contextAfterRestart); + } finally { + conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, false); + if (rm1 != null) { + rm1.close(); + } + if (rm2 != null) { + rm2.close(); + } + } + } + @Test (timeout = 60000) public void testDelegationTokenRestoredInDelegationTokenRenewer() throws Exception { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java index 8ff754b7a87..68a6a22370c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java @@ -70,13 +70,16 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.UnRegisterNodeManagerRequest; +import org.apache.hadoop.yarn.server.api.records.AppCollectorData; import org.apache.hadoop.yarn.server.api.records.NodeAction; +import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus; import org.apache.hadoop.yarn.server.api.records.NodeStatus; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NodeLabelsUtils; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; @@ -87,6 +90,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; +import org.apache.hadoop.yarn.server.timelineservice.collector.PerNodeTimelineCollectorsAuxService; +import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter; import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.YarnVersionInfo; @@ -989,6 +995,103 @@ public class TestResourceTrackerService extends NodeLabelTestBase { checkRebootedNMCount(rm, ++initialMetricCount); } + @Test + public void testNodeHeartbeatForAppCollectorsMap() throws Exception { + Configuration conf = new Configuration(); + conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); + // set version to 2 + conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f); + // enable aux-service based timeline collectors + conf.set(YarnConfiguration.NM_AUX_SERVICES, "timeline_collector"); + conf.set(YarnConfiguration.NM_AUX_SERVICES + "." + + "timeline_collector" + ".class", + PerNodeTimelineCollectorsAuxService.class.getName()); + conf.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS, + FileSystemTimelineWriterImpl.class, TimelineWriter.class); + + rm = new MockRM(conf); + rm.start(); + + MockNM nm1 = rm.registerNode("host1:1234", 5120); + MockNM nm2 = rm.registerNode("host2:1234", 2048); + + NodeHeartbeatResponse nodeHeartbeat1 = nm1.nodeHeartbeat(true); + NodeHeartbeatResponse nodeHeartbeat2 = nm2.nodeHeartbeat(true); + + RMNodeImpl node1 = + (RMNodeImpl) rm.getRMContext().getRMNodes().get(nm1.getNodeId()); + + RMNodeImpl node2 = + (RMNodeImpl) rm.getRMContext().getRMNodes().get(nm2.getNodeId()); + + RMAppImpl app1 = (RMAppImpl) rm.submitApp(1024); + String collectorAddr1 = "1.2.3.4:5"; + app1.setCollectorData(AppCollectorData.newInstance( + app1.getApplicationId(), collectorAddr1)); + + String collectorAddr2 = "5.4.3.2:1"; + RMAppImpl app2 = (RMAppImpl) rm.submitApp(1024); + app2.setCollectorData(AppCollectorData.newInstance( + app2.getApplicationId(), collectorAddr2)); + + String collectorAddr3 = "5.4.3.2:2"; + app2.setCollectorData(AppCollectorData.newInstance( + app2.getApplicationId(), collectorAddr3, 0, 1)); + + String collectorAddr4 = "5.4.3.2:3"; + app2.setCollectorData(AppCollectorData.newInstance( + app2.getApplicationId(), collectorAddr4, 1, 0)); + + // Create a running container for app1 running on nm1 + ContainerId runningContainerId1 = BuilderUtils.newContainerId( + BuilderUtils.newApplicationAttemptId( + app1.getApplicationId(), 0), 0); + + ContainerStatus status1 = ContainerStatus.newInstance(runningContainerId1, + ContainerState.RUNNING, "", 0); + List statusList = new ArrayList(); + statusList.add(status1); + NodeHealthStatus nodeHealth = NodeHealthStatus.newInstance(true, + "", System.currentTimeMillis()); + NodeStatus nodeStatus = NodeStatus.newInstance(nm1.getNodeId(), 0, + statusList, null, nodeHealth, null, null, null); + node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus, + nodeHeartbeat1)); + + Assert.assertEquals(1, node1.getRunningApps().size()); + Assert.assertEquals(app1.getApplicationId(), node1.getRunningApps().get(0)); + + // Create a running container for app2 running on nm2 + ContainerId runningContainerId2 = BuilderUtils.newContainerId( + BuilderUtils.newApplicationAttemptId( + app2.getApplicationId(), 0), 0); + + ContainerStatus status2 = ContainerStatus.newInstance(runningContainerId2, + ContainerState.RUNNING, "", 0); + statusList = new ArrayList(); + statusList.add(status2); + nodeStatus = NodeStatus.newInstance(nm1.getNodeId(), 0, + statusList, null, nodeHealth, null, null, null); + node2.handle(new RMNodeStatusEvent(nm2.getNodeId(), nodeStatus, + nodeHeartbeat2)); + Assert.assertEquals(1, node2.getRunningApps().size()); + Assert.assertEquals(app2.getApplicationId(), node2.getRunningApps().get(0)); + + nodeHeartbeat1 = nm1.nodeHeartbeat(true); + Map map1 + = nodeHeartbeat1.getAppCollectors(); + Assert.assertEquals(1, map1.size()); + Assert.assertEquals(collectorAddr1, + map1.get(app1.getApplicationId()).getCollectorAddr()); + + nodeHeartbeat2 = nm2.nodeHeartbeat(true); + Map map2 + = nodeHeartbeat2.getAppCollectors(); + Assert.assertEquals(1, map2.size()); + Assert.assertEquals(collectorAddr4, + map2.get(app2.getApplicationId()).getCollectorAddr()); + } + private void checkRebootedNMCount(MockRM rm2, int count) throws InterruptedException { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java index b9adf78e32d..f826631a21d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java @@ -31,6 +31,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport; import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType; +import org.apache.hadoop.yarn.api.records.CollectorInfo; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; @@ -43,6 +44,7 @@ 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.server.api.protocolrecords.LogAggregationReport; +import org.apache.hadoop.yarn.server.api.records.AppCollectorData; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics; @@ -97,6 +99,10 @@ public abstract class MockAsm extends MockApps { throw new UnsupportedOperationException("Not supported yet."); } @Override + public AppCollectorData getCollectorData() { + throw new UnsupportedOperationException("Not supported yet."); + } + @Override public ApplicationId getApplicationId() { throw new UnsupportedOperationException("Not supported yet."); } @@ -234,6 +240,11 @@ public abstract class MockAsm extends MockApps { public boolean isAppInCompletedStates() { throw new UnsupportedOperationException("Not supported yet."); } + + @Override + public CollectorInfo getCollectorInfo() { + throw new UnsupportedOperationException("Not supported yet."); + } } public static RMApp newApplication(int i) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java index a3dd9bb4099..677990b8c98 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java @@ -58,6 +58,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStatusEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType; +import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -90,9 +91,12 @@ public class TestRMAppLogAggregationStatus { rmContext = new RMContextImpl(rmDispatcher, null, null, null, null, null, null, null, null); - rmContext.setSystemMetricsPublisher(new SystemMetricsPublisher()); + rmContext.setSystemMetricsPublisher(mock(SystemMetricsPublisher.class)); rmContext.setRMApplicationHistoryWriter(mock(RMApplicationHistoryWriter.class)); + rmContext + .setRMTimelineCollectorManager(mock(RMTimelineCollectorManager.class)); + scheduler = mock(YarnScheduler.class); doAnswer( new Answer() { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java index 56dfe6eb0f0..30ad2e0d652 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java @@ -70,14 +70,14 @@ import org.junit.Test; public class TestSystemMetricsPublisher { private static ApplicationHistoryServer timelineServer; - private static SystemMetricsPublisher metricsPublisher; + private static TimelineServiceV1Publisher metricsPublisher; private static TimelineStore store; @BeforeClass public static void setup() throws Exception { YarnConfiguration conf = new YarnConfiguration(); conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); - conf.setBoolean(YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_ENABLED, true); + conf.setBoolean(YarnConfiguration.SYSTEM_METRICS_PUBLISHER_ENABLED, true); conf.setClass(YarnConfiguration.TIMELINE_SERVICE_STORE, MemoryTimelineStore.class, TimelineStore.class); conf.setClass(YarnConfiguration.TIMELINE_SERVICE_STATE_STORE_CLASS, @@ -91,7 +91,7 @@ public class TestSystemMetricsPublisher { timelineServer.start(); store = timelineServer.getTimelineStore(); - metricsPublisher = new SystemMetricsPublisher(); + metricsPublisher = new TimelineServiceV1Publisher(); metricsPublisher.init(conf); metricsPublisher.start(); } @@ -351,34 +351,34 @@ public class TestSystemMetricsPublisher { hasRegisteredEvent = true; Assert.assertEquals(appAttempt.getHost(), event.getEventInfo() - .get(AppAttemptMetricsConstants.HOST_EVENT_INFO)); + .get(AppAttemptMetricsConstants.HOST_INFO)); Assert .assertEquals(appAttempt.getRpcPort(), event.getEventInfo().get( - AppAttemptMetricsConstants.RPC_PORT_EVENT_INFO)); + AppAttemptMetricsConstants.RPC_PORT_INFO)); Assert.assertEquals( appAttempt.getMasterContainer().getId().toString(), event.getEventInfo().get( - AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO)); + AppAttemptMetricsConstants.MASTER_CONTAINER_INFO)); } else if (event.getEventType().equals( AppAttemptMetricsConstants.FINISHED_EVENT_TYPE)) { hasFinishedEvent = true; Assert.assertEquals(appAttempt.getDiagnostics(), event.getEventInfo() - .get(AppAttemptMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO)); + .get(AppAttemptMetricsConstants.DIAGNOSTICS_INFO)); Assert.assertEquals(appAttempt.getTrackingUrl(), event.getEventInfo() - .get(AppAttemptMetricsConstants.TRACKING_URL_EVENT_INFO)); + .get(AppAttemptMetricsConstants.TRACKING_URL_INFO)); Assert.assertEquals( appAttempt.getOriginalTrackingUrl(), event.getEventInfo().get( - AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_EVENT_INFO)); + AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_INFO)); Assert.assertEquals( FinalApplicationStatus.UNDEFINED.toString(), event.getEventInfo().get( - AppAttemptMetricsConstants.FINAL_STATUS_EVENT_INFO)); + AppAttemptMetricsConstants.FINAL_STATUS_INFO)); Assert.assertEquals( YarnApplicationAttemptState.FINISHED.toString(), event.getEventInfo().get( - AppAttemptMetricsConstants.STATE_EVENT_INFO)); + AppAttemptMetricsConstants.STATE_INFO)); } } Assert.assertTrue(hasRegisteredEvent && hasFinishedEvent); @@ -401,17 +401,17 @@ public class TestSystemMetricsPublisher { Assert.assertNotNull(entity.getOtherInfo()); Assert.assertEquals(2, entity.getOtherInfo().size()); Assert.assertNotNull(entity.getOtherInfo().get( - ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO)); + ContainerMetricsConstants.ALLOCATED_HOST_INFO)); Assert.assertNotNull(entity.getOtherInfo().get( - ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO)); + ContainerMetricsConstants.ALLOCATED_PORT_INFO)); Assert.assertEquals( container.getAllocatedNode().getHost(), entity.getOtherInfo().get( - ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO)); + ContainerMetricsConstants.ALLOCATED_HOST_INFO)); Assert.assertEquals( container.getAllocatedNode().getPort(), entity.getOtherInfo().get( - ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO)); + ContainerMetricsConstants.ALLOCATED_PORT_INFO)); } @Test(timeout = 10000) @@ -442,25 +442,25 @@ public class TestSystemMetricsPublisher { Assert.assertEquals( container.getAllocatedNode().getHost(), entity.getOtherInfo().get( - ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO)); + ContainerMetricsConstants.ALLOCATED_HOST_INFO)); Assert.assertEquals( container.getAllocatedNode().getPort(), entity.getOtherInfo().get( - ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO)); + ContainerMetricsConstants.ALLOCATED_PORT_INFO)); Assert.assertEquals(container.getAllocatedResource().getMemorySize(), // KeyValueBasedTimelineStore could cast long to integer, need make sure // variables for compare have same type. ((Integer) entity.getOtherInfo().get( - ContainerMetricsConstants.ALLOCATED_MEMORY_ENTITY_INFO)) + ContainerMetricsConstants.ALLOCATED_MEMORY_INFO)) .longValue()); Assert.assertEquals( container.getAllocatedResource().getVirtualCores(), entity.getOtherInfo().get( - ContainerMetricsConstants.ALLOCATED_VCORE_ENTITY_INFO)); + ContainerMetricsConstants.ALLOCATED_VCORE_INFO)); Assert.assertEquals( container.getAllocatedPriority().getPriority(), entity.getOtherInfo().get( - ContainerMetricsConstants.ALLOCATED_PRIORITY_ENTITY_INFO)); + ContainerMetricsConstants.ALLOCATED_PRIORITY_INFO)); boolean hasCreatedEvent = false; boolean hasFinishedEvent = false; for (TimelineEvent event : entity.getEvents()) { @@ -475,13 +475,13 @@ public class TestSystemMetricsPublisher { Assert.assertEquals( container.getDiagnosticsInfo(), event.getEventInfo().get( - ContainerMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO)); + ContainerMetricsConstants.DIAGNOSTICS_INFO)); Assert.assertEquals( container.getContainerExitStatus(), event.getEventInfo().get( - ContainerMetricsConstants.EXIT_STATUS_EVENT_INFO)); + ContainerMetricsConstants.EXIT_STATUS_INFO)); Assert.assertEquals(container.getContainerState().toString(), event - .getEventInfo().get(ContainerMetricsConstants.STATE_EVENT_INFO)); + .getEventInfo().get(ContainerMetricsConstants.STATE_INFO)); } } Assert.assertTrue(hasCreatedEvent && hasFinishedEvent); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java new file mode 100644 index 00000000000..b18a70221bc --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java @@ -0,0 +1,422 @@ +/** + * 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.metrics; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.util.Collections; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileContext; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; +import org.apache.hadoop.yarn.api.records.ContainerState; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +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.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.Dispatcher; +import org.apache.hadoop.yarn.event.DrainDispatcher; +import org.apache.hadoop.yarn.server.metrics.AppAttemptMetricsConstants; +import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants; +import org.apache.hadoop.yarn.server.metrics.ContainerMetricsConstants; +import org.apache.hadoop.yarn.server.resourcemanager.RMContext; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState; +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.rmcontainer.RMContainer; +import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager; +import org.apache.hadoop.yarn.server.timelineservice.collector.AppLevelTimelineCollector; +import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineReaderImpl; +import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter; +import org.apache.hadoop.yarn.util.TimelineServiceHelper; +import org.apache.hadoop.yarn.util.timeline.TimelineUtils; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +/** + * Tests publishing of entities and metrics from RM to ATSv2. + */ +public class TestSystemMetricsPublisherForV2 { + + /** + * The folder where the FileSystemTimelineWriterImpl writes the entities. + */ + private static File testRootDir = new File("target", + TestSystemMetricsPublisherForV2.class.getName() + "-localDir") + .getAbsoluteFile(); + + private static TimelineServiceV2Publisher metricsPublisher; + private static DrainDispatcher dispatcher = new DrainDispatcher(); + + private static ConcurrentMap rmAppsMapInContext; + + private static RMTimelineCollectorManager rmTimelineCollectorManager; + + @BeforeClass + public static void setup() throws Exception { + if (testRootDir.exists()) { + //cleanup before hand + FileContext.getLocalFSFileContext().delete( + new Path(testRootDir.getAbsolutePath()), true); + } + + RMContext rmContext = mock(RMContext.class); + rmAppsMapInContext = new ConcurrentHashMap(); + when(rmContext.getRMApps()).thenReturn(rmAppsMapInContext); + rmTimelineCollectorManager = new RMTimelineCollectorManager(rmContext); + when(rmContext.getRMTimelineCollectorManager()).thenReturn( + rmTimelineCollectorManager); + + Configuration conf = getTimelineV2Conf(); + conf.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS, + FileSystemTimelineWriterImpl.class, TimelineWriter.class); + rmTimelineCollectorManager.init(conf); + rmTimelineCollectorManager.start(); + + dispatcher.init(conf); + dispatcher.start(); + metricsPublisher = new TimelineServiceV2Publisher(rmContext) { + @Override + protected Dispatcher getDispatcher() { + return dispatcher; + } + }; + metricsPublisher.init(conf); + metricsPublisher.start(); + } + + @AfterClass + public static void tearDown() throws Exception { + if (testRootDir.exists()) { + FileContext.getLocalFSFileContext().delete( + new Path(testRootDir.getAbsolutePath()), true); + } + if (rmTimelineCollectorManager != null) { + rmTimelineCollectorManager.stop(); + } + if (metricsPublisher != null) { + metricsPublisher.stop(); + } + } + + private static Configuration getTimelineV2Conf() { + Configuration conf = new Configuration(); + conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); + conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f); + conf.setBoolean(YarnConfiguration.SYSTEM_METRICS_PUBLISHER_ENABLED, true); + conf.setInt( + YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_DISPATCHER_POOL_SIZE, 2); + conf.setBoolean(YarnConfiguration.RM_PUBLISH_CONTAINER_EVENTS_ENABLED, + true); + try { + conf.set(FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_DIR_ROOT, + testRootDir.getCanonicalPath()); + } catch (IOException e) { + e.printStackTrace(); + Assert + .fail("Exception while setting the " + + "TIMELINE_SERVICE_STORAGE_DIR_ROOT "); + } + return conf; + } + + @Test + public void testSystemMetricPublisherInitialization() { + @SuppressWarnings("resource") + TimelineServiceV2Publisher publisher = + new TimelineServiceV2Publisher(mock(RMContext.class)); + try { + Configuration conf = getTimelineV2Conf(); + conf.setBoolean(YarnConfiguration.RM_PUBLISH_CONTAINER_EVENTS_ENABLED, + YarnConfiguration.DEFAULT_RM_PUBLISH_CONTAINER_EVENTS_ENABLED); + publisher.init(conf); + assertFalse( + "Default configuration should not publish container events from RM", + publisher.isPublishContainerEvents()); + + publisher.stop(); + + publisher = new TimelineServiceV2Publisher(mock(RMContext.class)); + conf = getTimelineV2Conf(); + publisher.init(conf); + assertTrue("Expected to have registered event handlers and set ready to " + + "publish events after init", + publisher.isPublishContainerEvents()); + publisher.start(); + assertTrue("Expected to publish container events from RM", + publisher.isPublishContainerEvents()); + } finally { + publisher.stop(); + } + } + + @Test(timeout = 10000) + public void testPublishApplicationMetrics() throws Exception { + ApplicationId appId = ApplicationId.newInstance(0, 1); + RMApp app = createAppAndRegister(appId); + + metricsPublisher.appCreated(app, app.getStartTime()); + metricsPublisher.appACLsUpdated(app, "user1,user2", 4L); + metricsPublisher.appFinished(app, RMAppState.FINISHED, app.getFinishTime()); + dispatcher.await(); + + String outputDirApp = + getTimelineEntityDir(app) + "/" + TimelineEntityType.YARN_APPLICATION + + "/"; + + File entityFolder = new File(outputDirApp); + Assert.assertTrue(entityFolder.isDirectory()); + + // file name is .thist + String timelineServiceFileName = + appId.toString() + + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION; + File appFile = new File(outputDirApp, timelineServiceFileName); + Assert.assertTrue(appFile.exists()); + verifyEntity( + appFile, 3, ApplicationMetricsConstants.CREATED_EVENT_TYPE, 8, 0); + } + + @Test(timeout = 10000) + public void testPublishAppAttemptMetrics() throws Exception { + ApplicationId appId = ApplicationId.newInstance(0, 1); + RMApp app = rmAppsMapInContext.get(appId); + if (app == null) { + app = createAppAndRegister(appId); + } + ApplicationAttemptId appAttemptId = + ApplicationAttemptId.newInstance(appId, 1); + RMAppAttempt appAttempt = createRMAppAttempt(appAttemptId); + metricsPublisher.appAttemptRegistered(appAttempt, Integer.MAX_VALUE + 1L); + when(app.getFinalApplicationStatus()).thenReturn( + FinalApplicationStatus.UNDEFINED); + metricsPublisher.appAttemptFinished(appAttempt, RMAppAttemptState.FINISHED, + app, Integer.MAX_VALUE + 2L); + + dispatcher.await(); + + String outputDirApp = + getTimelineEntityDir(app) + "/" + + TimelineEntityType.YARN_APPLICATION_ATTEMPT + "/"; + + File entityFolder = new File(outputDirApp); + Assert.assertTrue(entityFolder.isDirectory()); + + // file name is .thist + String timelineServiceFileName = + appAttemptId.toString() + + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION; + File appFile = new File(outputDirApp, timelineServiceFileName); + Assert.assertTrue(appFile.exists()); + verifyEntity(appFile, 2, AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE, + 0, TimelineServiceHelper.invertLong(appAttemptId.getAttemptId())); + } + + @Test(timeout = 10000) + public void testPublishContainerMetrics() throws Exception { + ApplicationId appId = ApplicationId.newInstance(0, 1); + RMApp app = rmAppsMapInContext.get(appId); + if (app == null) { + app = createAppAndRegister(appId); + } + ContainerId containerId = + ContainerId.newContainerId(ApplicationAttemptId.newInstance( + appId, 1), 1); + RMContainer container = createRMContainer(containerId); + metricsPublisher.containerCreated(container, container.getCreationTime()); + metricsPublisher.containerFinished(container, container.getFinishTime()); + dispatcher.await(); + + String outputDirApp = + getTimelineEntityDir(app) + "/" + + TimelineEntityType.YARN_CONTAINER + "/"; + + File entityFolder = new File(outputDirApp); + Assert.assertTrue(entityFolder.isDirectory()); + + // file name is .thist + String timelineServiceFileName = + containerId.toString() + + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION; + File appFile = new File(outputDirApp, timelineServiceFileName); + Assert.assertTrue(appFile.exists()); + verifyEntity(appFile, 2, + ContainerMetricsConstants.CREATED_IN_RM_EVENT_TYPE, 0, 0); + } + + private RMApp createAppAndRegister(ApplicationId appId) { + RMApp app = createRMApp(appId); + + // some stuff which are currently taken care in RMAppImpl + rmAppsMapInContext.putIfAbsent(appId, app); + AppLevelTimelineCollector collector = new AppLevelTimelineCollector(appId); + rmTimelineCollectorManager.putIfAbsent(appId, collector); + return app; + } + + private static void verifyEntity(File entityFile, long expectedEvents, + String eventForCreatedTime, long expectedMetrics, long idPrefix) + throws IOException { + BufferedReader reader = null; + String strLine; + long count = 0; + long metricsCount = 0; + try { + reader = new BufferedReader(new FileReader(entityFile)); + while ((strLine = reader.readLine()) != null) { + if (strLine.trim().length() > 0) { + TimelineEntity entity = FileSystemTimelineReaderImpl. + getTimelineRecordFromJSON(strLine.trim(), TimelineEntity.class); + metricsCount = entity.getMetrics().size(); + assertEquals(idPrefix, entity.getIdPrefix()); + for (TimelineEvent event : entity.getEvents()) { + if (event.getId().equals(eventForCreatedTime)) { + assertTrue(entity.getCreatedTime() > 0); + break; + } + } + count++; + } + } + } finally { + reader.close(); + } + assertEquals("Expected " + expectedEvents + " events to be published", + expectedEvents, count); + assertEquals("Expected " + expectedMetrics + " metrics is incorrect", + expectedMetrics, metricsCount); + } + + private String getTimelineEntityDir(RMApp app) { + String outputDirApp = + testRootDir.getAbsolutePath() + "/" + + FileSystemTimelineWriterImpl.ENTITIES_DIR + "/" + + YarnConfiguration.DEFAULT_RM_CLUSTER_ID + "/" + + app.getUser() + "/" + + app.getName() + "/" + + TimelineUtils.DEFAULT_FLOW_VERSION + "/" + + app.getStartTime() + "/" + + app.getApplicationId(); + return outputDirApp; + } + + private static RMApp createRMApp(ApplicationId appId) { + RMApp app = mock(RMAppImpl.class); + when(app.getApplicationId()).thenReturn(appId); + when(app.getName()).thenReturn("test app"); + when(app.getApplicationType()).thenReturn("test app type"); + when(app.getUser()).thenReturn("testUser"); + when(app.getQueue()).thenReturn("test queue"); + when(app.getSubmitTime()).thenReturn(Integer.MAX_VALUE + 1L); + when(app.getStartTime()).thenReturn(Integer.MAX_VALUE + 2L); + when(app.getFinishTime()).thenReturn(Integer.MAX_VALUE + 3L); + when(app.getDiagnostics()).thenReturn( + new StringBuilder("test diagnostics info")); + RMAppAttempt appAttempt = mock(RMAppAttempt.class); + when(appAttempt.getAppAttemptId()).thenReturn( + ApplicationAttemptId.newInstance(appId, 1)); + when(app.getCurrentAppAttempt()).thenReturn(appAttempt); + when(app.getFinalApplicationStatus()).thenReturn( + FinalApplicationStatus.UNDEFINED); + when(app.getRMAppMetrics()).thenReturn( + new RMAppMetrics(Resource.newInstance(0, 0), 0, 0, Integer.MAX_VALUE, + Long.MAX_VALUE, Long.MAX_VALUE, Long.MAX_VALUE)); + when(app.getApplicationTags()).thenReturn(Collections. emptySet()); + ApplicationSubmissionContext appSubmissionContext = + mock(ApplicationSubmissionContext.class); + when(appSubmissionContext.getPriority()) + .thenReturn(Priority.newInstance(0)); + + ContainerLaunchContext containerLaunchContext = + mock(ContainerLaunchContext.class); + when(containerLaunchContext.getCommands()) + .thenReturn(Collections.singletonList("java -Xmx1024m")); + when(appSubmissionContext.getAMContainerSpec()) + .thenReturn(containerLaunchContext); + when(app.getApplicationPriority()).thenReturn(Priority.newInstance(10)); + + when(app.getApplicationSubmissionContext()) + .thenReturn(appSubmissionContext); + return app; + } + + private static RMAppAttempt createRMAppAttempt( + ApplicationAttemptId appAttemptId) { + RMAppAttempt appAttempt = mock(RMAppAttempt.class); + when(appAttempt.getAppAttemptId()).thenReturn(appAttemptId); + when(appAttempt.getHost()).thenReturn("test host"); + when(appAttempt.getRpcPort()).thenReturn(-100); + Container container = mock(Container.class); + when(container.getId()).thenReturn( + ContainerId.newContainerId(appAttemptId, 1)); + when(appAttempt.getMasterContainer()).thenReturn(container); + when(appAttempt.getDiagnostics()).thenReturn("test diagnostics info"); + when(appAttempt.getTrackingUrl()).thenReturn("test tracking url"); + when(appAttempt.getOriginalTrackingUrl()).thenReturn( + "test original tracking url"); + when(appAttempt.getStartTime()).thenReturn(200L); + return appAttempt; + } + + private static RMContainer createRMContainer(ContainerId containerId) { + RMContainer container = mock(RMContainer.class); + when(container.getContainerId()).thenReturn(containerId); + when(container.getAllocatedNode()).thenReturn( + NodeId.newInstance("test host", -100)); + when(container.getAllocatedResource()).thenReturn( + Resource.newInstance(-1, -1)); + when(container.getAllocatedPriority()).thenReturn(Priority.UNDEFINED); + when(container.getCreationTime()).thenReturn(Integer.MAX_VALUE + 1L); + when(container.getFinishTime()).thenReturn(Integer.MAX_VALUE + 2L); + when(container.getDiagnosticsInfo()).thenReturn("test diagnostics info"); + when(container.getContainerExitStatus()).thenReturn(-1); + when(container.getContainerState()).thenReturn(ContainerState.COMPLETE); + Container mockContainer = mock(Container.class); + when(container.getContainer()).thenReturn(mockContainer); + when(mockContainer.getNodeHttpAddress()) + .thenReturn("http://localhost:1234"); + return container; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java index d3639085977..17cafefc65f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java @@ -32,6 +32,7 @@ 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.ApplicationTimeoutType; +import org.apache.hadoop.yarn.api.records.CollectorInfo; import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; import org.apache.hadoop.yarn.api.records.LogAggregationStatus; import org.apache.hadoop.yarn.api.records.NodeId; @@ -43,6 +44,7 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport; +import org.apache.hadoop.yarn.server.api.records.AppCollectorData; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; @@ -319,4 +321,14 @@ public class MockRMApp implements RMApp { public boolean isAppInCompletedStates() { return false; } + + @Override + public AppCollectorData getCollectorData() { + throw new UnsupportedOperationException("Not supported yet."); + } + + @Override + public CollectorInfo getCollectorInfo() { + throw new UnsupportedOperationException("Not supported yet."); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java index e9a74deedc5..344f8bb7de0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java @@ -100,6 +100,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSec import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer; import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM; import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager; +import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager; import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.Records; import org.junit.Assert; @@ -245,6 +246,9 @@ public class TestRMAppTransitions { .getAppResourceUsageReport((ApplicationAttemptId)Matchers.any()); doReturn(resourceScheduler).when(rmContext).getScheduler(); + doReturn(mock(RMTimelineCollectorManager.class)).when(rmContext) + .getRMTimelineCollectorManager(); + rmDispatcher.register(RMAppAttemptEventType.class, new TestApplicationAttemptEventDispatcher(this.rmContext)); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/pom.xml index f1fda86880b..ca7b5fd3828 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/pom.xml @@ -56,6 +56,11 @@ junit test + + org.mockito + mockito-all + test + org.apache.hadoop @@ -86,6 +91,18 @@ test + + org.apache.hadoop + hadoop-yarn-server-timelineservice + + + + org.apache.hadoop + hadoop-yarn-server-timelineservice + test-jar + test + + org.apache.hadoop hadoop-minikdc @@ -109,6 +126,17 @@ commons-logging commons-logging + + org.bouncycastle + bcprov-jdk16 + test + + + org.apache.hadoop + hadoop-auth + test + test-jar + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java index 0fe57519bc6..de282fd0631 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java @@ -39,6 +39,7 @@ import org.apache.hadoop.ha.HAServiceProtocol; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.net.ServerSocketUtil; import org.apache.hadoop.service.AbstractService; import org.apache.hadoop.service.CompositeService; import org.apache.hadoop.util.Shell; @@ -557,6 +558,8 @@ public class MiniYARNCluster extends CompositeService { MiniYARNCluster.getHostname() + ":0"); config.set(YarnConfiguration.NM_LOCALIZER_ADDRESS, MiniYARNCluster.getHostname() + ":0"); + config.set(YarnConfiguration.NM_COLLECTOR_SERVICE_ADDRESS, + MiniYARNCluster.getHostname() + ":0"); WebAppUtils .setNMWebAppHostNameAndPort(config, MiniYARNCluster.getHostname(), 0); @@ -799,8 +802,8 @@ public class MiniYARNCluster extends CompositeService { if (!useFixedPorts) { String hostname = MiniYARNCluster.getHostname(); conf.set(YarnConfiguration.TIMELINE_SERVICE_ADDRESS, hostname + ":0"); - conf.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS, hostname - + ":0"); + conf.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS, + hostname + ":" + ServerSocketUtil.getPort(9188, 10)); } appHistoryServer.init(conf); super.serviceInit(conf); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestMiniYarnCluster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestMiniYarnCluster.java index 81b44427d71..ff7fafc2001 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestMiniYarnCluster.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestMiniYarnCluster.java @@ -71,8 +71,6 @@ public class TestMiniYarnCluster { String hostname = MiniYARNCluster.getHostname(); Assert.assertEquals(hostname + ":0", conf.get(YarnConfiguration.TIMELINE_SERVICE_ADDRESS)); - Assert.assertEquals(hostname + ":0", - conf.get(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS)); cluster.start(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestRMNMSecretKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestRMNMSecretKeys.java index 8cc2dee43ac..de3b0d7e6b6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestRMNMSecretKeys.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestRMNMSecretKeys.java @@ -18,10 +18,17 @@ package org.apache.hadoop.yarn.server; +import java.io.File; +import java.io.FileOutputStream; import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.charset.StandardCharsets; +import java.util.UUID; +import org.junit.AfterClass; import org.junit.Assert; - +import org.junit.BeforeClass; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.conf.YarnConfiguration; @@ -35,6 +42,44 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; import org.junit.Test; public class TestRMNMSecretKeys { + private static final String KRB5_CONF = "java.security.krb5.conf"; + private static final File KRB5_CONF_ROOT_DIR = new File( + System.getProperty("test.build.dir", "target/test-dir"), + UUID.randomUUID().toString()); + + @BeforeClass + public static void setup() throws IOException { + KRB5_CONF_ROOT_DIR.mkdir(); + File krb5ConfFile = new File(KRB5_CONF_ROOT_DIR, "krb5.conf"); + krb5ConfFile.createNewFile(); + String content = "[libdefaults]\n" + + " default_realm = APACHE.ORG\n" + + " udp_preference_limit = 1\n"+ + " extra_addresses = 127.0.0.1\n" + + "[realms]\n" + + " APACHE.ORG = {\n" + + " admin_server = localhost:88\n" + + " kdc = localhost:88\n}\n" + + "[domain_realm]\n" + + " localhost = APACHE.ORG"; + writeFile(content, krb5ConfFile); + System.setProperty(KRB5_CONF, krb5ConfFile.getAbsolutePath()); + } + + private static void writeFile(String content, File file) throws IOException { + FileOutputStream outputStream = new FileOutputStream(file); + FileChannel fc = outputStream.getChannel(); + + ByteBuffer buffer = + ByteBuffer.wrap(content.getBytes(StandardCharsets.UTF_8)); + fc.write(buffer); + outputStream.close(); + } + + @AfterClass + public static void tearDown() throws IOException { + KRB5_CONF_ROOT_DIR.delete(); + } @Test(timeout = 1000000) public void testNMUpdation() throws Exception { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java new file mode 100644 index 00000000000..8df2fcb5916 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java @@ -0,0 +1,192 @@ +/** + * 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.timelineservice; + + +import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.io.File; +import java.io.IOException; + +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.util.ExitUtil; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.CollectorInfo; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationAttemptEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.ClusterEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.ContainerEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.QueueEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.api.records.timelineservice.UserEntity; +import org.apache.hadoop.yarn.client.api.TimelineV2Client; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocol; +import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextResponse; +import org.apache.hadoop.yarn.server.timelineservice.collector.NodeTimelineCollectorManager; +import org.apache.hadoop.yarn.server.timelineservice.collector.PerNodeTimelineCollectorsAuxService; +import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +/** + * Tests timeline client integration with collector service. + */ +public class TestTimelineServiceClientIntegration { + private static final String ROOT_DIR = new File("target", + TestTimelineServiceClientIntegration.class.getSimpleName()). + getAbsolutePath(); + private static NodeTimelineCollectorManager collectorManager; + private static PerNodeTimelineCollectorsAuxService auxService; + private static Configuration conf; + + @BeforeClass + public static void setupClass() throws Exception { + try { + collectorManager = new MockNodeTimelineCollectorManager(); + conf = new YarnConfiguration(); + // enable timeline service v.2 + conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); + conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f); + conf.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS, + FileSystemTimelineWriterImpl.class, TimelineWriter.class); + conf.set(FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_DIR_ROOT, + ROOT_DIR); + auxService = + PerNodeTimelineCollectorsAuxService.launchServer(new String[0], + collectorManager, conf); + auxService.addApplication(ApplicationId.newInstance(0, 1), "user"); + } catch (ExitUtil.ExitException e) { + fail(); + } + } + + @AfterClass + public static void tearDownClass() throws Exception { + if (auxService != null) { + auxService.stop(); + } + FileUtils.deleteDirectory(new File(ROOT_DIR)); + } + + @Test + public void testPutEntities() throws Exception { + TimelineV2Client client = + TimelineV2Client.createTimelineClient(ApplicationId.newInstance(0, 1)); + try { + // Set the timeline service address manually. + client.setTimelineCollectorInfo(CollectorInfo.newInstance( + collectorManager.getRestServerBindAddress())); + client.init(conf); + client.start(); + TimelineEntity entity = new TimelineEntity(); + entity.setType("test entity type"); + entity.setId("test entity id"); + TimelineMetric metric = + new TimelineMetric(TimelineMetric.Type.TIME_SERIES); + metric.setId("test metric id"); + metric.addValue(1L, 1.0D); + metric.addValue(2L, 2.0D); + entity.addMetric(metric); + client.putEntities(entity); + client.putEntitiesAsync(entity); + } finally { + client.stop(); + } + } + + @Test + public void testPutExtendedEntities() throws Exception { + ApplicationId appId = ApplicationId.newInstance(0, 1); + TimelineV2Client client = + TimelineV2Client.createTimelineClient(appId); + try { + // Set the timeline service address manually. + client.setTimelineCollectorInfo(CollectorInfo.newInstance( + collectorManager.getRestServerBindAddress())); + client.init(conf); + client.start(); + ClusterEntity cluster = new ClusterEntity(); + cluster.setId(YarnConfiguration.DEFAULT_RM_CLUSTER_ID); + FlowRunEntity flow = new FlowRunEntity(); + flow.setUser(UserGroupInformation.getCurrentUser().getShortUserName()); + flow.setName("test_flow_name"); + flow.setVersion("test_flow_version"); + flow.setRunId(1L); + flow.setParent(cluster.getType(), cluster.getId()); + ApplicationEntity app = new ApplicationEntity(); + app.setId(appId.toString()); + flow.addChild(app.getType(), app.getId()); + ApplicationAttemptId attemptId = + ApplicationAttemptId.newInstance(appId, 1); + ApplicationAttemptEntity appAttempt = new ApplicationAttemptEntity(); + appAttempt.setId(attemptId.toString()); + ContainerId containerId = ContainerId.newContainerId(attemptId, 1); + ContainerEntity container = new ContainerEntity(); + container.setId(containerId.toString()); + UserEntity user = new UserEntity(); + user.setId(UserGroupInformation.getCurrentUser().getShortUserName()); + QueueEntity queue = new QueueEntity(); + queue.setId("default_queue"); + client.putEntities(cluster, flow, app, appAttempt, container, user, + queue); + client.putEntitiesAsync(cluster, flow, app, appAttempt, container, user, + queue); + } finally { + client.stop(); + } + } + + private static class MockNodeTimelineCollectorManager extends + NodeTimelineCollectorManager { + public MockNodeTimelineCollectorManager() { + super(); + } + + @Override + protected CollectorNodemanagerProtocol getNMCollectorService() { + CollectorNodemanagerProtocol protocol = + mock(CollectorNodemanagerProtocol.class); + try { + GetTimelineCollectorContextResponse response = + GetTimelineCollectorContextResponse.newInstance( + UserGroupInformation.getCurrentUser().getShortUserName(), + "test_flow_name", "test_flow_version", 1L); + when(protocol.getTimelineCollectorContext(any( + GetTimelineCollectorContextRequest.class))).thenReturn(response); + } catch (YarnException | IOException e) { + fail(); + } + return protocol; + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/security/TestTimelineAuthFilterForV2.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/security/TestTimelineAuthFilterForV2.java new file mode 100644 index 00000000000..8715e46fc5e --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/security/TestTimelineAuthFilterForV2.java @@ -0,0 +1,478 @@ +/** + * 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.timelineservice.security; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.UUID; +import java.util.concurrent.Callable; + +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.http.HttpConfig; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.minikdc.MiniKdc; +import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.authentication.KerberosTestUtils; +import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler; +import org.apache.hadoop.security.ssl.KeyStoreTestUtil; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.CollectorInfo; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.client.api.TimelineV2Client; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier; +import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocol; +import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextResponse; +import org.apache.hadoop.yarn.server.timeline.security.TimelineAuthenticationFilterInitializer; +import org.apache.hadoop.yarn.server.timelineservice.collector.AppLevelTimelineCollector; +import org.apache.hadoop.yarn.server.timelineservice.collector.NodeTimelineCollectorManager; +import org.apache.hadoop.yarn.server.timelineservice.collector.PerNodeTimelineCollectorsAuxService; +import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineReaderImpl; +import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +/** + * Tests timeline authentication filter based security for timeline service v2. + */ +@RunWith(Parameterized.class) +public class TestTimelineAuthFilterForV2 { + + private static final String FOO_USER = "foo"; + private static final String HTTP_USER = "HTTP"; + private static final File TEST_ROOT_DIR = new File( + System.getProperty("test.build.dir", "target" + File.separator + + "test-dir"), UUID.randomUUID().toString()); + private static final String BASEDIR = + System.getProperty("test.build.dir", "target/test-dir") + "/" + + TestTimelineAuthFilterForV2.class.getSimpleName(); + private static File httpSpnegoKeytabFile = new File(KerberosTestUtils. + getKeytabFile()); + private static String httpSpnegoPrincipal = KerberosTestUtils. + getServerPrincipal(); + + // First param indicates whether HTTPS access or HTTP access and second param + // indicates whether it is kerberos access or token based access. + @Parameterized.Parameters + public static Collection params() { + return Arrays.asList(new Object[][] {{false, true}, {false, false}, + {true, false}, {true, true}}); + } + + private static MiniKdc testMiniKDC; + private static String keystoresDir; + private static String sslConfDir; + private static Configuration conf; + private static UserGroupInformation nonKerberosUser; + static { + try { + nonKerberosUser = UserGroupInformation.getCurrentUser(); + } catch (IOException e) {} + } + // Indicates whether HTTPS or HTTP access. + private boolean withSsl; + // Indicates whether Kerberos based login is used or token based access is + // done. + private boolean withKerberosLogin; + private NodeTimelineCollectorManager collectorManager; + private PerNodeTimelineCollectorsAuxService auxService; + public TestTimelineAuthFilterForV2(boolean withSsl, + boolean withKerberosLogin) { + this.withSsl = withSsl; + this.withKerberosLogin = withKerberosLogin; + } + + @BeforeClass + public static void setup() { + try { + testMiniKDC = new MiniKdc(MiniKdc.createConf(), TEST_ROOT_DIR); + testMiniKDC.start(); + testMiniKDC.createPrincipal( + httpSpnegoKeytabFile, HTTP_USER + "/localhost"); + } catch (Exception e) { + fail("Couldn't setup MiniKDC."); + } + + // Setup timeline service v2. + try { + conf = new Configuration(false); + conf.setStrings(TimelineAuthenticationFilterInitializer.PREFIX + "type", + "kerberos"); + conf.set(TimelineAuthenticationFilterInitializer.PREFIX + + KerberosAuthenticationHandler.PRINCIPAL, httpSpnegoPrincipal); + conf.set(TimelineAuthenticationFilterInitializer.PREFIX + + KerberosAuthenticationHandler.KEYTAB, + httpSpnegoKeytabFile.getAbsolutePath()); + conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, + "kerberos"); + conf.set(YarnConfiguration.TIMELINE_SERVICE_PRINCIPAL, + httpSpnegoPrincipal); + conf.set(YarnConfiguration.TIMELINE_SERVICE_KEYTAB, + httpSpnegoKeytabFile.getAbsolutePath()); + // Enable timeline service v2 + conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); + conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f); + conf.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS, + FileSystemTimelineWriterImpl.class, TimelineWriter.class); + conf.set(YarnConfiguration.TIMELINE_SERVICE_BIND_HOST, "localhost"); + conf.set(FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_DIR_ROOT, + TEST_ROOT_DIR.getAbsolutePath()); + conf.set("hadoop.proxyuser.HTTP.hosts", "*"); + conf.set("hadoop.proxyuser.HTTP.users", FOO_USER); + UserGroupInformation.setConfiguration(conf); + } catch (Exception e) { + fail("Couldn't setup TimelineServer V2."); + } + } + + @Before + public void initialize() throws Exception { + if (withSsl) { + conf.set(YarnConfiguration.YARN_HTTP_POLICY_KEY, + HttpConfig.Policy.HTTPS_ONLY.name()); + File base = new File(BASEDIR); + FileUtil.fullyDelete(base); + base.mkdirs(); + keystoresDir = new File(BASEDIR).getAbsolutePath(); + sslConfDir = + KeyStoreTestUtil.getClasspathDir(TestTimelineAuthFilterForV2.class); + KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, conf, false); + } else { + conf.set(YarnConfiguration.YARN_HTTP_POLICY_KEY, + HttpConfig.Policy.HTTP_ONLY.name()); + } + if (!withKerberosLogin) { + // For timeline delegation token based access, set delegation token renew + // interval to 100 ms. to test if timeline delegation token for the app is + // renewed automatically if app is still alive. + conf.setLong( + YarnConfiguration.TIMELINE_DELEGATION_TOKEN_RENEW_INTERVAL, 100); + // Set token max lifetime to 4 seconds to test if timeline delegation + // token for the app is regenerated automatically if app is still alive. + conf.setLong( + YarnConfiguration.TIMELINE_DELEGATION_TOKEN_MAX_LIFETIME, 4000); + } + UserGroupInformation.setConfiguration(conf); + collectorManager = new DummyNodeTimelineCollectorManager(); + auxService = PerNodeTimelineCollectorsAuxService.launchServer( + new String[0], collectorManager, conf); + if (withKerberosLogin) { + SecurityUtil.login(conf, YarnConfiguration.TIMELINE_SERVICE_KEYTAB, + YarnConfiguration.TIMELINE_SERVICE_PRINCIPAL, "localhost"); + } + ApplicationId appId = ApplicationId.newInstance(0, 1); + auxService.addApplication( + appId, UserGroupInformation.getCurrentUser().getUserName()); + if (!withKerberosLogin) { + AppLevelTimelineCollector collector = + (AppLevelTimelineCollector)collectorManager.get(appId); + Token token = + collector.getDelegationTokenForApp(); + token.setService(new Text("localhost" + token.getService().toString(). + substring(token.getService().toString().indexOf(":")))); + UserGroupInformation.getCurrentUser().addToken(token); + } + } + + private TimelineV2Client createTimelineClientForUGI(ApplicationId appId) { + TimelineV2Client client = + TimelineV2Client.createTimelineClient(ApplicationId.newInstance(0, 1)); + // set the timeline service address. + String restBindAddr = collectorManager.getRestServerBindAddress(); + String addr = + "localhost" + restBindAddr.substring(restBindAddr.indexOf(":")); + client.setTimelineCollectorInfo(CollectorInfo.newInstance(addr)); + client.init(conf); + client.start(); + return client; + } + + @AfterClass + public static void tearDown() throws Exception { + if (testMiniKDC != null) { + testMiniKDC.stop(); + } + FileUtil.fullyDelete(TEST_ROOT_DIR); + } + + @After + public void destroy() throws Exception { + if (auxService != null) { + auxService.stop(); + } + if (withSsl) { + KeyStoreTestUtil.cleanupSSLConfig(keystoresDir, sslConfDir); + FileUtil.fullyDelete(new File(BASEDIR)); + } + if (withKerberosLogin) { + UserGroupInformation.getCurrentUser().logoutUserFromKeytab(); + } + // Reset the user for next run. + UserGroupInformation.setLoginUser( + UserGroupInformation.createRemoteUser(nonKerberosUser.getUserName())); + } + + private static TimelineEntity createEntity(String id, String type) { + TimelineEntity entityToStore = new TimelineEntity(); + entityToStore.setId(id); + entityToStore.setType(type); + entityToStore.setCreatedTime(0L); + return entityToStore; + } + + private static void verifyEntity(File entityTypeDir, String id, String type) + throws IOException { + File entityFile = new File(entityTypeDir, id + + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION); + assertTrue(entityFile.exists()); + TimelineEntity entity = readEntityFile(entityFile); + assertNotNull(entity); + assertEquals(id, entity.getId()); + assertEquals(type, entity.getType()); + } + + private static TimelineEntity readEntityFile(File entityFile) + throws IOException { + BufferedReader reader = null; + String strLine; + try { + reader = new BufferedReader(new FileReader(entityFile)); + while ((strLine = reader.readLine()) != null) { + if (strLine.trim().length() > 0) { + return FileSystemTimelineReaderImpl. + getTimelineRecordFromJSON(strLine.trim(), TimelineEntity.class); + } + } + return null; + } finally { + reader.close(); + } + } + + private void publishAndVerifyEntity(ApplicationId appId, File entityTypeDir, + String entityType, int numEntities) throws Exception { + TimelineV2Client client = createTimelineClientForUGI(appId); + try { + // Sync call. Results available immediately. + client.putEntities(createEntity("entity1", entityType)); + assertEquals(numEntities, entityTypeDir.listFiles().length); + verifyEntity(entityTypeDir, "entity1", entityType); + // Async call. + client.putEntitiesAsync(createEntity("entity2", entityType)); + } finally { + client.stop(); + } + } + + private boolean publishWithRetries(ApplicationId appId, File entityTypeDir, + String entityType, int numEntities) throws Exception { + for (int i = 0; i < 10; i++) { + try { + publishAndVerifyEntity(appId, entityTypeDir, entityType, numEntities); + } catch (YarnException e) { + Thread.sleep(50); + continue; + } + return true; + } + return false; + } + + @Test + public void testPutTimelineEntities() throws Exception { + final String entityType = "dummy_type"; + final ApplicationId appId = ApplicationId.newInstance(0, 1); + final File entityTypeDir = new File(TEST_ROOT_DIR.getAbsolutePath() + + File.separator + "entities" + File.separator + + YarnConfiguration.DEFAULT_RM_CLUSTER_ID + File.separator + + UserGroupInformation.getCurrentUser().getUserName() + + File.separator + "test_flow_name" + File.separator + + "test_flow_version" + File.separator + "1" + File.separator + + appId.toString() + File.separator + entityType); + try { + if (withKerberosLogin) { + KerberosTestUtils.doAs(HTTP_USER + "/localhost", new Callable() { + @Override + public Void call() throws Exception { + publishAndVerifyEntity(appId, entityTypeDir, entityType, 1); + return null; + } + }); + } else { + assertTrue("Entities should have been published successfully.", + publishWithRetries(appId, entityTypeDir, entityType, 1)); + + AppLevelTimelineCollector collector = + (AppLevelTimelineCollector) collectorManager.get(appId); + Token token = + collector.getDelegationTokenForApp(); + assertNotNull(token); + + // Verify if token is renewed automatically and entities can still be + // published. + Thread.sleep(1000); + // Entities should publish successfully after renewal. + assertTrue("Entities should have been published successfully.", + publishWithRetries(appId, entityTypeDir, entityType, 2)); + assertNotNull(collector); + verify(collectorManager.getTokenManagerService(), atLeastOnce()). + renewToken(eq(collector.getDelegationTokenForApp()), + any(String.class)); + + // Wait to ensure lifetime of token expires and ensure its regenerated + // automatically. + Thread.sleep(3000); + for (int i = 0; i < 40; i++) { + if (!token.equals(collector.getDelegationTokenForApp())) { + break; + } + Thread.sleep(50); + } + assertNotEquals("Token should have been regenerated.", token, + collector.getDelegationTokenForApp()); + Thread.sleep(1000); + // Try publishing with the old token in UGI. Publishing should fail due + // to invalid token. + try { + publishAndVerifyEntity(appId, entityTypeDir, entityType, 2); + fail("Exception should have been thrown due to Invalid Token."); + } catch (YarnException e) { + assertTrue("Exception thrown should have been due to Invalid Token.", + e.getCause().getMessage().contains("InvalidToken")); + } + + // Update the regenerated token in UGI and retry publishing entities. + Token regeneratedToken = + collector.getDelegationTokenForApp(); + regeneratedToken.setService(new Text("localhost" + + regeneratedToken.getService().toString().substring( + regeneratedToken.getService().toString().indexOf(":")))); + UserGroupInformation.getCurrentUser().addToken(regeneratedToken); + assertTrue("Entities should have been published successfully.", + publishWithRetries(appId, entityTypeDir, entityType, 2)); + // Token was generated twice, once when app collector was created and + // later after token lifetime expiry. + verify(collectorManager.getTokenManagerService(), times(2)). + generateToken(any(UserGroupInformation.class), any(String.class)); + assertEquals(1, ((DummyNodeTimelineCollectorManager) collectorManager). + getTokenExpiredCnt()); + } + // Wait for async entity to be published. + for (int i = 0; i < 50; i++) { + if (entityTypeDir.listFiles().length == 2) { + break; + } + Thread.sleep(50); + } + assertEquals(2, entityTypeDir.listFiles().length); + verifyEntity(entityTypeDir, "entity2", entityType); + AppLevelTimelineCollector collector = + (AppLevelTimelineCollector)collectorManager.get(appId); + assertNotNull(collector); + auxService.removeApplication(appId); + verify(collectorManager.getTokenManagerService()).cancelToken( + eq(collector.getDelegationTokenForApp()), any(String.class)); + } finally { + FileUtils.deleteQuietly(entityTypeDir); + } + } + + private static class DummyNodeTimelineCollectorManager extends + NodeTimelineCollectorManager { + private volatile int tokenExpiredCnt = 0; + DummyNodeTimelineCollectorManager() { + super(); + } + + private int getTokenExpiredCnt() { + return tokenExpiredCnt; + } + + @Override + protected TimelineV2DelegationTokenSecretManagerService + createTokenManagerService() { + return spy(new TimelineV2DelegationTokenSecretManagerService() { + @Override + protected AbstractDelegationTokenSecretManager + + createTimelineDelegationTokenSecretManager(long secretKeyInterval, + long tokenMaxLifetime, long tokenRenewInterval, + long tokenRemovalScanInterval) { + return spy(new TimelineV2DelegationTokenSecretManager( + secretKeyInterval, tokenMaxLifetime, tokenRenewInterval, 2000L) { + @Override + protected void logExpireToken( + TimelineDelegationTokenIdentifier ident) throws IOException { + tokenExpiredCnt++; + } + }); + } + }); + } + + @Override + protected CollectorNodemanagerProtocol getNMCollectorService() { + CollectorNodemanagerProtocol protocol = + mock(CollectorNodemanagerProtocol.class); + try { + GetTimelineCollectorContextResponse response = + GetTimelineCollectorContextResponse.newInstance( + UserGroupInformation.getCurrentUser().getUserName(), + "test_flow_name", "test_flow_version", 1L); + when(protocol.getTimelineCollectorContext(any( + GetTimelineCollectorContextRequest.class))).thenReturn(response); + } catch (YarnException | IOException e) { + fail(); + } + return protocol; + } + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/resources/krb5.conf b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/resources/krb5.conf deleted file mode 100644 index 121ac6d9b98..00000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/resources/krb5.conf +++ /dev/null @@ -1,28 +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. -# -[libdefaults] - default_realm = APACHE.ORG - udp_preference_limit = 1 - extra_addresses = 127.0.0.1 -[realms] - APACHE.ORG = { - admin_server = localhost:88 - kdc = localhost:88 - } -[domain_realm] - localhost = APACHE.ORG diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml new file mode 100644 index 00000000000..7bcf764242a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml @@ -0,0 +1,355 @@ + + + + + hadoop-yarn-server + org.apache.hadoop + 2.9.0-SNAPSHOT + + 4.0.0 + org.apache.hadoop + hadoop-yarn-server-timelineservice-hbase-tests + 2.9.0-SNAPSHOT + Apache Hadoop YARN Timeline Service HBase tests + + + + ${project.parent.parent.basedir} + + + + + junit + junit + test + + + + + org.apache.hadoop + hadoop-yarn-server-timelineservice + test + + + org.apache.hadoop + hadoop-common + + + + + + org.apache.hadoop + hadoop-yarn-server-timelineservice-hbase + test + + + org.apache.hadoop + hadoop-common + + + + + + org.apache.hadoop + hadoop-common + ${hbase-compatible-hadoop.version} + test + + + org.apache.hadoop + hadoop-auth + + + + + + + org.apache.hadoop + hadoop-auth + ${hbase-compatible-hadoop.version} + test + + + + org.apache.hadoop + hadoop-yarn-api + test + + + org.apache.hadoop + hadoop-common + + + + + + org.apache.hadoop + hadoop-yarn-common + test + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-auth + + + + + + org.apache.hadoop + hadoop-yarn-server-applicationhistoryservice + test + + + org.apache.hadoop + hadoop-common + + + + + + com.google.guava + guava + test + + + + com.sun.jersey + jersey-client + test + + + + javax.ws.rs + jsr311-api + 1.1.1 + + + + org.apache.hbase + hbase-common + test + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + + + + org.apache.hbase + hbase-client + test + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-auth + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + + + + org.apache.hbase + hbase-server + test + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-auth + + + org.apache.hadoop + hadoop-hdfs + + + org.apache.hadoop + hadoop-hdfs-client + + + org.apache.hadoop + hadoop-client + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + + + + org.apache.hbase + hbase-server + tests + test + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-auth + + + org.apache.hadoop + hadoop-hdfs + + + org.apache.hadoop + hadoop-hdfs-client + + + org.apache.hadoop + hadoop-client + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + + + + + org.apache.hadoop + hadoop-common + ${hbase-compatible-hadoop.version} + test-jar + test + + + org.apache.hadoop + hadoop-auth + + + + + + + org.apache.hadoop + hadoop-hdfs + ${hbase-compatible-hadoop.version} + test + + + + + org.apache.hadoop + hadoop-hdfs + ${hbase-compatible-hadoop.version} + test-jar + test + + + + org.apache.hbase + hbase-testing-util + test + true + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-auth + + + org.apache.hadoop + hadoop-client + + + org.apache.hadoop + hadoop-mapreduce-client-jobclient + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + + + + + + + org.codehaus.mojo + findbugs-maven-plugin + + true + + + + maven-jar-plugin + + + + test-jar + + test-compile + + + + + org.apache.maven.plugins + maven-javadoc-plugin + + + + junit + junit + 4.11 + + + + + + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/AbstractTimelineReaderHBaseTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/AbstractTimelineReaderHBaseTestBase.java new file mode 100644 index 00000000000..f9daf7680a2 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/AbstractTimelineReaderHBaseTestBase.java @@ -0,0 +1,174 @@ +/** + * 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.timelineservice.reader; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.lang.reflect.UndeclaredThrowableException; +import java.net.HttpURLConnection; +import java.net.URI; +import java.net.URL; +import java.util.List; + +import javax.ws.rs.core.MediaType; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.timelineservice.storage.DataGeneratorForTest; +import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider; +import org.junit.Assert; + +import com.sun.jersey.api.client.Client; +import com.sun.jersey.api.client.ClientResponse; +import com.sun.jersey.api.client.ClientResponse.Status; +import com.sun.jersey.api.client.GenericType; +import com.sun.jersey.api.client.config.ClientConfig; +import com.sun.jersey.api.client.config.DefaultClientConfig; +import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory; +import com.sun.jersey.client.urlconnection.URLConnectionClientHandler; + +/** + * Test Base for TimelineReaderServer HBase tests. + */ +public abstract class AbstractTimelineReaderHBaseTestBase { + private static int serverPort; + private static TimelineReaderServer server; + private static HBaseTestingUtility util; + + public static void setup() throws Exception { + util = new HBaseTestingUtility(); + Configuration conf = util.getConfiguration(); + conf.setInt("hfile.format.version", 3); + util.startMiniCluster(); + DataGeneratorForTest.createSchema(util.getConfiguration()); + } + + public static void tearDown() throws Exception { + if (server != null) { + server.stop(); + server = null; + } + if (util != null) { + util.shutdownMiniCluster(); + } + } + + protected static void initialize() throws Exception { + try { + Configuration config = util.getConfiguration(); + config.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); + config.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f); + config.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS, + "localhost:0"); + config.set(YarnConfiguration.RM_CLUSTER_ID, "cluster1"); + config.set(YarnConfiguration.TIMELINE_SERVICE_READER_CLASS, + "org.apache.hadoop.yarn.server.timelineservice.storage." + + "HBaseTimelineReaderImpl"); + config.setInt("hfile.format.version", 3); + server = new TimelineReaderServer() { + @Override + protected void addFilters(Configuration conf) { + // The parent code uses hadoop-common jar from this version of + // Hadoop, but the tests are using hadoop-common jar from + // ${hbase-compatible-hadoop.version}. Between these versions, there + // are many differences, including classnames and packages. + // We do nothing here, so that we don't cause a NoSuchMethodError or + // NoClassDefFoundError. + } + }; + server.init(config); + server.start(); + serverPort = server.getWebServerPort(); + } catch (Exception e) { + Assert.fail("Web server failed to start"); + } + } + + protected Client createClient() { + ClientConfig cfg = new DefaultClientConfig(); + cfg.getClasses().add(YarnJacksonJaxbJsonProvider.class); + return new Client( + new URLConnectionClientHandler(new DummyURLConnectionFactory()), cfg); + } + + protected ClientResponse getResponse(Client client, URI uri) + throws Exception { + ClientResponse resp = + client.resource(uri).accept(MediaType.APPLICATION_JSON) + .type(MediaType.APPLICATION_JSON).get(ClientResponse.class); + if (resp == null || + resp.getClientResponseStatus() != ClientResponse.Status.OK) { + String msg = ""; + if (resp != null) { + msg = String.valueOf(resp.getClientResponseStatus()); + } + throw new IOException( + "Incorrect response from timeline reader. " + "Status=" + msg); + } + return resp; + } + + protected void verifyHttpResponse(Client client, URI uri, Status status) { + ClientResponse resp = + client.resource(uri).accept(MediaType.APPLICATION_JSON) + .type(MediaType.APPLICATION_JSON).get(ClientResponse.class); + assertNotNull(resp); + assertTrue("Response from server should have been " + status, + resp.getClientResponseStatus() == status); + System.out.println("Response is: " + resp.getEntity(String.class)); + } + + protected List verifyFlowEntites(Client client, URI uri, + int noOfEntities) throws Exception { + ClientResponse resp = getResponse(client, uri); + List entities = + resp.getEntity(new GenericType>() { + }); + assertNotNull(entities); + assertEquals(noOfEntities, entities.size()); + return entities; + } + + protected static class DummyURLConnectionFactory + implements HttpURLConnectionFactory { + + @Override + public HttpURLConnection getHttpURLConnection(final URL url) + throws IOException { + try { + return (HttpURLConnection) url.openConnection(); + } catch (UndeclaredThrowableException e) { + throw new IOException(e.getCause()); + } + } + } + + protected static HBaseTestingUtility getHBaseTestingUtility() { + return util; + } + + public static int getServerPort() { + return serverPort; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java new file mode 100644 index 00000000000..426096881aa --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java @@ -0,0 +1,2577 @@ +/** + * 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.timelineservice.reader; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.net.URI; +import java.text.DateFormat; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import javax.ws.rs.core.MediaType; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric.Type; +import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants; +import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext; +import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils; +import org.apache.hadoop.yarn.server.utils.BuilderUtils; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Sets; +import com.sun.jersey.api.client.Client; +import com.sun.jersey.api.client.ClientResponse; +import com.sun.jersey.api.client.ClientResponse.Status; +import com.sun.jersey.api.client.GenericType; + +/** + * Test TimelineReder Web Service REST API's using HBase storage. + */ +public class TestTimelineReaderWebServicesHBaseStorage + extends AbstractTimelineReaderHBaseTestBase { + private static long ts = System.currentTimeMillis(); + private static long dayTs = + HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(ts); + private static String doAsUser = "remoteuser"; + + @BeforeClass + public static void setupBeforeClass() throws Exception { + setup(); + loadData(); + initialize(); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + tearDown(); + } + + private static void loadData() throws Exception { + String cluster = "cluster1"; + String user = "user1"; + String flow = "flow_name"; + String flowVersion = "CF7022C10F1354"; + Long runid = 1002345678919L; + Long runid1 = 1002345678920L; + + TimelineEntities te = new TimelineEntities(); + TimelineEntity entity = new TimelineEntity(); + String id = "application_1111111111_1111"; + String type = TimelineEntityType.YARN_APPLICATION.toString(); + entity.setId(id); + entity.setType(type); + Long cTime = 1425016501000L; + entity.setCreatedTime(cTime); + entity.addConfig("cfg2", "value1"); + + // add metrics + Set metrics = new HashSet<>(); + TimelineMetric m1 = new TimelineMetric(); + m1.setId("MAP_SLOT_MILLIS"); + Map metricValues = + ImmutableMap.of(ts - 100000, (Number)2, ts - 90000, 7, ts - 80000, 40); + m1.setType(Type.TIME_SERIES); + m1.setValues(metricValues); + metrics.add(m1); + m1 = new TimelineMetric(); + m1.setId("MAP1_SLOT_MILLIS"); + metricValues = + ImmutableMap.of(ts - 100000, (Number)2, ts - 90000, 9, ts - 80000, 40); + m1.setType(Type.TIME_SERIES); + m1.setValues(metricValues); + metrics.add(m1); + m1 = new TimelineMetric(); + m1.setId("HDFS_BYTES_READ"); + metricValues = ImmutableMap.of(ts - 100000, (Number)31, ts - 80000, 57); + m1.setType(Type.TIME_SERIES); + m1.setValues(metricValues); + metrics.add(m1); + entity.addMetrics(metrics); + + TimelineEvent event = new TimelineEvent(); + event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE); + event.setTimestamp(cTime); + String expKey = "foo_event"; + Object expVal = "test"; + event.addInfo(expKey, expVal); + entity.addEvent(event); + TimelineEvent event11 = new TimelineEvent(); + event11.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE); + Long expTs = 1425019501000L; + event11.setTimestamp(expTs); + entity.addEvent(event11); + + te.addEntity(entity); + + // write another application with same metric to this flow + TimelineEntities te1 = new TimelineEntities(); + TimelineEntity entity1 = new TimelineEntity(); + id = "application_1111111111_2222"; + type = TimelineEntityType.YARN_APPLICATION.toString(); + entity1.setId(id); + entity1.setType(type); + cTime = 1425016501000L; + entity1.setCreatedTime(cTime); + entity1.addConfig("cfg1", "value1"); + // add metrics + metrics.clear(); + TimelineMetric m2 = new TimelineMetric(); + m2.setId("MAP_SLOT_MILLIS"); + metricValues = new HashMap(); + metricValues.put(ts - 100000, 5L); + metricValues.put(ts - 80000, 101L); + m2.setType(Type.TIME_SERIES); + m2.setValues(metricValues); + metrics.add(m2); + entity1.addMetrics(metrics); + TimelineEvent event1 = new TimelineEvent(); + event1.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE); + event1.setTimestamp(cTime); + event1.addInfo(expKey, expVal); + entity1.addEvent(event1); + te1.addEntity(entity1); + + String flow2 = "flow_name2"; + String flowVersion2 = "CF7022C10F1454"; + Long runid2 = 2102356789046L; + TimelineEntities te3 = new TimelineEntities(); + TimelineEntity entity3 = new TimelineEntity(); + id = "application_11111111111111_2223"; + entity3.setId(id); + entity3.setType(type); + cTime = 1425016501037L; + entity3.setCreatedTime(cTime); + TimelineEvent event2 = new TimelineEvent(); + event2.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE); + event2.setTimestamp(cTime); + event2.addInfo("foo_event", "test"); + entity3.addEvent(event2); + te3.addEntity(entity3); + + TimelineEntities te4 = new TimelineEntities(); + TimelineEntity entity4 = new TimelineEntity(); + id = "application_1111111111_2224"; + entity4.setId(id); + entity4.setType(type); + cTime = 1425016501034L; + entity4.setCreatedTime(cTime); + TimelineEvent event4 = new TimelineEvent(); + event4.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE); + event4.setTimestamp(cTime); + event4.addInfo("foo_event", "test"); + entity4.addEvent(event4); + metrics.clear(); + m2 = new TimelineMetric(); + m2.setId("MAP_SLOT_MILLIS"); + metricValues = ImmutableMap.of(ts - 100000, (Number)5L, ts - 80000, 101L); + m2.setType(Type.TIME_SERIES); + m2.setValues(metricValues); + metrics.add(m2); + entity4.addMetrics(metrics); + te4.addEntity(entity4); + + TimelineEntities userEntities = new TimelineEntities(); + TimelineEntity entity5 = new TimelineEntity(); + entity5.setId("entity1"); + entity5.setType("type1"); + entity5.setCreatedTime(1425016501034L); + // add some config entries + entity5.addConfigs(ImmutableMap.of("config_param1", "value1", + "config_param2", "value2", "cfg_param1", "value3")); + entity5.addInfo(ImmutableMap.of("info1", (Object)"cluster1", + "info2", 2.0, "info3", 35000, "info4", 36000)); + metrics = new HashSet<>(); + m1 = new TimelineMetric(); + m1.setId("MAP_SLOT_MILLIS"); + metricValues = ImmutableMap.of(ts - 100000, (Number)2, ts - 80000, 40); + m1.setType(Type.TIME_SERIES); + m1.setValues(metricValues); + metrics.add(m1); + m1 = new TimelineMetric(); + m1.setId("HDFS_BYTES_READ"); + metricValues = ImmutableMap.of(ts - 100000, (Number)31, ts - 80000, 57); + m1.setType(Type.TIME_SERIES); + m1.setValues(metricValues); + metrics.add(m1); + entity5.addMetrics(metrics); + TimelineEvent event51 = new TimelineEvent(); + event51.setId("event1"); + event51.setTimestamp(cTime); + entity5.addEvent(event51); + TimelineEvent event52 = new TimelineEvent(); + event52.setId("event2"); + event52.setTimestamp(cTime); + entity5.addEvent(event52); + TimelineEvent event53 = new TimelineEvent(); + event53.setId("event3"); + event53.setTimestamp(cTime); + entity5.addEvent(event53); + TimelineEvent event54 = new TimelineEvent(); + event54.setId("event4"); + event54.setTimestamp(cTime); + entity5.addEvent(event54); + Map> isRelatedTo1 = new HashMap>(); + isRelatedTo1.put("type2", + Sets.newHashSet("entity21", "entity22", "entity23", "entity24")); + isRelatedTo1.put("type4", Sets.newHashSet("entity41", "entity42")); + isRelatedTo1.put("type1", Sets.newHashSet("entity14", "entity15")); + isRelatedTo1.put("type3", + Sets.newHashSet("entity31", "entity35", "entity32", "entity33")); + entity5.addIsRelatedToEntities(isRelatedTo1); + Map> relatesTo1 = new HashMap>(); + relatesTo1.put("type2", + Sets.newHashSet("entity21", "entity22", "entity23", "entity24")); + relatesTo1.put("type4", Sets.newHashSet("entity41", "entity42")); + relatesTo1.put("type1", Sets.newHashSet("entity14", "entity15")); + relatesTo1.put("type3", + Sets.newHashSet("entity31", "entity35", "entity32", "entity33")); + entity5.addRelatesToEntities(relatesTo1); + userEntities.addEntity(entity5); + + TimelineEntity entity6 = new TimelineEntity(); + entity6.setId("entity2"); + entity6.setType("type1"); + entity6.setCreatedTime(1425016501034L); + entity6.addConfigs(ImmutableMap.of("cfg_param3", "value1", + "configuration_param2", "value2", "config_param1", "value3")); + entity6.addInfo(ImmutableMap.of("info1", (Object)"cluster2", + "info2", 2.0, "info4", 35000)); + metrics = new HashSet<>(); + m1 = new TimelineMetric(); + m1.setId("MAP1_SLOT_MILLIS"); + metricValues = ImmutableMap.of(ts - 100000, (Number)12, ts - 80000, 140); + m1.setType(Type.TIME_SERIES); + m1.setValues(metricValues); + metrics.add(m1); + m1 = new TimelineMetric(); + m1.setId("HDFS_BYTES_READ"); + metricValues = ImmutableMap.of(ts - 100000, (Number)78, ts - 80000, 157); + m1.setType(Type.TIME_SERIES); + m1.setValues(metricValues); + metrics.add(m1); + m1 = new TimelineMetric(); + m1.setId("MAP11_SLOT_MILLIS"); + m1.setType(Type.SINGLE_VALUE); + m1.addValue(ts - 100000, 122); + metrics.add(m1); + entity6.addMetrics(metrics); + TimelineEvent event61 = new TimelineEvent(); + event61.setId("event1"); + event61.setTimestamp(cTime); + entity6.addEvent(event61); + TimelineEvent event62 = new TimelineEvent(); + event62.setId("event5"); + event62.setTimestamp(cTime); + entity6.addEvent(event62); + TimelineEvent event63 = new TimelineEvent(); + event63.setId("event3"); + event63.setTimestamp(cTime); + entity6.addEvent(event63); + TimelineEvent event64 = new TimelineEvent(); + event64.setId("event6"); + event64.setTimestamp(cTime); + entity6.addEvent(event64); + Map> isRelatedTo2 = new HashMap>(); + isRelatedTo2.put("type2", + Sets.newHashSet("entity21", "entity22", "entity23", "entity24")); + isRelatedTo2.put("type5", Sets.newHashSet("entity51", "entity52")); + isRelatedTo2.put("type6", Sets.newHashSet("entity61", "entity66")); + isRelatedTo2.put("type3", Sets.newHashSet("entity31")); + entity6.addIsRelatedToEntities(isRelatedTo2); + Map> relatesTo2 = new HashMap>(); + relatesTo2.put("type2", + Sets.newHashSet("entity21", "entity22", "entity23", "entity24")); + relatesTo2.put("type5", Sets.newHashSet("entity51", "entity52")); + relatesTo2.put("type6", Sets.newHashSet("entity61", "entity66")); + relatesTo2.put("type3", Sets.newHashSet("entity31")); + entity6.addRelatesToEntities(relatesTo2); + userEntities.addEntity(entity6); + + for (long i = 1; i <= 10; i++) { + TimelineEntity userEntity = new TimelineEntity(); + userEntity.setType("entitytype"); + userEntity.setId("entityid-" + i); + userEntity.setIdPrefix(11 - i); + userEntity.setCreatedTime(ts); + userEntities.addEntity(userEntity); + } + + HBaseTimelineWriterImpl hbi = null; + Configuration c1 = getHBaseTestingUtility().getConfiguration(); + UserGroupInformation remoteUser = + UserGroupInformation.createRemoteUser(doAsUser); + try { + hbi = new HBaseTimelineWriterImpl(); + hbi.init(c1); + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, entity.getId()), te, remoteUser); + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, entity1.getId()), te1, remoteUser); + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid1, entity4.getId()), te4, remoteUser); + hbi.write(new TimelineCollectorContext(cluster, user, flow2, flowVersion2, + runid2, entity3.getId()), te3, remoteUser); + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, "application_1111111111_1111"), userEntities, remoteUser); + writeApplicationEntities(hbi, ts); + hbi.flush(); + } finally { + if (hbi != null) { + hbi.close(); + } + } + } + + static void writeApplicationEntities(HBaseTimelineWriterImpl hbi, + long timestamp) throws IOException { + int count = 1; + for (long i = 1; i <= 3; i++) { + for (int j = 1; j <= 5; j++) { + TimelineEntities te = new TimelineEntities(); + ApplicationId appId = + BuilderUtils.newApplicationId(timestamp, count++); + ApplicationEntity appEntity = new ApplicationEntity(); + appEntity.setId( + HBaseTimelineStorageUtils.convertApplicationIdToString(appId)); + appEntity.setCreatedTime(timestamp); + + TimelineEvent created = new TimelineEvent(); + created.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE); + created.setTimestamp(timestamp); + appEntity.addEvent(created); + TimelineEvent finished = new TimelineEvent(); + finished.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE); + finished.setTimestamp(timestamp + i * j); + + appEntity.addEvent(finished); + te.addEntity(appEntity); + hbi.write(new TimelineCollectorContext("cluster1", "user1", "flow1", + "CF7022C10F1354", i, appEntity.getId()), te, + UserGroupInformation.createRemoteUser("user1")); + } + } + } + + private static TimelineEntity newEntity(String type, String id) { + TimelineEntity entity = new TimelineEntity(); + entity.setIdentifier(new TimelineEntity.Identifier(type, id)); + return entity; + } + + private static TimelineMetric newMetric(TimelineMetric.Type type, + String id, long t, Number value) { + TimelineMetric metric = new TimelineMetric(type); + metric.setId(id); + metric.addValue(t, value); + return metric; + } + + private static boolean verifyMetricValues(Map m1, + Map m2) { + for (Map.Entry entry : m1.entrySet()) { + if (!m2.containsKey(entry.getKey())) { + return false; + } + if (m2.get(entry.getKey()).equals(entry.getValue())) { + return false; + } + } + return true; + } + + private static boolean verifyMetrics( + TimelineMetric m, TimelineMetric... metrics) { + for (TimelineMetric metric : metrics) { + if (!metric.equals(m)) { + continue; + } + if (!verifyMetricValues(metric.getValues(), m.getValues())) { + continue; + } + return true; + } + return false; + } + + @Test + public void testGetFlowRun() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/runs/" + + "1002345678919"); + ClientResponse resp = getResponse(client, uri); + FlowRunEntity entity = resp.getEntity(FlowRunEntity.class); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entity); + assertEquals("user1@flow_name/1002345678919", entity.getId()); + assertEquals(3, entity.getMetrics().size()); + TimelineMetric m1 = newMetric(TimelineMetric.Type.SINGLE_VALUE, + "HDFS_BYTES_READ", ts - 80000, 57L); + TimelineMetric m2 = newMetric(TimelineMetric.Type.SINGLE_VALUE, + "MAP_SLOT_MILLIS", ts - 80000, 141L); + TimelineMetric m3 = newMetric(TimelineMetric.Type.SINGLE_VALUE, + "MAP1_SLOT_MILLIS", ts - 80000, 40L); + for (TimelineMetric metric : entity.getMetrics()) { + assertTrue(verifyMetrics(metric, m1, m2, m3)); + } + + // Query without specifying cluster ID. + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/users/user1/flows/flow_name/runs/1002345678919"); + resp = getResponse(client, uri); + entity = resp.getEntity(FlowRunEntity.class); + assertNotNull(entity); + assertEquals("user1@flow_name/1002345678919", entity.getId()); + assertEquals(3, entity.getMetrics().size()); + m1 = newMetric(TimelineMetric.Type.SINGLE_VALUE, + "HDFS_BYTES_READ", ts - 80000, 57L); + m2 = newMetric(TimelineMetric.Type.SINGLE_VALUE, + "MAP_SLOT_MILLIS", ts - 80000, 141L); + m3 = newMetric(TimelineMetric.Type.SINGLE_VALUE, + "MAP1_SLOT_MILLIS", ts - 80000, 40L); + for (TimelineMetric metric : entity.getMetrics()) { + assertTrue(verifyMetrics(metric, m1, m2, m3)); + } + } finally { + client.destroy(); + } + } + + @Test + public void testGetFlowRuns() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/runs"); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + assertEquals(2, entities.size()); + for (FlowRunEntity entity : entities) { + assertTrue("Id, run id or start time does not match.", + ((entity.getId().equals("user1@flow_name/1002345678919")) && + (entity.getRunId() == 1002345678919L) && + (entity.getStartTime() == 1425016501000L)) || + ((entity.getId().equals("user1@flow_name/1002345678920")) && + (entity.getRunId() == 1002345678920L) && + (entity.getStartTime() == 1425016501034L))); + assertEquals(0, entity.getMetrics().size()); + } + + uri = + URI.create("http://localhost:" + getServerPort() + "/ws/v2/timeline/" + + "clusters/cluster1/users/user1/flows/flow_name/runs?limit=1"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + assertEquals(1, entities.size()); + for (FlowRunEntity entity : entities) { + assertTrue("Id, run id or start time does not match.", + entity.getId().equals("user1@flow_name/1002345678920") && + entity.getRunId() == 1002345678920L && + entity.getStartTime() == 1425016501034L); + assertEquals(0, entity.getMetrics().size()); + } + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/runs?" + + "createdtimestart=1425016501030"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + assertEquals(1, entities.size()); + for (FlowRunEntity entity : entities) { + assertTrue("Id, run id or start time does not match.", + entity.getId().equals("user1@flow_name/1002345678920") && + entity.getRunId() == 1002345678920L && + entity.getStartTime() == 1425016501034L); + assertEquals(0, entity.getMetrics().size()); + } + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/runs?" + + "createdtimestart=1425016500999&createdtimeend=1425016501035"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + assertEquals(2, entities.size()); + for (FlowRunEntity entity : entities) { + assertTrue("Id, run id or start time does not match.", + ((entity.getId().equals("user1@flow_name/1002345678919")) && + (entity.getRunId() == 1002345678919L) && + (entity.getStartTime() == 1425016501000L)) || + ((entity.getId().equals("user1@flow_name/1002345678920")) && + (entity.getRunId() == 1002345678920L) && + (entity.getStartTime() == 1425016501034L))); + assertEquals(0, entity.getMetrics().size()); + } + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/runs?" + + "createdtimeend=1425016501030"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + assertEquals(1, entities.size()); + for (FlowRunEntity entity : entities) { + assertTrue("Id, run id or start time does not match.", + entity.getId().equals("user1@flow_name/1002345678919") && + entity.getRunId() == 1002345678919L && + entity.getStartTime() == 1425016501000L); + assertEquals(0, entity.getMetrics().size()); + } + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/runs?" + + "fields=metrics"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + assertEquals(2, entities.size()); + for (FlowRunEntity entity : entities) { + assertTrue("Id, run id or start time does not match.", + ((entity.getId().equals("user1@flow_name/1002345678919")) && + (entity.getRunId() == 1002345678919L) && + (entity.getStartTime() == 1425016501000L) && + (entity.getMetrics().size() == 3)) || + ((entity.getId().equals("user1@flow_name/1002345678920")) && + (entity.getRunId() == 1002345678920L) && + (entity.getStartTime() == 1425016501034L) && + (entity.getMetrics().size() == 1))); + } + + // fields as CONFIGS will lead to a HTTP 400 as it makes no sense for + // flow runs. + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/runs?" + + "fields=CONFIGS"); + verifyHttpResponse(client, uri, Status.BAD_REQUEST); + } finally { + client.destroy(); + } + } + + @Test + public void testGetFlowRunsMetricsToRetrieve() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/runs?" + + "metricstoretrieve=MAP_,HDFS_"); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + assertEquals(2, entities.size()); + int metricCnt = 0; + for (FlowRunEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + for (TimelineMetric metric : entity.getMetrics()) { + assertTrue(metric.getId().startsWith("MAP_") || + metric.getId().startsWith("HDFS_")); + } + } + assertEquals(3, metricCnt); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/runs?" + + "metricstoretrieve=!(MAP_,HDFS_)"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + assertEquals(2, entities.size()); + metricCnt = 0; + for (FlowRunEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + for (TimelineMetric metric : entity.getMetrics()) { + assertTrue(metric.getId().startsWith("MAP1_")); + } + } + assertEquals(1, metricCnt); + } finally { + client.destroy(); + } + } + + @Test + public void testGetEntitiesByUID() throws Exception { + Client client = createClient(); + try { + // Query all flows. + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/flows"); + ClientResponse resp = getResponse(client, uri); + Set flowEntities = + resp.getEntity(new GenericType>(){}); + assertNotNull(flowEntities); + assertEquals(3, flowEntities.size()); + List listFlowUIDs = new ArrayList(); + for (FlowActivityEntity entity : flowEntities) { + String flowUID = + (String) entity.getInfo().get(TimelineReaderUtils.UID_KEY); + listFlowUIDs.add(flowUID); + assertEquals(TimelineUIDConverter.FLOW_UID.encodeUID( + new TimelineReaderContext(entity.getCluster(), entity.getUser(), + entity.getFlowName(), null, null, null, null)), flowUID); + assertTrue((entity.getId().endsWith("@flow_name") && + entity.getFlowRuns().size() == 2) || + (entity.getId().endsWith("@flow_name2") && + entity.getFlowRuns().size() == 1) + || (entity.getId().endsWith("@flow1") + && entity.getFlowRuns().size() == 3)); + } + + // Query flowruns based on UID returned in query above. + List listFlowRunUIDs = new ArrayList(); + for (String flowUID : listFlowUIDs) { + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/flow-uid/" + flowUID + "/runs"); + resp = getResponse(client, uri); + Set frEntities = + resp.getEntity(new GenericType>(){}); + assertNotNull(frEntities); + for (FlowRunEntity entity : frEntities) { + String flowRunUID = + (String) entity.getInfo().get(TimelineReaderUtils.UID_KEY); + listFlowRunUIDs.add(flowRunUID); + assertEquals(TimelineUIDConverter.FLOWRUN_UID.encodeUID( + new TimelineReaderContext("cluster1", entity.getUser(), + entity.getName(), entity.getRunId(), null, null, null)), + flowRunUID); + } + } + assertEquals(6, listFlowRunUIDs.size()); + + // Query single flowrun based on UIDs' returned in query to get flowruns. + for (String flowRunUID : listFlowRunUIDs) { + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/run-uid/" + flowRunUID); + resp = getResponse(client, uri); + FlowRunEntity entity = resp.getEntity(FlowRunEntity.class); + assertNotNull(entity); + } + + // Query apps based on UIDs' returned in query to get flowruns. + List listAppUIDs = new ArrayList(); + for (String flowRunUID : listFlowRunUIDs) { + TimelineReaderContext context = + TimelineUIDConverter.FLOWRUN_UID.decodeUID(flowRunUID); + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/run-uid/" + flowRunUID + "/apps"); + resp = getResponse(client, uri); + Set appEntities = + resp.getEntity(new GenericType>(){}); + assertNotNull(appEntities); + for (TimelineEntity entity : appEntities) { + String appUID = + (String) entity.getInfo().get(TimelineReaderUtils.UID_KEY); + listAppUIDs.add(appUID); + assertEquals(TimelineUIDConverter.APPLICATION_UID.encodeUID( + new TimelineReaderContext(context.getClusterId(), + context.getUserId(), context.getFlowName(), + context.getFlowRunId(), entity.getId(), null, null)), appUID); + } + } + assertEquals(19, listAppUIDs.size()); + + // Query single app based on UIDs' returned in query to get apps. + for (String appUID : listAppUIDs) { + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/app-uid/" + appUID); + resp = getResponse(client, uri); + TimelineEntity entity = resp.getEntity(TimelineEntity.class); + assertNotNull(entity); + } + + // Query entities based on UIDs' returned in query to get apps and + // a specific entity type(in this case type1). + List listEntityUIDs = new ArrayList(); + for (String appUID : listAppUIDs) { + TimelineReaderContext context = + TimelineUIDConverter.APPLICATION_UID.decodeUID(appUID); + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/app-uid/" + appUID + "/entities/type1"); + resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + for (TimelineEntity entity : entities) { + String entityUID = + (String) entity.getInfo().get(TimelineReaderUtils.UID_KEY); + listEntityUIDs.add(entityUID); + assertEquals(TimelineUIDConverter.GENERIC_ENTITY_UID.encodeUID( + new TimelineReaderContext(context.getClusterId(), + context.getUserId(), context.getFlowName(), + context.getFlowRunId(), context.getAppId(), "type1", + entity.getIdPrefix(), + entity.getId())), entityUID); + } + } + assertEquals(2, listEntityUIDs.size()); + + // Query single entity based on UIDs' returned in query to get entities. + for (String entityUID : listEntityUIDs) { + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/entity-uid/" + entityUID); + resp = getResponse(client, uri); + TimelineEntity entity = resp.getEntity(TimelineEntity.class); + assertNotNull(entity); + } + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/flow-uid/dummy:flow/runs"); + verifyHttpResponse(client, uri, Status.BAD_REQUEST); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/run-uid/dummy:flowrun"); + verifyHttpResponse(client, uri, Status.BAD_REQUEST); + + // Run Id is not a numerical value. + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/run-uid/some:dummy:flow:123v456"); + verifyHttpResponse(client, uri, Status.BAD_REQUEST); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/run-uid/dummy:flowrun/apps"); + verifyHttpResponse(client, uri, Status.BAD_REQUEST); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/app-uid/dummy:app"); + verifyHttpResponse(client, uri, Status.BAD_REQUEST); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/app-uid/dummy:app/entities/type1"); + verifyHttpResponse(client, uri, Status.BAD_REQUEST); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/entity-uid/dummy:entity"); + verifyHttpResponse(client, uri, Status.BAD_REQUEST); + } finally { + client.destroy(); + } + } + + @Test + public void testUIDQueryWithAndWithoutFlowContextInfo() throws Exception { + Client client = createClient(); + try { + String appUIDWithFlowInfo = + "cluster1!user1!flow_name!1002345678919!application_1111111111_1111"; + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/"+ + "timeline/app-uid/" + appUIDWithFlowInfo); + ClientResponse resp = getResponse(client, uri); + TimelineEntity appEntity1 = resp.getEntity(TimelineEntity.class); + assertNotNull(appEntity1); + assertEquals( + TimelineEntityType.YARN_APPLICATION.toString(), appEntity1.getType()); + assertEquals("application_1111111111_1111", appEntity1.getId()); + + uri = + URI.create("http://localhost:" + getServerPort() + "/ws/v2/timeline/" + + "app-uid/" + appUIDWithFlowInfo + "/entities/type1"); + resp = getResponse(client, uri); + Set entities1 = + resp.getEntity(new GenericType>(){}); + assertNotNull(entities1); + assertEquals(2, entities1.size()); + for (TimelineEntity entity : entities1) { + assertNotNull(entity.getInfo()); + assertEquals(2, entity.getInfo().size()); + String uid = + (String) entity.getInfo().get(TimelineReaderUtils.UID_KEY); + assertNotNull(uid); + assertTrue(uid.equals(appUIDWithFlowInfo + "!type1!0!entity1") + || uid.equals(appUIDWithFlowInfo + "!type1!0!entity2")); + } + + String appUIDWithoutFlowInfo = "cluster1!application_1111111111_1111"; + uri = URI.create("http://localhost:" + getServerPort() + + "/ws/v2/timeline/" + "app-uid/" + appUIDWithoutFlowInfo); + resp = getResponse(client, uri); + TimelineEntity appEntity2 = resp.getEntity(TimelineEntity.class); + assertNotNull(appEntity2); + assertEquals( + TimelineEntityType.YARN_APPLICATION.toString(), appEntity2.getType()); + assertEquals("application_1111111111_1111", appEntity2.getId()); + + uri = + URI.create("http://localhost:" + getServerPort() + "/ws/v2/timeline/" + + "app-uid/" + appUIDWithoutFlowInfo + "/entities/type1"); + resp = getResponse(client, uri); + Set entities2 = + resp.getEntity(new GenericType>(){}); + assertNotNull(entities2); + assertEquals(2, entities2.size()); + for (TimelineEntity entity : entities2) { + assertNotNull(entity.getInfo()); + assertEquals(2, entity.getInfo().size()); + String uid = + (String) entity.getInfo().get(TimelineReaderUtils.UID_KEY); + assertNotNull(uid); + assertTrue(uid.equals(appUIDWithoutFlowInfo + "!type1!0!entity1") + || uid.equals(appUIDWithoutFlowInfo + "!type1!0!entity2")); + } + + String entityUIDWithFlowInfo = appUIDWithFlowInfo + "!type1!0!entity1"; + uri = URI.create("http://localhost:" + getServerPort() + + "/ws/v2/timeline/" + "entity-uid/" + entityUIDWithFlowInfo); + resp = getResponse(client, uri); + TimelineEntity singleEntity1 = resp.getEntity(TimelineEntity.class); + assertNotNull(singleEntity1); + assertEquals("type1", singleEntity1.getType()); + assertEquals("entity1", singleEntity1.getId()); + + String entityUIDWithoutFlowInfo = + appUIDWithoutFlowInfo + "!type1!0!entity1"; + uri = URI.create("http://localhost:" + getServerPort() + + "/ws/v2/timeline/" + "entity-uid/" + entityUIDWithoutFlowInfo); + resp = getResponse(client, uri); + TimelineEntity singleEntity2 = resp.getEntity(TimelineEntity.class); + assertNotNull(singleEntity2); + assertEquals("type1", singleEntity2.getType()); + assertEquals("entity1", singleEntity2.getId()); + } finally { + client.destroy(); + } + } + + @Test + public void testUIDNotProperlyEscaped() throws Exception { + Client client = createClient(); + try { + String appUID = + "cluster1!user*1!flow_name!1002345678919!application_1111111111_1111"; + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/"+ + "timeline/app-uid/" + appUID); + verifyHttpResponse(client, uri, Status.BAD_REQUEST); + } finally { + client.destroy(); + } + } + + @Test + public void testGetFlows() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/flows"); + + verifyFlowEntites(client, uri, 3, new int[] {3, 2, 1}, + new String[] {"flow1", "flow_name", "flow_name2"}); + + // Query without specifying cluster ID. + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/flows/"); + verifyFlowEntites(client, uri, 3, new int[] {3, 2, 1}, + new String[] {"flow1", "flow_name", "flow_name2"}); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/flows?limit=1"); + verifyFlowEntites(client, uri, 1, new int[] {3}, + new String[] {"flow1"}); + + long firstFlowActivity = + HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(1425016501000L); + + DateFormat fmt = TimelineReaderWebServices.DATE_FORMAT.get(); + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/flows?daterange=" + + fmt.format(firstFlowActivity) + "-" + + fmt.format(dayTs)); + verifyFlowEntites(client, uri, 3, new int[] {3, 2, 1}, + new String[] {"flow1", "flow_name", "flow_name2"}); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/flows?daterange=" + + fmt.format(dayTs + (4*86400000L))); + verifyFlowEntites(client, uri, 0, new int[] {}, new String[] {}); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/flows?daterange=-" + + fmt.format(dayTs)); + verifyFlowEntites(client, uri, 3, new int[] {3, 2, 1}, + new String[] {"flow1", "flow_name", "flow_name2"}); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/flows?daterange=" + + fmt.format(firstFlowActivity) + "-"); + verifyFlowEntites(client, uri, 3, new int[] {3, 2, 1}, + new String[] {"flow1", "flow_name", "flow_name2"}); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/flows?daterange=20150711:20150714"); + verifyHttpResponse(client, uri, Status.BAD_REQUEST); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/flows?daterange=20150714-20150711"); + verifyHttpResponse(client, uri, Status.BAD_REQUEST); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/flows?daterange=2015071129-20150712"); + verifyHttpResponse(client, uri, Status.BAD_REQUEST); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/flows?daterange=20150711-2015071243"); + verifyHttpResponse(client, uri, Status.BAD_REQUEST); + } finally { + client.destroy(); + } + } + + @Test + public void testGetFlowsForPagination() throws Exception { + Client client = createClient(); + int noOfEntities = 3; + int limit = 2; + try { + String flowURI = "http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/flows"; + URI uri = URI.create(flowURI); + List flowEntites = + verifyFlowEntites(client, uri, 3, new int[] {3, 2, 1}, + new String[] {"flow1", "flow_name", "flow_name2"}); + FlowActivityEntity fEntity1 = flowEntites.get(0); + FlowActivityEntity fEntity3 = flowEntites.get(noOfEntities - 1); + + uri = URI.create(flowURI + "?limit=" + limit); + flowEntites = verifyFlowEntites(client, uri, limit); + assertEquals(fEntity1, flowEntites.get(0)); + FlowActivityEntity fEntity2 = flowEntites.get(limit - 1); + + uri = URI + .create(flowURI + "?limit=" + limit + "&fromid=" + + fEntity2.getInfo().get(TimelineReaderUtils.FROMID_KEY)); + flowEntites = verifyFlowEntites(client, uri, noOfEntities - limit + 1); + assertEquals(fEntity2, flowEntites.get(0)); + assertEquals(fEntity3, flowEntites.get(noOfEntities - limit)); + + uri = URI + .create(flowURI + "?limit=" + limit + "&fromid=" + + fEntity3.getInfo().get(TimelineReaderUtils.FROMID_KEY)); + flowEntites = verifyFlowEntites(client, uri, 1); + assertEquals(fEntity3, flowEntites.get(0)); + } finally { + client.destroy(); + } + } + + @Test + public void testGetApp() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111?" + + "userid=user1&fields=ALL&flowname=flow_name&flowrunid=1002345678919"); + ClientResponse resp = getResponse(client, uri); + TimelineEntity entity = resp.getEntity(TimelineEntity.class); + assertNotNull(entity); + assertEquals("application_1111111111_1111", entity.getId()); + assertEquals(3, entity.getMetrics().size()); + TimelineMetric m1 = newMetric(TimelineMetric.Type.SINGLE_VALUE, + "HDFS_BYTES_READ", ts - 80000, 57L); + TimelineMetric m2 = newMetric(TimelineMetric.Type.SINGLE_VALUE, + "MAP_SLOT_MILLIS", ts - 80000, 40L); + TimelineMetric m3 = newMetric(TimelineMetric.Type.SINGLE_VALUE, + "MAP1_SLOT_MILLIS", ts - 80000, 40L); + for (TimelineMetric metric : entity.getMetrics()) { + assertTrue(verifyMetrics(metric, m1, m2, m3)); + } + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/apps/application_1111111111_2222?userid=user1" + + "&fields=metrics&flowname=flow_name&flowrunid=1002345678919"); + resp = getResponse(client, uri); + entity = resp.getEntity(TimelineEntity.class); + assertNotNull(entity); + assertEquals("application_1111111111_2222", entity.getId()); + assertEquals(1, entity.getMetrics().size()); + TimelineMetric m4 = newMetric(TimelineMetric.Type.SINGLE_VALUE, + "MAP_SLOT_MILLIS", ts - 80000, 101L); + for (TimelineMetric metric : entity.getMetrics()) { + assertTrue(verifyMetrics(metric, m4)); + } + } finally { + client.destroy(); + } + } + + @Test + public void testGetAppWithoutFlowInfo() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111?" + + "fields=ALL"); + ClientResponse resp = getResponse(client, uri); + TimelineEntity entity = resp.getEntity(TimelineEntity.class); + assertNotNull(entity); + assertEquals("application_1111111111_1111", entity.getId()); + assertEquals(1, entity.getConfigs().size()); + assertEquals(3, entity.getMetrics().size()); + TimelineMetric m1 = newMetric(TimelineMetric.Type.SINGLE_VALUE, + "HDFS_BYTES_READ", ts - 80000, 57L); + TimelineMetric m2 = newMetric(TimelineMetric.Type.SINGLE_VALUE, + "MAP_SLOT_MILLIS", ts - 80000, 40L); + TimelineMetric m3 = newMetric(TimelineMetric.Type.SINGLE_VALUE, + "MAP1_SLOT_MILLIS", ts - 80000, 40L); + for (TimelineMetric metric : entity.getMetrics()) { + assertTrue(verifyMetrics(metric, m1, m2, m3)); + } + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111?" + + "fields=ALL&metricslimit=10"); + resp = getResponse(client, uri); + entity = resp.getEntity(TimelineEntity.class); + assertNotNull(entity); + assertEquals("application_1111111111_1111", entity.getId()); + assertEquals(1, entity.getConfigs().size()); + assertEquals(3, entity.getMetrics().size()); + m1 = newMetric(TimelineMetric.Type.TIME_SERIES, "HDFS_BYTES_READ", + ts - 100000, 31L); + m1.addValue(ts - 80000, 57L); + m2 = newMetric(TimelineMetric.Type.TIME_SERIES, "MAP_SLOT_MILLIS", + ts - 100000, 2L); + m2.addValue(ts - 80000, 40L); + m3 = newMetric(TimelineMetric.Type.TIME_SERIES, "MAP1_SLOT_MILLIS", + ts - 100000, 2L); + m3.addValue(ts - 80000, 40L); + for (TimelineMetric metric : entity.getMetrics()) { + assertTrue(verifyMetrics(metric, m1, m2, m3)); + } + } finally { + client.destroy(); + } + } + + @Test + public void testGetEntityWithoutFlowInfo() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1/entity1"); + ClientResponse resp = getResponse(client, uri); + TimelineEntity entity = resp.getEntity(TimelineEntity.class); + assertNotNull(entity); + assertEquals("entity1", entity.getId()); + assertEquals("type1", entity.getType()); + } finally { + client.destroy(); + } + } + + @Test + public void testGetEntitiesWithoutFlowInfo() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1"); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + for (TimelineEntity entity : entities) { + assertTrue(entity.getId().equals("entity1") || + entity.getId().equals("entity2")); + } + } finally { + client.destroy(); + } + } + + /** + * Tests if specific configs and metrics are retrieve for getEntities call. + */ + @Test + public void testGetEntitiesDataToRetrieve() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?confstoretrieve=cfg_"); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + int cfgCnt = 0; + for (TimelineEntity entity : entities) { + cfgCnt += entity.getConfigs().size(); + for (String configKey : entity.getConfigs().keySet()) { + assertTrue(configKey.startsWith("cfg_")); + } + } + assertEquals(2, cfgCnt); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?confstoretrieve=cfg_,config_"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + cfgCnt = 0; + for (TimelineEntity entity : entities) { + cfgCnt += entity.getConfigs().size(); + for (String configKey : entity.getConfigs().keySet()) { + assertTrue(configKey.startsWith("cfg_") || + configKey.startsWith("config_")); + } + } + assertEquals(5, cfgCnt); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?confstoretrieve=!(cfg_,config_)"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + cfgCnt = 0; + for (TimelineEntity entity : entities) { + cfgCnt += entity.getConfigs().size(); + for (String configKey : entity.getConfigs().keySet()) { + assertTrue(configKey.startsWith("configuration_")); + } + } + assertEquals(1, cfgCnt); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?metricstoretrieve=MAP_"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + int metricCnt = 0; + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + for (TimelineMetric metric : entity.getMetrics()) { + assertTrue(metric.getId().startsWith("MAP_")); + } + } + assertEquals(1, metricCnt); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?metricstoretrieve=MAP1_,HDFS_"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + metricCnt = 0; + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + for (TimelineMetric metric : entity.getMetrics()) { + assertTrue(metric.getId().startsWith("MAP1_") || + metric.getId().startsWith("HDFS_")); + } + } + assertEquals(3, metricCnt); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?metricstoretrieve=!(MAP1_,HDFS_)"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + metricCnt = 0; + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + for (TimelineMetric metric : entity.getMetrics()) { + assertTrue(metric.getId().startsWith("MAP_") || + metric.getId().startsWith("MAP11_")); + } + } + assertEquals(2, metricCnt); + } finally { + client.destroy(); + } + } + + @Test + public void testGetEntitiesConfigFilters() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?conffilters=config_param1%20eq%20value1%20OR%20" + + "config_param1%20eq%20value3"); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + for (TimelineEntity entity : entities) { + assertTrue(entity.getId().equals("entity1") || + entity.getId().equals("entity2")); + } + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?conffilters=config_param1%20eq%20value1%20AND" + + "%20configuration_param2%20eq%20value2"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(0, entities.size()); + + // conffilters=(config_param1 eq value1 AND configuration_param2 eq + // value2) OR (config_param1 eq value3 AND cfg_param3 eq value1) + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?conffilters=(config_param1%20eq%20value1%20AND" + + "%20configuration_param2%20eq%20value2)%20OR%20(config_param1%20eq" + + "%20value3%20AND%20cfg_param3%20eq%20value1)"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(1, entities.size()); + int cfgCnt = 0; + for (TimelineEntity entity : entities) { + cfgCnt += entity.getConfigs().size(); + assertTrue(entity.getId().equals("entity2")); + } + assertEquals(0, cfgCnt); + + // conffilters=(config_param1 eq value1 AND configuration_param2 eq + // value2) OR (config_param1 eq value3 AND cfg_param3 eq value1) + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?conffilters=(config_param1%20eq%20value1%20AND" + + "%20configuration_param2%20eq%20value2)%20OR%20(config_param1%20eq" + + "%20value3%20AND%20cfg_param3%20eq%20value1)&fields=CONFIGS"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(1, entities.size()); + cfgCnt = 0; + for (TimelineEntity entity : entities) { + cfgCnt += entity.getConfigs().size(); + assertTrue(entity.getId().equals("entity2")); + } + assertEquals(3, cfgCnt); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?conffilters=(config_param1%20eq%20value1%20AND" + + "%20configuration_param2%20eq%20value2)%20OR%20(config_param1%20eq" + + "%20value3%20AND%20cfg_param3%20eq%20value1)&confstoretrieve=cfg_," + + "configuration_"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(1, entities.size()); + cfgCnt = 0; + for (TimelineEntity entity : entities) { + cfgCnt += entity.getConfigs().size(); + assertTrue(entity.getId().equals("entity2")); + for (String configKey : entity.getConfigs().keySet()) { + assertTrue(configKey.startsWith("cfg_") || + configKey.startsWith("configuration_")); + } + } + assertEquals(2, cfgCnt); + + // Test for behavior when compare op is ne(not equals) vs ene + // (exists and not equals). configuration_param2 does not exist for + // entity1. For ne, both entity1 and entity2 will be returned. For ene, + // only entity2 will be returned as we are checking for existence too. + // conffilters=configuration_param2 ne value3 + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?conffilters=configuration_param2%20ne%20value3"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + for (TimelineEntity entity : entities) { + assertTrue(entity.getId().equals("entity1") || + entity.getId().equals("entity2")); + } + // conffilters=configuration_param2 ene value3 + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?conffilters=configuration_param2%20ene%20value3"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(1, entities.size()); + for (TimelineEntity entity : entities) { + assertTrue(entity.getId().equals("entity2")); + } + } finally { + client.destroy(); + } + } + + @Test + public void testGetEntitiesInfoFilters() throws Exception { + Client client = createClient(); + try { + // infofilters=info1 eq cluster1 OR info1 eq cluster2 + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?infofilters=info1%20eq%20cluster1%20OR%20info1%20eq" + + "%20cluster2"); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + for (TimelineEntity entity : entities) { + assertTrue(entity.getId().equals("entity1") || + entity.getId().equals("entity2")); + } + + // infofilters=info1 eq cluster1 AND info4 eq 35000 + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?infofilters=info1%20eq%20cluster1%20AND%20info4%20" + + "eq%2035000"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(0, entities.size()); + + // infofilters=info4 eq 35000 OR info4 eq 36000 + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?infofilters=info4%20eq%2035000%20OR%20info4%20eq" + + "%2036000"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + for (TimelineEntity entity : entities) { + assertTrue(entity.getId().equals("entity1") || + entity.getId().equals("entity2")); + } + + // infofilters=(info1 eq cluster1 AND info4 eq 35000) OR + // (info1 eq cluster2 AND info2 eq 2.0) + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?infofilters=(info1%20eq%20cluster1%20AND%20info4%20" + + "eq%2035000)%20OR%20(info1%20eq%20cluster2%20AND%20info2%20eq%202.0" + + ")"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(1, entities.size()); + int infoCnt = 0; + for (TimelineEntity entity : entities) { + infoCnt += entity.getInfo().size(); + assertTrue(entity.getId().equals("entity2")); + } + // Includes UID and FROM_ID in info field even if fields not specified as + // INFO. + assertEquals(2, infoCnt); + + // infofilters=(info1 eq cluster1 AND info4 eq 35000) OR + // (info1 eq cluster2 AND info2 eq 2.0) + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?infofilters=(info1%20eq%20cluster1%20AND%20info4%20" + + "eq%2035000)%20OR%20(info1%20eq%20cluster2%20AND%20info2%20eq%20" + + "2.0)&fields=INFO"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(1, entities.size()); + infoCnt = 0; + for (TimelineEntity entity : entities) { + infoCnt += entity.getInfo().size(); + assertTrue(entity.getId().equals("entity2")); + } + // Includes UID and FROM_ID in info field. + assertEquals(5, infoCnt); + + // Test for behavior when compare op is ne(not equals) vs ene + // (exists and not equals). info3 does not exist for entity2. For ne, + // both entity1 and entity2 will be returned. For ene, only entity2 will + // be returned as we are checking for existence too. + // infofilters=info3 ne 39000 + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?infofilters=info3%20ne%2039000"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + for (TimelineEntity entity : entities) { + assertTrue(entity.getId().equals("entity1") || + entity.getId().equals("entity2")); + } + // infofilters=info3 ene 39000 + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?infofilters=info3%20ene%2039000"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(1, entities.size()); + for (TimelineEntity entity : entities) { + assertTrue(entity.getId().equals("entity1")); + } + } finally { + client.destroy(); + } + } + + @Test + public void testGetEntitiesMetricFilters() throws Exception { + Client client = createClient(); + try { + // metricfilters=HDFS_BYTES_READ lt 60 OR HDFS_BYTES_READ eq 157 + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?metricfilters=HDFS_BYTES_READ%20lt%2060%20OR%20" + + "HDFS_BYTES_READ%20eq%20157"); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + for (TimelineEntity entity : entities) { + assertTrue(entity.getId().equals("entity1") || + entity.getId().equals("entity2")); + } + + // metricfilters=HDFS_BYTES_READ lt 60 AND MAP_SLOT_MILLIS gt 40 + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?metricfilters=HDFS_BYTES_READ%20lt%2060%20AND%20" + + "MAP_SLOT_MILLIS%20gt%2040"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(0, entities.size()); + + // metricfilters=(HDFS_BYTES_READ lt 60 AND MAP_SLOT_MILLIS gt 40) OR + // (MAP1_SLOT_MILLIS ge 140 AND MAP11_SLOT_MILLIS le 122) + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?metricfilters=(HDFS_BYTES_READ%20lt%2060%20AND%20" + + "MAP_SLOT_MILLIS%20gt%2040)%20OR%20(MAP1_SLOT_MILLIS%20ge" + + "%20140%20AND%20MAP11_SLOT_MILLIS%20le%20122)"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(1, entities.size()); + int metricCnt = 0; + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + assertTrue(entity.getId().equals("entity2")); + } + assertEquals(0, metricCnt); + + // metricfilters=(HDFS_BYTES_READ lt 60 AND MAP_SLOT_MILLIS gt 40) OR + // (MAP1_SLOT_MILLIS ge 140 AND MAP11_SLOT_MILLIS le 122) + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?metricfilters=(HDFS_BYTES_READ%20lt%2060%20AND%20" + + "MAP_SLOT_MILLIS%20gt%2040)%20OR%20(MAP1_SLOT_MILLIS%20ge" + + "%20140%20AND%20MAP11_SLOT_MILLIS%20le%20122)&fields=METRICS"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(1, entities.size()); + metricCnt = 0; + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + assertTrue(entity.getId().equals("entity2")); + } + assertEquals(3, metricCnt); + + // metricfilters=(HDFS_BYTES_READ lt 60 AND MAP_SLOT_MILLIS gt 40) OR + // (MAP1_SLOT_MILLIS ge 140 AND MAP11_SLOT_MILLIS le 122) + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?metricfilters=(HDFS_BYTES_READ%20lt%2060%20AND%20" + + "MAP_SLOT_MILLIS%20gt%2040)%20OR%20(MAP1_SLOT_MILLIS%20ge" + + "%20140%20AND%20MAP11_SLOT_MILLIS%20le%20122)&metricstoretrieve=" + + "!(HDFS)"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(1, entities.size()); + metricCnt = 0; + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + assertTrue(entity.getId().equals("entity2")); + for (TimelineMetric metric : entity.getMetrics()) { + assertTrue(metric.getId().startsWith("MAP1")); + assertEquals(TimelineMetric.Type.SINGLE_VALUE, metric.getType()); + } + } + assertEquals(2, metricCnt); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?metricfilters=(HDFS_BYTES_READ%20lt%2060%20AND%20" + + "MAP_SLOT_MILLIS%20gt%2040)%20OR%20(MAP1_SLOT_MILLIS%20ge" + + "%20140%20AND%20MAP11_SLOT_MILLIS%20le%20122)&metricstoretrieve=" + + "!(HDFS)&metricslimit=10"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(1, entities.size()); + metricCnt = 0; + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + assertTrue(entity.getId().equals("entity2")); + for (TimelineMetric metric : entity.getMetrics()) { + assertTrue(metric.getId().startsWith("MAP1")); + if (metric.getId().equals("MAP1_SLOT_MILLIS")) { + assertEquals(2, metric.getValues().size()); + assertEquals(TimelineMetric.Type.TIME_SERIES, metric.getType()); + } else if (metric.getId().equals("MAP11_SLOT_MILLIS")) { + assertEquals(TimelineMetric.Type.SINGLE_VALUE, metric.getType()); + } else { + fail("Unexpected metric id"); + } + } + } + assertEquals(2, metricCnt); + + // Test for behavior when compare op is ne(not equals) vs ene + // (exists and not equals). MAP11_SLOT_MILLIS does not exist for + // entity1. For ne, both entity1 and entity2 will be returned. For ene, + // only entity2 will be returned as we are checking for existence too. + // metricfilters=MAP11_SLOT_MILLIS ne 100 + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?metricfilters=MAP11_SLOT_MILLIS%20ne%20100"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + for (TimelineEntity entity : entities) { + assertTrue(entity.getId().equals("entity1") || + entity.getId().equals("entity2")); + } + // metricfilters=MAP11_SLOT_MILLIS ene 100 + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?metricfilters=MAP11_SLOT_MILLIS%20ene%20100"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(1, entities.size()); + for (TimelineEntity entity : entities) { + assertTrue(entity.getId().equals("entity2")); + } + } finally { + client.destroy(); + } + } + + @Test + public void testGetEntitiesEventFilters() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?eventfilters=event1,event3"); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + for (TimelineEntity entity : entities) { + assertTrue(entity.getId().equals("entity1") || + entity.getId().equals("entity2")); + } + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?eventfilters=!(event1,event3)"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(0, entities.size()); + + // eventfilters=!(event1,event3) OR event5,event6 + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?eventfilters=!(event1,event3)%20OR%20event5,event6"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(1, entities.size()); + for (TimelineEntity entity : entities) { + assertTrue(entity.getId().equals("entity2")); + } + + // eventfilters=(!(event1,event3) OR event5,event6) OR + // (event1,event2 AND (event3,event4)) + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?eventfilters=(!(event1,event3)%20OR%20event5," + + "event6)%20OR%20(event1,event2%20AND%20(event3,event4))"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + for (TimelineEntity entity : entities) { + assertTrue(entity.getId().equals("entity1") || + entity.getId().equals("entity2")); + } + } finally { + client.destroy(); + } + } + + @Test + public void testGetEntitiesRelationFilters() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?isrelatedto=type3:entity31,type2:entity21:entity22"); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + for (TimelineEntity entity : entities) { + assertTrue(entity.getId().equals("entity1") || + entity.getId().equals("entity2")); + } + + uri = URI.create("http://localhost:" + getServerPort() + + "/ws/v2/timeline/" + + "clusters/cluster1/apps/application_1111111111_1111/entities/type1" + + "?isrelatedto=!(type3:entity31,type2:entity21:entity22)"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(0, entities.size()); + + // isrelatedto=!(type3:entity31,type2:entity21:entity22)OR type5:entity51, + // type6:entity61:entity66 + uri = URI.create("http://localhost:" + getServerPort() + + "/ws/v2/timeline/" + + "clusters/cluster1/apps/application_1111111111_1111/entities/type1" + + "?isrelatedto=!(type3:entity31,type2:entity21:entity22)%20OR%20" + + "type5:entity51,type6:entity61:entity66"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(1, entities.size()); + for (TimelineEntity entity : entities) { + assertTrue(entity.getId().equals("entity2")); + } + + // isrelatedto=(!(type3:entity31,type2:entity21:entity22)OR type5: + // entity51,type6:entity61:entity66) OR (type1:entity14,type2:entity21: + // entity22 AND (type3:entity32:entity35,type4:entity42)) + uri = URI.create("http://localhost:" + getServerPort() + + "/ws/v2/timeline/" + + "clusters/cluster1/apps/application_1111111111_1111/entities/type1" + + "?isrelatedto=(!(type3:entity31,type2:entity21:entity22)%20OR%20" + + "type5:entity51,type6:entity61:entity66)%20OR%20(type1:entity14," + + "type2:entity21:entity22%20AND%20(type3:entity32:entity35,"+ + "type4:entity42))"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + for (TimelineEntity entity : entities) { + assertTrue(entity.getId().equals("entity1") || + entity.getId().equals("entity2")); + } + + // relatesto=!(type3:entity31,type2:entity21:entity22)OR type5:entity51, + // type6:entity61:entity66 + uri = URI.create("http://localhost:" + getServerPort() + + "/ws/v2/timeline/" + + "clusters/cluster1/apps/application_1111111111_1111/entities/type1" + + "?relatesto=!%20(type3:entity31,type2:entity21:entity22%20)%20OR%20" + + "type5:entity51,type6:entity61:entity66"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(1, entities.size()); + for (TimelineEntity entity : entities) { + assertTrue(entity.getId().equals("entity2")); + } + + // relatesto=(!(type3:entity31,type2:entity21:entity22)OR type5:entity51, + // type6:entity61:entity66) OR (type1:entity14,type2:entity21:entity22 AND + // (type3:entity32:entity35 , type4:entity42)) + uri = + URI.create("http://localhost:" + getServerPort() + + "/ws/v2/timeline/" + + "clusters/cluster1/apps/application_1111111111_1111/entities/type1" + + "?relatesto=(!(%20type3:entity31,type2:entity21:entity22)%20OR%20" + + "type5:entity51,type6:entity61:entity66%20)%20OR%20(type1:entity14," + + "type2:entity21:entity22%20AND%20(type3:entity32:entity35%20,%20"+ + "type4:entity42))"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + for (TimelineEntity entity : entities) { + assertTrue(entity.getId().equals("entity1") || + entity.getId().equals("entity2")); + } + } finally { + client.destroy(); + } + } + + private static void verifyMetricCount(TimelineEntity entity, + int expectedMetricsCnt, int expectedMeticsValCnt) { + int metricsValCnt = 0; + for (TimelineMetric m : entity.getMetrics()) { + metricsValCnt += m.getValues().size(); + } + assertEquals(expectedMetricsCnt, entity.getMetrics().size()); + assertEquals(expectedMeticsValCnt, metricsValCnt); + } + + private static void verifyMetricsCount(Set entities, + int expectedMetricsCnt, int expectedMeticsValCnt) { + int metricsCnt = 0; + int metricsValCnt = 0; + for (TimelineEntity entity : entities) { + metricsCnt += entity.getMetrics().size(); + for (TimelineMetric m : entity.getMetrics()) { + metricsValCnt += m.getValues().size(); + } + } + assertEquals(expectedMetricsCnt, metricsCnt); + assertEquals(expectedMeticsValCnt, metricsValCnt); + } + + @Test + public void testGetEntitiesMetricsTimeRange() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?fields=ALL&metricslimit=100&metricstimestart=" + + (ts - 90000) + "&metricstimeend=" + (ts - 80000)); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + verifyMetricsCount(entities, 4, 4); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?fields=ALL&metricslimit=100&metricstimestart=" + + (ts - 100000) + "&metricstimeend=" + (ts - 80000)); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + verifyMetricsCount(entities, 5, 9); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?fields=ALL&metricslimit=100&metricstimestart=" + + (ts - 100000)); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + verifyMetricsCount(entities, 5, 9); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?fields=ALL&metricslimit=100&metricstimeend=" + + (ts - 90000)); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + verifyMetricsCount(entities, 5, 5); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?fields=ALL&metricstimestart=" + + (ts - 100000)); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + verifyMetricsCount(entities, 5, 5); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1/entity2?fields=ALL&metricstimestart=" + + (ts - 100000) + "&metricstimeend=" + (ts - 80000)); + resp = getResponse(client, uri); + TimelineEntity entity = resp.getEntity(TimelineEntity.class); + assertNotNull(entity); + verifyMetricCount(entity, 3, 3); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1/entity2?fields=ALL&metricslimit=5&metricstimestart=" + + (ts - 100000) + "&metricstimeend=" + (ts - 80000)); + resp = getResponse(client, uri); + entity = resp.getEntity(TimelineEntity.class); + assertNotNull(entity); + verifyMetricCount(entity, 3, 5); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1?fields=ALL&metricslimit=100&metricstimestart=" + + (ts - 80000) + "&metricstimeend=" + (ts - 90000)); + verifyHttpResponse(client, uri, Status.BAD_REQUEST); + } finally { + client.destroy(); + } + } + + /** + * Tests if specific configs and metrics are retrieve for getEntity call. + */ + @Test + public void testGetEntityDataToRetrieve() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1/entity2?confstoretrieve=cfg_,configuration_"); + ClientResponse resp = getResponse(client, uri); + TimelineEntity entity = resp.getEntity(TimelineEntity.class); + assertNotNull(entity); + assertEquals("entity2", entity.getId()); + assertEquals("type1", entity.getType()); + assertEquals(2, entity.getConfigs().size()); + for (String configKey : entity.getConfigs().keySet()) { + assertTrue(configKey.startsWith("configuration_") || + configKey.startsWith("cfg_")); + } + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1/entity2?confstoretrieve=!(cfg_,configuration_)"); + resp = getResponse(client, uri); + entity = resp.getEntity(TimelineEntity.class); + assertNotNull(entity); + assertEquals("entity2", entity.getId()); + assertEquals("type1", entity.getType()); + assertEquals(1, entity.getConfigs().size()); + for (String configKey : entity.getConfigs().keySet()) { + assertTrue(configKey.startsWith("config_")); + } + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1/entity2?metricstoretrieve=MAP1_,HDFS_"); + resp = getResponse(client, uri); + entity = resp.getEntity(TimelineEntity.class); + assertNotNull(entity); + assertEquals("entity2", entity.getId()); + assertEquals("type1", entity.getType()); + assertEquals(2, entity.getMetrics().size()); + for (TimelineMetric metric : entity.getMetrics()) { + assertTrue(metric.getId().startsWith("MAP1_") || + metric.getId().startsWith("HDFS_")); + } + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1/entity2?metricstoretrieve=!(MAP1_,HDFS_)"); + resp = getResponse(client, uri); + entity = resp.getEntity(TimelineEntity.class); + assertNotNull(entity); + assertEquals("entity2", entity.getId()); + assertEquals("type1", entity.getType()); + assertEquals(1, entity.getMetrics().size()); + for (TimelineMetric metric : entity.getMetrics()) { + assertTrue(metric.getId().startsWith("MAP11_")); + assertEquals(TimelineMetric.Type.SINGLE_VALUE, metric.getType()); + assertEquals(1, metric.getValues().size()); + } + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/type1/entity2?metricstoretrieve=!(MAP1_,HDFS_)&" + + "metricslimit=5"); + resp = getResponse(client, uri); + entity = resp.getEntity(TimelineEntity.class); + assertNotNull(entity); + assertEquals("entity2", entity.getId()); + assertEquals("type1", entity.getType()); + assertEquals(1, entity.getMetrics().size()); + for (TimelineMetric metric : entity.getMetrics()) { + assertTrue(metric.getId().startsWith("MAP11_")); + assertEquals(TimelineMetric.Type.SINGLE_VALUE, metric.getType()); + } + } finally { + client.destroy(); + } + } + + @Test + public void testGetFlowRunApps() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/runs/" + + "1002345678919/apps?fields=ALL"); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + for (TimelineEntity entity : entities) { + assertTrue("Unexpected app in result", + (entity.getId().equals("application_1111111111_1111") && + entity.getMetrics().size() == 3) || + (entity.getId().equals("application_1111111111_2222") && + entity.getMetrics().size() == 1)); + for (TimelineMetric metric : entity.getMetrics()) { + assertEquals(TimelineMetric.Type.SINGLE_VALUE, metric.getType()); + assertEquals(1, metric.getValues().size()); + } + } + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/runs/" + + "1002345678919/apps?fields=ALL&metricslimit=2"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + for (TimelineEntity entity : entities) { + assertTrue("Unexpected app in result", + (entity.getId().equals("application_1111111111_1111") && + entity.getMetrics().size() == 3) || + (entity.getId().equals("application_1111111111_2222") && + entity.getMetrics().size() == 1)); + for (TimelineMetric metric : entity.getMetrics()) { + assertTrue(metric.getValues().size() <= 2); + assertEquals(TimelineMetric.Type.TIME_SERIES, metric.getType()); + } + } + + // Query without specifying cluster ID. + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/users/user1/flows/flow_name/runs/1002345678919/apps"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/users/user1/flows/flow_name/runs/1002345678919/" + + "apps?limit=1"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(1, entities.size()); + } finally { + client.destroy(); + } + } + + @Test + public void testGetFlowApps() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/apps?" + + "fields=ALL"); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(3, entities.size()); + for (TimelineEntity entity : entities) { + assertTrue("Unexpected app in result", + (entity.getId().equals("application_1111111111_1111") && + entity.getConfigs().size() == 1 && + entity.getConfigs().equals(ImmutableMap.of("cfg2", "value1"))) || + (entity.getId().equals("application_1111111111_2222") && + entity.getConfigs().size() == 1 && + entity.getConfigs().equals(ImmutableMap.of("cfg1", "value1"))) || + (entity.getId().equals("application_1111111111_2224") && + entity.getConfigs().size() == 0)); + for (TimelineMetric metric : entity.getMetrics()) { + if (entity.getId().equals("application_1111111111_1111")) { + TimelineMetric m1 = newMetric(TimelineMetric.Type.SINGLE_VALUE, + "HDFS_BYTES_READ", ts - 80000, 57L); + TimelineMetric m2 = newMetric(TimelineMetric.Type.SINGLE_VALUE, + "MAP_SLOT_MILLIS", ts - 80000, 40L); + TimelineMetric m3 = newMetric(TimelineMetric.Type.SINGLE_VALUE, + "MAP1_SLOT_MILLIS", ts - 80000, 40L); + assertTrue(verifyMetrics(metric, m1, m2, m3)); + } else if (entity.getId().equals("application_1111111111_2222")) { + TimelineMetric m1 = newMetric(TimelineMetric.Type.SINGLE_VALUE, + "MAP_SLOT_MILLIS", ts - 80000, 101L); + assertTrue(verifyMetrics(metric, m1)); + } else if (entity.getId().equals("application_1111111111_2224")) { + TimelineMetric m1 = newMetric(TimelineMetric.Type.SINGLE_VALUE, + "MAP_SLOT_MILLIS", ts - 80000, 101L); + assertTrue(verifyMetrics(metric, m1)); + } + } + } + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/apps?" + + "fields=ALL&metricslimit=6"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(3, entities.size()); + for (TimelineEntity entity : entities) { + assertTrue("Unexpected app in result", + (entity.getId().equals("application_1111111111_1111") && + entity.getConfigs().size() == 1 && + entity.getConfigs().equals(ImmutableMap.of("cfg2", "value1"))) || + (entity.getId().equals("application_1111111111_2222") && + entity.getConfigs().size() == 1 && + entity.getConfigs().equals(ImmutableMap.of("cfg1", "value1"))) || + (entity.getId().equals("application_1111111111_2224") && + entity.getConfigs().size() == 0)); + for (TimelineMetric metric : entity.getMetrics()) { + if (entity.getId().equals("application_1111111111_1111")) { + TimelineMetric m1 = newMetric(TimelineMetric.Type.TIME_SERIES, + "HDFS_BYTES_READ", ts - 80000, 57L); + m1.addValue(ts - 100000, 31L); + TimelineMetric m2 = newMetric(TimelineMetric.Type.TIME_SERIES, + "MAP_SLOT_MILLIS", ts - 80000, 40L); + m2.addValue(ts - 100000, 2L); + TimelineMetric m3 = newMetric(TimelineMetric.Type.TIME_SERIES, + "MAP1_SLOT_MILLIS", ts - 80000, 40L); + m3.addValue(ts - 100000, 2L); + assertTrue(verifyMetrics(metric, m1, m2, m3)); + } else if (entity.getId().equals("application_1111111111_2222")) { + TimelineMetric m1 = newMetric(TimelineMetric.Type.TIME_SERIES, + "MAP_SLOT_MILLIS", ts - 80000, 101L); + m1.addValue(ts - 100000, 5L); + assertTrue(verifyMetrics(metric, m1)); + } else if (entity.getId().equals("application_1111111111_2224")) { + TimelineMetric m1 = newMetric(TimelineMetric.Type.TIME_SERIES, + "MAP_SLOT_MILLIS", ts - 80000, 101L); + m1.addValue(ts - 100000, 5L); + assertTrue(verifyMetrics(metric, m1)); + } + } + } + + // Query without specifying cluster ID. + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/users/user1/flows/flow_name/apps"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(3, entities.size()); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/users/user1/flows/flow_name/apps?limit=1"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(1, entities.size()); + } finally { + client.destroy(); + } + } + + @Test + public void testGetFlowAppsFilters() throws Exception { + Client client = createClient(); + try { + String entityType = TimelineEntityType.YARN_APPLICATION.toString(); + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/apps?" + + "eventfilters=" + ApplicationMetricsConstants.FINISHED_EVENT_TYPE); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(1, entities.size()); + assertTrue("Unexpected app in result", entities.contains( + newEntity(entityType, "application_1111111111_1111"))); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/apps?" + + "metricfilters=HDFS_BYTES_READ%20ge%200"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(1, entities.size()); + assertTrue("Unexpected app in result", entities.contains( + newEntity(entityType, "application_1111111111_1111"))); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/apps?" + + "conffilters=cfg1%20eq%20value1"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(1, entities.size()); + assertTrue("Unexpected app in result", entities.contains( + newEntity(entityType, "application_1111111111_2222"))); + } finally { + client.destroy(); + } + } + + @Test + public void testGetFlowRunNotPresent() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/runs/" + + "1002345678929"); + verifyHttpResponse(client, uri, Status.NOT_FOUND); + } finally { + client.destroy(); + } + } + + @Test + public void testGetFlowsNotPresent() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster2/flows"); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + assertEquals(0, entities.size()); + } finally { + client.destroy(); + } + } + + @Test + public void testGetAppNotPresent() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1378"); + verifyHttpResponse(client, uri, Status.NOT_FOUND); + } finally { + client.destroy(); + } + } + + @Test + public void testGetFlowRunAppsNotPresent() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster2/users/user1/flows/flow_name/runs/" + + "1002345678919/apps"); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + assertEquals(0, entities.size()); + } finally { + client.destroy(); + } + } + + @Test + public void testGetFlowAppsNotPresent() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster2/users/user1/flows/flow_name55/apps"); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + assertEquals(0, entities.size()); + } finally { + client.destroy(); + } + } + + @Test + public void testGenericEntitiesForPagination() throws Exception { + Client client = createClient(); + try { + String resourceUri = "http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/apps/application_1111111111_1111/" + + "entities/entitytype"; + verifyEntitiesForPagination(client, resourceUri); + resourceUri = "http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/" + doAsUser + + "/entities/entitytype"; + verifyEntitiesForPagination(client, resourceUri); + } finally { + client.destroy(); + } + } + + private void verifyEntitiesForPagination(Client client, String resourceUri) + throws Exception { + int limit = 10; + String queryParam = "?limit=" + limit; + URI uri = URI.create(resourceUri + queryParam); + + ClientResponse resp = getResponse(client, uri); + List entities = + resp.getEntity(new GenericType>() { + }); + // verify for entity-10 to entity-1 in descending order. + verifyPaginatedEntites(entities, limit, limit); + + limit = 4; + queryParam = "?limit=" + limit; + uri = URI.create(resourceUri + queryParam); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>() { + }); + // verify for entity-10 to entity-7 in descending order. + TimelineEntity entity = verifyPaginatedEntites(entities, limit, 10); + + queryParam = "?limit=" + limit + "&fromid=" + + entity.getInfo().get(TimelineReaderUtils.FROMID_KEY); + uri = URI.create(resourceUri + queryParam); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>() { + }); + // verify for entity-7 to entity-4 in descending order. + entity = verifyPaginatedEntites(entities, limit, 7); + + queryParam = "?limit=" + limit + "&fromid=" + + entity.getInfo().get(TimelineReaderUtils.FROMID_KEY); + uri = URI.create(resourceUri + queryParam); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>() { + }); + // verify for entity-4 to entity-1 in descending order. + entity = verifyPaginatedEntites(entities, limit, 4); + + queryParam = "?limit=" + limit + "&fromid=" + + entity.getInfo().get(TimelineReaderUtils.FROMID_KEY); + uri = URI.create(resourceUri + queryParam); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>() { + }); + // always entity-1 will be retrieved + entity = verifyPaginatedEntites(entities, 1, 1); + } + + private TimelineEntity verifyPaginatedEntites(List entities, + int limit, int startFrom) { + assertNotNull(entities); + assertEquals(limit, entities.size()); + TimelineEntity entity = null; + for (TimelineEntity timelineEntity : entities) { + assertEquals("entitytype", timelineEntity.getType()); + assertEquals("entityid-" + startFrom, timelineEntity.getId()); + assertEquals(11 - startFrom--, timelineEntity.getIdPrefix()); + entity = timelineEntity; + } + return entity; + } + + private List verifyFlowEntites(Client client, URI uri, + int noOfEntities, + int[] a, String[] flowsInSequence) throws Exception { + ClientResponse resp = getResponse(client, uri); + List entities = + resp.getEntity(new GenericType>() { + }); + assertNotNull(entities); + assertEquals(noOfEntities, entities.size()); + assertEquals(noOfEntities, flowsInSequence.length); + assertEquals(noOfEntities, a.length); + int count = 0; + for (FlowActivityEntity timelineEntity : entities) { + assertEquals(flowsInSequence[count], + timelineEntity.getInfo().get("SYSTEM_INFO_FLOW_NAME")); + assertEquals(a[count++], timelineEntity.getFlowRuns().size()); + } + return entities; + } + + @Test + public void testForFlowAppsPagination() throws Exception { + Client client = createClient(); + try { + // app entities stored is 15 during initialization. + int totalAppEntities = 15; + String resourceUri = "http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow1/apps"; + URI uri = URI.create(resourceUri); + ClientResponse resp = getResponse(client, uri); + List entities = + resp.getEntity(new GenericType>() { + }); + assertNotNull(entities); + assertEquals(totalAppEntities, entities.size()); + TimelineEntity entity1 = entities.get(0); + TimelineEntity entity15 = entities.get(totalAppEntities - 1); + + int limit = 10; + String queryParam = "?limit=" + limit; + uri = URI.create(resourceUri + queryParam); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>() { + }); + assertNotNull(entities); + assertEquals(limit, entities.size()); + assertEquals(entity1, entities.get(0)); + TimelineEntity entity10 = entities.get(limit - 1); + + uri = + URI.create(resourceUri + queryParam + "&fromid=" + + entity10.getInfo().get(TimelineReaderUtils.FROMID_KEY)); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>() { + }); + assertNotNull(entities); + assertEquals(6, entities.size()); + assertEquals(entity10, entities.get(0)); + assertEquals(entity15, entities.get(5)); + + } finally { + client.destroy(); + } + } + + @Test + public void testForFlowRunAppsPagination() throws Exception { + Client client = createClient(); + try { + // app entities stored is 15 during initialization. + int totalAppEntities = 5; + String resourceUri = "http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow1/runs/1/apps"; + URI uri = URI.create(resourceUri); + ClientResponse resp = getResponse(client, uri); + List entities = + resp.getEntity(new GenericType>() { + }); + assertNotNull(entities); + assertEquals(totalAppEntities, entities.size()); + TimelineEntity entity1 = entities.get(0); + TimelineEntity entity5 = entities.get(totalAppEntities - 1); + + int limit = 3; + String queryParam = "?limit=" + limit; + uri = URI.create(resourceUri + queryParam); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>() { + }); + assertNotNull(entities); + assertEquals(limit, entities.size()); + assertEquals(entity1, entities.get(0)); + TimelineEntity entity3 = entities.get(limit - 1); + + uri = + URI.create(resourceUri + queryParam + "&fromid=" + + entity3.getInfo().get(TimelineReaderUtils.FROMID_KEY)); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>() { + }); + assertNotNull(entities); + assertEquals(3, entities.size()); + assertEquals(entity3, entities.get(0)); + assertEquals(entity5, entities.get(2)); + + } finally { + client.destroy(); + } + } + + @Test + public void testForFlowRunsPagination() throws Exception { + Client client = createClient(); + try { + // app entities stored is 15 during initialization. + int totalRuns = 3; + String resourceUri = "http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow1/runs"; + URI uri = URI.create(resourceUri); + ClientResponse resp = getResponse(client, uri); + List entities = + resp.getEntity(new GenericType>() { + }); + assertNotNull(entities); + assertEquals(totalRuns, entities.size()); + TimelineEntity entity1 = entities.get(0); + TimelineEntity entity3 = entities.get(totalRuns - 1); + + int limit = 2; + String queryParam = "?limit=" + limit; + uri = URI.create(resourceUri + queryParam); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>() { + }); + assertNotNull(entities); + assertEquals(limit, entities.size()); + assertEquals(entity1, entities.get(0)); + TimelineEntity entity2 = entities.get(limit - 1); + + uri = URI.create(resourceUri + queryParam + "&fromid=" + + entity2.getInfo().get(TimelineReaderUtils.FROMID_KEY)); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>() { + }); + assertNotNull(entities); + assertEquals(limit, entities.size()); + assertEquals(entity2, entities.get(0)); + assertEquals(entity3, entities.get(1)); + + uri = URI.create(resourceUri + queryParam + "&fromid=" + + entity3.getInfo().get(TimelineReaderUtils.FROMID_KEY)); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>() { + }); + assertNotNull(entities); + assertEquals(1, entities.size()); + assertEquals(entity3, entities.get(0)); + } finally { + client.destroy(); + } + } + + @Test + public void testGetAppsMetricsRange() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/runs/" + + "1002345678919/apps?fields=ALL&metricslimit=100&metricstimestart=" + + (ts - 200000) + "&metricstimeend=" + (ts - 100000)); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + verifyMetricsCount(entities, 4, 4); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/runs/" + + "1002345678919/apps?fields=ALL&metricslimit=100"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + verifyMetricsCount(entities, 4, 10); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/" + + "apps?fields=ALL&metricslimit=100&metricstimestart=" + + (ts - 200000) + "&metricstimeend=" + (ts - 100000)); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(3, entities.size()); + verifyMetricsCount(entities, 5, 5); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/" + + "apps?fields=ALL&metricslimit=100"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(3, entities.size()); + verifyMetricsCount(entities, 5, 12); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/runs/" + + "1002345678919/apps?fields=ALL&metricslimit=100&metricstimestart=" + + (ts - 200000)); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + verifyMetricsCount(entities, 4, 10); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/runs/" + + "1002345678919/apps?fields=ALL&metricslimit=100&metricstimeend=" + + (ts - 100000)); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertNotNull(entities); + assertEquals(2, entities.size()); + verifyMetricsCount(entities, 4, 4); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/apps/application_1111111111_1111?userid=user1&fields=ALL" + + "&flowname=flow_name&flowrunid=1002345678919&metricslimit=100" + + "&metricstimestart=" +(ts - 200000) + "&metricstimeend=" + + (ts - 100000)); + resp = getResponse(client, uri); + TimelineEntity entity = resp.getEntity(TimelineEntity.class); + assertNotNull(entity); + verifyMetricCount(entity, 3, 3); + + uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" + + "timeline/clusters/cluster1/users/user1/flows/flow_name/" + + "apps?fields=ALL&metricslimit=100&metricstimestart=" + + (ts - 100000) + "&metricstimeend=" + (ts - 200000)); + verifyHttpResponse(client, uri, Status.BAD_REQUEST); + } finally { + client.destroy(); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/DataGeneratorForTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/DataGeneratorForTest.java new file mode 100644 index 00000000000..cf6a8544241 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/DataGeneratorForTest.java @@ -0,0 +1,516 @@ +/** + * 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.timelineservice.storage; + +import java.io.IOException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric.Type; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants; +import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext; + +/** + * Utility class that creates the schema and generates test data. + */ +public final class DataGeneratorForTest { + + // private constructor for utility class + private DataGeneratorForTest() { + } + + /** + * Creates the schema for timeline service. + * @param conf + * @throws IOException + */ + public static void createSchema(final Configuration conf) + throws IOException { + // set the jar location to null so that + // the coprocessor class is loaded from classpath + conf.set(YarnConfiguration.FLOW_RUN_COPROCESSOR_JAR_HDFS_LOCATION, " "); + // now create all tables + TimelineSchemaCreator.createAllTables(conf, false); + } + + public static void loadApps(HBaseTestingUtility util, long ts) + throws IOException { + TimelineEntities te = new TimelineEntities(); + TimelineEntity entity = new TimelineEntity(); + String id = "application_1111111111_2222"; + entity.setId(id); + entity.setType(TimelineEntityType.YARN_APPLICATION.toString()); + Long cTime = 1425016502000L; + entity.setCreatedTime(cTime); + // add the info map in Timeline Entity + entity.addInfo(getInfoMap3()); + // add the isRelatedToEntity info + Set isRelatedToSet = new HashSet<>(); + isRelatedToSet.add("relatedto1"); + Map> isRelatedTo = new HashMap<>(); + isRelatedTo.put("task", isRelatedToSet); + entity.setIsRelatedToEntities(isRelatedTo); + // add the relatesTo info + Set relatesToSet = new HashSet<>(); + relatesToSet.add("relatesto1"); + relatesToSet.add("relatesto3"); + Map> relatesTo = new HashMap<>(); + relatesTo.put("container", relatesToSet); + Set relatesToSet11 = new HashSet<>(); + relatesToSet11.add("relatesto4"); + relatesTo.put("container1", relatesToSet11); + entity.setRelatesToEntities(relatesTo); + // add some config entries + Map conf = new HashMap<>(); + conf.put("config_param1", "value1"); + conf.put("config_param2", "value2"); + conf.put("cfg_param1", "value3"); + entity.addConfigs(conf); + // add metrics + Set metrics = new HashSet<>(); + metrics.add(getMetric4(ts)); + + TimelineMetric m12 = new TimelineMetric(); + m12.setId("MAP1_BYTES"); + m12.addValue(ts, 50); + metrics.add(m12); + entity.addMetrics(metrics); + entity.addEvent(addStartEvent(ts)); + te.addEntity(entity); + TimelineEntities te1 = new TimelineEntities(); + TimelineEntity entity1 = new TimelineEntity(); + String id1 = "application_1111111111_3333"; + entity1.setId(id1); + entity1.setType(TimelineEntityType.YARN_APPLICATION.toString()); + entity1.setCreatedTime(cTime + 20L); + // add the info map in Timeline Entity + entity1.addInfo(getInfoMap4()); + + // add the isRelatedToEntity info + Set isRelatedToSet1 = new HashSet<>(); + isRelatedToSet1.add("relatedto3"); + isRelatedToSet1.add("relatedto5"); + Map> isRelatedTo1 = new HashMap<>(); + isRelatedTo1.put("task1", isRelatedToSet1); + Set isRelatedToSet11 = new HashSet<>(); + isRelatedToSet11.add("relatedto4"); + isRelatedTo1.put("task2", isRelatedToSet11); + entity1.setIsRelatedToEntities(isRelatedTo1); + + // add the relatesTo info + Set relatesToSet1 = new HashSet<>(); + relatesToSet1.add("relatesto1"); + relatesToSet1.add("relatesto2"); + Map> relatesTo1 = new HashMap<>(); + relatesTo1.put("container", relatesToSet1); + entity1.setRelatesToEntities(relatesTo1); + + // add some config entries + Map conf1 = new HashMap<>(); + conf1.put("cfg_param1", "value1"); + conf1.put("cfg_param2", "value2"); + entity1.addConfigs(conf1); + + // add metrics + entity1.addMetrics(getMetrics4(ts)); + TimelineEvent event11 = new TimelineEvent(); + event11.setId("end_event"); + event11.setTimestamp(ts); + entity1.addEvent(event11); + TimelineEvent event12 = new TimelineEvent(); + event12.setId("update_event"); + event12.setTimestamp(ts - 10); + entity1.addEvent(event12); + te1.addEntity(entity1); + + TimelineEntities te2 = new TimelineEntities(); + te2.addEntity(getEntity4(cTime, ts)); + HBaseTimelineWriterImpl hbi = null; + try { + hbi = new HBaseTimelineWriterImpl(); + hbi.init(util.getConfiguration()); + hbi.start(); + UserGroupInformation remoteUser = + UserGroupInformation.createRemoteUser("user1"); + hbi.write( + new TimelineCollectorContext("cluster1", "user1", "some_flow_name", + "AB7822C10F1111", 1002345678919L, "application_1111111111_2222"), + te, remoteUser); + hbi.write( + new TimelineCollectorContext("cluster1", "user1", "some_flow_name", + "AB7822C10F1111", 1002345678919L, "application_1111111111_3333"), + te1, remoteUser); + hbi.write( + new TimelineCollectorContext("cluster1", "user1", "some_flow_name", + "AB7822C10F1111", 1002345678919L, "application_1111111111_4444"), + te2, remoteUser); + hbi.stop(); + } finally { + if (hbi != null) { + hbi.stop(); + hbi.close(); + } + } + } + + private static Set getMetrics4(long ts) { + Set metrics1 = new HashSet<>(); + TimelineMetric m2 = new TimelineMetric(); + m2.setId("MAP1_SLOT_MILLIS"); + Map metricValues1 = new HashMap<>(); + metricValues1.put(ts - 120000, 100000000); + metricValues1.put(ts - 100000, 200000000); + metricValues1.put(ts - 80000, 300000000); + metricValues1.put(ts - 60000, 400000000); + metricValues1.put(ts - 40000, 50000000000L); + metricValues1.put(ts - 20000, 60000000000L); + m2.setType(Type.TIME_SERIES); + m2.setValues(metricValues1); + metrics1.add(m2); + return metrics1; + } + + private static TimelineEntity getEntity4(long cTime, long ts) { + TimelineEntity entity2 = new TimelineEntity(); + String id2 = "application_1111111111_4444"; + entity2.setId(id2); + entity2.setType(TimelineEntityType.YARN_APPLICATION.toString()); + entity2.setCreatedTime(cTime + 40L); + TimelineEvent event21 = new TimelineEvent(); + event21.setId("update_event"); + event21.setTimestamp(ts - 20); + entity2.addEvent(event21); + Set isRelatedToSet2 = new HashSet(); + isRelatedToSet2.add("relatedto3"); + Map> isRelatedTo2 = new HashMap<>(); + isRelatedTo2.put("task1", isRelatedToSet2); + entity2.setIsRelatedToEntities(isRelatedTo2); + Map> relatesTo3 = new HashMap<>(); + Set relatesToSet14 = new HashSet(); + relatesToSet14.add("relatesto7"); + relatesTo3.put("container2", relatesToSet14); + entity2.setRelatesToEntities(relatesTo3); + return entity2; + } + + private static Map getInfoMap4() { + Map infoMap1 = new HashMap<>(); + infoMap1.put("infoMapKey1", "infoMapValue1"); + infoMap1.put("infoMapKey2", 10); + return infoMap1; + } + + private static TimelineMetric getMetric4(long ts) { + TimelineMetric m1 = new TimelineMetric(); + m1.setId("MAP_SLOT_MILLIS"); + Map metricValues = new HashMap<>(); + metricValues.put(ts - 120000, 100000000); + metricValues.put(ts - 100000, 200000000); + metricValues.put(ts - 80000, 300000000); + metricValues.put(ts - 60000, 400000000); + metricValues.put(ts - 40000, 50000000000L); + metricValues.put(ts - 20000, 60000000000L); + m1.setType(Type.TIME_SERIES); + m1.setValues(metricValues); + return m1; + } + + private static Map getInfoMap3() { + Map infoMap = new HashMap<>(); + infoMap.put("infoMapKey1", "infoMapValue2"); + infoMap.put("infoMapKey2", 20); + infoMap.put("infoMapKey3", 85.85); + return infoMap; + } + + private static Map getInfoMap1() { + Map infoMap = new HashMap<>(); + infoMap.put("infoMapKey1", "infoMapValue2"); + infoMap.put("infoMapKey2", 20); + infoMap.put("infoMapKey3", 71.4); + return infoMap; + } + + private static Map> getRelatesTo1() { + Set relatesToSet = new HashSet(); + relatesToSet.add("relatesto1"); + relatesToSet.add("relatesto3"); + Map> relatesTo = new HashMap<>(); + relatesTo.put("container", relatesToSet); + Set relatesToSet11 = new HashSet<>(); + relatesToSet11.add("relatesto4"); + relatesTo.put("container1", relatesToSet11); + return relatesTo; + } + + private static Map getConfig1() { + Map conf = new HashMap<>(); + conf.put("config_param1", "value1"); + conf.put("config_param2", "value2"); + conf.put("cfg_param1", "value3"); + return conf; + } + + private static Map getConfig2() { + Map conf1 = new HashMap<>(); + conf1.put("cfg_param1", "value1"); + conf1.put("cfg_param2", "value2"); + return conf1; + } + + private static Map getInfoMap2() { + Map infoMap1 = new HashMap<>(); + infoMap1.put("infoMapKey1", "infoMapValue1"); + infoMap1.put("infoMapKey2", 10); + return infoMap1; + } + + private static Map> getIsRelatedTo1() { + Set isRelatedToSet = new HashSet<>(); + isRelatedToSet.add("relatedto1"); + Map> isRelatedTo = new HashMap<>(); + isRelatedTo.put("task", isRelatedToSet); + return isRelatedTo; + } + + private static Map getMetricValues1(long ts) { + Map metricValues = new HashMap<>(); + metricValues.put(ts - 120000, 100000000); + metricValues.put(ts - 100000, 200000000); + metricValues.put(ts - 80000, 300000000); + metricValues.put(ts - 60000, 400000000); + metricValues.put(ts - 40000, 50000000000L); + metricValues.put(ts - 20000, 70000000000L); + return metricValues; + } + + public static void loadEntities(HBaseTestingUtility util, long ts) + throws IOException { + TimelineEntities te = new TimelineEntities(); + TimelineEntity entity = new TimelineEntity(); + String id = "hello"; + String type = "world"; + entity.setId(id); + entity.setType(type); + Long cTime = 1425016502000L; + entity.setCreatedTime(cTime); + // add the info map in Timeline Entity + entity.addInfo(getInfoMap1()); + // add the isRelatedToEntity info + entity.setIsRelatedToEntities(getIsRelatedTo1()); + + // add the relatesTo info + entity.setRelatesToEntities(getRelatesTo1()); + + // add some config entries + entity.addConfigs(getConfig1()); + + // add metrics + Set metrics = new HashSet<>(); + TimelineMetric m1 = new TimelineMetric(); + m1.setId("MAP_SLOT_MILLIS"); + m1.setType(Type.TIME_SERIES); + m1.setValues(getMetricValues1(ts)); + metrics.add(m1); + + TimelineMetric m12 = new TimelineMetric(); + m12.setId("MAP1_BYTES"); + m12.addValue(ts, 50); + metrics.add(m12); + entity.addMetrics(metrics); + entity.addEvent(addStartEvent(ts)); + te.addEntity(entity); + + TimelineEntity entity1 = new TimelineEntity(); + String id1 = "hello1"; + entity1.setId(id1); + entity1.setType(type); + entity1.setCreatedTime(cTime + 20L); + + // add the info map in Timeline Entity + entity1.addInfo(getInfoMap2()); + + // add event. + TimelineEvent event11 = new TimelineEvent(); + event11.setId("end_event"); + event11.setTimestamp(ts); + entity1.addEvent(event11); + TimelineEvent event12 = new TimelineEvent(); + event12.setId("update_event"); + event12.setTimestamp(ts - 10); + entity1.addEvent(event12); + + + // add the isRelatedToEntity info + entity1.setIsRelatedToEntities(getIsRelatedTo2()); + + // add the relatesTo info + Set relatesToSet1 = new HashSet(); + relatesToSet1.add("relatesto1"); + relatesToSet1.add("relatesto2"); + Map> relatesTo1 = new HashMap<>(); + relatesTo1.put("container", relatesToSet1); + entity1.setRelatesToEntities(relatesTo1); + + // add some config entries + entity1.addConfigs(getConfig2()); + + // add metrics + Set metrics1 = new HashSet<>(); + TimelineMetric m2 = new TimelineMetric(); + m2.setId("MAP1_SLOT_MILLIS"); + m2.setType(Type.TIME_SERIES); + m2.setValues(getMetricValues2(ts)); + metrics1.add(m2); + entity1.addMetrics(metrics1); + te.addEntity(entity1); + + te.addEntity(getEntity2(type, cTime, ts)); + + // For listing types + for (int i = 0; i < 10; i++) { + TimelineEntity entity3 = new TimelineEntity(); + String id3 = "typeTest" + i; + entity3.setId(id3); + StringBuilder typeName = new StringBuilder("newType"); + for (int j = 0; j < (i % 3); j++) { + typeName.append(" ").append(j); + } + entity3.setType(typeName.toString()); + entity3.setCreatedTime(cTime + 80L + i); + te.addEntity(entity3); + } + + // Create app entity for app to flow table + TimelineEntities appTe1 = new TimelineEntities(); + TimelineEntity entityApp1 = new TimelineEntity(); + String appName1 = "application_1231111111_1111"; + entityApp1.setId(appName1); + entityApp1.setType(TimelineEntityType.YARN_APPLICATION.toString()); + entityApp1.setCreatedTime(cTime + 40L); + TimelineEvent appCreationEvent1 = new TimelineEvent(); + appCreationEvent1.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE); + appCreationEvent1.setTimestamp(cTime); + entityApp1.addEvent(appCreationEvent1); + appTe1.addEntity(entityApp1); + + TimelineEntities appTe2 = new TimelineEntities(); + TimelineEntity entityApp2 = new TimelineEntity(); + String appName2 = "application_1231111111_1112"; + entityApp2.setId(appName2); + entityApp2.setType(TimelineEntityType.YARN_APPLICATION.toString()); + entityApp2.setCreatedTime(cTime + 50L); + TimelineEvent appCreationEvent2 = new TimelineEvent(); + appCreationEvent2.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE); + appCreationEvent2.setTimestamp(cTime); + entityApp2.addEvent(appCreationEvent2); + appTe2.addEntity(entityApp2); + + HBaseTimelineWriterImpl hbi = null; + try { + hbi = new HBaseTimelineWriterImpl(); + hbi.init(util.getConfiguration()); + hbi.start(); + + UserGroupInformation user = + UserGroupInformation.createRemoteUser("user1"); + TimelineCollectorContext context = + new TimelineCollectorContext("cluster1", "user1", "some_flow_name", + "AB7822C10F1111", 1002345678919L, appName1); + hbi.write(context, te, user); + hbi.write(context, appTe1, user); + + context = new TimelineCollectorContext("cluster1", "user1", + "some_flow_name", "AB7822C10F1111", 1002345678919L, appName2); + hbi.write(context, te, user); + hbi.write(context, appTe2, user); + hbi.stop(); + } finally { + if (hbi != null) { + hbi.stop(); + hbi.close(); + } + } + } + + private static TimelineEntity getEntity2(String type, long cTime, + long ts) { + TimelineEntity entity2 = new TimelineEntity(); + String id2 = "hello2"; + entity2.setId(id2); + entity2.setType(type); + entity2.setCreatedTime(cTime + 40L); + TimelineEvent event21 = new TimelineEvent(); + event21.setId("update_event"); + event21.setTimestamp(ts - 20); + entity2.addEvent(event21); + Set isRelatedToSet2 = new HashSet<>(); + isRelatedToSet2.add("relatedto3"); + Map> isRelatedTo2 = new HashMap<>(); + isRelatedTo2.put("task1", isRelatedToSet2); + entity2.setIsRelatedToEntities(isRelatedTo2); + Map> relatesTo3 = new HashMap<>(); + Set relatesToSet14 = new HashSet<>(); + relatesToSet14.add("relatesto7"); + relatesTo3.put("container2", relatesToSet14); + entity2.setRelatesToEntities(relatesTo3); + return entity2; + } + + private static TimelineEvent addStartEvent(long ts) { + TimelineEvent event = new TimelineEvent(); + event.setId("start_event"); + event.setTimestamp(ts); + return event; + } + + private static Map getMetricValues2(long ts1) { + Map metricValues1 = new HashMap<>(); + metricValues1.put(ts1 - 120000, 100000000); + metricValues1.put(ts1 - 100000, 200000000); + metricValues1.put(ts1 - 80000, 300000000); + metricValues1.put(ts1 - 60000, 400000000); + metricValues1.put(ts1 - 40000, 50000000000L); + metricValues1.put(ts1 - 20000, 60000000000L); + return metricValues1; + } + + private static Map> getIsRelatedTo2() { + Set isRelatedToSet1 = new HashSet<>(); + isRelatedToSet1.add("relatedto3"); + isRelatedToSet1.add("relatedto5"); + Map> isRelatedTo1 = new HashMap<>(); + isRelatedTo1.put("task1", isRelatedToSet1); + Set isRelatedToSet11 = new HashSet<>(); + isRelatedToSet11.add("relatedto4"); + isRelatedTo1.put("task2", isRelatedToSet11); + return isRelatedTo1; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java new file mode 100644 index 00000000000..111008a5d4c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java @@ -0,0 +1,1941 @@ +/** + * 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.timelineservice.storage; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.Arrays; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.NavigableMap; +import java.util.NavigableSet; +import java.util.Set; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetricOperation; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric.Type; +import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants; +import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValuesFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList.Operator; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumn; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnName; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnNameConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +/** + * Tests for apps stored in TimelineStorage. + */ +public class TestHBaseTimelineStorageApps { + + private static HBaseTestingUtility util; + private HBaseTimelineReaderImpl reader; + private static final long CURRENT_TIME = System.currentTimeMillis(); + + @BeforeClass + public static void setupBeforeClass() throws Exception { + util = new HBaseTestingUtility(); + util.startMiniCluster(); + DataGeneratorForTest.createSchema(util.getConfiguration()); + DataGeneratorForTest.loadApps(util, CURRENT_TIME); + } + + @Before + public void init() throws Exception { + reader = new HBaseTimelineReaderImpl(); + reader.init(util.getConfiguration()); + reader.start(); + } + + @After + public void stop() throws Exception { + if (reader != null) { + reader.stop(); + reader.close(); + } + } + + private static void matchMetrics(Map m1, Map m2) { + assertEquals(m1.size(), m2.size()); + for (Map.Entry entry : m2.entrySet()) { + Number val = m1.get(entry.getKey()); + assertNotNull(val); + assertEquals(val.longValue(), entry.getValue().longValue()); + } + } + + private boolean isApplicationRowKeyCorrect(byte[] rowKey, String cluster, + String user, String flow, Long runid, String appName) { + + ApplicationRowKey key = ApplicationRowKey.parseRowKey(rowKey); + + assertEquals(cluster, key.getClusterId()); + assertEquals(user, key.getUserId()); + assertEquals(flow, key.getFlowName()); + assertEquals(runid, key.getFlowRunId()); + assertEquals(appName, key.getAppId()); + return true; + } + + @Test + public void testWriteNullApplicationToHBase() throws Exception { + TimelineEntities te = new TimelineEntities(); + ApplicationEntity entity = new ApplicationEntity(); + String appId = "application_1000178881110_2002"; + entity.setId(appId); + long cTime = 1425016501000L; + entity.setCreatedTime(cTime); + + // add the info map in Timeline Entity + Map infoMap = new HashMap(); + infoMap.put("in fo M apK ey1", "infoMapValue1"); + infoMap.put("infoMapKey2", 10); + entity.addInfo(infoMap); + + te.addEntity(entity); + HBaseTimelineWriterImpl hbi = null; + try { + Configuration c1 = util.getConfiguration(); + hbi = new HBaseTimelineWriterImpl(); + hbi.init(c1); + hbi.start(); + String cluster = "cluster_check_null_application"; + String user = "user1check_null_application"; + //set the flow name to null + String flow = null; + String flowVersion = "AB7822C10F1111"; + long runid = 1002345678919L; + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, appId), te, UserGroupInformation.createRemoteUser(user)); + hbi.stop(); + + // retrieve the row + Scan scan = new Scan(); + scan.setStartRow(Bytes.toBytes(cluster)); + scan.setStopRow(Bytes.toBytes(cluster + "1")); + Connection conn = ConnectionFactory.createConnection(c1); + ResultScanner resultScanner = new ApplicationTable() + .getResultScanner(c1, conn, scan); + + assertTrue(resultScanner != null); + // try to iterate over results + int count = 0; + for (Result rr = resultScanner.next(); rr != null; + rr = resultScanner.next()) { + count++; + } + // there should be no rows written + // no exceptions thrown during write + assertEquals(0, count); + } finally { + if (hbi != null) { + hbi.stop(); + hbi.close(); + } + } + } + + @Test + public void testWriteApplicationToHBase() throws Exception { + TimelineEntities te = new TimelineEntities(); + ApplicationEntity entity = new ApplicationEntity(); + String appId = "application_1000178881110_2002"; + entity.setId(appId); + Long cTime = 1425016501000L; + entity.setCreatedTime(cTime); + + // add the info map in Timeline Entity + Map infoMap = new HashMap(); + infoMap.put("infoMapKey1", "infoMapValue1"); + infoMap.put("infoMapKey2", 10); + entity.addInfo(infoMap); + + // add the isRelatedToEntity info + String key = "task"; + String value = "is_related_to_entity_id_here"; + Set isRelatedToSet = new HashSet(); + isRelatedToSet.add(value); + Map> isRelatedTo = new HashMap>(); + isRelatedTo.put(key, isRelatedToSet); + entity.setIsRelatedToEntities(isRelatedTo); + + // add the relatesTo info + key = "container"; + value = "relates_to_entity_id_here"; + Set relatesToSet = new HashSet(); + relatesToSet.add(value); + value = "relates_to_entity_id_here_Second"; + relatesToSet.add(value); + Map> relatesTo = new HashMap>(); + relatesTo.put(key, relatesToSet); + entity.setRelatesToEntities(relatesTo); + + // add some config entries + Map conf = new HashMap(); + conf.put("config_param1", "value1"); + conf.put("config_param2", "value2"); + entity.addConfigs(conf); + + // add metrics + Set metrics = new HashSet<>(); + TimelineMetric m1 = new TimelineMetric(); + m1.setId("MAP_SLOT_MILLIS"); + Map metricValues = new HashMap(); + metricValues.put(CURRENT_TIME - 120000, 100000000); + metricValues.put(CURRENT_TIME - 100000, 200000000); + metricValues.put(CURRENT_TIME - 80000, 300000000); + metricValues.put(CURRENT_TIME - 60000, 400000000); + metricValues.put(CURRENT_TIME - 40000, 50000000000L); + metricValues.put(CURRENT_TIME - 20000, 60000000000L); + m1.setType(Type.TIME_SERIES); + m1.setValues(metricValues); + metrics.add(m1); + entity.addMetrics(metrics); + + // add aggregated metrics + TimelineEntity aggEntity = new TimelineEntity(); + String type = TimelineEntityType.YARN_APPLICATION.toString(); + aggEntity.setId(appId); + aggEntity.setType(type); + long cTime2 = 1425016502000L; + aggEntity.setCreatedTime(cTime2); + + TimelineMetric aggMetric = new TimelineMetric(); + aggMetric.setId("MEM_USAGE"); + Map aggMetricValues = new HashMap(); + long aggTs = CURRENT_TIME; + aggMetricValues.put(aggTs - 120000, 102400000L); + aggMetric.setType(Type.SINGLE_VALUE); + aggMetric.setRealtimeAggregationOp(TimelineMetricOperation.SUM); + aggMetric.setValues(aggMetricValues); + Set aggMetrics = new HashSet<>(); + aggMetrics.add(aggMetric); + entity.addMetrics(aggMetrics); + te.addEntity(entity); + + HBaseTimelineWriterImpl hbi = null; + try { + Configuration c1 = util.getConfiguration(); + hbi = new HBaseTimelineWriterImpl(); + hbi.init(c1); + hbi.start(); + String cluster = "cluster_test_write_app"; + String user = "user1"; + String flow = "s!ome_f\tlow _n am!e"; + String flowVersion = "AB7822C10F1111"; + long runid = 1002345678919L; + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, appId), te, UserGroupInformation.createRemoteUser(user)); + + // Write entity again, this time without created time. + entity = new ApplicationEntity(); + appId = "application_1000178881110_2002"; + entity.setId(appId); + // add the info map in Timeline Entity + Map infoMap1 = new HashMap<>(); + infoMap1.put("infoMapKey3", "infoMapValue1"); + entity.addInfo(infoMap1); + te = new TimelineEntities(); + te.addEntity(entity); + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, appId), te, UserGroupInformation.createRemoteUser(user)); + hbi.stop(); + + infoMap.putAll(infoMap1); + // retrieve the row + ApplicationRowKey applicationRowKey = + new ApplicationRowKey(cluster, user, flow, runid, appId); + byte[] rowKey = applicationRowKey.getRowKey(); + Get get = new Get(rowKey); + get.setMaxVersions(Integer.MAX_VALUE); + Connection conn = ConnectionFactory.createConnection(c1); + Result result = new ApplicationTable().getResult(c1, conn, get); + + assertTrue(result != null); + assertEquals(17, result.size()); + + // check the row key + byte[] row1 = result.getRow(); + assertTrue(isApplicationRowKeyCorrect(row1, cluster, user, flow, runid, + appId)); + + // check info column family + String id1 = ApplicationColumn.ID.readResult(result).toString(); + assertEquals(appId, id1); + + Long cTime1 = + (Long) ApplicationColumn.CREATED_TIME.readResult(result); + assertEquals(cTime, cTime1); + + Map infoColumns = + ApplicationColumnPrefix.INFO.readResults(result, + new StringKeyConverter()); + assertEquals(infoMap, infoColumns); + + // Remember isRelatedTo is of type Map> + for (Map.Entry> isRelatedToEntry : isRelatedTo + .entrySet()) { + Object isRelatedToValue = + ApplicationColumnPrefix.IS_RELATED_TO.readResult(result, + isRelatedToEntry.getKey()); + String compoundValue = isRelatedToValue.toString(); + // id7?id9?id6 + Set isRelatedToValues = + new HashSet(Separator.VALUES.splitEncoded(compoundValue)); + assertEquals(isRelatedTo.get(isRelatedToEntry.getKey()).size(), + isRelatedToValues.size()); + for (String v : isRelatedToEntry.getValue()) { + assertTrue(isRelatedToValues.contains(v)); + } + } + + // RelatesTo + for (Map.Entry> relatesToEntry : relatesTo + .entrySet()) { + String compoundValue = + ApplicationColumnPrefix.RELATES_TO.readResult(result, + relatesToEntry.getKey()).toString(); + // id3?id4?id5 + Set relatesToValues = + new HashSet(Separator.VALUES.splitEncoded(compoundValue)); + assertEquals(relatesTo.get(relatesToEntry.getKey()).size(), + relatesToValues.size()); + for (String v : relatesToEntry.getValue()) { + assertTrue(relatesToValues.contains(v)); + } + } + + KeyConverter stringKeyConverter = new StringKeyConverter(); + // Configuration + Map configColumns = + ApplicationColumnPrefix.CONFIG + .readResults(result, stringKeyConverter); + assertEquals(conf, configColumns); + + NavigableMap> metricsResult = + ApplicationColumnPrefix.METRIC.readResultsWithTimestamps(result, + stringKeyConverter); + + NavigableMap metricMap = metricsResult.get(m1.getId()); + matchMetrics(metricValues, metricMap); + + // read the timeline entity using the reader this time. In metrics limit + // specify Integer MAX_VALUE. A TIME_SERIES will be returned(if more than + // one value exists for a metric). + TimelineEntity e1 = reader.getEntity( + new TimelineReaderContext(cluster, user, flow, runid, appId, + entity.getType(), entity.getId()), + new TimelineDataToRetrieve(null, null, + EnumSet.of(TimelineReader.Field.ALL), Integer.MAX_VALUE, null, null)); + assertNotNull(e1); + + // verify attributes + assertEquals(appId, e1.getId()); + assertEquals(TimelineEntityType.YARN_APPLICATION.toString(), + e1.getType()); + assertEquals(cTime, e1.getCreatedTime()); + Map infoMap2 = e1.getInfo(); + // fromid key is added by storage. Remove it for comparision. + infoMap2.remove("FROM_ID"); + assertEquals(infoMap, infoMap2); + + Map> isRelatedTo2 = e1.getIsRelatedToEntities(); + assertEquals(isRelatedTo, isRelatedTo2); + + Map> relatesTo2 = e1.getRelatesToEntities(); + assertEquals(relatesTo, relatesTo2); + + Map conf2 = e1.getConfigs(); + assertEquals(conf, conf2); + + Set metrics2 = e1.getMetrics(); + assertEquals(2, metrics2.size()); + for (TimelineMetric metric2 : metrics2) { + Map metricValues2 = metric2.getValues(); + assertTrue(metric2.getId().equals("MAP_SLOT_MILLIS") || + metric2.getId().equals("MEM_USAGE")); + if (metric2.getId().equals("MAP_SLOT_MILLIS")) { + assertEquals(6, metricValues2.size()); + matchMetrics(metricValues, metricValues2); + } + if (metric2.getId().equals("MEM_USAGE")) { + assertEquals(1, metricValues2.size()); + matchMetrics(aggMetricValues, metricValues2); + } + } + + // In metrics limit specify a value of 3. No more than 3 values for a + // metric will be returned. + e1 = reader.getEntity(new TimelineReaderContext(cluster, user, flow, + runid, appId, entity.getType(), entity.getId()), + new TimelineDataToRetrieve(null, null, + EnumSet.of(TimelineReader.Field.ALL), 3, null, null)); + assertNotNull(e1); + assertEquals(appId, e1.getId()); + assertEquals(TimelineEntityType.YARN_APPLICATION.toString(), + e1.getType()); + assertEquals(conf, e1.getConfigs()); + metrics2 = e1.getMetrics(); + assertEquals(2, metrics2.size()); + for (TimelineMetric metric2 : metrics2) { + Map metricValues2 = metric2.getValues(); + assertTrue(metricValues2.size() <= 3); + assertTrue(metric2.getId().equals("MAP_SLOT_MILLIS") || + metric2.getId().equals("MEM_USAGE")); + } + + // Check if single value(latest value) instead of time series is returned + // if metricslimit is not set(null), irrespective of number of metric + // values. + e1 = reader.getEntity( + new TimelineReaderContext(cluster, user, flow, runid, appId, + entity.getType(), entity.getId()), new TimelineDataToRetrieve( + null, null, EnumSet.of(TimelineReader.Field.ALL), null, null, null)); + assertNotNull(e1); + assertEquals(appId, e1.getId()); + assertEquals(TimelineEntityType.YARN_APPLICATION.toString(), + e1.getType()); + assertEquals(cTime, e1.getCreatedTime()); + infoMap2 = e1.getInfo(); + // fromid key is added by storage. Remove it for comparison. + infoMap2.remove("FROM_ID"); + assertEquals(infoMap, e1.getInfo()); + assertEquals(isRelatedTo, e1.getIsRelatedToEntities()); + assertEquals(relatesTo, e1.getRelatesToEntities()); + assertEquals(conf, e1.getConfigs()); + assertEquals(2, e1.getMetrics().size()); + for (TimelineMetric metric : e1.getMetrics()) { + assertEquals(1, metric.getValues().size()); + assertEquals(TimelineMetric.Type.SINGLE_VALUE, metric.getType()); + assertTrue(metric.getId().equals("MAP_SLOT_MILLIS") || + metric.getId().equals("MEM_USAGE")); + assertEquals(1, metric.getValues().size()); + if (metric.getId().equals("MAP_SLOT_MILLIS")) { + assertTrue(metric.getValues().containsKey(CURRENT_TIME - 20000)); + assertEquals(metricValues.get(CURRENT_TIME - 20000), + metric.getValues().get(CURRENT_TIME - 20000)); + } + if (metric.getId().equals("MEM_USAGE")) { + assertTrue(metric.getValues().containsKey(aggTs - 120000)); + assertEquals(aggMetricValues.get(aggTs - 120000), + metric.getValues().get(aggTs - 120000)); + } + } + } finally { + if (hbi != null) { + hbi.stop(); + hbi.close(); + } + } + } + + @Test + public void testEvents() throws IOException { + TimelineEvent event = new TimelineEvent(); + String eventId = ApplicationMetricsConstants.CREATED_EVENT_TYPE; + event.setId(eventId); + Long expTs = 1436512802000L; + event.setTimestamp(expTs); + String expKey = "foo_event"; + Object expVal = "test"; + event.addInfo(expKey, expVal); + + final TimelineEntity entity = new ApplicationEntity(); + entity.setId(HBaseTimelineStorageUtils.convertApplicationIdToString( + ApplicationId.newInstance(0, 1))); + entity.addEvent(event); + + TimelineEntities entities = new TimelineEntities(); + entities.addEntity(entity); + + HBaseTimelineWriterImpl hbi = null; + try { + Configuration c1 = util.getConfiguration(); + hbi = new HBaseTimelineWriterImpl(); + hbi.init(c1); + hbi.start(); + String cluster = "cluster_test_events"; + String user = "user2"; + String flow = "other_flow_name"; + String flowVersion = "1111F01C2287BA"; + long runid = 1009876543218L; + String appName = "application_123465899910_1001"; + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, appName), entities, + UserGroupInformation.createRemoteUser(user)); + hbi.stop(); + + // retrieve the row + ApplicationRowKey applicationRowKey = + new ApplicationRowKey(cluster, user, flow, runid, appName); + byte[] rowKey = applicationRowKey.getRowKey(); + Get get = new Get(rowKey); + get.setMaxVersions(Integer.MAX_VALUE); + Connection conn = ConnectionFactory.createConnection(c1); + Result result = new ApplicationTable().getResult(c1, conn, get); + + assertTrue(result != null); + + // check the row key + byte[] row1 = result.getRow(); + assertTrue(isApplicationRowKeyCorrect(row1, cluster, user, flow, runid, + appName)); + + Map eventsResult = + ApplicationColumnPrefix.EVENT.readResults(result, + new EventColumnNameConverter()); + // there should be only one event + assertEquals(1, eventsResult.size()); + for (Map.Entry e : eventsResult.entrySet()) { + EventColumnName eventColumnName = e.getKey(); + // the qualifier is a compound key + // hence match individual values + assertEquals(eventId, eventColumnName.getId()); + assertEquals(expTs, eventColumnName.getTimestamp()); + assertEquals(expKey, eventColumnName.getInfoKey()); + Object value = e.getValue(); + // there should be only one timestamp and value + assertEquals(expVal, value.toString()); + } + + // read the timeline entity using the reader this time + TimelineEntity e1 = reader.getEntity( + new TimelineReaderContext(cluster, user, flow, runid, appName, + entity.getType(), entity.getId()), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null, + null, null)); + TimelineEntity e2 = reader.getEntity( + new TimelineReaderContext(cluster, user, null, null, appName, + entity.getType(), entity.getId()), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null, + null, null)); + assertNotNull(e1); + assertNotNull(e2); + assertEquals(e1, e2); + + // check the events + NavigableSet events = e1.getEvents(); + // there should be only one event + assertEquals(1, events.size()); + for (TimelineEvent e : events) { + assertEquals(eventId, e.getId()); + assertEquals(expTs, Long.valueOf(e.getTimestamp())); + Map info = e.getInfo(); + assertEquals(1, info.size()); + for (Map.Entry infoEntry : info.entrySet()) { + assertEquals(expKey, infoEntry.getKey()); + assertEquals(expVal, infoEntry.getValue()); + } + } + } finally { + if (hbi != null) { + hbi.stop(); + hbi.close(); + } + } + } + + @Test + public void testNonIntegralMetricValues() throws IOException { + TimelineEntities teApp = new TimelineEntities(); + ApplicationEntity entityApp = new ApplicationEntity(); + String appId = "application_1000178881110_2002"; + entityApp.setId(appId); + entityApp.setCreatedTime(1425016501000L); + // add metrics with floating point values + Set metricsApp = new HashSet<>(); + TimelineMetric mApp = new TimelineMetric(); + mApp.setId("MAP_SLOT_MILLIS"); + Map metricAppValues = new HashMap(); + long ts = System.currentTimeMillis(); + metricAppValues.put(ts - 20, 10.5); + metricAppValues.put(ts - 10, 20.5); + mApp.setType(Type.TIME_SERIES); + mApp.setValues(metricAppValues); + metricsApp.add(mApp); + entityApp.addMetrics(metricsApp); + teApp.addEntity(entityApp); + + TimelineEntities teEntity = new TimelineEntities(); + TimelineEntity entity = new TimelineEntity(); + entity.setId("hello"); + entity.setType("world"); + entity.setCreatedTime(1425016501000L); + // add metrics with floating point values + Set metricsEntity = new HashSet<>(); + TimelineMetric mEntity = new TimelineMetric(); + mEntity.setId("MAP_SLOT_MILLIS"); + mEntity.addValue(ts - 20, 10.5); + metricsEntity.add(mEntity); + entity.addMetrics(metricsEntity); + teEntity.addEntity(entity); + + HBaseTimelineWriterImpl hbi = null; + try { + Configuration c1 = util.getConfiguration(); + hbi = new HBaseTimelineWriterImpl(); + hbi.init(c1); + hbi.start(); + // Writing application entity. + TimelineCollectorContext context = new TimelineCollectorContext("c1", + "u1", "f1", "v1", 1002345678919L, appId); + UserGroupInformation user = UserGroupInformation.createRemoteUser("u1"); + try { + hbi.write(context, teApp, user); + Assert.fail("Expected an exception as metric values are non integral"); + } catch (IOException e) {} + + // Writing generic entity. + try { + hbi.write(context, teEntity, user); + Assert.fail("Expected an exception as metric values are non integral"); + } catch (IOException e) {} + hbi.stop(); + } finally { + if (hbi != null) { + hbi.stop(); + hbi.close(); + } + } + } + + @Test + public void testReadApps() throws Exception { + TimelineEntity entity = reader.getEntity( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1111111111_2222", + TimelineEntityType.YARN_APPLICATION.toString(), null), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null, + null, null)); + assertNotNull(entity); + assertEquals(3, entity.getConfigs().size()); + assertEquals(1, entity.getIsRelatedToEntities().size()); + Set entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null, + null, null)); + assertEquals(3, entities.size()); + int cfgCnt = 0; + int metricCnt = 0; + int infoCnt = 0; + int eventCnt = 0; + int relatesToCnt = 0; + int isRelatedToCnt = 0; + for (TimelineEntity timelineEntity : entities) { + cfgCnt += (timelineEntity.getConfigs() == null) ? 0 : + timelineEntity.getConfigs().size(); + metricCnt += (timelineEntity.getMetrics() == null) ? 0 : + timelineEntity.getMetrics().size(); + infoCnt += (timelineEntity.getInfo() == null) ? 0 : + timelineEntity.getInfo().size(); + eventCnt += (timelineEntity.getEvents() == null) ? 0 : + timelineEntity.getEvents().size(); + relatesToCnt += (timelineEntity.getRelatesToEntities() == null) ? 0 : + timelineEntity.getRelatesToEntities().size(); + isRelatedToCnt += (timelineEntity.getIsRelatedToEntities() == null) ? 0 : + timelineEntity.getIsRelatedToEntities().size(); + } + assertEquals(5, cfgCnt); + assertEquals(3, metricCnt); + assertEquals(8, infoCnt); + assertEquals(4, eventCnt); + assertEquals(4, relatesToCnt); + assertEquals(4, isRelatedToCnt); + } + + @Test + public void testFilterAppsByCreatedTime() throws Exception { + Set entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().createdTimeBegin(1425016502000L) + .createTimeEnd(1425016502040L).build(), + new TimelineDataToRetrieve()); + assertEquals(3, entities.size()); + for (TimelineEntity entity : entities) { + if (!entity.getId().equals("application_1111111111_2222") && + !entity.getId().equals("application_1111111111_3333") && + !entity.getId().equals("application_1111111111_4444")) { + Assert.fail("Entities with ids' application_1111111111_2222, " + + "application_1111111111_3333 and application_1111111111_4444" + + " should be present"); + } + } + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().createdTimeBegin(1425016502015L) + .build(), + new TimelineDataToRetrieve()); + assertEquals(2, entities.size()); + for (TimelineEntity entity : entities) { + if (!entity.getId().equals("application_1111111111_3333") && + !entity.getId().equals("application_1111111111_4444")) { + Assert.fail("Apps with ids' application_1111111111_3333 and" + + " application_1111111111_4444 should be present"); + } + } + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().createTimeEnd(1425016502015L) + .build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + for (TimelineEntity entity : entities) { + if (!entity.getId().equals("application_1111111111_2222")) { + Assert.fail("App with id application_1111111111_2222 should" + + " be present"); + } + } + } + + @Test + public void testReadAppsDefaultView() throws Exception { + TimelineEntity e1 = reader.getEntity( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1111111111_2222", + TimelineEntityType.YARN_APPLICATION.toString(), null), + new TimelineDataToRetrieve()); + assertNotNull(e1); + assertEquals(1, e1.getInfo().size()); + assertTrue(e1.getConfigs().isEmpty() && + e1.getMetrics().isEmpty() && e1.getIsRelatedToEntities().isEmpty() && + e1.getRelatesToEntities().isEmpty()); + Set es1 = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().build(), + new TimelineDataToRetrieve()); + assertEquals(3, es1.size()); + for (TimelineEntity e : es1) { + assertEquals(1, e1.getInfo().size()); + assertTrue(e.getConfigs().isEmpty() && + e.getMetrics().isEmpty() && e.getIsRelatedToEntities().isEmpty() && + e.getRelatesToEntities().isEmpty()); + } + } + + @Test + public void testReadAppsByFields() throws Exception { + TimelineEntity e1 = reader.getEntity( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1111111111_2222", + TimelineEntityType.YARN_APPLICATION.toString(), null), + new TimelineDataToRetrieve( + null, null, EnumSet.of(Field.INFO, Field.CONFIGS), null, null, null)); + assertNotNull(e1); + assertEquals(3, e1.getConfigs().size()); + assertEquals(0, e1.getIsRelatedToEntities().size()); + Set es1 = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), new TimelineEntityFilters.Builder().build(), + new TimelineDataToRetrieve( + null, null, EnumSet.of(Field.IS_RELATED_TO, Field.METRICS), null, + null, null)); + assertEquals(3, es1.size()); + int metricsCnt = 0; + int isRelatedToCnt = 0; + int infoCnt = 0; + for (TimelineEntity entity : es1) { + metricsCnt += entity.getMetrics().size(); + isRelatedToCnt += entity.getIsRelatedToEntities().size(); + infoCnt += entity.getInfo().size(); + } + assertEquals(3, infoCnt); + assertEquals(4, isRelatedToCnt); + assertEquals(3, metricsCnt); + } + + @Test + public void testReadAppsIsRelatedTo() throws Exception { + TimelineFilterList irt = new TimelineFilterList(Operator.OR); + irt.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "task", + new HashSet(Arrays.asList("relatedto1")))); + irt.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "task2", + new HashSet(Arrays.asList("relatedto4")))); + Set entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().isRelatedTo(irt).build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null, + null, null)); + assertEquals(2, entities.size()); + int isRelatedToCnt = 0; + for (TimelineEntity timelineEntity : entities) { + isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size(); + if (!timelineEntity.getId().equals("application_1111111111_2222") && + !timelineEntity.getId().equals("application_1111111111_3333")) { + Assert.fail("Entity ids' should have been application_1111111111_2222" + + " and application_1111111111_3333"); + } + } + assertEquals(3, isRelatedToCnt); + + TimelineFilterList irt1 = new TimelineFilterList(); + irt1.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "task1", + new HashSet(Arrays.asList("relatedto3")))); + irt1.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.NOT_EQUAL, "task1", + new HashSet(Arrays.asList("relatedto5")))); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().isRelatedTo(irt1).build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + isRelatedToCnt = 0; + for (TimelineEntity timelineEntity : entities) { + isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size(); + if (!timelineEntity.getId().equals("application_1111111111_4444")) { + Assert.fail("Entity id should have been application_1111111111_4444"); + } + } + assertEquals(0, isRelatedToCnt); + + TimelineFilterList irt2 = new TimelineFilterList(Operator.OR); + irt2.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "task", + new HashSet(Arrays.asList("relatedto1")))); + irt2.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "task2", + new HashSet(Arrays.asList("relatedto4")))); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().isRelatedTo(irt2).build(), + new TimelineDataToRetrieve()); + assertEquals(2, entities.size()); + isRelatedToCnt = 0; + for (TimelineEntity timelineEntity : entities) { + isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size(); + if (!timelineEntity.getId().equals("application_1111111111_2222") && + !timelineEntity.getId().equals("application_1111111111_3333")) { + Assert.fail("Entity ids' should have been application_1111111111_2222" + + " and application_1111111111_3333"); + } + } + assertEquals(0, isRelatedToCnt); + + TimelineFilterList irt3 = new TimelineFilterList(); + irt3.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "task1", + new HashSet(Arrays.asList("relatedto3", "relatedto5")))); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().isRelatedTo(irt3).build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + isRelatedToCnt = 0; + for (TimelineEntity timelineEntity : entities) { + isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size(); + if (!timelineEntity.getId().equals("application_1111111111_3333")) { + Assert.fail("Entity id should have been application_1111111111_3333"); + } + } + assertEquals(0, isRelatedToCnt); + + TimelineFilterList irt4 = new TimelineFilterList(); + irt4.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "task1", + new HashSet(Arrays.asList("relatedto3")))); + irt4.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "dummy_task", + new HashSet(Arrays.asList("relatedto5")))); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().isRelatedTo(irt4).build(), + new TimelineDataToRetrieve()); + assertEquals(0, entities.size()); + + TimelineFilterList irt5 = new TimelineFilterList(); + irt5.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "task1", + new HashSet(Arrays.asList("relatedto3", "relatedto7")))); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().isRelatedTo(irt5).build(), + new TimelineDataToRetrieve()); + assertEquals(0, entities.size()); + + TimelineFilterList list1 = new TimelineFilterList(); + list1.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "task", + new HashSet(Arrays.asList("relatedto1")))); + list1.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "dummy_task", + new HashSet(Arrays.asList("relatedto4")))); + TimelineFilterList list2 = new TimelineFilterList(); + list2.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "task2", + new HashSet(Arrays.asList("relatedto4")))); + TimelineFilterList irt6 = new TimelineFilterList(Operator.OR, list1, list2); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().isRelatedTo(irt6).build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + isRelatedToCnt = 0; + for (TimelineEntity timelineEntity : entities) { + isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size(); + if (!timelineEntity.getId().equals("application_1111111111_3333")) { + Assert.fail("Entity id should have been application_1111111111_3333"); + } + } + assertEquals(0, isRelatedToCnt); + } + + + @Test + public void testReadAppsRelatesTo() throws Exception { + TimelineFilterList rt = new TimelineFilterList(Operator.OR); + rt.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container2", + new HashSet(Arrays.asList("relatesto7")))); + rt.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container1", + new HashSet(Arrays.asList("relatesto4")))); + Set entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().relatesTo(rt).build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null, + null, null)); + assertEquals(2, entities.size()); + int relatesToCnt = 0; + for (TimelineEntity timelineEntity : entities) { + relatesToCnt += timelineEntity.getRelatesToEntities().size(); + if (!timelineEntity.getId().equals("application_1111111111_2222") && + !timelineEntity.getId().equals("application_1111111111_4444")) { + Assert.fail("Entity ids' should have been application_1111111111_2222" + + " and application_1111111111_4444"); + } + } + assertEquals(3, relatesToCnt); + + TimelineFilterList rt1 = new TimelineFilterList(); + rt1.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container", + new HashSet(Arrays.asList("relatesto1")))); + rt1.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.NOT_EQUAL, "container", + new HashSet(Arrays.asList("relatesto3")))); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().relatesTo(rt1).build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + relatesToCnt = 0; + for (TimelineEntity timelineEntity : entities) { + relatesToCnt += timelineEntity.getRelatesToEntities().size(); + if (!timelineEntity.getId().equals("application_1111111111_3333")) { + Assert.fail("Entity id should have been application_1111111111_3333"); + } + } + assertEquals(0, relatesToCnt); + + TimelineFilterList rt2 = new TimelineFilterList(Operator.OR); + rt2.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container2", + new HashSet(Arrays.asList("relatesto7")))); + rt2.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container1", + new HashSet(Arrays.asList("relatesto4")))); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().relatesTo(rt2).build(), + new TimelineDataToRetrieve()); + assertEquals(2, entities.size()); + relatesToCnt = 0; + for (TimelineEntity timelineEntity : entities) { + relatesToCnt += timelineEntity.getRelatesToEntities().size(); + if (!timelineEntity.getId().equals("application_1111111111_2222") && + !timelineEntity.getId().equals("application_1111111111_4444")) { + Assert.fail("Entity ids' should have been application_1111111111_2222" + + " and application_1111111111_4444"); + } + } + assertEquals(0, relatesToCnt); + + TimelineFilterList rt3 = new TimelineFilterList(); + rt3.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container", + new HashSet(Arrays.asList("relatesto1", "relatesto3")))); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().relatesTo(rt3).build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + relatesToCnt = 0; + for (TimelineEntity timelineEntity : entities) { + relatesToCnt += timelineEntity.getRelatesToEntities().size(); + if (!timelineEntity.getId().equals("application_1111111111_2222")) { + Assert.fail("Entity id should have been application_1111111111_2222"); + } + } + assertEquals(0, relatesToCnt); + + TimelineFilterList rt4 = new TimelineFilterList(); + rt4.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container", + new HashSet(Arrays.asList("relatesto1")))); + rt4.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "dummy_container", + new HashSet(Arrays.asList("relatesto5")))); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().relatesTo(rt4).build(), + new TimelineDataToRetrieve()); + assertEquals(0, entities.size()); + + TimelineFilterList rt5 = new TimelineFilterList(); + rt5.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container", + new HashSet(Arrays.asList("relatedto1", "relatesto8")))); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().relatesTo(rt5).build(), + new TimelineDataToRetrieve()); + assertEquals(0, entities.size()); + + TimelineFilterList list1 = new TimelineFilterList(); + list1.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container2", + new HashSet(Arrays.asList("relatesto7")))); + list1.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "dummy_container", + new HashSet(Arrays.asList("relatesto4")))); + TimelineFilterList list2 = new TimelineFilterList(); + list2.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container1", + new HashSet(Arrays.asList("relatesto4")))); + TimelineFilterList rt6 = new TimelineFilterList(Operator.OR, list1, list2); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().relatesTo(rt6).build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + relatesToCnt = 0; + for (TimelineEntity timelineEntity : entities) { + relatesToCnt += timelineEntity.getRelatesToEntities().size(); + if (!timelineEntity.getId().equals("application_1111111111_2222")) { + Assert.fail("Entity id should have been application_1111111111_2222"); + } + } + assertEquals(0, relatesToCnt); + + TimelineFilterList list3 = new TimelineFilterList(); + list3.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container", + new HashSet(Arrays.asList("relatesto1")))); + list3.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container1", + new HashSet(Arrays.asList("relatesto4")))); + TimelineFilterList list4 = new TimelineFilterList(); + list4.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container", + new HashSet(Arrays.asList("relatesto1")))); + list4.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container", + new HashSet(Arrays.asList("relatesto2")))); + TimelineFilterList combinedList = + new TimelineFilterList(Operator.OR, list3, list4); + TimelineFilterList rt7 = new TimelineFilterList(Operator.AND, combinedList, + new TimelineKeyValuesFilter( + TimelineCompareOp.NOT_EQUAL, "container", + new HashSet(Arrays.asList("relatesto3")))); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().relatesTo(rt7).build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + relatesToCnt = 0; + for (TimelineEntity timelineEntity : entities) { + relatesToCnt += timelineEntity.getRelatesToEntities().size(); + if (!timelineEntity.getId().equals("application_1111111111_3333")) { + Assert.fail("Entity id should have been application_1111111111_3333"); + } + } + assertEquals(0, relatesToCnt); + } + + @Test + public void testReadAppsRelationsAndEventFiltersDefaultView() + throws Exception { + TimelineFilterList eventFilter = new TimelineFilterList(); + eventFilter.addFilter(new TimelineExistsFilter(TimelineCompareOp.NOT_EQUAL, + "end_event")); + TimelineFilterList relatesTo = new TimelineFilterList(Operator.OR); + relatesTo.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container2", + new HashSet(Arrays.asList("relatesto7")))); + relatesTo.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container1", + new HashSet(Arrays.asList("relatesto4")))); + TimelineFilterList isRelatedTo = new TimelineFilterList(); + isRelatedTo.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "task1", + new HashSet(Arrays.asList("relatedto3")))); + isRelatedTo.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.NOT_EQUAL, "task1", + new HashSet(Arrays.asList("relatedto5")))); + Set entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().relatesTo(relatesTo) + .isRelatedTo(isRelatedTo).eventFilters(eventFilter).build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + int eventCnt = 0; + int isRelatedToCnt = 0; + int relatesToCnt = 0; + for (TimelineEntity timelineEntity : entities) { + eventCnt += timelineEntity.getEvents().size(); + isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size(); + relatesToCnt += timelineEntity.getRelatesToEntities().size(); + if (!timelineEntity.getId().equals("application_1111111111_4444")) { + Assert.fail("Entity id should have been application_1111111111_4444"); + } + } + assertEquals(0, eventCnt); + assertEquals(0, isRelatedToCnt); + assertEquals(0, relatesToCnt); + } + + @Test + public void testReadAppsConfigFilters() throws Exception { + TimelineFilterList list1 = new TimelineFilterList(); + list1.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "cfg_param1", "value1")); + list1.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "cfg_param2", "value2")); + TimelineFilterList list2 = new TimelineFilterList(); + list2.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "cfg_param1", "value3")); + list2.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "config_param2", "value2")); + TimelineFilterList confFilterList = + new TimelineFilterList(Operator.OR, list1, list2); + Set entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().configFilters(confFilterList) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS), + null, null, null)); + assertEquals(2, entities.size()); + int cfgCnt = 0; + for (TimelineEntity entity : entities) { + cfgCnt += entity.getConfigs().size(); + } + assertEquals(5, cfgCnt); + + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().configFilters(confFilterList) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null, + null, null)); + assertEquals(2, entities.size()); + cfgCnt = 0; + for (TimelineEntity entity : entities) { + cfgCnt += entity.getConfigs().size(); + } + assertEquals(5, cfgCnt); + + TimelineFilterList confFilterList1 = new TimelineFilterList( + new TimelineKeyValueFilter( + TimelineCompareOp.NOT_EQUAL, "cfg_param1", "value1")); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().configFilters(confFilterList1) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS), + null, null, null)); + assertEquals(1, entities.size()); + cfgCnt = 0; + for (TimelineEntity entity : entities) { + cfgCnt += entity.getConfigs().size(); + } + assertEquals(3, cfgCnt); + + TimelineFilterList confFilterList2 = new TimelineFilterList( + new TimelineKeyValueFilter( + TimelineCompareOp.NOT_EQUAL, "cfg_param1", "value1"), + new TimelineKeyValueFilter( + TimelineCompareOp.NOT_EQUAL, "config_param2", "value2")); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().configFilters(confFilterList2) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS), + null, null, null)); + assertEquals(0, entities.size()); + + TimelineFilterList confFilterList3 = new TimelineFilterList( + new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "dummy_config", "value1")); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().configFilters(confFilterList3) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS), + null, null, null)); + assertEquals(0, entities.size()); + + TimelineFilterList confFilterList4 = new TimelineFilterList( + new TimelineKeyValueFilter( + TimelineCompareOp.NOT_EQUAL, "dummy_config", "value1")); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().configFilters(confFilterList4) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS), + null, null, null)); + assertEquals(0, entities.size()); + + TimelineFilterList confFilterList5 = new TimelineFilterList( + new TimelineKeyValueFilter( + TimelineCompareOp.NOT_EQUAL, "dummy_config", "value1", false)); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().configFilters(confFilterList5) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS), + null, null, null)); + assertEquals(3, entities.size()); + } + + @Test + public void testReadAppsEventFilters() throws Exception { + TimelineFilterList ef = new TimelineFilterList(); + ef.addFilter(new TimelineExistsFilter( + TimelineCompareOp.EQUAL, "update_event")); + ef.addFilter(new TimelineExistsFilter( + TimelineCompareOp.NOT_EQUAL, "end_event")); + Set entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().eventFilters(ef).build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null, + null, null)); + assertEquals(1, entities.size()); + int eventCnt = 0; + for (TimelineEntity timelineEntity : entities) { + eventCnt += timelineEntity.getEvents().size(); + if (!timelineEntity.getId().equals("application_1111111111_4444")) { + Assert.fail("Entity id should have been application_1111111111_4444"); + } + } + assertEquals(1, eventCnt); + + TimelineFilterList ef1 = new TimelineFilterList(); + ef1.addFilter(new TimelineExistsFilter( + TimelineCompareOp.EQUAL, "update_event")); + ef1.addFilter(new TimelineExistsFilter( + TimelineCompareOp.NOT_EQUAL, "end_event")); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().eventFilters(ef1).build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + eventCnt = 0; + for (TimelineEntity timelineEntity : entities) { + eventCnt += timelineEntity.getEvents().size(); + if (!timelineEntity.getId().equals("application_1111111111_4444")) { + Assert.fail("Entity id should have been application_1111111111_4444"); + } + } + assertEquals(0, eventCnt); + + TimelineFilterList ef2 = new TimelineFilterList(); + ef2.addFilter(new TimelineExistsFilter( + TimelineCompareOp.NOT_EQUAL, "end_event")); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().eventFilters(ef2).build(), + new TimelineDataToRetrieve()); + assertEquals(2, entities.size()); + eventCnt = 0; + for (TimelineEntity timelineEntity : entities) { + eventCnt += timelineEntity.getEvents().size(); + if (!timelineEntity.getId().equals("application_1111111111_2222") && + !timelineEntity.getId().equals("application_1111111111_4444")) { + Assert.fail("Entity ids' should have been application_1111111111_2222" + + " and application_1111111111_4444"); + } + } + assertEquals(0, eventCnt); + + TimelineFilterList ef3 = new TimelineFilterList(); + ef3.addFilter(new TimelineExistsFilter( + TimelineCompareOp.EQUAL, "update_event")); + ef3.addFilter(new TimelineExistsFilter( + TimelineCompareOp.EQUAL, "dummy_event")); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().eventFilters(ef3).build(), + new TimelineDataToRetrieve()); + assertEquals(0, entities.size()); + + TimelineFilterList list1 = new TimelineFilterList(); + list1.addFilter(new TimelineExistsFilter( + TimelineCompareOp.EQUAL, "update_event")); + list1.addFilter(new TimelineExistsFilter( + TimelineCompareOp.EQUAL, "dummy_event")); + TimelineFilterList list2 = new TimelineFilterList(); + list2.addFilter(new TimelineExistsFilter( + TimelineCompareOp.EQUAL, "start_event")); + TimelineFilterList ef4 = new TimelineFilterList(Operator.OR, list1, list2); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().eventFilters(ef4).build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + eventCnt = 0; + for (TimelineEntity timelineEntity : entities) { + eventCnt += timelineEntity.getEvents().size(); + if (!timelineEntity.getId().equals("application_1111111111_2222")) { + Assert.fail("Entity id should have been application_1111111111_2222"); + } + } + assertEquals(0, eventCnt); + + TimelineFilterList ef5 = new TimelineFilterList(); + ef5.addFilter(new TimelineExistsFilter( + TimelineCompareOp.NOT_EQUAL, "update_event")); + ef5.addFilter(new TimelineExistsFilter( + TimelineCompareOp.NOT_EQUAL, "end_event")); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().eventFilters(ef5).build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + eventCnt = 0; + for (TimelineEntity timelineEntity : entities) { + eventCnt += timelineEntity.getEvents().size(); + if (!timelineEntity.getId().equals("application_1111111111_2222")) { + Assert.fail("Entity id should have been application_1111111111_2222"); + } + } + assertEquals(0, eventCnt); + } + + @Test + public void testReadAppsConfigPrefix() throws Exception { + TimelineFilterList list = + new TimelineFilterList(Operator.OR, + new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_")); + TimelineEntity e1 = reader.getEntity( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1111111111_2222", + TimelineEntityType.YARN_APPLICATION.toString(), null), + new TimelineDataToRetrieve(list, null, null, null, null, null)); + assertNotNull(e1); + assertEquals(1, e1.getConfigs().size()); + Set es1 = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null) , + new TimelineEntityFilters.Builder().build(), + new TimelineDataToRetrieve(list, null, null, null, null, null)); + int cfgCnt = 0; + for (TimelineEntity entity : es1) { + cfgCnt += entity.getConfigs().size(); + for (String confKey : entity.getConfigs().keySet()) { + assertTrue("Config key returned should start with cfg_", + confKey.startsWith("cfg_")); + } + } + assertEquals(3, cfgCnt); + } + + @Test + public void testReadAppsConfigFilterPrefix() throws Exception { + TimelineFilterList confFilterList = new TimelineFilterList(); + confFilterList.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "cfg_param1", "value1")); + TimelineFilterList list = + new TimelineFilterList(Operator.OR, + new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_")); + Set entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().configFilters(confFilterList) + .build(), + new TimelineDataToRetrieve(list, null, null, null, null, null)); + assertEquals(1, entities.size()); + int cfgCnt = 0; + for (TimelineEntity entity : entities) { + cfgCnt += entity.getConfigs().size(); + for (String confKey : entity.getConfigs().keySet()) { + assertTrue("Config key returned should start with cfg_", + confKey.startsWith("cfg_")); + } + } + assertEquals(2, cfgCnt); + + TimelineFilterList list1 = new TimelineFilterList(); + list1.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "cfg_param1", "value1")); + list1.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "cfg_param2", "value2")); + TimelineFilterList list2 = new TimelineFilterList(); + list2.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "cfg_param1", "value3")); + list2.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "config_param2", "value2")); + TimelineFilterList confsToRetrieve = + new TimelineFilterList(Operator.OR, + new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "config_")); + TimelineFilterList confFilterList1 = + new TimelineFilterList(Operator.OR, list1, list2); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().configFilters(confFilterList1) + .build(), + new TimelineDataToRetrieve(confsToRetrieve, null, null, null, null, + null)); + assertEquals(2, entities.size()); + cfgCnt = 0; + for (TimelineEntity entity : entities) { + cfgCnt += entity.getConfigs().size(); + for (String confKey : entity.getConfigs().keySet()) { + assertTrue("Config key returned should start with config_", + confKey.startsWith("config_")); + } + } + assertEquals(2, cfgCnt); + } + + @Test + public void testReadAppsMetricFilters() throws Exception { + TimelineFilterList list1 = new TimelineFilterList(); + list1.addFilter(new TimelineCompareFilter( + TimelineCompareOp.GREATER_OR_EQUAL, "MAP1_SLOT_MILLIS", 50000000900L)); + TimelineFilterList list2 = new TimelineFilterList(); + list2.addFilter(new TimelineCompareFilter( + TimelineCompareOp.LESS_THAN, "MAP_SLOT_MILLIS", 80000000000L)); + list2.addFilter(new TimelineCompareFilter( + TimelineCompareOp.EQUAL, "MAP1_BYTES", 50)); + TimelineFilterList metricFilterList = + new TimelineFilterList(Operator.OR, list1, list2); + Set entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS), + null, null, null)); + assertEquals(2, entities.size()); + int metricCnt = 0; + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + } + assertEquals(3, metricCnt); + + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null, + null, null)); + assertEquals(2, entities.size()); + metricCnt = 0; + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + } + assertEquals(3, metricCnt); + + TimelineFilterList metricFilterList1 = new TimelineFilterList( + new TimelineCompareFilter( + TimelineCompareOp.LESS_OR_EQUAL, "MAP_SLOT_MILLIS", 80000000000L), + new TimelineCompareFilter( + TimelineCompareOp.NOT_EQUAL, "MAP1_BYTES", 30)); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList1) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS), + null, null, null)); + assertEquals(1, entities.size()); + metricCnt = 0; + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + } + assertEquals(2, metricCnt); + + TimelineFilterList metricFilterList2 = new TimelineFilterList( + new TimelineCompareFilter( + TimelineCompareOp.LESS_THAN, "MAP_SLOT_MILLIS", 40000000000L), + new TimelineCompareFilter( + TimelineCompareOp.NOT_EQUAL, "MAP1_BYTES", 30)); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList2) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS), + null, null, null)); + assertEquals(0, entities.size()); + + TimelineFilterList metricFilterList3 = new TimelineFilterList( + new TimelineCompareFilter( + TimelineCompareOp.EQUAL, "dummy_metric", 5)); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList3) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS), + null, null, null)); + assertEquals(0, entities.size()); + + TimelineFilterList metricFilterList4 = new TimelineFilterList( + new TimelineCompareFilter( + TimelineCompareOp.NOT_EQUAL, "dummy_metric", 5)); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList4) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS), + null, null, null)); + assertEquals(0, entities.size()); + + TimelineFilterList metricFilterList5 = new TimelineFilterList( + new TimelineCompareFilter( + TimelineCompareOp.NOT_EQUAL, "dummy_metric", 5, false)); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList5) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS), + null, null, null)); + assertEquals(3, entities.size()); + } + + @Test + public void testReadAppsMetricPrefix() throws Exception { + TimelineFilterList list = + new TimelineFilterList(Operator.OR, + new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_")); + TimelineEntity e1 = reader.getEntity( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1111111111_2222", + TimelineEntityType.YARN_APPLICATION.toString(), null), + new TimelineDataToRetrieve(null, list, null, null, null, null)); + assertNotNull(e1); + assertEquals(1, e1.getMetrics().size()); + Set es1 = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().build(), + new TimelineDataToRetrieve(null, list, null, null, null, null)); + int metricCnt = 0; + for (TimelineEntity entity : es1) { + metricCnt += entity.getMetrics().size(); + for (TimelineMetric metric : entity.getMetrics()) { + assertTrue("Metric Id returned should start with MAP1_", + metric.getId().startsWith("MAP1_")); + } + } + assertEquals(2, metricCnt); + } + + @Test + public void testReadAppsMetricFilterPrefix() throws Exception { + TimelineFilterList list = + new TimelineFilterList(Operator.OR, + new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_")); + TimelineFilterList metricFilterList = new TimelineFilterList(); + metricFilterList.addFilter(new TimelineCompareFilter( + TimelineCompareOp.GREATER_OR_EQUAL, "MAP1_SLOT_MILLIS", 0L)); + Set entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList) + .build(), + new TimelineDataToRetrieve(null, list, null, null, null, null)); + int metricCnt = 0; + assertEquals(1, entities.size()); + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + } + assertEquals(1, metricCnt); + + TimelineFilterList list1 = new TimelineFilterList(); + list1.addFilter(new TimelineCompareFilter( + TimelineCompareOp.GREATER_OR_EQUAL, "MAP1_SLOT_MILLIS", 50000000900L)); + TimelineFilterList list2 = new TimelineFilterList(); + list2.addFilter(new TimelineCompareFilter( + TimelineCompareOp.LESS_THAN, "MAP_SLOT_MILLIS", 80000000000L)); + list2.addFilter(new TimelineCompareFilter( + TimelineCompareOp.EQUAL, "MAP1_BYTES", 50)); + TimelineFilterList metricsToRetrieve = new TimelineFilterList(Operator.OR, + new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_")); + TimelineFilterList metricFilterList1 = + new TimelineFilterList(Operator.OR, list1, list2); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList1) + .build(), + new TimelineDataToRetrieve(null, metricsToRetrieve, null, null, null, + null)); + metricCnt = 0; + assertEquals(2, entities.size()); + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + for (TimelineMetric metric : entity.getMetrics()) { + assertTrue("Metric Id returned should start with MAP1_", + metric.getId().startsWith("MAP1_")); + } + } + assertEquals(2, metricCnt); + + entities = reader.getEntities(new TimelineReaderContext("cluster1", "user1", + "some_flow_name", 1002345678919L, null, + TimelineEntityType.YARN_APPLICATION.toString(), null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList1) + .build(), + new TimelineDataToRetrieve(null, metricsToRetrieve, + EnumSet.of(Field.METRICS), Integer.MAX_VALUE, null, null)); + metricCnt = 0; + int metricValCnt = 0; + assertEquals(2, entities.size()); + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + for (TimelineMetric metric : entity.getMetrics()) { + metricValCnt += metric.getValues().size(); + assertTrue("Metric Id returned should start with MAP1_", + metric.getId().startsWith("MAP1_")); + } + } + assertEquals(2, metricCnt); + assertEquals(7, metricValCnt); + } + + @Test + public void testReadAppsMetricTimeRange() throws Exception { + Set entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), new TimelineEntityFilters.Builder().build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS), + 100, null, null)); + assertEquals(3, entities.size()); + int metricTimeSeriesCnt = 0; + int metricCnt = 0; + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + for (TimelineMetric m : entity.getMetrics()) { + metricTimeSeriesCnt += m.getValues().size(); + } + } + assertEquals(3, metricCnt); + assertEquals(13, metricTimeSeriesCnt); + + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), new TimelineEntityFilters.Builder().build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS), + 100, CURRENT_TIME - 40000, CURRENT_TIME)); + assertEquals(3, entities.size()); + metricCnt = 0; + metricTimeSeriesCnt = 0; + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + for (TimelineMetric m : entity.getMetrics()) { + for (Long ts : m.getValues().keySet()) { + assertTrue(ts >= CURRENT_TIME - 40000 && ts <= CURRENT_TIME); + } + metricTimeSeriesCnt += m.getValues().size(); + } + } + assertEquals(3, metricCnt); + assertEquals(5, metricTimeSeriesCnt); + + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), new TimelineEntityFilters.Builder().build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS), + null, CURRENT_TIME - 40000, CURRENT_TIME)); + assertEquals(3, entities.size()); + metricCnt = 0; + metricTimeSeriesCnt = 0; + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + for (TimelineMetric m : entity.getMetrics()) { + for (Long ts : m.getValues().keySet()) { + assertTrue(ts >= CURRENT_TIME - 40000 && ts <= CURRENT_TIME); + } + metricTimeSeriesCnt += m.getValues().size(); + } + } + assertEquals(3, metricCnt); + assertEquals(3, metricTimeSeriesCnt); + + TimelineEntity entity = reader.getEntity(new TimelineReaderContext( + "cluster1", "user1", "some_flow_name", 1002345678919L, + "application_1111111111_2222", + TimelineEntityType.YARN_APPLICATION.toString(), null), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS), 100, + CURRENT_TIME - 40000, CURRENT_TIME)); + assertNotNull(entity); + assertEquals(2, entity.getMetrics().size()); + metricTimeSeriesCnt = 0; + for (TimelineMetric m : entity.getMetrics()) { + for (Long ts : m.getValues().keySet()) { + assertTrue(ts >= CURRENT_TIME - 40000 && ts <= CURRENT_TIME); + } + metricTimeSeriesCnt += m.getValues().size(); + } + assertEquals(3, metricTimeSeriesCnt); + } + + @Test + public void testReadAppsInfoFilters() throws Exception { + TimelineFilterList list1 = new TimelineFilterList(); + list1.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "infoMapKey3", 85.85)); + list1.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "infoMapKey1", "infoMapValue2")); + TimelineFilterList list2 = new TimelineFilterList(); + list2.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "infoMapKey1", "infoMapValue1")); + list2.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "infoMapKey2", 10)); + TimelineFilterList infoFilterList = + new TimelineFilterList(Operator.OR, list1, list2); + Set entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().infoFilters(infoFilterList).build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null, + null, null)); + assertEquals(2, entities.size()); + int infoCnt = 0; + for (TimelineEntity entity : entities) { + infoCnt += entity.getInfo().size(); + } + assertEquals(7, infoCnt); + + TimelineFilterList infoFilterList1 = new TimelineFilterList( + new TimelineKeyValueFilter( + TimelineCompareOp.NOT_EQUAL, "infoMapKey1", "infoMapValue1")); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().infoFilters(infoFilterList1) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null, + null, null)); + assertEquals(1, entities.size()); + infoCnt = 0; + for (TimelineEntity entity : entities) { + infoCnt += entity.getInfo().size(); + } + assertEquals(4, infoCnt); + + TimelineFilterList infoFilterList2 = new TimelineFilterList( + new TimelineKeyValueFilter( + TimelineCompareOp.NOT_EQUAL, "infoMapKey1", "infoMapValue2"), + new TimelineKeyValueFilter( + TimelineCompareOp.NOT_EQUAL, "infoMapKey3", 85.85)); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().infoFilters(infoFilterList2) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null, + null, null)); + assertEquals(0, entities.size()); + + TimelineFilterList infoFilterList3 = new TimelineFilterList( + new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "dummy_info", "some_value")); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().infoFilters(infoFilterList3) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null, + null, null)); + assertEquals(0, entities.size()); + + TimelineFilterList infoFilterList4 = new TimelineFilterList( + new TimelineKeyValueFilter( + TimelineCompareOp.NOT_EQUAL, "dummy_info", "some_value")); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().infoFilters(infoFilterList4) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null, + null, null)); + assertEquals(0, entities.size()); + + TimelineFilterList infoFilterList5 = new TimelineFilterList( + new TimelineKeyValueFilter( + TimelineCompareOp.NOT_EQUAL, "dummy_info", "some_value", false)); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(), + null), + new TimelineEntityFilters.Builder().infoFilters(infoFilterList5) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null, + null, null)); + assertEquals(3, entities.size()); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + util.shutdownMiniCluster(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java new file mode 100644 index 00000000000..ae8f8212270 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java @@ -0,0 +1,1886 @@ +/** + * 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.timelineservice.storage; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.Arrays; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.NavigableMap; +import java.util.NavigableSet; +import java.util.Set; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric.Type; +import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants; +import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList.Operator; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValuesFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnName; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnNameConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKeyPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationColumn; +import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationRowKeyPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationTable; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +/** + * Various tests to test writing entities to HBase and reading them back from + * it. + * + * It uses a single HBase mini-cluster for all tests which is a little more + * realistic, and helps test correctness in the presence of other data. + * + * Each test uses a different cluster name to be able to handle its own data + * even if other records exist in the table. Use a different cluster name if + * you add a new test. + */ +public class TestHBaseTimelineStorageEntities { + + private static HBaseTestingUtility util; + private HBaseTimelineReaderImpl reader; + private static final long CURRENT_TIME = System.currentTimeMillis(); + + @BeforeClass + public static void setupBeforeClass() throws Exception { + util = new HBaseTestingUtility(); + util.startMiniCluster(); + DataGeneratorForTest.createSchema(util.getConfiguration()); + DataGeneratorForTest.loadEntities(util, CURRENT_TIME); + } + + @Before + public void init() throws Exception { + reader = new HBaseTimelineReaderImpl(); + reader.init(util.getConfiguration()); + reader.start(); + } + + @After + public void stop() throws Exception { + if (reader != null) { + reader.stop(); + reader.close(); + } + } + + private static void matchMetrics(Map m1, Map m2) { + assertEquals(m1.size(), m2.size()); + for (Map.Entry entry : m2.entrySet()) { + Number val = m1.get(entry.getKey()); + assertNotNull(val); + assertEquals(val.longValue(), entry.getValue().longValue()); + } + } + + @Test + public void testWriteEntityToHBase() throws Exception { + TimelineEntities te = new TimelineEntities(); + TimelineEntity entity = new TimelineEntity(); + String id = "hello"; + String type = "world"; + entity.setId(id); + entity.setType(type); + Long cTime = 1425016501000L; + entity.setCreatedTime(cTime); + + // add the info map in Timeline Entity + Map infoMap = new HashMap(); + infoMap.put("infoMapKey1", "infoMapValue1"); + infoMap.put("infoMapKey2", 10); + entity.addInfo(infoMap); + + // add the isRelatedToEntity info + String key = "task"; + String value = "is_related_to_entity_id_here"; + Set isRelatedToSet = new HashSet(); + isRelatedToSet.add(value); + Map> isRelatedTo = new HashMap>(); + isRelatedTo.put(key, isRelatedToSet); + entity.setIsRelatedToEntities(isRelatedTo); + + // add the relatesTo info + key = "container"; + value = "relates_to_entity_id_here"; + Set relatesToSet = new HashSet(); + relatesToSet.add(value); + value = "relates_to_entity_id_here_Second"; + relatesToSet.add(value); + Map> relatesTo = new HashMap>(); + relatesTo.put(key, relatesToSet); + entity.setRelatesToEntities(relatesTo); + + // add some config entries + Map conf = new HashMap(); + conf.put("config_param1", "value1"); + conf.put("config_param2", "value2"); + entity.addConfigs(conf); + + // add metrics + Set metrics = new HashSet<>(); + TimelineMetric m1 = new TimelineMetric(); + m1.setId("MAP_SLOT_MILLIS"); + Map metricValues = new HashMap(); + long ts = System.currentTimeMillis(); + metricValues.put(ts - 120000, 100000000); + metricValues.put(ts - 100000, 200000000); + metricValues.put(ts - 80000, 300000000); + metricValues.put(ts - 60000, 400000000); + metricValues.put(ts - 40000, 50000000000L); + metricValues.put(ts - 20000, 60000000000L); + m1.setType(Type.TIME_SERIES); + m1.setValues(metricValues); + metrics.add(m1); + entity.addMetrics(metrics); + te.addEntity(entity); + + HBaseTimelineWriterImpl hbi = null; + try { + Configuration c1 = util.getConfiguration(); + hbi = new HBaseTimelineWriterImpl(); + hbi.init(c1); + hbi.start(); + String cluster = "cluster_test_write_entity"; + String user = "user1"; + String subAppUser = "subAppUser1"; + String flow = "some_flow_name"; + String flowVersion = "AB7822C10F1111"; + long runid = 1002345678919L; + String appName = HBaseTimelineStorageUtils.convertApplicationIdToString( + ApplicationId.newInstance(System.currentTimeMillis() + 9000000L, 1)); + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, appName), te, + UserGroupInformation.createRemoteUser(subAppUser)); + hbi.stop(); + + // scan the table and see that entity exists + Scan s = new Scan(); + byte[] startRow = + new EntityRowKeyPrefix(cluster, user, flow, runid, appName) + .getRowKeyPrefix(); + s.setStartRow(startRow); + s.setMaxVersions(Integer.MAX_VALUE); + Connection conn = ConnectionFactory.createConnection(c1); + ResultScanner scanner = new EntityTable().getResultScanner(c1, conn, s); + + int rowCount = 0; + int colCount = 0; + KeyConverter stringKeyConverter = new StringKeyConverter(); + for (Result result : scanner) { + if (result != null && !result.isEmpty()) { + rowCount++; + colCount += result.size(); + byte[] row1 = result.getRow(); + assertTrue(isRowKeyCorrect(row1, cluster, user, flow, runid, appName, + entity)); + + // check info column family + String id1 = EntityColumn.ID.readResult(result).toString(); + assertEquals(id, id1); + + String type1 = EntityColumn.TYPE.readResult(result).toString(); + assertEquals(type, type1); + + Long cTime1 = (Long) EntityColumn.CREATED_TIME.readResult(result); + assertEquals(cTime1, cTime); + + Map infoColumns = + EntityColumnPrefix.INFO.readResults(result, + new StringKeyConverter()); + assertEquals(infoMap, infoColumns); + + // Remember isRelatedTo is of type Map> + for (Map.Entry> isRelatedToEntry : isRelatedTo + .entrySet()) { + Object isRelatedToValue = + EntityColumnPrefix.IS_RELATED_TO.readResult(result, + isRelatedToEntry.getKey()); + String compoundValue = isRelatedToValue.toString(); + // id7?id9?id6 + Set isRelatedToValues = + new HashSet( + Separator.VALUES.splitEncoded(compoundValue)); + assertEquals(isRelatedTo.get(isRelatedToEntry.getKey()).size(), + isRelatedToValues.size()); + for (String v : isRelatedToEntry.getValue()) { + assertTrue(isRelatedToValues.contains(v)); + } + } + + // RelatesTo + for (Map.Entry> relatesToEntry : relatesTo + .entrySet()) { + String compoundValue = EntityColumnPrefix.RELATES_TO + .readResult(result, relatesToEntry.getKey()).toString(); + // id3?id4?id5 + Set relatesToValues = + new HashSet( + Separator.VALUES.splitEncoded(compoundValue)); + assertEquals(relatesTo.get(relatesToEntry.getKey()).size(), + relatesToValues.size()); + for (String v : relatesToEntry.getValue()) { + assertTrue(relatesToValues.contains(v)); + } + } + + // Configuration + Map configColumns = + EntityColumnPrefix.CONFIG.readResults(result, stringKeyConverter); + assertEquals(conf, configColumns); + + NavigableMap> metricsResult = + EntityColumnPrefix.METRIC.readResultsWithTimestamps(result, + stringKeyConverter); + + NavigableMap metricMap = metricsResult.get(m1.getId()); + matchMetrics(metricValues, metricMap); + } + } + assertEquals(1, rowCount); + assertEquals(16, colCount); + + // read the timeline entity using the reader this time + TimelineEntity e1 = reader.getEntity( + new TimelineReaderContext(cluster, user, flow, runid, appName, + entity.getType(), entity.getId()), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), + Integer.MAX_VALUE, null, null)); + Set es1 = reader.getEntities( + new TimelineReaderContext(cluster, user, flow, runid, appName, + entity.getType(), null), + new TimelineEntityFilters.Builder().build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), + Integer.MAX_VALUE, null, null)); + assertNotNull(e1); + assertEquals(1, es1.size()); + + // verify attributes + assertEquals(id, e1.getId()); + assertEquals(type, e1.getType()); + assertEquals(cTime, e1.getCreatedTime()); + Map infoMap2 = e1.getInfo(); + // fromid key is added by storage. Remove it for comparison. + infoMap2.remove("FROM_ID"); + assertEquals(infoMap, infoMap2); + + Map> isRelatedTo2 = e1.getIsRelatedToEntities(); + assertEquals(isRelatedTo, isRelatedTo2); + + Map> relatesTo2 = e1.getRelatesToEntities(); + assertEquals(relatesTo, relatesTo2); + + Map conf2 = e1.getConfigs(); + assertEquals(conf, conf2); + + Set metrics2 = e1.getMetrics(); + assertEquals(metrics, metrics2); + for (TimelineMetric metric2 : metrics2) { + Map metricValues2 = metric2.getValues(); + matchMetrics(metricValues, metricValues2); + } + + e1 = reader.getEntity(new TimelineReaderContext(cluster, user, flow, + runid, appName, entity.getType(), entity.getId()), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null, + null, null)); + assertNotNull(e1); + assertEquals(id, e1.getId()); + assertEquals(type, e1.getType()); + assertEquals(cTime, e1.getCreatedTime()); + infoMap2 = e1.getInfo(); + // fromid key is added by storage. Remove it for comparision. + infoMap2.remove("FROM_ID"); + assertEquals(infoMap, infoMap2); + assertEquals(isRelatedTo, e1.getIsRelatedToEntities()); + assertEquals(relatesTo, e1.getRelatesToEntities()); + assertEquals(conf, e1.getConfigs()); + for (TimelineMetric metric : e1.getMetrics()) { + assertEquals(TimelineMetric.Type.SINGLE_VALUE, metric.getType()); + assertEquals(1, metric.getValues().size()); + assertTrue(metric.getValues().containsKey(ts - 20000)); + assertEquals(metricValues.get(ts - 20000), + metric.getValues().get(ts - 20000)); + } + + // verify for sub application table entities. + verifySubApplicationTableEntities(cluster, user, flow, flowVersion, runid, + appName, subAppUser, c1, entity, id, type, infoMap, isRelatedTo, + relatesTo, conf, metricValues, metrics, cTime, m1); + } finally { + if (hbi != null) { + hbi.stop(); + hbi.close(); + } + } + } + + private void verifySubApplicationTableEntities(String cluster, String user, + String flow, String flowVersion, Long runid, String appName, + String subAppUser, Configuration c1, TimelineEntity entity, String id, + String type, Map infoMap, + Map> isRelatedTo, Map> relatesTo, + Map conf, Map metricValues, + Set metrics, Long cTime, TimelineMetric m1) + throws IOException { + Scan s = new Scan(); + // read from SubApplicationTable + byte[] startRow = new SubApplicationRowKeyPrefix(cluster, subAppUser, null, + null, null, null).getRowKeyPrefix(); + s.setStartRow(startRow); + s.setMaxVersions(Integer.MAX_VALUE); + Connection conn = ConnectionFactory.createConnection(c1); + ResultScanner scanner = + new SubApplicationTable().getResultScanner(c1, conn, s); + + int rowCount = 0; + int colCount = 0; + KeyConverter stringKeyConverter = new StringKeyConverter(); + for (Result result : scanner) { + if (result != null && !result.isEmpty()) { + rowCount++; + colCount += result.size(); + byte[] row1 = result.getRow(); + assertTrue(verifyRowKeyForSubApplication(row1, subAppUser, cluster, + user, entity)); + + // check info column family + String id1 = SubApplicationColumn.ID.readResult(result).toString(); + assertEquals(id, id1); + + String type1 = SubApplicationColumn.TYPE.readResult(result).toString(); + assertEquals(type, type1); + + Long cTime1 = + (Long) SubApplicationColumn.CREATED_TIME.readResult(result); + assertEquals(cTime1, cTime); + + Map infoColumns = SubApplicationColumnPrefix.INFO + .readResults(result, new StringKeyConverter()); + assertEquals(infoMap, infoColumns); + + // Remember isRelatedTo is of type Map> + for (Map.Entry> isRelatedToEntry : isRelatedTo + .entrySet()) { + Object isRelatedToValue = SubApplicationColumnPrefix.IS_RELATED_TO + .readResult(result, isRelatedToEntry.getKey()); + String compoundValue = isRelatedToValue.toString(); + // id7?id9?id6 + Set isRelatedToValues = + new HashSet(Separator.VALUES.splitEncoded(compoundValue)); + assertEquals(isRelatedTo.get(isRelatedToEntry.getKey()).size(), + isRelatedToValues.size()); + for (String v : isRelatedToEntry.getValue()) { + assertTrue(isRelatedToValues.contains(v)); + } + } + + // RelatesTo + for (Map.Entry> relatesToEntry : relatesTo + .entrySet()) { + String compoundValue = SubApplicationColumnPrefix.RELATES_TO + .readResult(result, relatesToEntry.getKey()).toString(); + // id3?id4?id5 + Set relatesToValues = + new HashSet(Separator.VALUES.splitEncoded(compoundValue)); + assertEquals(relatesTo.get(relatesToEntry.getKey()).size(), + relatesToValues.size()); + for (String v : relatesToEntry.getValue()) { + assertTrue(relatesToValues.contains(v)); + } + } + + // Configuration + Map configColumns = SubApplicationColumnPrefix.CONFIG + .readResults(result, stringKeyConverter); + assertEquals(conf, configColumns); + + NavigableMap> metricsResult = + SubApplicationColumnPrefix.METRIC.readResultsWithTimestamps(result, + stringKeyConverter); + + NavigableMap metricMap = metricsResult.get(m1.getId()); + matchMetrics(metricValues, metricMap); + } + } + assertEquals(1, rowCount); + assertEquals(16, colCount); + } + + private boolean isRowKeyCorrect(byte[] rowKey, String cluster, String user, + String flow, Long runid, String appName, TimelineEntity te) { + + EntityRowKey key = EntityRowKey.parseRowKey(rowKey); + + assertEquals(user, key.getUserId()); + assertEquals(cluster, key.getClusterId()); + assertEquals(flow, key.getFlowName()); + assertEquals(runid, key.getFlowRunId()); + assertEquals(appName, key.getAppId()); + assertEquals(te.getType(), key.getEntityType()); + assertEquals(te.getId(), key.getEntityId()); + return true; + } + + @Test + public void testEventsWithEmptyInfo() throws IOException { + TimelineEvent event = new TimelineEvent(); + String eventId = "foo_ev e nt_id"; + event.setId(eventId); + Long expTs = 1436512802000L; + event.setTimestamp(expTs); + + final TimelineEntity entity = new TimelineEntity(); + entity.setId("attempt_1329348432655_0001_m_000008_18"); + entity.setType("FOO_ATTEMPT"); + entity.addEvent(event); + + TimelineEntities entities = new TimelineEntities(); + entities.addEntity(entity); + + HBaseTimelineWriterImpl hbi = null; + try { + Configuration c1 = util.getConfiguration(); + hbi = new HBaseTimelineWriterImpl(); + hbi.init(c1); + hbi.start(); + String cluster = "cluster_test_empty_eventkey"; + String user = "user_emptyeventkey"; + String flow = "other_flow_name"; + String flowVersion = "1111F01C2287BA"; + long runid = 1009876543218L; + String appName = HBaseTimelineStorageUtils.convertApplicationIdToString( + ApplicationId.newInstance(System.currentTimeMillis() + 9000000L, 1)); + byte[] startRow = + new EntityRowKeyPrefix(cluster, user, flow, runid, appName) + .getRowKeyPrefix(); + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, appName), entities, + UserGroupInformation.createRemoteUser(user)); + hbi.stop(); + // scan the table and see that entity exists + Scan s = new Scan(); + s.setStartRow(startRow); + s.addFamily(EntityColumnFamily.INFO.getBytes()); + Connection conn = ConnectionFactory.createConnection(c1); + ResultScanner scanner = new EntityTable().getResultScanner(c1, conn, s); + + int rowCount = 0; + for (Result result : scanner) { + if (result != null && !result.isEmpty()) { + rowCount++; + + // check the row key + byte[] row1 = result.getRow(); + assertTrue(isRowKeyCorrect(row1, cluster, user, flow, runid, appName, + entity)); + + Map eventsResult = + EntityColumnPrefix.EVENT.readResults(result, + new EventColumnNameConverter()); + // there should be only one event + assertEquals(1, eventsResult.size()); + for (Map.Entry e : eventsResult.entrySet()) { + EventColumnName eventColumnName = e.getKey(); + // the qualifier is a compound key + // hence match individual values + assertEquals(eventId, eventColumnName.getId()); + assertEquals(expTs, eventColumnName.getTimestamp()); + // key must be empty + assertNull(eventColumnName.getInfoKey()); + Object value = e.getValue(); + // value should be empty + assertEquals("", value.toString()); + } + } + } + assertEquals(1, rowCount); + + // read the timeline entity using the reader this time + TimelineEntity e1 = reader.getEntity( + new TimelineReaderContext(cluster, user, flow, runid, appName, + entity.getType(), entity.getId()), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null, + null, null)); + Set es1 = reader.getEntities( + new TimelineReaderContext(cluster, user, flow, runid, appName, + entity.getType(), null), + new TimelineEntityFilters.Builder().build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null, + null, null)); + assertNotNull(e1); + assertEquals(1, es1.size()); + + // check the events + NavigableSet events = e1.getEvents(); + // there should be only one event + assertEquals(1, events.size()); + for (TimelineEvent e : events) { + assertEquals(eventId, e.getId()); + assertEquals(expTs, Long.valueOf(e.getTimestamp())); + Map info = e.getInfo(); + assertTrue(info == null || info.isEmpty()); + } + } finally { + if (hbi != null) { + hbi.stop(); + hbi.close(); + } + } + } + + @Test + public void testEventsEscapeTs() throws IOException { + TimelineEvent event = new TimelineEvent(); + String eventId = ApplicationMetricsConstants.CREATED_EVENT_TYPE; + event.setId(eventId); + long expTs = 1463567041056L; + event.setTimestamp(expTs); + String expKey = "f==o o_e ve\tnt"; + Object expVal = "test"; + event.addInfo(expKey, expVal); + + final TimelineEntity entity = new ApplicationEntity(); + entity.setId( + HBaseTimelineStorageUtils.convertApplicationIdToString( + ApplicationId.newInstance(0, 1))); + entity.addEvent(event); + + TimelineEntities entities = new TimelineEntities(); + entities.addEntity(entity); + + HBaseTimelineWriterImpl hbi = null; + try { + Configuration c1 = util.getConfiguration(); + hbi = new HBaseTimelineWriterImpl(); + hbi.init(c1); + hbi.start(); + String cluster = "clus!ter_\ttest_ev ents"; + String user = "user2"; + String flow = "other_flow_name"; + String flowVersion = "1111F01C2287BA"; + long runid = 1009876543218L; + String appName = "application_123465899910_2001"; + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, appName), entities, + UserGroupInformation.createRemoteUser(user)); + hbi.stop(); + + // read the timeline entity using the reader this time + TimelineEntity e1 = reader.getEntity( + new TimelineReaderContext(cluster, user, flow, runid, appName, + entity.getType(), entity.getId()), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null, + null, null)); + assertNotNull(e1); + // check the events + NavigableSet events = e1.getEvents(); + // there should be only one event + assertEquals(1, events.size()); + for (TimelineEvent e : events) { + assertEquals(eventId, e.getId()); + assertEquals(expTs, e.getTimestamp()); + Map info = e.getInfo(); + assertEquals(1, info.size()); + for (Map.Entry infoEntry : info.entrySet()) { + assertEquals(expKey, infoEntry.getKey()); + assertEquals(expVal, infoEntry.getValue()); + } + } + } finally { + if (hbi != null) { + hbi.stop(); + hbi.close(); + } + } + } + + @Test + public void testReadEntities() throws Exception { + TimelineEntity entity = reader.getEntity( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", "hello"), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null, + null, null)); + assertNotNull(entity); + assertEquals(3, entity.getConfigs().size()); + assertEquals(1, entity.getIsRelatedToEntities().size()); + Set entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", + null), new TimelineEntityFilters.Builder().build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null, + null, null)); + assertEquals(3, entities.size()); + int cfgCnt = 0; + int metricCnt = 0; + int infoCnt = 0; + int eventCnt = 0; + int relatesToCnt = 0; + int isRelatedToCnt = 0; + for (TimelineEntity timelineEntity : entities) { + cfgCnt += (timelineEntity.getConfigs() == null) ? 0 : + timelineEntity.getConfigs().size(); + metricCnt += (timelineEntity.getMetrics() == null) ? 0 : + timelineEntity.getMetrics().size(); + infoCnt += (timelineEntity.getInfo() == null) ? 0 : + timelineEntity.getInfo().size(); + eventCnt += (timelineEntity.getEvents() == null) ? 0 : + timelineEntity.getEvents().size(); + relatesToCnt += (timelineEntity.getRelatesToEntities() == null) ? 0 : + timelineEntity.getRelatesToEntities().size(); + isRelatedToCnt += (timelineEntity.getIsRelatedToEntities() == null) ? 0 : + timelineEntity.getIsRelatedToEntities().size(); + } + assertEquals(5, cfgCnt); + assertEquals(3, metricCnt); + assertEquals(8, infoCnt); + assertEquals(4, eventCnt); + assertEquals(4, relatesToCnt); + assertEquals(4, isRelatedToCnt); + } + + @Test + public void testFilterEntitiesByCreatedTime() throws Exception { + Set entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().createdTimeBegin(1425016502000L) + .createTimeEnd(1425016502040L).build(), + new TimelineDataToRetrieve()); + assertEquals(3, entities.size()); + for (TimelineEntity entity : entities) { + if (!entity.getId().equals("hello") && !entity.getId().equals("hello1") && + !entity.getId().equals("hello2")) { + Assert.fail("Entities with ids' hello, hello1 and hello2 should be" + + " present"); + } + } + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().createdTimeBegin(1425016502015L) + .build(), + new TimelineDataToRetrieve()); + assertEquals(2, entities.size()); + for (TimelineEntity entity : entities) { + if (!entity.getId().equals("hello1") && + !entity.getId().equals("hello2")) { + Assert.fail("Entities with ids' hello1 and hello2 should be present"); + } + } + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().createTimeEnd(1425016502015L) + .build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + for (TimelineEntity entity : entities) { + if (!entity.getId().equals("hello")) { + Assert.fail("Entity with id hello should be present"); + } + } + } + + @Test + public void testReadEntitiesRelationsAndEventFiltersDefaultView() + throws Exception { + TimelineFilterList eventFilter = new TimelineFilterList(); + eventFilter.addFilter(new TimelineExistsFilter(TimelineCompareOp.NOT_EQUAL, + "end_event")); + TimelineFilterList relatesTo = new TimelineFilterList(Operator.OR); + relatesTo.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container2", + new HashSet(Arrays.asList("relatesto7")))); + relatesTo.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container1", + new HashSet(Arrays.asList("relatesto4")))); + TimelineFilterList isRelatedTo = new TimelineFilterList(); + isRelatedTo.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "task1", + new HashSet(Arrays.asList("relatedto3")))); + isRelatedTo.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.NOT_EQUAL, "task1", + new HashSet(Arrays.asList("relatedto5")))); + Set entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().relatesTo(relatesTo) + .isRelatedTo(isRelatedTo).eventFilters(eventFilter).build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + int eventCnt = 0; + int isRelatedToCnt = 0; + int relatesToCnt = 0; + for (TimelineEntity timelineEntity : entities) { + eventCnt += timelineEntity.getEvents().size(); + isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size(); + relatesToCnt += timelineEntity.getRelatesToEntities().size(); + if (!timelineEntity.getId().equals("hello2")) { + Assert.fail("Entity id should have been hello2"); + } + } + assertEquals(0, eventCnt); + assertEquals(0, isRelatedToCnt); + assertEquals(0, relatesToCnt); + } + + @Test + public void testReadEntitiesEventFilters() throws Exception { + TimelineFilterList ef = new TimelineFilterList(); + ef.addFilter(new TimelineExistsFilter( + TimelineCompareOp.EQUAL, "update_event")); + ef.addFilter(new TimelineExistsFilter( + TimelineCompareOp.NOT_EQUAL, "end_event")); + Set entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().eventFilters(ef).build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null, + null, null)); + assertEquals(1, entities.size()); + int eventCnt = 0; + for (TimelineEntity timelineEntity : entities) { + eventCnt += timelineEntity.getEvents().size(); + if (!timelineEntity.getId().equals("hello2")) { + Assert.fail("Entity id should have been hello2"); + } + } + assertEquals(1, eventCnt); + + TimelineFilterList ef1 = new TimelineFilterList(); + ef1.addFilter(new TimelineExistsFilter( + TimelineCompareOp.EQUAL, "update_event")); + ef1.addFilter(new TimelineExistsFilter( + TimelineCompareOp.NOT_EQUAL, "end_event")); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().eventFilters(ef1).build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + eventCnt = 0; + for (TimelineEntity timelineEntity : entities) { + eventCnt += timelineEntity.getEvents().size(); + if (!timelineEntity.getId().equals("hello2")) { + Assert.fail("Entity id should have been hello2"); + } + } + assertEquals(0, eventCnt); + + TimelineFilterList ef2 = new TimelineFilterList(); + ef2.addFilter(new TimelineExistsFilter( + TimelineCompareOp.NOT_EQUAL, "end_event")); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().eventFilters(ef2).build(), + new TimelineDataToRetrieve()); + assertEquals(2, entities.size()); + eventCnt = 0; + for (TimelineEntity timelineEntity : entities) { + eventCnt += timelineEntity.getEvents().size(); + if (!timelineEntity.getId().equals("hello") && + !timelineEntity.getId().equals("hello2")) { + Assert.fail("Entity ids' should have been hello and hello2"); + } + } + assertEquals(0, eventCnt); + + TimelineFilterList ef3 = new TimelineFilterList(); + ef3.addFilter(new TimelineExistsFilter( + TimelineCompareOp.EQUAL, "update_event")); + ef3.addFilter(new TimelineExistsFilter( + TimelineCompareOp.EQUAL, "dummy_event")); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().eventFilters(ef3).build(), + new TimelineDataToRetrieve()); + assertEquals(0, entities.size()); + + TimelineFilterList list1 = new TimelineFilterList(); + list1.addFilter(new TimelineExistsFilter( + TimelineCompareOp.EQUAL, "update_event")); + list1.addFilter(new TimelineExistsFilter( + TimelineCompareOp.EQUAL, "dummy_event")); + TimelineFilterList list2 = new TimelineFilterList(); + list2.addFilter(new TimelineExistsFilter( + TimelineCompareOp.EQUAL, "start_event")); + TimelineFilterList ef4 = new TimelineFilterList(Operator.OR, list1, list2); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().eventFilters(ef4).build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + eventCnt = 0; + for (TimelineEntity timelineEntity : entities) { + eventCnt += timelineEntity.getEvents().size(); + if (!timelineEntity.getId().equals("hello")) { + Assert.fail("Entity id should have been hello"); + } + } + assertEquals(0, eventCnt); + + TimelineFilterList ef5 = new TimelineFilterList(); + ef5.addFilter(new TimelineExistsFilter( + TimelineCompareOp.NOT_EQUAL, "update_event")); + ef5.addFilter(new TimelineExistsFilter( + TimelineCompareOp.NOT_EQUAL, "end_event")); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().eventFilters(ef5).build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + eventCnt = 0; + for (TimelineEntity timelineEntity : entities) { + eventCnt += timelineEntity.getEvents().size(); + if (!timelineEntity.getId().equals("hello")) { + Assert.fail("Entity id should have been hello"); + } + } + assertEquals(0, eventCnt); + } + + @Test + public void testReadEntitiesIsRelatedTo() throws Exception { + TimelineFilterList irt = new TimelineFilterList(Operator.OR); + irt.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "task", + new HashSet(Arrays.asList("relatedto1")))); + irt.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "task2", + new HashSet(Arrays.asList("relatedto4")))); + Set entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().isRelatedTo(irt).build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null, + null, null)); + assertEquals(2, entities.size()); + int isRelatedToCnt = 0; + for (TimelineEntity timelineEntity : entities) { + isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size(); + if (!timelineEntity.getId().equals("hello") && + !timelineEntity.getId().equals("hello1")) { + Assert.fail("Entity ids' should have been hello and hello1"); + } + } + assertEquals(3, isRelatedToCnt); + + TimelineFilterList irt1 = new TimelineFilterList(); + irt1.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "task1", + new HashSet(Arrays.asList("relatedto3")))); + irt1.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.NOT_EQUAL, "task1", + new HashSet(Arrays.asList("relatedto5")))); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().isRelatedTo(irt1).build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + isRelatedToCnt = 0; + for (TimelineEntity timelineEntity : entities) { + isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size(); + if (!timelineEntity.getId().equals("hello2")) { + Assert.fail("Entity id should have been hello2"); + } + } + assertEquals(0, isRelatedToCnt); + + TimelineFilterList irt2 = new TimelineFilterList(Operator.OR); + irt2.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "task", + new HashSet(Arrays.asList("relatedto1")))); + irt2.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "task2", + new HashSet(Arrays.asList("relatedto4")))); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().isRelatedTo(irt2).build(), + new TimelineDataToRetrieve()); + assertEquals(2, entities.size()); + isRelatedToCnt = 0; + for (TimelineEntity timelineEntity : entities) { + isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size(); + if (!timelineEntity.getId().equals("hello") && + !timelineEntity.getId().equals("hello1")) { + Assert.fail("Entity ids' should have been hello and hello1"); + } + } + assertEquals(0, isRelatedToCnt); + + TimelineFilterList irt3 = new TimelineFilterList(); + irt3.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "task1", + new HashSet(Arrays.asList("relatedto3", "relatedto5")))); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().isRelatedTo(irt3).build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + isRelatedToCnt = 0; + for (TimelineEntity timelineEntity : entities) { + isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size(); + if (!timelineEntity.getId().equals("hello1")) { + Assert.fail("Entity id should have been hello1"); + } + } + assertEquals(0, isRelatedToCnt); + + TimelineFilterList irt4 = new TimelineFilterList(); + irt4.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "task1", + new HashSet(Arrays.asList("relatedto3")))); + irt4.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "dummy_task", + new HashSet(Arrays.asList("relatedto5")))); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().isRelatedTo(irt4).build(), + new TimelineDataToRetrieve()); + assertEquals(0, entities.size()); + + TimelineFilterList irt5 = new TimelineFilterList(); + irt5.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "task1", + new HashSet(Arrays.asList("relatedto3", "relatedto7")))); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().isRelatedTo(irt5).build(), + new TimelineDataToRetrieve()); + assertEquals(0, entities.size()); + + TimelineFilterList list1 = new TimelineFilterList(); + list1.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "task", + new HashSet(Arrays.asList("relatedto1")))); + list1.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "dummy_task", + new HashSet(Arrays.asList("relatedto4")))); + TimelineFilterList list2 = new TimelineFilterList(); + list2.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "task2", + new HashSet(Arrays.asList("relatedto4")))); + TimelineFilterList irt6 = new TimelineFilterList(Operator.OR, list1, list2); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().isRelatedTo(irt6).build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + isRelatedToCnt = 0; + for (TimelineEntity timelineEntity : entities) { + isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size(); + if (!timelineEntity.getId().equals("hello1")) { + Assert.fail("Entity id should have been hello1"); + } + } + assertEquals(0, isRelatedToCnt); + } + + @Test + public void testReadEntitiesRelatesTo() throws Exception { + TimelineFilterList rt = new TimelineFilterList(Operator.OR); + rt.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container2", + new HashSet(Arrays.asList("relatesto7")))); + rt.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container1", + new HashSet(Arrays.asList("relatesto4")))); + Set entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().relatesTo(rt).build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null, + null, null)); + assertEquals(2, entities.size()); + int relatesToCnt = 0; + for (TimelineEntity timelineEntity : entities) { + relatesToCnt += timelineEntity.getRelatesToEntities().size(); + if (!timelineEntity.getId().equals("hello") && + !timelineEntity.getId().equals("hello2")) { + Assert.fail("Entity ids' should have been hello and hello2"); + } + } + assertEquals(3, relatesToCnt); + + TimelineFilterList rt1 = new TimelineFilterList(); + rt1.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container", + new HashSet(Arrays.asList("relatesto1")))); + rt1.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.NOT_EQUAL, "container", + new HashSet(Arrays.asList("relatesto3")))); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().relatesTo(rt1).build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + relatesToCnt = 0; + for (TimelineEntity timelineEntity : entities) { + relatesToCnt += timelineEntity.getRelatesToEntities().size(); + if (!timelineEntity.getId().equals("hello1")) { + Assert.fail("Entity id should have been hello1"); + } + } + assertEquals(0, relatesToCnt); + + TimelineFilterList rt2 = new TimelineFilterList(Operator.OR); + rt2.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container2", + new HashSet(Arrays.asList("relatesto7")))); + rt2.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container1", + new HashSet(Arrays.asList("relatesto4")))); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().relatesTo(rt2).build(), + new TimelineDataToRetrieve()); + assertEquals(2, entities.size()); + relatesToCnt = 0; + for (TimelineEntity timelineEntity : entities) { + relatesToCnt += timelineEntity.getRelatesToEntities().size(); + if (!timelineEntity.getId().equals("hello") && + !timelineEntity.getId().equals("hello2")) { + Assert.fail("Entity ids' should have been hello and hello2"); + } + } + assertEquals(0, relatesToCnt); + + TimelineFilterList rt3 = new TimelineFilterList(); + rt3.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container", + new HashSet(Arrays.asList("relatesto1", "relatesto3")))); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().relatesTo(rt3).build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + relatesToCnt = 0; + for (TimelineEntity timelineEntity : entities) { + relatesToCnt += timelineEntity.getRelatesToEntities().size(); + if (!timelineEntity.getId().equals("hello")) { + Assert.fail("Entity id should have been hello"); + } + } + assertEquals(0, relatesToCnt); + + TimelineFilterList rt4 = new TimelineFilterList(); + rt4.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container", + new HashSet(Arrays.asList("relatesto1")))); + rt4.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "dummy_container", + new HashSet(Arrays.asList("relatesto5")))); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().relatesTo(rt4).build(), + new TimelineDataToRetrieve()); + assertEquals(0, entities.size()); + + TimelineFilterList rt5 = new TimelineFilterList(); + rt5.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container", + new HashSet(Arrays.asList("relatedto1", "relatesto8")))); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().relatesTo(rt5).build(), + new TimelineDataToRetrieve()); + assertEquals(0, entities.size()); + + TimelineFilterList list1 = new TimelineFilterList(); + list1.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container2", + new HashSet(Arrays.asList("relatesto7")))); + list1.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "dummy_container", + new HashSet(Arrays.asList("relatesto4")))); + TimelineFilterList list2 = new TimelineFilterList(); + list2.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container1", + new HashSet(Arrays.asList("relatesto4")))); + TimelineFilterList rt6 = new TimelineFilterList(Operator.OR, list1, list2); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().relatesTo(rt6).build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + relatesToCnt = 0; + for (TimelineEntity timelineEntity : entities) { + relatesToCnt += timelineEntity.getRelatesToEntities().size(); + if (!timelineEntity.getId().equals("hello")) { + Assert.fail("Entity id should have been hello"); + } + } + assertEquals(0, relatesToCnt); + + TimelineFilterList list3 = new TimelineFilterList(); + list3.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container", + new HashSet(Arrays.asList("relatesto1")))); + list3.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container1", + new HashSet(Arrays.asList("relatesto4")))); + TimelineFilterList list4 = new TimelineFilterList(); + list4.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container", + new HashSet(Arrays.asList("relatesto1")))); + list4.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "container", + new HashSet(Arrays.asList("relatesto2")))); + TimelineFilterList combinedList = + new TimelineFilterList(Operator.OR, list3, list4); + TimelineFilterList rt7 = new TimelineFilterList(Operator.AND, combinedList, + new TimelineKeyValuesFilter( + TimelineCompareOp.NOT_EQUAL, "container", + new HashSet(Arrays.asList("relatesto3")))); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().relatesTo(rt7).build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + relatesToCnt = 0; + for (TimelineEntity timelineEntity : entities) { + relatesToCnt += timelineEntity.getRelatesToEntities().size(); + if (!timelineEntity.getId().equals("hello1")) { + Assert.fail("Entity id should have been hello1"); + } + } + assertEquals(0, relatesToCnt); + } + + @Test + public void testReadEntitiesDefaultView() throws Exception { + TimelineEntity e1 = reader.getEntity( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", "hello"), + new TimelineDataToRetrieve()); + assertNotNull(e1); + assertEquals(1, e1.getInfo().size()); + assertTrue(e1.getConfigs().isEmpty() && + e1.getMetrics().isEmpty() && e1.getIsRelatedToEntities().isEmpty() && + e1.getRelatesToEntities().isEmpty()); + Set es1 = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().build(), + new TimelineDataToRetrieve()); + assertEquals(3, es1.size()); + for (TimelineEntity e : es1) { + assertTrue(e.getConfigs().isEmpty() && + e.getMetrics().isEmpty() && e.getIsRelatedToEntities().isEmpty() && + e.getRelatesToEntities().isEmpty()); + assertEquals(1, e.getInfo().size()); + } + } + + @Test + public void testReadEntitiesByFields() throws Exception { + TimelineEntity e1 = reader.getEntity( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", "hello"), + new TimelineDataToRetrieve( + null, null, EnumSet.of(Field.INFO, Field.CONFIGS), null, null, null)); + assertNotNull(e1); + assertEquals(3, e1.getConfigs().size()); + assertEquals(0, e1.getIsRelatedToEntities().size()); + Set es1 = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.IS_RELATED_TO, + Field.METRICS), null, null, null)); + assertEquals(3, es1.size()); + int metricsCnt = 0; + int isRelatedToCnt = 0; + int infoCnt = 0; + for (TimelineEntity entity : es1) { + metricsCnt += entity.getMetrics().size(); + isRelatedToCnt += entity.getIsRelatedToEntities().size(); + infoCnt += entity.getInfo().size(); + } + assertEquals(3, infoCnt); + assertEquals(4, isRelatedToCnt); + assertEquals(3, metricsCnt); + } + + @Test + public void testReadEntitiesConfigPrefix() throws Exception { + TimelineFilterList list = + new TimelineFilterList(Operator.OR, + new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_")); + TimelineEntity e1 = reader.getEntity( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", "hello"), + new TimelineDataToRetrieve(list, null, null, null, null, null)); + assertNotNull(e1); + assertEquals(1, e1.getConfigs().size()); + Set es1 = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().build(), + new TimelineDataToRetrieve(list, null, null, null, null, null)); + int cfgCnt = 0; + for (TimelineEntity entity : es1) { + cfgCnt += entity.getConfigs().size(); + for (String confKey : entity.getConfigs().keySet()) { + assertTrue("Config key returned should start with cfg_", + confKey.startsWith("cfg_")); + } + } + assertEquals(3, cfgCnt); + } + + @Test + public void testReadEntitiesConfigFilters() throws Exception { + TimelineFilterList list1 = new TimelineFilterList(); + list1.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "cfg_param1", "value1")); + list1.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "cfg_param2", "value2")); + TimelineFilterList list2 = new TimelineFilterList(); + list2.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "cfg_param1", "value3")); + list2.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "config_param2", "value2")); + TimelineFilterList confFilterList = + new TimelineFilterList(Operator.OR, list1, list2); + Set entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().configFilters(confFilterList) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS), + null, null, null)); + assertEquals(2, entities.size()); + int cfgCnt = 0; + for (TimelineEntity entity : entities) { + cfgCnt += entity.getConfigs().size(); + } + assertEquals(5, cfgCnt); + + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().configFilters(confFilterList) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null, + null, null)); + assertEquals(2, entities.size()); + cfgCnt = 0; + for (TimelineEntity entity : entities) { + cfgCnt += entity.getConfigs().size(); + } + assertEquals(5, cfgCnt); + + TimelineFilterList confFilterList1 = new TimelineFilterList( + new TimelineKeyValueFilter( + TimelineCompareOp.NOT_EQUAL, "cfg_param1", "value1")); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().configFilters(confFilterList1) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS), + null, null, null)); + assertEquals(1, entities.size()); + cfgCnt = 0; + for (TimelineEntity entity : entities) { + cfgCnt += entity.getConfigs().size(); + } + assertEquals(3, cfgCnt); + + TimelineFilterList confFilterList2 = new TimelineFilterList( + new TimelineKeyValueFilter( + TimelineCompareOp.NOT_EQUAL, "cfg_param1", "value1"), + new TimelineKeyValueFilter( + TimelineCompareOp.NOT_EQUAL, "config_param2", "value2")); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().configFilters(confFilterList2) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS), + null, null, null)); + assertEquals(0, entities.size()); + + TimelineFilterList confFilterList3 = new TimelineFilterList( + new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "dummy_config", "value1")); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().configFilters(confFilterList3) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS), + null, null, null)); + assertEquals(0, entities.size()); + + TimelineFilterList confFilterList4 = new TimelineFilterList( + new TimelineKeyValueFilter( + TimelineCompareOp.NOT_EQUAL, "dummy_config", "value1")); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().configFilters(confFilterList4) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS), + null, null, null)); + assertEquals(0, entities.size()); + + TimelineFilterList confFilterList5 = new TimelineFilterList( + new TimelineKeyValueFilter( + TimelineCompareOp.NOT_EQUAL, "dummy_config", "value1", false)); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().configFilters(confFilterList5) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS), + null, null, null)); + assertEquals(3, entities.size()); + } + + @Test + public void testReadEntitiesConfigFilterPrefix() throws Exception { + TimelineFilterList confFilterList = new TimelineFilterList(); + confFilterList.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "cfg_param1", "value1")); + TimelineFilterList list = + new TimelineFilterList(Operator.OR, + new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_")); + Set entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().configFilters(confFilterList) + .build(), + new TimelineDataToRetrieve(list, null, null, null, null, null)); + assertEquals(1, entities.size()); + int cfgCnt = 0; + for (TimelineEntity entity : entities) { + cfgCnt += entity.getConfigs().size(); + for (String confKey : entity.getConfigs().keySet()) { + assertTrue("Config key returned should start with cfg_", + confKey.startsWith("cfg_")); + } + } + assertEquals(2, cfgCnt); + TimelineFilterList list1 = new TimelineFilterList(); + list1.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "cfg_param1", "value1")); + list1.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "cfg_param2", "value2")); + TimelineFilterList list2 = new TimelineFilterList(); + list2.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "cfg_param1", "value3")); + list2.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "config_param2", "value2")); + TimelineFilterList confFilterList1 = + new TimelineFilterList(Operator.OR, list1, list2); + TimelineFilterList confsToRetrieve = + new TimelineFilterList(Operator.OR, + new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "config_")); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().configFilters(confFilterList1) + .build(), + new TimelineDataToRetrieve(confsToRetrieve, null, null, null, null, + null)); + assertEquals(2, entities.size()); + cfgCnt = 0; + for (TimelineEntity entity : entities) { + cfgCnt += entity.getConfigs().size(); + for (String confKey : entity.getConfigs().keySet()) { + assertTrue("Config key returned should start with config_", + confKey.startsWith("config_")); + } + } + assertEquals(2, cfgCnt); + } + + @Test + public void testReadEntitiesMetricPrefix() throws Exception { + TimelineFilterList list = + new TimelineFilterList(Operator.OR, + new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_")); + TimelineEntity e1 = reader.getEntity( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", "hello"), + new TimelineDataToRetrieve(null, list, null, null, null, null)); + assertNotNull(e1); + assertEquals(1, e1.getMetrics().size()); + Set es1 = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().build(), + new TimelineDataToRetrieve(null, list, null, null, null, null)); + int metricCnt = 0; + for (TimelineEntity entity : es1) { + metricCnt += entity.getMetrics().size(); + for (TimelineMetric metric : entity.getMetrics()) { + assertTrue("Metric Id returned should start with MAP1_", + metric.getId().startsWith("MAP1_")); + } + } + assertEquals(2, metricCnt); + } + + @Test + public void testReadEntitiesMetricTimeRange() throws Exception { + Set entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS), + 100, null, null)); + assertEquals(3, entities.size()); + int metricTimeSeriesCnt = 0; + int metricCnt = 0; + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + for (TimelineMetric m : entity.getMetrics()) { + metricTimeSeriesCnt += m.getValues().size(); + } + } + assertEquals(3, metricCnt); + assertEquals(13, metricTimeSeriesCnt); + + entities = reader.getEntities(new TimelineReaderContext("cluster1", "user1", + "some_flow_name", 1002345678919L, "application_1231111111_1111", + "world", null), new TimelineEntityFilters.Builder().build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS), + 100, CURRENT_TIME - 40000, CURRENT_TIME)); + assertEquals(3, entities.size()); + metricCnt = 0; + metricTimeSeriesCnt = 0; + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + for (TimelineMetric m : entity.getMetrics()) { + for (Long ts : m.getValues().keySet()) { + assertTrue(ts >= CURRENT_TIME - 40000 && ts <= CURRENT_TIME); + } + metricTimeSeriesCnt += m.getValues().size(); + } + } + assertEquals(3, metricCnt); + assertEquals(5, metricTimeSeriesCnt); + + TimelineEntity entity = reader.getEntity(new TimelineReaderContext( + "cluster1", "user1", "some_flow_name", 1002345678919L, + "application_1231111111_1111", "world", "hello"), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS), 100, + CURRENT_TIME - 40000, CURRENT_TIME)); + assertNotNull(entity); + assertEquals(2, entity.getMetrics().size()); + metricTimeSeriesCnt = 0; + for (TimelineMetric m : entity.getMetrics()) { + for (Long ts : m.getValues().keySet()) { + assertTrue(ts >= CURRENT_TIME - 40000 && ts <= CURRENT_TIME); + } + metricTimeSeriesCnt += m.getValues().size(); + } + assertEquals(3, metricTimeSeriesCnt); + } + + @Test + public void testReadEntitiesMetricFilters() throws Exception { + TimelineFilterList list1 = new TimelineFilterList(); + list1.addFilter(new TimelineCompareFilter( + TimelineCompareOp.GREATER_OR_EQUAL, "MAP1_SLOT_MILLIS", 50000000900L)); + TimelineFilterList list2 = new TimelineFilterList(); + list2.addFilter(new TimelineCompareFilter( + TimelineCompareOp.LESS_THAN, "MAP_SLOT_MILLIS", 80000000000L)); + list2.addFilter(new TimelineCompareFilter( + TimelineCompareOp.EQUAL, "MAP1_BYTES", 50)); + TimelineFilterList metricFilterList = + new TimelineFilterList(Operator.OR, list1, list2); + Set entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS), + null, null, null)); + assertEquals(2, entities.size()); + int metricCnt = 0; + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + } + assertEquals(3, metricCnt); + + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null, + null, null)); + assertEquals(2, entities.size()); + metricCnt = 0; + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + } + assertEquals(3, metricCnt); + + TimelineFilterList metricFilterList1 = new TimelineFilterList( + new TimelineCompareFilter( + TimelineCompareOp.LESS_OR_EQUAL, "MAP_SLOT_MILLIS", 80000000000L), + new TimelineCompareFilter( + TimelineCompareOp.NOT_EQUAL, "MAP1_BYTES", 30)); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList1) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS), + null, null, null)); + assertEquals(1, entities.size()); + metricCnt = 0; + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + } + assertEquals(2, metricCnt); + + TimelineFilterList metricFilterList2 = new TimelineFilterList( + new TimelineCompareFilter( + TimelineCompareOp.LESS_THAN, "MAP_SLOT_MILLIS", 40000000000L), + new TimelineCompareFilter( + TimelineCompareOp.NOT_EQUAL, "MAP1_BYTES", 30)); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList2) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS), + null, null, null)); + assertEquals(0, entities.size()); + + TimelineFilterList metricFilterList3 = new TimelineFilterList( + new TimelineCompareFilter( + TimelineCompareOp.EQUAL, "dummy_metric", 5)); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList3) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS), + null, null, null)); + assertEquals(0, entities.size()); + + TimelineFilterList metricFilterList4 = new TimelineFilterList( + new TimelineCompareFilter( + TimelineCompareOp.NOT_EQUAL, "dummy_metric", 5)); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList4) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS), + null, null, null)); + assertEquals(0, entities.size()); + + TimelineFilterList metricFilterList5 = new TimelineFilterList( + new TimelineCompareFilter( + TimelineCompareOp.NOT_EQUAL, "dummy_metric", 5, false)); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList5) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS), + null, null, null)); + assertEquals(3, entities.size()); + } + + @Test + public void testReadEntitiesMetricFilterPrefix() throws Exception { + TimelineFilterList metricFilterList = new TimelineFilterList(); + metricFilterList.addFilter(new TimelineCompareFilter( + TimelineCompareOp.GREATER_OR_EQUAL, "MAP1_SLOT_MILLIS", 0L)); + TimelineFilterList list = + new TimelineFilterList(Operator.OR, + new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_")); + Set entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList) + .build(), + new TimelineDataToRetrieve(null, list, null, null, null, null)); + assertEquals(1, entities.size()); + int metricCnt = 0; + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + for (TimelineMetric metric : entity.getMetrics()) { + assertTrue("Metric Id returned should start with MAP1_", + metric.getId().startsWith("MAP1_")); + } + } + assertEquals(1, metricCnt); + + TimelineFilterList list1 = new TimelineFilterList(); + list1.addFilter(new TimelineCompareFilter( + TimelineCompareOp.GREATER_OR_EQUAL, "MAP1_SLOT_MILLIS", 50000000900L)); + TimelineFilterList list2 = new TimelineFilterList(); + list2.addFilter(new TimelineCompareFilter( + TimelineCompareOp.LESS_THAN, "MAP_SLOT_MILLIS", 80000000000L)); + list2.addFilter(new TimelineCompareFilter( + TimelineCompareOp.EQUAL, "MAP1_BYTES", 50)); + TimelineFilterList metricFilterList1 = + new TimelineFilterList(Operator.OR, list1, list2); + TimelineFilterList metricsToRetrieve = new TimelineFilterList(Operator.OR, + new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_")); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList1) + .build(), + new TimelineDataToRetrieve( + null, metricsToRetrieve, EnumSet.of(Field.METRICS), null, null, null)); + assertEquals(2, entities.size()); + metricCnt = 0; + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + for (TimelineMetric metric : entity.getMetrics()) { + assertEquals(TimelineMetric.Type.SINGLE_VALUE, metric.getType()); + assertEquals(1, metric.getValues().size()); + assertTrue("Metric Id returned should start with MAP1_", + metric.getId().startsWith("MAP1_")); + } + } + assertEquals(2, metricCnt); + + entities = reader.getEntities(new TimelineReaderContext("cluster1", "user1", + "some_flow_name", 1002345678919L, "application_1231111111_1111", + "world", null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList1) + .build(), + new TimelineDataToRetrieve(null, metricsToRetrieve, + EnumSet.of(Field.METRICS), Integer.MAX_VALUE, null, null)); + assertEquals(2, entities.size()); + metricCnt = 0; + int metricValCnt = 0; + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + for (TimelineMetric metric : entity.getMetrics()) { + metricValCnt += metric.getValues().size(); + assertTrue("Metric Id returned should start with MAP1_", + metric.getId().startsWith("MAP1_")); + } + } + assertEquals(2, metricCnt); + assertEquals(7, metricValCnt); + } + + @Test + public void testReadEntitiesInfoFilters() throws Exception { + TimelineFilterList list1 = new TimelineFilterList(); + list1.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "infoMapKey3", 71.4)); + list1.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "infoMapKey1", "infoMapValue2")); + TimelineFilterList list2 = new TimelineFilterList(); + list2.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "infoMapKey1", "infoMapValue1")); + list2.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "infoMapKey2", 10)); + TimelineFilterList infoFilterList = + new TimelineFilterList(Operator.OR, list1, list2); + Set entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().infoFilters(infoFilterList).build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null, + null, null)); + assertEquals(2, entities.size()); + int infoCnt = 0; + for (TimelineEntity entity : entities) { + infoCnt += entity.getInfo().size(); + } + assertEquals(7, infoCnt); + + TimelineFilterList infoFilterList1 = new TimelineFilterList( + new TimelineKeyValueFilter( + TimelineCompareOp.NOT_EQUAL, "infoMapKey1", "infoMapValue1")); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().infoFilters(infoFilterList1) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null, + null, null)); + assertEquals(1, entities.size()); + infoCnt = 0; + for (TimelineEntity entity : entities) { + infoCnt += entity.getInfo().size(); + } + assertEquals(4, infoCnt); + + TimelineFilterList infoFilterList2 = new TimelineFilterList( + new TimelineKeyValueFilter( + TimelineCompareOp.NOT_EQUAL, "infoMapKey1", "infoMapValue2"), + new TimelineKeyValueFilter( + TimelineCompareOp.NOT_EQUAL, "infoMapKey3", 71.4)); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().infoFilters(infoFilterList2) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null, + null, null)); + assertEquals(0, entities.size()); + + TimelineFilterList infoFilterList3 = new TimelineFilterList( + new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "dummy_info", "some_value")); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().infoFilters(infoFilterList3) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null, + null, null)); + assertEquals(0, entities.size()); + + TimelineFilterList infoFilterList4 = new TimelineFilterList( + new TimelineKeyValueFilter( + TimelineCompareOp.NOT_EQUAL, "dummy_info", "some_value")); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().infoFilters(infoFilterList4) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null, + null, null)); + assertEquals(0, entities.size()); + + TimelineFilterList infoFilterList5 = new TimelineFilterList( + new TimelineKeyValueFilter( + TimelineCompareOp.NOT_EQUAL, "dummy_info", "some_value", false)); + entities = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", "world", null), + new TimelineEntityFilters.Builder().infoFilters(infoFilterList5) + .build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null, + null, null)); + assertEquals(3, entities.size()); + } + + @Test(timeout = 90000) + public void testListTypesInApp() throws Exception { + Set types = reader.getEntityTypes( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1111", null, null)); + assertEquals(4, types.size()); + + types = reader.getEntityTypes( + new TimelineReaderContext("cluster1", null, null, + null, "application_1231111111_1111", null, null)); + assertEquals(4, types.size()); + + types = reader.getEntityTypes( + new TimelineReaderContext("cluster1", null, null, + null, "application_1231111111_1112", null, null)); + assertEquals(4, types.size()); + + types = reader.getEntityTypes( + new TimelineReaderContext("cluster1", "user1", "some_flow_name", + 1002345678919L, "application_1231111111_1113", null, null)); + assertEquals(0, types.size()); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + util.shutdownMiniCluster(); + } + + private boolean verifyRowKeyForSubApplication(byte[] rowKey, String suAppUser, + String cluster, String user, TimelineEntity te) { + SubApplicationRowKey key = SubApplicationRowKey.parseRowKey(rowKey); + assertEquals(suAppUser, key.getSubAppUserId()); + assertEquals(cluster, key.getClusterId()); + assertEquals(te.getType(), key.getEntityType()); + assertEquals(te.getId(), key.getEntityId()); + assertEquals(user, key.getUserId()); + return true; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageSchema.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageSchema.java new file mode 100644 index 00000000000..0dcd171ec52 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageSchema.java @@ -0,0 +1,135 @@ +/** + * 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.timelineservice.storage; + +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Table; + +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable; + +/** + * Unit tests for checking different schema prefixes. + */ +public class TestHBaseTimelineStorageSchema { + private static HBaseTestingUtility util; + + @BeforeClass + public static void setupBeforeClass() throws Exception { + util = new HBaseTestingUtility(); + Configuration conf = util.getConfiguration(); + conf.setInt("hfile.format.version", 3); + util.startMiniCluster(); + } + + @Test + public void createWithDefaultPrefix() throws IOException { + Configuration hbaseConf = util.getConfiguration(); + DataGeneratorForTest.createSchema(hbaseConf); + Connection conn = null; + conn = ConnectionFactory.createConnection(hbaseConf); + Admin admin = conn.getAdmin(); + + TableName entityTableName = BaseTable.getTableName(hbaseConf, + EntityTable.TABLE_NAME_CONF_NAME, EntityTable.DEFAULT_TABLE_NAME); + assertTrue(admin.tableExists(entityTableName)); + assertTrue(entityTableName.getNameAsString().startsWith( + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_HBASE_SCHEMA_PREFIX)); + Table entityTable = conn.getTable(BaseTable.getTableName(hbaseConf, + EntityTable.TABLE_NAME_CONF_NAME, EntityTable.DEFAULT_TABLE_NAME)); + assertNotNull(entityTable); + + TableName flowRunTableName = BaseTable.getTableName(hbaseConf, + FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME); + assertTrue(admin.tableExists(flowRunTableName)); + assertTrue(flowRunTableName.getNameAsString().startsWith( + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_HBASE_SCHEMA_PREFIX)); + Table flowRunTable = conn.getTable(BaseTable.getTableName(hbaseConf, + FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME)); + assertNotNull(flowRunTable); + } + + @Test + public void createWithSetPrefix() throws IOException { + Configuration hbaseConf = util.getConfiguration(); + String prefix = "unit-test."; + hbaseConf.set(YarnConfiguration.TIMELINE_SERVICE_HBASE_SCHEMA_PREFIX_NAME, + prefix); + DataGeneratorForTest.createSchema(hbaseConf); + Connection conn = null; + conn = ConnectionFactory.createConnection(hbaseConf); + Admin admin = conn.getAdmin(); + + TableName entityTableName = BaseTable.getTableName(hbaseConf, + EntityTable.TABLE_NAME_CONF_NAME, EntityTable.DEFAULT_TABLE_NAME); + assertTrue(admin.tableExists(entityTableName)); + assertTrue(entityTableName.getNameAsString().startsWith(prefix)); + Table entityTable = conn.getTable(BaseTable.getTableName(hbaseConf, + EntityTable.TABLE_NAME_CONF_NAME, EntityTable.DEFAULT_TABLE_NAME)); + assertNotNull(entityTable); + + TableName flowRunTableName = BaseTable.getTableName(hbaseConf, + FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME); + assertTrue(admin.tableExists(flowRunTableName)); + assertTrue(flowRunTableName.getNameAsString().startsWith(prefix)); + Table flowRunTable = conn.getTable(BaseTable.getTableName(hbaseConf, + FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME)); + assertNotNull(flowRunTable); + + // create another set with a diff prefix + hbaseConf + .unset(YarnConfiguration.TIMELINE_SERVICE_HBASE_SCHEMA_PREFIX_NAME); + prefix = "yet-another-unit-test."; + hbaseConf.set(YarnConfiguration.TIMELINE_SERVICE_HBASE_SCHEMA_PREFIX_NAME, + prefix); + DataGeneratorForTest.createSchema(hbaseConf); + entityTableName = BaseTable.getTableName(hbaseConf, + EntityTable.TABLE_NAME_CONF_NAME, EntityTable.DEFAULT_TABLE_NAME); + assertTrue(admin.tableExists(entityTableName)); + assertTrue(entityTableName.getNameAsString().startsWith(prefix)); + entityTable = conn.getTable(BaseTable.getTableName(hbaseConf, + EntityTable.TABLE_NAME_CONF_NAME, EntityTable.DEFAULT_TABLE_NAME)); + assertNotNull(entityTable); + + flowRunTableName = BaseTable.getTableName(hbaseConf, + FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME); + assertTrue(admin.tableExists(flowRunTableName)); + assertTrue(flowRunTableName.getNameAsString().startsWith(prefix)); + flowRunTable = conn.getTable(BaseTable.getTableName(hbaseConf, + FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME)); + assertNotNull(flowRunTable); + hbaseConf + .unset(YarnConfiguration.TIMELINE_SERVICE_HBASE_SCHEMA_PREFIX_NAME); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestFlowDataGenerator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestFlowDataGenerator.java new file mode 100644 index 00000000000..8ab32dfff31 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestFlowDataGenerator.java @@ -0,0 +1,386 @@ +/** + * 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.timelineservice.storage.flow; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric.Type; +import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants; +import org.apache.hadoop.conf.Configuration; + +/** + * Generates the data/entities for the FlowRun and FlowActivity Tables. + */ +public final class TestFlowDataGenerator { + private TestFlowDataGenerator() { + } + + private static final String METRIC_1 = "MAP_SLOT_MILLIS"; + private static final String METRIC_2 = "HDFS_BYTES_READ"; + public static final long END_TS_INCR = 10000L; + + public static TimelineEntity getEntityMetricsApp1(long insertTs, + Configuration c1) { + TimelineEntity entity = new TimelineEntity(); + String id = "flowRunMetrics_test"; + String type = TimelineEntityType.YARN_APPLICATION.toString(); + entity.setId(id); + entity.setType(type); + long cTime = 1425016501000L; + entity.setCreatedTime(cTime); + + // add metrics + Set metrics = new HashSet<>(); + TimelineMetric m1 = new TimelineMetric(); + m1.setId(METRIC_1); + Map metricValues = new HashMap(); + long ts = insertTs; + + for (int k = 1; k < 100; k++) { + metricValues.put(ts - k * 200000L, 20L); + } + metricValues.put(ts - 80000, 40L); + m1.setType(Type.TIME_SERIES); + m1.setValues(metricValues); + metrics.add(m1); + + TimelineMetric m2 = new TimelineMetric(); + m2.setId(METRIC_2); + metricValues = new HashMap(); + ts = System.currentTimeMillis(); + for (int k = 1; k < 100; k++) { + metricValues.put(ts - k*100000L, 31L); + } + + metricValues.put(ts - 80000, 57L); + m2.setType(Type.TIME_SERIES); + m2.setValues(metricValues); + metrics.add(m2); + + entity.addMetrics(metrics); + return entity; + } + + + public static TimelineEntity getEntityMetricsApp1Complete(long insertTs, + Configuration c1) { + TimelineEntity entity = new TimelineEntity(); + String id = "flowRunMetrics_test"; + String type = TimelineEntityType.YARN_APPLICATION.toString(); + entity.setId(id); + entity.setType(type); + long cTime = 1425016501000L; + entity.setCreatedTime(cTime); + + // add metrics + Set metrics = new HashSet<>(); + TimelineMetric m1 = new TimelineMetric(); + m1.setId(METRIC_1); + Map metricValues = new HashMap(); + long ts = insertTs; + + metricValues.put(ts - 80000, 40L); + m1.setType(Type.TIME_SERIES); + m1.setValues(metricValues); + metrics.add(m1); + + TimelineMetric m2 = new TimelineMetric(); + m2.setId(METRIC_2); + metricValues = new HashMap(); + ts = insertTs; + metricValues.put(ts - 80000, 57L); + m2.setType(Type.TIME_SERIES); + m2.setValues(metricValues); + metrics.add(m2); + + entity.addMetrics(metrics); + + TimelineEvent event = new TimelineEvent(); + event.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE); + event.setTimestamp(insertTs); + event.addInfo("done", "insertTs=" + insertTs); + entity.addEvent(event); + return entity; + } + + + public static TimelineEntity getEntityMetricsApp1(long insertTs) { + TimelineEntity entity = new TimelineEntity(); + String id = "flowRunMetrics_test"; + String type = TimelineEntityType.YARN_APPLICATION.toString(); + entity.setId(id); + entity.setType(type); + long cTime = 1425016501000L; + entity.setCreatedTime(cTime); + + // add metrics + Set metrics = new HashSet<>(); + TimelineMetric m1 = new TimelineMetric(); + m1.setId(METRIC_1); + Map metricValues = new HashMap(); + long ts = insertTs; + metricValues.put(ts - 100000, 2L); + metricValues.put(ts - 80000, 40L); + m1.setType(Type.TIME_SERIES); + m1.setValues(metricValues); + metrics.add(m1); + + TimelineMetric m2 = new TimelineMetric(); + m2.setId(METRIC_2); + metricValues = new HashMap(); + ts = insertTs; + metricValues.put(ts - 100000, 31L); + metricValues.put(ts - 80000, 57L); + m2.setType(Type.TIME_SERIES); + m2.setValues(metricValues); + metrics.add(m2); + + entity.addMetrics(metrics); + TimelineEvent event = new TimelineEvent(); + event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE); + long endTs = 1439379885000L; + event.setTimestamp(endTs); + String expKey = "foo_event_greater"; + String expVal = "test_app_greater"; + event.addInfo(expKey, expVal); + entity.addEvent(event); + return entity; + } + + public static TimelineEntity getEntityMetricsApp2(long insertTs) { + TimelineEntity entity = new TimelineEntity(); + String id = "flowRunMetrics_test"; + String type = TimelineEntityType.YARN_APPLICATION.toString(); + entity.setId(id); + entity.setType(type); + long cTime = 1425016501000L; + entity.setCreatedTime(cTime); + // add metrics + Set metrics = new HashSet<>(); + TimelineMetric m1 = new TimelineMetric(); + m1.setId(METRIC_1); + Map metricValues = new HashMap(); + long ts = insertTs; + metricValues.put(ts - 100000, 5L); + metricValues.put(ts - 80000, 101L); + m1.setType(Type.TIME_SERIES); + m1.setValues(metricValues); + metrics.add(m1); + entity.addMetrics(metrics); + TimelineEvent event = new TimelineEvent(); + event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE); + long endTs = 1439379885000L; + event.setTimestamp(endTs); + String expKey = "foo_event_greater"; + String expVal = "test_app_greater"; + event.addInfo(expKey, expVal); + entity.addEvent(event); + return entity; + } + + public static TimelineEntity getEntity1() { + TimelineEntity entity = new TimelineEntity(); + String id = "flowRunHello"; + String type = TimelineEntityType.YARN_APPLICATION.toString(); + entity.setId(id); + entity.setType(type); + long cTime = 1425026901000L; + entity.setCreatedTime(cTime); + // add metrics + Set metrics = new HashSet<>(); + TimelineMetric m1 = new TimelineMetric(); + m1.setId(METRIC_1); + Map metricValues = new HashMap(); + long ts = System.currentTimeMillis(); + metricValues.put(ts - 120000, 100000000L); + metricValues.put(ts - 100000, 200000000L); + metricValues.put(ts - 80000, 300000000L); + metricValues.put(ts - 60000, 400000000L); + metricValues.put(ts - 40000, 50000000000L); + metricValues.put(ts - 20000, 60000000000L); + m1.setType(Type.TIME_SERIES); + m1.setValues(metricValues); + metrics.add(m1); + entity.addMetrics(metrics); + + TimelineEvent event = new TimelineEvent(); + event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE); + event.setTimestamp(cTime); + String expKey = "foo_event"; + Object expVal = "test"; + event.addInfo(expKey, expVal); + entity.addEvent(event); + + event = new TimelineEvent(); + event.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE); + long expTs = cTime + 21600000; // start time + 6hrs + event.setTimestamp(expTs); + event.addInfo(expKey, expVal); + entity.addEvent(event); + + return entity; + } + + public static TimelineEntity getAFullEntity(long ts, long endTs) { + TimelineEntity entity = new TimelineEntity(); + String id = "flowRunFullEntity"; + String type = TimelineEntityType.YARN_APPLICATION.toString(); + entity.setId(id); + entity.setType(type); + entity.setCreatedTime(ts); + // add metrics + Set metrics = new HashSet<>(); + TimelineMetric m1 = new TimelineMetric(); + m1.setId(METRIC_1); + Map metricValues = new HashMap(); + metricValues.put(ts - 120000, 100000000L); + metricValues.put(ts - 100000, 200000000L); + metricValues.put(ts - 80000, 300000000L); + metricValues.put(ts - 60000, 400000000L); + metricValues.put(ts - 40000, 50000000000L); + metricValues.put(ts - 20000, 60000000000L); + m1.setType(Type.TIME_SERIES); + m1.setValues(metricValues); + metrics.add(m1); + TimelineMetric m2 = new TimelineMetric(); + m2.setId(METRIC_2); + metricValues = new HashMap(); + metricValues.put(ts - 900000, 31L); + metricValues.put(ts - 30000, 57L); + m2.setType(Type.TIME_SERIES); + m2.setValues(metricValues); + metrics.add(m2); + entity.addMetrics(metrics); + + TimelineEvent event = new TimelineEvent(); + event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE); + event.setTimestamp(ts); + String expKey = "foo_event"; + Object expVal = "test"; + event.addInfo(expKey, expVal); + entity.addEvent(event); + + event = new TimelineEvent(); + event.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE); + long expTs = ts + 21600000; // start time + 6hrs + event.setTimestamp(expTs); + event.addInfo(expKey, expVal); + entity.addEvent(event); + + return entity; + } + + public static TimelineEntity getEntityGreaterStartTime(long startTs) { + TimelineEntity entity = new TimelineEntity(); + entity.setCreatedTime(startTs); + entity.setId("flowRunHello with greater start time"); + String type = TimelineEntityType.YARN_APPLICATION.toString(); + entity.setType(type); + TimelineEvent event = new TimelineEvent(); + event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE); + event.setTimestamp(startTs); + String expKey = "foo_event_greater"; + String expVal = "test_app_greater"; + event.addInfo(expKey, expVal); + entity.addEvent(event); + return entity; + } + + public static TimelineEntity getEntityMaxEndTime(long endTs) { + TimelineEntity entity = new TimelineEntity(); + entity.setId("flowRunHello Max End time"); + entity.setType(TimelineEntityType.YARN_APPLICATION.toString()); + TimelineEvent event = new TimelineEvent(); + event.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE); + event.setTimestamp(endTs); + String expKey = "foo_even_max_ finished"; + String expVal = "test_app_max_finished"; + event.addInfo(expKey, expVal); + entity.addEvent(event); + return entity; + } + + public static TimelineEntity getEntityMinStartTime(long startTs) { + TimelineEntity entity = new TimelineEntity(); + String id = "flowRunHelloMInStartTime"; + String type = TimelineEntityType.YARN_APPLICATION.toString(); + entity.setId(id); + entity.setType(type); + entity.setCreatedTime(startTs); + TimelineEvent event = new TimelineEvent(); + event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE); + event.setTimestamp(startTs); + entity.addEvent(event); + return entity; + } + + public static TimelineEntity getMinFlushEntity(long startTs) { + TimelineEntity entity = new TimelineEntity(); + String id = "flowRunHelloFlushEntityMin"; + String type = TimelineEntityType.YARN_APPLICATION.toString(); + entity.setId(id); + entity.setType(type); + entity.setCreatedTime(startTs); + TimelineEvent event = new TimelineEvent(); + event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE); + event.setTimestamp(startTs); + entity.addEvent(event); + return entity; + } + + public static TimelineEntity getMaxFlushEntity(long startTs) { + TimelineEntity entity = new TimelineEntity(); + String id = "flowRunHelloFlushEntityMax"; + String type = TimelineEntityType.YARN_APPLICATION.toString(); + entity.setId(id); + entity.setType(type); + entity.setCreatedTime(startTs); + + TimelineEvent event = new TimelineEvent(); + event.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE); + event.setTimestamp(startTs + END_TS_INCR); + entity.addEvent(event); + return entity; + } + + public static TimelineEntity getFlowApp1(long appCreatedTime) { + TimelineEntity entity = new TimelineEntity(); + String id = "flowActivity_test"; + String type = TimelineEntityType.YARN_APPLICATION.toString(); + entity.setId(id); + entity.setType(type); + entity.setCreatedTime(appCreatedTime); + + TimelineEvent event = new TimelineEvent(); + event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE); + event.setTimestamp(appCreatedTime); + String expKey = "foo_event"; + Object expVal = "test"; + event.addInfo(expKey, expVal); + entity.addEvent(event); + + return entity; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java new file mode 100644 index 00000000000..4bf221e2ad6 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java @@ -0,0 +1,497 @@ +/** + * 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.timelineservice.storage.flow; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.util.Map; +import java.util.NavigableSet; +import java.util.Set; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; +import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper; +import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext; +import org.apache.hadoop.yarn.server.timelineservice.storage.DataGeneratorForTest; +import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineReaderImpl; +import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +/** + * Tests the FlowRun and FlowActivity Tables. + */ +public class TestHBaseStorageFlowActivity { + + private static HBaseTestingUtility util; + + @BeforeClass + public static void setupBeforeClass() throws Exception { + util = new HBaseTestingUtility(); + Configuration conf = util.getConfiguration(); + conf.setInt("hfile.format.version", 3); + util.startMiniCluster(); + DataGeneratorForTest.createSchema(util.getConfiguration()); + } + + /** + * Writes 4 timeline entities belonging to one flow run through the + * {@link HBaseTimelineWriterImpl} + * + * Checks the flow run table contents + * + * The first entity has a created event, metrics and a finish event. + * + * The second entity has a created event and this is the entity with smallest + * start time. This should be the start time for the flow run. + * + * The third entity has a finish event and this is the entity with the max end + * time. This should be the end time for the flow run. + * + * The fourth entity has a created event which has a start time that is + * greater than min start time. + * + * The test also checks in the flow activity table that one entry has been + * made for all of these 4 application entities since they belong to the same + * flow run. + */ + @Test + public void testWriteFlowRunMinMax() throws Exception { + + TimelineEntities te = new TimelineEntities(); + te.addEntity(TestFlowDataGenerator.getEntity1()); + + HBaseTimelineWriterImpl hbi = null; + Configuration c1 = util.getConfiguration(); + String cluster = "testWriteFlowRunMinMaxToHBase_cluster1"; + String user = "testWriteFlowRunMinMaxToHBase_user1"; + String flow = "testing_flowRun_flow_name"; + String flowVersion = "CF7022C10F1354"; + long runid = 1002345678919L; + String appName = "application_100000000000_1111"; + long minStartTs = 1424995200300L; + long greaterStartTs = 1424995200300L + 864000L; + long endTs = 1424995200300L + 86000000L; + TimelineEntity entityMinStartTime = TestFlowDataGenerator + .getEntityMinStartTime(minStartTs); + + try { + hbi = new HBaseTimelineWriterImpl(); + hbi.init(c1); + + UserGroupInformation remoteUser = + UserGroupInformation.createRemoteUser(user); + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, appName), te, remoteUser); + + // write another entity with the right min start time + te = new TimelineEntities(); + te.addEntity(entityMinStartTime); + appName = "application_100000000000_3333"; + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, appName), te, remoteUser); + + // writer another entity for max end time + TimelineEntity entityMaxEndTime = TestFlowDataGenerator + .getEntityMaxEndTime(endTs); + te = new TimelineEntities(); + te.addEntity(entityMaxEndTime); + appName = "application_100000000000_4444"; + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, appName), te, remoteUser); + + // writer another entity with greater start time + TimelineEntity entityGreaterStartTime = TestFlowDataGenerator + .getEntityGreaterStartTime(greaterStartTs); + te = new TimelineEntities(); + te.addEntity(entityGreaterStartTime); + appName = "application_1000000000000000_2222"; + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, appName), te, remoteUser); + + // flush everything to hbase + hbi.flush(); + } finally { + if (hbi != null) { + hbi.close(); + } + } + + Connection conn = ConnectionFactory.createConnection(c1); + // check in flow activity table + Table table1 = conn.getTable( + BaseTable.getTableName(c1, FlowActivityTable.TABLE_NAME_CONF_NAME, + FlowActivityTable.DEFAULT_TABLE_NAME)); + byte[] startRow = + new FlowActivityRowKey(cluster, minStartTs, user, flow).getRowKey(); + Get g = new Get(startRow); + Result r1 = table1.get(g); + assertNotNull(r1); + assertTrue(!r1.isEmpty()); + Map values = r1.getFamilyMap(FlowActivityColumnFamily.INFO + .getBytes()); + assertEquals(1, values.size()); + byte[] row = r1.getRow(); + FlowActivityRowKey flowActivityRowKey = FlowActivityRowKey.parseRowKey(row); + assertNotNull(flowActivityRowKey); + assertEquals(cluster, flowActivityRowKey.getClusterId()); + assertEquals(user, flowActivityRowKey.getUserId()); + assertEquals(flow, flowActivityRowKey.getFlowName()); + Long dayTs = HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(minStartTs); + assertEquals(dayTs, flowActivityRowKey.getDayTimestamp()); + assertEquals(1, values.size()); + checkFlowActivityRunId(runid, flowVersion, values); + + // use the timeline reader to verify data + HBaseTimelineReaderImpl hbr = null; + try { + hbr = new HBaseTimelineReaderImpl(); + hbr.init(c1); + hbr.start(); + // get the flow activity entity + Set entities = hbr.getEntities( + new TimelineReaderContext(cluster, null, null, null, null, + TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), null), + new TimelineEntityFilters.Builder().entityLimit(10L).build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + for (TimelineEntity e : entities) { + FlowActivityEntity flowActivity = (FlowActivityEntity)e; + assertEquals(cluster, flowActivity.getCluster()); + assertEquals(user, flowActivity.getUser()); + assertEquals(flow, flowActivity.getFlowName()); + assertEquals(dayTs, Long.valueOf(flowActivity.getDate().getTime())); + Set flowRuns = flowActivity.getFlowRuns(); + assertEquals(1, flowRuns.size()); + } + } finally { + if (hbr != null) { + hbr.close(); + } + } + } + + /** + * Write 1 application entity and checks the record for today in the flow + * activity table. + */ + @Test + public void testWriteFlowActivityOneFlow() throws Exception { + String cluster = "testWriteFlowActivityOneFlow_cluster1"; + String user = "testWriteFlowActivityOneFlow_user1"; + String flow = "flow_activity_test_flow_name"; + String flowVersion = "A122110F135BC4"; + long runid = 1001111178919L; + + TimelineEntities te = new TimelineEntities(); + long appCreatedTime = 1425016501000L; + TimelineEntity entityApp1 = + TestFlowDataGenerator.getFlowApp1(appCreatedTime); + te.addEntity(entityApp1); + + HBaseTimelineWriterImpl hbi = null; + Configuration c1 = util.getConfiguration(); + try { + hbi = new HBaseTimelineWriterImpl(); + hbi.init(c1); + String appName = "application_1111999999_1234"; + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, appName), te, UserGroupInformation.createRemoteUser(user)); + hbi.flush(); + } finally { + if (hbi != null) { + hbi.close(); + } + } + // check flow activity + checkFlowActivityTable(cluster, user, flow, flowVersion, runid, c1, + appCreatedTime); + + // use the reader to verify the data + HBaseTimelineReaderImpl hbr = null; + try { + hbr = new HBaseTimelineReaderImpl(); + hbr.init(c1); + hbr.start(); + + Set entities = hbr.getEntities( + new TimelineReaderContext(cluster, user, flow, null, null, + TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), null), + new TimelineEntityFilters.Builder().entityLimit(10L).build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + for (TimelineEntity e : entities) { + FlowActivityEntity entity = (FlowActivityEntity)e; + NavigableSet flowRuns = entity.getFlowRuns(); + assertEquals(1, flowRuns.size()); + for (FlowRunEntity flowRun : flowRuns) { + assertEquals(runid, flowRun.getRunId()); + assertEquals(flowVersion, flowRun.getVersion()); + } + } + } finally { + if (hbr != null) { + hbr.close(); + } + } + } + + private void checkFlowActivityTable(String cluster, String user, String flow, + String flowVersion, long runid, Configuration c1, long appCreatedTime) + throws IOException { + Scan s = new Scan(); + s.addFamily(FlowActivityColumnFamily.INFO.getBytes()); + byte[] startRow = + new FlowActivityRowKey(cluster, appCreatedTime, user, flow).getRowKey(); + s.setStartRow(startRow); + String clusterStop = cluster + "1"; + byte[] stopRow = + new FlowActivityRowKey(clusterStop, appCreatedTime, user, flow) + .getRowKey(); + s.setStopRow(stopRow); + Connection conn = ConnectionFactory.createConnection(c1); + Table table1 = conn.getTable( + BaseTable.getTableName(c1, FlowActivityTable.TABLE_NAME_CONF_NAME, + FlowActivityTable.DEFAULT_TABLE_NAME)); + ResultScanner scanner = table1.getScanner(s); + int rowCount = 0; + for (Result result : scanner) { + assertNotNull(result); + assertTrue(!result.isEmpty()); + Map values = result + .getFamilyMap(FlowActivityColumnFamily.INFO.getBytes()); + rowCount++; + byte[] row = result.getRow(); + FlowActivityRowKey flowActivityRowKey = FlowActivityRowKey + .parseRowKey(row); + assertNotNull(flowActivityRowKey); + assertEquals(cluster, flowActivityRowKey.getClusterId()); + assertEquals(user, flowActivityRowKey.getUserId()); + assertEquals(flow, flowActivityRowKey.getFlowName()); + Long dayTs = HBaseTimelineStorageUtils.getTopOfTheDayTimestamp( + appCreatedTime); + assertEquals(dayTs, flowActivityRowKey.getDayTimestamp()); + assertEquals(1, values.size()); + checkFlowActivityRunId(runid, flowVersion, values); + } + assertEquals(1, rowCount); + } + + /** + * Writes 3 applications each with a different run id and version for the same + * {cluster, user, flow}. + * + * They should be getting inserted into one record in the flow activity table + * with 3 columns, one per run id. + */ + @Test + public void testFlowActivityTableOneFlowMultipleRunIds() throws IOException { + String cluster = "testManyRunsFlowActivity_cluster1"; + String user = "testManyRunsFlowActivity_c_user1"; + String flow = "flow_activity_test_flow_name"; + String flowVersion1 = "A122110F135BC4"; + long runid1 = 11111111111L; + + String flowVersion2 = "A12222222222C4"; + long runid2 = 2222222222222L; + + String flowVersion3 = "A1333333333C4"; + long runid3 = 3333333333333L; + + TimelineEntities te = new TimelineEntities(); + long appCreatedTime = 1425016501000L; + TimelineEntity entityApp1 = + TestFlowDataGenerator.getFlowApp1(appCreatedTime); + te.addEntity(entityApp1); + + HBaseTimelineWriterImpl hbi = null; + Configuration c1 = util.getConfiguration(); + try { + hbi = new HBaseTimelineWriterImpl(); + hbi.init(c1); + + UserGroupInformation remoteUser = + UserGroupInformation.createRemoteUser(user); + + String appName = "application_11888888888_1111"; + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion1, + runid1, appName), te, remoteUser); + + // write an application with to this flow but a different runid/ version + te = new TimelineEntities(); + te.addEntity(entityApp1); + appName = "application_11888888888_2222"; + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion2, + runid2, appName), te, remoteUser); + + // write an application with to this flow but a different runid/ version + te = new TimelineEntities(); + te.addEntity(entityApp1); + appName = "application_11888888888_3333"; + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion3, + runid3, appName), te, remoteUser); + + hbi.flush(); + } finally { + if (hbi != null) { + hbi.close(); + } + } + // check flow activity + checkFlowActivityTableSeveralRuns(cluster, user, flow, c1, flowVersion1, + runid1, flowVersion2, runid2, flowVersion3, runid3, appCreatedTime); + + // use the timeline reader to verify data + HBaseTimelineReaderImpl hbr = null; + try { + hbr = new HBaseTimelineReaderImpl(); + hbr.init(c1); + hbr.start(); + + Set entities = hbr.getEntities( + new TimelineReaderContext(cluster, null, null, null, null, + TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), null), + new TimelineEntityFilters.Builder().entityLimit(10L).build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + for (TimelineEntity e : entities) { + FlowActivityEntity flowActivity = (FlowActivityEntity)e; + assertEquals(cluster, flowActivity.getCluster()); + assertEquals(user, flowActivity.getUser()); + assertEquals(flow, flowActivity.getFlowName()); + long dayTs = + HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(appCreatedTime); + assertEquals(dayTs, flowActivity.getDate().getTime()); + Set flowRuns = flowActivity.getFlowRuns(); + assertEquals(3, flowRuns.size()); + for (FlowRunEntity flowRun : flowRuns) { + long runId = flowRun.getRunId(); + String version = flowRun.getVersion(); + if (runId == runid1) { + assertEquals(flowVersion1, version); + } else if (runId == runid2) { + assertEquals(flowVersion2, version); + } else if (runId == runid3) { + assertEquals(flowVersion3, version); + } else { + fail("unknown run id: " + runId); + } + } + } + } finally { + if (hbr != null) { + hbr.close(); + } + } + } + + private void checkFlowActivityTableSeveralRuns(String cluster, String user, + String flow, Configuration c1, String flowVersion1, long runid1, + String flowVersion2, long runid2, String flowVersion3, long runid3, + long appCreatedTime) + throws IOException { + Scan s = new Scan(); + s.addFamily(FlowActivityColumnFamily.INFO.getBytes()); + byte[] startRow = + new FlowActivityRowKey(cluster, appCreatedTime, user, flow).getRowKey(); + s.setStartRow(startRow); + String clusterStop = cluster + "1"; + byte[] stopRow = new FlowActivityRowKey(clusterStop, appCreatedTime, user, + flow).getRowKey(); + s.setStopRow(stopRow); + Connection conn = ConnectionFactory.createConnection(c1); + Table table1 = conn.getTable( + BaseTable.getTableName(c1, FlowActivityTable.TABLE_NAME_CONF_NAME, + FlowActivityTable.DEFAULT_TABLE_NAME)); + ResultScanner scanner = table1.getScanner(s); + int rowCount = 0; + for (Result result : scanner) { + assertNotNull(result); + assertTrue(!result.isEmpty()); + byte[] row = result.getRow(); + FlowActivityRowKey flowActivityRowKey = FlowActivityRowKey + .parseRowKey(row); + assertNotNull(flowActivityRowKey); + assertEquals(cluster, flowActivityRowKey.getClusterId()); + assertEquals(user, flowActivityRowKey.getUserId()); + assertEquals(flow, flowActivityRowKey.getFlowName()); + Long dayTs = HBaseTimelineStorageUtils.getTopOfTheDayTimestamp( + appCreatedTime); + assertEquals(dayTs, flowActivityRowKey.getDayTimestamp()); + + Map values = result + .getFamilyMap(FlowActivityColumnFamily.INFO.getBytes()); + rowCount++; + assertEquals(3, values.size()); + checkFlowActivityRunId(runid1, flowVersion1, values); + checkFlowActivityRunId(runid2, flowVersion2, values); + checkFlowActivityRunId(runid3, flowVersion3, values); + } + // the flow activity table is such that it will insert + // into current day's record + // hence, if this test runs across the midnight boundary, + // it may fail since it would insert into two records + // one for each day + assertEquals(1, rowCount); + } + + private void checkFlowActivityRunId(long runid, String flowVersion, + Map values) throws IOException { + byte[] rq = ColumnHelper.getColumnQualifier( + FlowActivityColumnPrefix.RUN_ID.getColumnPrefixBytes(), + GenericObjectMapper.write(runid)); + for (Map.Entry k : values.entrySet()) { + String actualQ = Bytes.toString(k.getKey()); + if (Bytes.toString(rq).equals(actualQ)) { + String actualV = (String) GenericObjectMapper.read(k.getValue()); + assertEquals(flowVersion, actualV); + } + } + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + util.shutdownMiniCluster(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java new file mode 100644 index 00000000000..1ad02e1886c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java @@ -0,0 +1,1083 @@ +/** + * 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.timelineservice.storage.flow; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.util.EnumSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList.Operator; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter; +import org.apache.hadoop.yarn.server.timelineservice.storage.DataGeneratorForTest; +import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineReaderImpl; +import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +/** + * Tests the FlowRun and FlowActivity Tables. + */ +public class TestHBaseStorageFlowRun { + + private static HBaseTestingUtility util; + + private static final String METRIC1 = "MAP_SLOT_MILLIS"; + private static final String METRIC2 = "HDFS_BYTES_READ"; + + @BeforeClass + public static void setupBeforeClass() throws Exception { + util = new HBaseTestingUtility(); + Configuration conf = util.getConfiguration(); + conf.setInt("hfile.format.version", 3); + util.startMiniCluster(); + DataGeneratorForTest.createSchema(util.getConfiguration()); + } + + @Test + public void checkCoProcessorOff() throws IOException, InterruptedException { + Configuration hbaseConf = util.getConfiguration(); + TableName table = BaseTable.getTableName(hbaseConf, + FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME); + Connection conn = null; + conn = ConnectionFactory.createConnection(hbaseConf); + Admin admin = conn.getAdmin(); + if (admin == null) { + throw new IOException("Can't check tables since admin is null"); + } + if (admin.tableExists(table)) { + // check the regions. + // check in flow run table + util.waitUntilAllRegionsAssigned(table); + checkCoprocessorExists(table, true); + } + + table = BaseTable.getTableName(hbaseConf, + FlowActivityTable.TABLE_NAME_CONF_NAME, + FlowActivityTable.DEFAULT_TABLE_NAME); + if (admin.tableExists(table)) { + // check the regions. + // check in flow activity table + util.waitUntilAllRegionsAssigned(table); + checkCoprocessorExists(table, false); + } + + table = BaseTable.getTableName(hbaseConf, EntityTable.TABLE_NAME_CONF_NAME, + EntityTable.DEFAULT_TABLE_NAME); + if (admin.tableExists(table)) { + // check the regions. + // check in entity run table + util.waitUntilAllRegionsAssigned(table); + checkCoprocessorExists(table, false); + } + } + + private void checkCoprocessorExists(TableName table, boolean exists) + throws IOException, InterruptedException { + HRegionServer server = util.getRSForFirstRegionInTable(table); + List regions = server.getOnlineRegions(table); + for (Region region : regions) { + boolean found = false; + Set coprocs = region.getCoprocessorHost().getCoprocessors(); + for (String coprocName : coprocs) { + if (coprocName.contains("FlowRunCoprocessor")) { + found = true; + } + } + assertEquals(found, exists); + } + } + + /** + * Writes 4 timeline entities belonging to one flow run through the + * {@link HBaseTimelineWriterImpl} + * + * Checks the flow run table contents + * + * The first entity has a created event, metrics and a finish event. + * + * The second entity has a created event and this is the entity with smallest + * start time. This should be the start time for the flow run. + * + * The third entity has a finish event and this is the entity with the max end + * time. This should be the end time for the flow run. + * + * The fourth entity has a created event which has a start time that is + * greater than min start time. + * + */ + @Test + public void testWriteFlowRunMinMax() throws Exception { + + TimelineEntities te = new TimelineEntities(); + te.addEntity(TestFlowDataGenerator.getEntity1()); + + HBaseTimelineWriterImpl hbi = null; + Configuration c1 = util.getConfiguration(); + String cluster = "testWriteFlowRunMinMaxToHBase_cluster1"; + String user = "testWriteFlowRunMinMaxToHBase_user1"; + String flow = "testing_flowRun_flow_name"; + String flowVersion = "CF7022C10F1354"; + long runid = 1002345678919L; + String appName = "application_100000000000_1111"; + long minStartTs = 1425026900000L; + long greaterStartTs = 30000000000000L; + long endTs = 1439750690000L; + TimelineEntity entityMinStartTime = TestFlowDataGenerator + .getEntityMinStartTime(minStartTs); + + try { + hbi = new HBaseTimelineWriterImpl(); + hbi.init(c1); + + UserGroupInformation remoteUser = + UserGroupInformation.createRemoteUser(user); + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, appName), te, remoteUser); + + // write another entity with the right min start time + te = new TimelineEntities(); + te.addEntity(entityMinStartTime); + appName = "application_100000000000_3333"; + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, appName), te, remoteUser); + + // writer another entity for max end time + TimelineEntity entityMaxEndTime = TestFlowDataGenerator + .getEntityMaxEndTime(endTs); + te = new TimelineEntities(); + te.addEntity(entityMaxEndTime); + appName = "application_100000000000_4444"; + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, appName), te, remoteUser); + + // writer another entity with greater start time + TimelineEntity entityGreaterStartTime = TestFlowDataGenerator + .getEntityGreaterStartTime(greaterStartTs); + te = new TimelineEntities(); + te.addEntity(entityGreaterStartTime); + appName = "application_1000000000000000_2222"; + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, appName), te, remoteUser); + + // flush everything to hbase + hbi.flush(); + } finally { + if (hbi != null) { + hbi.close(); + } + } + + Connection conn = ConnectionFactory.createConnection(c1); + // check in flow run table + Table table1 = conn.getTable(BaseTable.getTableName(c1, + FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME)); + // scan the table and see that we get back the right min and max + // timestamps + byte[] startRow = new FlowRunRowKey(cluster, user, flow, runid).getRowKey(); + Get g = new Get(startRow); + g.addColumn(FlowRunColumnFamily.INFO.getBytes(), + FlowRunColumn.MIN_START_TIME.getColumnQualifierBytes()); + g.addColumn(FlowRunColumnFamily.INFO.getBytes(), + FlowRunColumn.MAX_END_TIME.getColumnQualifierBytes()); + Result r1 = table1.get(g); + assertNotNull(r1); + assertTrue(!r1.isEmpty()); + Map values = r1.getFamilyMap(FlowRunColumnFamily.INFO + .getBytes()); + + assertEquals(2, r1.size()); + long starttime = Bytes.toLong(values.get( + FlowRunColumn.MIN_START_TIME.getColumnQualifierBytes())); + assertEquals(minStartTs, starttime); + assertEquals(endTs, Bytes.toLong(values + .get(FlowRunColumn.MAX_END_TIME.getColumnQualifierBytes()))); + + // use the timeline reader to verify data + HBaseTimelineReaderImpl hbr = null; + try { + hbr = new HBaseTimelineReaderImpl(); + hbr.init(c1); + hbr.start(); + // get the flow run entity + TimelineEntity entity = hbr.getEntity( + new TimelineReaderContext(cluster, user, flow, runid, null, + TimelineEntityType.YARN_FLOW_RUN.toString(), null), + new TimelineDataToRetrieve()); + assertTrue(TimelineEntityType.YARN_FLOW_RUN.matches(entity.getType())); + FlowRunEntity flowRun = (FlowRunEntity)entity; + assertEquals(minStartTs, flowRun.getStartTime()); + assertEquals(endTs, flowRun.getMaxEndTime()); + } finally { + if (hbr != null) { + hbr.close(); + } + } + } + + /** + * Writes two application entities of the same flow run. Each application has + * two metrics: slot millis and hdfs bytes read. Each metric has values at two + * timestamps. + * + * Checks the metric values of the flow in the flow run table. Flow metric + * values should be the sum of individual metric values that belong to the + * latest timestamp for that metric + */ + @Test + public void testWriteFlowRunMetricsOneFlow() throws Exception { + String cluster = "testWriteFlowRunMetricsOneFlow_cluster1"; + String user = "testWriteFlowRunMetricsOneFlow_user1"; + String flow = "testing_flowRun_metrics_flow_name"; + String flowVersion = "CF7022C10F1354"; + long runid = 1002345678919L; + + TimelineEntities te = new TimelineEntities(); + TimelineEntity entityApp1 = TestFlowDataGenerator + .getEntityMetricsApp1(System.currentTimeMillis()); + te.addEntity(entityApp1); + + HBaseTimelineWriterImpl hbi = null; + Configuration c1 = util.getConfiguration(); + try { + hbi = new HBaseTimelineWriterImpl(); + hbi.init(c1); + UserGroupInformation remoteUser = + UserGroupInformation.createRemoteUser(user); + String appName = "application_11111111111111_1111"; + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, appName), te, remoteUser); + // write another application with same metric to this flow + te = new TimelineEntities(); + TimelineEntity entityApp2 = TestFlowDataGenerator + .getEntityMetricsApp2(System.currentTimeMillis()); + te.addEntity(entityApp2); + appName = "application_11111111111111_2222"; + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, appName), te, remoteUser); + hbi.flush(); + } finally { + if (hbi != null) { + hbi.close(); + } + } + + // check flow run + checkFlowRunTable(cluster, user, flow, runid, c1); + + // check various batch limits in scanning the table for this flow + checkFlowRunTableBatchLimit(cluster, user, flow, runid, c1); + + // use the timeline reader to verify data + HBaseTimelineReaderImpl hbr = null; + try { + hbr = new HBaseTimelineReaderImpl(); + hbr.init(c1); + hbr.start(); + TimelineEntity entity = hbr.getEntity( + new TimelineReaderContext(cluster, user, flow, runid, null, + TimelineEntityType.YARN_FLOW_RUN.toString(), null), + new TimelineDataToRetrieve()); + assertTrue(TimelineEntityType.YARN_FLOW_RUN.matches(entity.getType())); + Set metrics = entity.getMetrics(); + assertEquals(2, metrics.size()); + for (TimelineMetric metric : metrics) { + String id = metric.getId(); + Map values = metric.getValues(); + assertEquals(1, values.size()); + Number value = null; + for (Number n : values.values()) { + value = n; + } + switch (id) { + case METRIC1: + assertEquals(141L, value); + break; + case METRIC2: + assertEquals(57L, value); + break; + default: + fail("unrecognized metric: " + id); + } + } + } finally { + if (hbr != null) { + hbr.close(); + } + } + } + + /* + * checks the batch limits on a scan + */ + void checkFlowRunTableBatchLimit(String cluster, String user, String flow, + long runid, Configuration c1) throws IOException { + + Scan s = new Scan(); + s.addFamily(FlowRunColumnFamily.INFO.getBytes()); + byte[] startRow = new FlowRunRowKey(cluster, user, flow, runid) + .getRowKey(); + s.setStartRow(startRow); + // set a batch limit + int batchLimit = 2; + s.setBatch(batchLimit); + String clusterStop = cluster + "1"; + byte[] stopRow = new FlowRunRowKey(clusterStop, user, flow, runid) + .getRowKey(); + s.setStopRow(stopRow); + Connection conn = ConnectionFactory.createConnection(c1); + Table table1 = conn.getTable(BaseTable.getTableName(c1, + FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME)); + ResultScanner scanner = table1.getScanner(s); + + int loopCount = 0; + for (Result result : scanner) { + assertNotNull(result); + assertTrue(!result.isEmpty()); + assertTrue(result.rawCells().length <= batchLimit); + Map values = result + .getFamilyMap(FlowRunColumnFamily.INFO.getBytes()); + assertNotNull(values); + assertTrue(values.size() <= batchLimit); + loopCount++; + } + assertTrue(loopCount > 0); + + // test with a diff batch limit + s = new Scan(); + s.addFamily(FlowRunColumnFamily.INFO.getBytes()); + s.setStartRow(startRow); + // set a batch limit + batchLimit = 1; + s.setBatch(batchLimit); + s.setMaxResultsPerColumnFamily(2); + s.setStopRow(stopRow); + scanner = table1.getScanner(s); + + loopCount = 0; + for (Result result : scanner) { + assertNotNull(result); + assertTrue(!result.isEmpty()); + assertEquals(batchLimit, result.rawCells().length); + Map values = result + .getFamilyMap(FlowRunColumnFamily.INFO.getBytes()); + assertNotNull(values); + assertEquals(batchLimit, values.size()); + loopCount++; + } + assertTrue(loopCount > 0); + + // test with a diff batch limit + // set it high enough + // we expect back 3 since there are + // column = m!HDFS_BYTES_READ value=57 + // column = m!MAP_SLOT_MILLIS value=141 + // column min_start_time value=1425016501000 + s = new Scan(); + s.addFamily(FlowRunColumnFamily.INFO.getBytes()); + s.setStartRow(startRow); + // set a batch limit + batchLimit = 100; + s.setBatch(batchLimit); + s.setStopRow(stopRow); + scanner = table1.getScanner(s); + + loopCount = 0; + for (Result result : scanner) { + assertNotNull(result); + assertTrue(!result.isEmpty()); + assertTrue(result.rawCells().length <= batchLimit); + Map values = result + .getFamilyMap(FlowRunColumnFamily.INFO.getBytes()); + assertNotNull(values); + // assert that with every next invocation + // we get back <= batchLimit values + assertTrue(values.size() <= batchLimit); + assertTrue(values.size() == 3); // see comment above + loopCount++; + } + // should loop through only once + assertTrue(loopCount == 1); + + // set it to a negative number + // we expect all 3 back since there are + // column = m!HDFS_BYTES_READ value=57 + // column = m!MAP_SLOT_MILLIS value=141 + // column min_start_time value=1425016501000 + s = new Scan(); + s.addFamily(FlowRunColumnFamily.INFO.getBytes()); + s.setStartRow(startRow); + // set a batch limit + batchLimit = -671; + s.setBatch(batchLimit); + s.setStopRow(stopRow); + scanner = table1.getScanner(s); + + loopCount = 0; + for (Result result : scanner) { + assertNotNull(result); + assertTrue(!result.isEmpty()); + assertEquals(3, result.rawCells().length); + Map values = result + .getFamilyMap(FlowRunColumnFamily.INFO.getBytes()); + assertNotNull(values); + // assert that with every next invocation + // we get back <= batchLimit values + assertEquals(3, values.size()); + loopCount++; + } + // should loop through only once + assertEquals(1, loopCount); + + // set it to 0 + // we expect all 3 back since there are + // column = m!HDFS_BYTES_READ value=57 + // column = m!MAP_SLOT_MILLIS value=141 + // column min_start_time value=1425016501000 + s = new Scan(); + s.addFamily(FlowRunColumnFamily.INFO.getBytes()); + s.setStartRow(startRow); + // set a batch limit + batchLimit = 0; + s.setBatch(batchLimit); + s.setStopRow(stopRow); + scanner = table1.getScanner(s); + + loopCount = 0; + for (Result result : scanner) { + assertNotNull(result); + assertTrue(!result.isEmpty()); + assertEquals(3, result.rawCells().length); + Map values = result + .getFamilyMap(FlowRunColumnFamily.INFO.getBytes()); + assertNotNull(values); + // assert that with every next invocation + // we get back <= batchLimit values + assertEquals(3, values.size()); + loopCount++; + } + // should loop through only once + assertEquals(1, loopCount); + } + + private void checkFlowRunTable(String cluster, String user, String flow, + long runid, Configuration c1) throws IOException { + Scan s = new Scan(); + s.addFamily(FlowRunColumnFamily.INFO.getBytes()); + byte[] startRow = new FlowRunRowKey(cluster, user, flow, runid).getRowKey(); + s.setStartRow(startRow); + String clusterStop = cluster + "1"; + byte[] stopRow = + new FlowRunRowKey(clusterStop, user, flow, runid).getRowKey(); + s.setStopRow(stopRow); + Connection conn = ConnectionFactory.createConnection(c1); + Table table1 = conn.getTable(BaseTable.getTableName(c1, + FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME)); + ResultScanner scanner = table1.getScanner(s); + + int rowCount = 0; + for (Result result : scanner) { + assertNotNull(result); + assertTrue(!result.isEmpty()); + Map values = result.getFamilyMap(FlowRunColumnFamily.INFO + .getBytes()); + rowCount++; + // check metric1 + byte[] q = ColumnHelper.getColumnQualifier( + FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(), METRIC1); + assertTrue(values.containsKey(q)); + assertEquals(141L, Bytes.toLong(values.get(q))); + + // check metric2 + assertEquals(3, values.size()); + q = ColumnHelper.getColumnQualifier( + FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(), METRIC2); + assertTrue(values.containsKey(q)); + assertEquals(57L, Bytes.toLong(values.get(q))); + } + assertEquals(1, rowCount); + } + + @Test + public void testWriteFlowRunMetricsPrefix() throws Exception { + String cluster = "testWriteFlowRunMetricsPrefix_cluster1"; + String user = "testWriteFlowRunMetricsPrefix_user1"; + String flow = "testWriteFlowRunMetricsPrefix_flow_name"; + String flowVersion = "CF7022C10F1354"; + + TimelineEntities te = new TimelineEntities(); + TimelineEntity entityApp1 = TestFlowDataGenerator + .getEntityMetricsApp1(System.currentTimeMillis()); + te.addEntity(entityApp1); + + HBaseTimelineWriterImpl hbi = null; + Configuration c1 = util.getConfiguration(); + try { + hbi = new HBaseTimelineWriterImpl(); + hbi.init(c1); + UserGroupInformation remoteUser = + UserGroupInformation.createRemoteUser(user); + + String appName = "application_11111111111111_1111"; + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + 1002345678919L, appName), te, + remoteUser); + // write another application with same metric to this flow + te = new TimelineEntities(); + TimelineEntity entityApp2 = TestFlowDataGenerator + .getEntityMetricsApp2(System.currentTimeMillis()); + te.addEntity(entityApp2); + appName = "application_11111111111111_2222"; + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + 1002345678918L, appName), te, + remoteUser); + hbi.flush(); + } finally { + if (hbi != null) { + hbi.close(); + } + } + + // use the timeline reader to verify data + HBaseTimelineReaderImpl hbr = null; + try { + hbr = new HBaseTimelineReaderImpl(); + hbr.init(c1); + hbr.start(); + TimelineFilterList metricsToRetrieve = new TimelineFilterList( + Operator.OR, new TimelinePrefixFilter(TimelineCompareOp.EQUAL, + METRIC1.substring(0, METRIC1.indexOf("_") + 1))); + TimelineEntity entity = hbr.getEntity( + new TimelineReaderContext(cluster, user, flow, 1002345678919L, null, + TimelineEntityType.YARN_FLOW_RUN.toString(), null), + new TimelineDataToRetrieve(null, metricsToRetrieve, null, null, null, + null)); + assertTrue(TimelineEntityType.YARN_FLOW_RUN.matches(entity.getType())); + Set metrics = entity.getMetrics(); + assertEquals(1, metrics.size()); + for (TimelineMetric metric : metrics) { + String id = metric.getId(); + Map values = metric.getValues(); + assertEquals(1, values.size()); + Number value = null; + for (Number n : values.values()) { + value = n; + } + switch (id) { + case METRIC1: + assertEquals(40L, value); + break; + default: + fail("unrecognized metric: " + id); + } + } + + Set entities = hbr.getEntities( + new TimelineReaderContext(cluster, user, flow, null, null, + TimelineEntityType.YARN_FLOW_RUN.toString(), null), + new TimelineEntityFilters.Builder().build(), + new TimelineDataToRetrieve(null, metricsToRetrieve, null, null, null, + null)); + assertEquals(2, entities.size()); + int metricCnt = 0; + for (TimelineEntity timelineEntity : entities) { + metricCnt += timelineEntity.getMetrics().size(); + } + assertEquals(2, metricCnt); + } finally { + if (hbr != null) { + hbr.close(); + } + } + } + + @Test + public void testWriteFlowRunsMetricFields() throws Exception { + String cluster = "testWriteFlowRunsMetricFields_cluster1"; + String user = "testWriteFlowRunsMetricFields_user1"; + String flow = "testWriteFlowRunsMetricFields_flow_name"; + String flowVersion = "CF7022C10F1354"; + long runid = 1002345678919L; + + TimelineEntities te = new TimelineEntities(); + TimelineEntity entityApp1 = TestFlowDataGenerator + .getEntityMetricsApp1(System.currentTimeMillis()); + te.addEntity(entityApp1); + + HBaseTimelineWriterImpl hbi = null; + Configuration c1 = util.getConfiguration(); + try { + hbi = new HBaseTimelineWriterImpl(); + hbi.init(c1); + UserGroupInformation remoteUser = + UserGroupInformation.createRemoteUser(user); + + String appName = "application_11111111111111_1111"; + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, appName), te, remoteUser); + // write another application with same metric to this flow + te = new TimelineEntities(); + TimelineEntity entityApp2 = TestFlowDataGenerator + .getEntityMetricsApp2(System.currentTimeMillis()); + te.addEntity(entityApp2); + appName = "application_11111111111111_2222"; + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, appName), te, remoteUser); + hbi.flush(); + } finally { + if (hbi != null) { + hbi.close(); + } + } + + // check flow run + checkFlowRunTable(cluster, user, flow, runid, c1); + + // use the timeline reader to verify data + HBaseTimelineReaderImpl hbr = null; + try { + hbr = new HBaseTimelineReaderImpl(); + hbr.init(c1); + hbr.start(); + Set entities = hbr.getEntities( + new TimelineReaderContext(cluster, user, flow, runid, null, + TimelineEntityType.YARN_FLOW_RUN.toString(), null), + new TimelineEntityFilters.Builder().build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + for (TimelineEntity timelineEntity : entities) { + assertEquals(0, timelineEntity.getMetrics().size()); + } + + entities = hbr.getEntities( + new TimelineReaderContext(cluster, user, flow, runid, null, + TimelineEntityType.YARN_FLOW_RUN.toString(), null), + new TimelineEntityFilters.Builder().build(), + new TimelineDataToRetrieve(null, null, + EnumSet.of(Field.METRICS), null, null, null)); + assertEquals(1, entities.size()); + for (TimelineEntity timelineEntity : entities) { + Set timelineMetrics = timelineEntity.getMetrics(); + assertEquals(2, timelineMetrics.size()); + for (TimelineMetric metric : timelineMetrics) { + String id = metric.getId(); + Map values = metric.getValues(); + assertEquals(1, values.size()); + Number value = null; + for (Number n : values.values()) { + value = n; + } + switch (id) { + case METRIC1: + assertEquals(141L, value); + break; + case METRIC2: + assertEquals(57L, value); + break; + default: + fail("unrecognized metric: " + id); + } + } + } + } finally { + if (hbr != null) { + hbr.close(); + } + } + } + + @Test + public void testWriteFlowRunFlush() throws Exception { + String cluster = "atestFlushFlowRun_cluster1"; + String user = "atestFlushFlowRun__user1"; + String flow = "atestFlushFlowRun_flow_name"; + String flowVersion = "AF1021C19F1351"; + long runid = 1449526652000L; + + int start = 10; + int count = 20000; + int appIdSuffix = 1; + HBaseTimelineWriterImpl hbi = null; + long insertTs = 1449796654827L - count; + long minTS = insertTs + 1; + long startTs = insertTs; + Configuration c1 = util.getConfiguration(); + TimelineEntities te1 = null; + TimelineEntity entityApp1 = null; + TimelineEntity entityApp2 = null; + try { + hbi = new HBaseTimelineWriterImpl(); + hbi.init(c1); + UserGroupInformation remoteUser = + UserGroupInformation.createRemoteUser(user); + + for (int i = start; i < count; i++) { + String appName = "application_1060350000000_" + appIdSuffix; + insertTs++; + te1 = new TimelineEntities(); + entityApp1 = TestFlowDataGenerator.getMinFlushEntity(insertTs); + te1.addEntity(entityApp1); + entityApp2 = TestFlowDataGenerator.getMaxFlushEntity(insertTs); + te1.addEntity(entityApp2); + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, appName), te1, remoteUser); + Thread.sleep(1); + + appName = "application_1001199480000_7" + appIdSuffix; + insertTs++; + appIdSuffix++; + te1 = new TimelineEntities(); + entityApp1 = TestFlowDataGenerator.getMinFlushEntity(insertTs); + te1.addEntity(entityApp1); + entityApp2 = TestFlowDataGenerator.getMaxFlushEntity(insertTs); + te1.addEntity(entityApp2); + + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, appName), te1, + remoteUser); + if (i % 1000 == 0) { + hbi.flush(); + checkMinMaxFlush(c1, minTS, startTs, count, cluster, user, flow, + runid, false); + } + } + } finally { + if (hbi != null) { + hbi.flush(); + hbi.close(); + } + checkMinMaxFlush(c1, minTS, startTs, count, cluster, user, flow, runid, + true); + } + } + + private void checkMinMaxFlush(Configuration c1, long minTS, long startTs, + int count, String cluster, String user, String flow, long runid, + boolean checkMax) throws IOException { + Connection conn = ConnectionFactory.createConnection(c1); + // check in flow run table + Table table1 = conn.getTable(BaseTable.getTableName(c1, + FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME)); + // scan the table and see that we get back the right min and max + // timestamps + byte[] startRow = new FlowRunRowKey(cluster, user, flow, runid).getRowKey(); + Get g = new Get(startRow); + g.addColumn(FlowRunColumnFamily.INFO.getBytes(), + FlowRunColumn.MIN_START_TIME.getColumnQualifierBytes()); + g.addColumn(FlowRunColumnFamily.INFO.getBytes(), + FlowRunColumn.MAX_END_TIME.getColumnQualifierBytes()); + + Result r1 = table1.get(g); + assertNotNull(r1); + assertTrue(!r1.isEmpty()); + Map values = r1.getFamilyMap(FlowRunColumnFamily.INFO + .getBytes()); + int start = 10; + assertEquals(2, r1.size()); + long starttime = Bytes.toLong(values + .get(FlowRunColumn.MIN_START_TIME.getColumnQualifierBytes())); + assertEquals(minTS, starttime); + if (checkMax) { + assertEquals(startTs + 2 * (count - start) + + TestFlowDataGenerator.END_TS_INCR, + Bytes.toLong(values + .get(FlowRunColumn.MAX_END_TIME.getColumnQualifierBytes()))); + } + } + + @Test + public void testFilterFlowRunsByCreatedTime() throws Exception { + String cluster = "cluster2"; + String user = "user2"; + String flow = "flow_name2"; + + TimelineEntities te = new TimelineEntities(); + TimelineEntity entityApp1 = TestFlowDataGenerator.getEntityMetricsApp1( + System.currentTimeMillis()); + entityApp1.setCreatedTime(1425016501000L); + te.addEntity(entityApp1); + + HBaseTimelineWriterImpl hbi = null; + Configuration c1 = util.getConfiguration(); + try { + hbi = new HBaseTimelineWriterImpl(); + hbi.init(c1); + UserGroupInformation remoteUser = + UserGroupInformation.createRemoteUser(user); + + hbi.write( + new TimelineCollectorContext(cluster, user, flow, "CF7022C10F1354", + 1002345678919L, "application_11111111111111_1111"), + te, remoteUser); + // write another application with same metric to this flow + te = new TimelineEntities(); + TimelineEntity entityApp2 = TestFlowDataGenerator.getEntityMetricsApp2( + System.currentTimeMillis()); + entityApp2.setCreatedTime(1425016502000L); + te.addEntity(entityApp2); + hbi.write( + new TimelineCollectorContext(cluster, user, flow, "CF7022C10F1354", + 1002345678918L, "application_11111111111111_2222"), + te, remoteUser); + hbi.flush(); + } finally { + if (hbi != null) { + hbi.close(); + } + } + + // use the timeline reader to verify data + HBaseTimelineReaderImpl hbr = null; + try { + hbr = new HBaseTimelineReaderImpl(); + hbr.init(c1); + hbr.start(); + + Set entities = hbr.getEntities( + new TimelineReaderContext(cluster, user, flow, + null, null, TimelineEntityType.YARN_FLOW_RUN.toString(), null), + new TimelineEntityFilters.Builder().createdTimeBegin(1425016501000L) + .createTimeEnd(1425016502001L).build(), + new TimelineDataToRetrieve()); + assertEquals(2, entities.size()); + for (TimelineEntity entity : entities) { + if (!entity.getId().equals("user2@flow_name2/1002345678918") && + !entity.getId().equals("user2@flow_name2/1002345678919")) { + fail("Entities with flow runs 1002345678918 and 1002345678919" + + "should be present."); + } + } + entities = hbr.getEntities( + new TimelineReaderContext(cluster, user, flow, null, null, + TimelineEntityType.YARN_FLOW_RUN.toString(), null), + new TimelineEntityFilters.Builder().createdTimeBegin(1425016501050L) + .build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + for (TimelineEntity entity : entities) { + if (!entity.getId().equals("user2@flow_name2/1002345678918")) { + fail("Entity with flow run 1002345678918 should be present."); + } + } + entities = hbr.getEntities( + new TimelineReaderContext(cluster, user, flow, null, null, + TimelineEntityType.YARN_FLOW_RUN.toString(), null), + new TimelineEntityFilters.Builder().createTimeEnd(1425016501050L) + .build(), + new TimelineDataToRetrieve()); + assertEquals(1, entities.size()); + for (TimelineEntity entity : entities) { + if (!entity.getId().equals("user2@flow_name2/1002345678919")) { + fail("Entity with flow run 1002345678919 should be present."); + } + } + } finally { + if (hbr != null) { + hbr.close(); + } + } + } + + @Test + public void testMetricFilters() throws Exception { + String cluster = "cluster1"; + String user = "user1"; + String flow = "flow_name1"; + + TimelineEntities te = new TimelineEntities(); + TimelineEntity entityApp1 = TestFlowDataGenerator.getEntityMetricsApp1( + System.currentTimeMillis()); + te.addEntity(entityApp1); + + HBaseTimelineWriterImpl hbi = null; + Configuration c1 = util.getConfiguration(); + try { + hbi = new HBaseTimelineWriterImpl(); + hbi.init(c1); + UserGroupInformation remoteUser = + UserGroupInformation.createRemoteUser(user); + + hbi.write( + new TimelineCollectorContext(cluster, user, flow, "CF7022C10F1354", + 1002345678919L, "application_11111111111111_1111"), + te, remoteUser); + // write another application with same metric to this flow + te = new TimelineEntities(); + TimelineEntity entityApp2 = TestFlowDataGenerator.getEntityMetricsApp2( + System.currentTimeMillis()); + te.addEntity(entityApp2); + hbi.write( + new TimelineCollectorContext(cluster, user, flow, "CF7022C10F1354", + 1002345678918L, "application_11111111111111_2222"), + te, remoteUser); + hbi.flush(); + } finally { + if (hbi != null) { + hbi.close(); + } + } + + // use the timeline reader to verify data + HBaseTimelineReaderImpl hbr = null; + try { + hbr = new HBaseTimelineReaderImpl(); + hbr.init(c1); + hbr.start(); + + TimelineFilterList list1 = new TimelineFilterList(); + list1.addFilter(new TimelineCompareFilter( + TimelineCompareOp.GREATER_OR_EQUAL, METRIC1, 101)); + TimelineFilterList list2 = new TimelineFilterList(); + list2.addFilter(new TimelineCompareFilter( + TimelineCompareOp.LESS_THAN, METRIC1, 43)); + list2.addFilter(new TimelineCompareFilter( + TimelineCompareOp.EQUAL, METRIC2, 57)); + TimelineFilterList metricFilterList = + new TimelineFilterList(Operator.OR, list1, list2); + Set entities = hbr.getEntities( + new TimelineReaderContext(cluster, user, flow, null, + null, TimelineEntityType.YARN_FLOW_RUN.toString(), null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList) + .build(), + new TimelineDataToRetrieve(null, null, + EnumSet.of(Field.METRICS), null, null, null)); + assertEquals(2, entities.size()); + int metricCnt = 0; + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + } + assertEquals(3, metricCnt); + + TimelineFilterList metricFilterList1 = new TimelineFilterList( + new TimelineCompareFilter( + TimelineCompareOp.LESS_OR_EQUAL, METRIC1, 127), + new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL, METRIC2, 30)); + entities = hbr.getEntities( + new TimelineReaderContext(cluster, user, flow, null, null, + TimelineEntityType.YARN_FLOW_RUN.toString(), null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList1) + .build(), + new TimelineDataToRetrieve(null, null, + EnumSet.of(Field.METRICS), null, null, null)); + assertEquals(1, entities.size()); + metricCnt = 0; + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + } + assertEquals(2, metricCnt); + + TimelineFilterList metricFilterList2 = new TimelineFilterList( + new TimelineCompareFilter(TimelineCompareOp.LESS_THAN, METRIC1, 32), + new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL, METRIC2, 57)); + entities = hbr.getEntities( + new TimelineReaderContext(cluster, user, flow, null, null, + TimelineEntityType.YARN_FLOW_RUN.toString(), null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList2) + .build(), + new TimelineDataToRetrieve(null, null, + EnumSet.of(Field.METRICS), null, null, null)); + assertEquals(0, entities.size()); + + TimelineFilterList metricFilterList3 = new TimelineFilterList( + new TimelineCompareFilter(TimelineCompareOp.EQUAL, "s_metric", 32)); + entities = hbr.getEntities( + new TimelineReaderContext(cluster, user, flow, null, null, + TimelineEntityType.YARN_FLOW_RUN.toString(), null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList3) + .build(), + new TimelineDataToRetrieve(null, null, + EnumSet.of(Field.METRICS), null, null, null)); + assertEquals(0, entities.size()); + + TimelineFilterList list3 = new TimelineFilterList(); + list3.addFilter(new TimelineCompareFilter( + TimelineCompareOp.GREATER_OR_EQUAL, METRIC1, 101)); + TimelineFilterList list4 = new TimelineFilterList(); + list4.addFilter(new TimelineCompareFilter( + TimelineCompareOp.LESS_THAN, METRIC1, 43)); + list4.addFilter(new TimelineCompareFilter( + TimelineCompareOp.EQUAL, METRIC2, 57)); + TimelineFilterList metricFilterList4 = + new TimelineFilterList(Operator.OR, list3, list4); + TimelineFilterList metricsToRetrieve = new TimelineFilterList(Operator.OR, + new TimelinePrefixFilter(TimelineCompareOp.EQUAL, + METRIC2.substring(0, METRIC2.indexOf("_") + 1))); + entities = hbr.getEntities( + new TimelineReaderContext(cluster, user, flow, null, null, + TimelineEntityType.YARN_FLOW_RUN.toString(), null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList4) + .build(), + new TimelineDataToRetrieve(null, metricsToRetrieve, + EnumSet.of(Field.ALL), null, null, null)); + assertEquals(2, entities.size()); + metricCnt = 0; + for (TimelineEntity entity : entities) { + metricCnt += entity.getMetrics().size(); + } + assertEquals(1, metricCnt); + } finally { + if (hbr != null) { + hbr.close(); + } + } + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + util.shutdownMiniCluster(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java new file mode 100644 index 00000000000..0ef8260dee4 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java @@ -0,0 +1,834 @@ +/** + * 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.timelineservice.storage.flow; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.SortedSet; +import java.util.TreeSet; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.Tag; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext; +import org.apache.hadoop.yarn.server.timelineservice.storage.DataGeneratorForTest; +import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimestampGenerator; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +/** + * Tests the FlowRun and FlowActivity Tables. + */ +public class TestHBaseStorageFlowRunCompaction { + + private static HBaseTestingUtility util; + + private static final String METRIC1 = "MAP_SLOT_MILLIS"; + private static final String METRIC2 = "HDFS_BYTES_READ"; + + private final byte[] aRowKey = Bytes.toBytes("a"); + private final byte[] aFamily = Bytes.toBytes("family"); + private final byte[] aQualifier = Bytes.toBytes("qualifier"); + + @BeforeClass + public static void setupBeforeClass() throws Exception { + util = new HBaseTestingUtility(); + Configuration conf = util.getConfiguration(); + conf.setInt("hfile.format.version", 3); + util.startMiniCluster(); + DataGeneratorForTest.createSchema(util.getConfiguration()); + } + + /** + * writes non numeric data into flow run table. + * reads it back + * + * @throws Exception + */ + @Test + public void testWriteNonNumericData() throws Exception { + String rowKey = "nonNumericRowKey"; + String column = "nonNumericColumnName"; + String value = "nonNumericValue"; + byte[] rowKeyBytes = Bytes.toBytes(rowKey); + byte[] columnNameBytes = Bytes.toBytes(column); + byte[] valueBytes = Bytes.toBytes(value); + Put p = new Put(rowKeyBytes); + p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnNameBytes, + valueBytes); + Configuration hbaseConf = util.getConfiguration(); + Connection conn = null; + conn = ConnectionFactory.createConnection(hbaseConf); + Table flowRunTable = conn.getTable(BaseTable.getTableName(hbaseConf, + FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME)); + flowRunTable.put(p); + + Get g = new Get(rowKeyBytes); + Result r = flowRunTable.get(g); + assertNotNull(r); + assertTrue(r.size() >= 1); + Cell actualValue = r.getColumnLatestCell( + FlowRunColumnFamily.INFO.getBytes(), columnNameBytes); + assertNotNull(CellUtil.cloneValue(actualValue)); + assertEquals(Bytes.toString(CellUtil.cloneValue(actualValue)), value); + } + + @Test + public void testWriteScanBatchLimit() throws Exception { + String rowKey = "nonNumericRowKey"; + String column = "nonNumericColumnName"; + String value = "nonNumericValue"; + String column2 = "nonNumericColumnName2"; + String value2 = "nonNumericValue2"; + String column3 = "nonNumericColumnName3"; + String value3 = "nonNumericValue3"; + String column4 = "nonNumericColumnName4"; + String value4 = "nonNumericValue4"; + + byte[] rowKeyBytes = Bytes.toBytes(rowKey); + byte[] columnNameBytes = Bytes.toBytes(column); + byte[] valueBytes = Bytes.toBytes(value); + byte[] columnName2Bytes = Bytes.toBytes(column2); + byte[] value2Bytes = Bytes.toBytes(value2); + byte[] columnName3Bytes = Bytes.toBytes(column3); + byte[] value3Bytes = Bytes.toBytes(value3); + byte[] columnName4Bytes = Bytes.toBytes(column4); + byte[] value4Bytes = Bytes.toBytes(value4); + + Put p = new Put(rowKeyBytes); + p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnNameBytes, + valueBytes); + p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnName2Bytes, + value2Bytes); + p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnName3Bytes, + value3Bytes); + p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnName4Bytes, + value4Bytes); + + Configuration hbaseConf = util.getConfiguration(); + Connection conn = null; + conn = ConnectionFactory.createConnection(hbaseConf); + Table flowRunTable = conn.getTable(BaseTable.getTableName(hbaseConf, + FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME)); + flowRunTable.put(p); + + String rowKey2 = "nonNumericRowKey2"; + byte[] rowKey2Bytes = Bytes.toBytes(rowKey2); + p = new Put(rowKey2Bytes); + p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnNameBytes, + valueBytes); + p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnName2Bytes, + value2Bytes); + p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnName3Bytes, + value3Bytes); + p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnName4Bytes, + value4Bytes); + flowRunTable.put(p); + + String rowKey3 = "nonNumericRowKey3"; + byte[] rowKey3Bytes = Bytes.toBytes(rowKey3); + p = new Put(rowKey3Bytes); + p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnNameBytes, + valueBytes); + p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnName2Bytes, + value2Bytes); + p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnName3Bytes, + value3Bytes); + p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnName4Bytes, + value4Bytes); + flowRunTable.put(p); + + Scan s = new Scan(); + s.addFamily(FlowRunColumnFamily.INFO.getBytes()); + s.setStartRow(rowKeyBytes); + // set number of cells to fetch per scanner next invocation + int batchLimit = 2; + s.setBatch(batchLimit); + ResultScanner scanner = flowRunTable.getScanner(s); + for (Result result : scanner) { + assertNotNull(result); + assertTrue(!result.isEmpty()); + assertTrue(result.rawCells().length <= batchLimit); + Map values = result + .getFamilyMap(FlowRunColumnFamily.INFO.getBytes()); + assertTrue(values.size() <= batchLimit); + } + + s = new Scan(); + s.addFamily(FlowRunColumnFamily.INFO.getBytes()); + s.setStartRow(rowKeyBytes); + // set number of cells to fetch per scanner next invocation + batchLimit = 3; + s.setBatch(batchLimit); + scanner = flowRunTable.getScanner(s); + for (Result result : scanner) { + assertNotNull(result); + assertTrue(!result.isEmpty()); + assertTrue(result.rawCells().length <= batchLimit); + Map values = result + .getFamilyMap(FlowRunColumnFamily.INFO.getBytes()); + assertTrue(values.size() <= batchLimit); + } + + s = new Scan(); + s.addFamily(FlowRunColumnFamily.INFO.getBytes()); + s.setStartRow(rowKeyBytes); + // set number of cells to fetch per scanner next invocation + batchLimit = 1000; + s.setBatch(batchLimit); + scanner = flowRunTable.getScanner(s); + int rowCount = 0; + for (Result result : scanner) { + assertNotNull(result); + assertTrue(!result.isEmpty()); + assertTrue(result.rawCells().length <= batchLimit); + Map values = result + .getFamilyMap(FlowRunColumnFamily.INFO.getBytes()); + assertTrue(values.size() <= batchLimit); + // we expect all back in one next call + assertEquals(4, values.size()); + rowCount++; + } + // should get back 1 row with each invocation + // if scan batch is set sufficiently high + assertEquals(3, rowCount); + + // test with a negative number + // should have same effect as setting it to a high number + s = new Scan(); + s.addFamily(FlowRunColumnFamily.INFO.getBytes()); + s.setStartRow(rowKeyBytes); + // set number of cells to fetch per scanner next invocation + batchLimit = -2992; + s.setBatch(batchLimit); + scanner = flowRunTable.getScanner(s); + rowCount = 0; + for (Result result : scanner) { + assertNotNull(result); + assertTrue(!result.isEmpty()); + assertEquals(4, result.rawCells().length); + Map values = result + .getFamilyMap(FlowRunColumnFamily.INFO.getBytes()); + // we expect all back in one next call + assertEquals(4, values.size()); + rowCount++; + } + // should get back 1 row with each invocation + // if scan batch is set sufficiently high + assertEquals(3, rowCount); + } + + @Test + public void testWriteFlowRunCompaction() throws Exception { + String cluster = "kompaction_cluster1"; + String user = "kompaction_FlowRun__user1"; + String flow = "kompaction_flowRun_flow_name"; + String flowVersion = "AF1021C19F1351"; + long runid = 1449526652000L; + + int start = 10; + int count = 2000; + int appIdSuffix = 1; + HBaseTimelineWriterImpl hbi = null; + long insertTs = System.currentTimeMillis() - count; + Configuration c1 = util.getConfiguration(); + TimelineEntities te1 = null; + TimelineEntity entityApp1 = null; + UserGroupInformation remoteUser = + UserGroupInformation.createRemoteUser(user); + try { + hbi = new HBaseTimelineWriterImpl(); + hbi.init(c1); + + // now insert count * ( 100 + 100) metrics + // each call to getEntityMetricsApp1 brings back 100 values + // of metric1 and 100 of metric2 + for (int i = start; i < start + count; i++) { + String appName = "application_10240000000000_" + appIdSuffix; + insertTs++; + te1 = new TimelineEntities(); + entityApp1 = TestFlowDataGenerator.getEntityMetricsApp1(insertTs, c1); + te1.addEntity(entityApp1); + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, appName), te1, remoteUser); + + appName = "application_2048000000000_7" + appIdSuffix; + insertTs++; + te1 = new TimelineEntities(); + entityApp1 = TestFlowDataGenerator.getEntityMetricsApp2(insertTs); + te1.addEntity(entityApp1); + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, appName), te1, remoteUser); + } + } finally { + String appName = "application_10240000000000_" + appIdSuffix; + te1 = new TimelineEntities(); + entityApp1 = TestFlowDataGenerator.getEntityMetricsApp1Complete( + insertTs + 1, c1); + te1.addEntity(entityApp1); + if (hbi != null) { + hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion, + runid, appName), te1, remoteUser); + hbi.flush(); + hbi.close(); + } + } + + // check in flow run table + HRegionServer server = util.getRSForFirstRegionInTable( + BaseTable.getTableName(c1, FlowRunTable.TABLE_NAME_CONF_NAME, + FlowRunTable.DEFAULT_TABLE_NAME)); + List regions = server.getOnlineRegions(BaseTable.getTableName(c1, + FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME)); + assertTrue("Didn't find any regions for primary table!", + regions.size() > 0); + // flush and compact all the regions of the primary table + for (Region region : regions) { + region.flush(true); + region.compact(true); + } + + // check flow run for one flow many apps + checkFlowRunTable(cluster, user, flow, runid, c1, 4); + } + + + private void checkFlowRunTable(String cluster, String user, String flow, + long runid, Configuration c1, int valueCount) throws IOException { + Scan s = new Scan(); + s.addFamily(FlowRunColumnFamily.INFO.getBytes()); + byte[] startRow = new FlowRunRowKey(cluster, user, flow, runid).getRowKey(); + s.setStartRow(startRow); + String clusterStop = cluster + "1"; + byte[] stopRow = + new FlowRunRowKey(clusterStop, user, flow, runid).getRowKey(); + s.setStopRow(stopRow); + Connection conn = ConnectionFactory.createConnection(c1); + Table table1 = conn.getTable(BaseTable.getTableName(c1, + FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME)); + ResultScanner scanner = table1.getScanner(s); + + int rowCount = 0; + for (Result result : scanner) { + assertNotNull(result); + assertTrue(!result.isEmpty()); + Map values = result.getFamilyMap(FlowRunColumnFamily.INFO + .getBytes()); + assertEquals(valueCount, values.size()); + + rowCount++; + // check metric1 + byte[] q = ColumnHelper.getColumnQualifier( + FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(), METRIC1); + assertTrue(values.containsKey(q)); + assertEquals(141, Bytes.toLong(values.get(q))); + + // check metric2 + q = ColumnHelper.getColumnQualifier( + FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(), METRIC2); + assertTrue(values.containsKey(q)); + assertEquals(57, Bytes.toLong(values.get(q))); + } + assertEquals(1, rowCount); + } + + + private FlowScanner getFlowScannerForTestingCompaction() { + // create a FlowScanner object with the sole purpose of invoking a process + // summation; + CompactionRequest request = new CompactionRequest(); + request.setIsMajor(true, true); + // okay to pass in nulls for the constructor arguments + // because all we want to do is invoke the process summation + FlowScanner fs = new FlowScanner(null, null, + (request.isMajor() ? FlowScannerOperation.MAJOR_COMPACTION + : FlowScannerOperation.MINOR_COMPACTION)); + assertNotNull(fs); + return fs; + } + + @Test + public void checkProcessSummationMoreCellsSumFinal2() + throws IOException { + long cellValue1 = 1236L; + long cellValue2 = 28L; + long cellValue3 = 1236L; + long cellValue4 = 1236L; + FlowScanner fs = getFlowScannerForTestingCompaction(); + + // note down the current timestamp + long currentTimestamp = System.currentTimeMillis(); + long cell1Ts = 1200120L; + long cell2Ts = TimestampGenerator.getSupplementedTimestamp( + System.currentTimeMillis(), "application_123746661110_11202"); + long cell3Ts = 1277719L; + long cell4Ts = currentTimestamp - 10; + + SortedSet currentColumnCells = new TreeSet(KeyValue.COMPARATOR); + + List tags = new ArrayList<>(); + Tag t = new Tag(AggregationOperation.SUM_FINAL.getTagType(), + "application_1234588888_91188"); + tags.add(t); + byte[] tagByteArray = Tag.fromList(tags); + // create a cell with a VERY old timestamp and attribute SUM_FINAL + Cell c1 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily, + aQualifier, cell1Ts, Bytes.toBytes(cellValue1), tagByteArray); + currentColumnCells.add(c1); + + tags = new ArrayList<>(); + t = new Tag(AggregationOperation.SUM_FINAL.getTagType(), + "application_12700000001_29102"); + tags.add(t); + tagByteArray = Tag.fromList(tags); + // create a cell with a recent timestamp and attribute SUM_FINAL + Cell c2 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily, + aQualifier, cell2Ts, Bytes.toBytes(cellValue2), tagByteArray); + currentColumnCells.add(c2); + + tags = new ArrayList<>(); + t = new Tag(AggregationOperation.SUM.getTagType(), + "application_191780000000001_8195"); + tags.add(t); + tagByteArray = Tag.fromList(tags); + // create a cell with a VERY old timestamp but has attribute SUM + Cell c3 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily, + aQualifier, cell3Ts, Bytes.toBytes(cellValue3), tagByteArray); + currentColumnCells.add(c3); + + tags = new ArrayList<>(); + t = new Tag(AggregationOperation.SUM.getTagType(), + "application_191780000000001_98104"); + tags.add(t); + tagByteArray = Tag.fromList(tags); + // create a cell with a VERY old timestamp but has attribute SUM + Cell c4 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily, + aQualifier, cell4Ts, Bytes.toBytes(cellValue4), tagByteArray); + currentColumnCells.add(c4); + + List cells = + fs.processSummationMajorCompaction(currentColumnCells, + new LongConverter(), currentTimestamp); + assertNotNull(cells); + + // we should be getting back 4 cells + // one is the flow sum cell + // two are the cells with SUM attribute + // one cell with SUM_FINAL + assertEquals(4, cells.size()); + + for (int i = 0; i < cells.size(); i++) { + Cell returnedCell = cells.get(0); + assertNotNull(returnedCell); + + long returnTs = returnedCell.getTimestamp(); + long returnValue = Bytes.toLong(CellUtil + .cloneValue(returnedCell)); + if (returnValue == cellValue2) { + assertTrue(returnTs == cell2Ts); + } else if (returnValue == cellValue3) { + assertTrue(returnTs == cell3Ts); + } else if (returnValue == cellValue4) { + assertTrue(returnTs == cell4Ts); + } else if (returnValue == cellValue1) { + assertTrue(returnTs != cell1Ts); + assertTrue(returnTs > cell1Ts); + assertTrue(returnTs >= currentTimestamp); + } else { + // raise a failure since we expect only these two values back + Assert.fail(); + } + } + } + + // tests with many cells + // of type SUM and SUM_FINAL + // all cells of SUM_FINAL will expire + @Test + public void checkProcessSummationMoreCellsSumFinalMany() throws IOException { + FlowScanner fs = getFlowScannerForTestingCompaction(); + int count = 200000; + + long cellValueFinal = 1000L; + long cellValueNotFinal = 28L; + + // note down the current timestamp + long currentTimestamp = System.currentTimeMillis(); + long cellTsFinalStart = 10001120L; + long cellTsFinal = cellTsFinalStart; + long cellTsNotFinalStart = currentTimestamp - 5; + long cellTsNotFinal = cellTsNotFinalStart; + + SortedSet currentColumnCells = new TreeSet(KeyValue.COMPARATOR); + List tags = null; + Tag t = null; + Cell c1 = null; + + // insert SUM_FINAL cells + for (int i = 0; i < count; i++) { + tags = new ArrayList<>(); + t = new Tag(AggregationOperation.SUM_FINAL.getTagType(), + "application_123450000" + i + "01_19" + i); + tags.add(t); + byte[] tagByteArray = Tag.fromList(tags); + // create a cell with a VERY old timestamp and attribute SUM_FINAL + c1 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier, + cellTsFinal, Bytes.toBytes(cellValueFinal), tagByteArray); + currentColumnCells.add(c1); + cellTsFinal++; + } + + // add SUM cells + for (int i = 0; i < count; i++) { + tags = new ArrayList<>(); + t = new Tag(AggregationOperation.SUM.getTagType(), + "application_1987650000" + i + "83_911" + i); + tags.add(t); + byte[] tagByteArray = Tag.fromList(tags); + // create a cell with attribute SUM + c1 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier, + cellTsNotFinal, Bytes.toBytes(cellValueNotFinal), tagByteArray); + currentColumnCells.add(c1); + cellTsNotFinal++; + } + + List cells = + fs.processSummationMajorCompaction(currentColumnCells, + new LongConverter(), currentTimestamp); + assertNotNull(cells); + + // we should be getting back count + 1 cells + // one is the flow sum cell + // others are the cells with SUM attribute + assertEquals(count + 1, cells.size()); + + for (int i = 0; i < cells.size(); i++) { + Cell returnedCell = cells.get(0); + assertNotNull(returnedCell); + + long returnTs = returnedCell.getTimestamp(); + long returnValue = Bytes.toLong(CellUtil + .cloneValue(returnedCell)); + if (returnValue == (count * cellValueFinal)) { + assertTrue(returnTs > (cellTsFinalStart + count)); + assertTrue(returnTs >= currentTimestamp); + } else if ((returnValue >= cellValueNotFinal) + && (returnValue <= cellValueNotFinal * count)) { + assertTrue(returnTs >= cellTsNotFinalStart); + assertTrue(returnTs <= cellTsNotFinalStart * count); + } else { + // raise a failure since we expect only these values back + Assert.fail(); + } + } + } + + // tests with many cells + // of type SUM and SUM_FINAL + // NOT cells of SUM_FINAL will expire + @Test + public void checkProcessSummationMoreCellsSumFinalVariedTags() + throws IOException { + FlowScanner fs = getFlowScannerForTestingCompaction(); + int countFinal = 20100; + int countNotFinal = 1000; + int countFinalNotExpire = 7009; + + long cellValueFinal = 1000L; + long cellValueNotFinal = 28L; + + // note down the current timestamp + long currentTimestamp = System.currentTimeMillis(); + long cellTsFinalStart = 10001120L; + long cellTsFinal = cellTsFinalStart; + + long cellTsFinalStartNotExpire = TimestampGenerator + .getSupplementedTimestamp(System.currentTimeMillis(), + "application_10266666661166_118821"); + long cellTsFinalNotExpire = cellTsFinalStartNotExpire; + + long cellTsNotFinalStart = currentTimestamp - 5; + long cellTsNotFinal = cellTsNotFinalStart; + + SortedSet currentColumnCells = new TreeSet(KeyValue.COMPARATOR); + List tags = null; + Tag t = null; + Cell c1 = null; + + // insert SUM_FINAL cells which will expire + for (int i = 0; i < countFinal; i++) { + tags = new ArrayList<>(); + t = new Tag(AggregationOperation.SUM_FINAL.getTagType(), + "application_123450000" + i + "01_19" + i); + tags.add(t); + byte[] tagByteArray = Tag.fromList(tags); + // create a cell with a VERY old timestamp and attribute SUM_FINAL + c1 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier, + cellTsFinal, Bytes.toBytes(cellValueFinal), tagByteArray); + currentColumnCells.add(c1); + cellTsFinal++; + } + + // insert SUM_FINAL cells which will NOT expire + for (int i = 0; i < countFinalNotExpire; i++) { + tags = new ArrayList<>(); + t = new Tag(AggregationOperation.SUM_FINAL.getTagType(), + "application_123450000" + i + "01_19" + i); + tags.add(t); + byte[] tagByteArray = Tag.fromList(tags); + // create a cell with a VERY old timestamp and attribute SUM_FINAL + c1 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier, + cellTsFinalNotExpire, Bytes.toBytes(cellValueFinal), tagByteArray); + currentColumnCells.add(c1); + cellTsFinalNotExpire++; + } + + // add SUM cells + for (int i = 0; i < countNotFinal; i++) { + tags = new ArrayList<>(); + t = new Tag(AggregationOperation.SUM.getTagType(), + "application_1987650000" + i + "83_911" + i); + tags.add(t); + byte[] tagByteArray = Tag.fromList(tags); + // create a cell with attribute SUM + c1 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier, + cellTsNotFinal, Bytes.toBytes(cellValueNotFinal), tagByteArray); + currentColumnCells.add(c1); + cellTsNotFinal++; + } + + List cells = + fs.processSummationMajorCompaction(currentColumnCells, + new LongConverter(), currentTimestamp); + assertNotNull(cells); + + // we should be getting back + // countNotFinal + countFinalNotExpire + 1 cells + // one is the flow sum cell + // count = the cells with SUM attribute + // count = the cells with SUM_FINAL attribute but not expired + assertEquals(countFinalNotExpire + countNotFinal + 1, cells.size()); + + for (int i = 0; i < cells.size(); i++) { + Cell returnedCell = cells.get(0); + assertNotNull(returnedCell); + + long returnTs = returnedCell.getTimestamp(); + long returnValue = Bytes.toLong(CellUtil + .cloneValue(returnedCell)); + if (returnValue == (countFinal * cellValueFinal)) { + assertTrue(returnTs > (cellTsFinalStart + countFinal)); + assertTrue(returnTs >= currentTimestamp); + } else if (returnValue == cellValueNotFinal) { + assertTrue(returnTs >= cellTsNotFinalStart); + assertTrue(returnTs <= cellTsNotFinalStart + countNotFinal); + } else if (returnValue == cellValueFinal){ + assertTrue(returnTs >= cellTsFinalStartNotExpire); + assertTrue(returnTs <= cellTsFinalStartNotExpire + countFinalNotExpire); + } else { + // raise a failure since we expect only these values back + Assert.fail(); + } + } + } + + @Test + public void testProcessSummationMoreCellsSumFinal() throws IOException { + FlowScanner fs = getFlowScannerForTestingCompaction(); + // note down the current timestamp + long currentTimestamp = System.currentTimeMillis(); + long cellValue1 = 1236L; + long cellValue2 = 28L; + + List tags = new ArrayList<>(); + Tag t = new Tag(AggregationOperation.SUM_FINAL.getTagType(), + "application_1234588888_999888"); + tags.add(t); + byte[] tagByteArray = Tag.fromList(tags); + SortedSet currentColumnCells = new TreeSet(KeyValue.COMPARATOR); + + // create a cell with a VERY old timestamp and attribute SUM_FINAL + Cell c1 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily, + aQualifier, 120L, Bytes.toBytes(cellValue1), tagByteArray); + currentColumnCells.add(c1); + + tags = new ArrayList<>(); + t = new Tag(AggregationOperation.SUM.getTagType(), + "application_100000000001_119101"); + tags.add(t); + tagByteArray = Tag.fromList(tags); + + // create a cell with a VERY old timestamp but has attribute SUM + Cell c2 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily, + aQualifier, 130L, Bytes.toBytes(cellValue2), tagByteArray); + currentColumnCells.add(c2); + List cells = fs.processSummationMajorCompaction(currentColumnCells, + new LongConverter(), currentTimestamp); + assertNotNull(cells); + + // we should be getting back two cells + // one is the flow sum cell + // another is the cell with SUM attribute + assertEquals(2, cells.size()); + + Cell returnedCell = cells.get(0); + assertNotNull(returnedCell); + long inputTs1 = c1.getTimestamp(); + long inputTs2 = c2.getTimestamp(); + + long returnTs = returnedCell.getTimestamp(); + long returnValue = Bytes.toLong(CellUtil + .cloneValue(returnedCell)); + // the returned Ts will be far greater than input ts as well as the noted + // current timestamp + if (returnValue == cellValue2) { + assertTrue(returnTs == inputTs2); + } else if (returnValue == cellValue1) { + assertTrue(returnTs >= currentTimestamp); + assertTrue(returnTs != inputTs1); + } else { + // raise a failure since we expect only these two values back + Assert.fail(); + } + } + + @Test + public void testProcessSummationOneCellSumFinal() throws IOException { + FlowScanner fs = getFlowScannerForTestingCompaction(); + + // note down the current timestamp + long currentTimestamp = System.currentTimeMillis(); + List tags = new ArrayList<>(); + Tag t = new Tag(AggregationOperation.SUM_FINAL.getTagType(), + "application_123458888888_999888"); + tags.add(t); + byte[] tagByteArray = Tag.fromList(tags); + SortedSet currentColumnCells = new TreeSet(KeyValue.COMPARATOR); + + // create a cell with a VERY old timestamp + Cell c1 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily, + aQualifier, 120L, Bytes.toBytes(1110L), tagByteArray); + currentColumnCells.add(c1); + + List cells = fs.processSummationMajorCompaction(currentColumnCells, + new LongConverter(), currentTimestamp); + assertNotNull(cells); + // we should not get the same cell back + // but we get back the flow cell + assertEquals(1, cells.size()); + + Cell returnedCell = cells.get(0); + // it's NOT the same cell + assertNotEquals(c1, returnedCell); + long inputTs = c1.getTimestamp(); + long returnTs = returnedCell.getTimestamp(); + // the returned Ts will be far greater than input ts as well as the noted + // current timestamp + assertTrue(returnTs > inputTs); + assertTrue(returnTs >= currentTimestamp); + } + + @Test + public void testProcessSummationOneCell() throws IOException { + FlowScanner fs = getFlowScannerForTestingCompaction(); + + // note down the current timestamp + long currentTimestamp = System.currentTimeMillis(); + + // try for 1 cell with tag SUM + List tags = new ArrayList<>(); + Tag t = new Tag(AggregationOperation.SUM.getTagType(), + "application_123458888888_999888"); + tags.add(t); + byte[] tagByteArray = Tag.fromList(tags); + + SortedSet currentColumnCells = new TreeSet(KeyValue.COMPARATOR); + + Cell c1 = HBaseTimelineStorageUtils.createNewCell(aRowKey, aFamily, + aQualifier, currentTimestamp, Bytes.toBytes(1110L), tagByteArray); + currentColumnCells.add(c1); + List cells = fs.processSummationMajorCompaction(currentColumnCells, + new LongConverter(), currentTimestamp); + assertNotNull(cells); + // we expect the same cell back + assertEquals(1, cells.size()); + Cell c2 = cells.get(0); + assertEquals(c1, c2); + assertEquals(currentTimestamp, c2.getTimestamp()); + } + + @Test + public void testProcessSummationEmpty() throws IOException { + FlowScanner fs = getFlowScannerForTestingCompaction(); + long currentTimestamp = System.currentTimeMillis(); + + LongConverter longConverter = new LongConverter(); + + SortedSet currentColumnCells = null; + List cells = + fs.processSummationMajorCompaction(currentColumnCells, longConverter, + currentTimestamp); + assertNotNull(cells); + assertEquals(0, cells.size()); + + currentColumnCells = new TreeSet(KeyValue.COMPARATOR); + cells = + fs.processSummationMajorCompaction(currentColumnCells, longConverter, + currentTimestamp); + assertNotNull(cells); + assertEquals(0, cells.size()); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + util.shutdownMiniCluster(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/resources/log4j.properties b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/resources/log4j.properties new file mode 100644 index 00000000000..81a3f6ad5d2 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/resources/log4j.properties @@ -0,0 +1,19 @@ +# Licensed 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. + +# log4j configuration used during build and unit tests + +log4j.rootLogger=info,stdout +log4j.threshold=ALL +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2} (%F:%M(%L)) - %m%n diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/pom.xml new file mode 100644 index 00000000000..6c568cedd09 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/pom.xml @@ -0,0 +1,213 @@ + + + + + hadoop-yarn-server + org.apache.hadoop + 2.9.0-SNAPSHOT + + 4.0.0 + hadoop-yarn-server-timelineservice-hbase + Apache Hadoop YARN TimelineService HBase Backend + + + + ${project.parent.parent.basedir} + + + + + commons-logging + commons-logging + + + + commons-lang + commons-lang + + + + commons-cli + commons-cli + + + + com.google.guava + guava + + + + org.apache.hadoop + hadoop-annotations + provided + + + + org.apache.hadoop + hadoop-common + provided + + + + org.apache.hadoop + hadoop-common + test-jar + test + + + + org.apache.hadoop + hadoop-yarn-api + provided + + + + org.apache.hadoop + hadoop-yarn-common + provided + + + + org.apache.hadoop + hadoop-yarn-server-applicationhistoryservice + provided + + + + org.apache.hadoop + hadoop-yarn-server-timelineservice + + + + org.apache.hbase + hbase-common + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + org.mortbay.jetty + jetty-util + + + + + + org.apache.hbase + hbase-client + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + + + + org.apache.hbase + hbase-server + provided + + + org.apache.hadoop + hadoop-hdfs + + + org.apache.hadoop + hadoop-hdfs-client + + + org.apache.hadoop + hadoop-client + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + org.mortbay.jetty + jetty + + + org.mortbay.jetty + jetty-util + + + org.mortbay.jetty + jetty-sslengine + + + + + + junit + junit + test + + + + + + + maven-jar-plugin + + + + test-jar + + test-compile + + + + + org.apache.maven.plugins + maven-javadoc-plugin + + + + junit + junit + 4.11 + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + package + + copy-dependencies + + + runtime + org.slf4j,org.apache.hadoop,com.github.stephenc.findbugs + ${project.build.directory}/lib + + + + + + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilterUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilterUtils.java new file mode 100644 index 00000000000..8b46d324135 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilterUtils.java @@ -0,0 +1,308 @@ +/** + * 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.timelineservice.reader.filter; + +import java.io.IOException; +import java.util.HashSet; +import java.util.Set; + +import org.apache.hadoop.hbase.filter.BinaryComparator; +import org.apache.hadoop.hbase.filter.BinaryPrefixComparator; +import org.apache.hadoop.hbase.filter.FamilyFilter; +import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; +import org.apache.hadoop.hbase.filter.Filter; +import org.apache.hadoop.hbase.filter.FilterList; +import org.apache.hadoop.hbase.filter.FilterList.Operator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Column; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; +import org.apache.hadoop.hbase.filter.QualifierFilter; +import org.apache.hadoop.hbase.filter.SingleColumnValueFilter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Set of utility methods used by timeline filter classes. + */ +public final class TimelineFilterUtils { + + private static final Logger LOG = + LoggerFactory.getLogger(TimelineFilterUtils.class); + + private TimelineFilterUtils() { + } + + /** + * Returns the equivalent HBase filter list's {@link Operator}. + * + * @param op timeline filter list operator. + * @return HBase filter list's Operator. + */ + private static Operator getHBaseOperator(TimelineFilterList.Operator op) { + switch (op) { + case AND: + return Operator.MUST_PASS_ALL; + case OR: + return Operator.MUST_PASS_ONE; + default: + throw new IllegalArgumentException("Invalid operator"); + } + } + + /** + * Returns the equivalent HBase compare filter's {@link CompareOp}. + * + * @param op timeline compare op. + * @return HBase compare filter's CompareOp. + */ + private static CompareOp getHBaseCompareOp( + TimelineCompareOp op) { + switch (op) { + case LESS_THAN: + return CompareOp.LESS; + case LESS_OR_EQUAL: + return CompareOp.LESS_OR_EQUAL; + case EQUAL: + return CompareOp.EQUAL; + case NOT_EQUAL: + return CompareOp.NOT_EQUAL; + case GREATER_OR_EQUAL: + return CompareOp.GREATER_OR_EQUAL; + case GREATER_THAN: + return CompareOp.GREATER; + default: + throw new IllegalArgumentException("Invalid compare operator"); + } + } + + /** + * Converts a {@link TimelinePrefixFilter} to an equivalent HBase + * {@link QualifierFilter}. + * @param colPrefix + * @param filter + * @return a {@link QualifierFilter} object + */ + private static Filter createHBaseColQualPrefixFilter( + ColumnPrefix colPrefix, TimelinePrefixFilter filter) { + return new QualifierFilter(getHBaseCompareOp(filter.getCompareOp()), + new BinaryPrefixComparator( + colPrefix.getColumnPrefixBytes(filter.getPrefix()))); + } + + /** + * Create a HBase {@link QualifierFilter} for the passed column prefix and + * compare op. + * + * @param Describes the type of column prefix. + * @param compareOp compare op. + * @param columnPrefix column prefix. + * @return a column qualifier filter. + */ + public static Filter createHBaseQualifierFilter(CompareOp compareOp, + ColumnPrefix columnPrefix) { + return new QualifierFilter(compareOp, + new BinaryPrefixComparator( + columnPrefix.getColumnPrefixBytes(""))); + } + + /** + * Create filters for confs or metrics to retrieve. This list includes a + * configs/metrics family filter and relevant filters for confs/metrics to + * retrieve, if present. + * + * @param Describes the type of column prefix. + * @param confsOrMetricToRetrieve configs/metrics to retrieve. + * @param columnFamily config or metric column family. + * @param columnPrefix config or metric column prefix. + * @return a filter list. + * @throws IOException if any problem occurs while creating the filters. + */ + public static Filter createFilterForConfsOrMetricsToRetrieve( + TimelineFilterList confsOrMetricToRetrieve, ColumnFamily columnFamily, + ColumnPrefix columnPrefix) throws IOException { + Filter familyFilter = new FamilyFilter(CompareOp.EQUAL, + new BinaryComparator(columnFamily.getBytes())); + if (confsOrMetricToRetrieve != null && + !confsOrMetricToRetrieve.getFilterList().isEmpty()) { + // If confsOrMetricsToRetrive are specified, create a filter list based + // on it and family filter. + FilterList filter = new FilterList(familyFilter); + filter.addFilter( + createHBaseFilterList(columnPrefix, confsOrMetricToRetrieve)); + return filter; + } else { + // Only the family filter needs to be added. + return familyFilter; + } + } + + /** + * Create 2 HBase {@link SingleColumnValueFilter} filters for the specified + * value range represented by start and end value and wraps them inside a + * filter list. Start and end value should not be null. + * + * @param Describes the type of column prefix. + * @param column Column for which single column value filter is to be created. + * @param startValue Start value. + * @param endValue End value. + * @return 2 single column value filters wrapped in a filter list. + * @throws IOException if any problem is encountered while encoding value. + */ + public static FilterList createSingleColValueFiltersByRange( + Column column, Object startValue, Object endValue) throws IOException { + FilterList list = new FilterList(); + Filter singleColValFilterStart = createHBaseSingleColValueFilter( + column.getColumnFamilyBytes(), column.getColumnQualifierBytes(), + column.getValueConverter().encodeValue(startValue), + CompareOp.GREATER_OR_EQUAL, true); + list.addFilter(singleColValFilterStart); + + Filter singleColValFilterEnd = createHBaseSingleColValueFilter( + column.getColumnFamilyBytes(), column.getColumnQualifierBytes(), + column.getValueConverter().encodeValue(endValue), + CompareOp.LESS_OR_EQUAL, true); + list.addFilter(singleColValFilterEnd); + return list; + } + + /** + * Creates a HBase {@link SingleColumnValueFilter} with specified column. + * @param Describes the type of column prefix. + * @param column Column which value to be filtered. + * @param value Value to be filtered. + * @param op Compare operator + * @return a SingleColumnValue Filter + * @throws IOException if any exception. + */ + public static Filter createHBaseSingleColValueFilter(Column column, + Object value, CompareOp op) throws IOException { + Filter singleColValFilter = createHBaseSingleColValueFilter( + column.getColumnFamilyBytes(), column.getColumnQualifierBytes(), + column.getValueConverter().encodeValue(value), op, true); + return singleColValFilter; + } + + /** + * Creates a HBase {@link SingleColumnValueFilter}. + * + * @param columnFamily Column Family represented as bytes. + * @param columnQualifier Column Qualifier represented as bytes. + * @param value Value. + * @param compareOp Compare operator. + * @param filterIfMissing This flag decides if we should filter the row if the + * specified column is missing. This is based on the filter's keyMustExist + * field. + * @return a {@link SingleColumnValueFilter} object + * @throws IOException + */ + private static SingleColumnValueFilter createHBaseSingleColValueFilter( + byte[] columnFamily, byte[] columnQualifier, byte[] value, + CompareOp compareOp, boolean filterIfMissing) throws IOException { + SingleColumnValueFilter singleColValFilter = + new SingleColumnValueFilter(columnFamily, columnQualifier, compareOp, + new BinaryComparator(value)); + singleColValFilter.setLatestVersionOnly(true); + singleColValFilter.setFilterIfMissing(filterIfMissing); + return singleColValFilter; + } + + /** + * Fetch columns from filter list containing exists and multivalue equality + * filters. This is done to fetch only required columns from back-end and + * then match event filters or relationships in reader. + * + * @param filterList filter list. + * @return set of columns. + */ + public static Set fetchColumnsFromFilterList( + TimelineFilterList filterList) { + Set strSet = new HashSet(); + for (TimelineFilter filter : filterList.getFilterList()) { + switch(filter.getFilterType()) { + case LIST: + strSet.addAll(fetchColumnsFromFilterList((TimelineFilterList)filter)); + break; + case KEY_VALUES: + strSet.add(((TimelineKeyValuesFilter)filter).getKey()); + break; + case EXISTS: + strSet.add(((TimelineExistsFilter)filter).getValue()); + break; + default: + LOG.info("Unexpected filter type " + filter.getFilterType()); + break; + } + } + return strSet; + } + + /** + * Creates equivalent HBase {@link FilterList} from {@link TimelineFilterList} + * while converting different timeline filters(of type {@link TimelineFilter}) + * into their equivalent HBase filters. + * + * @param Describes the type of column prefix. + * @param colPrefix column prefix which will be used for conversion. + * @param filterList timeline filter list which has to be converted. + * @return A {@link FilterList} object. + * @throws IOException if any problem occurs while creating the filter list. + */ + public static FilterList createHBaseFilterList(ColumnPrefix colPrefix, + TimelineFilterList filterList) throws IOException { + FilterList list = + new FilterList(getHBaseOperator(filterList.getOperator())); + for (TimelineFilter filter : filterList.getFilterList()) { + switch(filter.getFilterType()) { + case LIST: + list.addFilter(createHBaseFilterList(colPrefix, + (TimelineFilterList)filter)); + break; + case PREFIX: + list.addFilter(createHBaseColQualPrefixFilter(colPrefix, + (TimelinePrefixFilter)filter)); + break; + case COMPARE: + TimelineCompareFilter compareFilter = (TimelineCompareFilter)filter; + list.addFilter( + createHBaseSingleColValueFilter( + colPrefix.getColumnFamilyBytes(), + colPrefix.getColumnPrefixBytes(compareFilter.getKey()), + colPrefix.getValueConverter(). + encodeValue(compareFilter.getValue()), + getHBaseCompareOp(compareFilter.getCompareOp()), + compareFilter.getKeyMustExist())); + break; + case KEY_VALUE: + TimelineKeyValueFilter kvFilter = (TimelineKeyValueFilter)filter; + list.addFilter( + createHBaseSingleColValueFilter( + colPrefix.getColumnFamilyBytes(), + colPrefix.getColumnPrefixBytes(kvFilter.getKey()), + colPrefix.getValueConverter().encodeValue(kvFilter.getValue()), + getHBaseCompareOp(kvFilter.getCompareOp()), + kvFilter.getKeyMustExist())); + break; + default: + LOG.info("Unexpected filter type " + filter.getFilterType()); + break; + } + } + return list; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/package-info.java new file mode 100644 index 00000000000..f7c07057d7c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/package-info.java @@ -0,0 +1,28 @@ +/* + * 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.server.timelineservice.reader.filter stores + * timeline filter implementations. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.timelineservice.reader.filter; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java new file mode 100644 index 00000000000..1ebfab29cad --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java @@ -0,0 +1,96 @@ +/** + * 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.timelineservice.storage; + + +import java.io.IOException; +import java.util.Set; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.reader.EntityTypeReader; +import org.apache.hadoop.yarn.server.timelineservice.storage.reader.TimelineEntityReader; +import org.apache.hadoop.yarn.server.timelineservice.storage.reader.TimelineEntityReaderFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * HBase based implementation for {@link TimelineReader}. + */ +public class HBaseTimelineReaderImpl + extends AbstractService implements TimelineReader { + + private static final Logger LOG = LoggerFactory + .getLogger(HBaseTimelineReaderImpl.class); + + private Configuration hbaseConf = null; + private Connection conn; + + public HBaseTimelineReaderImpl() { + super(HBaseTimelineReaderImpl.class.getName()); + } + + @Override + public void serviceInit(Configuration conf) throws Exception { + super.serviceInit(conf); + hbaseConf = HBaseTimelineStorageUtils.getTimelineServiceHBaseConf(conf); + conn = ConnectionFactory.createConnection(hbaseConf); + } + + @Override + protected void serviceStop() throws Exception { + if (conn != null) { + LOG.info("closing the hbase Connection"); + conn.close(); + } + super.serviceStop(); + } + + @Override + public TimelineEntity getEntity(TimelineReaderContext context, + TimelineDataToRetrieve dataToRetrieve) throws IOException { + TimelineEntityReader reader = + TimelineEntityReaderFactory.createSingleEntityReader(context, + dataToRetrieve); + return reader.readEntity(hbaseConf, conn); + } + + @Override + public Set getEntities(TimelineReaderContext context, + TimelineEntityFilters filters, TimelineDataToRetrieve dataToRetrieve) + throws IOException { + TimelineEntityReader reader = + TimelineEntityReaderFactory.createMultipleEntitiesReader(context, + filters, dataToRetrieve); + return reader.readEntities(hbaseConf, conn); + } + + @Override + public Set getEntityTypes(TimelineReaderContext context) + throws IOException { + EntityTypeReader reader = new EntityTypeReader(context); + return reader.readEntityTypes(hbaseConf, conn); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java new file mode 100644 index 00000000000..9e9134cf3eb --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java @@ -0,0 +1,588 @@ +/** + * 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.timelineservice.storage; + +import java.io.IOException; +import java.util.Map; +import java.util.Set; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineWriteResponse; +import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants; +import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumn; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnName; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongKeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationCompactionDimension; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationOperation; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumn; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationColumn; +import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This implements a hbase based backend for storing the timeline entity + * information. + * It writes to multiple tables at the backend + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public class HBaseTimelineWriterImpl extends AbstractService implements + TimelineWriter { + + private static final Logger LOG = LoggerFactory + .getLogger(HBaseTimelineWriterImpl.class); + + private Connection conn; + private TypedBufferedMutator entityTable; + private TypedBufferedMutator appToFlowTable; + private TypedBufferedMutator applicationTable; + private TypedBufferedMutator flowActivityTable; + private TypedBufferedMutator flowRunTable; + private TypedBufferedMutator subApplicationTable; + + /** + * Used to convert strings key components to and from storage format. + */ + private final KeyConverter stringKeyConverter = + new StringKeyConverter(); + + /** + * Used to convert Long key components to and from storage format. + */ + private final KeyConverter longKeyConverter = new LongKeyConverter(); + + private enum Tables { + APPLICATION_TABLE, ENTITY_TABLE, SUBAPPLICATION_TABLE + }; + + public HBaseTimelineWriterImpl() { + super(HBaseTimelineWriterImpl.class.getName()); + } + + /** + * initializes the hbase connection to write to the entity table. + */ + @Override + protected void serviceInit(Configuration conf) throws Exception { + super.serviceInit(conf); + Configuration hbaseConf = + HBaseTimelineStorageUtils.getTimelineServiceHBaseConf(conf); + conn = ConnectionFactory.createConnection(hbaseConf); + entityTable = new EntityTable().getTableMutator(hbaseConf, conn); + appToFlowTable = new AppToFlowTable().getTableMutator(hbaseConf, conn); + applicationTable = new ApplicationTable().getTableMutator(hbaseConf, conn); + flowRunTable = new FlowRunTable().getTableMutator(hbaseConf, conn); + flowActivityTable = + new FlowActivityTable().getTableMutator(hbaseConf, conn); + subApplicationTable = + new SubApplicationTable().getTableMutator(hbaseConf, conn); + } + + /** + * Stores the entire information in TimelineEntities to the timeline store. + */ + @Override + public TimelineWriteResponse write(TimelineCollectorContext context, + TimelineEntities data, UserGroupInformation callerUgi) + throws IOException { + + TimelineWriteResponse putStatus = new TimelineWriteResponse(); + + String clusterId = context.getClusterId(); + String userId = context.getUserId(); + String flowName = context.getFlowName(); + String flowVersion = context.getFlowVersion(); + long flowRunId = context.getFlowRunId(); + String appId = context.getAppId(); + String subApplicationUser = callerUgi.getShortUserName(); + + // defensive coding to avoid NPE during row key construction + if ((flowName == null) || (appId == null) || (clusterId == null) + || (userId == null)) { + LOG.warn("Found null for one of: flowName=" + flowName + " appId=" + appId + + " userId=" + userId + " clusterId=" + clusterId + + " . Not proceeding with writing to hbase"); + return putStatus; + } + + for (TimelineEntity te : data.getEntities()) { + + // a set can have at most 1 null + if (te == null) { + continue; + } + + // if the entity is the application, the destination is the application + // table + boolean isApplication = ApplicationEntity.isApplicationEntity(te); + byte[] rowKey; + if (isApplication) { + ApplicationRowKey applicationRowKey = + new ApplicationRowKey(clusterId, userId, flowName, flowRunId, + appId); + rowKey = applicationRowKey.getRowKey(); + store(rowKey, te, flowVersion, Tables.APPLICATION_TABLE); + } else { + EntityRowKey entityRowKey = + new EntityRowKey(clusterId, userId, flowName, flowRunId, appId, + te.getType(), te.getIdPrefix(), te.getId()); + rowKey = entityRowKey.getRowKey(); + store(rowKey, te, flowVersion, Tables.ENTITY_TABLE); + } + + if (!isApplication && !userId.equals(subApplicationUser)) { + SubApplicationRowKey subApplicationRowKey = + new SubApplicationRowKey(subApplicationUser, clusterId, + te.getType(), te.getIdPrefix(), te.getId(), userId); + rowKey = subApplicationRowKey.getRowKey(); + store(rowKey, te, flowVersion, Tables.SUBAPPLICATION_TABLE); + } + + if (isApplication) { + TimelineEvent event = + ApplicationEntity.getApplicationEvent(te, + ApplicationMetricsConstants.CREATED_EVENT_TYPE); + FlowRunRowKey flowRunRowKey = + new FlowRunRowKey(clusterId, userId, flowName, flowRunId); + if (event != null) { + onApplicationCreated(flowRunRowKey, clusterId, appId, userId, + flowVersion, te, event.getTimestamp()); + } + // if it's an application entity, store metrics + storeFlowMetricsAppRunning(flowRunRowKey, appId, te); + // if application has finished, store it's finish time and write final + // values of all metrics + event = ApplicationEntity.getApplicationEvent(te, + ApplicationMetricsConstants.FINISHED_EVENT_TYPE); + if (event != null) { + onApplicationFinished(flowRunRowKey, flowVersion, appId, te, + event.getTimestamp()); + } + } + } + return putStatus; + } + + private void onApplicationCreated(FlowRunRowKey flowRunRowKey, + String clusterId, String appId, String userId, String flowVersion, + TimelineEntity te, long appCreatedTimeStamp) + throws IOException { + + String flowName = flowRunRowKey.getFlowName(); + Long flowRunId = flowRunRowKey.getFlowRunId(); + + // store in App to flow table + AppToFlowRowKey appToFlowRowKey = new AppToFlowRowKey(appId); + byte[] rowKey = appToFlowRowKey.getRowKey(); + AppToFlowColumnPrefix.FLOW_NAME.store(rowKey, appToFlowTable, clusterId, + null, flowName); + AppToFlowColumnPrefix.FLOW_RUN_ID.store(rowKey, appToFlowTable, clusterId, + null, flowRunId); + AppToFlowColumnPrefix.USER_ID.store(rowKey, appToFlowTable, clusterId, null, + userId); + + // store in flow run table + storeAppCreatedInFlowRunTable(flowRunRowKey, appId, te); + + // store in flow activity table + byte[] flowActivityRowKeyBytes = + new FlowActivityRowKey(flowRunRowKey.getClusterId(), + appCreatedTimeStamp, flowRunRowKey.getUserId(), flowName) + .getRowKey(); + byte[] qualifier = longKeyConverter.encode(flowRunRowKey.getFlowRunId()); + FlowActivityColumnPrefix.RUN_ID.store(flowActivityRowKeyBytes, + flowActivityTable, qualifier, null, flowVersion, + AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId)); + } + + /* + * updates the {@link FlowRunTable} with Application Created information + */ + private void storeAppCreatedInFlowRunTable(FlowRunRowKey flowRunRowKey, + String appId, TimelineEntity te) throws IOException { + byte[] rowKey = flowRunRowKey.getRowKey(); + FlowRunColumn.MIN_START_TIME.store(rowKey, flowRunTable, null, + te.getCreatedTime(), + AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId)); + } + + + /* + * updates the {@link FlowRunTable} and {@link FlowActivityTable} when an + * application has finished + */ + private void onApplicationFinished(FlowRunRowKey flowRunRowKey, + String flowVersion, String appId, TimelineEntity te, + long appFinishedTimeStamp) throws IOException { + // store in flow run table + storeAppFinishedInFlowRunTable(flowRunRowKey, appId, te, + appFinishedTimeStamp); + + // indicate in the flow activity table that the app has finished + byte[] rowKey = + new FlowActivityRowKey(flowRunRowKey.getClusterId(), + appFinishedTimeStamp, flowRunRowKey.getUserId(), + flowRunRowKey.getFlowName()).getRowKey(); + byte[] qualifier = longKeyConverter.encode(flowRunRowKey.getFlowRunId()); + FlowActivityColumnPrefix.RUN_ID.store(rowKey, flowActivityTable, qualifier, + null, flowVersion, + AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId)); + } + + /* + * Update the {@link FlowRunTable} with Application Finished information + */ + private void storeAppFinishedInFlowRunTable(FlowRunRowKey flowRunRowKey, + String appId, TimelineEntity te, long appFinishedTimeStamp) + throws IOException { + byte[] rowKey = flowRunRowKey.getRowKey(); + Attribute attributeAppId = + AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId); + FlowRunColumn.MAX_END_TIME.store(rowKey, flowRunTable, null, + appFinishedTimeStamp, attributeAppId); + + // store the final value of metrics since application has finished + Set metrics = te.getMetrics(); + if (metrics != null) { + storeFlowMetrics(rowKey, metrics, attributeAppId, + AggregationOperation.SUM_FINAL.getAttribute()); + } + } + + /* + * Updates the {@link FlowRunTable} with Application Metrics + */ + private void storeFlowMetricsAppRunning(FlowRunRowKey flowRunRowKey, + String appId, TimelineEntity te) throws IOException { + Set metrics = te.getMetrics(); + if (metrics != null) { + byte[] rowKey = flowRunRowKey.getRowKey(); + storeFlowMetrics(rowKey, metrics, + AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId), + AggregationOperation.SUM.getAttribute()); + } + } + + private void storeFlowMetrics(byte[] rowKey, Set metrics, + Attribute... attributes) throws IOException { + for (TimelineMetric metric : metrics) { + byte[] metricColumnQualifier = stringKeyConverter.encode(metric.getId()); + Map timeseries = metric.getValues(); + for (Map.Entry timeseriesEntry : timeseries.entrySet()) { + Long timestamp = timeseriesEntry.getKey(); + FlowRunColumnPrefix.METRIC.store(rowKey, flowRunTable, + metricColumnQualifier, timestamp, timeseriesEntry.getValue(), + attributes); + } + } + } + + /** + * Stores the Relations from the {@linkplain TimelineEntity} object. + */ + private void storeRelations(byte[] rowKey, + Map> connectedEntities, ColumnPrefix columnPrefix, + TypedBufferedMutator table) throws IOException { + if (connectedEntities != null) { + for (Map.Entry> connectedEntity : connectedEntities + .entrySet()) { + // id3?id4?id5 + String compoundValue = + Separator.VALUES.joinEncoded(connectedEntity.getValue()); + columnPrefix.store(rowKey, table, + stringKeyConverter.encode(connectedEntity.getKey()), null, + compoundValue); + } + } + } + + /** + * Stores information from the {@linkplain TimelineEntity} object. + */ + private void store(byte[] rowKey, TimelineEntity te, + String flowVersion, + Tables table) throws IOException { + switch (table) { + case APPLICATION_TABLE: + ApplicationColumn.ID.store(rowKey, applicationTable, null, te.getId()); + ApplicationColumn.CREATED_TIME.store(rowKey, applicationTable, null, + te.getCreatedTime()); + ApplicationColumn.FLOW_VERSION.store(rowKey, applicationTable, null, + flowVersion); + storeInfo(rowKey, te.getInfo(), flowVersion, ApplicationColumnPrefix.INFO, + applicationTable); + storeMetrics(rowKey, te.getMetrics(), ApplicationColumnPrefix.METRIC, + applicationTable); + storeEvents(rowKey, te.getEvents(), ApplicationColumnPrefix.EVENT, + applicationTable); + storeConfig(rowKey, te.getConfigs(), ApplicationColumnPrefix.CONFIG, + applicationTable); + storeRelations(rowKey, te.getIsRelatedToEntities(), + ApplicationColumnPrefix.IS_RELATED_TO, applicationTable); + storeRelations(rowKey, te.getRelatesToEntities(), + ApplicationColumnPrefix.RELATES_TO, applicationTable); + break; + case ENTITY_TABLE: + EntityColumn.ID.store(rowKey, entityTable, null, te.getId()); + EntityColumn.TYPE.store(rowKey, entityTable, null, te.getType()); + EntityColumn.CREATED_TIME.store(rowKey, entityTable, null, + te.getCreatedTime()); + EntityColumn.FLOW_VERSION.store(rowKey, entityTable, null, flowVersion); + storeInfo(rowKey, te.getInfo(), flowVersion, EntityColumnPrefix.INFO, + entityTable); + storeMetrics(rowKey, te.getMetrics(), EntityColumnPrefix.METRIC, + entityTable); + storeEvents(rowKey, te.getEvents(), EntityColumnPrefix.EVENT, + entityTable); + storeConfig(rowKey, te.getConfigs(), EntityColumnPrefix.CONFIG, + entityTable); + storeRelations(rowKey, te.getIsRelatedToEntities(), + EntityColumnPrefix.IS_RELATED_TO, entityTable); + storeRelations(rowKey, te.getRelatesToEntities(), + EntityColumnPrefix.RELATES_TO, entityTable); + break; + case SUBAPPLICATION_TABLE: + SubApplicationColumn.ID.store(rowKey, subApplicationTable, null, + te.getId()); + SubApplicationColumn.TYPE.store(rowKey, subApplicationTable, null, + te.getType()); + SubApplicationColumn.CREATED_TIME.store(rowKey, subApplicationTable, null, + te.getCreatedTime()); + SubApplicationColumn.FLOW_VERSION.store(rowKey, subApplicationTable, null, + flowVersion); + storeInfo(rowKey, te.getInfo(), flowVersion, + SubApplicationColumnPrefix.INFO, subApplicationTable); + storeMetrics(rowKey, te.getMetrics(), SubApplicationColumnPrefix.METRIC, + subApplicationTable); + storeEvents(rowKey, te.getEvents(), SubApplicationColumnPrefix.EVENT, + subApplicationTable); + storeConfig(rowKey, te.getConfigs(), SubApplicationColumnPrefix.CONFIG, + subApplicationTable); + storeRelations(rowKey, te.getIsRelatedToEntities(), + SubApplicationColumnPrefix.IS_RELATED_TO, subApplicationTable); + storeRelations(rowKey, te.getRelatesToEntities(), + SubApplicationColumnPrefix.RELATES_TO, subApplicationTable); + break; + default: + LOG.info("Invalid table name provided."); + break; + } + } + + /** + * stores the info information from {@linkplain TimelineEntity}. + */ + private void storeInfo(byte[] rowKey, Map info, + String flowVersion, ColumnPrefix columnPrefix, + TypedBufferedMutator table) throws IOException { + if (info != null) { + for (Map.Entry entry : info.entrySet()) { + columnPrefix.store(rowKey, table, + stringKeyConverter.encode(entry.getKey()), null, entry.getValue()); + } + } + } + + /** + * stores the config information from {@linkplain TimelineEntity}. + */ + private void storeConfig(byte[] rowKey, Map config, + ColumnPrefix columnPrefix, TypedBufferedMutator table) + throws IOException { + if (config != null) { + for (Map.Entry entry : config.entrySet()) { + byte[] configKey = stringKeyConverter.encode(entry.getKey()); + columnPrefix.store(rowKey, table, configKey, null, entry.getValue()); + } + } + } + + /** + * stores the {@linkplain TimelineMetric} information from the + * {@linkplain TimelineEvent} object. + */ + private void storeMetrics(byte[] rowKey, Set metrics, + ColumnPrefix columnPrefix, TypedBufferedMutator table) + throws IOException { + if (metrics != null) { + for (TimelineMetric metric : metrics) { + byte[] metricColumnQualifier = + stringKeyConverter.encode(metric.getId()); + Map timeseries = metric.getValues(); + for (Map.Entry timeseriesEntry : timeseries.entrySet()) { + Long timestamp = timeseriesEntry.getKey(); + columnPrefix.store(rowKey, table, metricColumnQualifier, timestamp, + timeseriesEntry.getValue()); + } + } + } + } + + /** + * Stores the events from the {@linkplain TimelineEvent} object. + */ + private void storeEvents(byte[] rowKey, Set events, + ColumnPrefix columnPrefix, TypedBufferedMutator table) + throws IOException { + if (events != null) { + for (TimelineEvent event : events) { + if (event != null) { + String eventId = event.getId(); + if (eventId != null) { + long eventTimestamp = event.getTimestamp(); + // if the timestamp is not set, use the current timestamp + if (eventTimestamp == TimelineEvent.INVALID_TIMESTAMP) { + LOG.warn("timestamp is not set for event " + eventId + + "! Using the current timestamp"); + eventTimestamp = System.currentTimeMillis(); + } + Map eventInfo = event.getInfo(); + if ((eventInfo == null) || (eventInfo.size() == 0)) { + byte[] columnQualifierBytes = + new EventColumnName(eventId, eventTimestamp, null) + .getColumnQualifier(); + columnPrefix.store(rowKey, table, columnQualifierBytes, null, + Separator.EMPTY_BYTES); + } else { + for (Map.Entry info : eventInfo.entrySet()) { + // eventId=infoKey + byte[] columnQualifierBytes = + new EventColumnName(eventId, eventTimestamp, info.getKey()) + .getColumnQualifier(); + columnPrefix.store(rowKey, table, columnQualifierBytes, null, + info.getValue()); + } // for info: eventInfo + } + } + } + } // event : events + } + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage + * .TimelineWriter#aggregate + * (org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity, + * org.apache + * .hadoop.yarn.server.timelineservice.storage.TimelineAggregationTrack) + */ + @Override + public TimelineWriteResponse aggregate(TimelineEntity data, + TimelineAggregationTrack track) throws IOException { + return null; + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter#flush + * () + */ + @Override + public void flush() throws IOException { + // flush all buffered mutators + entityTable.flush(); + appToFlowTable.flush(); + applicationTable.flush(); + flowRunTable.flush(); + flowActivityTable.flush(); + subApplicationTable.flush(); + } + + /** + * close the hbase connections The close APIs perform flushing and release any + * resources held. + */ + @Override + protected void serviceStop() throws Exception { + if (entityTable != null) { + LOG.info("closing the entity table"); + // The close API performs flushing and releases any resources held + entityTable.close(); + } + if (appToFlowTable != null) { + LOG.info("closing the app_flow table"); + // The close API performs flushing and releases any resources held + appToFlowTable.close(); + } + if (applicationTable != null) { + LOG.info("closing the application table"); + applicationTable.close(); + } + if (flowRunTable != null) { + LOG.info("closing the flow run table"); + // The close API performs flushing and releases any resources held + flowRunTable.close(); + } + if (flowActivityTable != null) { + LOG.info("closing the flowActivityTable table"); + // The close API performs flushing and releases any resources held + flowActivityTable.close(); + } + if (subApplicationTable != null) { + subApplicationTable.close(); + } + if (conn != null) { + LOG.info("closing the hbase Connection"); + conn.close(); + } + super.serviceStop(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java new file mode 100644 index 00000000000..c9f7cecdf56 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java @@ -0,0 +1,367 @@ +/** + * 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.timelineservice.storage; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.CommandLineParser; +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.ParseException; +import org.apache.commons.cli.PosixParser; +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.util.GenericOptionsParser; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationTable; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This creates the schema for a hbase based backend for storing application + * timeline information. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public final class TimelineSchemaCreator { + private TimelineSchemaCreator() { + } + + final static String NAME = TimelineSchemaCreator.class.getSimpleName(); + private static final Logger LOG = + LoggerFactory.getLogger(TimelineSchemaCreator.class); + private static final String SKIP_EXISTING_TABLE_OPTION_SHORT = "s"; + private static final String APP_METRICS_TTL_OPTION_SHORT = "ma"; + private static final String SUB_APP_METRICS_TTL_OPTION_SHORT = "msa"; + private static final String APP_TABLE_NAME_SHORT = "a"; + private static final String SUB_APP_TABLE_NAME_SHORT = "sa"; + private static final String APP_TO_FLOW_TABLE_NAME_SHORT = "a2f"; + private static final String ENTITY_METRICS_TTL_OPTION_SHORT = "me"; + private static final String ENTITY_TABLE_NAME_SHORT = "e"; + private static final String HELP_SHORT = "h"; + private static final String CREATE_TABLES_SHORT = "c"; + + public static void main(String[] args) throws Exception { + + LOG.info("Starting the schema creation"); + Configuration hbaseConf = + HBaseTimelineStorageUtils.getTimelineServiceHBaseConf( + new YarnConfiguration()); + // Grab input args and allow for -Dxyz style arguments + String[] otherArgs = new GenericOptionsParser(hbaseConf, args) + .getRemainingArgs(); + + // Grab the arguments we're looking for. + CommandLine commandLine = parseArgs(otherArgs); + + if (commandLine.hasOption(HELP_SHORT)) { + // -help option has the highest precedence + printUsage(); + } else if (commandLine.hasOption(CREATE_TABLES_SHORT)) { + // Grab the entityTableName argument + String entityTableName = commandLine.getOptionValue( + ENTITY_TABLE_NAME_SHORT); + if (StringUtils.isNotBlank(entityTableName)) { + hbaseConf.set(EntityTable.TABLE_NAME_CONF_NAME, entityTableName); + } + // Grab the entity metrics TTL + String entityTableMetricsTTL = commandLine.getOptionValue( + ENTITY_METRICS_TTL_OPTION_SHORT); + if (StringUtils.isNotBlank(entityTableMetricsTTL)) { + int entityMetricsTTL = Integer.parseInt(entityTableMetricsTTL); + new EntityTable().setMetricsTTL(entityMetricsTTL, hbaseConf); + } + // Grab the appToflowTableName argument + String appToflowTableName = commandLine.getOptionValue( + APP_TO_FLOW_TABLE_NAME_SHORT); + if (StringUtils.isNotBlank(appToflowTableName)) { + hbaseConf.set(AppToFlowTable.TABLE_NAME_CONF_NAME, appToflowTableName); + } + // Grab the applicationTableName argument + String applicationTableName = commandLine.getOptionValue( + APP_TABLE_NAME_SHORT); + if (StringUtils.isNotBlank(applicationTableName)) { + hbaseConf.set(ApplicationTable.TABLE_NAME_CONF_NAME, + applicationTableName); + } + // Grab the application metrics TTL + String applicationTableMetricsTTL = commandLine.getOptionValue( + APP_METRICS_TTL_OPTION_SHORT); + if (StringUtils.isNotBlank(applicationTableMetricsTTL)) { + int appMetricsTTL = Integer.parseInt(applicationTableMetricsTTL); + new ApplicationTable().setMetricsTTL(appMetricsTTL, hbaseConf); + } + + // Grab the subApplicationTableName argument + String subApplicationTableName = commandLine.getOptionValue( + SUB_APP_TABLE_NAME_SHORT); + if (StringUtils.isNotBlank(subApplicationTableName)) { + hbaseConf.set(SubApplicationTable.TABLE_NAME_CONF_NAME, + subApplicationTableName); + } + // Grab the subApplication metrics TTL + String subApplicationTableMetricsTTL = commandLine + .getOptionValue(SUB_APP_METRICS_TTL_OPTION_SHORT); + if (StringUtils.isNotBlank(subApplicationTableMetricsTTL)) { + int subAppMetricsTTL = Integer.parseInt(subApplicationTableMetricsTTL); + new SubApplicationTable().setMetricsTTL(subAppMetricsTTL, hbaseConf); + } + + // create all table schemas in hbase + final boolean skipExisting = commandLine.hasOption( + SKIP_EXISTING_TABLE_OPTION_SHORT); + createAllSchemas(hbaseConf, skipExisting); + } else { + // print usage information if -create is not specified + printUsage(); + } + } + + /** + * Parse command-line arguments. + * + * @param args + * command line arguments passed to program. + * @return parsed command line. + * @throws ParseException + */ + private static CommandLine parseArgs(String[] args) throws ParseException { + Options options = new Options(); + + // Input + Option o = new Option(HELP_SHORT, "help", false, "print help information"); + o.setRequired(false); + options.addOption(o); + + o = new Option(CREATE_TABLES_SHORT, "create", false, + "a mandatory option to create hbase tables"); + o.setRequired(false); + options.addOption(o); + + o = new Option(ENTITY_TABLE_NAME_SHORT, "entityTableName", true, + "entity table name"); + o.setArgName("entityTableName"); + o.setRequired(false); + options.addOption(o); + + o = new Option(ENTITY_METRICS_TTL_OPTION_SHORT, "entityMetricsTTL", true, + "TTL for metrics column family"); + o.setArgName("entityMetricsTTL"); + o.setRequired(false); + options.addOption(o); + + o = new Option(APP_TO_FLOW_TABLE_NAME_SHORT, "appToflowTableName", true, + "app to flow table name"); + o.setArgName("appToflowTableName"); + o.setRequired(false); + options.addOption(o); + + o = new Option(APP_TABLE_NAME_SHORT, "applicationTableName", true, + "application table name"); + o.setArgName("applicationTableName"); + o.setRequired(false); + options.addOption(o); + + o = new Option(APP_METRICS_TTL_OPTION_SHORT, "applicationMetricsTTL", true, + "TTL for metrics column family"); + o.setArgName("applicationMetricsTTL"); + o.setRequired(false); + options.addOption(o); + + o = new Option(SUB_APP_TABLE_NAME_SHORT, "subApplicationTableName", true, + "subApplication table name"); + o.setArgName("subApplicationTableName"); + o.setRequired(false); + options.addOption(o); + + o = new Option(SUB_APP_METRICS_TTL_OPTION_SHORT, "subApplicationMetricsTTL", + true, "TTL for metrics column family"); + o.setArgName("subApplicationMetricsTTL"); + o.setRequired(false); + options.addOption(o); + + // Options without an argument + // No need to set arg name since we do not need an argument here + o = new Option(SKIP_EXISTING_TABLE_OPTION_SHORT, "skipExistingTable", + false, "skip existing Hbase tables and continue to create new tables"); + o.setRequired(false); + options.addOption(o); + + CommandLineParser parser = new PosixParser(); + CommandLine commandLine = null; + try { + commandLine = parser.parse(options, args); + } catch (Exception e) { + LOG.error("ERROR: " + e.getMessage() + "\n"); + HelpFormatter formatter = new HelpFormatter(); + formatter.printHelp(NAME + " ", options, true); + System.exit(-1); + } + + return commandLine; + } + + private static void printUsage() { + StringBuilder usage = new StringBuilder("Command Usage: \n"); + usage.append("TimelineSchemaCreator [-help] Display help info" + + " for all commands. Or\n"); + usage.append("TimelineSchemaCreator -create [OPTIONAL_OPTIONS]" + + " Create hbase tables.\n\n"); + usage.append("The Optional options for creating tables include: \n"); + usage.append("[-entityTableName ] " + + "The name of the Entity table\n"); + usage.append("[-entityMetricsTTL ]" + + " TTL for metrics in the Entity table\n"); + usage.append("[-appToflowTableName ]" + + " The name of the AppToFlow table\n"); + usage.append("[-applicationTableName ]" + + " The name of the Application table\n"); + usage.append("[-applicationMetricsTTL ]" + + " TTL for metrics in the Application table\n"); + usage.append("[-subApplicationTableName ]" + + " The name of the SubApplication table\n"); + usage.append("[-subApplicationMetricsTTL " + + " ]" + + " TTL for metrics in the SubApplication table\n"); + usage.append("[-skipExistingTable] Whether to skip existing" + + " hbase tables\n"); + System.out.println(usage.toString()); + } + + /** + * Create all table schemas and log success or exception if failed. + * @param hbaseConf the hbase configuration to create tables with + * @param skipExisting whether to skip existing hbase tables + */ + private static void createAllSchemas(Configuration hbaseConf, + boolean skipExisting) { + List exceptions = new ArrayList<>(); + try { + if (skipExisting) { + LOG.info("Will skip existing tables and continue on htable creation " + + "exceptions!"); + } + createAllTables(hbaseConf, skipExisting); + LOG.info("Successfully created HBase schema. "); + } catch (IOException e) { + LOG.error("Error in creating hbase tables: ", e); + exceptions.add(e); + } + + if (exceptions.size() > 0) { + LOG.warn("Schema creation finished with the following exceptions"); + for (Exception e : exceptions) { + LOG.warn(e.getMessage()); + } + System.exit(-1); + } else { + LOG.info("Schema creation finished successfully"); + } + } + + @VisibleForTesting + public static void createAllTables(Configuration hbaseConf, + boolean skipExisting) throws IOException { + + Connection conn = null; + try { + conn = ConnectionFactory.createConnection(hbaseConf); + Admin admin = conn.getAdmin(); + if (admin == null) { + throw new IOException("Cannot create table since admin is null"); + } + try { + new EntityTable().createTable(admin, hbaseConf); + } catch (IOException e) { + if (skipExisting) { + LOG.warn("Skip and continue on: " + e.getMessage()); + } else { + throw e; + } + } + try { + new AppToFlowTable().createTable(admin, hbaseConf); + } catch (IOException e) { + if (skipExisting) { + LOG.warn("Skip and continue on: " + e.getMessage()); + } else { + throw e; + } + } + try { + new ApplicationTable().createTable(admin, hbaseConf); + } catch (IOException e) { + if (skipExisting) { + LOG.warn("Skip and continue on: " + e.getMessage()); + } else { + throw e; + } + } + try { + new FlowRunTable().createTable(admin, hbaseConf); + } catch (IOException e) { + if (skipExisting) { + LOG.warn("Skip and continue on: " + e.getMessage()); + } else { + throw e; + } + } + try { + new FlowActivityTable().createTable(admin, hbaseConf); + } catch (IOException e) { + if (skipExisting) { + LOG.warn("Skip and continue on: " + e.getMessage()); + } else { + throw e; + } + } + try { + new SubApplicationTable().createTable(admin, hbaseConf); + } catch (IOException e) { + if (skipExisting) { + LOG.warn("Skip and continue on: " + e.getMessage()); + } else { + throw e; + } + } + } finally { + if (conn != null) { + conn.close(); + } + } + } + + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumn.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumn.java new file mode 100644 index 00000000000..00eaa7eb3fc --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumn.java @@ -0,0 +1,108 @@ +/** + * 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.timelineservice.storage.application; + +import java.io.IOException; + +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Column; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; + +/** + * Identifies fully qualified columns for the {@link ApplicationTable}. + */ +public enum ApplicationColumn implements Column { + + /** + * App id. + */ + ID(ApplicationColumnFamily.INFO, "id"), + + /** + * When the application was created. + */ + CREATED_TIME(ApplicationColumnFamily.INFO, "created_time", + new LongConverter()), + + /** + * The version of the flow that this app belongs to. + */ + FLOW_VERSION(ApplicationColumnFamily.INFO, "flow_version"); + + private final ColumnHelper column; + private final ColumnFamily columnFamily; + private final String columnQualifier; + private final byte[] columnQualifierBytes; + + private ApplicationColumn(ColumnFamily columnFamily, + String columnQualifier) { + this(columnFamily, columnQualifier, GenericConverter.getInstance()); + } + + private ApplicationColumn(ColumnFamily columnFamily, + String columnQualifier, ValueConverter converter) { + this.columnFamily = columnFamily; + this.columnQualifier = columnQualifier; + // Future-proof by ensuring the right column prefix hygiene. + this.columnQualifierBytes = + Bytes.toBytes(Separator.SPACE.encode(columnQualifier)); + this.column = new ColumnHelper(columnFamily, converter); + } + + /** + * @return the column name value + */ + private String getColumnQualifier() { + return columnQualifier; + } + + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, Long timestamp, + Object inputValue, Attribute... attributes) throws IOException { + column.store(rowKey, tableMutator, columnQualifierBytes, timestamp, + inputValue, attributes); + } + + public Object readResult(Result result) throws IOException { + return column.readResult(result, columnQualifierBytes); + } + + @Override + public byte[] getColumnQualifierBytes() { + return columnQualifierBytes.clone(); + } + + @Override + public byte[] getColumnFamilyBytes() { + return columnFamily.getBytes(); + } + + @Override + public ValueConverter getValueConverter() { + return column.getValueConverter(); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnFamily.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnFamily.java new file mode 100644 index 00000000000..97e5f7b56cc --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnFamily.java @@ -0,0 +1,65 @@ +/** + * 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.timelineservice.storage.application; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; + +/** + * Represents the application table column families. + */ +public enum ApplicationColumnFamily implements ColumnFamily { + + /** + * Info column family houses known columns, specifically ones included in + * columnfamily filters. + */ + INFO("i"), + + /** + * Configurations are in a separate column family for two reasons: a) the size + * of the config values can be very large and b) we expect that config values + * are often separately accessed from other metrics and info columns. + */ + CONFIGS("c"), + + /** + * Metrics have a separate column family, because they have a separate TTL. + */ + METRICS("m"); + + /** + * Byte representation of this column family. + */ + private final byte[] bytes; + + /** + * @param value create a column family with this name. Must be lower case and + * without spaces. + */ + private ApplicationColumnFamily(String value) { + // column families should be lower case and not contain any spaces. + this.bytes = Bytes.toBytes(Separator.SPACE.encode(value)); + } + + public byte[] getBytes() { + return Bytes.copy(bytes); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.java new file mode 100644 index 00000000000..8297dc5b762 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.java @@ -0,0 +1,236 @@ +/** + * 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.timelineservice.storage.application; + +import java.io.IOException; +import java.util.Map; +import java.util.NavigableMap; + +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; + +/** + * Identifies partially qualified columns for the application table. + */ +public enum ApplicationColumnPrefix implements ColumnPrefix { + + /** + * To store TimelineEntity getIsRelatedToEntities values. + */ + IS_RELATED_TO(ApplicationColumnFamily.INFO, "s"), + + /** + * To store TimelineEntity getRelatesToEntities values. + */ + RELATES_TO(ApplicationColumnFamily.INFO, "r"), + + /** + * To store TimelineEntity info values. + */ + INFO(ApplicationColumnFamily.INFO, "i"), + + /** + * Lifecycle events for an application. + */ + EVENT(ApplicationColumnFamily.INFO, "e"), + + /** + * Config column stores configuration with config key as the column name. + */ + CONFIG(ApplicationColumnFamily.CONFIGS, null), + + /** + * Metrics are stored with the metric name as the column name. + */ + METRIC(ApplicationColumnFamily.METRICS, null, new LongConverter()); + + private final ColumnHelper column; + private final ColumnFamily columnFamily; + + /** + * Can be null for those cases where the provided column qualifier is the + * entire column name. + */ + private final String columnPrefix; + private final byte[] columnPrefixBytes; + + /** + * Private constructor, meant to be used by the enum definition. + * + * @param columnFamily that this column is stored in. + * @param columnPrefix for this column. + */ + private ApplicationColumnPrefix(ColumnFamily columnFamily, + String columnPrefix) { + this(columnFamily, columnPrefix, GenericConverter.getInstance()); + } + + /** + * Private constructor, meant to be used by the enum definition. + * + * @param columnFamily that this column is stored in. + * @param columnPrefix for this column. + * @param converter used to encode/decode values to be stored in HBase for + * this column prefix. + */ + private ApplicationColumnPrefix(ColumnFamily columnFamily, + String columnPrefix, ValueConverter converter) { + column = new ColumnHelper(columnFamily, converter); + this.columnFamily = columnFamily; + this.columnPrefix = columnPrefix; + if (columnPrefix == null) { + this.columnPrefixBytes = null; + } else { + // Future-proof by ensuring the right column prefix hygiene. + this.columnPrefixBytes = + Bytes.toBytes(Separator.SPACE.encode(columnPrefix)); + } + } + + /** + * @return the column name value + */ + private String getColumnPrefix() { + return columnPrefix; + } + + @Override + public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) { + return ColumnHelper.getColumnQualifier( + this.columnPrefixBytes, qualifierPrefix); + } + + @Override + public byte[] getColumnPrefixBytes(String qualifierPrefix) { + return ColumnHelper.getColumnQualifier( + this.columnPrefixBytes, qualifierPrefix); + } + + @Override + public byte[] getColumnFamilyBytes() { + return columnFamily.getBytes(); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #store(byte[], + * org.apache.hadoop.yarn.server.timelineservice.storage.common. + * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object) + */ + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, byte[] qualifier, + Long timestamp, Object inputValue, Attribute... attributes) + throws IOException { + + // Null check + if (qualifier == null) { + throw new IOException("Cannot store column with null qualifier in " + + tableMutator.getName().getNameAsString()); + } + + byte[] columnQualifier = getColumnPrefixBytes(qualifier); + + column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue, + attributes); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #store(byte[], + * org.apache.hadoop.yarn.server.timelineservice.storage.common. + * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object) + */ + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, String qualifier, + Long timestamp, Object inputValue, Attribute...attributes) + throws IOException { + + // Null check + if (qualifier == null) { + throw new IOException("Cannot store column with null qualifier in " + + tableMutator.getName().getNameAsString()); + } + + byte[] columnQualifier = getColumnPrefixBytes(qualifier); + + column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue, + attributes); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResult(org.apache.hadoop.hbase.client.Result, java.lang.String) + */ + public Object readResult(Result result, String qualifier) throws IOException { + byte[] columnQualifier = + ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier); + return column.readResult(result, columnQualifier); + } + + public ValueConverter getValueConverter() { + return column.getValueConverter(); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResults(org.apache.hadoop.hbase.client.Result, + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter) + */ + public Map readResults(Result result, + KeyConverter keyConverter) throws IOException { + return column.readResults(result, columnPrefixBytes, keyConverter); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result, + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter) + */ + public NavigableMap> + readResultsWithTimestamps(Result result, KeyConverter keyConverter) + throws IOException { + return column.readResultsWithTimestamps(result, columnPrefixBytes, + keyConverter); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKey.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKey.java new file mode 100644 index 00000000000..e89a6a7433d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKey.java @@ -0,0 +1,251 @@ +/** + * 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.timelineservice.storage.application; + +import java.util.List; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.AppIdKeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverterToString; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; + +/** + * Represents a rowkey for the application table. + */ +public class ApplicationRowKey { + private final String clusterId; + private final String userId; + private final String flowName; + private final Long flowRunId; + private final String appId; + private final ApplicationRowKeyConverter appRowKeyConverter = + new ApplicationRowKeyConverter(); + + public ApplicationRowKey(String clusterId, String userId, String flowName, + Long flowRunId, String appId) { + this.clusterId = clusterId; + this.userId = userId; + this.flowName = flowName; + this.flowRunId = flowRunId; + this.appId = appId; + } + + public String getClusterId() { + return clusterId; + } + + public String getUserId() { + return userId; + } + + public String getFlowName() { + return flowName; + } + + public Long getFlowRunId() { + return flowRunId; + } + + public String getAppId() { + return appId; + } + + /** + * Constructs a row key for the application table as follows: + * {@code clusterId!userName!flowName!flowRunId!AppId}. + * + * @return byte array with the row key + */ + public byte[] getRowKey() { + return appRowKeyConverter.encode(this); + } + + /** + * Given the raw row key as bytes, returns the row key as an object. + * + * @param rowKey Byte representation of row key. + * @return An ApplicationRowKey object. + */ + public static ApplicationRowKey parseRowKey(byte[] rowKey) { + return new ApplicationRowKeyConverter().decode(rowKey); + } + + /** + * Constructs a row key for the application table as follows: + * {@code clusterId!userName!flowName!flowRunId!AppId}. + * @return String representation of row key. + */ + public String getRowKeyAsString() { + return appRowKeyConverter.encodeAsString(this); + } + + /** + * Given the encoded row key as string, returns the row key as an object. + * @param encodedRowKey String representation of row key. + * @return A ApplicationRowKey object. + */ + public static ApplicationRowKey parseRowKeyFromString(String encodedRowKey) { + return new ApplicationRowKeyConverter().decodeFromString(encodedRowKey); + } + + /** + * Encodes and decodes row key for application table. The row key is of the + * form: clusterId!userName!flowName!flowRunId!appId. flowRunId is a long, + * appId is encoded and decoded using {@link AppIdKeyConverter} and rest are + * strings. + *

+ */ + final private static class ApplicationRowKeyConverter implements + KeyConverter, KeyConverterToString { + + private final KeyConverter appIDKeyConverter = + new AppIdKeyConverter(); + + /** + * Intended for use in ApplicationRowKey only. + */ + private ApplicationRowKeyConverter() { + } + + /** + * Application row key is of the form + * clusterId!userName!flowName!flowRunId!appId with each segment separated + * by !. The sizes below indicate sizes of each one of these segements in + * sequence. clusterId, userName and flowName are strings. flowrunId is a + * long hence 8 bytes in size. app id is represented as 12 bytes with + * cluster timestamp part of appid takes 8 bytes(long) and seq id takes 4 + * bytes(int). Strings are variable in size (i.e. end whenever separator is + * encountered). This is used while decoding and helps in determining where + * to split. + */ + private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE, + Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, + AppIdKeyConverter.getKeySize() }; + + /* + * (non-Javadoc) + * + * Encodes ApplicationRowKey object into a byte array with each + * component/field in ApplicationRowKey separated by Separator#QUALIFIERS. + * This leads to an application table row key of the form + * clusterId!userName!flowName!flowRunId!appId If flowRunId in passed + * ApplicationRowKey object is null (and the fields preceding it i.e. + * clusterId, userId and flowName are not null), this returns a row key + * prefix of the form clusterId!userName!flowName! and if appId in + * ApplicationRowKey is null (other 4 components all are not null), this + * returns a row key prefix of the form + * clusterId!userName!flowName!flowRunId! flowRunId is inverted while + * encoding as it helps maintain a descending order for row keys in the + * application table. + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common + * .KeyConverter#encode(java.lang.Object) + */ + @Override + public byte[] encode(ApplicationRowKey rowKey) { + byte[] cluster = + Separator.encode(rowKey.getClusterId(), Separator.SPACE, + Separator.TAB, Separator.QUALIFIERS); + byte[] user = + Separator.encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB, + Separator.QUALIFIERS); + byte[] flow = + Separator.encode(rowKey.getFlowName(), Separator.SPACE, + Separator.TAB, Separator.QUALIFIERS); + byte[] first = Separator.QUALIFIERS.join(cluster, user, flow); + // Note that flowRunId is a long, so we can't encode them all at the same + // time. + if (rowKey.getFlowRunId() == null) { + return Separator.QUALIFIERS.join(first, Separator.EMPTY_BYTES); + } + byte[] second = + Bytes.toBytes(LongConverter.invertLong( + rowKey.getFlowRunId())); + if (rowKey.getAppId() == null || rowKey.getAppId().isEmpty()) { + return Separator.QUALIFIERS.join(first, second, Separator.EMPTY_BYTES); + } + byte[] third = appIDKeyConverter.encode(rowKey.getAppId()); + return Separator.QUALIFIERS.join(first, second, third); + } + + /* + * (non-Javadoc) + * + * Decodes an application row key of the form + * clusterId!userName!flowName!flowRunId!appId represented in byte format + * and converts it into an ApplicationRowKey object.flowRunId is inverted + * while decoding as it was inverted while encoding. + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common + * .KeyConverter#decode(byte[]) + */ + @Override + public ApplicationRowKey decode(byte[] rowKey) { + byte[][] rowKeyComponents = + Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES); + if (rowKeyComponents.length != 5) { + throw new IllegalArgumentException("the row key is not valid for " + + "an application"); + } + String clusterId = + Separator.decode(Bytes.toString(rowKeyComponents[0]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + String userId = + Separator.decode(Bytes.toString(rowKeyComponents[1]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + String flowName = + Separator.decode(Bytes.toString(rowKeyComponents[2]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + Long flowRunId = + LongConverter.invertLong(Bytes.toLong(rowKeyComponents[3])); + String appId = appIDKeyConverter.decode(rowKeyComponents[4]); + return new ApplicationRowKey(clusterId, userId, flowName, flowRunId, + appId); + } + + @Override + public String encodeAsString(ApplicationRowKey key) { + if (key.clusterId == null || key.userId == null || key.flowName == null + || key.flowRunId == null || key.appId == null) { + throw new IllegalArgumentException(); + } + return TimelineReaderUtils + .joinAndEscapeStrings(new String[] {key.clusterId, key.userId, + key.flowName, key.flowRunId.toString(), key.appId}); + } + + @Override + public ApplicationRowKey decodeFromString(String encodedRowKey) { + List split = TimelineReaderUtils.split(encodedRowKey); + if (split == null || split.size() != 5) { + throw new IllegalArgumentException( + "Invalid row key for application table."); + } + Long flowRunId = Long.valueOf(split.get(3)); + return new ApplicationRowKey(split.get(0), split.get(1), split.get(2), + flowRunId, split.get(4)); + } + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKeyPrefix.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKeyPrefix.java new file mode 100644 index 00000000000..f61b0e9cf8b --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKeyPrefix.java @@ -0,0 +1,69 @@ +/** + * 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.timelineservice.storage.application; + +import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix; + +/** + * Represents a partial rowkey (without flowName or without flowName and + * flowRunId) for the application table. + */ +public class ApplicationRowKeyPrefix extends ApplicationRowKey implements + RowKeyPrefix { + + /** + * Creates a prefix which generates the following rowKeyPrefixes for the + * application table: {@code clusterId!userName!flowName!}. + * + * @param clusterId the cluster on which applications ran + * @param userId the user that ran applications + * @param flowName the name of the flow that was run by the user on the + * cluster + */ + public ApplicationRowKeyPrefix(String clusterId, String userId, + String flowName) { + super(clusterId, userId, flowName, null, null); + } + + /** + * Creates a prefix which generates the following rowKeyPrefixes for the + * application table: {@code clusterId!userName!flowName!flowRunId!}. + * + * @param clusterId identifying the cluster + * @param userId identifying the user + * @param flowName identifying the flow + * @param flowRunId identifying the instance of this flow + */ + public ApplicationRowKeyPrefix(String clusterId, String userId, + String flowName, Long flowRunId) { + super(clusterId, userId, flowName, flowRunId, null); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.application. + * RowKeyPrefix#getRowKeyPrefix() + */ + @Override + public byte[] getRowKeyPrefix() { + return super.getRowKey(); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java new file mode 100644 index 00000000000..4da720e7a54 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java @@ -0,0 +1,170 @@ +/** + * 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.timelineservice.storage.application; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The application table as column families info, config and metrics. Info + * stores information about a YARN application entity, config stores + * configuration data of a YARN application, metrics stores the metrics of a + * YARN application. This table is entirely analogous to the entity table but + * created for better performance. + * + * Example application table record: + * + *

+ * |-------------------------------------------------------------------------|
+ * |  Row       | Column Family                | Column Family| Column Family|
+ * |  key       | info                         | metrics      | config       |
+ * |-------------------------------------------------------------------------|
+ * | clusterId! | id:appId                     | metricId1:   | configKey1:  |
+ * | userName!  |                              | metricValue1 | configValue1 |
+ * | flowName!  | created_time:                | @timestamp1  |              |
+ * | flowRunId! | 1392993084018                |              | configKey2:  |
+ * | AppId      |                              | metriciD1:   | configValue2 |
+ * |            | i!infoKey:                   | metricValue2 |              |
+ * |            | infoValue                    | @timestamp2  |              |
+ * |            |                              |              |              |
+ * |            | r!relatesToKey:              | metricId2:   |              |
+ * |            | id3=id4=id5                  | metricValue1 |              |
+ * |            |                              | @timestamp2  |              |
+ * |            | s!isRelatedToKey:            |              |              |
+ * |            | id7=id9=id6                  |              |              |
+ * |            |                              |              |              |
+ * |            | e!eventId=timestamp=infoKey: |              |              |
+ * |            | eventInfoValue               |              |              |
+ * |            |                              |              |              |
+ * |            | flowVersion:                 |              |              |
+ * |            | versionValue                 |              |              |
+ * |-------------------------------------------------------------------------|
+ * 
+ */ +public class ApplicationTable extends BaseTable { + /** application prefix. */ + private static final String PREFIX = + YarnConfiguration.TIMELINE_SERVICE_PREFIX + "application"; + + /** config param name that specifies the application table name. */ + public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; + + /** + * config param name that specifies the TTL for metrics column family in + * application table. + */ + private static final String METRICS_TTL_CONF_NAME = PREFIX + + ".table.metrics.ttl"; + + /** + * config param name that specifies max-versions for metrics column family in + * entity table. + */ + private static final String METRICS_MAX_VERSIONS = + PREFIX + ".table.metrics.max-versions"; + + /** default value for application table name. */ + private static final String DEFAULT_TABLE_NAME = + "timelineservice.application"; + + /** default TTL is 30 days for metrics timeseries. */ + private static final int DEFAULT_METRICS_TTL = 2592000; + + /** default max number of versions. */ + private static final int DEFAULT_METRICS_MAX_VERSIONS = 10000; + + private static final Logger LOG = + LoggerFactory.getLogger(ApplicationTable.class); + + public ApplicationTable() { + super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable + * (org.apache.hadoop.hbase.client.Admin, + * org.apache.hadoop.conf.Configuration) + */ + public void createTable(Admin admin, Configuration hbaseConf) + throws IOException { + + TableName table = getTableName(hbaseConf); + if (admin.tableExists(table)) { + // do not disable / delete existing table + // similar to the approach taken by map-reduce jobs when + // output directory exists + throw new IOException("Table " + table.getNameAsString() + + " already exists."); + } + + HTableDescriptor applicationTableDescp = new HTableDescriptor(table); + HColumnDescriptor infoCF = + new HColumnDescriptor(ApplicationColumnFamily.INFO.getBytes()); + infoCF.setBloomFilterType(BloomType.ROWCOL); + applicationTableDescp.addFamily(infoCF); + + HColumnDescriptor configCF = + new HColumnDescriptor(ApplicationColumnFamily.CONFIGS.getBytes()); + configCF.setBloomFilterType(BloomType.ROWCOL); + configCF.setBlockCacheEnabled(true); + applicationTableDescp.addFamily(configCF); + + HColumnDescriptor metricsCF = + new HColumnDescriptor(ApplicationColumnFamily.METRICS.getBytes()); + applicationTableDescp.addFamily(metricsCF); + metricsCF.setBlockCacheEnabled(true); + // always keep 1 version (the latest) + metricsCF.setMinVersions(1); + metricsCF.setMaxVersions( + hbaseConf.getInt(METRICS_MAX_VERSIONS, DEFAULT_METRICS_MAX_VERSIONS)); + metricsCF.setTimeToLive(hbaseConf.getInt(METRICS_TTL_CONF_NAME, + DEFAULT_METRICS_TTL)); + applicationTableDescp.setRegionSplitPolicyClassName( + "org.apache.hadoop.hbase.regionserver.KeyPrefixRegionSplitPolicy"); + applicationTableDescp.setValue("KeyPrefixRegionSplitPolicy.prefix_length", + TimelineHBaseSchemaConstants.USERNAME_SPLIT_KEY_PREFIX_LENGTH); + admin.createTable(applicationTableDescp, + TimelineHBaseSchemaConstants.getUsernameSplits()); + LOG.info("Status of table creation for " + table.getNameAsString() + "=" + + admin.tableExists(table)); + } + + /** + * @param metricsTTL time to live parameter for the metrics in this table. + * @param hbaseConf configuration in which to set the metrics TTL config + * variable. + */ + public void setMetricsTTL(int metricsTTL, Configuration hbaseConf) { + hbaseConf.setInt(METRICS_TTL_CONF_NAME, metricsTTL); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/package-info.java new file mode 100644 index 00000000000..03f508f1314 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/package-info.java @@ -0,0 +1,28 @@ +/* + * 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.timelineservice.storage.application + * contains classes related to implementation for application table. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.timelineservice.storage.application; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnFamily.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnFamily.java new file mode 100644 index 00000000000..f3f045e1a47 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnFamily.java @@ -0,0 +1,51 @@ +/** + * 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.timelineservice.storage.apptoflow; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; + +/** + * Represents the app_flow table column families. + */ +public enum AppToFlowColumnFamily implements ColumnFamily { + /** + * Mapping column family houses known columns such as flowName and flowRunId. + */ + MAPPING("m"); + + /** + * Byte representation of this column family. + */ + private final byte[] bytes; + + /** + * @param value create a column family with this name. Must be lower case and + * without spaces. + */ + AppToFlowColumnFamily(String value) { + // column families should be lower case and not contain any spaces. + this.bytes = Bytes.toBytes(Separator.SPACE.encode(value)); + } + + public byte[] getBytes() { + return Bytes.copy(bytes); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnPrefix.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnPrefix.java new file mode 100644 index 00000000000..f1e44956087 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnPrefix.java @@ -0,0 +1,206 @@ +/** + * 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.timelineservice.storage.apptoflow; + +import java.io.IOException; +import java.util.Map; +import java.util.NavigableMap; + +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; + +/** + * Identifies partially qualified columns for the app-to-flow table. + */ +public enum AppToFlowColumnPrefix implements ColumnPrefix { + + /** + * The flow name. + */ + FLOW_NAME(AppToFlowColumnFamily.MAPPING, "flow_name"), + + /** + * The flow run ID. + */ + FLOW_RUN_ID(AppToFlowColumnFamily.MAPPING, "flow_run_id"), + + /** + * The user. + */ + USER_ID(AppToFlowColumnFamily.MAPPING, "user_id"); + + private final ColumnHelper column; + private final ColumnFamily columnFamily; + private final String columnPrefix; + private final byte[] columnPrefixBytes; + + private AppToFlowColumnPrefix(ColumnFamily columnFamily, + String columnPrefix) { + this.columnFamily = columnFamily; + this.columnPrefix = columnPrefix; + if (columnPrefix == null) { + this.columnPrefixBytes = null; + } else { + // Future-proof by ensuring the right column prefix hygiene. + this.columnPrefixBytes = + Bytes.toBytes(Separator.SPACE.encode(columnPrefix)); + } + this.column = new ColumnHelper(columnFamily); + } + + @Override + public byte[] getColumnPrefixBytes(String qualifierPrefix) { + return ColumnHelper.getColumnQualifier( + columnPrefixBytes, qualifierPrefix); + } + + @Override + public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) { + return ColumnHelper.getColumnQualifier( + columnPrefixBytes, qualifierPrefix); + } + + @Override + public byte[] getColumnFamilyBytes() { + return columnFamily.getBytes(); + } + + @Override + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, byte[] qualifier, + Long timestamp, Object inputValue, Attribute... attributes) + throws IOException { + + // Null check + if (qualifier == null) { + throw new IOException("Cannot store column with null qualifier in " + + tableMutator.getName().getNameAsString()); + } + + byte[] columnQualifier = getColumnPrefixBytes(qualifier); + + column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue, + attributes); + } + + @Override + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, String qualifier, + Long timestamp, Object inputValue, Attribute... attributes) + throws IOException { + + // Null check + if (qualifier == null) { + throw new IOException("Cannot store column with null qualifier in " + + tableMutator.getName().getNameAsString()); + } + + byte[] columnQualifier = getColumnPrefixBytes(qualifier); + + column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue, + attributes); + } + + @Override + public ValueConverter getValueConverter() { + return column.getValueConverter(); + } + + @Override + public Object readResult(Result result, String qualifier) throws IOException { + byte[] columnQualifier = + ColumnHelper.getColumnQualifier(columnPrefixBytes, qualifier); + return column.readResult(result, columnQualifier); + } + + @Override + public Map readResults(Result result, + KeyConverter keyConverter) + throws IOException { + return column.readResults(result, columnPrefixBytes, keyConverter); + } + + @Override + public NavigableMap> + readResultsWithTimestamps(Result result, + KeyConverter keyConverter) throws IOException { + return column.readResultsWithTimestamps(result, columnPrefixBytes, + keyConverter); + } + + /** + * Retrieve an {@link AppToFlowColumnPrefix} given a name, or null if there + * is no match. The following holds true: {@code columnFor(x) == columnFor(y)} + * if and only if {@code x.equals(y)} or {@code (x == y == null)} + * + * @param columnPrefix Name of the column to retrieve + * @return the corresponding {@link AppToFlowColumnPrefix} or null + */ + public static final AppToFlowColumnPrefix columnFor(String columnPrefix) { + + // Match column based on value, assume column family matches. + for (AppToFlowColumnPrefix afcp : AppToFlowColumnPrefix.values()) { + // Find a match based only on name. + if (afcp.columnPrefix.equals(columnPrefix)) { + return afcp; + } + } + + // Default to null + return null; + } + + /** + * Retrieve an {@link AppToFlowColumnPrefix} given a name, or null if there + * is no match. The following holds true: + * {@code columnFor(a,x) == columnFor(b,y)} if and only if + * {@code (x == y == null)} or {@code a.equals(b) & x.equals(y)} + * + * @param columnFamily The columnFamily for which to retrieve the column. + * @param columnPrefix Name of the column to retrieve + * @return the corresponding {@link AppToFlowColumnPrefix} or null if both + * arguments don't match. + */ + public static final AppToFlowColumnPrefix columnFor( + AppToFlowColumnFamily columnFamily, String columnPrefix) { + + // TODO: needs unit test to confirm and need to update javadoc to explain + // null prefix case. + + for (AppToFlowColumnPrefix afcp : AppToFlowColumnPrefix.values()) { + // Find a match based column family and on name. + if (afcp.columnFamily.equals(columnFamily) + && (((columnPrefix == null) && (afcp.columnPrefix == null)) || + (afcp.columnPrefix.equals(columnPrefix)))) { + return afcp; + } + } + + // Default to null + return null; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowRowKey.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowRowKey.java new file mode 100644 index 00000000000..146c47520d4 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowRowKey.java @@ -0,0 +1,58 @@ +/** + * 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.timelineservice.storage.apptoflow; + +import org.apache.hadoop.yarn.server.timelineservice.storage.common.AppIdKeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; + +/** + * Represents a row key for the app_flow table, which is the app id. + */ +public class AppToFlowRowKey { + private final String appId; + private final KeyConverter appIdKeyConverter = + new AppIdKeyConverter(); + + public AppToFlowRowKey(String appId) { + this.appId = appId; + } + + public String getAppId() { + return appId; + } + + /** + * Constructs a row key prefix for the app_flow table. + * + * @return byte array with the row key + */ + public byte[] getRowKey() { + return appIdKeyConverter.encode(appId); + } + + /** + * Given the raw row key as bytes, returns the row key as an object. + * + * @param rowKey a rowkey represented as a byte array. + * @return an AppToFlowRowKey object. + */ + public static AppToFlowRowKey parseRowKey(byte[] rowKey) { + String appId = new AppIdKeyConverter().decode(rowKey); + return new AppToFlowRowKey(appId); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java new file mode 100644 index 00000000000..04da5c70d40 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java @@ -0,0 +1,125 @@ +/** + * 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.timelineservice.storage.apptoflow; + + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +/** + * The app_flow table as column families mapping. Mapping stores + * appId to flowName and flowRunId mapping information + * + * Example app_flow table record: + * + *
+ * |--------------------------------------|
+ * |  Row       | Column Family           |
+ * |  key       | mapping                 |
+ * |--------------------------------------|
+ * | appId      | flow_name!cluster1:     |
+ * |            | foo@daily_hive_report   |
+ * |            |                         |
+ * |            | flow_run_id!cluster1:   |
+ * |            | 1452828720457           |
+ * |            |                         |
+ * |            | user_id!cluster1:       |
+ * |            | admin                   |
+ * |            |                         |
+ * |            | flow_name!cluster2:     |
+ * |            | bar@ad_hoc_query        |
+ * |            |                         |
+ * |            | flow_run_id!cluster2:   |
+ * |            | 1452828498752           |
+ * |            |                         |
+ * |            | user_id!cluster2:       |
+ * |            | joe                     |
+ * |            |                         |
+ * |--------------------------------------|
+ * 
+ * + * It is possible (although unlikely) in a multi-cluster environment that there + * may be more than one applications for a given app id. Different clusters are + * recorded as different sets of columns. + */ +public class AppToFlowTable extends BaseTable { + /** app_flow prefix. */ + private static final String PREFIX = + YarnConfiguration.TIMELINE_SERVICE_PREFIX + "app-flow"; + + /** config param name that specifies the app_flow table name. */ + public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; + + /** default value for app_flow table name. */ + private static final String DEFAULT_TABLE_NAME = "timelineservice.app_flow"; + + private static final Logger LOG = + LoggerFactory.getLogger(AppToFlowTable.class); + + public AppToFlowTable() { + super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable + * (org.apache.hadoop.hbase.client.Admin, + * org.apache.hadoop.conf.Configuration) + */ + public void createTable(Admin admin, Configuration hbaseConf) + throws IOException { + + TableName table = getTableName(hbaseConf); + if (admin.tableExists(table)) { + // do not disable / delete existing table + // similar to the approach taken by map-reduce jobs when + // output directory exists + throw new IOException("Table " + table.getNameAsString() + + " already exists."); + } + + HTableDescriptor appToFlowTableDescp = new HTableDescriptor(table); + HColumnDescriptor mappCF = + new HColumnDescriptor(AppToFlowColumnFamily.MAPPING.getBytes()); + mappCF.setBloomFilterType(BloomType.ROWCOL); + appToFlowTableDescp.addFamily(mappCF); + + appToFlowTableDescp + .setRegionSplitPolicyClassName( + "org.apache.hadoop.hbase.regionserver.KeyPrefixRegionSplitPolicy"); + appToFlowTableDescp.setValue("KeyPrefixRegionSplitPolicy.prefix_length", + TimelineHBaseSchemaConstants.USERNAME_SPLIT_KEY_PREFIX_LENGTH); + admin.createTable(appToFlowTableDescp, + TimelineHBaseSchemaConstants.getUsernameSplits()); + LOG.info("Status of table creation for " + table.getNameAsString() + "=" + + admin.tableExists(table)); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/package-info.java new file mode 100644 index 00000000000..f01d9826040 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/package-info.java @@ -0,0 +1,28 @@ +/* + * 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.timelineservice.storage.apptoflow + * contains classes related to implementation for app to flow table. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java new file mode 100644 index 00000000000..51604f012cc --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java @@ -0,0 +1,97 @@ +/** + * 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.timelineservice.storage.common; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.api.records.ApplicationId; + +/** + * Encodes and decodes {@link ApplicationId} for row keys. + * App ID is stored in row key as 12 bytes, cluster timestamp section of app id + * (long - 8 bytes) followed by sequence id section of app id (int - 4 bytes). + */ +public final class AppIdKeyConverter implements KeyConverter { + + public AppIdKeyConverter() { + } + + /* + * (non-Javadoc) + * + * Converts/encodes a string app Id into a byte representation for (row) keys. + * For conversion, we extract cluster timestamp and sequence id from the + * string app id (calls ConverterUtils#toApplicationId(String) for + * conversion) and then store it in a byte array of length 12 (8 bytes (long) + * for cluster timestamp followed 4 bytes(int) for sequence id). Both cluster + * timestamp and sequence id are inverted so that the most recent cluster + * timestamp and highest sequence id appears first in the table (i.e. + * application id appears in a descending order). + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter + * #encode(java.lang.Object) + */ + @Override + public byte[] encode(String appIdStr) { + ApplicationId appId = ApplicationId.fromString(appIdStr); + byte[] appIdBytes = new byte[getKeySize()]; + byte[] clusterTs = Bytes.toBytes( + LongConverter.invertLong(appId.getClusterTimestamp())); + System.arraycopy(clusterTs, 0, appIdBytes, 0, Bytes.SIZEOF_LONG); + byte[] seqId = Bytes.toBytes( + HBaseTimelineStorageUtils.invertInt(appId.getId())); + System.arraycopy(seqId, 0, appIdBytes, Bytes.SIZEOF_LONG, Bytes.SIZEOF_INT); + return appIdBytes; + } + + /* + * (non-Javadoc) + * + * Converts/decodes a 12 byte representation of app id for (row) keys to an + * app id in string format which can be returned back to client. + * For decoding, 12 bytes are interpreted as 8 bytes of inverted cluster + * timestamp(long) followed by 4 bytes of inverted sequence id(int). Calls + * ApplicationId#toString to generate string representation of app id. + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter + * #decode(byte[]) + */ + @Override + public String decode(byte[] appIdBytes) { + if (appIdBytes.length != getKeySize()) { + throw new IllegalArgumentException("Invalid app id in byte format"); + } + long clusterTs = LongConverter.invertLong( + Bytes.toLong(appIdBytes, 0, Bytes.SIZEOF_LONG)); + int seqId = HBaseTimelineStorageUtils.invertInt( + Bytes.toInt(appIdBytes, Bytes.SIZEOF_LONG, Bytes.SIZEOF_INT)); + return HBaseTimelineStorageUtils.convertApplicationIdToString( + ApplicationId.newInstance(clusterTs, seqId)); + } + + /** + * Returns the size of app id after encoding. + * + * @return size of app id after encoding. + */ + public static int getKeySize() { + return Bytes.SIZEOF_LONG + Bytes.SIZEOF_INT; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BaseTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BaseTable.java new file mode 100644 index 00000000000..93d809c003e --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BaseTable.java @@ -0,0 +1,167 @@ +/** + * 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.timelineservice.storage.common; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.BufferedMutator; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.yarn.conf.YarnConfiguration; + +/** + * Implements behavior common to tables used in the timeline service storage. It + * is thread-safe, and can be used by multiple threads concurrently. + * + * @param reference to the table instance class itself for type safety. + */ +public abstract class BaseTable { + + /** + * Name of config variable that is used to point to this table. + */ + private final String tableNameConfName; + + /** + * Unless the configuration overrides, this will be the default name for the + * table when it is created. + */ + private final String defaultTableName; + + /** + * @param tableNameConfName name of config variable that is used to point to + * this table. + * @param defaultTableName Default table name if table from config is not + * found. + */ + protected BaseTable(String tableNameConfName, String defaultTableName) { + this.tableNameConfName = tableNameConfName; + this.defaultTableName = defaultTableName; + } + + /** + * Used to create a type-safe mutator for this table. + * + * @param hbaseConf used to read table name. + * @param conn used to create a table from. + * @return a type safe {@link BufferedMutator} for the entity table. + * @throws IOException if any exception occurs while creating mutator for the + * table. + */ + public TypedBufferedMutator getTableMutator(Configuration hbaseConf, + Connection conn) throws IOException { + + TableName tableName = this.getTableName(hbaseConf); + + // Plain buffered mutator + BufferedMutator bufferedMutator = conn.getBufferedMutator(tableName); + + // Now make this thing type safe. + // This is how service initialization should hang on to this variable, with + // the proper type + TypedBufferedMutator table = + new BufferedMutatorDelegator(bufferedMutator); + + return table; + } + + /** + * @param hbaseConf used to read settings that override defaults + * @param conn used to create table from + * @param scan that specifies what you want to read from this table. + * @return scanner for the table. + * @throws IOException if any exception occurs while getting the scanner. + */ + public ResultScanner getResultScanner(Configuration hbaseConf, + Connection conn, Scan scan) throws IOException { + Table table = conn.getTable(getTableName(hbaseConf)); + return table.getScanner(scan); + } + + /** + * + * @param hbaseConf used to read settings that override defaults + * @param conn used to create table from + * @param get that specifies what single row you want to get from this table + * @return result of get operation + * @throws IOException if any exception occurs while getting the result. + */ + public Result getResult(Configuration hbaseConf, Connection conn, Get get) + throws IOException { + Table table = conn.getTable(getTableName(hbaseConf)); + return table.get(get); + } + + /** + * Get the table name for the input table. + * + * @param conf HBase configuration from which table name will be fetched. + * @param tableName name of the table to be fetched + * @return A {@link TableName} object. + */ + public static TableName getTableName(Configuration conf, String tableName) { + String tableSchemaPrefix = conf.get( + YarnConfiguration.TIMELINE_SERVICE_HBASE_SCHEMA_PREFIX_NAME, + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_HBASE_SCHEMA_PREFIX); + return TableName.valueOf(tableSchemaPrefix + tableName); + } + + /** + * Get the table name for this table. + * + * @param conf HBase configuration from which table name will be fetched. + * @return A {@link TableName} object. + */ + public TableName getTableName(Configuration conf) { + String tableName = conf.get(tableNameConfName, defaultTableName); + return getTableName(conf, tableName); + } + + /** + * Get the table name based on the input config parameters. + * + * @param conf HBase configuration from which table name will be fetched. + * @param tableNameInConf the table name parameter in conf. + * @param defaultTableName the default table name. + * @return A {@link TableName} object. + */ + public static TableName getTableName(Configuration conf, + String tableNameInConf, String defaultTableName) { + String tableName = conf.get(tableNameInConf, defaultTableName); + return getTableName(conf, tableName); + } + + /** + * Used to create the table in HBase. Should be called only once (per HBase + * instance). + * + * @param admin Used for doing HBase table operations. + * @param hbaseConf Hbase configuration. + * @throws IOException if any exception occurs while creating the table. + */ + public abstract void createTable(Admin admin, Configuration hbaseConf) + throws IOException; + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BufferedMutatorDelegator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BufferedMutatorDelegator.java new file mode 100644 index 00000000000..cf469a54282 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BufferedMutatorDelegator.java @@ -0,0 +1,73 @@ +/** + * 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.timelineservice.storage.common; + +import java.io.IOException; +import java.util.List; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.BufferedMutator; +import org.apache.hadoop.hbase.client.Mutation; + +/** + * To be used to wrap an actual {@link BufferedMutator} in a type safe manner. + * + * @param The class referring to the table to be written to. + */ +class BufferedMutatorDelegator implements TypedBufferedMutator { + + private final BufferedMutator bufferedMutator; + + /** + * @param bufferedMutator the mutator to be wrapped for delegation. Shall not + * be null. + */ + public BufferedMutatorDelegator(BufferedMutator bufferedMutator) { + this.bufferedMutator = bufferedMutator; + } + + public TableName getName() { + return bufferedMutator.getName(); + } + + public Configuration getConfiguration() { + return bufferedMutator.getConfiguration(); + } + + public void mutate(Mutation mutation) throws IOException { + bufferedMutator.mutate(mutation); + } + + public void mutate(List mutations) throws IOException { + bufferedMutator.mutate(mutations); + } + + public void close() throws IOException { + bufferedMutator.close(); + } + + public void flush() throws IOException { + bufferedMutator.flush(); + } + + public long getWriteBufferSize() { + return bufferedMutator.getWriteBufferSize(); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Column.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Column.java new file mode 100644 index 00000000000..90f2de4ef3f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Column.java @@ -0,0 +1,80 @@ +/** + * 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.timelineservice.storage.common; + +import java.io.IOException; + +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; + +/** + * A Column represents the way to store a fully qualified column in a specific + * table. + */ +public interface Column { + + /** + * Sends a Mutation to the table. The mutations will be buffered and sent over + * the wire as part of a batch. + * + * @param rowKey identifying the row to write. Nothing gets written when null. + * @param tableMutator used to modify the underlying HBase table. Caller is + * responsible to pass a mutator for the table that actually has this + * column. + * @param timestamp version timestamp. When null the server timestamp will be + * used. + * @param attributes Map of attributes for this mutation. used in the + * coprocessor to set/read the cell tags. Can be null. + * @param inputValue the value to write to the rowKey and column qualifier. + * Nothing gets written when null. + * @throws IOException if there is any exception encountered during store. + */ + void store(byte[] rowKey, TypedBufferedMutator tableMutator, + Long timestamp, Object inputValue, Attribute... attributes) + throws IOException; + + /** + * Get the latest version of this specified column. Note: this call clones the + * value content of the hosting {@link org.apache.hadoop.hbase.Cell Cell}. + * + * @param result Cannot be null + * @return result object (can be cast to whatever object was written to), or + * null when result doesn't contain this column. + * @throws IOException if there is any exception encountered while reading + * result. + */ + Object readResult(Result result) throws IOException; + + /** + * Returns column family name(as bytes) associated with this column. + * @return a byte array encoding column family for this column qualifier. + */ + byte[] getColumnFamilyBytes(); + + /** + * Get byte representation for this column qualifier. + * @return a byte array representing column qualifier. + */ + byte[] getColumnQualifierBytes(); + + /** + * Returns value converter implementation associated with this column. + * @return a {@link ValueConverter} implementation. + */ + ValueConverter getValueConverter(); +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnFamily.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnFamily.java new file mode 100644 index 00000000000..452adcdcb4a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnFamily.java @@ -0,0 +1,34 @@ +/** + * 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.timelineservice.storage.common; + +/** + * Type safe column family. + * + * @param refers to the table for which this column family is used for. + */ +public interface ColumnFamily { + + /** + * Keep a local copy if you need to avoid overhead of repeated cloning. + * + * @return a clone of the byte representation of the column family. + */ + byte[] getBytes(); + +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java new file mode 100644 index 00000000000..9f95d445271 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java @@ -0,0 +1,414 @@ +/** + * 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.timelineservice.storage.common; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; +import java.util.NavigableMap; +import java.util.TreeMap; + +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationCompactionDimension; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class is meant to be used only by explicit Columns, and not directly to + * write by clients. + * + * @param refers to the table. + */ +public class ColumnHelper { + private static final Logger LOG = + LoggerFactory.getLogger(ColumnHelper.class); + + private final ColumnFamily columnFamily; + + /** + * Local copy of bytes representation of columnFamily so that we can avoid + * cloning a new copy over and over. + */ + private final byte[] columnFamilyBytes; + + private final ValueConverter converter; + + private final boolean supplementTs; + + public ColumnHelper(ColumnFamily columnFamily) { + this(columnFamily, GenericConverter.getInstance()); + } + + public ColumnHelper(ColumnFamily columnFamily, ValueConverter converter) { + this(columnFamily, converter, false); + } + + /** + * @param columnFamily column family implementation. + * @param converter converter use to encode/decode values stored in the column + * or column prefix. + * @param needSupplementTs flag to indicate if cell timestamp needs to be + * modified for this column by calling + * {@link TimestampGenerator#getSupplementedTimestamp(long, String)}. This + * would be required for columns(such as metrics in flow run table) where + * potential collisions can occur due to same timestamp. + */ + public ColumnHelper(ColumnFamily columnFamily, ValueConverter converter, + boolean needSupplementTs) { + this.columnFamily = columnFamily; + columnFamilyBytes = columnFamily.getBytes(); + if (converter == null) { + this.converter = GenericConverter.getInstance(); + } else { + this.converter = converter; + } + this.supplementTs = needSupplementTs; + } + + /** + * Sends a Mutation to the table. The mutations will be buffered and sent over + * the wire as part of a batch. + * + * @param rowKey + * identifying the row to write. Nothing gets written when null. + * @param tableMutator + * used to modify the underlying HBase table + * @param columnQualifier + * column qualifier. Nothing gets written when null. + * @param timestamp + * version timestamp. When null the current timestamp multiplied with + * TimestampGenerator.TS_MULTIPLIER and added with last 3 digits of + * app id will be used + * @param inputValue + * the value to write to the rowKey and column qualifier. Nothing + * gets written when null. + * @param attributes Attributes to be set for HBase Put. + * @throws IOException if any problem occurs during store operation(sending + * mutation to table). + */ + public void store(byte[] rowKey, TypedBufferedMutator tableMutator, + byte[] columnQualifier, Long timestamp, Object inputValue, + Attribute... attributes) throws IOException { + if ((rowKey == null) || (columnQualifier == null) || (inputValue == null)) { + return; + } + Put p = new Put(rowKey); + timestamp = getPutTimestamp(timestamp, attributes); + p.addColumn(columnFamilyBytes, columnQualifier, timestamp, + converter.encodeValue(inputValue)); + if ((attributes != null) && (attributes.length > 0)) { + for (Attribute attribute : attributes) { + p.setAttribute(attribute.getName(), attribute.getValue()); + } + } + tableMutator.mutate(p); + } + + /* + * Figures out the cell timestamp used in the Put For storing. + * Will supplement the timestamp if required. Typically done for flow run + * table.If we supplement the timestamp, we left shift the timestamp and + * supplement it with the AppId id so that there are no collisions in the flow + * run table's cells. + */ + private long getPutTimestamp(Long timestamp, Attribute[] attributes) { + if (timestamp == null) { + timestamp = System.currentTimeMillis(); + } + if (!this.supplementTs) { + return timestamp; + } else { + String appId = getAppIdFromAttributes(attributes); + long supplementedTS = TimestampGenerator.getSupplementedTimestamp( + timestamp, appId); + return supplementedTS; + } + } + + private String getAppIdFromAttributes(Attribute[] attributes) { + if (attributes == null) { + return null; + } + String appId = null; + for (Attribute attribute : attributes) { + if (AggregationCompactionDimension.APPLICATION_ID.toString().equals( + attribute.getName())) { + appId = Bytes.toString(attribute.getValue()); + } + } + return appId; + } + + /** + * @return the column family for this column implementation. + */ + public ColumnFamily getColumnFamily() { + return columnFamily; + } + + /** + * Get the latest version of this specified column. Note: this call clones the + * value content of the hosting {@link org.apache.hadoop.hbase.Cell Cell}. + * + * @param result from which to read the value. Cannot be null + * @param columnQualifierBytes referring to the column to be read. + * @return latest version of the specified column of whichever object was + * written. + * @throws IOException if any problem occurs while reading result. + */ + public Object readResult(Result result, byte[] columnQualifierBytes) + throws IOException { + if (result == null || columnQualifierBytes == null) { + return null; + } + + // Would have preferred to be able to use getValueAsByteBuffer and get a + // ByteBuffer to avoid copy, but GenericObjectMapper doesn't seem to like + // that. + byte[] value = result.getValue(columnFamilyBytes, columnQualifierBytes); + return converter.decodeValue(value); + } + + /** + * @param result from which to reads data with timestamps + * @param columnPrefixBytes optional prefix to limit columns. If null all + * columns are returned. + * @param identifies the type of column name(indicated by type of key + * converter). + * @param the type of the values. The values will be cast into that type. + * @param keyConverter used to convert column bytes to the appropriate key + * type. + * @return the cell values at each respective time in for form + * {@literal {idA={timestamp1->value1}, idA={timestamp2->value2}, + * idB={timestamp3->value3}, idC={timestamp1->value4}}} + * @throws IOException if any problem occurs while reading results. + */ + @SuppressWarnings("unchecked") + public NavigableMap> + readResultsWithTimestamps(Result result, byte[] columnPrefixBytes, + KeyConverter keyConverter) throws IOException { + + NavigableMap> results = new TreeMap<>(); + + if (result != null) { + NavigableMap< + byte[], NavigableMap>> resultMap = + result.getMap(); + + NavigableMap> columnCellMap = + resultMap.get(columnFamilyBytes); + // could be that there is no such column family. + if (columnCellMap != null) { + for (Entry> entry : columnCellMap + .entrySet()) { + K converterColumnKey = null; + if (columnPrefixBytes == null) { + if (LOG.isDebugEnabled()) { + LOG.debug("null prefix was specified; returning all columns"); + } + try { + converterColumnKey = keyConverter.decode(entry.getKey()); + } catch (IllegalArgumentException iae) { + LOG.error("Illegal column found, skipping this column.", iae); + continue; + } + } else { + // A non-null prefix means columns are actually of the form + // prefix!columnNameRemainder + byte[][] columnNameParts = + Separator.QUALIFIERS.split(entry.getKey(), 2); + byte[] actualColumnPrefixBytes = columnNameParts[0]; + if (Bytes.equals(columnPrefixBytes, actualColumnPrefixBytes) + && columnNameParts.length == 2) { + try { + // This is the prefix that we want + converterColumnKey = keyConverter.decode(columnNameParts[1]); + } catch (IllegalArgumentException iae) { + LOG.error("Illegal column found, skipping this column.", iae); + continue; + } + } + } + + // If this column has the prefix we want + if (converterColumnKey != null) { + NavigableMap cellResults = + new TreeMap(); + NavigableMap cells = entry.getValue(); + if (cells != null) { + for (Entry cell : cells.entrySet()) { + V value = + (V) converter.decodeValue(cell.getValue()); + Long ts = supplementTs ? TimestampGenerator. + getTruncatedTimestamp(cell.getKey()) : cell.getKey(); + cellResults.put(ts, value); + } + } + results.put(converterColumnKey, cellResults); + } + } // for entry : columnCellMap + } // if columnCellMap != null + } // if result != null + return results; + } + + /** + * @param identifies the type of column name(indicated by type of key + * converter). + * @param result from which to read columns + * @param columnPrefixBytes optional prefix to limit columns. If null all + * columns are returned. + * @param keyConverter used to convert column bytes to the appropriate key + * type. + * @return the latest values of columns in the column family. If the column + * prefix is null, the column qualifier is returned as Strings. For a + * non-null column prefix bytes, the column qualifier is returned as + * a list of parts, each part a byte[]. This is to facilitate + * returning byte arrays of values that were not Strings. + * @throws IOException if any problem occurs while reading results. + */ + public Map readResults(Result result, + byte[] columnPrefixBytes, KeyConverter keyConverter) + throws IOException { + Map results = new HashMap(); + + if (result != null) { + Map columns = result.getFamilyMap(columnFamilyBytes); + for (Entry entry : columns.entrySet()) { + byte[] columnKey = entry.getKey(); + if (columnKey != null && columnKey.length > 0) { + + K converterColumnKey = null; + if (columnPrefixBytes == null) { + try { + converterColumnKey = keyConverter.decode(columnKey); + } catch (IllegalArgumentException iae) { + LOG.error("Illegal column found, skipping this column.", iae); + continue; + } + } else { + // A non-null prefix means columns are actually of the form + // prefix!columnNameRemainder + byte[][] columnNameParts = Separator.QUALIFIERS.split(columnKey, 2); + if (columnNameParts.length > 0) { + byte[] actualColumnPrefixBytes = columnNameParts[0]; + // If this is the prefix that we want + if (Bytes.equals(columnPrefixBytes, actualColumnPrefixBytes) + && columnNameParts.length == 2) { + try { + converterColumnKey = keyConverter.decode(columnNameParts[1]); + } catch (IllegalArgumentException iae) { + LOG.error("Illegal column found, skipping this column.", iae); + continue; + } + } + } + } // if-else + + // If the columnPrefix is null (we want all columns), or the actual + // prefix matches the given prefix we want this column + if (converterColumnKey != null) { + Object value = converter.decodeValue(entry.getValue()); + // we return the columnQualifier in parts since we don't know + // which part is of which data type. + results.put(converterColumnKey, value); + } + } + } // for entry + } + return results; + } + + /** + * @param columnPrefixBytes The byte representation for the column prefix. + * Should not contain {@link Separator#QUALIFIERS}. + * @param qualifier for the remainder of the column. + * {@link Separator#QUALIFIERS} is permissible in the qualifier + * as it is joined only with the column prefix bytes. + * @return fully sanitized column qualifier that is a combination of prefix + * and qualifier. If prefix is null, the result is simply the encoded + * qualifier without any separator. + */ + public static byte[] getColumnQualifier(byte[] columnPrefixBytes, + String qualifier) { + + // We don't want column names to have spaces / tabs. + byte[] encodedQualifier = + Separator.encode(qualifier, Separator.SPACE, Separator.TAB); + if (columnPrefixBytes == null) { + return encodedQualifier; + } + + // Convert qualifier to lower case, strip of separators and tag on column + // prefix. + byte[] columnQualifier = + Separator.QUALIFIERS.join(columnPrefixBytes, encodedQualifier); + return columnQualifier; + } + + /** + * @param columnPrefixBytes The byte representation for the column prefix. + * Should not contain {@link Separator#QUALIFIERS}. + * @param qualifier for the remainder of the column. + * @return fully sanitized column qualifier that is a combination of prefix + * and qualifier. If prefix is null, the result is simply the encoded + * qualifier without any separator. + */ + public static byte[] getColumnQualifier(byte[] columnPrefixBytes, + long qualifier) { + + if (columnPrefixBytes == null) { + return Bytes.toBytes(qualifier); + } + + // Convert qualifier to lower case, strip of separators and tag on column + // prefix. + byte[] columnQualifier = + Separator.QUALIFIERS.join(columnPrefixBytes, Bytes.toBytes(qualifier)); + return columnQualifier; + } + + public ValueConverter getValueConverter() { + return converter; + } + + /** + * @param columnPrefixBytes The byte representation for the column prefix. + * Should not contain {@link Separator#QUALIFIERS}. + * @param qualifier the byte representation for the remainder of the column. + * @return fully sanitized column qualifier that is a combination of prefix + * and qualifier. If prefix is null, the result is simply the encoded + * qualifier without any separator. + */ + public static byte[] getColumnQualifier(byte[] columnPrefixBytes, + byte[] qualifier) { + + if (columnPrefixBytes == null) { + return qualifier; + } + + byte[] columnQualifier = + Separator.QUALIFIERS.join(columnPrefixBytes, qualifier); + return columnQualifier; + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnPrefix.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnPrefix.java new file mode 100644 index 00000000000..89aa013a1be --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnPrefix.java @@ -0,0 +1,145 @@ +/** + * 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.timelineservice.storage.common; + +import java.io.IOException; +import java.util.Map; +import java.util.NavigableMap; + +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; + +/** + * Used to represent a partially qualified column, where the actual column name + * will be composed of a prefix and the remainder of the column qualifier. The + * prefix can be null, in which case the column qualifier will be completely + * determined when the values are stored. + */ +public interface ColumnPrefix { + + /** + * Sends a Mutation to the table. The mutations will be buffered and sent over + * the wire as part of a batch. + * + * @param rowKey identifying the row to write. Nothing gets written when null. + * @param tableMutator used to modify the underlying HBase table. Caller is + * responsible to pass a mutator for the table that actually has this + * column. + * @param qualifier column qualifier. Nothing gets written when null. + * @param timestamp version timestamp. When null the server timestamp will be + * used. + * @param attributes attributes for the mutation that are used by the + * coprocessor to set/read the cell tags. + * @param inputValue the value to write to the rowKey and column qualifier. + * Nothing gets written when null. + * @throws IOException if there is any exception encountered while doing + * store operation(sending mutation to the table). + */ + void store(byte[] rowKey, TypedBufferedMutator tableMutator, + byte[] qualifier, Long timestamp, Object inputValue, + Attribute... attributes) throws IOException; + + /** + * Sends a Mutation to the table. The mutations will be buffered and sent over + * the wire as part of a batch. + * + * @param rowKey identifying the row to write. Nothing gets written when null. + * @param tableMutator used to modify the underlying HBase table. Caller is + * responsible to pass a mutator for the table that actually has this + * column. + * @param qualifier column qualifier. Nothing gets written when null. + * @param timestamp version timestamp. When null the server timestamp will be + * used. + * @param attributes attributes for the mutation that are used by the + * coprocessor to set/read the cell tags. + * @param inputValue the value to write to the rowKey and column qualifier. + * Nothing gets written when null. + * @throws IOException if there is any exception encountered while doing + * store operation(sending mutation to the table). + */ + void store(byte[] rowKey, TypedBufferedMutator tableMutator, + String qualifier, Long timestamp, Object inputValue, + Attribute... attributes) throws IOException; + + /** + * Get the latest version of this specified column. Note: this call clones the + * value content of the hosting {@link org.apache.hadoop.hbase.Cell Cell}. + * + * @param result Cannot be null + * @param qualifier column qualifier. Nothing gets read when null. + * @return result object (can be cast to whatever object was written to) or + * null when specified column qualifier for this prefix doesn't exist + * in the result. + * @throws IOException if there is any exception encountered while reading + * result. + */ + Object readResult(Result result, String qualifier) throws IOException; + + /** + * + * @param identifies the type of key converter. + * @param result from which to read columns. + * @param keyConverter used to convert column bytes to the appropriate key + * type + * @return the latest values of columns in the column family with this prefix + * (or all of them if the prefix value is null). + * @throws IOException if there is any exception encountered while reading + * results. + */ + Map readResults(Result result, KeyConverter keyConverter) + throws IOException; + + /** + * @param result from which to reads data with timestamps. + * @param identifies the type of key converter. + * @param the type of the values. The values will be cast into that type. + * @param keyConverter used to convert column bytes to the appropriate key + * type. + * @return the cell values at each respective time in for form + * {@literal {idA={timestamp1->value1}, idA={timestamp2->value2}, + * idB={timestamp3->value3}, idC={timestamp1->value4}}} + * @throws IOException if there is any exception encountered while reading + * result. + */ + NavigableMap> readResultsWithTimestamps( + Result result, KeyConverter keyConverter) throws IOException; + + /** + * @param qualifierPrefix Column qualifier or prefix of qualifier. + * @return a byte array encoding column prefix and qualifier/prefix passed. + */ + byte[] getColumnPrefixBytes(String qualifierPrefix); + + /** + * @param qualifierPrefix Column qualifier or prefix of qualifier. + * @return a byte array encoding column prefix and qualifier/prefix passed. + */ + byte[] getColumnPrefixBytes(byte[] qualifierPrefix); + + /** + * Returns column family name(as bytes) associated with this column prefix. + * @return a byte array encoding column family for this prefix. + */ + byte[] getColumnFamilyBytes(); + + /** + * Returns value converter implementation associated with this column prefix. + * @return a {@link ValueConverter} implementation. + */ + ValueConverter getValueConverter(); +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnName.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnName.java new file mode 100644 index 00000000000..84455757a4a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnName.java @@ -0,0 +1,63 @@ +/** + * 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.timelineservice.storage.common; + +/** + * Encapsulates information about Event column names for application and entity + * tables. Used while encoding/decoding event column names. + */ +public class EventColumnName { + + private final String id; + private final Long timestamp; + private final String infoKey; + private final KeyConverter eventColumnNameConverter = + new EventColumnNameConverter(); + + public EventColumnName(String id, Long timestamp, String infoKey) { + this.id = id; + this.timestamp = timestamp; + this.infoKey = infoKey; + } + + public String getId() { + return id; + } + + public Long getTimestamp() { + return timestamp; + } + + public String getInfoKey() { + return infoKey; + } + + /** + * @return a byte array with each components/fields separated by + * Separator#VALUES. This leads to an event column name of the form + * eventId=timestamp=infokey. If both timestamp and infokey are null, + * then a qualifier of the form eventId=timestamp= is returned. If + * only infokey is null, then a qualifier of the form eventId= is + * returned. These prefix forms are useful for queries that intend to + * retrieve more than one specific column name. + */ + public byte[] getColumnQualifier() { + return eventColumnNameConverter.encode(this); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnNameConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnNameConverter.java new file mode 100644 index 00000000000..d3ef89749b9 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnNameConverter.java @@ -0,0 +1,99 @@ +/** + * 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.timelineservice.storage.common; + +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Encodes and decodes event column names for application and entity tables. + * The event column name is of the form : eventId=timestamp=infokey. + * If info is not associated with the event, event column name is of the form : + * eventId=timestamp= + * Event timestamp is long and rest are strings. + * Column prefixes are not part of the eventcolumn name passed for encoding. It + * is added later, if required in the associated ColumnPrefix implementations. + */ +public final class EventColumnNameConverter + implements KeyConverter { + + public EventColumnNameConverter() { + } + + // eventId=timestamp=infokey are of types String, Long String + // Strings are variable in size (i.e. end whenever separator is encountered). + // This is used while decoding and helps in determining where to split. + private static final int[] SEGMENT_SIZES = { + Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, Separator.VARIABLE_SIZE }; + + /* + * (non-Javadoc) + * + * Encodes EventColumnName into a byte array with each component/field in + * EventColumnName separated by Separator#VALUES. This leads to an event + * column name of the form eventId=timestamp=infokey. + * If timestamp in passed EventColumnName object is null (eventId is not null) + * this returns a column prefix of the form eventId= and if infokey in + * EventColumnName is null (other 2 components are not null), this returns a + * column name of the form eventId=timestamp= + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter + * #encode(java.lang.Object) + */ + @Override + public byte[] encode(EventColumnName key) { + byte[] first = Separator.encode(key.getId(), Separator.SPACE, Separator.TAB, + Separator.VALUES); + if (key.getTimestamp() == null) { + return Separator.VALUES.join(first, Separator.EMPTY_BYTES); + } + byte[] second = Bytes.toBytes( + LongConverter.invertLong(key.getTimestamp())); + if (key.getInfoKey() == null) { + return Separator.VALUES.join(first, second, Separator.EMPTY_BYTES); + } + return Separator.VALUES.join(first, second, Separator.encode( + key.getInfoKey(), Separator.SPACE, Separator.TAB, Separator.VALUES)); + } + + /* + * (non-Javadoc) + * + * Decodes an event column name of the form eventId=timestamp= or + * eventId=timestamp=infoKey represented in byte format and converts it into + * an EventColumnName object. + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter + * #decode(byte[]) + */ + @Override + public EventColumnName decode(byte[] bytes) { + byte[][] components = Separator.VALUES.split(bytes, SEGMENT_SIZES); + if (components.length != 3) { + throw new IllegalArgumentException("the column name is not valid"); + } + String id = Separator.decode(Bytes.toString(components[0]), + Separator.VALUES, Separator.TAB, Separator.SPACE); + Long ts = LongConverter.invertLong(Bytes.toLong(components[1])); + String infoKey = components[2].length == 0 ? null : + Separator.decode(Bytes.toString(components[2]), + Separator.VALUES, Separator.TAB, Separator.SPACE); + return new EventColumnName(id, ts, infoKey); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/GenericConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/GenericConverter.java new file mode 100644 index 00000000000..c34bfcb6d92 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/GenericConverter.java @@ -0,0 +1,48 @@ +/** + * 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.timelineservice.storage.common; + +import java.io.IOException; + +import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper; + +/** + * Uses GenericObjectMapper to encode objects as bytes and decode bytes as + * objects. + */ +public final class GenericConverter implements ValueConverter { + private static final GenericConverter INSTANCE = new GenericConverter(); + + private GenericConverter() { + } + + public static GenericConverter getInstance() { + return INSTANCE; + } + + @Override + public byte[] encodeValue(Object value) throws IOException { + return GenericObjectMapper.write(value); + } + + @Override + public Object decodeValue(byte[] bytes) throws IOException { + return GenericObjectMapper.read(bytes); + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java new file mode 100644 index 00000000000..0e5ff59bd75 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java @@ -0,0 +1,354 @@ +/** + * 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.timelineservice.storage.common; + +import java.io.IOException; +import java.net.MalformedURLException; +import java.net.URL; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.Tag; +import org.apache.hadoop.hbase.client.Query; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationCompactionDimension; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationOperation; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.text.NumberFormat; + +/** + * A bunch of utility functions used in HBase TimelineService backend. + */ +public final class HBaseTimelineStorageUtils { + + private static final Logger LOG = + LoggerFactory.getLogger(HBaseTimelineStorageUtils.class); + /** milliseconds in one day. */ + public static final long MILLIS_ONE_DAY = 86400000L; + + private HBaseTimelineStorageUtils() { + } + + /** + * Combines the input array of attributes and the input aggregation operation + * into a new array of attributes. + * + * @param attributes Attributes to be combined. + * @param aggOp Aggregation operation. + * @return array of combined attributes. + */ + public static Attribute[] combineAttributes(Attribute[] attributes, + AggregationOperation aggOp) { + int newLength = getNewLengthCombinedAttributes(attributes, aggOp); + Attribute[] combinedAttributes = new Attribute[newLength]; + + if (attributes != null) { + System.arraycopy(attributes, 0, combinedAttributes, 0, attributes.length); + } + + if (aggOp != null) { + Attribute a2 = aggOp.getAttribute(); + combinedAttributes[newLength - 1] = a2; + } + return combinedAttributes; + } + + /** + * Returns a number for the new array size. The new array is the combination + * of input array of attributes and the input aggregation operation. + * + * @param attributes Attributes. + * @param aggOp Aggregation operation. + * @return the size for the new array + */ + private static int getNewLengthCombinedAttributes(Attribute[] attributes, + AggregationOperation aggOp) { + int oldLength = getAttributesLength(attributes); + int aggLength = getAppOpLength(aggOp); + return oldLength + aggLength; + } + + private static int getAppOpLength(AggregationOperation aggOp) { + if (aggOp != null) { + return 1; + } + return 0; + } + + private static int getAttributesLength(Attribute[] attributes) { + if (attributes != null) { + return attributes.length; + } + return 0; + } + + /** + * Returns the first seen aggregation operation as seen in the list of input + * tags or null otherwise. + * + * @param tags list of HBase tags. + * @return AggregationOperation + */ + public static AggregationOperation getAggregationOperationFromTagsList( + List tags) { + for (AggregationOperation aggOp : AggregationOperation.values()) { + for (Tag tag : tags) { + if (tag.getType() == aggOp.getTagType()) { + return aggOp; + } + } + } + return null; + } + + /** + * Creates a {@link Tag} from the input attribute. + * + * @param attribute Attribute from which tag has to be fetched. + * @return a HBase Tag. + */ + public static Tag getTagFromAttribute(Map.Entry attribute) { + // attribute could be either an Aggregation Operation or + // an Aggregation Dimension + // Get the Tag type from either + AggregationOperation aggOp = AggregationOperation + .getAggregationOperation(attribute.getKey()); + if (aggOp != null) { + Tag t = new Tag(aggOp.getTagType(), attribute.getValue()); + return t; + } + + AggregationCompactionDimension aggCompactDim = + AggregationCompactionDimension.getAggregationCompactionDimension( + attribute.getKey()); + if (aggCompactDim != null) { + Tag t = new Tag(aggCompactDim.getTagType(), attribute.getValue()); + return t; + } + return null; + } + + /** + * creates a new cell based on the input cell but with the new value. + * + * @param origCell Original cell + * @param newValue new cell value + * @return cell + * @throws IOException while creating new cell. + */ + public static Cell createNewCell(Cell origCell, byte[] newValue) + throws IOException { + return CellUtil.createCell(CellUtil.cloneRow(origCell), + CellUtil.cloneFamily(origCell), CellUtil.cloneQualifier(origCell), + origCell.getTimestamp(), KeyValue.Type.Put.getCode(), newValue); + } + + /** + * creates a cell with the given inputs. + * + * @param row row of the cell to be created + * @param family column family name of the new cell + * @param qualifier qualifier for the new cell + * @param ts timestamp of the new cell + * @param newValue value of the new cell + * @param tags tags in the new cell + * @return cell + * @throws IOException while creating the cell. + */ + public static Cell createNewCell(byte[] row, byte[] family, byte[] qualifier, + long ts, byte[] newValue, byte[] tags) throws IOException { + return CellUtil.createCell(row, family, qualifier, ts, KeyValue.Type.Put, + newValue, tags); + } + + /** + * returns app id from the list of tags. + * + * @param tags cell tags to be looked into + * @return App Id as the AggregationCompactionDimension + */ + public static String getAggregationCompactionDimension(List tags) { + String appId = null; + for (Tag t : tags) { + if (AggregationCompactionDimension.APPLICATION_ID.getTagType() == t + .getType()) { + appId = Bytes.toString(t.getValue()); + return appId; + } + } + return appId; + } + + /** + * Converts an int into it's inverse int to be used in (row) keys + * where we want to have the largest int value in the top of the table + * (scans start at the largest int first). + * + * @param key value to be inverted so that the latest version will be first in + * a scan. + * @return inverted int + */ + public static int invertInt(int key) { + return Integer.MAX_VALUE - key; + } + + /** + * returns the timestamp of that day's start (which is midnight 00:00:00 AM) + * for a given input timestamp. + * + * @param ts Timestamp. + * @return timestamp of that day's beginning (midnight) + */ + public static long getTopOfTheDayTimestamp(long ts) { + long dayTimestamp = ts - (ts % MILLIS_ONE_DAY); + return dayTimestamp; + } + + private static final ThreadLocal APP_ID_FORMAT = + new ThreadLocal() { + @Override + public NumberFormat initialValue() { + NumberFormat fmt = NumberFormat.getInstance(); + fmt.setGroupingUsed(false); + fmt.setMinimumIntegerDigits(4); + return fmt; + } + }; + + /** + * A utility method that converts ApplicationId to string without using + * FastNumberFormat in order to avoid the incompatibility issue caused + * by mixing hadoop-common 2.5.1 and hadoop-yarn-api 3.0 in this module. + * This is a work-around implementation as discussed in YARN-6905. + * + * @param appId application id + * @return the string representation of the given application id + * + */ + public static String convertApplicationIdToString(ApplicationId appId) { + StringBuilder sb = new StringBuilder(64); + sb.append(ApplicationId.appIdStrPrefix); + sb.append("_"); + sb.append(appId.getClusterTimestamp()); + sb.append('_'); + sb.append(APP_ID_FORMAT.get().format(appId.getId())); + return sb.toString(); + } + + /** + * @param conf Yarn configuration. Used to see if there is an explicit config + * pointing to the HBase config file to read. It should not be null + * or a NullPointerException will be thrown. + * @return a configuration with the HBase configuration from the classpath, + * optionally overwritten by the timeline service configuration URL if + * specified. + * @throws MalformedURLException if a timeline service HBase configuration URL + * is specified but is a malformed URL. + */ + public static Configuration getTimelineServiceHBaseConf(Configuration conf) + throws MalformedURLException { + if (conf == null) { + throw new NullPointerException(); + } + + Configuration hbaseConf; + String timelineServiceHBaseConfFileURL = + conf.get(YarnConfiguration.TIMELINE_SERVICE_HBASE_CONFIGURATION_FILE); + if (timelineServiceHBaseConfFileURL != null + && timelineServiceHBaseConfFileURL.length() > 0) { + LOG.info("Using hbase configuration at " + + timelineServiceHBaseConfFileURL); + // create a clone so that we don't mess with out input one + hbaseConf = new Configuration(conf); + Configuration plainHBaseConf = new Configuration(false); + URL hbaseSiteXML = new URL(timelineServiceHBaseConfFileURL); + plainHBaseConf.addResource(hbaseSiteXML); + HBaseConfiguration.merge(hbaseConf, plainHBaseConf); + } else { + // default to what is on the classpath + hbaseConf = HBaseConfiguration.create(conf); + } + return hbaseConf; + } + + /** + * Given a row key prefix stored in a byte array, return a byte array for its + * immediate next row key. + * + * @param rowKeyPrefix The provided row key prefix, represented in an array. + * @return the closest next row key of the provided row key. + */ + public static byte[] calculateTheClosestNextRowKeyForPrefix( + byte[] rowKeyPrefix) { + // Essentially we are treating it like an 'unsigned very very long' and + // doing +1 manually. + // Search for the place where the trailing 0xFFs start + int offset = rowKeyPrefix.length; + while (offset > 0) { + if (rowKeyPrefix[offset - 1] != (byte) 0xFF) { + break; + } + offset--; + } + + if (offset == 0) { + // We got an 0xFFFF... (only FFs) stopRow value which is + // the last possible prefix before the end of the table. + // So set it to stop at the 'end of the table' + return HConstants.EMPTY_END_ROW; + } + + // Copy the right length of the original + byte[] newStopRow = Arrays.copyOfRange(rowKeyPrefix, 0, offset); + // And increment the last one + newStopRow[newStopRow.length - 1]++; + return newStopRow; + } + + /** + * Checks if passed object is of integral type(Short/Integer/Long). + * + * @param obj Object to be checked. + * @return true if object passed is of type Short or Integer or Long, false + * otherwise. + */ + public static boolean isIntegralValue(Object obj) { + return (obj instanceof Short) || (obj instanceof Integer) || + (obj instanceof Long); + } + + public static void setMetricsTimeRange(Query query, byte[] metricsCf, + long tsBegin, long tsEnd) { + if (tsBegin != 0 || tsEnd != Long.MAX_VALUE) { + query.setColumnFamilyTimeRange(metricsCf, + tsBegin, ((tsEnd == Long.MAX_VALUE) ? Long.MAX_VALUE : (tsEnd + 1))); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/KeyConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/KeyConverter.java new file mode 100644 index 00000000000..4229e81ad13 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/KeyConverter.java @@ -0,0 +1,41 @@ +/** + * 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.timelineservice.storage.common; + +/** + * Interface which has to be implemented for encoding and decoding row keys and + * columns. + */ +public interface KeyConverter { + /** + * Encodes a key as a byte array. + * + * @param key key to be encoded. + * @return a byte array. + */ + byte[] encode(T key); + + /** + * Decodes a byte array and returns a key of type T. + * + * @param bytes byte representation + * @return an object(key) of type T which has been constructed after decoding + * the bytes. + */ + T decode(byte[] bytes); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/KeyConverterToString.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/KeyConverterToString.java new file mode 100644 index 00000000000..1f52a7b6f89 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/KeyConverterToString.java @@ -0,0 +1,38 @@ +/** + * 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.timelineservice.storage.common; + +/** + * Interface which has to be implemented for encoding and decoding row keys or + * column qualifiers as string. + */ +public interface KeyConverterToString { + /** + * Encode key as string. + * @param key of type T to be encoded as string. + * @return encoded value as string. + */ + String encodeAsString(T key); + + /** + * Decode row key from string to a key of type T. + * @param encodedKey string representation of row key + * @return type T which has been constructed after decoding string. + */ + T decodeFromString(String encodedKey); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongConverter.java new file mode 100644 index 00000000000..6ab69f7de15 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongConverter.java @@ -0,0 +1,94 @@ +/** + * 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.timelineservice.storage.common; + +import java.io.IOException; +import java.io.Serializable; + +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Encodes a value by interpreting it as a Long and converting it to bytes and + * decodes a set of bytes as a Long. + */ +public final class LongConverter implements NumericValueConverter, + Serializable { + + /** + * Added because we implement Comparator. + */ + private static final long serialVersionUID = 1L; + + public LongConverter() { + } + + @Override + public byte[] encodeValue(Object value) throws IOException { + if (!HBaseTimelineStorageUtils.isIntegralValue(value)) { + throw new IOException("Expected integral value"); + } + return Bytes.toBytes(((Number)value).longValue()); + } + + @Override + public Object decodeValue(byte[] bytes) throws IOException { + if (bytes == null) { + return null; + } + return Bytes.toLong(bytes); + } + + /** + * Compares two numbers as longs. If either number is null, it will be taken + * as 0. + * + * @param num1 the first {@code Long} to compare. + * @param num2 the second {@code Long} to compare. + * @return -1 if num1 is less than num2, 0 if num1 is equal to num2 and 1 if + * num1 is greater than num2. + */ + @Override + public int compare(Number num1, Number num2) { + return Long.compare((num1 == null) ? 0L : num1.longValue(), + (num2 == null) ? 0L : num2.longValue()); + } + + @Override + public Number add(Number num1, Number num2, Number...numbers) { + long sum = ((num1 == null) ? 0L : num1.longValue()) + + ((num2 == null) ? 0L : num2.longValue()); + for (Number num : numbers) { + sum = sum + ((num == null) ? 0L : num.longValue()); + } + return sum; + } + + /** + * Converts a timestamp into it's inverse timestamp to be used in (row) keys + * where we want to have the most recent timestamp in the top of the table + * (scans start at the most recent timestamp first). + * + * @param key value to be inverted so that the latest version will be first in + * a scan. + * @return inverted long + */ + public static long invertLong(long key) { + return Long.MAX_VALUE - key; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongKeyConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongKeyConverter.java new file mode 100644 index 00000000000..4a724d6b877 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongKeyConverter.java @@ -0,0 +1,68 @@ +/** + * 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.timelineservice.storage.common; + +import java.io.IOException; +/** + * Encodes and decodes column names / row keys which are long. + */ +public final class LongKeyConverter implements KeyConverter { + + /** + * To delegate the actual work to. + */ + private final LongConverter longConverter = new LongConverter(); + + public LongKeyConverter() { + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter + * #encode(java.lang.Object) + */ + @Override + public byte[] encode(Long key) { + try { + // IOException will not be thrown here as we are explicitly passing + // Long. + return longConverter.encodeValue(key); + } catch (IOException e) { + return null; + } + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter + * #decode(byte[]) + */ + @Override + public Long decode(byte[] bytes) { + try { + return (Long) longConverter.decodeValue(bytes); + } catch (IOException e) { + return null; + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/NumericValueConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/NumericValueConverter.java new file mode 100644 index 00000000000..8fb65368f5f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/NumericValueConverter.java @@ -0,0 +1,39 @@ +/** + * 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.timelineservice.storage.common; + +import java.util.Comparator; + +/** + * Extends ValueConverter interface for numeric converters to support numerical + * operations such as comparison, addition, etc. + */ +public interface NumericValueConverter extends ValueConverter, + Comparator { + /** + * Adds two or more numbers. If either of the numbers are null, it is taken as + * 0. + * + * @param num1 the first number to add. + * @param num2 the second number to add. + * @param numbers Rest of the numbers to be added. + * @return result after adding up the numbers. + */ + Number add(Number num1, Number num2, Number...numbers); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Range.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Range.java new file mode 100644 index 00000000000..8a2e01a6a01 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Range.java @@ -0,0 +1,62 @@ +/** + * 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.timelineservice.storage.common; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Encapsulates a range with start and end indices. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public class Range { + private final int startIdx; + private final int endIdx; + + /** + * Defines a range from start index (inclusive) to end index (exclusive). + * + * @param start + * Starting index position + * @param end + * Ending index position (exclusive) + */ + public Range(int start, int end) { + if (start < 0 || end < start) { + throw new IllegalArgumentException( + "Invalid range, required that: 0 <= start <= end; start=" + start + + ", end=" + end); + } + + this.startIdx = start; + this.endIdx = end; + } + + public int start() { + return startIdx; + } + + public int end() { + return endIdx; + } + + public int length() { + return endIdx - startIdx; + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/RowKeyPrefix.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/RowKeyPrefix.java new file mode 100644 index 00000000000..6159dc71731 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/RowKeyPrefix.java @@ -0,0 +1,42 @@ +/** + * 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.timelineservice.storage.common; + +/** + * In queries where a single result is needed, an exact rowkey can be used + * through the corresponding rowkey#getRowKey() method. For queries that need to + * scan over a range of rowkeys, a partial (the initial part) of rowkeys are + * used. Classes implementing RowKeyPrefix indicate that they are the initial + * part of rowkeys, with different constructors with fewer number of argument to + * form a partial rowkey, a prefix. + * + * @param indicating the type of rowkey that a particular implementation is + * a prefix for. + */ +public interface RowKeyPrefix { + + /** + * Create a row key prefix, meaning a partial rowkey that can be used in range + * scans. Which fields are included in the prefix will depend on the + * constructor of the specific instance that was used. Output depends on which + * constructor was used. + * @return a prefix of the following form {@code fist!second!...!last!} + */ + byte[] getRowKeyPrefix(); + +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Separator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Separator.java new file mode 100644 index 00000000000..5090b4d60c5 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Separator.java @@ -0,0 +1,575 @@ +/** + * 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.timelineservice.storage.common; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Used to separate row qualifiers, column qualifiers and compound fields. + */ +public enum Separator { + + /** + * separator in key or column qualifier fields. + */ + QUALIFIERS("!", "%0$"), + + /** + * separator in values, and/or compound key/column qualifier fields. + */ + VALUES("=", "%1$"), + + /** + * separator in values, often used to avoid having these in qualifiers and + * names. Note that if we use HTML form encoding through URLEncoder, we end up + * getting a + for a space, which may already occur in strings, so we don't + * want that. + */ + SPACE(" ", "%2$"), + + /** + * separator in values, often used to avoid having these in qualifiers and + * names. + */ + TAB("\t", "%3$"); + + // a reserved character that starts each of the encoded values and is encoded + // first in order to escape naturally occurring instances of encoded values + // although it can be expressed as an enum instance, we define them as private + // variables to hide it from callers + private static final String PERCENT = "%"; + private static final String PERCENT_ENCODED = "%9$"; + + private static final Pattern PERCENT_PATTERN = + Pattern.compile(PERCENT, Pattern.LITERAL); + private static final String PERCENT_REPLACEMENT = + Matcher.quoteReplacement(PERCENT); + + private static final Pattern PERCENT_ENCODED_PATTERN = + Pattern.compile(PERCENT_ENCODED, Pattern.LITERAL); + private static final String PERCENT_ENCODED_REPLACEMENT = + Matcher.quoteReplacement(PERCENT_ENCODED); + + /** + * The string value of this separator. + */ + private final String value; + + /** + * The bye representation of value. + */ + private final byte[] bytes; + + // pre-compiled patterns and quoted replacements for optimization + private final Pattern valuePattern; + private final String valueReplacement; + + private final Pattern encodedValuePattern; + private final String encodedValueReplacement; + + /** + * Indicator for variable size of an individual segment in a split. The + * segment ends wherever separator is encountered. + * Typically used for string. + * Also used to indicate that there is no fixed number of splits which need to + * be returned. If split limit is specified as this, all possible splits are + * returned. + */ + public static final int VARIABLE_SIZE = 0; + + + /** empty string. */ + public static final String EMPTY_STRING = ""; + + /** empty bytes. */ + public static final byte[] EMPTY_BYTES = new byte[0]; + + /** + * @param value of the separator to use. Cannot be null or empty string. + * @param encodedValue choose something that isn't likely to occur in the data + * itself. Cannot be null or empty string. + */ + private Separator(String value, String encodedValue) { + this.value = value; + + // validation + if (value == null || value.length() == 0 || encodedValue == null + || encodedValue.length() == 0) { + throw new IllegalArgumentException( + "Cannot create separator from null or empty string."); + } + + this.bytes = Bytes.toBytes(value); + this.valuePattern = Pattern.compile(value, Pattern.LITERAL); + this.valueReplacement = Matcher.quoteReplacement(value); + + this.encodedValuePattern = Pattern.compile(encodedValue, Pattern.LITERAL); + this.encodedValueReplacement = Matcher.quoteReplacement(encodedValue); + } + + /** + * @return the original value of the separator + */ + public String getValue() { + return value; + } + + /** + * Used to make token safe to be used with this separator without collisions. + * It must be paired with {@link #decode(String)} for it to be + * decoded correctly. + *

+ * If you need to encode a given string for multiple separators, + * {@link #encode(String, Separator...)} should be used over successive + * invocations of this method. It will result in a more compact version of the + * encoded value. + * + * @param token Token to be encoded. + * @return the token with any occurrences of this separator URLEncoded. + */ + public String encode(String token) { + if (token == null || token.length() == 0) { + // Nothing to replace + return token; + } + // first encode the percent to escape naturally occurring encoded values + String escaped = encodePercent(token); + return encodeSingle(escaped, this); + } + + private static String replace(String token, Pattern pattern, + String replacement) { + return pattern.matcher(token).replaceAll(replacement); + } + + private static String encodeSingle(String token, Separator separator) { + return replace(token, separator.valuePattern, + separator.encodedValueReplacement); + } + + private static String encodePercent(String token) { + return replace(token, PERCENT_PATTERN, PERCENT_ENCODED_REPLACEMENT); + } + + /** + * Decode the token encoded using {@link #encode(String)}. It must be + * used for the result encoded with {@link #encode(String)} to be able to + * recover the original. + * + * @param token Token to be decoded. + * @return the token with any occurrences of the encoded separator replaced by + * the separator itself. + */ + public String decode(String token) { + if (token == null || token.length() == 0) { + // Nothing to replace + return token; + } + String escaped = decodeSingle(token, this); + // decode percent to de-escape + return decodePercent(escaped); + } + + private static String decodeSingle(String token, Separator separator) { + return replace(token, separator.encodedValuePattern, + separator.valueReplacement); + } + + private static String decodePercent(String token) { + return replace(token, PERCENT_ENCODED_PATTERN, PERCENT_REPLACEMENT); + } + + /** + * Encode the given separators in the token with their encoding equivalents. + * It must be paired with {@link #decode(byte[], Separator...)} or + * {@link #decode(String, Separator...)} with the same separators for it to be + * decoded correctly. + *

+ * If you need to encode a given string for multiple separators, this form of + * encoding should be used over successive invocations of + * {@link #encode(String)}. It will result in a more compact version of the + * encoded value. + * + * @param token containing possible separators that need to be encoded. + * @param separators to be encoded in the token with their URLEncoding + * equivalent. + * @return non-null byte representation of the token with occurrences of the + * separators encoded. + */ + public static byte[] encode(String token, Separator... separators) { + if (token == null || token.length() == 0) { + return EMPTY_BYTES; + } + String result = token; + // first encode the percent to escape naturally occurring encoded values + result = encodePercent(token); + for (Separator separator : separators) { + if (separator != null) { + result = encodeSingle(result, separator); + } + } + return Bytes.toBytes(result); + } + + /** + * Decode the given separators in the token with their decoding equivalents. + * It must be used for the result encoded with + * {@link #encode(String, Separator...)} with the same separators to be able + * to recover the original. + * + * @param token containing possible separators that need to be encoded. + * @param separators to be encoded in the token with their URLEncoding + * equivalent. + * @return String representation of the token with occurrences of the URL + * encoded separators decoded. + */ + public static String decode(byte[] token, Separator... separators) { + if (token == null) { + return null; + } + return decode(Bytes.toString(token), separators); + } + + /** + * Decode the given separators in the token with their decoding equivalents. + * It must be used for the result encoded with + * {@link #encode(String, Separator...)} with the same separators to be able + * to recover the original. + * + * @param token containing possible separators that need to be encoded. + * @param separators to be encoded in the token with their URLEncoding + * equivalent. + * @return String representation of the token with occurrences of the URL + * encoded separators decoded. + */ + public static String decode(String token, Separator... separators) { + if (token == null) { + return null; + } + String result = token; + for (Separator separator : separators) { + if (separator != null) { + result = decodeSingle(result, separator); + } + } + // decode percent to de-escape + return decodePercent(result); + } + + /** + * Returns a single byte array containing all of the individual arrays + * components separated by this separator. + * + * @param components Byte array components to be joined together. + * @return byte array after joining the components + */ + public byte[] join(byte[]... components) { + if (components == null || components.length == 0) { + return EMPTY_BYTES; + } + + int finalSize = 0; + finalSize = this.value.length() * (components.length - 1); + for (byte[] comp : components) { + if (comp != null) { + finalSize += comp.length; + } + } + + byte[] buf = new byte[finalSize]; + int offset = 0; + for (int i = 0; i < components.length; i++) { + if (components[i] != null) { + System.arraycopy(components[i], 0, buf, offset, components[i].length); + offset += components[i].length; + } + if (i < (components.length - 1)) { + System.arraycopy(this.bytes, 0, buf, offset, this.value.length()); + offset += this.value.length(); + } + } + return buf; + } + + /** + * Concatenates items (as String), using this separator. + * + * @param items Items join, {@code toString()} will be called in each item. + * Any occurrence of the separator in the individual strings will be + * first encoded. Cannot be null. + * @return non-null joined result. Note that when separator is {@literal null} + * the result is simply all items concatenated and the process is not + * reversible through {@link #splitEncoded(String)} + */ + public String joinEncoded(String... items) { + if (items == null || items.length == 0) { + return ""; + } + + StringBuilder sb = new StringBuilder(encode(items[0].toString())); + // Start at 1, we've already grabbed the first value at index 0 + for (int i = 1; i < items.length; i++) { + sb.append(this.value); + sb.append(encode(items[i].toString())); + } + + return sb.toString(); + } + + /** + * Concatenates items (as String), using this separator. + * + * @param items Items join, {@code toString()} will be called in each item. + * Any occurrence of the separator in the individual strings will be + * first encoded. Cannot be null. + * @return non-null joined result. Note that when separator is {@literal null} + * the result is simply all items concatenated and the process is not + * reversible through {@link #splitEncoded(String)} + */ + public String joinEncoded(Iterable items) { + if (items == null) { + return ""; + } + Iterator i = items.iterator(); + if (!i.hasNext()) { + return ""; + } + + StringBuilder sb = new StringBuilder(encode(i.next().toString())); + while (i.hasNext()) { + sb.append(this.value); + sb.append(encode(i.next().toString())); + } + + return sb.toString(); + } + + /** + * @param compoundValue containing individual values separated by this + * separator, which have that separator encoded. + * @return non-null set of values from the compoundValue with the separator + * decoded. + */ + public Collection splitEncoded(String compoundValue) { + List result = new ArrayList(); + if (compoundValue != null) { + for (String val : valuePattern.split(compoundValue)) { + result.add(decode(val)); + } + } + return result; + } + + /** + * Splits the source array into multiple array segments using this separator, + * up to a maximum of count items. This will naturally produce copied byte + * arrays for each of the split segments. + * + * @param source to be split + * @param limit on how many segments are supposed to be returned. A + * non-positive value indicates no limit on number of segments. + * @return source split by this separator. + */ + public byte[][] split(byte[] source, int limit) { + return split(source, this.bytes, limit); + } + + /** + * Splits the source array into multiple array segments using this separator. + * The sizes indicate the sizes of the relative components/segments. + * In case one of the segments contains this separator before the specified + * size is reached, the separator will be considered part of that segment and + * we will continue till size is reached. + * Variable length strings cannot contain this separator and are indiced with + * a size of {@value #VARIABLE_SIZE}. Such strings are encoded for this + * separator and decoded after the results from split is returned. + * + * @param source byte array to be split. + * @param sizes sizes of relative components/segments. + * @return source split by this separator as per the sizes specified.. + */ + public byte[][] split(byte[] source, int[] sizes) { + return split(source, this.bytes, sizes); + } + + /** + * Splits the source array into multiple array segments using this separator, + * as many times as splits are found. This will naturally produce copied byte + * arrays for each of the split segments. + * + * @param source byte array to be split + * @return source split by this separator. + */ + public byte[][] split(byte[] source) { + return split(source, this.bytes); + } + + /** + * Returns a list of ranges identifying [start, end) -- closed, open -- + * positions within the source byte array that would be split using the + * separator byte array. + * The sizes indicate the sizes of the relative components/segments. + * In case one of the segments contains this separator before the specified + * size is reached, the separator will be considered part of that segment and + * we will continue till size is reached. + * Variable length strings cannot contain this separator and are indiced with + * a size of {@value #VARIABLE_SIZE}. Such strings are encoded for this + * separator and decoded after the results from split is returned. + * + * @param source the source data + * @param separator the separator pattern to look for + * @param sizes indicate the sizes of the relative components/segments. + * @return a list of ranges. + */ + private static List splitRanges(byte[] source, byte[] separator, + int[] sizes) { + List segments = new ArrayList(); + if (source == null || separator == null) { + return segments; + } + // VARIABLE_SIZE here indicates that there is no limit to number of segments + // to return. + int limit = VARIABLE_SIZE; + if (sizes != null && sizes.length > 0) { + limit = sizes.length; + } + int start = 0; + int currentSegment = 0; + itersource: for (int i = 0; i < source.length; i++) { + for (int j = 0; j < separator.length; j++) { + if (source[i + j] != separator[j]) { + continue itersource; + } + } + // all separator elements matched + if (limit > VARIABLE_SIZE) { + if (segments.size() >= (limit - 1)) { + // everything else goes in one final segment + break; + } + if (sizes != null) { + int currentSegExpectedSize = sizes[currentSegment]; + if (currentSegExpectedSize > VARIABLE_SIZE) { + int currentSegSize = i - start; + if (currentSegSize < currentSegExpectedSize) { + // Segment not yet complete. More bytes to parse. + continue itersource; + } else if (currentSegSize > currentSegExpectedSize) { + // Segment is not as per size. + throw new IllegalArgumentException( + "Segments not separated as per expected sizes"); + } + } + } + } + segments.add(new Range(start, i)); + start = i + separator.length; + // i will be incremented again in outer for loop + i += separator.length - 1; + currentSegment++; + } + // add in remaining to a final range + if (start <= source.length) { + if (sizes != null) { + // Check if final segment is as per size specified. + if (sizes[currentSegment] > VARIABLE_SIZE && + source.length - start > sizes[currentSegment]) { + // Segment is not as per size. + throw new IllegalArgumentException( + "Segments not separated as per expected sizes"); + } + } + segments.add(new Range(start, source.length)); + } + return segments; + } + + /** + * Splits based on segments calculated based on limit/sizes specified for the + * separator. + * + * @param source byte array to be split. + * @param segments specifies the range for each segment. + * @return a byte[][] split as per the segment ranges. + */ + private static byte[][] split(byte[] source, List segments) { + byte[][] splits = new byte[segments.size()][]; + for (int i = 0; i < segments.size(); i++) { + Range r = segments.get(i); + byte[] tmp = new byte[r.length()]; + if (tmp.length > 0) { + System.arraycopy(source, r.start(), tmp, 0, r.length()); + } + splits[i] = tmp; + } + return splits; + } + + /** + * Splits the source array into multiple array segments using the given + * separator based on the sizes. This will naturally produce copied byte + * arrays for each of the split segments. + * + * @param source source array. + * @param separator separator represented as a byte array. + * @param sizes sizes of relative components/segments. + * @return byte[][] after splitting the source. + */ + private static byte[][] split(byte[] source, byte[] separator, int[] sizes) { + List segments = splitRanges(source, separator, sizes); + return split(source, segments); + } + + /** + * Splits the source array into multiple array segments using the given + * separator. This will naturally produce copied byte arrays for each of the + * split segments. + * + * @param source Source array. + * @param separator Separator represented as a byte array. + * @return byte[][] after splitting the source. + */ + private static byte[][] split(byte[] source, byte[] separator) { + return split(source, separator, (int[]) null); + } + + /** + * Splits the source array into multiple array segments using the given + * separator, up to a maximum of count items. This will naturally produce + * copied byte arrays for each of the split segments. + * + * @param source Source array. + * @param separator Separator represented as a byte array. + * @param limit a non-positive value indicates no limit on number of segments. + * @return byte[][] after splitting the input source. + */ + private static byte[][] split(byte[] source, byte[] separator, int limit) { + int[] sizes = null; + if (limit > VARIABLE_SIZE) { + sizes = new int[limit]; + } + List segments = splitRanges(source, separator, sizes); + return split(source, segments); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/StringKeyConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/StringKeyConverter.java new file mode 100644 index 00000000000..282848e3acc --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/StringKeyConverter.java @@ -0,0 +1,54 @@ +/** + * 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.timelineservice.storage.common; + +/** + * Encodes and decodes column names / row keys which are merely strings. + * Column prefixes are not part of the column name passed for encoding. It is + * added later, if required in the associated ColumnPrefix implementations. + */ +public final class StringKeyConverter implements KeyConverter { + + public StringKeyConverter() { + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter + * #encode(java.lang.Object) + */ + @Override + public byte[] encode(String key) { + return Separator.encode(key, Separator.SPACE, Separator.TAB); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter + * #decode(byte[]) + */ + @Override + public String decode(byte[] bytes) { + return Separator.decode(bytes, Separator.TAB, Separator.SPACE); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineHBaseSchemaConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineHBaseSchemaConstants.java new file mode 100644 index 00000000000..8e6c2595e58 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineHBaseSchemaConstants.java @@ -0,0 +1,71 @@ +/** + * 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.timelineservice.storage.common; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * contains the constants used in the context of schema accesses for + * {@link org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity} + * information. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public final class TimelineHBaseSchemaConstants { + private TimelineHBaseSchemaConstants() { + } + + /** + * Used to create a pre-split for tables starting with a username in the + * prefix. TODO: this may have to become a config variable (string with + * separators) so that different installations can presplit based on their own + * commonly occurring names. + */ + private final static byte[][] USERNAME_SPLITS = { + Bytes.toBytes("a"), Bytes.toBytes("ad"), Bytes.toBytes("an"), + Bytes.toBytes("b"), Bytes.toBytes("ca"), Bytes.toBytes("cl"), + Bytes.toBytes("d"), Bytes.toBytes("e"), Bytes.toBytes("f"), + Bytes.toBytes("g"), Bytes.toBytes("h"), Bytes.toBytes("i"), + Bytes.toBytes("j"), Bytes.toBytes("k"), Bytes.toBytes("l"), + Bytes.toBytes("m"), Bytes.toBytes("n"), Bytes.toBytes("o"), + Bytes.toBytes("q"), Bytes.toBytes("r"), Bytes.toBytes("s"), + Bytes.toBytes("se"), Bytes.toBytes("t"), Bytes.toBytes("u"), + Bytes.toBytes("v"), Bytes.toBytes("w"), Bytes.toBytes("x"), + Bytes.toBytes("y"), Bytes.toBytes("z") + }; + + /** + * The length at which keys auto-split. + */ + public static final String USERNAME_SPLIT_KEY_PREFIX_LENGTH = "4"; + + /** + * @return splits for splits where a user is a prefix. + */ + public static byte[][] getUsernameSplits() { + byte[][] kloon = USERNAME_SPLITS.clone(); + // Deep copy. + for (int row = 0; row < USERNAME_SPLITS.length; row++) { + kloon[row] = Bytes.copy(USERNAME_SPLITS[row]); + } + return kloon; + } + +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimestampGenerator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimestampGenerator.java new file mode 100644 index 00000000000..d03b37dbb3c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimestampGenerator.java @@ -0,0 +1,116 @@ +/** + * 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.timelineservice.storage.common; + +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.hadoop.yarn.api.records.ApplicationId; + +/** + * Utility class that allows HBase coprocessors to interact with unique + * timestamps. + */ +public class TimestampGenerator { + + /* + * if this is changed, then reading cell timestamps written with older + * multiplier value will not work + */ + public static final long TS_MULTIPLIER = 1000000L; + + private final AtomicLong lastTimestamp = new AtomicLong(); + + /** + * Returns the current wall clock time in milliseconds, multiplied by the + * required precision. + * + * @return current timestamp. + */ + public long currentTime() { + // We want to align cell timestamps with current time. + // cell timestamps are not be less than + // System.currentTimeMillis() * TS_MULTIPLIER. + return System.currentTimeMillis() * TS_MULTIPLIER; + } + + /** + * Returns a timestamp value unique within the scope of this + * {@code TimestampGenerator} instance. For usage by HBase + * {@code RegionObserver} coprocessors, this normally means unique within a + * given region. + * + * Unlikely scenario of generating a non-unique timestamp: if there is a + * sustained rate of more than 1M hbase writes per second AND if region fails + * over within that time range of timestamps being generated then there may be + * collisions writing to a cell version of the same column. + * + * @return unique timestamp. + */ + public long getUniqueTimestamp() { + long lastTs; + long nextTs; + do { + lastTs = lastTimestamp.get(); + nextTs = Math.max(lastTs + 1, currentTime()); + } while (!lastTimestamp.compareAndSet(lastTs, nextTs)); + return nextTs; + } + + /** + * Returns a timestamp multiplied with TS_MULTIPLIER and last few digits of + * application id. + * + * Unlikely scenario of generating a timestamp that is a duplicate: If more + * than a 1M concurrent apps are running in one flow run AND write to same + * column at the same time, then say appId of 1M and 1 will overlap + * with appId of 001 and there may be collisions for that flow run's + * specific column. + * + * @param incomingTS Timestamp to be converted. + * @param appId Application Id. + * @return a timestamp multiplied with TS_MULTIPLIER and last few digits of + * application id + */ + public static long getSupplementedTimestamp(long incomingTS, String appId) { + long suffix = getAppIdSuffix(appId); + long outgoingTS = incomingTS * TS_MULTIPLIER + suffix; + return outgoingTS; + + } + + private static long getAppIdSuffix(String appIdStr) { + if (appIdStr == null) { + return 0L; + } + ApplicationId appId = ApplicationId.fromString(appIdStr); + long id = appId.getId() % TS_MULTIPLIER; + return id; + } + + /** + * truncates the last few digits of the timestamp which were supplemented by + * the TimestampGenerator#getSupplementedTimestamp function. + * + * @param incomingTS Timestamp to be truncated. + * @return a truncated timestamp value + */ + public static long getTruncatedTimestamp(long incomingTS) { + return incomingTS / TS_MULTIPLIER; + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TypedBufferedMutator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TypedBufferedMutator.java new file mode 100644 index 00000000000..64a11f8f24d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TypedBufferedMutator.java @@ -0,0 +1,28 @@ +/** + * 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.timelineservice.storage.common; + +import org.apache.hadoop.hbase.client.BufferedMutator; + +/** + * Just a typed wrapper around {@link BufferedMutator} used to ensure that + * columns can write only to the table mutator for the right table. + */ +public interface TypedBufferedMutator extends BufferedMutator { + // This class is intentionally left (almost) blank +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ValueConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ValueConverter.java new file mode 100644 index 00000000000..757a6d3e54d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ValueConverter.java @@ -0,0 +1,47 @@ +/** + * 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.timelineservice.storage.common; + +import java.io.IOException; + +/** + * Converter used to encode/decode value associated with a column prefix or a + * column. + */ +public interface ValueConverter { + + /** + * Encode an object as a byte array depending on the converter implementation. + * + * @param value Value to be encoded. + * @return a byte array + * @throws IOException if any problem is encountered while encoding. + */ + byte[] encodeValue(Object value) throws IOException; + + /** + * Decode a byte array and convert it into an object depending on the + * converter implementation. + * + * @param bytes Byte array to be decoded. + * @return an object + * @throws IOException if any problem is encountered while decoding. + */ + Object decodeValue(byte[] bytes) throws IOException; +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/package-info.java new file mode 100644 index 00000000000..dcccf74e054 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/package-info.java @@ -0,0 +1,28 @@ +/* + * 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.timelineservice.storage contains + * classes which define and implement reading and writing to backend storage. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java new file mode 100644 index 00000000000..b228d848064 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java @@ -0,0 +1,112 @@ +/** + * 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.timelineservice.storage.entity; + +import java.io.IOException; + +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Column; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; + +/** + * Identifies fully qualified columns for the {@link EntityTable}. + */ +public enum EntityColumn implements Column { + + /** + * Identifier for the entity. + */ + ID(EntityColumnFamily.INFO, "id"), + + /** + * The type of entity. + */ + TYPE(EntityColumnFamily.INFO, "type"), + + /** + * When the entity was created. + */ + CREATED_TIME(EntityColumnFamily.INFO, "created_time", new LongConverter()), + + /** + * The version of the flow that this entity belongs to. + */ + FLOW_VERSION(EntityColumnFamily.INFO, "flow_version"); + + private final ColumnHelper column; + private final ColumnFamily columnFamily; + private final String columnQualifier; + private final byte[] columnQualifierBytes; + + EntityColumn(ColumnFamily columnFamily, + String columnQualifier) { + this(columnFamily, columnQualifier, GenericConverter.getInstance()); + } + + EntityColumn(ColumnFamily columnFamily, + String columnQualifier, ValueConverter converter) { + this.columnFamily = columnFamily; + this.columnQualifier = columnQualifier; + // Future-proof by ensuring the right column prefix hygiene. + this.columnQualifierBytes = + Bytes.toBytes(Separator.SPACE.encode(columnQualifier)); + this.column = new ColumnHelper(columnFamily, converter); + } + + /** + * @return the column name value + */ + private String getColumnQualifier() { + return columnQualifier; + } + + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, Long timestamp, + Object inputValue, Attribute... attributes) throws IOException { + column.store(rowKey, tableMutator, columnQualifierBytes, timestamp, + inputValue, attributes); + } + + public Object readResult(Result result) throws IOException { + return column.readResult(result, columnQualifierBytes); + } + + @Override + public byte[] getColumnQualifierBytes() { + return columnQualifierBytes.clone(); + } + + @Override + public byte[] getColumnFamilyBytes() { + return columnFamily.getBytes(); + } + + @Override + public ValueConverter getValueConverter() { + return column.getValueConverter(); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnFamily.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnFamily.java new file mode 100644 index 00000000000..7c637276046 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnFamily.java @@ -0,0 +1,65 @@ +/** + * 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.timelineservice.storage.entity; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; + +/** + * Represents the entity table column families. + */ +public enum EntityColumnFamily implements ColumnFamily { + + /** + * Info column family houses known columns, specifically ones included in + * columnfamily filters. + */ + INFO("i"), + + /** + * Configurations are in a separate column family for two reasons: a) the size + * of the config values can be very large and b) we expect that config values + * are often separately accessed from other metrics and info columns. + */ + CONFIGS("c"), + + /** + * Metrics have a separate column family, because they have a separate TTL. + */ + METRICS("m"); + + /** + * Byte representation of this column family. + */ + private final byte[] bytes; + + /** + * @param value create a column family with this name. Must be lower case and + * without spaces. + */ + EntityColumnFamily(String value) { + // column families should be lower case and not contain any spaces. + this.bytes = Bytes.toBytes(Separator.SPACE.encode(value)); + } + + public byte[] getBytes() { + return Bytes.copy(bytes); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java new file mode 100644 index 00000000000..d3851089c8f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java @@ -0,0 +1,249 @@ +/** + * 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.timelineservice.storage.entity; + +import java.io.IOException; +import java.util.Map; +import java.util.NavigableMap; + +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; + +/** + * Identifies partially qualified columns for the entity table. + */ +public enum EntityColumnPrefix implements ColumnPrefix { + + /** + * To store TimelineEntity getIsRelatedToEntities values. + */ + IS_RELATED_TO(EntityColumnFamily.INFO, "s"), + + /** + * To store TimelineEntity getRelatesToEntities values. + */ + RELATES_TO(EntityColumnFamily.INFO, "r"), + + /** + * To store TimelineEntity info values. + */ + INFO(EntityColumnFamily.INFO, "i"), + + /** + * Lifecycle events for an entity. + */ + EVENT(EntityColumnFamily.INFO, "e", true), + + /** + * Config column stores configuration with config key as the column name. + */ + CONFIG(EntityColumnFamily.CONFIGS, null), + + /** + * Metrics are stored with the metric name as the column name. + */ + METRIC(EntityColumnFamily.METRICS, null, new LongConverter()); + + private final ColumnHelper column; + private final ColumnFamily columnFamily; + + /** + * Can be null for those cases where the provided column qualifier is the + * entire column name. + */ + private final String columnPrefix; + private final byte[] columnPrefixBytes; + + /** + * Private constructor, meant to be used by the enum definition. + * + * @param columnFamily that this column is stored in. + * @param columnPrefix for this column. + */ + EntityColumnPrefix(ColumnFamily columnFamily, + String columnPrefix) { + this(columnFamily, columnPrefix, false, GenericConverter.getInstance()); + } + + EntityColumnPrefix(ColumnFamily columnFamily, + String columnPrefix, boolean compondColQual) { + this(columnFamily, columnPrefix, compondColQual, + GenericConverter.getInstance()); + } + + EntityColumnPrefix(ColumnFamily columnFamily, + String columnPrefix, ValueConverter converter) { + this(columnFamily, columnPrefix, false, converter); + } + + /** + * Private constructor, meant to be used by the enum definition. + * + * @param columnFamily that this column is stored in. + * @param columnPrefix for this column. + * @param converter used to encode/decode values to be stored in HBase for + * this column prefix. + */ + EntityColumnPrefix(ColumnFamily columnFamily, + String columnPrefix, boolean compondColQual, ValueConverter converter) { + column = new ColumnHelper(columnFamily, converter); + this.columnFamily = columnFamily; + this.columnPrefix = columnPrefix; + if (columnPrefix == null) { + this.columnPrefixBytes = null; + } else { + // Future-proof by ensuring the right column prefix hygiene. + this.columnPrefixBytes = + Bytes.toBytes(Separator.SPACE.encode(columnPrefix)); + } + } + + /** + * @return the column name value + */ + public String getColumnPrefix() { + return columnPrefix; + } + + @Override + public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) { + return ColumnHelper.getColumnQualifier( + this.columnPrefixBytes, qualifierPrefix); + } + + @Override + public byte[] getColumnPrefixBytes(String qualifierPrefix) { + return ColumnHelper.getColumnQualifier( + this.columnPrefixBytes, qualifierPrefix); + } + + @Override + public byte[] getColumnFamilyBytes() { + return columnFamily.getBytes(); + } + + @Override + public ValueConverter getValueConverter() { + return column.getValueConverter(); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #store(byte[], + * org.apache.hadoop.yarn.server.timelineservice.storage.common. + * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object, + * org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute[]) + */ + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, String qualifier, + Long timestamp, Object inputValue, Attribute... attributes) + throws IOException { + + // Null check + if (qualifier == null) { + throw new IOException("Cannot store column with null qualifier in " + + tableMutator.getName().getNameAsString()); + } + + byte[] columnQualifier = getColumnPrefixBytes(qualifier); + + column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue, + attributes); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #store(byte[], + * org.apache.hadoop.yarn.server.timelineservice.storage.common. + * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object) + */ + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, byte[] qualifier, + Long timestamp, Object inputValue, Attribute... attributes) + throws IOException { + + // Null check + if (qualifier == null) { + throw new IOException("Cannot store column with null qualifier in " + + tableMutator.getName().getNameAsString()); + } + + byte[] columnQualifier = getColumnPrefixBytes(qualifier); + + column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue, + attributes); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResult(org.apache.hadoop.hbase.client.Result, java.lang.String) + */ + public Object readResult(Result result, String qualifier) throws IOException { + byte[] columnQualifier = + ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier); + return column.readResult(result, columnQualifier); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResults(org.apache.hadoop.hbase.client.Result, + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter) + */ + public Map readResults(Result result, + KeyConverter keyConverter) throws IOException { + return column.readResults(result, columnPrefixBytes, keyConverter); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result, + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter) + */ + public NavigableMap> + readResultsWithTimestamps(Result result, KeyConverter keyConverter) + throws IOException { + return column.readResultsWithTimestamps(result, columnPrefixBytes, + keyConverter); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.java new file mode 100644 index 00000000000..b85a9b09c0a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.java @@ -0,0 +1,299 @@ +/** + * 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.timelineservice.storage.entity; + +import java.util.List; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.AppIdKeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverterToString; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; + +/** + * Represents a rowkey for the entity table. + */ +public class EntityRowKey { + private final String clusterId; + private final String userId; + private final String flowName; + private final Long flowRunId; + private final String appId; + private final String entityType; + private final Long entityIdPrefix; + private final String entityId; + private final EntityRowKeyConverter entityRowKeyConverter = + new EntityRowKeyConverter(); + + public EntityRowKey(String clusterId, String userId, String flowName, + Long flowRunId, String appId, String entityType, Long entityIdPrefix, + String entityId) { + this.clusterId = clusterId; + this.userId = userId; + this.flowName = flowName; + this.flowRunId = flowRunId; + this.appId = appId; + this.entityType = entityType; + this.entityIdPrefix = entityIdPrefix; + this.entityId = entityId; + } + + public String getClusterId() { + return clusterId; + } + + public String getUserId() { + return userId; + } + + public String getFlowName() { + return flowName; + } + + public Long getFlowRunId() { + return flowRunId; + } + + public String getAppId() { + return appId; + } + + public String getEntityType() { + return entityType; + } + + public String getEntityId() { + return entityId; + } + + public Long getEntityIdPrefix() { + return entityIdPrefix; + } + + /** + * Constructs a row key for the entity table as follows: + * {@code userName!clusterId!flowName!flowRunId!AppId!entityType!entityId}. + * Typically used while querying a specific entity. + * + * @return byte array with the row key. + */ + public byte[] getRowKey() { + return entityRowKeyConverter.encode(this); + } + + /** + * Given the raw row key as bytes, returns the row key as an object. + * @param rowKey byte representation of row key. + * @return An EntityRowKey object. + */ + public static EntityRowKey parseRowKey(byte[] rowKey) { + return new EntityRowKeyConverter().decode(rowKey); + } + + /** + * Constructs a row key for the entity table as follows: + *

+ * {@code userName!clusterId!flowName!flowRunId!AppId! + * entityType!entityIdPrefix!entityId}. + *

+ * @return String representation of row key. + */ + public String getRowKeyAsString() { + return entityRowKeyConverter.encodeAsString(this); + } + + /** + * Given the encoded row key as string, returns the row key as an object. + * @param encodedRowKey String representation of row key. + * @return A EntityRowKey object. + */ + public static EntityRowKey parseRowKeyFromString(String encodedRowKey) { + return new EntityRowKeyConverter().decodeFromString(encodedRowKey); + } + + /** + * Encodes and decodes row key for entity table. The row key is of the form : + * userName!clusterId!flowName!flowRunId!appId!entityType!entityId. flowRunId + * is a long, appId is encoded/decoded using {@link AppIdKeyConverter} and + * rest are strings. + *

+ */ + final private static class EntityRowKeyConverter implements + KeyConverter, KeyConverterToString { + + private final AppIdKeyConverter appIDKeyConverter = new AppIdKeyConverter(); + + private EntityRowKeyConverter() { + } + + /** + * Entity row key is of the form + * userName!clusterId!flowName!flowRunId!appId!entityType!entityId w. each + * segment separated by !. The sizes below indicate sizes of each one of + * these segments in sequence. clusterId, userName, flowName, entityType and + * entityId are strings. flowrunId is a long hence 8 bytes in size. app id + * is represented as 12 bytes with cluster timestamp part of appid being 8 + * bytes (long) and seq id being 4 bytes(int). Strings are variable in size + * (i.e. end whenever separator is encountered). This is used while decoding + * and helps in determining where to split. + */ + private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE, + Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, + AppIdKeyConverter.getKeySize(), Separator.VARIABLE_SIZE, + Bytes.SIZEOF_LONG, Separator.VARIABLE_SIZE }; + + /* + * (non-Javadoc) + * + * Encodes EntityRowKey object into a byte array with each component/field + * in EntityRowKey separated by Separator#QUALIFIERS. This leads to an + * entity table row key of the form + * userName!clusterId!flowName!flowRunId!appId!entityType!entityId If + * entityType in passed EntityRowKey object is null (and the fields + * preceding it i.e. clusterId, userId and flowName, flowRunId and appId + * are not null), this returns a row key prefix of the form + * userName!clusterId!flowName!flowRunId!appId! and if entityId in + * EntityRowKey is null (other 6 components are not null), this returns a + * row key prefix of the form + * userName!clusterId!flowName!flowRunId!appId!entityType! flowRunId is + * inverted while encoding as it helps maintain a descending order for row + * keys in entity table. + * + * @see org.apache.hadoop.yarn.server.timelineservice.storage.common + * .KeyConverter#encode(java.lang.Object) + */ + @Override + public byte[] encode(EntityRowKey rowKey) { + byte[] user = + Separator.encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB, + Separator.QUALIFIERS); + byte[] cluster = + Separator.encode(rowKey.getClusterId(), Separator.SPACE, + Separator.TAB, Separator.QUALIFIERS); + byte[] flow = + Separator.encode(rowKey.getFlowName(), Separator.SPACE, + Separator.TAB, Separator.QUALIFIERS); + byte[] first = Separator.QUALIFIERS.join(user, cluster, flow); + // Note that flowRunId is a long, so we can't encode them all at the same + // time. + byte[] second = + Bytes.toBytes(LongConverter.invertLong(rowKey.getFlowRunId())); + byte[] third = appIDKeyConverter.encode(rowKey.getAppId()); + if (rowKey.getEntityType() == null) { + return Separator.QUALIFIERS.join(first, second, third, + Separator.EMPTY_BYTES); + } + byte[] entityType = + Separator.encode(rowKey.getEntityType(), Separator.SPACE, + Separator.TAB, Separator.QUALIFIERS); + + if (rowKey.getEntityIdPrefix() == null) { + return Separator.QUALIFIERS.join(first, second, third, entityType, + Separator.EMPTY_BYTES); + } + + byte[] entityIdPrefix = Bytes.toBytes(rowKey.getEntityIdPrefix()); + + if (rowKey.getEntityId() == null) { + return Separator.QUALIFIERS.join(first, second, third, entityType, + entityIdPrefix, Separator.EMPTY_BYTES); + } + + byte[] entityId = Separator.encode(rowKey.getEntityId(), Separator.SPACE, + Separator.TAB, Separator.QUALIFIERS); + + byte[] fourth = + Separator.QUALIFIERS.join(entityType, entityIdPrefix, entityId); + + return Separator.QUALIFIERS.join(first, second, third, fourth); + } + + /* + * (non-Javadoc) + * + * Decodes an application row key of the form + * userName!clusterId!flowName!flowRunId!appId!entityType!entityId + * represented in byte format and converts it into an EntityRowKey object. + * flowRunId is inverted while decoding as it was inverted while encoding. + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common + * .KeyConverter#decode(byte[]) + */ + @Override + public EntityRowKey decode(byte[] rowKey) { + byte[][] rowKeyComponents = + Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES); + if (rowKeyComponents.length != 8) { + throw new IllegalArgumentException("the row key is not valid for " + + "an entity"); + } + String userId = + Separator.decode(Bytes.toString(rowKeyComponents[0]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + String clusterId = + Separator.decode(Bytes.toString(rowKeyComponents[1]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + String flowName = + Separator.decode(Bytes.toString(rowKeyComponents[2]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + Long flowRunId = + LongConverter.invertLong(Bytes.toLong(rowKeyComponents[3])); + String appId = appIDKeyConverter.decode(rowKeyComponents[4]); + String entityType = + Separator.decode(Bytes.toString(rowKeyComponents[5]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + + Long entityPrefixId = Bytes.toLong(rowKeyComponents[6]); + + String entityId = + Separator.decode(Bytes.toString(rowKeyComponents[7]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + return new EntityRowKey(clusterId, userId, flowName, flowRunId, appId, + entityType, entityPrefixId, entityId); + } + + @Override + public String encodeAsString(EntityRowKey key) { + if (key.clusterId == null || key.userId == null || key.flowName == null + || key.flowRunId == null || key.appId == null + || key.entityType == null || key.entityIdPrefix == null + || key.entityId == null) { + throw new IllegalArgumentException(); + } + return TimelineReaderUtils + .joinAndEscapeStrings(new String[] {key.clusterId, key.userId, + key.flowName, key.flowRunId.toString(), key.appId, key.entityType, + key.entityIdPrefix.toString(), key.entityId}); + } + + @Override + public EntityRowKey decodeFromString(String encodedRowKey) { + List split = TimelineReaderUtils.split(encodedRowKey); + if (split == null || split.size() != 8) { + throw new IllegalArgumentException("Invalid row key for entity table."); + } + Long flowRunId = Long.valueOf(split.get(3)); + Long entityIdPrefix = Long.valueOf(split.get(6)); + return new EntityRowKey(split.get(0), split.get(1), split.get(2), + flowRunId, split.get(4), split.get(5), entityIdPrefix, split.get(7)); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKeyPrefix.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKeyPrefix.java new file mode 100644 index 00000000000..47a1789bf95 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKeyPrefix.java @@ -0,0 +1,77 @@ +/** + * 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.timelineservice.storage.entity; + +import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix; + +/** + * Represents a partial rowkey without the entityId or without entityType and + * entityId for the entity table. + * + */ +public class EntityRowKeyPrefix extends EntityRowKey implements + RowKeyPrefix { + + /** + * Creates a prefix which generates the following rowKeyPrefixes for the + * entity table: + * {@code userName!clusterId!flowName!flowRunId!AppId!entityType!}. + * @param clusterId identifying the cluster + * @param userId identifying the user + * @param flowName identifying the flow + * @param flowRunId identifying the individual run of this flow + * @param appId identifying the application + * @param entityType which entity type + * @param entityIdPrefix for entityId + * @param entityId for an entity + */ + public EntityRowKeyPrefix(String clusterId, String userId, String flowName, + Long flowRunId, String appId, String entityType, Long entityIdPrefix, + String entityId) { + super(clusterId, userId, flowName, flowRunId, appId, entityType, + entityIdPrefix, entityId); + } + + /** + * Creates a prefix which generates the following rowKeyPrefixes for the + * entity table: + * {@code userName!clusterId!flowName!flowRunId!AppId!entityType!entityId}. + * + * @param clusterId identifying the cluster + * @param userId identifying the user + * @param flowName identifying the flow + * @param flowRunId identifying the individual run of this flow + * @param appId identifying the application + */ + public EntityRowKeyPrefix(String clusterId, String userId, String flowName, + Long flowRunId, String appId) { + this(clusterId, userId, flowName, flowRunId, appId, null, null, null); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.application. + * RowKeyPrefix#getRowKeyPrefix() + */ + public byte[] getRowKeyPrefix() { + return super.getRowKey(); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java new file mode 100644 index 00000000000..988bba2852f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java @@ -0,0 +1,170 @@ +/** + * 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.timelineservice.storage.entity; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The entity table as column families info, config and metrics. Info stores + * information about a timeline entity object config stores configuration data + * of a timeline entity object metrics stores the metrics of a timeline entity + * object + * + * Example entity table record: + * + *

+ * |-------------------------------------------------------------------------|
+ * |  Row       | Column Family                | Column Family| Column Family|
+ * |  key       | info                         | metrics      | config       |
+ * |-------------------------------------------------------------------------|
+ * | userName!  | id:entityId                  | metricId1:   | configKey1:  |
+ * | clusterId! |                              | metricValue1 | configValue1 |
+ * | flowName!  | type:entityType              | @timestamp1  |              |
+ * | flowRunId! |                              |              | configKey2:  |
+ * | AppId!     | created_time:                | metricId1:   | configValue2 |
+ * | entityType!| 1392993084018                | metricValue2 |              |
+ * | idPrefix!  |                              | @timestamp2  |              |
+ * | entityId   | i!infoKey:                   |              |              |
+ * |            | infoValue                    | metricId1:   |              |
+ * |            |                              | metricValue1 |              |
+ * |            | r!relatesToKey:              | @timestamp2  |              |
+ * |            | id3=id4=id5                  |              |              |
+ * |            |                              |              |              |
+ * |            | s!isRelatedToKey             |              |              |
+ * |            | id7=id9=id6                  |              |              |
+ * |            |                              |              |              |
+ * |            | e!eventId=timestamp=infoKey: |              |              |
+ * |            | eventInfoValue               |              |              |
+ * |            |                              |              |              |
+ * |            | flowVersion:                 |              |              |
+ * |            | versionValue                 |              |              |
+ * |-------------------------------------------------------------------------|
+ * 
+ */ +public class EntityTable extends BaseTable { + /** entity prefix. */ + private static final String PREFIX = + YarnConfiguration.TIMELINE_SERVICE_PREFIX + "entity"; + + /** config param name that specifies the entity table name. */ + public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; + + /** + * config param name that specifies the TTL for metrics column family in + * entity table. + */ + private static final String METRICS_TTL_CONF_NAME = PREFIX + + ".table.metrics.ttl"; + + /** + * config param name that specifies max-versions for metrics column family in + * entity table. + */ + private static final String METRICS_MAX_VERSIONS = + PREFIX + ".table.metrics.max-versions"; + + /** default value for entity table name. */ + public static final String DEFAULT_TABLE_NAME = "timelineservice.entity"; + + /** default TTL is 30 days for metrics timeseries. */ + private static final int DEFAULT_METRICS_TTL = 2592000; + + /** default max number of versions. */ + private static final int DEFAULT_METRICS_MAX_VERSIONS = 10000; + + private static final Logger LOG = + LoggerFactory.getLogger(EntityTable.class); + + public EntityTable() { + super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable + * (org.apache.hadoop.hbase.client.Admin, + * org.apache.hadoop.conf.Configuration) + */ + public void createTable(Admin admin, Configuration hbaseConf) + throws IOException { + + TableName table = getTableName(hbaseConf); + if (admin.tableExists(table)) { + // do not disable / delete existing table + // similar to the approach taken by map-reduce jobs when + // output directory exists + throw new IOException("Table " + table.getNameAsString() + + " already exists."); + } + + HTableDescriptor entityTableDescp = new HTableDescriptor(table); + HColumnDescriptor infoCF = + new HColumnDescriptor(EntityColumnFamily.INFO.getBytes()); + infoCF.setBloomFilterType(BloomType.ROWCOL); + entityTableDescp.addFamily(infoCF); + + HColumnDescriptor configCF = + new HColumnDescriptor(EntityColumnFamily.CONFIGS.getBytes()); + configCF.setBloomFilterType(BloomType.ROWCOL); + configCF.setBlockCacheEnabled(true); + entityTableDescp.addFamily(configCF); + + HColumnDescriptor metricsCF = + new HColumnDescriptor(EntityColumnFamily.METRICS.getBytes()); + entityTableDescp.addFamily(metricsCF); + metricsCF.setBlockCacheEnabled(true); + // always keep 1 version (the latest) + metricsCF.setMinVersions(1); + metricsCF.setMaxVersions( + hbaseConf.getInt(METRICS_MAX_VERSIONS, DEFAULT_METRICS_MAX_VERSIONS)); + metricsCF.setTimeToLive(hbaseConf.getInt(METRICS_TTL_CONF_NAME, + DEFAULT_METRICS_TTL)); + entityTableDescp.setRegionSplitPolicyClassName( + "org.apache.hadoop.hbase.regionserver.KeyPrefixRegionSplitPolicy"); + entityTableDescp.setValue("KeyPrefixRegionSplitPolicy.prefix_length", + TimelineHBaseSchemaConstants.USERNAME_SPLIT_KEY_PREFIX_LENGTH); + admin.createTable(entityTableDescp, + TimelineHBaseSchemaConstants.getUsernameSplits()); + LOG.info("Status of table creation for " + table.getNameAsString() + "=" + + admin.tableExists(table)); + } + + /** + * @param metricsTTL time to live parameter for the metricss in this table. + * @param hbaseConf configururation in which to set the metrics TTL config + * variable. + */ + public void setMetricsTTL(int metricsTTL, Configuration hbaseConf) { + hbaseConf.setInt(METRICS_TTL_CONF_NAME, metricsTTL); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/package-info.java new file mode 100644 index 00000000000..bb0e33133e2 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/package-info.java @@ -0,0 +1,28 @@ +/* + * 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.timelineservice.storage.entity + * contains classes related to implementation for entity table. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.timelineservice.storage.entity; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationCompactionDimension.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationCompactionDimension.java new file mode 100644 index 00000000000..4e2cf2da68c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationCompactionDimension.java @@ -0,0 +1,63 @@ +/** + * 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.timelineservice.storage.flow; + +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Identifies the compaction dimensions for the data in the {@link FlowRunTable} + * . + */ +public enum AggregationCompactionDimension { + + /** + * the application id. + */ + APPLICATION_ID((byte) 101); + + private byte tagType; + private byte[] inBytes; + + private AggregationCompactionDimension(byte tagType) { + this.tagType = tagType; + this.inBytes = Bytes.toBytes(this.name()); + } + + public Attribute getAttribute(String attributeValue) { + return new Attribute(this.name(), Bytes.toBytes(attributeValue)); + } + + public byte getTagType() { + return tagType; + } + + public byte[] getInBytes() { + return this.inBytes.clone(); + } + + public static AggregationCompactionDimension + getAggregationCompactionDimension(String aggCompactDimStr) { + for (AggregationCompactionDimension aggDim : AggregationCompactionDimension + .values()) { + if (aggDim.name().equals(aggCompactDimStr)) { + return aggDim; + } + } + return null; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationOperation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationOperation.java new file mode 100644 index 00000000000..40cdd2c972b --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationOperation.java @@ -0,0 +1,94 @@ +/** + * 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.timelineservice.storage.flow; + +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Identifies the attributes to be set for puts into the {@link FlowRunTable}. + * The numbers used for tagType are prime numbers. + */ +public enum AggregationOperation { + + /** + * When the flow was started. + */ + GLOBAL_MIN((byte) 71), + + /** + * When it ended. + */ + GLOBAL_MAX((byte) 73), + + /** + * The metrics of the flow. + */ + SUM((byte) 79), + + /** + * application running. + */ + SUM_FINAL((byte) 83), + + /** + * Min value as per the latest timestamp + * seen for a given app. + */ + LATEST_MIN((byte) 89), + + /** + * Max value as per the latest timestamp + * seen for a given app. + */ + LATEST_MAX((byte) 97); + + private byte tagType; + private byte[] inBytes; + + private AggregationOperation(byte tagType) { + this.tagType = tagType; + this.inBytes = Bytes.toBytes(this.name()); + } + + public Attribute getAttribute() { + return new Attribute(this.name(), this.inBytes); + } + + public byte getTagType() { + return tagType; + } + + public byte[] getInBytes() { + return this.inBytes.clone(); + } + + /** + * returns the AggregationOperation enum that represents that string. + * @param aggOpStr Aggregation operation. + * @return the AggregationOperation enum that represents that string + */ + public static AggregationOperation getAggregationOperation(String aggOpStr) { + for (AggregationOperation aggOp : AggregationOperation.values()) { + if (aggOp.name().equals(aggOpStr)) { + return aggOp; + } + } + return null; + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationACLsUpdatedEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/Attribute.java similarity index 57% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationACLsUpdatedEvent.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/Attribute.java index c8b314c221e..d3de51877be 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationACLsUpdatedEvent.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/Attribute.java @@ -15,31 +15,25 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +package org.apache.hadoop.yarn.server.timelineservice.storage.flow; -package org.apache.hadoop.yarn.server.resourcemanager.metrics; +/** + * Defines the attribute tuple to be set for puts into the {@link FlowRunTable}. + */ +public class Attribute { + private final String name; + private final byte[] value; -import org.apache.hadoop.yarn.api.records.ApplicationId; - - -public class ApplicationACLsUpdatedEvent extends SystemMetricsEvent { - - private ApplicationId appId; - private String viewAppACLs; - - public ApplicationACLsUpdatedEvent(ApplicationId appId, - String viewAppACLs, - long updatedTime) { - super(SystemMetricsEventType.APP_ACLS_UPDATED, updatedTime); - this.appId = appId; - this.viewAppACLs = viewAppACLs; + public Attribute(String name, byte[] value) { + this.name = name; + this.value = value.clone(); } - public ApplicationId getApplicationId() { - return appId; + public String getName() { + return name; } - public String getViewAppACLs() { - return viewAppACLs; + public byte[] getValue() { + return value.clone(); } - } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnFamily.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnFamily.java new file mode 100644 index 00000000000..f9eb5b40cfe --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnFamily.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.timelineservice.storage.flow; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; + +/** + * Represents the flow run table column families. + */ +public enum FlowActivityColumnFamily + implements ColumnFamily { + + /** + * Info column family houses known columns, specifically ones included in + * columnfamily filters. + */ + INFO("i"); + + /** + * Byte representation of this column family. + */ + private final byte[] bytes; + + /** + * @param value + * create a column family with this name. Must be lower case and + * without spaces. + */ + private FlowActivityColumnFamily(String value) { + // column families should be lower case and not contain any spaces. + this.bytes = Bytes.toBytes(Separator.SPACE.encode(value)); + } + + public byte[] getBytes() { + return Bytes.copy(bytes); + } + +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnPrefix.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnPrefix.java new file mode 100644 index 00000000000..706b002f93a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnPrefix.java @@ -0,0 +1,221 @@ +/** + * 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.timelineservice.storage.flow; + +import java.io.IOException; +import java.util.Map; +import java.util.NavigableMap; + +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; + +/** + * Identifies partially qualified columns for the {@link FlowActivityTable}. + */ +public enum FlowActivityColumnPrefix + implements ColumnPrefix { + + /** + * To store run ids of the flows. + */ + RUN_ID(FlowActivityColumnFamily.INFO, "r", null); + + private final ColumnHelper column; + private final ColumnFamily columnFamily; + + /** + * Can be null for those cases where the provided column qualifier is the + * entire column name. + */ + private final String columnPrefix; + private final byte[] columnPrefixBytes; + + private final AggregationOperation aggOp; + + /** + * Private constructor, meant to be used by the enum definition. + * + * @param columnFamily + * that this column is stored in. + * @param columnPrefix + * for this column. + */ + private FlowActivityColumnPrefix( + ColumnFamily columnFamily, String columnPrefix, + AggregationOperation aggOp) { + this(columnFamily, columnPrefix, aggOp, false); + } + + private FlowActivityColumnPrefix( + ColumnFamily columnFamily, String columnPrefix, + AggregationOperation aggOp, boolean compoundColQual) { + column = new ColumnHelper(columnFamily); + this.columnFamily = columnFamily; + this.columnPrefix = columnPrefix; + if (columnPrefix == null) { + this.columnPrefixBytes = null; + } else { + // Future-proof by ensuring the right column prefix hygiene. + this.columnPrefixBytes = Bytes.toBytes(Separator.SPACE + .encode(columnPrefix)); + } + this.aggOp = aggOp; + } + + /** + * @return the column name value + */ + public String getColumnPrefix() { + return columnPrefix; + } + + @Override + public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) { + return ColumnHelper.getColumnQualifier( + this.columnPrefixBytes, qualifierPrefix); + } + + @Override + public byte[] getColumnPrefixBytes(String qualifierPrefix) { + return ColumnHelper.getColumnQualifier( + this.columnPrefixBytes, qualifierPrefix); + } + + public byte[] getColumnPrefixBytes() { + return columnPrefixBytes.clone(); + } + + @Override + public byte[] getColumnFamilyBytes() { + return columnFamily.getBytes(); + } + + @Override + public ValueConverter getValueConverter() { + return column.getValueConverter(); + } + + public AggregationOperation getAttribute() { + return aggOp; + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #store(byte[], + * org.apache.hadoop.yarn.server.timelineservice.storage.common. + * TypedBufferedMutator, byte[], java.lang.Long, java.lang.Object, + * org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute[]) + */ + @Override + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, byte[] qualifier, + Long timestamp, Object inputValue, Attribute... attributes) + throws IOException { + // Null check + if (qualifier == null) { + throw new IOException("Cannot store column with null qualifier in " + + tableMutator.getName().getNameAsString()); + } + + byte[] columnQualifier = getColumnPrefixBytes(qualifier); + Attribute[] combinedAttributes = + HBaseTimelineStorageUtils.combineAttributes(attributes, this.aggOp); + column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue, + combinedAttributes); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResult(org.apache.hadoop.hbase.client.Result, java.lang.String) + */ + public Object readResult(Result result, String qualifier) throws IOException { + byte[] columnQualifier = ColumnHelper.getColumnQualifier( + this.columnPrefixBytes, qualifier); + return column.readResult(result, columnQualifier); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResults(org.apache.hadoop.hbase.client.Result, + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter) + */ + public Map readResults(Result result, + KeyConverter keyConverter) throws IOException { + return column.readResults(result, columnPrefixBytes, keyConverter); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result, + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter) + */ + public NavigableMap> + readResultsWithTimestamps(Result result, KeyConverter keyConverter) + throws IOException { + return column.readResultsWithTimestamps(result, columnPrefixBytes, + keyConverter); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #store(byte[], + * org.apache.hadoop.yarn.server.timelineservice.storage.common. + * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object, + * org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute[]) + */ + @Override + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, String qualifier, + Long timestamp, Object inputValue, Attribute... attributes) + throws IOException { + // Null check + if (qualifier == null) { + throw new IOException("Cannot store column with null qualifier in " + + tableMutator.getName().getNameAsString()); + } + + byte[] columnQualifier = getColumnPrefixBytes(qualifier); + Attribute[] combinedAttributes = + HBaseTimelineStorageUtils.combineAttributes(attributes, this.aggOp); + column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue, + combinedAttributes); + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.java new file mode 100644 index 00000000000..b8a5dba6cd6 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.java @@ -0,0 +1,247 @@ +/** + * 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.timelineservice.storage.flow; + +import java.util.List; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverterToString; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; + +/** + * Represents a rowkey for the flow activity table. + */ +public class FlowActivityRowKey { + + private final String clusterId; + private final Long dayTs; + private final String userId; + private final String flowName; + private final FlowActivityRowKeyConverter + flowActivityRowKeyConverter = new FlowActivityRowKeyConverter(); + + /** + * @param clusterId identifying the cluster + * @param dayTs to be converted to the top of the day timestamp + * @param userId identifying user + * @param flowName identifying the flow + */ + public FlowActivityRowKey(String clusterId, Long dayTs, String userId, + String flowName) { + this(clusterId, dayTs, userId, flowName, true); + } + + /** + * @param clusterId identifying the cluster + * @param timestamp when the flow activity happened. May be converted to the + * top of the day depending on the convertDayTsToTopOfDay argument. + * @param userId identifying user + * @param flowName identifying the flow + * @param convertDayTsToTopOfDay if true and timestamp isn't null, then + * timestamp will be converted to the top-of-the day timestamp + */ + protected FlowActivityRowKey(String clusterId, Long timestamp, String userId, + String flowName, boolean convertDayTsToTopOfDay) { + this.clusterId = clusterId; + if (convertDayTsToTopOfDay && (timestamp != null)) { + this.dayTs = HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(timestamp); + } else { + this.dayTs = timestamp; + } + this.userId = userId; + this.flowName = flowName; + } + + public String getClusterId() { + return clusterId; + } + + public Long getDayTimestamp() { + return dayTs; + } + + public String getUserId() { + return userId; + } + + public String getFlowName() { + return flowName; + } + + /** + * Constructs a row key for the flow activity table as follows: + * {@code clusterId!dayTimestamp!user!flowName}. + * + * @return byte array for the row key + */ + public byte[] getRowKey() { + return flowActivityRowKeyConverter.encode(this); + } + + /** + * Given the raw row key as bytes, returns the row key as an object. + * + * @param rowKey Byte representation of row key. + * @return A FlowActivityRowKey object. + */ + public static FlowActivityRowKey parseRowKey(byte[] rowKey) { + return new FlowActivityRowKeyConverter().decode(rowKey); + } + + /** + * Constructs a row key for the flow activity table as follows: + * {@code clusterId!dayTimestamp!user!flowName}. + * @return String representation of row key + */ + public String getRowKeyAsString() { + return flowActivityRowKeyConverter.encodeAsString(this); + } + + /** + * Given the raw row key as string, returns the row key as an object. + * @param encodedRowKey String representation of row key. + * @return A FlowActivityRowKey object. + */ + public static FlowActivityRowKey parseRowKeyFromString(String encodedRowKey) { + return new FlowActivityRowKeyConverter().decodeFromString(encodedRowKey); + } + + /** + * Encodes and decodes row key for flow activity table. The row key is of the + * form : clusterId!dayTimestamp!user!flowName. dayTimestamp(top of the day + * timestamp) is a long and rest are strings. + *

+ */ + final private static class FlowActivityRowKeyConverter + implements KeyConverter, + KeyConverterToString { + + private FlowActivityRowKeyConverter() { + } + + /** + * The flow activity row key is of the form + * clusterId!dayTimestamp!user!flowName with each segment separated by !. + * The sizes below indicate sizes of each one of these segements in + * sequence. clusterId, user and flowName are strings. Top of the day + * timestamp is a long hence 8 bytes in size. Strings are variable in size + * (i.e. they end whenever separator is encountered). This is used while + * decoding and helps in determining where to split. + */ + private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE, + Bytes.SIZEOF_LONG, Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE }; + + /* + * (non-Javadoc) + * + * Encodes FlowActivityRowKey object into a byte array with each + * component/field in FlowActivityRowKey separated by Separator#QUALIFIERS. + * This leads to an flow activity table row key of the form + * clusterId!dayTimestamp!user!flowName. If dayTimestamp in passed + * FlowActivityRowKey object is null and clusterId is not null, then this + * returns a row key prefix as clusterId! and if userId in + * FlowActivityRowKey is null (and the fields preceding it i.e. clusterId + * and dayTimestamp are not null), this returns a row key prefix as + * clusterId!dayTimeStamp! dayTimestamp is inverted while encoding as it + * helps maintain a descending order for row keys in flow activity table. + * + * @see org.apache.hadoop.yarn.server.timelineservice.storage.common + * .KeyConverter#encode(java.lang.Object) + */ + @Override + public byte[] encode(FlowActivityRowKey rowKey) { + if (rowKey.getDayTimestamp() == null) { + return Separator.QUALIFIERS.join(Separator.encode( + rowKey.getClusterId(), Separator.SPACE, Separator.TAB, + Separator.QUALIFIERS), Separator.EMPTY_BYTES); + } + if (rowKey.getUserId() == null) { + return Separator.QUALIFIERS.join(Separator.encode( + rowKey.getClusterId(), Separator.SPACE, Separator.TAB, + Separator.QUALIFIERS), Bytes.toBytes(LongConverter + .invertLong(rowKey.getDayTimestamp())), Separator.EMPTY_BYTES); + } + return Separator.QUALIFIERS.join(Separator.encode(rowKey.getClusterId(), + Separator.SPACE, Separator.TAB, Separator.QUALIFIERS), Bytes + .toBytes(LongConverter.invertLong(rowKey.getDayTimestamp())), + Separator.encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB, + Separator.QUALIFIERS), Separator.encode(rowKey.getFlowName(), + Separator.SPACE, Separator.TAB, Separator.QUALIFIERS)); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common + * .KeyConverter#decode(byte[]) + */ + @Override + public FlowActivityRowKey decode(byte[] rowKey) { + byte[][] rowKeyComponents = + Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES); + if (rowKeyComponents.length != 4) { + throw new IllegalArgumentException("the row key is not valid for " + + "a flow activity"); + } + String clusterId = + Separator.decode(Bytes.toString(rowKeyComponents[0]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + Long dayTs = LongConverter.invertLong(Bytes.toLong(rowKeyComponents[1])); + String userId = + Separator.decode(Bytes.toString(rowKeyComponents[2]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + String flowName = + Separator.decode(Bytes.toString(rowKeyComponents[3]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + return new FlowActivityRowKey(clusterId, dayTs, userId, flowName); + } + + @Override + public String encodeAsString(FlowActivityRowKey key) { + if (key.getDayTimestamp() == null) { + return TimelineReaderUtils + .joinAndEscapeStrings(new String[] {key.clusterId}); + } else if (key.getUserId() == null) { + return TimelineReaderUtils.joinAndEscapeStrings( + new String[] {key.clusterId, key.dayTs.toString()}); + } else if (key.getFlowName() == null) { + return TimelineReaderUtils.joinAndEscapeStrings( + new String[] {key.clusterId, key.dayTs.toString(), key.userId}); + } + return TimelineReaderUtils.joinAndEscapeStrings(new String[] { + key.clusterId, key.dayTs.toString(), key.userId, key.flowName}); + } + + @Override + public FlowActivityRowKey decodeFromString(String encodedRowKey) { + List split = TimelineReaderUtils.split(encodedRowKey); + if (split == null || split.size() != 4) { + throw new IllegalArgumentException( + "Invalid row key for flow activity."); + } + Long dayTs = Long.valueOf(split.get(1)); + return new FlowActivityRowKey(split.get(0), dayTs, split.get(2), + split.get(3)); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKeyPrefix.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKeyPrefix.java new file mode 100644 index 00000000000..eb88e546bfd --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKeyPrefix.java @@ -0,0 +1,60 @@ +/** + * 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.timelineservice.storage.flow; + +import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix; + +/** + * A prefix partial rowkey for flow activities. + */ +public class FlowActivityRowKeyPrefix extends FlowActivityRowKey implements + RowKeyPrefix { + + /** + * Constructs a row key prefix for the flow activity table as follows: + * {@code clusterId!dayTimestamp!}. + * + * @param clusterId Cluster Id. + * @param dayTs Start of the day timestamp. + */ + public FlowActivityRowKeyPrefix(String clusterId, Long dayTs) { + super(clusterId, dayTs, null, null, false); + } + + /** + * Constructs a row key prefix for the flow activity table as follows: + * {@code clusterId!}. + * + * @param clusterId identifying the cluster + */ + public FlowActivityRowKeyPrefix(String clusterId) { + super(clusterId, null, null, null, false); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.application. + * RowKeyPrefix#getRowKeyPrefix() + */ + public byte[] getRowKeyPrefix() { + return super.getRowKey(); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java new file mode 100644 index 00000000000..e646eb26a56 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java @@ -0,0 +1,109 @@ +/** + * 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.timelineservice.storage.flow; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The flow activity table has column family info + * Stores the daily activity record for flows + * Useful as a quick lookup of what flows were + * running on a given day + * + * Example flow activity table record: + * + *

+ * |-------------------------------------------|
+ * |  Row key   | Column Family                |
+ * |            | info                         |
+ * |-------------------------------------------|
+ * | clusterId! | r!runid1:version1            |
+ * | inv Top of |                              |
+ * | Day!       | r!runid2:version7            |
+ * | userName!  |                              |
+ * | flowName   |                              |
+ * |-------------------------------------------|
+ * 
+ */ +public class FlowActivityTable extends BaseTable { + /** flow activity table prefix. */ + private static final String PREFIX = + YarnConfiguration.TIMELINE_SERVICE_PREFIX + ".flowactivity"; + + /** config param name that specifies the flowactivity table name. */ + public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; + + /** default value for flowactivity table name. */ + public static final String DEFAULT_TABLE_NAME = + "timelineservice.flowactivity"; + + private static final Logger LOG = + LoggerFactory.getLogger(FlowActivityTable.class); + + /** default max number of versions. */ + public static final int DEFAULT_METRICS_MAX_VERSIONS = Integer.MAX_VALUE; + + public FlowActivityTable() { + super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable + * (org.apache.hadoop.hbase.client.Admin, + * org.apache.hadoop.conf.Configuration) + */ + public void createTable(Admin admin, Configuration hbaseConf) + throws IOException { + + TableName table = getTableName(hbaseConf); + if (admin.tableExists(table)) { + // do not disable / delete existing table + // similar to the approach taken by map-reduce jobs when + // output directory exists + throw new IOException("Table " + table.getNameAsString() + + " already exists."); + } + + HTableDescriptor flowActivityTableDescp = new HTableDescriptor(table); + HColumnDescriptor infoCF = + new HColumnDescriptor(FlowActivityColumnFamily.INFO.getBytes()); + infoCF.setBloomFilterType(BloomType.ROWCOL); + flowActivityTableDescp.addFamily(infoCF); + infoCF.setMinVersions(1); + infoCF.setMaxVersions(DEFAULT_METRICS_MAX_VERSIONS); + + // TODO: figure the split policy before running in production + admin.createTable(flowActivityTableDescp); + LOG.info("Status of table creation for " + table.getNameAsString() + "=" + + admin.tableExists(table)); + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumn.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumn.java new file mode 100644 index 00000000000..3797fafedc8 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumn.java @@ -0,0 +1,131 @@ +/** + * 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.timelineservice.storage.flow; + +import java.io.IOException; + +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Column; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; + +/** + * Identifies fully qualified columns for the {@link FlowRunTable}. + */ +public enum FlowRunColumn implements Column { + + /** + * When the flow was started. This is the minimum of currently known + * application start times. + */ + MIN_START_TIME(FlowRunColumnFamily.INFO, "min_start_time", + AggregationOperation.GLOBAL_MIN, new LongConverter()), + + /** + * When the flow ended. This is the maximum of currently known application end + * times. + */ + MAX_END_TIME(FlowRunColumnFamily.INFO, "max_end_time", + AggregationOperation.GLOBAL_MAX, new LongConverter()), + + /** + * The version of the flow that this flow belongs to. + */ + FLOW_VERSION(FlowRunColumnFamily.INFO, "flow_version", null); + + private final ColumnHelper column; + private final ColumnFamily columnFamily; + private final String columnQualifier; + private final byte[] columnQualifierBytes; + private final AggregationOperation aggOp; + + private FlowRunColumn(ColumnFamily columnFamily, + String columnQualifier, AggregationOperation aggOp) { + this(columnFamily, columnQualifier, aggOp, + GenericConverter.getInstance()); + } + + private FlowRunColumn(ColumnFamily columnFamily, + String columnQualifier, AggregationOperation aggOp, + ValueConverter converter) { + this.columnFamily = columnFamily; + this.columnQualifier = columnQualifier; + this.aggOp = aggOp; + // Future-proof by ensuring the right column prefix hygiene. + this.columnQualifierBytes = Bytes.toBytes(Separator.SPACE + .encode(columnQualifier)); + this.column = new ColumnHelper(columnFamily, converter, true); + } + + /** + * @return the column name value + */ + private String getColumnQualifier() { + return columnQualifier; + } + + @Override + public byte[] getColumnQualifierBytes() { + return columnQualifierBytes.clone(); + } + + @Override + public byte[] getColumnFamilyBytes() { + return columnFamily.getBytes(); + } + + public AggregationOperation getAggregationOperation() { + return aggOp; + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.Column#store + * (byte[], org.apache.hadoop.yarn.server.timelineservice.storage.common. + * TypedBufferedMutator, java.lang.Long, java.lang.Object, + * org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute[]) + */ + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, Long timestamp, + Object inputValue, Attribute... attributes) throws IOException { + + Attribute[] combinedAttributes = + HBaseTimelineStorageUtils.combineAttributes(attributes, aggOp); + column.store(rowKey, tableMutator, columnQualifierBytes, timestamp, + inputValue, combinedAttributes); + } + + public Object readResult(Result result) throws IOException { + return column.readResult(result, columnQualifierBytes); + } + + @Override + public ValueConverter getValueConverter() { + return column.getValueConverter(); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnFamily.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnFamily.java new file mode 100644 index 00000000000..8faf5f8576b --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnFamily.java @@ -0,0 +1,54 @@ +/** + * 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.timelineservice.storage.flow; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; + +/** + * Represents the flow run table column families. + */ +public enum FlowRunColumnFamily implements ColumnFamily { + + /** + * Info column family houses known columns, specifically ones included in + * columnfamily filters. + */ + INFO("i"); + + /** + * Byte representation of this column family. + */ + private final byte[] bytes; + + /** + * @param value + * create a column family with this name. Must be lower case and + * without spaces. + */ + private FlowRunColumnFamily(String value) { + // column families should be lower case and not contain any spaces. + this.bytes = Bytes.toBytes(Separator.SPACE.encode(value)); + } + + public byte[] getBytes() { + return Bytes.copy(bytes); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnPrefix.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnPrefix.java new file mode 100644 index 00000000000..f521cd71844 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnPrefix.java @@ -0,0 +1,217 @@ +/** + * 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.timelineservice.storage.flow; + +import java.io.IOException; +import java.util.Map; +import java.util.NavigableMap; + +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; + +/** + * Identifies partially qualified columns for the {@link FlowRunTable}. + */ +public enum FlowRunColumnPrefix implements ColumnPrefix { + + /** + * To store flow run info values. + */ + METRIC(FlowRunColumnFamily.INFO, "m", null, new LongConverter()); + + private final ColumnHelper column; + private final ColumnFamily columnFamily; + + /** + * Can be null for those cases where the provided column qualifier is the + * entire column name. + */ + private final String columnPrefix; + private final byte[] columnPrefixBytes; + + private final AggregationOperation aggOp; + + /** + * Private constructor, meant to be used by the enum definition. + * + * @param columnFamily that this column is stored in. + * @param columnPrefix for this column. + */ + private FlowRunColumnPrefix(ColumnFamily columnFamily, + String columnPrefix, AggregationOperation fra, ValueConverter converter) { + this(columnFamily, columnPrefix, fra, converter, false); + } + + private FlowRunColumnPrefix(ColumnFamily columnFamily, + String columnPrefix, AggregationOperation fra, ValueConverter converter, + boolean compoundColQual) { + column = new ColumnHelper(columnFamily, converter, true); + this.columnFamily = columnFamily; + this.columnPrefix = columnPrefix; + if (columnPrefix == null) { + this.columnPrefixBytes = null; + } else { + // Future-proof by ensuring the right column prefix hygiene. + this.columnPrefixBytes = + Bytes.toBytes(Separator.SPACE.encode(columnPrefix)); + } + this.aggOp = fra; + } + + /** + * @return the column name value + */ + public String getColumnPrefix() { + return columnPrefix; + } + + public byte[] getColumnPrefixBytes() { + return columnPrefixBytes.clone(); + } + + @Override + public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) { + return ColumnHelper.getColumnQualifier(this.columnPrefixBytes, + qualifierPrefix); + } + + @Override + public byte[] getColumnPrefixBytes(String qualifierPrefix) { + return ColumnHelper.getColumnQualifier(this.columnPrefixBytes, + qualifierPrefix); + } + + @Override + public byte[] getColumnFamilyBytes() { + return columnFamily.getBytes(); + } + + public AggregationOperation getAttribute() { + return aggOp; + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #store(byte[], + * org.apache.hadoop.yarn.server.timelineservice.storage.common. + * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object) + */ + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, String qualifier, + Long timestamp, Object inputValue, Attribute... attributes) + throws IOException { + + // Null check + if (qualifier == null) { + throw new IOException("Cannot store column with null qualifier in " + + tableMutator.getName().getNameAsString()); + } + + byte[] columnQualifier = getColumnPrefixBytes(qualifier); + Attribute[] combinedAttributes = + HBaseTimelineStorageUtils.combineAttributes(attributes, this.aggOp); + column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue, + combinedAttributes); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #store(byte[], + * org.apache.hadoop.yarn.server.timelineservice.storage.common. + * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object) + */ + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, byte[] qualifier, + Long timestamp, Object inputValue, Attribute... attributes) + throws IOException { + + // Null check + if (qualifier == null) { + throw new IOException("Cannot store column with null qualifier in " + + tableMutator.getName().getNameAsString()); + } + + byte[] columnQualifier = getColumnPrefixBytes(qualifier); + Attribute[] combinedAttributes = + HBaseTimelineStorageUtils.combineAttributes(attributes, this.aggOp); + column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue, + combinedAttributes); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResult(org.apache.hadoop.hbase.client.Result, java.lang.String) + */ + public Object readResult(Result result, String qualifier) throws IOException { + byte[] columnQualifier = + ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier); + return column.readResult(result, columnQualifier); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResults(org.apache.hadoop.hbase.client.Result, + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter) + */ + public Map readResults(Result result, + KeyConverter keyConverter) throws IOException { + return column.readResults(result, columnPrefixBytes, keyConverter); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result, + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter) + */ + public NavigableMap> + readResultsWithTimestamps(Result result, KeyConverter keyConverter) + throws IOException { + return column.readResultsWithTimestamps(result, columnPrefixBytes, + keyConverter); + } + + @Override + public ValueConverter getValueConverter() { + return column.getValueConverter(); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java new file mode 100644 index 00000000000..359eec98fa2 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java @@ -0,0 +1,275 @@ +/** + * 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.timelineservice.storage.flow; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.TreeMap; + +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.Tag; +import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver; +import org.apache.hadoop.hbase.coprocessor.ObserverContext; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; +import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.regionserver.RegionScanner; +import org.apache.hadoop.hbase.regionserver.ScanType; +import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; +import org.apache.hadoop.hbase.regionserver.wal.WALEdit; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimestampGenerator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Coprocessor for flow run table. + */ +public class FlowRunCoprocessor extends BaseRegionObserver { + + private static final Logger LOG = + LoggerFactory.getLogger(FlowRunCoprocessor.class); + + private Region region; + /** + * generate a timestamp that is unique per row in a region this is per region. + */ + private final TimestampGenerator timestampGenerator = + new TimestampGenerator(); + + @Override + public void start(CoprocessorEnvironment e) throws IOException { + if (e instanceof RegionCoprocessorEnvironment) { + RegionCoprocessorEnvironment env = (RegionCoprocessorEnvironment) e; + this.region = env.getRegion(); + } + } + + /* + * (non-Javadoc) + * + * This method adds the tags onto the cells in the Put. It is presumed that + * all the cells in one Put have the same set of Tags. The existing cell + * timestamp is overwritten for non-metric cells and each such cell gets a new + * unique timestamp generated by {@link TimestampGenerator} + * + * @see + * org.apache.hadoop.hbase.coprocessor.BaseRegionObserver#prePut(org.apache + * .hadoop.hbase.coprocessor.ObserverContext, + * org.apache.hadoop.hbase.client.Put, + * org.apache.hadoop.hbase.regionserver.wal.WALEdit, + * org.apache.hadoop.hbase.client.Durability) + */ + @Override + public void prePut(ObserverContext e, Put put, + WALEdit edit, Durability durability) throws IOException { + Map attributes = put.getAttributesMap(); + // Assumption is that all the cells in a put are the same operation. + List tags = new ArrayList<>(); + if ((attributes != null) && (attributes.size() > 0)) { + for (Map.Entry attribute : attributes.entrySet()) { + Tag t = HBaseTimelineStorageUtils.getTagFromAttribute(attribute); + tags.add(t); + } + byte[] tagByteArray = Tag.fromList(tags); + NavigableMap> newFamilyMap = new TreeMap<>( + Bytes.BYTES_COMPARATOR); + for (Map.Entry> entry : put.getFamilyCellMap() + .entrySet()) { + List newCells = new ArrayList<>(entry.getValue().size()); + for (Cell cell : entry.getValue()) { + // for each cell in the put add the tags + // Assumption is that all the cells in + // one put are the same operation + // also, get a unique cell timestamp for non-metric cells + // this way we don't inadvertently overwrite cell versions + long cellTimestamp = getCellTimestamp(cell.getTimestamp(), tags); + newCells.add(CellUtil.createCell(CellUtil.cloneRow(cell), + CellUtil.cloneFamily(cell), CellUtil.cloneQualifier(cell), + cellTimestamp, KeyValue.Type.Put, CellUtil.cloneValue(cell), + tagByteArray)); + } + newFamilyMap.put(entry.getKey(), newCells); + } // for each entry + // Update the family map for the Put + put.setFamilyCellMap(newFamilyMap); + } + } + + /** + * Determines if the current cell's timestamp is to be used or a new unique + * cell timestamp is to be used. The reason this is done is to inadvertently + * overwrite cells when writes come in very fast. But for metric cells, the + * cell timestamp signifies the metric timestamp. Hence we don't want to + * overwrite it. + * + * @param timestamp + * @param tags + * @return cell timestamp + */ + private long getCellTimestamp(long timestamp, List tags) { + // if ts not set (hbase sets to HConstants.LATEST_TIMESTAMP by default) + // then use the generator + if (timestamp == HConstants.LATEST_TIMESTAMP) { + return timestampGenerator.getUniqueTimestamp(); + } else { + return timestamp; + } + } + + /* + * (non-Javadoc) + * + * Creates a {@link FlowScanner} Scan so that it can correctly process the + * contents of {@link FlowRunTable}. + * + * @see + * org.apache.hadoop.hbase.coprocessor.BaseRegionObserver#preGetOp(org.apache + * .hadoop.hbase.coprocessor.ObserverContext, + * org.apache.hadoop.hbase.client.Get, java.util.List) + */ + @Override + public void preGetOp(ObserverContext e, + Get get, List results) throws IOException { + Scan scan = new Scan(get); + scan.setMaxVersions(); + RegionScanner scanner = null; + try { + scanner = new FlowScanner(e.getEnvironment(), scan, + region.getScanner(scan), FlowScannerOperation.READ); + scanner.next(results); + e.bypass(); + } finally { + if (scanner != null) { + scanner.close(); + } + } + } + + /* + * (non-Javadoc) + * + * Ensures that max versions are set for the Scan so that metrics can be + * correctly aggregated and min/max can be correctly determined. + * + * @see + * org.apache.hadoop.hbase.coprocessor.BaseRegionObserver#preScannerOpen(org + * .apache.hadoop.hbase.coprocessor.ObserverContext, + * org.apache.hadoop.hbase.client.Scan, + * org.apache.hadoop.hbase.regionserver.RegionScanner) + */ + @Override + public RegionScanner preScannerOpen( + ObserverContext e, Scan scan, + RegionScanner scanner) throws IOException { + // set max versions for scan to see all + // versions to aggregate for metrics + scan.setMaxVersions(); + return scanner; + } + + /* + * (non-Javadoc) + * + * Creates a {@link FlowScanner} Scan so that it can correctly process the + * contents of {@link FlowRunTable}. + * + * @see + * org.apache.hadoop.hbase.coprocessor.BaseRegionObserver#postScannerOpen( + * org.apache.hadoop.hbase.coprocessor.ObserverContext, + * org.apache.hadoop.hbase.client.Scan, + * org.apache.hadoop.hbase.regionserver.RegionScanner) + */ + @Override + public RegionScanner postScannerOpen( + ObserverContext e, Scan scan, + RegionScanner scanner) throws IOException { + return new FlowScanner(e.getEnvironment(), scan, + scanner, FlowScannerOperation.READ); + } + + @Override + public InternalScanner preFlush( + ObserverContext c, Store store, + InternalScanner scanner) throws IOException { + if (LOG.isDebugEnabled()) { + if (store != null) { + LOG.debug("preFlush store = " + store.getColumnFamilyName() + + " flushableSize=" + store.getFlushableSize() + + " flushedCellsCount=" + store.getFlushedCellsCount() + + " compactedCellsCount=" + store.getCompactedCellsCount() + + " majorCompactedCellsCount=" + + store.getMajorCompactedCellsCount() + " memstoreFlushSize=" + + store.getMemstoreFlushSize() + " memstoreSize=" + + store.getMemStoreSize() + " size=" + store.getSize() + + " storeFilesCount=" + store.getStorefilesCount()); + } + } + return new FlowScanner(c.getEnvironment(), scanner, + FlowScannerOperation.FLUSH); + } + + @Override + public void postFlush(ObserverContext c, + Store store, StoreFile resultFile) { + if (LOG.isDebugEnabled()) { + if (store != null) { + LOG.debug("postFlush store = " + store.getColumnFamilyName() + + " flushableSize=" + store.getFlushableSize() + + " flushedCellsCount=" + store.getFlushedCellsCount() + + " compactedCellsCount=" + store.getCompactedCellsCount() + + " majorCompactedCellsCount=" + + store.getMajorCompactedCellsCount() + " memstoreFlushSize=" + + store.getMemstoreFlushSize() + " memstoreSize=" + + store.getMemStoreSize() + " size=" + store.getSize() + + " storeFilesCount=" + store.getStorefilesCount()); + } + } + } + + @Override + public InternalScanner preCompact( + ObserverContext e, Store store, + InternalScanner scanner, ScanType scanType, CompactionRequest request) + throws IOException { + + FlowScannerOperation requestOp = FlowScannerOperation.MINOR_COMPACTION; + if (request != null) { + requestOp = (request.isMajor() ? FlowScannerOperation.MAJOR_COMPACTION + : FlowScannerOperation.MINOR_COMPACTION); + LOG.info("Compactionrequest= " + request.toString() + " " + + requestOp.toString() + " RegionName=" + e.getEnvironment() + .getRegion().getRegionInfo().getRegionNameAsString()); + } + return new FlowScanner(e.getEnvironment(), scanner, requestOp); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKey.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKey.java new file mode 100644 index 00000000000..7ce91cfed5c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKey.java @@ -0,0 +1,233 @@ +/** + * 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.timelineservice.storage.flow; + +import java.util.List; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverterToString; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; + +/** + * Represents a rowkey for the flow run table. + */ +public class FlowRunRowKey { + private final String clusterId; + private final String userId; + private final String flowName; + private final Long flowRunId; + private final FlowRunRowKeyConverter flowRunRowKeyConverter = + new FlowRunRowKeyConverter(); + + public FlowRunRowKey(String clusterId, String userId, String flowName, + Long flowRunId) { + this.clusterId = clusterId; + this.userId = userId; + this.flowName = flowName; + this.flowRunId = flowRunId; + } + + public String getClusterId() { + return clusterId; + } + + public String getUserId() { + return userId; + } + + public String getFlowName() { + return flowName; + } + + public Long getFlowRunId() { + return flowRunId; + } + + /** + * Constructs a row key for the entity table as follows: { + * clusterId!userId!flowName!Inverted Flow Run Id}. + * + * @return byte array with the row key + */ + public byte[] getRowKey() { + return flowRunRowKeyConverter.encode(this); + } + + + /** + * Given the raw row key as bytes, returns the row key as an object. + * @param rowKey Byte representation of row key. + * @return A FlowRunRowKey object. + */ + public static FlowRunRowKey parseRowKey(byte[] rowKey) { + return new FlowRunRowKeyConverter().decode(rowKey); + } + + /** + * Constructs a row key for the flow run table as follows: + * {@code clusterId!userId!flowName!Flow Run Id}. + * @return String representation of row key + */ + public String getRowKeyAsString() { + return flowRunRowKeyConverter.encodeAsString(this); + } + + /** + * Given the encoded row key as string, returns the row key as an object. + * @param encodedRowKey String representation of row key. + * @return A FlowRunRowKey object. + */ + public static FlowRunRowKey parseRowKeyFromString(String encodedRowKey) { + return new FlowRunRowKeyConverter().decodeFromString(encodedRowKey); + } + + /** + * returns the Flow Key as a verbose String output. + * @return String + */ + @Override + public String toString() { + StringBuilder flowKeyStr = new StringBuilder(); + flowKeyStr.append("{clusterId=" + clusterId); + flowKeyStr.append(" userId=" + userId); + flowKeyStr.append(" flowName=" + flowName); + flowKeyStr.append(" flowRunId="); + flowKeyStr.append(flowRunId); + flowKeyStr.append("}"); + return flowKeyStr.toString(); + } + + /** + * Encodes and decodes row key for flow run table. + * The row key is of the form : clusterId!userId!flowName!flowrunId. + * flowrunId is a long and rest are strings. + *

+ */ + final private static class FlowRunRowKeyConverter implements + KeyConverter, KeyConverterToString { + + private FlowRunRowKeyConverter() { + } + + /** + * The flow run row key is of the form clusterId!userId!flowName!flowrunId + * with each segment separated by !. The sizes below indicate sizes of each + * one of these segments in sequence. clusterId, userId and flowName are + * strings. flowrunId is a long hence 8 bytes in size. Strings are variable + * in size (i.e. end whenever separator is encountered). This is used while + * decoding and helps in determining where to split. + */ + private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE, + Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG }; + + /* + * (non-Javadoc) + * + * Encodes FlowRunRowKey object into a byte array with each component/field + * in FlowRunRowKey separated by Separator#QUALIFIERS. This leads to an flow + * run row key of the form clusterId!userId!flowName!flowrunId If flowRunId + * in passed FlowRunRowKey object is null (and the fields preceding it i.e. + * clusterId, userId and flowName are not null), this returns a row key + * prefix of the form clusterId!userName!flowName! flowRunId is inverted + * while encoding as it helps maintain a descending order for flow keys in + * flow run table. + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common + * .KeyConverter#encode(java.lang.Object) + */ + @Override + public byte[] encode(FlowRunRowKey rowKey) { + byte[] first = + Separator.QUALIFIERS.join(Separator.encode(rowKey.getClusterId(), + Separator.SPACE, Separator.TAB, Separator.QUALIFIERS), Separator + .encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB, + Separator.QUALIFIERS), Separator.encode(rowKey.getFlowName(), + Separator.SPACE, Separator.TAB, Separator.QUALIFIERS)); + if (rowKey.getFlowRunId() == null) { + return Separator.QUALIFIERS.join(first, Separator.EMPTY_BYTES); + } else { + // Note that flowRunId is a long, so we can't encode them all at the + // same + // time. + byte[] second = + Bytes.toBytes(LongConverter.invertLong(rowKey.getFlowRunId())); + return Separator.QUALIFIERS.join(first, second); + } + } + + /* + * (non-Javadoc) + * + * Decodes an flow run row key of the form + * clusterId!userId!flowName!flowrunId represented in byte format and + * converts it into an FlowRunRowKey object. flowRunId is inverted while + * decoding as it was inverted while encoding. + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common + * .KeyConverter#decode(byte[]) + */ + @Override + public FlowRunRowKey decode(byte[] rowKey) { + byte[][] rowKeyComponents = + Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES); + if (rowKeyComponents.length != 4) { + throw new IllegalArgumentException("the row key is not valid for " + + "a flow run"); + } + String clusterId = + Separator.decode(Bytes.toString(rowKeyComponents[0]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + String userId = + Separator.decode(Bytes.toString(rowKeyComponents[1]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + String flowName = + Separator.decode(Bytes.toString(rowKeyComponents[2]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + Long flowRunId = + LongConverter.invertLong(Bytes.toLong(rowKeyComponents[3])); + return new FlowRunRowKey(clusterId, userId, flowName, flowRunId); + } + + @Override + public String encodeAsString(FlowRunRowKey key) { + if (key.clusterId == null || key.userId == null || key.flowName == null + || key.flowRunId == null) { + throw new IllegalArgumentException(); + } + return TimelineReaderUtils.joinAndEscapeStrings(new String[] { + key.clusterId, key.userId, key.flowName, key.flowRunId.toString()}); + } + + @Override + public FlowRunRowKey decodeFromString(String encodedRowKey) { + List split = TimelineReaderUtils.split(encodedRowKey); + if (split == null || split.size() != 4) { + throw new IllegalArgumentException( + "Invalid row key for flow run table."); + } + Long flowRunId = Long.valueOf(split.get(3)); + return new FlowRunRowKey(split.get(0), split.get(1), split.get(2), + flowRunId); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKeyPrefix.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKeyPrefix.java new file mode 100644 index 00000000000..23ebc6692ec --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKeyPrefix.java @@ -0,0 +1,54 @@ +/** + * 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.timelineservice.storage.flow; + +import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix; + +/** + * Represents a partial rowkey (without the flowRunId) for the flow run table. + */ +public class FlowRunRowKeyPrefix extends FlowRunRowKey implements + RowKeyPrefix { + + /** + * Constructs a row key prefix for the flow run table as follows: + * {@code clusterId!userI!flowName!}. + * + * @param clusterId identifying the cluster + * @param userId identifying the user + * @param flowName identifying the flow + */ + public FlowRunRowKeyPrefix(String clusterId, String userId, + String flowName) { + super(clusterId, userId, flowName, null); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.application. + * RowKeyPrefix#getRowKeyPrefix() + */ + public byte[] getRowKeyPrefix() { + // We know we're a FlowRunRowKey with null florRunId, so we can simply + // delegate + return super.getRowKey(); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java new file mode 100644 index 00000000000..a1d32ee78cd --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java @@ -0,0 +1,151 @@ +/** + * 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.timelineservice.storage.flow; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Coprocessor; + +/** + * The flow run table has column family info + * Stores per flow run information + * aggregated across applications. + * + * Metrics are also stored in the info column family. + * + * Example flow run table record: + * + *

+ * flow_run table
+ * |-------------------------------------------|
+ * |  Row key   | Column Family                |
+ * |            | info                         |
+ * |-------------------------------------------|
+ * | clusterId! | flow_version:version7        |
+ * | userName!  |                              |
+ * | flowName!  | running_apps:1               |
+ * | flowRunId  |                              |
+ * |            | min_start_time:1392995080000 |
+ * |            | #0:""                        |
+ * |            |                              |
+ * |            | min_start_time:1392995081012 |
+ * |            | #0:appId2                    |
+ * |            |                              |
+ * |            | min_start_time:1392993083210 |
+ * |            | #0:appId3                    |
+ * |            |                              |
+ * |            |                              |
+ * |            | max_end_time:1392993084018   |
+ * |            | #0:""                        |
+ * |            |                              |
+ * |            |                              |
+ * |            | m!mapInputRecords:127        |
+ * |            | #0:""                        |
+ * |            |                              |
+ * |            | m!mapInputRecords:31         |
+ * |            | #2:appId2                    |
+ * |            |                              |
+ * |            | m!mapInputRecords:37         |
+ * |            | #1:appId3                    |
+ * |            |                              |
+ * |            |                              |
+ * |            | m!mapOutputRecords:181       |
+ * |            | #0:""                        |
+ * |            |                              |
+ * |            | m!mapOutputRecords:37        |
+ * |            | #1:appId3                    |
+ * |            |                              |
+ * |            |                              |
+ * |-------------------------------------------|
+ * 
+ */ +public class FlowRunTable extends BaseTable { + /** entity prefix. */ + private static final String PREFIX = + YarnConfiguration.TIMELINE_SERVICE_PREFIX + ".flowrun"; + + /** config param name that specifies the flowrun table name. */ + public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; + + /** default value for flowrun table name. */ + public static final String DEFAULT_TABLE_NAME = "timelineservice.flowrun"; + + private static final Logger LOG = + LoggerFactory.getLogger(FlowRunTable.class); + + /** default max number of versions. */ + public static final int DEFAULT_METRICS_MAX_VERSIONS = Integer.MAX_VALUE; + + public FlowRunTable() { + super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable + * (org.apache.hadoop.hbase.client.Admin, + * org.apache.hadoop.conf.Configuration) + */ + public void createTable(Admin admin, Configuration hbaseConf) + throws IOException { + + TableName table = getTableName(hbaseConf); + if (admin.tableExists(table)) { + // do not disable / delete existing table + // similar to the approach taken by map-reduce jobs when + // output directory exists + throw new IOException("Table " + table.getNameAsString() + + " already exists."); + } + + HTableDescriptor flowRunTableDescp = new HTableDescriptor(table); + HColumnDescriptor infoCF = + new HColumnDescriptor(FlowRunColumnFamily.INFO.getBytes()); + infoCF.setBloomFilterType(BloomType.ROWCOL); + flowRunTableDescp.addFamily(infoCF); + infoCF.setMinVersions(1); + infoCF.setMaxVersions(DEFAULT_METRICS_MAX_VERSIONS); + + // TODO: figure the split policy + String coprocessorJarPathStr = hbaseConf.get( + YarnConfiguration.FLOW_RUN_COPROCESSOR_JAR_HDFS_LOCATION, + YarnConfiguration.DEFAULT_HDFS_LOCATION_FLOW_RUN_COPROCESSOR_JAR); + + Path coprocessorJarPath = new Path(coprocessorJarPathStr); + LOG.info("CoprocessorJarPath=" + coprocessorJarPath.toString()); + flowRunTableDescp.addCoprocessor( + FlowRunCoprocessor.class.getCanonicalName(), coprocessorJarPath, + Coprocessor.PRIORITY_USER, null); + admin.createTable(flowRunTableDescp); + LOG.info("Status of table creation for " + table.getNameAsString() + "=" + + admin.tableExists(table)); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java new file mode 100644 index 00000000000..dbd04843eb1 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java @@ -0,0 +1,729 @@ +/** + * 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.timelineservice.storage.flow; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.Tag; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; +import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.RegionScanner; +import org.apache.hadoop.hbase.regionserver.ScannerContext; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.NumericValueConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimestampGenerator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Invoked via the coprocessor when a Get or a Scan is issued for flow run + * table. Looks through the list of cells per row, checks their tags and does + * operation on those cells as per the cell tags. Transforms reads of the stored + * metrics into calculated sums for each column Also, finds the min and max for + * start and end times in a flow run. + */ +class FlowScanner implements RegionScanner, Closeable { + + private static final Logger LOG = + LoggerFactory.getLogger(FlowScanner.class); + + /** + * use a special application id to represent the flow id this is needed since + * TimestampGenerator parses the app id to generate a cell timestamp. + */ + private static final String FLOW_APP_ID = "application_00000000000_0000"; + + private final Region region; + private final InternalScanner flowRunScanner; + private final int batchSize; + private final long appFinalValueRetentionThreshold; + private RegionScanner regionScanner; + private boolean hasMore; + private byte[] currentRow; + private List availableCells = new ArrayList<>(); + private int currentIndex; + private FlowScannerOperation action = FlowScannerOperation.READ; + + FlowScanner(RegionCoprocessorEnvironment env, InternalScanner internalScanner, + FlowScannerOperation action) { + this(env, null, internalScanner, action); + } + + FlowScanner(RegionCoprocessorEnvironment env, Scan incomingScan, + InternalScanner internalScanner, FlowScannerOperation action) { + this.batchSize = incomingScan == null ? -1 : incomingScan.getBatch(); + // TODO initialize other scan attributes like Scan#maxResultSize + this.flowRunScanner = internalScanner; + if (internalScanner instanceof RegionScanner) { + this.regionScanner = (RegionScanner) internalScanner; + } + this.action = action; + if (env == null) { + this.appFinalValueRetentionThreshold = + YarnConfiguration.DEFAULT_APP_FINAL_VALUE_RETENTION_THRESHOLD; + this.region = null; + } else { + this.region = env.getRegion(); + Configuration hbaseConf = env.getConfiguration(); + this.appFinalValueRetentionThreshold = hbaseConf.getLong( + YarnConfiguration.APP_FINAL_VALUE_RETENTION_THRESHOLD, + YarnConfiguration.DEFAULT_APP_FINAL_VALUE_RETENTION_THRESHOLD); + } + if (LOG.isDebugEnabled()) { + LOG.debug(" batch size=" + batchSize); + } + } + + + /* + * (non-Javadoc) + * + * @see org.apache.hadoop.hbase.regionserver.RegionScanner#getRegionInfo() + */ + @Override + public HRegionInfo getRegionInfo() { + return region.getRegionInfo(); + } + + @Override + public boolean nextRaw(List cells) throws IOException { + return nextRaw(cells, ScannerContext.newBuilder().build()); + } + + @Override + public boolean nextRaw(List cells, ScannerContext scannerContext) + throws IOException { + return nextInternal(cells, scannerContext); + } + + @Override + public boolean next(List cells) throws IOException { + return next(cells, ScannerContext.newBuilder().build()); + } + + @Override + public boolean next(List cells, ScannerContext scannerContext) + throws IOException { + return nextInternal(cells, scannerContext); + } + + /** + * Get value converter associated with a column or a column prefix. If nothing + * matches, generic converter is returned. + * @param colQualifierBytes + * @return value converter implementation. + */ + private static ValueConverter getValueConverter(byte[] colQualifierBytes) { + // Iterate over all the column prefixes for flow run table and get the + // appropriate converter for the column qualifier passed if prefix matches. + for (FlowRunColumnPrefix colPrefix : FlowRunColumnPrefix.values()) { + byte[] colPrefixBytes = colPrefix.getColumnPrefixBytes(""); + if (Bytes.compareTo(colPrefixBytes, 0, colPrefixBytes.length, + colQualifierBytes, 0, colPrefixBytes.length) == 0) { + return colPrefix.getValueConverter(); + } + } + // Iterate over all the columns for flow run table and get the + // appropriate converter for the column qualifier passed if match occurs. + for (FlowRunColumn column : FlowRunColumn.values()) { + if (Bytes.compareTo( + column.getColumnQualifierBytes(), colQualifierBytes) == 0) { + return column.getValueConverter(); + } + } + // Return generic converter if nothing matches. + return GenericConverter.getInstance(); + } + + /** + * This method loops through the cells in a given row of the + * {@link FlowRunTable}. It looks at the tags of each cell to figure out how + * to process the contents. It then calculates the sum or min or max for each + * column or returns the cell as is. + * + * @param cells + * @param scannerContext + * @return true if next row is available for the scanner, false otherwise + * @throws IOException + */ + private boolean nextInternal(List cells, ScannerContext scannerContext) + throws IOException { + Cell cell = null; + startNext(); + // Loop through all the cells in this row + // For min/max/metrics we do need to scan the entire set of cells to get the + // right one + // But with flush/compaction, the number of cells being scanned will go down + // cells are grouped per column qualifier then sorted by cell timestamp + // (latest to oldest) per column qualifier + // So all cells in one qualifier come one after the other before we see the + // next column qualifier + ByteArrayComparator comp = new ByteArrayComparator(); + byte[] previousColumnQualifier = Separator.EMPTY_BYTES; + AggregationOperation currentAggOp = null; + SortedSet currentColumnCells = new TreeSet<>(KeyValue.COMPARATOR); + Set alreadySeenAggDim = new HashSet<>(); + int addedCnt = 0; + long currentTimestamp = System.currentTimeMillis(); + ValueConverter converter = null; + int limit = batchSize; + + while (limit <= 0 || addedCnt < limit) { + cell = peekAtNextCell(scannerContext); + if (cell == null) { + break; + } + byte[] currentColumnQualifier = CellUtil.cloneQualifier(cell); + if (previousColumnQualifier == null) { + // first time in loop + previousColumnQualifier = currentColumnQualifier; + } + + converter = getValueConverter(currentColumnQualifier); + if (comp.compare(previousColumnQualifier, currentColumnQualifier) != 0) { + addedCnt += emitCells(cells, currentColumnCells, currentAggOp, + converter, currentTimestamp); + resetState(currentColumnCells, alreadySeenAggDim); + previousColumnQualifier = currentColumnQualifier; + currentAggOp = getCurrentAggOp(cell); + converter = getValueConverter(currentColumnQualifier); + } + collectCells(currentColumnCells, currentAggOp, cell, alreadySeenAggDim, + converter, scannerContext); + nextCell(scannerContext); + } + if ((!currentColumnCells.isEmpty()) && ((limit <= 0 || addedCnt < limit))) { + addedCnt += emitCells(cells, currentColumnCells, currentAggOp, converter, + currentTimestamp); + if (LOG.isDebugEnabled()) { + if (addedCnt > 0) { + LOG.debug("emitted cells. " + addedCnt + " for " + this.action + + " rowKey=" + + FlowRunRowKey.parseRowKey(CellUtil.cloneRow(cells.get(0)))); + } else { + LOG.debug("emitted no cells for " + this.action); + } + } + } + return hasMore(); + } + + private AggregationOperation getCurrentAggOp(Cell cell) { + List tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(), + cell.getTagsLength()); + // We assume that all the operations for a particular column are the same + return HBaseTimelineStorageUtils.getAggregationOperationFromTagsList(tags); + } + + /** + * resets the parameters to an initialized state for next loop iteration. + * + * @param cell + * @param currentAggOp + * @param currentColumnCells + * @param alreadySeenAggDim + * @param collectedButNotEmitted + */ + private void resetState(SortedSet currentColumnCells, + Set alreadySeenAggDim) { + currentColumnCells.clear(); + alreadySeenAggDim.clear(); + } + + private void collectCells(SortedSet currentColumnCells, + AggregationOperation currentAggOp, Cell cell, + Set alreadySeenAggDim, ValueConverter converter, + ScannerContext scannerContext) throws IOException { + + if (currentAggOp == null) { + // not a min/max/metric cell, so just return it as is + currentColumnCells.add(cell); + return; + } + + switch (currentAggOp) { + case GLOBAL_MIN: + if (currentColumnCells.size() == 0) { + currentColumnCells.add(cell); + } else { + Cell currentMinCell = currentColumnCells.first(); + Cell newMinCell = compareCellValues(currentMinCell, cell, currentAggOp, + (NumericValueConverter) converter); + if (!currentMinCell.equals(newMinCell)) { + currentColumnCells.remove(currentMinCell); + currentColumnCells.add(newMinCell); + } + } + break; + case GLOBAL_MAX: + if (currentColumnCells.size() == 0) { + currentColumnCells.add(cell); + } else { + Cell currentMaxCell = currentColumnCells.first(); + Cell newMaxCell = compareCellValues(currentMaxCell, cell, currentAggOp, + (NumericValueConverter) converter); + if (!currentMaxCell.equals(newMaxCell)) { + currentColumnCells.remove(currentMaxCell); + currentColumnCells.add(newMaxCell); + } + } + break; + case SUM: + case SUM_FINAL: + if (LOG.isTraceEnabled()) { + LOG.trace("In collect cells " + + " FlowSannerOperation=" + + this.action + + " currentAggOp=" + + currentAggOp + + " cell qualifier=" + + Bytes.toString(CellUtil.cloneQualifier(cell)) + + " cell value= " + + converter.decodeValue(CellUtil.cloneValue(cell)) + + " timestamp=" + cell.getTimestamp()); + } + + // only if this app has not been seen yet, add to current column cells + List tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(), + cell.getTagsLength()); + String aggDim = HBaseTimelineStorageUtils + .getAggregationCompactionDimension(tags); + if (!alreadySeenAggDim.contains(aggDim)) { + // if this agg dimension has already been seen, + // since they show up in sorted order + // we drop the rest which are older + // in other words, this cell is older than previously seen cells + // for that agg dim + // but when this agg dim is not seen, + // consider this cell in our working set + currentColumnCells.add(cell); + alreadySeenAggDim.add(aggDim); + } + break; + default: + break; + } // end of switch case + } + + /* + * Processes the cells in input param currentColumnCells and populates + * List cells as the output based on the input AggregationOperation + * parameter. + */ + private int emitCells(List cells, SortedSet currentColumnCells, + AggregationOperation currentAggOp, ValueConverter converter, + long currentTimestamp) throws IOException { + if ((currentColumnCells == null) || (currentColumnCells.size() == 0)) { + return 0; + } + if (currentAggOp == null) { + cells.addAll(currentColumnCells); + return currentColumnCells.size(); + } + if (LOG.isTraceEnabled()) { + LOG.trace("In emitCells " + this.action + " currentColumnCells size= " + + currentColumnCells.size() + " currentAggOp" + currentAggOp); + } + + switch (currentAggOp) { + case GLOBAL_MIN: + case GLOBAL_MAX: + cells.addAll(currentColumnCells); + return currentColumnCells.size(); + case SUM: + case SUM_FINAL: + switch (action) { + case FLUSH: + case MINOR_COMPACTION: + cells.addAll(currentColumnCells); + return currentColumnCells.size(); + case READ: + Cell sumCell = processSummation(currentColumnCells, + (NumericValueConverter) converter); + cells.add(sumCell); + return 1; + case MAJOR_COMPACTION: + List finalCells = processSummationMajorCompaction( + currentColumnCells, (NumericValueConverter) converter, + currentTimestamp); + cells.addAll(finalCells); + return finalCells.size(); + default: + cells.addAll(currentColumnCells); + return currentColumnCells.size(); + } + default: + cells.addAll(currentColumnCells); + return currentColumnCells.size(); + } + } + + /* + * Returns a cell whose value is the sum of all cell values in the input set. + * The new cell created has the timestamp of the most recent metric cell. The + * sum of a metric for a flow run is the summation at the point of the last + * metric update in that flow till that time. + */ + private Cell processSummation(SortedSet currentColumnCells, + NumericValueConverter converter) throws IOException { + Number sum = 0; + Number currentValue = 0; + long ts = 0L; + long mostCurrentTimestamp = 0L; + Cell mostRecentCell = null; + for (Cell cell : currentColumnCells) { + currentValue = (Number) converter.decodeValue(CellUtil.cloneValue(cell)); + ts = cell.getTimestamp(); + if (mostCurrentTimestamp < ts) { + mostCurrentTimestamp = ts; + mostRecentCell = cell; + } + sum = converter.add(sum, currentValue); + } + byte[] sumBytes = converter.encodeValue(sum); + Cell sumCell = + HBaseTimelineStorageUtils.createNewCell(mostRecentCell, sumBytes); + return sumCell; + } + + + /** + * Returns a list of cells that contains + * + * A) the latest cells for applications that haven't finished yet + * B) summation + * for the flow, based on applications that have completed and are older than + * a certain time + * + * The new cell created has the timestamp of the most recent metric cell. The + * sum of a metric for a flow run is the summation at the point of the last + * metric update in that flow till that time. + */ + @VisibleForTesting + List processSummationMajorCompaction( + SortedSet currentColumnCells, NumericValueConverter converter, + long currentTimestamp) + throws IOException { + Number sum = 0; + Number currentValue = 0; + long ts = 0L; + boolean summationDone = false; + List finalCells = new ArrayList(); + if (currentColumnCells == null) { + return finalCells; + } + + if (LOG.isDebugEnabled()) { + LOG.debug("In processSummationMajorCompaction," + + " will drop cells older than " + currentTimestamp + + " CurrentColumnCells size=" + currentColumnCells.size()); + } + + for (Cell cell : currentColumnCells) { + AggregationOperation cellAggOp = getCurrentAggOp(cell); + // if this is the existing flow sum cell + List tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(), + cell.getTagsLength()); + String appId = HBaseTimelineStorageUtils + .getAggregationCompactionDimension(tags); + if (appId == FLOW_APP_ID) { + sum = converter.add(sum, currentValue); + summationDone = true; + if (LOG.isTraceEnabled()) { + LOG.trace("reading flow app id sum=" + sum); + } + } else { + currentValue = (Number) converter.decodeValue(CellUtil + .cloneValue(cell)); + // read the timestamp truncated by the generator + ts = TimestampGenerator.getTruncatedTimestamp(cell.getTimestamp()); + if ((cellAggOp == AggregationOperation.SUM_FINAL) + && ((ts + this.appFinalValueRetentionThreshold) + < currentTimestamp)) { + sum = converter.add(sum, currentValue); + summationDone = true; + if (LOG.isTraceEnabled()) { + LOG.trace("MAJOR COMPACTION loop sum= " + sum + + " discarding now: " + " qualifier=" + + Bytes.toString(CellUtil.cloneQualifier(cell)) + " value=" + + converter.decodeValue(CellUtil.cloneValue(cell)) + + " timestamp=" + cell.getTimestamp() + " " + this.action); + } + } else { + // not a final value but it's the latest cell for this app + // so include this cell in the list of cells to write back + finalCells.add(cell); + } + } + } + if (summationDone) { + Cell anyCell = currentColumnCells.first(); + List tags = new ArrayList(); + Tag t = new Tag(AggregationOperation.SUM_FINAL.getTagType(), + Bytes.toBytes(FLOW_APP_ID)); + tags.add(t); + t = new Tag(AggregationCompactionDimension.APPLICATION_ID.getTagType(), + Bytes.toBytes(FLOW_APP_ID)); + tags.add(t); + byte[] tagByteArray = Tag.fromList(tags); + Cell sumCell = HBaseTimelineStorageUtils.createNewCell( + CellUtil.cloneRow(anyCell), + CellUtil.cloneFamily(anyCell), + CellUtil.cloneQualifier(anyCell), + TimestampGenerator.getSupplementedTimestamp( + System.currentTimeMillis(), FLOW_APP_ID), + converter.encodeValue(sum), tagByteArray); + finalCells.add(sumCell); + if (LOG.isTraceEnabled()) { + LOG.trace("MAJOR COMPACTION final sum= " + sum + " for " + + Bytes.toString(CellUtil.cloneQualifier(sumCell)) + + " " + this.action); + } + LOG.info("After major compaction for qualifier=" + + Bytes.toString(CellUtil.cloneQualifier(sumCell)) + + " with currentColumnCells.size=" + + currentColumnCells.size() + + " returning finalCells.size=" + finalCells.size() + + " with sum=" + sum.longValue() + + " with cell timestamp " + sumCell.getTimestamp()); + } else { + String qualifier = ""; + LOG.info("After major compaction for qualifier=" + qualifier + + " with currentColumnCells.size=" + + currentColumnCells.size() + + " returning finalCells.size=" + finalCells.size() + + " with zero sum=" + + sum.longValue()); + } + return finalCells; + } + + /** + * Determines which cell is to be returned based on the values in each cell + * and the comparison operation MIN or MAX. + * + * @param previouslyChosenCell + * @param currentCell + * @param currentAggOp + * @return the cell which is the min (or max) cell + * @throws IOException + */ + private Cell compareCellValues(Cell previouslyChosenCell, Cell currentCell, + AggregationOperation currentAggOp, NumericValueConverter converter) + throws IOException { + if (previouslyChosenCell == null) { + return currentCell; + } + try { + Number previouslyChosenCellValue = (Number)converter.decodeValue( + CellUtil.cloneValue(previouslyChosenCell)); + Number currentCellValue = (Number) converter.decodeValue(CellUtil + .cloneValue(currentCell)); + switch (currentAggOp) { + case GLOBAL_MIN: + if (converter.compare( + currentCellValue, previouslyChosenCellValue) < 0) { + // new value is minimum, hence return this cell + return currentCell; + } else { + // previously chosen value is miniumum, hence return previous min cell + return previouslyChosenCell; + } + case GLOBAL_MAX: + if (converter.compare( + currentCellValue, previouslyChosenCellValue) > 0) { + // new value is max, hence return this cell + return currentCell; + } else { + // previously chosen value is max, hence return previous max cell + return previouslyChosenCell; + } + default: + return currentCell; + } + } catch (IllegalArgumentException iae) { + LOG.error("caught iae during conversion to long ", iae); + return currentCell; + } + } + + @Override + public void close() throws IOException { + if (flowRunScanner != null) { + flowRunScanner.close(); + } else { + LOG.warn("scanner close called but scanner is null"); + } + } + + /** + * Called to signal the start of the next() call by the scanner. + */ + public void startNext() { + currentRow = null; + } + + /** + * Returns whether or not the underlying scanner has more rows. + */ + public boolean hasMore() { + return currentIndex < availableCells.size() ? true : hasMore; + } + + /** + * Returns the next available cell for the current row and advances the + * pointer to the next cell. This method can be called multiple times in a row + * to advance through all the available cells. + * + * @param scannerContext + * context information for the batch of cells under consideration + * @return the next available cell or null if no more cells are available for + * the current row + * @throws IOException + */ + public Cell nextCell(ScannerContext scannerContext) throws IOException { + Cell cell = peekAtNextCell(scannerContext); + if (cell != null) { + currentIndex++; + } + return cell; + } + + /** + * Returns the next available cell for the current row, without advancing the + * pointer. Calling this method multiple times in a row will continue to + * return the same cell. + * + * @param scannerContext + * context information for the batch of cells under consideration + * @return the next available cell or null if no more cells are available for + * the current row + * @throws IOException if any problem is encountered while grabbing the next + * cell. + */ + public Cell peekAtNextCell(ScannerContext scannerContext) throws IOException { + if (currentIndex >= availableCells.size()) { + // done with current batch + availableCells.clear(); + currentIndex = 0; + hasMore = flowRunScanner.next(availableCells, scannerContext); + } + Cell cell = null; + if (currentIndex < availableCells.size()) { + cell = availableCells.get(currentIndex); + if (currentRow == null) { + currentRow = CellUtil.cloneRow(cell); + } else if (!CellUtil.matchingRow(cell, currentRow)) { + // moved on to the next row + // don't use the current cell + // also signal no more cells for this row + return null; + } + } + return cell; + } + + /* + * (non-Javadoc) + * + * @see org.apache.hadoop.hbase.regionserver.RegionScanner#getMaxResultSize() + */ + @Override + public long getMaxResultSize() { + if (regionScanner == null) { + throw new IllegalStateException( + "RegionScanner.isFilterDone() called when the flow " + + "scanner's scanner is not a RegionScanner"); + } + return regionScanner.getMaxResultSize(); + } + + /* + * (non-Javadoc) + * + * @see org.apache.hadoop.hbase.regionserver.RegionScanner#getMvccReadPoint() + */ + @Override + public long getMvccReadPoint() { + if (regionScanner == null) { + throw new IllegalStateException( + "RegionScanner.isFilterDone() called when the flow " + + "scanner's internal scanner is not a RegionScanner"); + } + return regionScanner.getMvccReadPoint(); + } + + /* + * (non-Javadoc) + * + * @see org.apache.hadoop.hbase.regionserver.RegionScanner#isFilterDone() + */ + @Override + public boolean isFilterDone() throws IOException { + if (regionScanner == null) { + throw new IllegalStateException( + "RegionScanner.isFilterDone() called when the flow " + + "scanner's internal scanner is not a RegionScanner"); + } + return regionScanner.isFilterDone(); + + } + + /* + * (non-Javadoc) + * + * @see org.apache.hadoop.hbase.regionserver.RegionScanner#reseek(byte[]) + */ + @Override + public boolean reseek(byte[] bytes) throws IOException { + if (regionScanner == null) { + throw new IllegalStateException( + "RegionScanner.reseek() called when the flow " + + "scanner's internal scanner is not a RegionScanner"); + } + return regionScanner.reseek(bytes); + } + + @Override + public int getBatch() { + return batchSize; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScannerOperation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScannerOperation.java new file mode 100644 index 00000000000..73c666fa9ad --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScannerOperation.java @@ -0,0 +1,46 @@ +/** + * 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.timelineservice.storage.flow; + + +/** + * Identifies the scanner operation on the {@link FlowRunTable}. + */ +public enum FlowScannerOperation { + + /** + * If the scanner is opened for reading + * during preGet or preScan. + */ + READ, + + /** + * If the scanner is opened during preFlush. + */ + FLUSH, + + /** + * If the scanner is opened during minor Compaction. + */ + MINOR_COMPACTION, + + /** + * If the scanner is opened during major Compaction. + */ + MAJOR_COMPACTION +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/package-info.java new file mode 100644 index 00000000000..04963f3f1d3 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/package-info.java @@ -0,0 +1,29 @@ +/* + * 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.timelineservice.storage.flow + * contains classes related to implementation for flow related tables, viz. flow + * run table and flow activity table. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.timelineservice.storage.flow; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/package-info.java new file mode 100644 index 00000000000..e78db2a1ef5 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/package-info.java @@ -0,0 +1,28 @@ +/* + * 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.timelineservice.storage contains + * classes which define and implement reading and writing to backend storage. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.timelineservice.storage; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/AbstractTimelineStorageReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/AbstractTimelineStorageReader.java new file mode 100644 index 00000000000..5bacf66fb45 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/AbstractTimelineStorageReader.java @@ -0,0 +1,158 @@ +/** + * 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.timelineservice.storage.reader; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext; +import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTable; +import org.apache.hadoop.yarn.webapp.NotFoundException; + +/** + * The base class for reading timeline data from the HBase storage. This class + * provides basic support to validate and augment reader context. + */ +public abstract class AbstractTimelineStorageReader { + + private final TimelineReaderContext context; + /** + * Used to look up the flow context. + */ + private final AppToFlowTable appToFlowTable = new AppToFlowTable(); + + public AbstractTimelineStorageReader(TimelineReaderContext ctxt) { + context = ctxt; + } + + protected TimelineReaderContext getContext() { + return context; + } + + /** + * Looks up flow context from AppToFlow table. + * + * @param appToFlowRowKey to identify Cluster and App Ids. + * @param clusterId the cluster id. + * @param hbaseConf HBase configuration. + * @param conn HBase Connection. + * @return flow context information. + * @throws IOException if any problem occurs while fetching flow information. + */ + protected FlowContext lookupFlowContext(AppToFlowRowKey appToFlowRowKey, + String clusterId, Configuration hbaseConf, Connection conn) + throws IOException { + byte[] rowKey = appToFlowRowKey.getRowKey(); + Get get = new Get(rowKey); + Result result = appToFlowTable.getResult(hbaseConf, conn, get); + if (result != null && !result.isEmpty()) { + Object flowName = + AppToFlowColumnPrefix.FLOW_NAME.readResult(result, clusterId); + Object flowRunId = + AppToFlowColumnPrefix.FLOW_RUN_ID.readResult(result, clusterId); + Object userId = + AppToFlowColumnPrefix.USER_ID.readResult(result, clusterId); + if (flowName == null || userId == null || flowRunId == null) { + throw new NotFoundException( + "Unable to find the context flow name, and flow run id, " + + "and user id for clusterId=" + clusterId + + ", appId=" + appToFlowRowKey.getAppId()); + } + return new FlowContext((String)userId, (String)flowName, + ((Number)flowRunId).longValue()); + } else { + throw new NotFoundException( + "Unable to find the context flow name, and flow run id, " + + "and user id for clusterId=" + clusterId + + ", appId=" + appToFlowRowKey.getAppId()); + } + } + + /** + * Sets certain parameters to defaults if the values are not provided. + * + * @param hbaseConf HBase Configuration. + * @param conn HBase Connection. + * @throws IOException if any exception is encountered while setting params. + */ + protected void augmentParams(Configuration hbaseConf, Connection conn) + throws IOException { + defaultAugmentParams(hbaseConf, conn); + } + + /** + * Default behavior for all timeline readers to augment parameters. + * + * @param hbaseConf HBase Configuration. + * @param conn HBase Connection. + * @throws IOException if any exception is encountered while setting params. + */ + final protected void defaultAugmentParams(Configuration hbaseConf, + Connection conn) throws IOException { + // In reality all three should be null or neither should be null + if (context.getFlowName() == null || context.getFlowRunId() == null + || context.getUserId() == null) { + // Get flow context information from AppToFlow table. + AppToFlowRowKey appToFlowRowKey = + new AppToFlowRowKey(context.getAppId()); + FlowContext flowContext = + lookupFlowContext(appToFlowRowKey, context.getClusterId(), hbaseConf, + conn); + context.setFlowName(flowContext.flowName); + context.setFlowRunId(flowContext.flowRunId); + context.setUserId(flowContext.userId); + } + } + + /** + * Validates the required parameters to read the entities. + */ + protected abstract void validateParams(); + + /** + * Encapsulates flow context information. + */ + protected static class FlowContext { + private final String userId; + private final String flowName; + private final Long flowRunId; + + public FlowContext(String user, String flowName, Long flowRunId) { + this.userId = user; + this.flowName = flowName; + this.flowRunId = flowRunId; + } + + protected String getUserId() { + return userId; + } + + protected String getFlowName() { + return flowName; + } + + protected Long getFlowRunId() { + return flowRunId; + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java new file mode 100644 index 00000000000..0edd6a52680 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java @@ -0,0 +1,520 @@ +/** + * 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.timelineservice.storage.reader; + +import java.io.IOException; +import java.util.EnumSet; +import java.util.Set; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Query; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.filter.BinaryComparator; +import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; +import org.apache.hadoop.hbase.filter.FamilyFilter; +import org.apache.hadoop.hbase.filter.FilterList; +import org.apache.hadoop.hbase.filter.FilterList.Operator; +import org.apache.hadoop.hbase.filter.PageFilter; +import org.apache.hadoop.hbase.filter.QualifierFilter; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderUtils; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumn; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKeyPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils; +import org.apache.hadoop.yarn.webapp.BadRequestException; + +import com.google.common.base.Preconditions; + +/** + * Timeline entity reader for application entities that are stored in the + * application table. + */ +class ApplicationEntityReader extends GenericEntityReader { + private static final ApplicationTable APPLICATION_TABLE = + new ApplicationTable(); + + public ApplicationEntityReader(TimelineReaderContext ctxt, + TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) { + super(ctxt, entityFilters, toRetrieve); + } + + public ApplicationEntityReader(TimelineReaderContext ctxt, + TimelineDataToRetrieve toRetrieve) { + super(ctxt, toRetrieve); + } + + /** + * Uses the {@link ApplicationTable}. + */ + protected BaseTable getTable() { + return APPLICATION_TABLE; + } + + /** + * This method is called only for multiple entity reads. + */ + @Override + protected FilterList constructFilterListBasedOnFilters() throws IOException { + // Filters here cannot be null for multiple entity reads as they are set in + // augmentParams if null. + TimelineEntityFilters filters = getFilters(); + FilterList listBasedOnFilters = new FilterList(); + // Create filter list based on created time range and add it to + // listBasedOnFilters. + long createdTimeBegin = filters.getCreatedTimeBegin(); + long createdTimeEnd = filters.getCreatedTimeEnd(); + if (createdTimeBegin != 0 || createdTimeEnd != Long.MAX_VALUE) { + listBasedOnFilters.addFilter( + TimelineFilterUtils.createSingleColValueFiltersByRange( + ApplicationColumn.CREATED_TIME, createdTimeBegin, createdTimeEnd)); + } + // Create filter list based on metric filters and add it to + // listBasedOnFilters. + TimelineFilterList metricFilters = filters.getMetricFilters(); + if (metricFilters != null && !metricFilters.getFilterList().isEmpty()) { + listBasedOnFilters.addFilter( + TimelineFilterUtils.createHBaseFilterList( + ApplicationColumnPrefix.METRIC, metricFilters)); + } + // Create filter list based on config filters and add it to + // listBasedOnFilters. + TimelineFilterList configFilters = filters.getConfigFilters(); + if (configFilters != null && !configFilters.getFilterList().isEmpty()) { + listBasedOnFilters.addFilter( + TimelineFilterUtils.createHBaseFilterList( + ApplicationColumnPrefix.CONFIG, configFilters)); + } + // Create filter list based on info filters and add it to listBasedOnFilters + TimelineFilterList infoFilters = filters.getInfoFilters(); + if (infoFilters != null && !infoFilters.getFilterList().isEmpty()) { + listBasedOnFilters.addFilter( + TimelineFilterUtils.createHBaseFilterList( + ApplicationColumnPrefix.INFO, infoFilters)); + } + return listBasedOnFilters; + } + + /** + * Add {@link QualifierFilter} filters to filter list for each column of + * application table. + * + * @param list filter list to which qualifier filters have to be added. + */ + @Override + protected void updateFixedColumns(FilterList list) { + for (ApplicationColumn column : ApplicationColumn.values()) { + list.addFilter(new QualifierFilter(CompareOp.EQUAL, + new BinaryComparator(column.getColumnQualifierBytes()))); + } + } + + /** + * Creates a filter list which indicates that only some of the column + * qualifiers in the info column family will be returned in result. + * + * @return filter list. + * @throws IOException if any problem occurs while creating filter list. + */ + private FilterList createFilterListForColsOfInfoFamily() + throws IOException { + FilterList infoFamilyColsFilter = new FilterList(Operator.MUST_PASS_ONE); + // Add filters for each column in entity table. + updateFixedColumns(infoFamilyColsFilter); + EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); + // If INFO field has to be retrieved, add a filter for fetching columns + // with INFO column prefix. + if (hasField(fieldsToRetrieve, Field.INFO)) { + infoFamilyColsFilter.addFilter( + TimelineFilterUtils.createHBaseQualifierFilter( + CompareOp.EQUAL, ApplicationColumnPrefix.INFO)); + } + TimelineFilterList relatesTo = getFilters().getRelatesTo(); + if (hasField(fieldsToRetrieve, Field.RELATES_TO)) { + // If RELATES_TO field has to be retrieved, add a filter for fetching + // columns with RELATES_TO column prefix. + infoFamilyColsFilter.addFilter( + TimelineFilterUtils.createHBaseQualifierFilter( + CompareOp.EQUAL, ApplicationColumnPrefix.RELATES_TO)); + } else if (relatesTo != null && !relatesTo.getFilterList().isEmpty()) { + // Even if fields to retrieve does not contain RELATES_TO, we still + // need to have a filter to fetch some of the column qualifiers if + // relatesTo filters are specified. relatesTo filters will then be + // matched after fetching rows from HBase. + Set relatesToCols = + TimelineFilterUtils.fetchColumnsFromFilterList(relatesTo); + infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers( + ApplicationColumnPrefix.RELATES_TO, relatesToCols)); + } + TimelineFilterList isRelatedTo = getFilters().getIsRelatedTo(); + if (hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) { + // If IS_RELATED_TO field has to be retrieved, add a filter for fetching + // columns with IS_RELATED_TO column prefix. + infoFamilyColsFilter.addFilter( + TimelineFilterUtils.createHBaseQualifierFilter( + CompareOp.EQUAL, ApplicationColumnPrefix.IS_RELATED_TO)); + } else if (isRelatedTo != null && !isRelatedTo.getFilterList().isEmpty()) { + // Even if fields to retrieve does not contain IS_RELATED_TO, we still + // need to have a filter to fetch some of the column qualifiers if + // isRelatedTo filters are specified. isRelatedTo filters will then be + // matched after fetching rows from HBase. + Set isRelatedToCols = + TimelineFilterUtils.fetchColumnsFromFilterList(isRelatedTo); + infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers( + ApplicationColumnPrefix.IS_RELATED_TO, isRelatedToCols)); + } + TimelineFilterList eventFilters = getFilters().getEventFilters(); + if (hasField(fieldsToRetrieve, Field.EVENTS)) { + // If EVENTS field has to be retrieved, add a filter for fetching columns + // with EVENT column prefix. + infoFamilyColsFilter.addFilter( + TimelineFilterUtils.createHBaseQualifierFilter( + CompareOp.EQUAL, ApplicationColumnPrefix.EVENT)); + } else if (eventFilters != null && !eventFilters.getFilterList().isEmpty()){ + // Even if fields to retrieve does not contain EVENTS, we still need to + // have a filter to fetch some of the column qualifiers on the basis of + // event filters specified. Event filters will then be matched after + // fetching rows from HBase. + Set eventCols = + TimelineFilterUtils.fetchColumnsFromFilterList(eventFilters); + infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers( + ApplicationColumnPrefix.EVENT, eventCols)); + } + return infoFamilyColsFilter; + } + + /** + * Exclude column prefixes via filters which are not required(based on fields + * to retrieve) from info column family. These filters are added to filter + * list which contains a filter for getting info column family. + * + * @param infoColFamilyList filter list for info column family. + */ + private void excludeFieldsFromInfoColFamily(FilterList infoColFamilyList) { + EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); + // Events not required. + if (!hasField(fieldsToRetrieve, Field.EVENTS)) { + infoColFamilyList.addFilter( + TimelineFilterUtils.createHBaseQualifierFilter( + CompareOp.NOT_EQUAL, ApplicationColumnPrefix.EVENT)); + } + // info not required. + if (!hasField(fieldsToRetrieve, Field.INFO)) { + infoColFamilyList.addFilter( + TimelineFilterUtils.createHBaseQualifierFilter( + CompareOp.NOT_EQUAL, ApplicationColumnPrefix.INFO)); + } + // is related to not required. + if (!hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) { + infoColFamilyList.addFilter( + TimelineFilterUtils.createHBaseQualifierFilter( + CompareOp.NOT_EQUAL, ApplicationColumnPrefix.IS_RELATED_TO)); + } + // relates to not required. + if (!hasField(fieldsToRetrieve, Field.RELATES_TO)) { + infoColFamilyList.addFilter( + TimelineFilterUtils.createHBaseQualifierFilter( + CompareOp.NOT_EQUAL, ApplicationColumnPrefix.RELATES_TO)); + } + } + + /** + * Updates filter list based on fields for confs and metrics to retrieve. + * + * @param listBasedOnFields filter list based on fields. + * @throws IOException if any problem occurs while updating filter list. + */ + private void updateFilterForConfsAndMetricsToRetrieve( + FilterList listBasedOnFields) throws IOException { + TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve(); + // Please note that if confsToRetrieve is specified, we would have added + // CONFS to fields to retrieve in augmentParams() even if not specified. + if (dataToRetrieve.getFieldsToRetrieve().contains(Field.CONFIGS)) { + // Create a filter list for configs. + listBasedOnFields.addFilter(TimelineFilterUtils. + createFilterForConfsOrMetricsToRetrieve( + dataToRetrieve.getConfsToRetrieve(), + ApplicationColumnFamily.CONFIGS, ApplicationColumnPrefix.CONFIG)); + } + + // Please note that if metricsToRetrieve is specified, we would have added + // METRICS to fields to retrieve in augmentParams() even if not specified. + if (dataToRetrieve.getFieldsToRetrieve().contains(Field.METRICS)) { + // Create a filter list for metrics. + listBasedOnFields.addFilter(TimelineFilterUtils. + createFilterForConfsOrMetricsToRetrieve( + dataToRetrieve.getMetricsToRetrieve(), + ApplicationColumnFamily.METRICS, ApplicationColumnPrefix.METRIC)); + } + } + + @Override + protected FilterList constructFilterListBasedOnFields() throws IOException { + if (!needCreateFilterListBasedOnFields()) { + // Fetch all the columns. No need of a filter. + return null; + } + FilterList listBasedOnFields = new FilterList(Operator.MUST_PASS_ONE); + FilterList infoColFamilyList = new FilterList(); + // By default fetch everything in INFO column family. + FamilyFilter infoColumnFamily = + new FamilyFilter(CompareOp.EQUAL, + new BinaryComparator(ApplicationColumnFamily.INFO.getBytes())); + infoColFamilyList.addFilter(infoColumnFamily); + if (!isSingleEntityRead() && fetchPartialColsFromInfoFamily()) { + // We can fetch only some of the columns from info family. + infoColFamilyList.addFilter(createFilterListForColsOfInfoFamily()); + } else { + // Exclude column prefixes in info column family which are not required + // based on fields to retrieve. + excludeFieldsFromInfoColFamily(infoColFamilyList); + } + listBasedOnFields.addFilter(infoColFamilyList); + + updateFilterForConfsAndMetricsToRetrieve(listBasedOnFields); + return listBasedOnFields; + } + + @Override + protected Result getResult(Configuration hbaseConf, Connection conn, + FilterList filterList) throws IOException { + TimelineReaderContext context = getContext(); + ApplicationRowKey applicationRowKey = + new ApplicationRowKey(context.getClusterId(), context.getUserId(), + context.getFlowName(), context.getFlowRunId(), context.getAppId()); + byte[] rowKey = applicationRowKey.getRowKey(); + Get get = new Get(rowKey); + // Set time range for metric values. + setMetricsTimeRange(get); + get.setMaxVersions(getDataToRetrieve().getMetricsLimit()); + if (filterList != null && !filterList.getFilters().isEmpty()) { + get.setFilter(filterList); + } + return getTable().getResult(hbaseConf, conn, get); + } + + @Override + protected void validateParams() { + Preconditions.checkNotNull(getContext(), "context shouldn't be null"); + Preconditions.checkNotNull( + getDataToRetrieve(), "data to retrieve shouldn't be null"); + Preconditions.checkNotNull(getContext().getClusterId(), + "clusterId shouldn't be null"); + Preconditions.checkNotNull(getContext().getEntityType(), + "entityType shouldn't be null"); + if (isSingleEntityRead()) { + Preconditions.checkNotNull(getContext().getAppId(), + "appId shouldn't be null"); + } else { + Preconditions.checkNotNull(getContext().getUserId(), + "userId shouldn't be null"); + Preconditions.checkNotNull(getContext().getFlowName(), + "flowName shouldn't be null"); + } + } + + @Override + protected void augmentParams(Configuration hbaseConf, Connection conn) + throws IOException { + if (isSingleEntityRead()) { + // Get flow context information from AppToFlow table. + defaultAugmentParams(hbaseConf, conn); + } + // Add configs/metrics to fields to retrieve if confsToRetrieve and/or + // metricsToRetrieve are specified. + getDataToRetrieve().addFieldsBasedOnConfsAndMetricsToRetrieve(); + if (!isSingleEntityRead()) { + createFiltersIfNull(); + } + } + + private void setMetricsTimeRange(Query query) { + // Set time range for metric values. + HBaseTimelineStorageUtils.setMetricsTimeRange( + query, ApplicationColumnFamily.METRICS.getBytes(), + getDataToRetrieve().getMetricsTimeBegin(), + getDataToRetrieve().getMetricsTimeEnd()); + } + + @Override + protected ResultScanner getResults(Configuration hbaseConf, + Connection conn, FilterList filterList) throws IOException { + Scan scan = new Scan(); + TimelineReaderContext context = getContext(); + RowKeyPrefix applicationRowKeyPrefix = null; + + // Whether or not flowRunID is null doesn't matter, the + // ApplicationRowKeyPrefix will do the right thing. + // default mode, will always scans from beginning of entity type. + if (getFilters().getFromId() == null) { + applicationRowKeyPrefix = new ApplicationRowKeyPrefix( + context.getClusterId(), context.getUserId(), context.getFlowName(), + context.getFlowRunId()); + scan.setRowPrefixFilter(applicationRowKeyPrefix.getRowKeyPrefix()); + } else { + ApplicationRowKey applicationRowKey = null; + try { + applicationRowKey = + ApplicationRowKey.parseRowKeyFromString(getFilters().getFromId()); + } catch (IllegalArgumentException e) { + throw new BadRequestException("Invalid filter fromid is provided."); + } + if (!context.getClusterId().equals(applicationRowKey.getClusterId())) { + throw new BadRequestException( + "fromid doesn't belong to clusterId=" + context.getClusterId()); + } + + // set start row + scan.setStartRow(applicationRowKey.getRowKey()); + + // get the bytes for stop row + applicationRowKeyPrefix = new ApplicationRowKeyPrefix( + context.getClusterId(), context.getUserId(), context.getFlowName(), + context.getFlowRunId()); + + // set stop row + scan.setStopRow( + HBaseTimelineStorageUtils.calculateTheClosestNextRowKeyForPrefix( + applicationRowKeyPrefix.getRowKeyPrefix())); + } + + FilterList newList = new FilterList(); + newList.addFilter(new PageFilter(getFilters().getLimit())); + if (filterList != null && !filterList.getFilters().isEmpty()) { + newList.addFilter(filterList); + } + scan.setFilter(newList); + + // Set time range for metric values. + setMetricsTimeRange(scan); + scan.setMaxVersions(getDataToRetrieve().getMetricsLimit()); + return getTable().getResultScanner(hbaseConf, conn, scan); + } + + @Override + protected TimelineEntity parseEntity(Result result) throws IOException { + if (result == null || result.isEmpty()) { + return null; + } + TimelineEntity entity = new TimelineEntity(); + entity.setType(TimelineEntityType.YARN_APPLICATION.toString()); + String entityId = ApplicationColumn.ID.readResult(result).toString(); + entity.setId(entityId); + + TimelineEntityFilters filters = getFilters(); + // fetch created time + Long createdTime = (Long) ApplicationColumn.CREATED_TIME.readResult(result); + entity.setCreatedTime(createdTime); + + EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); + // fetch is related to entities and match isRelatedTo filter. If isRelatedTo + // filters do not match, entity would be dropped. We have to match filters + // locally as relevant HBase filters to filter out rows on the basis of + // isRelatedTo are not set in HBase scan. + boolean checkIsRelatedTo = + !isSingleEntityRead() && filters.getIsRelatedTo() != null && + filters.getIsRelatedTo().getFilterList().size() > 0; + if (hasField(fieldsToRetrieve, Field.IS_RELATED_TO) || checkIsRelatedTo) { + readRelationship(entity, result, ApplicationColumnPrefix.IS_RELATED_TO, + true); + if (checkIsRelatedTo && !TimelineStorageUtils.matchIsRelatedTo(entity, + filters.getIsRelatedTo())) { + return null; + } + if (!hasField(fieldsToRetrieve, + Field.IS_RELATED_TO)) { + entity.getIsRelatedToEntities().clear(); + } + } + + // fetch relates to entities and match relatesTo filter. If relatesTo + // filters do not match, entity would be dropped. We have to match filters + // locally as relevant HBase filters to filter out rows on the basis of + // relatesTo are not set in HBase scan. + boolean checkRelatesTo = + !isSingleEntityRead() && filters.getRelatesTo() != null && + filters.getRelatesTo().getFilterList().size() > 0; + if (hasField(fieldsToRetrieve, Field.RELATES_TO) || + checkRelatesTo) { + readRelationship(entity, result, ApplicationColumnPrefix.RELATES_TO, + false); + if (checkRelatesTo && !TimelineStorageUtils.matchRelatesTo(entity, + filters.getRelatesTo())) { + return null; + } + if (!hasField(fieldsToRetrieve, Field.RELATES_TO)) { + entity.getRelatesToEntities().clear(); + } + } + + // fetch info if fieldsToRetrieve contains INFO or ALL. + if (hasField(fieldsToRetrieve, Field.INFO)) { + readKeyValuePairs(entity, result, ApplicationColumnPrefix.INFO, false); + } + + // fetch configs if fieldsToRetrieve contains CONFIGS or ALL. + if (hasField(fieldsToRetrieve, Field.CONFIGS)) { + readKeyValuePairs(entity, result, ApplicationColumnPrefix.CONFIG, true); + } + + // fetch events and match event filters if they exist. If event filters do + // not match, entity would be dropped. We have to match filters locally + // as relevant HBase filters to filter out rows on the basis of events + // are not set in HBase scan. + boolean checkEvents = + !isSingleEntityRead() && filters.getEventFilters() != null && + filters.getEventFilters().getFilterList().size() > 0; + if (hasField(fieldsToRetrieve, Field.EVENTS) || checkEvents) { + readEvents(entity, result, ApplicationColumnPrefix.EVENT); + if (checkEvents && !TimelineStorageUtils.matchEventFilters(entity, + filters.getEventFilters())) { + return null; + } + if (!hasField(fieldsToRetrieve, Field.EVENTS)) { + entity.getEvents().clear(); + } + } + + // fetch metrics if fieldsToRetrieve contains METRICS or ALL. + if (hasField(fieldsToRetrieve, Field.METRICS)) { + readMetrics(entity, result, ApplicationColumnPrefix.METRIC); + } + + ApplicationRowKey rowKey = ApplicationRowKey.parseRowKey(result.getRow()); + entity.getInfo().put(TimelineReaderUtils.FROMID_KEY, + rowKey.getRowKeyAsString()); + return entity; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/EntityTypeReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/EntityTypeReader.java new file mode 100644 index 00000000000..0764775b968 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/EntityTypeReader.java @@ -0,0 +1,176 @@ +/** + * 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.timelineservice.storage.reader; + +import com.google.common.base.Preconditions; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.filter.FilterList; +import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter; +import org.apache.hadoop.hbase.filter.KeyOnlyFilter; +import org.apache.hadoop.hbase.filter.PageFilter; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKeyPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Set; +import java.util.TreeSet; + +/** + * Timeline entity reader for listing all available entity types given one + * reader context. Right now only supports listing all entity types within one + * YARN application. + */ +public final class EntityTypeReader extends AbstractTimelineStorageReader { + + private static final Logger LOG = + LoggerFactory.getLogger(EntityTypeReader.class); + private static final EntityTable ENTITY_TABLE = new EntityTable(); + + public EntityTypeReader(TimelineReaderContext context) { + super(context); + } + + /** + * Reads a set of timeline entity types from the HBase storage for the given + * context. + * + * @param hbaseConf HBase Configuration. + * @param conn HBase Connection. + * @return a set of TimelineEntity objects, with only type field + * set. + * @throws IOException if any exception is encountered while reading entities. + */ + public Set readEntityTypes(Configuration hbaseConf, + Connection conn) throws IOException { + + validateParams(); + augmentParams(hbaseConf, conn); + + Set types = new TreeSet<>(); + TimelineReaderContext context = getContext(); + EntityRowKeyPrefix prefix = new EntityRowKeyPrefix(context.getClusterId(), + context.getUserId(), context.getFlowName(), context.getFlowRunId(), + context.getAppId()); + byte[] currRowKey = prefix.getRowKeyPrefix(); + byte[] nextRowKey = prefix.getRowKeyPrefix(); + nextRowKey[nextRowKey.length - 1]++; + + FilterList typeFilterList = new FilterList(); + typeFilterList.addFilter(new FirstKeyOnlyFilter()); + typeFilterList.addFilter(new KeyOnlyFilter()); + typeFilterList.addFilter(new PageFilter(1)); + LOG.debug("FilterList created for scan is - {}", typeFilterList); + + int counter = 0; + while (true) { + try (ResultScanner results + = getResult(hbaseConf, conn, typeFilterList, currRowKey, nextRowKey)) + { + TimelineEntity entity = parseEntityForType(results.next()); + if (entity == null) { + break; + } + ++counter; + if (!types.add(entity.getType())) { + LOG.warn("Failed to add type " + entity.getType() + + " to the result set because there is a duplicated copy. "); + } + String currType = entity.getType(); + if (LOG.isDebugEnabled()) { + LOG.debug("Current row key: " + Arrays.toString(currRowKey)); + LOG.debug("New entity type discovered: " + currType); + } + currRowKey = getNextRowKey(prefix.getRowKeyPrefix(), currType); + } + } + LOG.debug("Scanned {} records for {} types", counter, types.size()); + return types; + } + + @Override + protected void validateParams() { + Preconditions.checkNotNull(getContext(), "context shouldn't be null"); + Preconditions.checkNotNull(getContext().getClusterId(), + "clusterId shouldn't be null"); + Preconditions.checkNotNull(getContext().getAppId(), + "appId shouldn't be null"); + } + + /** + * Gets the possibly next row key prefix given current prefix and type. + * + * @param currRowKeyPrefix The current prefix that contains user, cluster, + * flow, run, and application id. + * @param entityType Current entity type. + * @return A new prefix for the possibly immediately next row key. + */ + private static byte[] getNextRowKey(byte[] currRowKeyPrefix, + String entityType) { + if (currRowKeyPrefix == null || entityType == null) { + return null; + } + + byte[] entityTypeEncoded = Separator.QUALIFIERS.join( + Separator.encode(entityType, Separator.SPACE, Separator.TAB, + Separator.QUALIFIERS), + Separator.EMPTY_BYTES); + + byte[] currRowKey + = new byte[currRowKeyPrefix.length + entityTypeEncoded.length]; + System.arraycopy(currRowKeyPrefix, 0, currRowKey, 0, + currRowKeyPrefix.length); + System.arraycopy(entityTypeEncoded, 0, currRowKey, currRowKeyPrefix.length, + entityTypeEncoded.length); + + return HBaseTimelineStorageUtils.calculateTheClosestNextRowKeyForPrefix( + currRowKey); + } + + private ResultScanner getResult(Configuration hbaseConf, Connection conn, + FilterList filterList, byte[] startPrefix, byte[] endPrefix) + throws IOException { + Scan scan = new Scan(startPrefix, endPrefix); + scan.setFilter(filterList); + scan.setSmall(true); + return ENTITY_TABLE.getResultScanner(hbaseConf, conn, scan); + } + + private TimelineEntity parseEntityForType(Result result) + throws IOException { + if (result == null || result.isEmpty()) { + return null; + } + TimelineEntity entity = new TimelineEntity(); + EntityRowKey newRowKey = EntityRowKey.parseRowKey(result.getRow()); + entity.setType(newRowKey.getEntityType()); + return entity; + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowActivityEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowActivityEntityReader.java new file mode 100644 index 00000000000..a1cdb29b124 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowActivityEntityReader.java @@ -0,0 +1,185 @@ +/** + * 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.timelineservice.storage.reader; + +import java.io.IOException; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.filter.FilterList; +import org.apache.hadoop.hbase.filter.PageFilter; +import org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongKeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKeyPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTable; +import org.apache.hadoop.yarn.webapp.BadRequestException; + +import com.google.common.base.Preconditions; + +/** + * Timeline entity reader for flow activity entities that are stored in the + * flow activity table. + */ +class FlowActivityEntityReader extends TimelineEntityReader { + private static final FlowActivityTable FLOW_ACTIVITY_TABLE = + new FlowActivityTable(); + + /** + * Used to convert Long key components to and from storage format. + */ + private final KeyConverter longKeyConverter = new LongKeyConverter(); + + + public FlowActivityEntityReader(TimelineReaderContext ctxt, + TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) { + super(ctxt, entityFilters, toRetrieve); + } + + public FlowActivityEntityReader(TimelineReaderContext ctxt, + TimelineDataToRetrieve toRetrieve) { + super(ctxt, toRetrieve); + } + + /** + * Uses the {@link FlowActivityTable}. + */ + @Override + protected BaseTable getTable() { + return FLOW_ACTIVITY_TABLE; + } + + @Override + protected void validateParams() { + Preconditions.checkNotNull(getContext().getClusterId(), + "clusterId shouldn't be null"); + } + + @Override + protected void augmentParams(Configuration hbaseConf, Connection conn) + throws IOException { + createFiltersIfNull(); + } + + @Override + protected FilterList constructFilterListBasedOnFilters() throws IOException { + return null; + } + + @Override + protected FilterList constructFilterListBasedOnFields() { + return null; + } + + @Override + protected Result getResult(Configuration hbaseConf, Connection conn, + FilterList filterList) throws IOException { + throw new UnsupportedOperationException( + "we don't support a single entity query"); + } + + @Override + protected ResultScanner getResults(Configuration hbaseConf, + Connection conn, FilterList filterList) throws IOException { + Scan scan = new Scan(); + String clusterId = getContext().getClusterId(); + if (getFilters().getFromId() == null + && getFilters().getCreatedTimeBegin() == 0L + && getFilters().getCreatedTimeEnd() == Long.MAX_VALUE) { + // All records have to be chosen. + scan.setRowPrefixFilter(new FlowActivityRowKeyPrefix(clusterId) + .getRowKeyPrefix()); + } else if (getFilters().getFromId() != null) { + FlowActivityRowKey key = null; + try { + key = + FlowActivityRowKey.parseRowKeyFromString(getFilters().getFromId()); + } catch (IllegalArgumentException e) { + throw new BadRequestException("Invalid filter fromid is provided."); + } + if (!clusterId.equals(key.getClusterId())) { + throw new BadRequestException( + "fromid doesn't belong to clusterId=" + clusterId); + } + scan.setStartRow(key.getRowKey()); + scan.setStopRow( + new FlowActivityRowKeyPrefix(clusterId, + (getFilters().getCreatedTimeBegin() <= 0 ? 0 + : (getFilters().getCreatedTimeBegin() - 1))) + .getRowKeyPrefix()); + } else { + scan.setStartRow(new FlowActivityRowKeyPrefix(clusterId, getFilters() + .getCreatedTimeEnd()).getRowKeyPrefix()); + scan.setStopRow(new FlowActivityRowKeyPrefix(clusterId, (getFilters() + .getCreatedTimeBegin() <= 0 ? 0 + : (getFilters().getCreatedTimeBegin() - 1))).getRowKeyPrefix()); + } + // use the page filter to limit the result to the page size + // the scanner may still return more than the limit; therefore we need to + // read the right number as we iterate + scan.setFilter(new PageFilter(getFilters().getLimit())); + return getTable().getResultScanner(hbaseConf, conn, scan); + } + + @Override + protected TimelineEntity parseEntity(Result result) throws IOException { + FlowActivityRowKey rowKey = FlowActivityRowKey.parseRowKey(result.getRow()); + + Long time = rowKey.getDayTimestamp(); + String user = rowKey.getUserId(); + String flowName = rowKey.getFlowName(); + + FlowActivityEntity flowActivity = new FlowActivityEntity( + getContext().getClusterId(), time, user, flowName); + // set the id + flowActivity.setId(flowActivity.getId()); + // get the list of run ids along with the version that are associated with + // this flow on this day + Map runIdsMap = + FlowActivityColumnPrefix.RUN_ID.readResults(result, longKeyConverter); + for (Map.Entry e : runIdsMap.entrySet()) { + Long runId = e.getKey(); + String version = (String)e.getValue(); + FlowRunEntity flowRun = new FlowRunEntity(); + flowRun.setUser(user); + flowRun.setName(flowName); + flowRun.setRunId(runId); + flowRun.setVersion(version); + // set the id + flowRun.setId(flowRun.getId()); + flowActivity.addFlowRun(flowRun); + } + flowActivity.getInfo().put(TimelineReaderUtils.FROMID_KEY, + rowKey.getRowKeyAsString()); + return flowActivity; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java new file mode 100644 index 00000000000..af043b3bc7a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java @@ -0,0 +1,294 @@ +/** + * 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.timelineservice.storage.reader; + +import java.io.IOException; +import java.util.EnumSet; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.filter.BinaryComparator; +import org.apache.hadoop.hbase.filter.BinaryPrefixComparator; +import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; +import org.apache.hadoop.hbase.filter.FamilyFilter; +import org.apache.hadoop.hbase.filter.FilterList; +import org.apache.hadoop.hbase.filter.FilterList.Operator; +import org.apache.hadoop.hbase.filter.PageFilter; +import org.apache.hadoop.hbase.filter.QualifierFilter; +import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderUtils; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumn; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKeyPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable; +import org.apache.hadoop.yarn.webapp.BadRequestException; + +import com.google.common.base.Preconditions; + +/** + * Timeline entity reader for flow run entities that are stored in the flow run + * table. + */ +class FlowRunEntityReader extends TimelineEntityReader { + private static final FlowRunTable FLOW_RUN_TABLE = new FlowRunTable(); + + public FlowRunEntityReader(TimelineReaderContext ctxt, + TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) { + super(ctxt, entityFilters, toRetrieve); + } + + public FlowRunEntityReader(TimelineReaderContext ctxt, + TimelineDataToRetrieve toRetrieve) { + super(ctxt, toRetrieve); + } + + /** + * Uses the {@link FlowRunTable}. + */ + @Override + protected BaseTable getTable() { + return FLOW_RUN_TABLE; + } + + @Override + protected void validateParams() { + Preconditions.checkNotNull(getContext(), "context shouldn't be null"); + Preconditions.checkNotNull(getDataToRetrieve(), + "data to retrieve shouldn't be null"); + Preconditions.checkNotNull(getContext().getClusterId(), + "clusterId shouldn't be null"); + Preconditions.checkNotNull(getContext().getUserId(), + "userId shouldn't be null"); + Preconditions.checkNotNull(getContext().getFlowName(), + "flowName shouldn't be null"); + if (isSingleEntityRead()) { + Preconditions.checkNotNull(getContext().getFlowRunId(), + "flowRunId shouldn't be null"); + } + EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); + if (!isSingleEntityRead() && fieldsToRetrieve != null) { + for (Field field : fieldsToRetrieve) { + if (field != Field.ALL && field != Field.METRICS) { + throw new BadRequestException("Invalid field " + field + + " specified while querying flow runs."); + } + } + } + } + + @Override + protected void augmentParams(Configuration hbaseConf, Connection conn) { + // Add metrics to fields to retrieve if metricsToRetrieve is specified. + getDataToRetrieve().addFieldsBasedOnConfsAndMetricsToRetrieve(); + if (!isSingleEntityRead()) { + createFiltersIfNull(); + } + } + + protected FilterList constructFilterListBasedOnFilters() throws IOException { + FilterList listBasedOnFilters = new FilterList(); + // Filter based on created time range. + Long createdTimeBegin = getFilters().getCreatedTimeBegin(); + Long createdTimeEnd = getFilters().getCreatedTimeEnd(); + if (createdTimeBegin != 0 || createdTimeEnd != Long.MAX_VALUE) { + listBasedOnFilters.addFilter(TimelineFilterUtils + .createSingleColValueFiltersByRange(FlowRunColumn.MIN_START_TIME, + createdTimeBegin, createdTimeEnd)); + } + // Filter based on metric filters. + TimelineFilterList metricFilters = getFilters().getMetricFilters(); + if (metricFilters != null && !metricFilters.getFilterList().isEmpty()) { + listBasedOnFilters.addFilter(TimelineFilterUtils.createHBaseFilterList( + FlowRunColumnPrefix.METRIC, metricFilters)); + } + return listBasedOnFilters; + } + + /** + * Add {@link QualifierFilter} filters to filter list for each column of flow + * run table. + * + * @return filter list to which qualifier filters have been added. + */ + private FilterList updateFixedColumns() { + FilterList columnsList = new FilterList(Operator.MUST_PASS_ONE); + for (FlowRunColumn column : FlowRunColumn.values()) { + columnsList.addFilter(new QualifierFilter(CompareOp.EQUAL, + new BinaryComparator(column.getColumnQualifierBytes()))); + } + return columnsList; + } + + @Override + protected FilterList constructFilterListBasedOnFields() throws IOException { + FilterList list = new FilterList(Operator.MUST_PASS_ONE); + // By default fetch everything in INFO column family. + FamilyFilter infoColumnFamily = + new FamilyFilter(CompareOp.EQUAL, new BinaryComparator( + FlowRunColumnFamily.INFO.getBytes())); + TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve(); + // If multiple entities have to be retrieved, check if metrics have to be + // retrieved and if not, add a filter so that metrics can be excluded. + // Metrics are always returned if we are reading a single entity. + if (!isSingleEntityRead() + && !hasField(dataToRetrieve.getFieldsToRetrieve(), Field.METRICS)) { + FilterList infoColFamilyList = new FilterList(Operator.MUST_PASS_ONE); + infoColFamilyList.addFilter(infoColumnFamily); + infoColFamilyList.addFilter(new QualifierFilter(CompareOp.NOT_EQUAL, + new BinaryPrefixComparator(FlowRunColumnPrefix.METRIC + .getColumnPrefixBytes("")))); + list.addFilter(infoColFamilyList); + } else { + // Check if metricsToRetrieve are specified and if they are, create a + // filter list for info column family by adding flow run tables columns + // and a list for metrics to retrieve. Pls note that fieldsToRetrieve + // will have METRICS added to it if metricsToRetrieve are specified + // (in augmentParams()). + TimelineFilterList metricsToRetrieve = + dataToRetrieve.getMetricsToRetrieve(); + if (metricsToRetrieve != null + && !metricsToRetrieve.getFilterList().isEmpty()) { + FilterList infoColFamilyList = new FilterList(); + infoColFamilyList.addFilter(infoColumnFamily); + FilterList columnsList = updateFixedColumns(); + columnsList.addFilter(TimelineFilterUtils.createHBaseFilterList( + FlowRunColumnPrefix.METRIC, metricsToRetrieve)); + infoColFamilyList.addFilter(columnsList); + list.addFilter(infoColFamilyList); + } + } + return list; + } + + @Override + protected Result getResult(Configuration hbaseConf, Connection conn, + FilterList filterList) throws IOException { + TimelineReaderContext context = getContext(); + FlowRunRowKey flowRunRowKey = + new FlowRunRowKey(context.getClusterId(), context.getUserId(), + context.getFlowName(), context.getFlowRunId()); + byte[] rowKey = flowRunRowKey.getRowKey(); + Get get = new Get(rowKey); + get.setMaxVersions(Integer.MAX_VALUE); + if (filterList != null && !filterList.getFilters().isEmpty()) { + get.setFilter(filterList); + } + return getTable().getResult(hbaseConf, conn, get); + } + + @Override + protected ResultScanner getResults(Configuration hbaseConf, Connection conn, + FilterList filterList) throws IOException { + Scan scan = new Scan(); + TimelineReaderContext context = getContext(); + RowKeyPrefix flowRunRowKeyPrefix = null; + if (getFilters().getFromId() == null) { + flowRunRowKeyPrefix = new FlowRunRowKeyPrefix(context.getClusterId(), + context.getUserId(), context.getFlowName()); + scan.setRowPrefixFilter(flowRunRowKeyPrefix.getRowKeyPrefix()); + } else { + FlowRunRowKey flowRunRowKey = null; + try { + flowRunRowKey = + FlowRunRowKey.parseRowKeyFromString(getFilters().getFromId()); + } catch (IllegalArgumentException e) { + throw new BadRequestException("Invalid filter fromid is provided."); + } + if (!context.getClusterId().equals(flowRunRowKey.getClusterId())) { + throw new BadRequestException( + "fromid doesn't belong to clusterId=" + context.getClusterId()); + } + // set start row + scan.setStartRow(flowRunRowKey.getRowKey()); + + // get the bytes for stop row + flowRunRowKeyPrefix = new FlowRunRowKeyPrefix(context.getClusterId(), + context.getUserId(), context.getFlowName()); + + // set stop row + scan.setStopRow( + HBaseTimelineStorageUtils.calculateTheClosestNextRowKeyForPrefix( + flowRunRowKeyPrefix.getRowKeyPrefix())); + } + + FilterList newList = new FilterList(); + newList.addFilter(new PageFilter(getFilters().getLimit())); + if (filterList != null && !filterList.getFilters().isEmpty()) { + newList.addFilter(filterList); + } + scan.setFilter(newList); + scan.setMaxVersions(Integer.MAX_VALUE); + return getTable().getResultScanner(hbaseConf, conn, scan); + } + + @Override + protected TimelineEntity parseEntity(Result result) throws IOException { + FlowRunEntity flowRun = new FlowRunEntity(); + FlowRunRowKey rowKey = FlowRunRowKey.parseRowKey(result.getRow()); + flowRun.setRunId(rowKey.getFlowRunId()); + flowRun.setUser(rowKey.getUserId()); + flowRun.setName(rowKey.getFlowName()); + + // read the start time + Long startTime = (Long) FlowRunColumn.MIN_START_TIME.readResult(result); + if (startTime != null) { + flowRun.setStartTime(startTime.longValue()); + } + + // read the end time if available + Long endTime = (Long) FlowRunColumn.MAX_END_TIME.readResult(result); + if (endTime != null) { + flowRun.setMaxEndTime(endTime.longValue()); + } + + // read the flow version + String version = (String) FlowRunColumn.FLOW_VERSION.readResult(result); + if (version != null) { + flowRun.setVersion(version); + } + + // read metrics if its a single entity query or if METRICS are part of + // fieldsToRetrieve. + if (isSingleEntityRead() + || hasField(getDataToRetrieve().getFieldsToRetrieve(), Field.METRICS)) { + readMetrics(flowRun, result, FlowRunColumnPrefix.METRIC); + } + + // set the id + flowRun.setId(flowRun.getId()); + flowRun.getInfo().put(TimelineReaderUtils.FROMID_KEY, + rowKey.getRowKeyAsString()); + return flowRun; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java new file mode 100644 index 00000000000..3a444455bd9 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java @@ -0,0 +1,651 @@ +/** + * 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.timelineservice.storage.reader; + +import java.io.IOException; +import java.util.EnumSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Query; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.filter.BinaryComparator; +import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; +import org.apache.hadoop.hbase.filter.FamilyFilter; +import org.apache.hadoop.hbase.filter.FilterList; +import org.apache.hadoop.hbase.filter.PageFilter; +import org.apache.hadoop.hbase.filter.FilterList.Operator; +import org.apache.hadoop.hbase.filter.QualifierFilter; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderUtils; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKeyPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable; +import org.apache.hadoop.yarn.webapp.BadRequestException; + +import com.google.common.base.Preconditions; + +/** + * Timeline entity reader for generic entities that are stored in the entity + * table. + */ +class GenericEntityReader extends TimelineEntityReader { + private static final EntityTable ENTITY_TABLE = new EntityTable(); + + /** + * Used to convert strings key components to and from storage format. + */ + private final KeyConverter stringKeyConverter = + new StringKeyConverter(); + + public GenericEntityReader(TimelineReaderContext ctxt, + TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) { + super(ctxt, entityFilters, toRetrieve); + } + + public GenericEntityReader(TimelineReaderContext ctxt, + TimelineDataToRetrieve toRetrieve) { + super(ctxt, toRetrieve); + } + + /** + * Uses the {@link EntityTable}. + */ + protected BaseTable getTable() { + return ENTITY_TABLE; + } + + @Override + protected FilterList constructFilterListBasedOnFilters() throws IOException { + // Filters here cannot be null for multiple entity reads as they are set in + // augmentParams if null. + FilterList listBasedOnFilters = new FilterList(); + TimelineEntityFilters filters = getFilters(); + // Create filter list based on created time range and add it to + // listBasedOnFilters. + long createdTimeBegin = filters.getCreatedTimeBegin(); + long createdTimeEnd = filters.getCreatedTimeEnd(); + if (createdTimeBegin != 0 || createdTimeEnd != Long.MAX_VALUE) { + listBasedOnFilters.addFilter(TimelineFilterUtils + .createSingleColValueFiltersByRange(EntityColumn.CREATED_TIME, + createdTimeBegin, createdTimeEnd)); + } + // Create filter list based on metric filters and add it to + // listBasedOnFilters. + TimelineFilterList metricFilters = filters.getMetricFilters(); + if (metricFilters != null && !metricFilters.getFilterList().isEmpty()) { + listBasedOnFilters.addFilter(TimelineFilterUtils.createHBaseFilterList( + EntityColumnPrefix.METRIC, metricFilters)); + } + // Create filter list based on config filters and add it to + // listBasedOnFilters. + TimelineFilterList configFilters = filters.getConfigFilters(); + if (configFilters != null && !configFilters.getFilterList().isEmpty()) { + listBasedOnFilters.addFilter(TimelineFilterUtils.createHBaseFilterList( + EntityColumnPrefix.CONFIG, configFilters)); + } + // Create filter list based on info filters and add it to listBasedOnFilters + TimelineFilterList infoFilters = filters.getInfoFilters(); + if (infoFilters != null && !infoFilters.getFilterList().isEmpty()) { + listBasedOnFilters.addFilter(TimelineFilterUtils.createHBaseFilterList( + EntityColumnPrefix.INFO, infoFilters)); + } + return listBasedOnFilters; + } + + /** + * Check if we need to fetch only some of the event columns. + * + * @return true if we need to fetch some of the columns, false otherwise. + */ + protected boolean fetchPartialEventCols(TimelineFilterList eventFilters, + EnumSet fieldsToRetrieve) { + return (eventFilters != null && !eventFilters.getFilterList().isEmpty() && + !hasField(fieldsToRetrieve, Field.EVENTS)); + } + + /** + * Check if we need to fetch only some of the relates_to columns. + * + * @return true if we need to fetch some of the columns, false otherwise. + */ + protected boolean fetchPartialRelatesToCols(TimelineFilterList relatesTo, + EnumSet fieldsToRetrieve) { + return (relatesTo != null && !relatesTo.getFilterList().isEmpty() && + !hasField(fieldsToRetrieve, Field.RELATES_TO)); + } + + /** + * Check if we need to fetch only some of the is_related_to columns. + * + * @return true if we need to fetch some of the columns, false otherwise. + */ + private boolean fetchPartialIsRelatedToCols(TimelineFilterList isRelatedTo, + EnumSet fieldsToRetrieve) { + return (isRelatedTo != null && !isRelatedTo.getFilterList().isEmpty() && + !hasField(fieldsToRetrieve, Field.IS_RELATED_TO)); + } + + /** + * Check if we need to fetch only some of the columns based on event filters, + * relatesto and isrelatedto from info family. + * + * @return true, if we need to fetch only some of the columns, false if we + * need to fetch all the columns under info column family. + */ + protected boolean fetchPartialColsFromInfoFamily() { + EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); + TimelineEntityFilters filters = getFilters(); + return fetchPartialEventCols(filters.getEventFilters(), fieldsToRetrieve) + || fetchPartialRelatesToCols(filters.getRelatesTo(), fieldsToRetrieve) + || fetchPartialIsRelatedToCols(filters.getIsRelatedTo(), + fieldsToRetrieve); + } + + /** + * Check if we need to create filter list based on fields. We need to create a + * filter list iff all fields need not be retrieved or we have some specific + * fields or metrics to retrieve. We also need to create a filter list if we + * have relationships(relatesTo/isRelatedTo) and event filters specified for + * the query. + * + * @return true if we need to create the filter list, false otherwise. + */ + protected boolean needCreateFilterListBasedOnFields() { + TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve(); + // Check if all fields are to be retrieved or not. If all fields have to + // be retrieved, also check if we have some metrics or configs to + // retrieve specified for the query because then a filter list will have + // to be created. + boolean flag = + !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) + || (dataToRetrieve.getConfsToRetrieve() != null && !dataToRetrieve + .getConfsToRetrieve().getFilterList().isEmpty()) + || (dataToRetrieve.getMetricsToRetrieve() != null && !dataToRetrieve + .getMetricsToRetrieve().getFilterList().isEmpty()); + // Filters need to be checked only if we are reading multiple entities. If + // condition above is false, we check if there are relationships(relatesTo/ + // isRelatedTo) and event filters specified for the query. + if (!flag && !isSingleEntityRead()) { + TimelineEntityFilters filters = getFilters(); + flag = + (filters.getEventFilters() != null && !filters.getEventFilters() + .getFilterList().isEmpty()) + || (filters.getIsRelatedTo() != null && !filters.getIsRelatedTo() + .getFilterList().isEmpty()) + || (filters.getRelatesTo() != null && !filters.getRelatesTo() + .getFilterList().isEmpty()); + } + return flag; + } + + /** + * Add {@link QualifierFilter} filters to filter list for each column of + * entity table. + * + * @param list filter list to which qualifier filters have to be added. + */ + protected void updateFixedColumns(FilterList list) { + for (EntityColumn column : EntityColumn.values()) { + list.addFilter(new QualifierFilter(CompareOp.EQUAL, new BinaryComparator( + column.getColumnQualifierBytes()))); + } + } + + /** + * Creates a filter list which indicates that only some of the column + * qualifiers in the info column family will be returned in result. + * + * @param isApplication If true, it means operations are to be performed for + * application table, otherwise for entity table. + * @return filter list. + * @throws IOException if any problem occurs while creating filter list. + */ + private FilterList createFilterListForColsOfInfoFamily() throws IOException { + FilterList infoFamilyColsFilter = new FilterList(Operator.MUST_PASS_ONE); + // Add filters for each column in entity table. + updateFixedColumns(infoFamilyColsFilter); + EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); + // If INFO field has to be retrieved, add a filter for fetching columns + // with INFO column prefix. + if (hasField(fieldsToRetrieve, Field.INFO)) { + infoFamilyColsFilter + .addFilter(TimelineFilterUtils.createHBaseQualifierFilter( + CompareOp.EQUAL, EntityColumnPrefix.INFO)); + } + TimelineFilterList relatesTo = getFilters().getRelatesTo(); + if (hasField(fieldsToRetrieve, Field.RELATES_TO)) { + // If RELATES_TO field has to be retrieved, add a filter for fetching + // columns with RELATES_TO column prefix. + infoFamilyColsFilter.addFilter(TimelineFilterUtils + .createHBaseQualifierFilter(CompareOp.EQUAL, + EntityColumnPrefix.RELATES_TO)); + } else if (relatesTo != null && !relatesTo.getFilterList().isEmpty()) { + // Even if fields to retrieve does not contain RELATES_TO, we still + // need to have a filter to fetch some of the column qualifiers if + // relatesTo filters are specified. relatesTo filters will then be + // matched after fetching rows from HBase. + Set relatesToCols = + TimelineFilterUtils.fetchColumnsFromFilterList(relatesTo); + infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers( + EntityColumnPrefix.RELATES_TO, relatesToCols)); + } + TimelineFilterList isRelatedTo = getFilters().getIsRelatedTo(); + if (hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) { + // If IS_RELATED_TO field has to be retrieved, add a filter for fetching + // columns with IS_RELATED_TO column prefix. + infoFamilyColsFilter.addFilter(TimelineFilterUtils + .createHBaseQualifierFilter(CompareOp.EQUAL, + EntityColumnPrefix.IS_RELATED_TO)); + } else if (isRelatedTo != null && !isRelatedTo.getFilterList().isEmpty()) { + // Even if fields to retrieve does not contain IS_RELATED_TO, we still + // need to have a filter to fetch some of the column qualifiers if + // isRelatedTo filters are specified. isRelatedTo filters will then be + // matched after fetching rows from HBase. + Set isRelatedToCols = + TimelineFilterUtils.fetchColumnsFromFilterList(isRelatedTo); + infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers( + EntityColumnPrefix.IS_RELATED_TO, isRelatedToCols)); + } + TimelineFilterList eventFilters = getFilters().getEventFilters(); + if (hasField(fieldsToRetrieve, Field.EVENTS)) { + // If EVENTS field has to be retrieved, add a filter for fetching columns + // with EVENT column prefix. + infoFamilyColsFilter + .addFilter(TimelineFilterUtils.createHBaseQualifierFilter( + CompareOp.EQUAL, EntityColumnPrefix.EVENT)); + } else if (eventFilters != null && + !eventFilters.getFilterList().isEmpty()) { + // Even if fields to retrieve does not contain EVENTS, we still need to + // have a filter to fetch some of the column qualifiers on the basis of + // event filters specified. Event filters will then be matched after + // fetching rows from HBase. + Set eventCols = + TimelineFilterUtils.fetchColumnsFromFilterList(eventFilters); + infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers( + EntityColumnPrefix.EVENT, eventCols)); + } + return infoFamilyColsFilter; + } + + /** + * Exclude column prefixes via filters which are not required(based on fields + * to retrieve) from info column family. These filters are added to filter + * list which contains a filter for getting info column family. + * + * @param infoColFamilyList filter list for info column family. + */ + private void excludeFieldsFromInfoColFamily(FilterList infoColFamilyList) { + EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); + // Events not required. + if (!hasField(fieldsToRetrieve, Field.EVENTS)) { + infoColFamilyList.addFilter(TimelineFilterUtils + .createHBaseQualifierFilter(CompareOp.NOT_EQUAL, + EntityColumnPrefix.EVENT)); + } + // info not required. + if (!hasField(fieldsToRetrieve, Field.INFO)) { + infoColFamilyList.addFilter(TimelineFilterUtils + .createHBaseQualifierFilter(CompareOp.NOT_EQUAL, + EntityColumnPrefix.INFO)); + } + // is related to not required. + if (!hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) { + infoColFamilyList.addFilter(TimelineFilterUtils + .createHBaseQualifierFilter(CompareOp.NOT_EQUAL, + EntityColumnPrefix.IS_RELATED_TO)); + } + // relates to not required. + if (!hasField(fieldsToRetrieve, Field.RELATES_TO)) { + infoColFamilyList.addFilter(TimelineFilterUtils + .createHBaseQualifierFilter(CompareOp.NOT_EQUAL, + EntityColumnPrefix.RELATES_TO)); + } + } + + /** + * Updates filter list based on fields for confs and metrics to retrieve. + * + * @param listBasedOnFields filter list based on fields. + * @throws IOException if any problem occurs while updating filter list. + */ + private void updateFilterForConfsAndMetricsToRetrieve( + FilterList listBasedOnFields) throws IOException { + TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve(); + // Please note that if confsToRetrieve is specified, we would have added + // CONFS to fields to retrieve in augmentParams() even if not specified. + if (dataToRetrieve.getFieldsToRetrieve().contains(Field.CONFIGS)) { + // Create a filter list for configs. + listBasedOnFields.addFilter(TimelineFilterUtils + .createFilterForConfsOrMetricsToRetrieve( + dataToRetrieve.getConfsToRetrieve(), EntityColumnFamily.CONFIGS, + EntityColumnPrefix.CONFIG)); + } + + // Please note that if metricsToRetrieve is specified, we would have added + // METRICS to fields to retrieve in augmentParams() even if not specified. + if (dataToRetrieve.getFieldsToRetrieve().contains(Field.METRICS)) { + // Create a filter list for metrics. + listBasedOnFields.addFilter(TimelineFilterUtils + .createFilterForConfsOrMetricsToRetrieve( + dataToRetrieve.getMetricsToRetrieve(), + EntityColumnFamily.METRICS, EntityColumnPrefix.METRIC)); + } + } + + @Override + protected FilterList constructFilterListBasedOnFields() throws IOException { + if (!needCreateFilterListBasedOnFields()) { + // Fetch all the columns. No need of a filter. + return null; + } + FilterList listBasedOnFields = new FilterList(Operator.MUST_PASS_ONE); + FilterList infoColFamilyList = new FilterList(); + // By default fetch everything in INFO column family. + FamilyFilter infoColumnFamily = + new FamilyFilter(CompareOp.EQUAL, new BinaryComparator( + EntityColumnFamily.INFO.getBytes())); + infoColFamilyList.addFilter(infoColumnFamily); + if (!isSingleEntityRead() && fetchPartialColsFromInfoFamily()) { + // We can fetch only some of the columns from info family. + infoColFamilyList.addFilter(createFilterListForColsOfInfoFamily()); + } else { + // Exclude column prefixes in info column family which are not required + // based on fields to retrieve. + excludeFieldsFromInfoColFamily(infoColFamilyList); + } + listBasedOnFields.addFilter(infoColFamilyList); + updateFilterForConfsAndMetricsToRetrieve(listBasedOnFields); + return listBasedOnFields; + } + + @Override + protected void validateParams() { + Preconditions.checkNotNull(getContext(), "context shouldn't be null"); + Preconditions.checkNotNull(getDataToRetrieve(), + "data to retrieve shouldn't be null"); + Preconditions.checkNotNull(getContext().getClusterId(), + "clusterId shouldn't be null"); + Preconditions.checkNotNull(getContext().getAppId(), + "appId shouldn't be null"); + Preconditions.checkNotNull(getContext().getEntityType(), + "entityType shouldn't be null"); + if (isSingleEntityRead()) { + Preconditions.checkNotNull(getContext().getEntityId(), + "entityId shouldn't be null"); + } + } + + @Override + protected void augmentParams(Configuration hbaseConf, Connection conn) + throws IOException { + defaultAugmentParams(hbaseConf, conn); + // Add configs/metrics to fields to retrieve if confsToRetrieve and/or + // metricsToRetrieve are specified. + getDataToRetrieve().addFieldsBasedOnConfsAndMetricsToRetrieve(); + if (!isSingleEntityRead()) { + createFiltersIfNull(); + } + } + + @Override + protected Result getResult(Configuration hbaseConf, Connection conn, + FilterList filterList) throws IOException { + TimelineReaderContext context = getContext(); + Result result = null; + if (context.getEntityIdPrefix() != null) { + byte[] rowKey = new EntityRowKey(context.getClusterId(), + context.getUserId(), context.getFlowName(), context.getFlowRunId(), + context.getAppId(), context.getEntityType(), + context.getEntityIdPrefix(), context.getEntityId()).getRowKey(); + Get get = new Get(rowKey); + setMetricsTimeRange(get); + get.setMaxVersions(getDataToRetrieve().getMetricsLimit()); + if (filterList != null && !filterList.getFilters().isEmpty()) { + get.setFilter(filterList); + } + result = getTable().getResult(hbaseConf, conn, get); + + } else { + // Prepare for range scan + // create single SingleColumnValueFilter and add to existing filters. + FilterList filter = new FilterList(Operator.MUST_PASS_ALL); + if (filterList != null && !filterList.getFilters().isEmpty()) { + filter.addFilter(filterList); + } + FilterList newFilter = new FilterList(); + newFilter.addFilter(TimelineFilterUtils.createHBaseSingleColValueFilter( + EntityColumn.ID, context.getEntityId(), CompareOp.EQUAL)); + newFilter.addFilter(new PageFilter(1)); + filter.addFilter(newFilter); + + ResultScanner results = getResults(hbaseConf, conn, filter); + try { + Iterator iterator = results.iterator(); + if (iterator.hasNext()) { + result = iterator.next(); + } + } finally { + results.close(); + } + } + return result; + } + + private void setMetricsTimeRange(Query query) { + // Set time range for metric values. + HBaseTimelineStorageUtils.setMetricsTimeRange( + query, EntityColumnFamily.METRICS.getBytes(), + getDataToRetrieve().getMetricsTimeBegin(), + getDataToRetrieve().getMetricsTimeEnd()); + } + + @Override + protected ResultScanner getResults(Configuration hbaseConf, Connection conn, + FilterList filterList) throws IOException { + // Scan through part of the table to find the entities belong to one app + // and one type + Scan scan = new Scan(); + TimelineReaderContext context = getContext(); + RowKeyPrefix entityRowKeyPrefix = null; + // default mode, will always scans from beginning of entity type. + if (getFilters() == null || getFilters().getFromId() == null) { + entityRowKeyPrefix = new EntityRowKeyPrefix(context.getClusterId(), + context.getUserId(), context.getFlowName(), context.getFlowRunId(), + context.getAppId(), context.getEntityType(), null, null); + scan.setRowPrefixFilter(entityRowKeyPrefix.getRowKeyPrefix()); + } else { // pagination mode, will scan from given entityIdPrefix!enitityId + + EntityRowKey entityRowKey = null; + try { + entityRowKey = + EntityRowKey.parseRowKeyFromString(getFilters().getFromId()); + } catch (IllegalArgumentException e) { + throw new BadRequestException("Invalid filter fromid is provided."); + } + if (!context.getClusterId().equals(entityRowKey.getClusterId())) { + throw new BadRequestException( + "fromid doesn't belong to clusterId=" + context.getClusterId()); + } + + // set start row + scan.setStartRow(entityRowKey.getRowKey()); + + // get the bytes for stop row + entityRowKeyPrefix = new EntityRowKeyPrefix(context.getClusterId(), + context.getUserId(), context.getFlowName(), context.getFlowRunId(), + context.getAppId(), context.getEntityType(), null, null); + + // set stop row + scan.setStopRow( + HBaseTimelineStorageUtils.calculateTheClosestNextRowKeyForPrefix( + entityRowKeyPrefix.getRowKeyPrefix())); + + // set page filter to limit. This filter has to set only in pagination + // mode. + filterList.addFilter(new PageFilter(getFilters().getLimit())); + } + setMetricsTimeRange(scan); + scan.setMaxVersions(getDataToRetrieve().getMetricsLimit()); + if (filterList != null && !filterList.getFilters().isEmpty()) { + scan.setFilter(filterList); + } + return getTable().getResultScanner(hbaseConf, conn, scan); + } + + @Override + protected TimelineEntity parseEntity(Result result) throws IOException { + if (result == null || result.isEmpty()) { + return null; + } + TimelineEntity entity = new TimelineEntity(); + EntityRowKey parseRowKey = EntityRowKey.parseRowKey(result.getRow()); + entity.setType(parseRowKey.getEntityType()); + entity.setId(parseRowKey.getEntityId()); + entity.setIdPrefix(parseRowKey.getEntityIdPrefix().longValue()); + + TimelineEntityFilters filters = getFilters(); + // fetch created time + Long createdTime = (Long) EntityColumn.CREATED_TIME.readResult(result); + entity.setCreatedTime(createdTime); + + EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); + // fetch is related to entities and match isRelatedTo filter. If isRelatedTo + // filters do not match, entity would be dropped. We have to match filters + // locally as relevant HBase filters to filter out rows on the basis of + // isRelatedTo are not set in HBase scan. + boolean checkIsRelatedTo = + !isSingleEntityRead() && filters.getIsRelatedTo() != null + && filters.getIsRelatedTo().getFilterList().size() > 0; + if (hasField(fieldsToRetrieve, Field.IS_RELATED_TO) || checkIsRelatedTo) { + readRelationship(entity, result, EntityColumnPrefix.IS_RELATED_TO, true); + if (checkIsRelatedTo + && !TimelineStorageUtils.matchIsRelatedTo(entity, + filters.getIsRelatedTo())) { + return null; + } + if (!hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) { + entity.getIsRelatedToEntities().clear(); + } + } + + // fetch relates to entities and match relatesTo filter. If relatesTo + // filters do not match, entity would be dropped. We have to match filters + // locally as relevant HBase filters to filter out rows on the basis of + // relatesTo are not set in HBase scan. + boolean checkRelatesTo = + !isSingleEntityRead() && filters.getRelatesTo() != null + && filters.getRelatesTo().getFilterList().size() > 0; + if (hasField(fieldsToRetrieve, Field.RELATES_TO) + || checkRelatesTo) { + readRelationship(entity, result, EntityColumnPrefix.RELATES_TO, false); + if (checkRelatesTo + && !TimelineStorageUtils.matchRelatesTo(entity, + filters.getRelatesTo())) { + return null; + } + if (!hasField(fieldsToRetrieve, Field.RELATES_TO)) { + entity.getRelatesToEntities().clear(); + } + } + + // fetch info if fieldsToRetrieve contains INFO or ALL. + if (hasField(fieldsToRetrieve, Field.INFO)) { + readKeyValuePairs(entity, result, EntityColumnPrefix.INFO, false); + } + + // fetch configs if fieldsToRetrieve contains CONFIGS or ALL. + if (hasField(fieldsToRetrieve, Field.CONFIGS)) { + readKeyValuePairs(entity, result, EntityColumnPrefix.CONFIG, true); + } + + // fetch events and match event filters if they exist. If event filters do + // not match, entity would be dropped. We have to match filters locally + // as relevant HBase filters to filter out rows on the basis of events + // are not set in HBase scan. + boolean checkEvents = + !isSingleEntityRead() && filters.getEventFilters() != null + && filters.getEventFilters().getFilterList().size() > 0; + if (hasField(fieldsToRetrieve, Field.EVENTS) || checkEvents) { + readEvents(entity, result, EntityColumnPrefix.EVENT); + if (checkEvents + && !TimelineStorageUtils.matchEventFilters(entity, + filters.getEventFilters())) { + return null; + } + if (!hasField(fieldsToRetrieve, Field.EVENTS)) { + entity.getEvents().clear(); + } + } + + // fetch metrics if fieldsToRetrieve contains METRICS or ALL. + if (hasField(fieldsToRetrieve, Field.METRICS)) { + readMetrics(entity, result, EntityColumnPrefix.METRIC); + } + + entity.getInfo().put(TimelineReaderUtils.FROMID_KEY, + parseRowKey.getRowKeyAsString()); + return entity; + } + + /** + * Helper method for reading key-value pairs for either info or config. + * + * @param Describes the type of column prefix. + * @param entity entity to fill. + * @param result result from HBase. + * @param prefix column prefix. + * @param isConfig if true, means we are reading configs, otherwise info. + * @throws IOException if any problem is encountered while reading result. + */ + protected void readKeyValuePairs(TimelineEntity entity, Result result, + ColumnPrefix prefix, boolean isConfig) throws IOException { + // info and configuration are of type Map + Map columns = + prefix.readResults(result, stringKeyConverter); + if (isConfig) { + for (Map.Entry column : columns.entrySet()) { + entity.addConfig(column.getKey(), column.getValue().toString()); + } + } else { + entity.addInfo(columns); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/SubApplicationEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/SubApplicationEntityReader.java new file mode 100644 index 00000000000..e780dccd3a8 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/SubApplicationEntityReader.java @@ -0,0 +1,488 @@ +/** + * 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.timelineservice.storage.reader; + +import java.io.IOException; +import java.util.EnumSet; +import java.util.Set; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.Query; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.filter.BinaryComparator; +import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; +import org.apache.hadoop.hbase.filter.FamilyFilter; +import org.apache.hadoop.hbase.filter.FilterList; +import org.apache.hadoop.hbase.filter.FilterList.Operator; +import org.apache.hadoop.hbase.filter.PageFilter; +import org.apache.hadoop.hbase.filter.QualifierFilter; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderUtils; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationColumn; +import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationRowKeyPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationTable; +import org.apache.hadoop.yarn.webapp.BadRequestException; + +import com.google.common.base.Preconditions; + +class SubApplicationEntityReader extends GenericEntityReader { + private static final SubApplicationTable SUB_APPLICATION_TABLE = + new SubApplicationTable(); + + SubApplicationEntityReader(TimelineReaderContext ctxt, + TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) { + super(ctxt, entityFilters, toRetrieve); + } + + SubApplicationEntityReader(TimelineReaderContext ctxt, + TimelineDataToRetrieve toRetrieve) { + super(ctxt, toRetrieve); + } + + /** + * Uses the {@link SubApplicationTable}. + */ + protected BaseTable getTable() { + return SUB_APPLICATION_TABLE; + } + + @Override + protected FilterList constructFilterListBasedOnFilters() throws IOException { + // Filters here cannot be null for multiple entity reads as they are set in + // augmentParams if null. + FilterList listBasedOnFilters = new FilterList(); + TimelineEntityFilters filters = getFilters(); + // Create filter list based on created time range and add it to + // listBasedOnFilters. + long createdTimeBegin = filters.getCreatedTimeBegin(); + long createdTimeEnd = filters.getCreatedTimeEnd(); + if (createdTimeBegin != 0 || createdTimeEnd != Long.MAX_VALUE) { + listBasedOnFilters.addFilter(TimelineFilterUtils + .createSingleColValueFiltersByRange(SubApplicationColumn.CREATED_TIME, + createdTimeBegin, createdTimeEnd)); + } + // Create filter list based on metric filters and add it to + // listBasedOnFilters. + TimelineFilterList metricFilters = filters.getMetricFilters(); + if (metricFilters != null && !metricFilters.getFilterList().isEmpty()) { + listBasedOnFilters.addFilter(TimelineFilterUtils.createHBaseFilterList( + SubApplicationColumnPrefix.METRIC, metricFilters)); + } + // Create filter list based on config filters and add it to + // listBasedOnFilters. + TimelineFilterList configFilters = filters.getConfigFilters(); + if (configFilters != null && !configFilters.getFilterList().isEmpty()) { + listBasedOnFilters.addFilter(TimelineFilterUtils.createHBaseFilterList( + SubApplicationColumnPrefix.CONFIG, configFilters)); + } + // Create filter list based on info filters and add it to listBasedOnFilters + TimelineFilterList infoFilters = filters.getInfoFilters(); + if (infoFilters != null && !infoFilters.getFilterList().isEmpty()) { + listBasedOnFilters.addFilter(TimelineFilterUtils + .createHBaseFilterList(SubApplicationColumnPrefix.INFO, infoFilters)); + } + return listBasedOnFilters; + } + + /** + * Add {@link QualifierFilter} filters to filter list for each column of + * entity table. + * + * @param list filter list to which qualifier filters have to be added. + */ + protected void updateFixedColumns(FilterList list) { + for (SubApplicationColumn column : SubApplicationColumn.values()) { + list.addFilter(new QualifierFilter(CompareOp.EQUAL, + new BinaryComparator(column.getColumnQualifierBytes()))); + } + } + + /** + * Creates a filter list which indicates that only some of the column + * qualifiers in the info column family will be returned in result. + * + * @param isApplication If true, it means operations are to be performed for + * application table, otherwise for entity table. + * @return filter list. + * @throws IOException if any problem occurs while creating filter list. + */ + private FilterList createFilterListForColsOfInfoFamily() throws IOException { + FilterList infoFamilyColsFilter = new FilterList(Operator.MUST_PASS_ONE); + // Add filters for each column in entity table. + updateFixedColumns(infoFamilyColsFilter); + EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); + // If INFO field has to be retrieved, add a filter for fetching columns + // with INFO column prefix. + if (hasField(fieldsToRetrieve, Field.INFO)) { + infoFamilyColsFilter.addFilter( + TimelineFilterUtils.createHBaseQualifierFilter(CompareOp.EQUAL, + SubApplicationColumnPrefix.INFO)); + } + TimelineFilterList relatesTo = getFilters().getRelatesTo(); + if (hasField(fieldsToRetrieve, Field.RELATES_TO)) { + // If RELATES_TO field has to be retrieved, add a filter for fetching + // columns with RELATES_TO column prefix. + infoFamilyColsFilter.addFilter( + TimelineFilterUtils.createHBaseQualifierFilter(CompareOp.EQUAL, + SubApplicationColumnPrefix.RELATES_TO)); + } else if (relatesTo != null && !relatesTo.getFilterList().isEmpty()) { + // Even if fields to retrieve does not contain RELATES_TO, we still + // need to have a filter to fetch some of the column qualifiers if + // relatesTo filters are specified. relatesTo filters will then be + // matched after fetching rows from HBase. + Set relatesToCols = + TimelineFilterUtils.fetchColumnsFromFilterList(relatesTo); + infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers( + SubApplicationColumnPrefix.RELATES_TO, relatesToCols)); + } + TimelineFilterList isRelatedTo = getFilters().getIsRelatedTo(); + if (hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) { + // If IS_RELATED_TO field has to be retrieved, add a filter for fetching + // columns with IS_RELATED_TO column prefix. + infoFamilyColsFilter.addFilter( + TimelineFilterUtils.createHBaseQualifierFilter(CompareOp.EQUAL, + SubApplicationColumnPrefix.IS_RELATED_TO)); + } else if (isRelatedTo != null && !isRelatedTo.getFilterList().isEmpty()) { + // Even if fields to retrieve does not contain IS_RELATED_TO, we still + // need to have a filter to fetch some of the column qualifiers if + // isRelatedTo filters are specified. isRelatedTo filters will then be + // matched after fetching rows from HBase. + Set isRelatedToCols = + TimelineFilterUtils.fetchColumnsFromFilterList(isRelatedTo); + infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers( + SubApplicationColumnPrefix.IS_RELATED_TO, isRelatedToCols)); + } + TimelineFilterList eventFilters = getFilters().getEventFilters(); + if (hasField(fieldsToRetrieve, Field.EVENTS)) { + // If EVENTS field has to be retrieved, add a filter for fetching columns + // with EVENT column prefix. + infoFamilyColsFilter.addFilter( + TimelineFilterUtils.createHBaseQualifierFilter(CompareOp.EQUAL, + SubApplicationColumnPrefix.EVENT)); + } else if (eventFilters != null + && !eventFilters.getFilterList().isEmpty()) { + // Even if fields to retrieve does not contain EVENTS, we still need to + // have a filter to fetch some of the column qualifiers on the basis of + // event filters specified. Event filters will then be matched after + // fetching rows from HBase. + Set eventCols = + TimelineFilterUtils.fetchColumnsFromFilterList(eventFilters); + infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers( + SubApplicationColumnPrefix.EVENT, eventCols)); + } + return infoFamilyColsFilter; + } + + /** + * Exclude column prefixes via filters which are not required(based on fields + * to retrieve) from info column family. These filters are added to filter + * list which contains a filter for getting info column family. + * + * @param infoColFamilyList filter list for info column family. + */ + private void excludeFieldsFromInfoColFamily(FilterList infoColFamilyList) { + EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); + // Events not required. + if (!hasField(fieldsToRetrieve, Field.EVENTS)) { + infoColFamilyList.addFilter( + TimelineFilterUtils.createHBaseQualifierFilter(CompareOp.NOT_EQUAL, + SubApplicationColumnPrefix.EVENT)); + } + // info not required. + if (!hasField(fieldsToRetrieve, Field.INFO)) { + infoColFamilyList.addFilter( + TimelineFilterUtils.createHBaseQualifierFilter(CompareOp.NOT_EQUAL, + SubApplicationColumnPrefix.INFO)); + } + // is related to not required. + if (!hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) { + infoColFamilyList.addFilter( + TimelineFilterUtils.createHBaseQualifierFilter(CompareOp.NOT_EQUAL, + SubApplicationColumnPrefix.IS_RELATED_TO)); + } + // relates to not required. + if (!hasField(fieldsToRetrieve, Field.RELATES_TO)) { + infoColFamilyList.addFilter( + TimelineFilterUtils.createHBaseQualifierFilter(CompareOp.NOT_EQUAL, + SubApplicationColumnPrefix.RELATES_TO)); + } + } + + /** + * Updates filter list based on fields for confs and metrics to retrieve. + * + * @param listBasedOnFields filter list based on fields. + * @throws IOException if any problem occurs while updating filter list. + */ + private void updateFilterForConfsAndMetricsToRetrieve( + FilterList listBasedOnFields) throws IOException { + TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve(); + // Please note that if confsToRetrieve is specified, we would have added + // CONFS to fields to retrieve in augmentParams() even if not specified. + if (dataToRetrieve.getFieldsToRetrieve().contains(Field.CONFIGS)) { + // Create a filter list for configs. + listBasedOnFields.addFilter( + TimelineFilterUtils.createFilterForConfsOrMetricsToRetrieve( + dataToRetrieve.getConfsToRetrieve(), + SubApplicationColumnFamily.CONFIGS, + SubApplicationColumnPrefix.CONFIG)); + } + + // Please note that if metricsToRetrieve is specified, we would have added + // METRICS to fields to retrieve in augmentParams() even if not specified. + if (dataToRetrieve.getFieldsToRetrieve().contains(Field.METRICS)) { + // Create a filter list for metrics. + listBasedOnFields.addFilter( + TimelineFilterUtils.createFilterForConfsOrMetricsToRetrieve( + dataToRetrieve.getMetricsToRetrieve(), + SubApplicationColumnFamily.METRICS, + SubApplicationColumnPrefix.METRIC)); + } + } + + @Override + protected FilterList constructFilterListBasedOnFields() throws IOException { + if (!needCreateFilterListBasedOnFields()) { + // Fetch all the columns. No need of a filter. + return null; + } + FilterList listBasedOnFields = new FilterList(Operator.MUST_PASS_ONE); + FilterList infoColFamilyList = new FilterList(); + // By default fetch everything in INFO column family. + FamilyFilter infoColumnFamily = new FamilyFilter(CompareOp.EQUAL, + new BinaryComparator(SubApplicationColumnFamily.INFO.getBytes())); + infoColFamilyList.addFilter(infoColumnFamily); + if (fetchPartialColsFromInfoFamily()) { + // We can fetch only some of the columns from info family. + infoColFamilyList.addFilter(createFilterListForColsOfInfoFamily()); + } else { + // Exclude column prefixes in info column family which are not required + // based on fields to retrieve. + excludeFieldsFromInfoColFamily(infoColFamilyList); + } + listBasedOnFields.addFilter(infoColFamilyList); + updateFilterForConfsAndMetricsToRetrieve(listBasedOnFields); + return listBasedOnFields; + } + + @Override + protected void validateParams() { + Preconditions.checkNotNull(getContext(), "context shouldn't be null"); + Preconditions.checkNotNull(getDataToRetrieve(), + "data to retrieve shouldn't be null"); + Preconditions.checkNotNull(getContext().getClusterId(), + "clusterId shouldn't be null"); + Preconditions.checkNotNull(getContext().getDoAsUser(), + "DoAsUser shouldn't be null"); + Preconditions.checkNotNull(getContext().getEntityType(), + "entityType shouldn't be null"); + } + + @Override + protected void augmentParams(Configuration hbaseConf, Connection conn) + throws IOException { + getDataToRetrieve().addFieldsBasedOnConfsAndMetricsToRetrieve(); + createFiltersIfNull(); + } + + private void setMetricsTimeRange(Query query) { + // Set time range for metric values. + HBaseTimelineStorageUtils.setMetricsTimeRange(query, + SubApplicationColumnFamily.METRICS.getBytes(), + getDataToRetrieve().getMetricsTimeBegin(), + getDataToRetrieve().getMetricsTimeEnd()); + } + + @Override + protected ResultScanner getResults(Configuration hbaseConf, Connection conn, + FilterList filterList) throws IOException { + + // Scan through part of the table to find the entities belong to one app + // and one type + Scan scan = new Scan(); + TimelineReaderContext context = getContext(); + if (context.getDoAsUser() == null) { + throw new BadRequestException("Invalid user!"); + } + + RowKeyPrefix subApplicationRowKeyPrefix = null; + // default mode, will always scans from beginning of entity type. + if (getFilters() == null || getFilters().getFromId() == null) { + subApplicationRowKeyPrefix = new SubApplicationRowKeyPrefix( + context.getDoAsUser(), context.getClusterId(), + context.getEntityType(), null, null, null); + scan.setRowPrefixFilter(subApplicationRowKeyPrefix.getRowKeyPrefix()); + } else { // pagination mode, will scan from given entityIdPrefix!enitityId + + SubApplicationRowKey entityRowKey = null; + try { + entityRowKey = SubApplicationRowKey + .parseRowKeyFromString(getFilters().getFromId()); + } catch (IllegalArgumentException e) { + throw new BadRequestException("Invalid filter fromid is provided."); + } + if (!context.getClusterId().equals(entityRowKey.getClusterId())) { + throw new BadRequestException( + "fromid doesn't belong to clusterId=" + context.getClusterId()); + } + + // set start row + scan.setStartRow(entityRowKey.getRowKey()); + + // get the bytes for stop row + subApplicationRowKeyPrefix = new SubApplicationRowKeyPrefix( + context.getDoAsUser(), context.getClusterId(), + context.getEntityType(), null, null, null); + + // set stop row + scan.setStopRow( + HBaseTimelineStorageUtils.calculateTheClosestNextRowKeyForPrefix( + subApplicationRowKeyPrefix.getRowKeyPrefix())); + + // set page filter to limit. This filter has to set only in pagination + // mode. + filterList.addFilter(new PageFilter(getFilters().getLimit())); + } + setMetricsTimeRange(scan); + scan.setMaxVersions(getDataToRetrieve().getMetricsLimit()); + if (filterList != null && !filterList.getFilters().isEmpty()) { + scan.setFilter(filterList); + } + return getTable().getResultScanner(hbaseConf, conn, scan); + } + + @Override + protected Result getResult(Configuration hbaseConf, Connection conn, + FilterList filterList) throws IOException { + throw new UnsupportedOperationException( + "we don't support a single entity query"); + } + + @Override + protected TimelineEntity parseEntity(Result result) throws IOException { + if (result == null || result.isEmpty()) { + return null; + } + TimelineEntity entity = new TimelineEntity(); + SubApplicationRowKey parseRowKey = + SubApplicationRowKey.parseRowKey(result.getRow()); + entity.setType(parseRowKey.getEntityType()); + entity.setId(parseRowKey.getEntityId()); + entity.setIdPrefix(parseRowKey.getEntityIdPrefix().longValue()); + + TimelineEntityFilters filters = getFilters(); + // fetch created time + Long createdTime = + (Long) SubApplicationColumn.CREATED_TIME.readResult(result); + entity.setCreatedTime(createdTime); + + EnumSet fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve(); + // fetch is related to entities and match isRelatedTo filter. If isRelatedTo + // filters do not match, entity would be dropped. We have to match filters + // locally as relevant HBase filters to filter out rows on the basis of + // isRelatedTo are not set in HBase scan. + boolean checkIsRelatedTo = + filters.getIsRelatedTo() != null + && filters.getIsRelatedTo().getFilterList().size() > 0; + if (hasField(fieldsToRetrieve, Field.IS_RELATED_TO) || checkIsRelatedTo) { + readRelationship(entity, result, SubApplicationColumnPrefix.IS_RELATED_TO, + true); + if (checkIsRelatedTo && !TimelineStorageUtils.matchIsRelatedTo(entity, + filters.getIsRelatedTo())) { + return null; + } + if (!hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) { + entity.getIsRelatedToEntities().clear(); + } + } + + // fetch relates to entities and match relatesTo filter. If relatesTo + // filters do not match, entity would be dropped. We have to match filters + // locally as relevant HBase filters to filter out rows on the basis of + // relatesTo are not set in HBase scan. + boolean checkRelatesTo = + !isSingleEntityRead() && filters.getRelatesTo() != null + && filters.getRelatesTo().getFilterList().size() > 0; + if (hasField(fieldsToRetrieve, Field.RELATES_TO) || checkRelatesTo) { + readRelationship(entity, result, SubApplicationColumnPrefix.RELATES_TO, + false); + if (checkRelatesTo && !TimelineStorageUtils.matchRelatesTo(entity, + filters.getRelatesTo())) { + return null; + } + if (!hasField(fieldsToRetrieve, Field.RELATES_TO)) { + entity.getRelatesToEntities().clear(); + } + } + + // fetch info if fieldsToRetrieve contains INFO or ALL. + if (hasField(fieldsToRetrieve, Field.INFO)) { + readKeyValuePairs(entity, result, SubApplicationColumnPrefix.INFO, false); + } + + // fetch configs if fieldsToRetrieve contains CONFIGS or ALL. + if (hasField(fieldsToRetrieve, Field.CONFIGS)) { + readKeyValuePairs(entity, result, SubApplicationColumnPrefix.CONFIG, + true); + } + + // fetch events and match event filters if they exist. If event filters do + // not match, entity would be dropped. We have to match filters locally + // as relevant HBase filters to filter out rows on the basis of events + // are not set in HBase scan. + boolean checkEvents = + !isSingleEntityRead() && filters.getEventFilters() != null + && filters.getEventFilters().getFilterList().size() > 0; + if (hasField(fieldsToRetrieve, Field.EVENTS) || checkEvents) { + readEvents(entity, result, SubApplicationColumnPrefix.EVENT); + if (checkEvents && !TimelineStorageUtils.matchEventFilters(entity, + filters.getEventFilters())) { + return null; + } + if (!hasField(fieldsToRetrieve, Field.EVENTS)) { + entity.getEvents().clear(); + } + } + + // fetch metrics if fieldsToRetrieve contains METRICS or ALL. + if (hasField(fieldsToRetrieve, Field.METRICS)) { + readMetrics(entity, result, SubApplicationColumnPrefix.METRIC); + } + + entity.getInfo().put(TimelineReaderUtils.FROMID_KEY, + parseRowKey.getRowKeyAsString()); + return entity; + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java new file mode 100644 index 00000000000..07e84234254 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java @@ -0,0 +1,459 @@ +/** + * 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.timelineservice.storage.reader; + +import java.io.IOException; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Set; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.filter.BinaryPrefixComparator; +import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; +import org.apache.hadoop.hbase.filter.FilterList; +import org.apache.hadoop.hbase.filter.FilterList.Operator; +import org.apache.hadoop.hbase.filter.QualifierFilter; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnName; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnNameConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The base class for reading and deserializing timeline entities from the + * HBase storage. Different types can be defined for different types of the + * entities that are being requested. + */ +public abstract class TimelineEntityReader extends + AbstractTimelineStorageReader { + private static final Logger LOG = + LoggerFactory.getLogger(TimelineEntityReader.class); + + private final boolean singleEntityRead; + private TimelineDataToRetrieve dataToRetrieve; + // used only for multiple entity read mode + private TimelineEntityFilters filters; + + /** + * Main table the entity reader uses. + */ + private BaseTable table; + + /** + * Used to convert strings key components to and from storage format. + */ + private final KeyConverter stringKeyConverter = + new StringKeyConverter(); + + /** + * Instantiates a reader for multiple-entity reads. + * + * @param ctxt Reader context which defines the scope in which query has to be + * made. + * @param entityFilters Filters which limit the entities returned. + * @param toRetrieve Data to retrieve for each entity. + */ + protected TimelineEntityReader(TimelineReaderContext ctxt, + TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) { + super(ctxt); + this.singleEntityRead = false; + this.dataToRetrieve = toRetrieve; + this.filters = entityFilters; + + this.setTable(getTable()); + } + + /** + * Instantiates a reader for single-entity reads. + * + * @param ctxt Reader context which defines the scope in which query has to be + * made. + * @param toRetrieve Data to retrieve for each entity. + */ + protected TimelineEntityReader(TimelineReaderContext ctxt, + TimelineDataToRetrieve toRetrieve) { + super(ctxt); + this.singleEntityRead = true; + this.dataToRetrieve = toRetrieve; + + this.setTable(getTable()); + } + + /** + * Creates a {@link FilterList} based on fields, confs and metrics to + * retrieve. This filter list will be set in Scan/Get objects to trim down + * results fetched from HBase back-end storage. This is called only for + * multiple entity reads. + * + * @return a {@link FilterList} object. + * @throws IOException if any problem occurs while creating filter list. + */ + protected abstract FilterList constructFilterListBasedOnFields() + throws IOException; + + /** + * Creates a {@link FilterList} based on info, config and metric filters. This + * filter list will be set in HBase Get to trim down results fetched from + * HBase back-end storage. + * + * @return a {@link FilterList} object. + * @throws IOException if any problem occurs while creating filter list. + */ + protected abstract FilterList constructFilterListBasedOnFilters() + throws IOException; + + /** + * Combines filter lists created based on fields and based on filters. + * + * @return a {@link FilterList} object if it can be constructed. Returns null, + * if filter list cannot be created either on the basis of filters or on the + * basis of fields. + * @throws IOException if any problem occurs while creating filter list. + */ + private FilterList createFilterList() throws IOException { + FilterList listBasedOnFilters = constructFilterListBasedOnFilters(); + boolean hasListBasedOnFilters = listBasedOnFilters != null && + !listBasedOnFilters.getFilters().isEmpty(); + FilterList listBasedOnFields = constructFilterListBasedOnFields(); + boolean hasListBasedOnFields = listBasedOnFields != null && + !listBasedOnFields.getFilters().isEmpty(); + // If filter lists based on both filters and fields can be created, + // combine them in a new filter list and return it. + // If either one of them has been created, return that filter list. + // Return null, if none of the filter lists can be created. This indicates + // that no filter list needs to be added to HBase Scan as filters are not + // specified for the query or only the default view of entity needs to be + // returned. + if (hasListBasedOnFilters && hasListBasedOnFields) { + FilterList list = new FilterList(); + list.addFilter(listBasedOnFilters); + list.addFilter(listBasedOnFields); + return list; + } else if (hasListBasedOnFilters) { + return listBasedOnFilters; + } else if (hasListBasedOnFields) { + return listBasedOnFields; + } + return null; + } + + protected TimelineDataToRetrieve getDataToRetrieve() { + return dataToRetrieve; + } + + protected TimelineEntityFilters getFilters() { + return filters; + } + + /** + * Create a {@link TimelineEntityFilters} object with default values for + * filters. + */ + protected void createFiltersIfNull() { + if (filters == null) { + filters = new TimelineEntityFilters.Builder().build(); + } + } + + /** + * Reads and deserializes a single timeline entity from the HBase storage. + * + * @param hbaseConf HBase Configuration. + * @param conn HBase Connection. + * @return A TimelineEntity object. + * @throws IOException if there is any exception encountered while reading + * entity. + */ + public TimelineEntity readEntity(Configuration hbaseConf, Connection conn) + throws IOException { + validateParams(); + augmentParams(hbaseConf, conn); + + FilterList filterList = constructFilterListBasedOnFields(); + if (LOG.isDebugEnabled() && filterList != null) { + LOG.debug("FilterList created for get is - " + filterList); + } + Result result = getResult(hbaseConf, conn, filterList); + if (result == null || result.isEmpty()) { + // Could not find a matching row. + LOG.info("Cannot find matching entity of type " + + getContext().getEntityType()); + return null; + } + return parseEntity(result); + } + + /** + * Reads and deserializes a set of timeline entities from the HBase storage. + * It goes through all the results available, and returns the number of + * entries as specified in the limit in the entity's natural sort order. + * + * @param hbaseConf HBase Configuration. + * @param conn HBase Connection. + * @return a set of TimelineEntity objects. + * @throws IOException if any exception is encountered while reading entities. + */ + public Set readEntities(Configuration hbaseConf, + Connection conn) throws IOException { + validateParams(); + augmentParams(hbaseConf, conn); + + Set entities = new LinkedHashSet<>(); + FilterList filterList = createFilterList(); + if (LOG.isDebugEnabled() && filterList != null) { + LOG.debug("FilterList created for scan is - " + filterList); + } + ResultScanner results = getResults(hbaseConf, conn, filterList); + try { + for (Result result : results) { + TimelineEntity entity = parseEntity(result); + if (entity == null) { + continue; + } + entities.add(entity); + if (entities.size() == filters.getLimit()) { + break; + } + } + return entities; + } finally { + results.close(); + } + } + + /** + * Returns the main table to be used by the entity reader. + * + * @return A reference to the table. + */ + protected BaseTable getTable() { + return table; + } + + /** + * Fetches a {@link Result} instance for a single-entity read. + * + * @param hbaseConf HBase Configuration. + * @param conn HBase Connection. + * @param filterList filter list which will be applied to HBase Get. + * @return the {@link Result} instance or null if no such record is found. + * @throws IOException if any exception is encountered while getting result. + */ + protected abstract Result getResult(Configuration hbaseConf, Connection conn, + FilterList filterList) throws IOException; + + /** + * Fetches a {@link ResultScanner} for a multi-entity read. + * + * @param hbaseConf HBase Configuration. + * @param conn HBase Connection. + * @param filterList filter list which will be applied to HBase Scan. + * @return the {@link ResultScanner} instance. + * @throws IOException if any exception is encountered while getting results. + */ + protected abstract ResultScanner getResults(Configuration hbaseConf, + Connection conn, FilterList filterList) throws IOException; + + /** + * Parses the result retrieved from HBase backend and convert it into a + * {@link TimelineEntity} object. + * + * @param result Single row result of a Get/Scan. + * @return the TimelineEntity instance or null if the entity is + * filtered. + * @throws IOException if any exception is encountered while parsing entity. + */ + protected abstract TimelineEntity parseEntity(Result result) + throws IOException; + + /** + * Helper method for reading and deserializing {@link TimelineMetric} objects + * using the specified column prefix. The timeline metrics then are added to + * the given timeline entity. + * + * @param entity {@link TimelineEntity} object. + * @param result {@link Result} object retrieved from backend. + * @param columnPrefix Metric column prefix + * @throws IOException if any exception is encountered while reading metrics. + */ + protected void readMetrics(TimelineEntity entity, Result result, + ColumnPrefix columnPrefix) throws IOException { + NavigableMap> metricsResult = + columnPrefix.readResultsWithTimestamps( + result, stringKeyConverter); + for (Map.Entry> metricResult: + metricsResult.entrySet()) { + TimelineMetric metric = new TimelineMetric(); + metric.setId(metricResult.getKey()); + // Simply assume that if the value set contains more than 1 elements, the + // metric is a TIME_SERIES metric, otherwise, it's a SINGLE_VALUE metric + TimelineMetric.Type metricType = metricResult.getValue().size() > 1 ? + TimelineMetric.Type.TIME_SERIES : TimelineMetric.Type.SINGLE_VALUE; + metric.setType(metricType); + metric.addValues(metricResult.getValue()); + entity.addMetric(metric); + } + } + + /** + * Checks whether the reader has been created to fetch single entity or + * multiple entities. + * + * @return true, if query is for single entity, false otherwise. + */ + public boolean isSingleEntityRead() { + return singleEntityRead; + } + + protected void setTable(BaseTable baseTable) { + this.table = baseTable; + } + + /** + * Check if we have a certain field amongst fields to retrieve. This method + * checks against {@link Field#ALL} as well because that would mean field + * passed needs to be matched. + * + * @param fieldsToRetrieve fields to be retrieved. + * @param requiredField fields to be checked in fieldsToRetrieve. + * @return true if has the required field, false otherwise. + */ + protected boolean hasField(EnumSet fieldsToRetrieve, + Field requiredField) { + return fieldsToRetrieve.contains(Field.ALL) || + fieldsToRetrieve.contains(requiredField); + } + + /** + * Create a filter list of qualifier filters based on passed set of columns. + * + * @param Describes the type of column prefix. + * @param colPrefix Column Prefix. + * @param columns set of column qualifiers. + * @return filter list. + */ + protected FilterList createFiltersFromColumnQualifiers( + ColumnPrefix colPrefix, Set columns) { + FilterList list = new FilterList(Operator.MUST_PASS_ONE); + for (String column : columns) { + // For columns which have compound column qualifiers (eg. events), we need + // to include the required separator. + byte[] compoundColQual = createColQualifierPrefix(colPrefix, column); + list.addFilter(new QualifierFilter(CompareOp.EQUAL, + new BinaryPrefixComparator(colPrefix + .getColumnPrefixBytes(compoundColQual)))); + } + return list; + } + + protected byte[] createColQualifierPrefix(ColumnPrefix colPrefix, + String column) { + if (colPrefix == ApplicationColumnPrefix.EVENT + || colPrefix == EntityColumnPrefix.EVENT) { + return new EventColumnName(column, null, null).getColumnQualifier(); + } else { + return stringKeyConverter.encode(column); + } + } + + /** + * Helper method for reading relationship. + * + * @param Describes the type of column prefix. + * @param entity entity to fill. + * @param result result from HBase. + * @param prefix column prefix. + * @param isRelatedTo if true, means relationship is to be added to + * isRelatedTo, otherwise its added to relatesTo. + * @throws IOException if any problem is encountered while reading result. + */ + protected void readRelationship(TimelineEntity entity, Result result, + ColumnPrefix prefix, boolean isRelatedTo) throws IOException { + // isRelatedTo and relatesTo are of type Map> + Map columns = + prefix.readResults(result, stringKeyConverter); + for (Map.Entry column : columns.entrySet()) { + for (String id : Separator.VALUES.splitEncoded(column.getValue() + .toString())) { + if (isRelatedTo) { + entity.addIsRelatedToEntity(column.getKey(), id); + } else { + entity.addRelatesToEntity(column.getKey(), id); + } + } + } + } + + /** + * Read events from the entity table or the application table. The column name + * is of the form "eventId=timestamp=infoKey" where "infoKey" may be omitted + * if there is no info associated with the event. + * + * @param Describes the type of column prefix. + * @param entity entity to fill. + * @param result HBase Result. + * @param prefix column prefix. + * @throws IOException if any problem is encountered while reading result. + */ + protected static void readEvents(TimelineEntity entity, Result result, + ColumnPrefix prefix) throws IOException { + Map eventsMap = new HashMap<>(); + Map eventsResult = + prefix.readResults(result, new EventColumnNameConverter()); + for (Map.Entry + eventResult : eventsResult.entrySet()) { + EventColumnName eventColumnName = eventResult.getKey(); + String key = eventColumnName.getId() + + Long.toString(eventColumnName.getTimestamp()); + // Retrieve previously seen event to add to it + TimelineEvent event = eventsMap.get(key); + if (event == null) { + // First time we're seeing this event, add it to the eventsMap + event = new TimelineEvent(); + event.setId(eventColumnName.getId()); + event.setTimestamp(eventColumnName.getTimestamp()); + eventsMap.put(key, event); + } + if (eventColumnName.getInfoKey() != null) { + event.addInfo(eventColumnName.getInfoKey(), eventResult.getValue()); + } + } + Set eventsSet = new HashSet<>(eventsMap.values()); + entity.addEvents(eventsSet); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReaderFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReaderFactory.java new file mode 100644 index 00000000000..fa16077c2ed --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReaderFactory.java @@ -0,0 +1,105 @@ +/** + * 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.timelineservice.storage.reader; + +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext; + +/** + * Factory methods for instantiating a timeline entity reader. + */ +public final class TimelineEntityReaderFactory { + private TimelineEntityReaderFactory() { + } + + /** + * Creates a timeline entity reader instance for reading a single entity with + * the specified input. + * + * @param context Reader context which defines the scope in which query has to + * be made. + * @param dataToRetrieve Data to retrieve for each entity. + * @return An implementation of TimelineEntityReader object + * depending on entity type. + */ + public static TimelineEntityReader createSingleEntityReader( + TimelineReaderContext context, TimelineDataToRetrieve dataToRetrieve) { + // currently the types that are handled separate from the generic entity + // table are application, flow run, and flow activity entities + if (TimelineEntityType.YARN_APPLICATION.matches(context.getEntityType())) { + return new ApplicationEntityReader(context, dataToRetrieve); + } else if (TimelineEntityType. + YARN_FLOW_RUN.matches(context.getEntityType())) { + return new FlowRunEntityReader(context, dataToRetrieve); + } else if (TimelineEntityType. + YARN_FLOW_ACTIVITY.matches(context.getEntityType())) { + return new FlowActivityEntityReader(context, dataToRetrieve); + } else { + // assume we're dealing with a generic entity read + return new GenericEntityReader(context, dataToRetrieve); + } + } + + /** + * Creates a timeline entity reader instance for reading set of entities with + * the specified input and predicates. + * + * @param context Reader context which defines the scope in which query has to + * be made. + * @param filters Filters which limit the entities returned. + * @param dataToRetrieve Data to retrieve for each entity. + * @return An implementation of TimelineEntityReader object + * depending on entity type. + */ + public static TimelineEntityReader createMultipleEntitiesReader( + TimelineReaderContext context, TimelineEntityFilters filters, + TimelineDataToRetrieve dataToRetrieve) { + // currently the types that are handled separate from the generic entity + // table are application, flow run, and flow activity entities + if (TimelineEntityType.YARN_APPLICATION.matches(context.getEntityType())) { + return new ApplicationEntityReader(context, filters, dataToRetrieve); + } else if (TimelineEntityType. + YARN_FLOW_ACTIVITY.matches(context.getEntityType())) { + return new FlowActivityEntityReader(context, filters, dataToRetrieve); + } else if (TimelineEntityType. + YARN_FLOW_RUN.matches(context.getEntityType())) { + return new FlowRunEntityReader(context, filters, dataToRetrieve); + } else { + if (context.getDoAsUser() != null) { + return new SubApplicationEntityReader(context, filters, dataToRetrieve); + } + // assume we're dealing with a generic entity read + return new GenericEntityReader(context, filters, dataToRetrieve); + } + } + + /** + * Creates a timeline entity type reader that will read all available entity + * types within the specified context. + * + * @param context Reader context which defines the scope in which query has to + * be made. Limited to application level only. + * @return an EntityTypeReader object + */ + public static EntityTypeReader createEntityTypeReader( + TimelineReaderContext context) { + return new EntityTypeReader(context); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/package-info.java new file mode 100644 index 00000000000..9814d6d3351 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/package-info.java @@ -0,0 +1,28 @@ +/* + * 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.timelineservice.storage.reader + * contains classes used to read entities from backend based on query type. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.timelineservice.storage.reader; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationColumn.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationColumn.java new file mode 100644 index 00000000000..46b0cc90e43 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationColumn.java @@ -0,0 +1,108 @@ +/** + * 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.timelineservice.storage.subapplication; + +import java.io.IOException; + +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Column; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; + +/** + * Identifies fully qualified columns for the {@link SubApplicationTable}. + */ +public enum SubApplicationColumn implements Column { + + /** + * Identifier for the sub application. + */ + ID(SubApplicationColumnFamily.INFO, "id"), + + /** + * The type of sub application. + */ + TYPE(SubApplicationColumnFamily.INFO, "type"), + + /** + * When the sub application was created. + */ + CREATED_TIME(SubApplicationColumnFamily.INFO, "created_time", + new LongConverter()), + + /** + * The version of the flow that this sub application belongs to. + */ + FLOW_VERSION(SubApplicationColumnFamily.INFO, "flow_version"); + + private final ColumnHelper column; + private final ColumnFamily columnFamily; + private final String columnQualifier; + private final byte[] columnQualifierBytes; + + SubApplicationColumn(ColumnFamily columnFamily, + String columnQualifier) { + this(columnFamily, columnQualifier, GenericConverter.getInstance()); + } + + SubApplicationColumn(ColumnFamily columnFamily, + String columnQualifier, ValueConverter converter) { + this.columnFamily = columnFamily; + this.columnQualifier = columnQualifier; + // Future-proof by ensuring the right column prefix hygiene. + this.columnQualifierBytes = + Bytes.toBytes(Separator.SPACE.encode(columnQualifier)); + this.column = new ColumnHelper(columnFamily, + converter); + } + + + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, Long timestamp, + Object inputValue, Attribute... attributes) throws IOException { + column.store(rowKey, tableMutator, columnQualifierBytes, timestamp, + inputValue, attributes); + } + + public Object readResult(Result result) throws IOException { + return column.readResult(result, columnQualifierBytes); + } + + @Override + public byte[] getColumnQualifierBytes() { + return columnQualifierBytes.clone(); + } + + @Override + public byte[] getColumnFamilyBytes() { + return columnFamily.getBytes(); + } + + @Override + public ValueConverter getValueConverter() { + return column.getValueConverter(); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationColumnFamily.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationColumnFamily.java new file mode 100644 index 00000000000..1d7f8fdf3da --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationColumnFamily.java @@ -0,0 +1,68 @@ +/** + * 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.timelineservice.storage.subapplication; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; + +/** + * Represents the sub application table column families. + */ +public enum SubApplicationColumnFamily + implements ColumnFamily { + + /** + * Info column family houses known columns, specifically ones included in + * columnfamily filters. + */ + INFO("i"), + + /** + * Configurations are in a separate column family for two reasons: + * a) the size of the config values can be very large and + * b) we expect that config values + * are often separately accessed from other metrics and info columns. + */ + CONFIGS("c"), + + /** + * Metrics have a separate column family, because they have a separate TTL. + */ + METRICS("m"); + + /** + * Byte representation of this column family. + */ + private final byte[] bytes; + + /** + * @param value + * create a column family with this name. Must be lower case and + * without spaces. + */ + SubApplicationColumnFamily(String value) { + // column families should be lower case and not contain any spaces. + this.bytes = Bytes.toBytes(Separator.SPACE.encode(value)); + } + + public byte[] getBytes() { + return Bytes.copy(bytes); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationColumnPrefix.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationColumnPrefix.java new file mode 100644 index 00000000000..06eccedf3a5 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationColumnPrefix.java @@ -0,0 +1,250 @@ +/** + * 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.timelineservice.storage.subapplication; + +import java.io.IOException; +import java.util.Map; +import java.util.NavigableMap; + +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; + +/** + * Identifies partially qualified columns for the sub app table. + */ +public enum SubApplicationColumnPrefix + implements ColumnPrefix { + + /** + * To store TimelineEntity getIsRelatedToEntities values. + */ + IS_RELATED_TO(SubApplicationColumnFamily.INFO, "s"), + + /** + * To store TimelineEntity getRelatesToEntities values. + */ + RELATES_TO(SubApplicationColumnFamily.INFO, "r"), + + /** + * To store TimelineEntity info values. + */ + INFO(SubApplicationColumnFamily.INFO, "i"), + + /** + * Lifecycle events for an entity. + */ + EVENT(SubApplicationColumnFamily.INFO, "e", true), + + /** + * Config column stores configuration with config key as the column name. + */ + CONFIG(SubApplicationColumnFamily.CONFIGS, null), + + /** + * Metrics are stored with the metric name as the column name. + */ + METRIC(SubApplicationColumnFamily.METRICS, null, new LongConverter()); + + private final ColumnHelper column; + private final ColumnFamily columnFamily; + + /** + * Can be null for those cases where the provided column qualifier is the + * entire column name. + */ + private final String columnPrefix; + private final byte[] columnPrefixBytes; + + /** + * Private constructor, meant to be used by the enum definition. + * + * @param columnFamily that this column is stored in. + * @param columnPrefix for this column. + */ + SubApplicationColumnPrefix(ColumnFamily columnFamily, + String columnPrefix) { + this(columnFamily, columnPrefix, false, GenericConverter.getInstance()); + } + + SubApplicationColumnPrefix(ColumnFamily columnFamily, + String columnPrefix, boolean compondColQual) { + this(columnFamily, columnPrefix, compondColQual, + GenericConverter.getInstance()); + } + + SubApplicationColumnPrefix(ColumnFamily columnFamily, + String columnPrefix, ValueConverter converter) { + this(columnFamily, columnPrefix, false, converter); + } + + /** + * Private constructor, meant to be used by the enum definition. + * + * @param columnFamily that this column is stored in. + * @param columnPrefix for this column. + * @param converter used to encode/decode values to be stored in HBase for + * this column prefix. + */ + SubApplicationColumnPrefix(ColumnFamily columnFamily, + String columnPrefix, boolean compondColQual, ValueConverter converter) { + column = new ColumnHelper(columnFamily, converter); + this.columnFamily = columnFamily; + this.columnPrefix = columnPrefix; + if (columnPrefix == null) { + this.columnPrefixBytes = null; + } else { + // Future-proof by ensuring the right column prefix hygiene. + this.columnPrefixBytes = + Bytes.toBytes(Separator.SPACE.encode(columnPrefix)); + } + } + + /** + * @return the column name value + */ + public String getColumnPrefix() { + return columnPrefix; + } + + @Override + public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) { + return ColumnHelper.getColumnQualifier( + this.columnPrefixBytes, qualifierPrefix); + } + + @Override + public byte[] getColumnPrefixBytes(String qualifierPrefix) { + return ColumnHelper.getColumnQualifier( + this.columnPrefixBytes, qualifierPrefix); + } + + @Override + public byte[] getColumnFamilyBytes() { + return columnFamily.getBytes(); + } + + @Override + public ValueConverter getValueConverter() { + return column.getValueConverter(); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #store(byte[], + * org.apache.hadoop.yarn.server.timelineservice.storage.common. + * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object, + * org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute[]) + */ + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, String qualifier, + Long timestamp, Object inputValue, Attribute... attributes) + throws IOException { + + // Null check + if (qualifier == null) { + throw new IOException("Cannot store column with null qualifier in " + + tableMutator.getName().getNameAsString()); + } + + byte[] columnQualifier = getColumnPrefixBytes(qualifier); + + column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue, + attributes); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #store(byte[], + * org.apache.hadoop.yarn.server.timelineservice.storage.common. + * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object) + */ + public void store(byte[] rowKey, + TypedBufferedMutator tableMutator, byte[] qualifier, + Long timestamp, Object inputValue, Attribute... attributes) + throws IOException { + + // Null check + if (qualifier == null) { + throw new IOException("Cannot store column with null qualifier in " + + tableMutator.getName().getNameAsString()); + } + + byte[] columnQualifier = getColumnPrefixBytes(qualifier); + + column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue, + attributes); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResult(org.apache.hadoop.hbase.client.Result, java.lang.String) + */ + public Object readResult(Result result, String qualifier) throws IOException { + byte[] columnQualifier = + ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier); + return column.readResult(result, columnQualifier); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResults(org.apache.hadoop.hbase.client.Result, + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter) + */ + public Map readResults(Result result, + KeyConverter keyConverter) throws IOException { + return column.readResults(result, columnPrefixBytes, keyConverter); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix + * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result, + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter) + */ + public NavigableMap> + readResultsWithTimestamps(Result result, KeyConverter keyConverter) + throws IOException { + return column.readResultsWithTimestamps(result, columnPrefixBytes, + keyConverter); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationRowKey.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationRowKey.java new file mode 100644 index 00000000000..fb1f774eaa0 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationRowKey.java @@ -0,0 +1,290 @@ +/** + * 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.timelineservice.storage.subapplication; + +import java.util.List; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverterToString; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; + +/** + * Represents a rowkey for the sub app table. + */ +public class SubApplicationRowKey { + private final String subAppUserId; + private final String clusterId; + private final String entityType; + private final Long entityIdPrefix; + private final String entityId; + private final String userId; + private final SubApplicationRowKeyConverter subAppRowKeyConverter = + new SubApplicationRowKeyConverter(); + + public SubApplicationRowKey(String subAppUserId, String clusterId, + String entityType, Long entityIdPrefix, String entityId, String userId) { + this.subAppUserId = subAppUserId; + this.clusterId = clusterId; + this.entityType = entityType; + this.entityIdPrefix = entityIdPrefix; + this.entityId = entityId; + this.userId = userId; + } + + public String getClusterId() { + return clusterId; + } + + public String getSubAppUserId() { + return subAppUserId; + } + + public String getEntityType() { + return entityType; + } + + public String getEntityId() { + return entityId; + } + + public Long getEntityIdPrefix() { + return entityIdPrefix; + } + + public String getUserId() { + return userId; + } + + /** + * Constructs a row key for the sub app table as follows: + * {@code subAppUserId!clusterId!entityType + * !entityPrefix!entityId!userId}. + * Typically used while querying a specific sub app. + * + * subAppUserId is usually the doAsUser. + * userId is the yarn user that the AM runs as. + * + * @return byte array with the row key. + */ + public byte[] getRowKey() { + return subAppRowKeyConverter.encode(this); + } + + /** + * Given the raw row key as bytes, returns the row key as an object. + * + * @param rowKey byte representation of row key. + * @return An SubApplicationRowKey object. + */ + public static SubApplicationRowKey parseRowKey(byte[] rowKey) { + return new SubApplicationRowKeyConverter().decode(rowKey); + } + + /** + * Constructs a row key for the sub app table as follows: + *

+ * {@code subAppUserId!clusterId! + * entityType!entityIdPrefix!entityId!userId}. + * + * subAppUserId is usually the doAsUser. + * userId is the yarn user that that the AM runs as. + * + *

+ * + * @return String representation of row key. + */ + public String getRowKeyAsString() { + return subAppRowKeyConverter.encodeAsString(this); + } + + /** + * Given the encoded row key as string, returns the row key as an object. + * + * @param encodedRowKey String representation of row key. + * @return A SubApplicationRowKey object. + */ + public static SubApplicationRowKey parseRowKeyFromString( + String encodedRowKey) { + return new SubApplicationRowKeyConverter().decodeFromString(encodedRowKey); + } + + /** + * Encodes and decodes row key for sub app table. + * The row key is of the form : + * subAppUserId!clusterId!flowRunId!appId!entityType!entityId!userId + * + * subAppUserId is usually the doAsUser. + * userId is the yarn user that the AM runs as. + * + *

+ */ + final private static class SubApplicationRowKeyConverter + implements KeyConverter, + KeyConverterToString { + + private SubApplicationRowKeyConverter() { + } + + /** + * sub app row key is of the form + * subAppUserId!clusterId!entityType!entityPrefix!entityId!userId + * w. each segment separated by !. + * + * subAppUserId is usually the doAsUser. + * userId is the yarn user that the AM runs as. + * + * The sizes below indicate sizes of each one of these + * segments in sequence. clusterId, subAppUserId, entityType, + * entityId and userId are strings. + * entity prefix is a long hence 8 bytes in size. Strings are + * variable in size (i.e. end whenever separator is encountered). + * This is used while decoding and helps in determining where to split. + */ + private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE, + Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, + Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE}; + + /* + * (non-Javadoc) + * + * Encodes SubApplicationRowKey object into a byte array with each + * component/field in SubApplicationRowKey separated by + * Separator#QUALIFIERS. + * This leads to an sub app table row key of the form + * subAppUserId!clusterId!entityType!entityPrefix!entityId!userId + * + * subAppUserId is usually the doAsUser. + * userId is the yarn user that the AM runs as. + * + * If entityType in passed SubApplicationRowKey object is null (and the + * fields preceding it are not null i.e. clusterId, subAppUserId), this + * returns a row key prefix of the form subAppUserId!clusterId! + * If entityId in SubApplicationRowKey is null + * (other components are not null), this returns a row key prefix + * of the form subAppUserId!clusterId!entityType! + * + * @see org.apache.hadoop.yarn.server.timelineservice.storage.common + * .KeyConverter#encode(java.lang.Object) + */ + @Override + public byte[] encode(SubApplicationRowKey rowKey) { + byte[] subAppUser = Separator.encode(rowKey.getSubAppUserId(), + Separator.SPACE, Separator.TAB, Separator.QUALIFIERS); + byte[] cluster = Separator.encode(rowKey.getClusterId(), Separator.SPACE, + Separator.TAB, Separator.QUALIFIERS); + byte[] first = Separator.QUALIFIERS.join(subAppUser, cluster); + if (rowKey.getEntityType() == null) { + return first; + } + byte[] entityType = Separator.encode(rowKey.getEntityType(), + Separator.SPACE, Separator.TAB, Separator.QUALIFIERS); + + if (rowKey.getEntityIdPrefix() == null) { + return Separator.QUALIFIERS.join(first, entityType, + Separator.EMPTY_BYTES); + } + + byte[] entityIdPrefix = Bytes.toBytes(rowKey.getEntityIdPrefix()); + + if (rowKey.getEntityId() == null) { + return Separator.QUALIFIERS.join(first, entityType, entityIdPrefix, + Separator.EMPTY_BYTES); + } + + byte[] entityId = Separator.encode(rowKey.getEntityId(), Separator.SPACE, + Separator.TAB, Separator.QUALIFIERS); + + byte[] userId = Separator.encode(rowKey.getUserId(), + Separator.SPACE, Separator.TAB, Separator.QUALIFIERS); + + byte[] second = Separator.QUALIFIERS.join(entityType, entityIdPrefix, + entityId, userId); + + return Separator.QUALIFIERS.join(first, second); + } + + /* + * (non-Javadoc) + * + * Decodes a sub application row key of the form + * subAppUserId!clusterId!entityType!entityPrefix!entityId!userId + * + * subAppUserId is usually the doAsUser. + * userId is the yarn user that the AM runs as. + * + * represented in byte format + * and converts it into an SubApplicationRowKey object. + * + * @see org.apache.hadoop.yarn.server.timelineservice.storage.common + * .KeyConverter#decode(byte[]) + */ + @Override + public SubApplicationRowKey decode(byte[] rowKey) { + byte[][] rowKeyComponents = + Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES); + if (rowKeyComponents.length != 6) { + throw new IllegalArgumentException( + "the row key is not valid for " + "a sub app"); + } + String subAppUserId = + Separator.decode(Bytes.toString(rowKeyComponents[0]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + String clusterId = Separator.decode(Bytes.toString(rowKeyComponents[1]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + String entityType = Separator.decode(Bytes.toString(rowKeyComponents[2]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + + Long entityPrefixId = Bytes.toLong(rowKeyComponents[3]); + + String entityId = Separator.decode(Bytes.toString(rowKeyComponents[4]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + String userId = + Separator.decode(Bytes.toString(rowKeyComponents[5]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + + return new SubApplicationRowKey(subAppUserId, clusterId, entityType, + entityPrefixId, entityId, userId); + } + + @Override + public String encodeAsString(SubApplicationRowKey key) { + if (key.subAppUserId == null || key.clusterId == null + || key.entityType == null || key.entityIdPrefix == null + || key.entityId == null || key.userId == null) { + throw new IllegalArgumentException(); + } + return TimelineReaderUtils.joinAndEscapeStrings( + new String[] {key.subAppUserId, key.clusterId, key.entityType, + key.entityIdPrefix.toString(), key.entityId, key.userId}); + } + + @Override + public SubApplicationRowKey decodeFromString(String encodedRowKey) { + List split = TimelineReaderUtils.split(encodedRowKey); + if (split == null || split.size() != 6) { + throw new IllegalArgumentException( + "Invalid row key for sub app table."); + } + Long entityIdPrefix = Long.valueOf(split.get(3)); + return new SubApplicationRowKey(split.get(0), split.get(1), + split.get(2), entityIdPrefix, split.get(4), split.get(5)); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationRowKeyPrefix.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationRowKeyPrefix.java new file mode 100644 index 00000000000..0c049596583 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationRowKeyPrefix.java @@ -0,0 +1,69 @@ +/** + * 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.timelineservice.storage.subapplication; + +import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix; + +/** + * Represents a partial rowkey without the entityId or without entityType and + * entityId for the sub application table. + * + */ +public class SubApplicationRowKeyPrefix extends SubApplicationRowKey + implements RowKeyPrefix { + + /** + * Creates a prefix which generates the following rowKeyPrefixes for the sub + * application table: + * {@code subAppUserId!clusterId!entityType!entityPrefix!userId}. + * + * @param subAppUserId + * identifying the subApp User + * @param clusterId + * identifying the cluster + * @param entityType + * which entity type + * @param entityIdPrefix + * for entityId + * @param entityId + * for an entity + * @param userId + * for the user who runs the AM + * + * subAppUserId is usually the doAsUser. + * userId is the yarn user that the AM runs as. + * + */ + public SubApplicationRowKeyPrefix(String subAppUserId, String clusterId, + String entityType, Long entityIdPrefix, String entityId, + String userId) { + super(subAppUserId, clusterId, entityType, entityIdPrefix, entityId, + userId); + } + + /* + * (non-Javadoc) + * + * @see org.apache.hadoop.yarn.server.timelineservice.storage.subapplication. + * RowKeyPrefix#getRowKeyPrefix() + */ + public byte[] getRowKeyPrefix() { + return super.getRowKey(); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationTable.java new file mode 100644 index 00000000000..785a243d93d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationTable.java @@ -0,0 +1,174 @@ +/** + * 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.timelineservice.storage.subapplication; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The sub application table has column families: + * info, config and metrics. + * Info stores information about a timeline entity object + * config stores configuration data of a timeline entity object + * metrics stores the metrics of a timeline entity object + * + * Example sub application table record: + * + *

+ * |-------------------------------------------------------------------------|
+ * |  Row          | Column Family             | Column Family| Column Family|
+ * |  key          | info                      | metrics      | config       |
+ * |-------------------------------------------------------------------------|
+ * | subAppUserId! | id:entityId               | metricId1:   | configKey1:  |
+ * | clusterId!    | type:entityType           | metricValue1 | configValue1 |
+ * | entityType!   |                           | @timestamp1  |              |
+ * | idPrefix!|    |                           |              | configKey2:  |
+ * | entityId!     | created_time:             | metricId1:   | configValue2 |
+ * | userId        | 1392993084018             | metricValue2 |              |
+ * |               |                           | @timestamp2  |              |
+ * |               | i!infoKey:                |              |              |
+ * |               | infoValue                 | metricId1:   |              |
+ * |               |                           | metricValue1 |              |
+ * |               |                           | @timestamp2  |              |
+ * |               | e!eventId=timestamp=      |              |              |
+ * |               | infoKey:                  |              |              |
+ * |               | eventInfoValue            |              |              |
+ * |               |                           |              |              |
+ * |               | r!relatesToKey:           |              |              |
+ * |               | id3=id4=id5               |              |              |
+ * |               |                           |              |              |
+ * |               | s!isRelatedToKey          |              |              |
+ * |               | id7=id9=id6               |              |              |
+ * |               |                           |              |              |
+ * |               | flowVersion:              |              |              |
+ * |               | versionValue              |              |              |
+ * |-------------------------------------------------------------------------|
+ * 
+ */ +public class SubApplicationTable extends BaseTable { + /** sub app prefix. */ + private static final String PREFIX = + YarnConfiguration.TIMELINE_SERVICE_PREFIX + "subapplication"; + + /** config param name that specifies the subapplication table name. */ + public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name"; + + /** + * config param name that specifies the TTL for metrics column family in + * subapplication table. + */ + private static final String METRICS_TTL_CONF_NAME = PREFIX + + ".table.metrics.ttl"; + + /** + * config param name that specifies max-versions for + * metrics column family in subapplication table. + */ + private static final String METRICS_MAX_VERSIONS = + PREFIX + ".table.metrics.max-versions"; + + /** default value for subapplication table name. */ + public static final String DEFAULT_TABLE_NAME = + "timelineservice.subapplication"; + + /** default TTL is 30 days for metrics timeseries. */ + private static final int DEFAULT_METRICS_TTL = 2592000; + + /** default max number of versions. */ + private static final int DEFAULT_METRICS_MAX_VERSIONS = 10000; + + private static final Logger LOG = LoggerFactory.getLogger( + SubApplicationTable.class); + + public SubApplicationTable() { + super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable + * (org.apache.hadoop.hbase.client.Admin, + * org.apache.hadoop.conf.Configuration) + */ + public void createTable(Admin admin, Configuration hbaseConf) + throws IOException { + + TableName table = getTableName(hbaseConf); + if (admin.tableExists(table)) { + // do not disable / delete existing table + // similar to the approach taken by map-reduce jobs when + // output directory exists + throw new IOException("Table " + table.getNameAsString() + + " already exists."); + } + + HTableDescriptor subAppTableDescp = new HTableDescriptor(table); + HColumnDescriptor infoCF = + new HColumnDescriptor(SubApplicationColumnFamily.INFO.getBytes()); + infoCF.setBloomFilterType(BloomType.ROWCOL); + subAppTableDescp.addFamily(infoCF); + + HColumnDescriptor configCF = + new HColumnDescriptor(SubApplicationColumnFamily.CONFIGS.getBytes()); + configCF.setBloomFilterType(BloomType.ROWCOL); + configCF.setBlockCacheEnabled(true); + subAppTableDescp.addFamily(configCF); + + HColumnDescriptor metricsCF = + new HColumnDescriptor(SubApplicationColumnFamily.METRICS.getBytes()); + subAppTableDescp.addFamily(metricsCF); + metricsCF.setBlockCacheEnabled(true); + // always keep 1 version (the latest) + metricsCF.setMinVersions(1); + metricsCF.setMaxVersions( + hbaseConf.getInt(METRICS_MAX_VERSIONS, DEFAULT_METRICS_MAX_VERSIONS)); + metricsCF.setTimeToLive(hbaseConf.getInt(METRICS_TTL_CONF_NAME, + DEFAULT_METRICS_TTL)); + subAppTableDescp.setRegionSplitPolicyClassName( + "org.apache.hadoop.hbase.regionserver.KeyPrefixRegionSplitPolicy"); + subAppTableDescp.setValue("KeyPrefixRegionSplitPolicy.prefix_length", + TimelineHBaseSchemaConstants.USERNAME_SPLIT_KEY_PREFIX_LENGTH); + admin.createTable(subAppTableDescp, + TimelineHBaseSchemaConstants.getUsernameSplits()); + LOG.info("Status of table creation for " + table.getNameAsString() + "=" + + admin.tableExists(table)); + } + + /** + * @param metricsTTL time to live parameter for the metricss in this table. + * @param hbaseConf configururation in which to set the metrics TTL config + * variable. + */ + public void setMetricsTTL(int metricsTTL, Configuration hbaseConf) { + hbaseConf.setInt(METRICS_TTL_CONF_NAME, metricsTTL); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/package-info.java new file mode 100644 index 00000000000..52cc39964d5 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/package-info.java @@ -0,0 +1,28 @@ +/* + * 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.timelineservice.storage.subapplication + * contains classes related to implementation for subapplication table. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.timelineservice.storage.subapplication; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationUpdatedEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestCustomApplicationIdConversion.java similarity index 50% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationUpdatedEvent.java rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestCustomApplicationIdConversion.java index 9e5e1fd985f..73bc29efc48 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationUpdatedEvent.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestCustomApplicationIdConversion.java @@ -15,40 +15,25 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - -package org.apache.hadoop.yarn.server.resourcemanager.metrics; +package org.apache.hadoop.yarn.server.timelineservice.storage.common; import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.Priority; +import org.junit.Assert; +import org.junit.Test; -public class ApplicationUpdatedEvent extends SystemMetricsEvent { - - private ApplicationId appId; - private String queue; - private Priority applicationPriority; - - public ApplicationUpdatedEvent(ApplicationId appId, String queue, - long updatedTime, Priority applicationPriority) { - super(SystemMetricsEventType.APP_UPDATED, updatedTime); - this.appId = appId; - this.queue = queue; - this.applicationPriority = applicationPriority; +/** + * Test for HBaseTimelineStorageUtils.convertApplicationIdToString(), + * a custom conversion from ApplicationId to String that avoids the + * incompatibility issue caused by mixing hadoop-common 2.5.1 and + * hadoop-yarn-api 3.0. See YARN-6905. + */ +public class TestCustomApplicationIdConversion { + @Test + public void testConvertAplicationIdToString() { + ApplicationId applicationId = ApplicationId.newInstance(0, 1); + String applicationIdStr = + HBaseTimelineStorageUtils.convertApplicationIdToString(applicationId); + Assert.assertEquals(applicationId, + ApplicationId.fromString(applicationIdStr)); } - - @Override - public int hashCode() { - return appId.hashCode(); - } - - public ApplicationId getApplicationId() { - return appId; - } - - public String getQueue() { - return queue; - } - - public Priority getApplicationPriority() { - return applicationPriority; - } -} \ No newline at end of file +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestHBaseTimelineStorageUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestHBaseTimelineStorageUtils.java new file mode 100644 index 00000000000..402a89bf380 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestHBaseTimelineStorageUtils.java @@ -0,0 +1,33 @@ +/** + * 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.timelineservice.storage.common; + +import org.junit.Test; + +/** + * Unit tests for HBaseTimelineStorageUtils static methos. + */ +public class TestHBaseTimelineStorageUtils { + + @Test(expected=NullPointerException.class) + public void testGetTimelineServiceHBaseConfNullArgument() throws Exception { + HBaseTimelineStorageUtils.getTimelineServiceHBaseConf(null); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestKeyConverters.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestKeyConverters.java new file mode 100644 index 00000000000..1bd363fd73a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestKeyConverters.java @@ -0,0 +1,134 @@ +/** + * 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.timelineservice.storage.common; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.junit.Test; + +/** + * Unit tests for key converters for various tables' row keys. + * + */ +public class TestKeyConverters { + + @Test + public void testAppIdKeyConverter() { + AppIdKeyConverter appIdKeyConverter = new AppIdKeyConverter(); + long currentTs = System.currentTimeMillis(); + ApplicationId appId1 = ApplicationId.newInstance(currentTs, 1); + ApplicationId appId2 = ApplicationId.newInstance(currentTs, 2); + ApplicationId appId3 = ApplicationId.newInstance(currentTs + 300, 1); + String appIdStr1 = appId1.toString(); + String appIdStr2 = appId2.toString(); + String appIdStr3 = appId3.toString(); + byte[] appIdBytes1 = appIdKeyConverter.encode(appIdStr1); + byte[] appIdBytes2 = appIdKeyConverter.encode(appIdStr2); + byte[] appIdBytes3 = appIdKeyConverter.encode(appIdStr3); + // App ids' should be encoded in a manner wherein descending order + // is maintained. + assertTrue( + "Ordering of app ids' is incorrect", + Bytes.compareTo(appIdBytes1, appIdBytes2) > 0 + && Bytes.compareTo(appIdBytes1, appIdBytes3) > 0 + && Bytes.compareTo(appIdBytes2, appIdBytes3) > 0); + String decodedAppId1 = appIdKeyConverter.decode(appIdBytes1); + String decodedAppId2 = appIdKeyConverter.decode(appIdBytes2); + String decodedAppId3 = appIdKeyConverter.decode(appIdBytes3); + assertTrue("Decoded app id is not same as the app id encoded", + appIdStr1.equals(decodedAppId1)); + assertTrue("Decoded app id is not same as the app id encoded", + appIdStr2.equals(decodedAppId2)); + assertTrue("Decoded app id is not same as the app id encoded", + appIdStr3.equals(decodedAppId3)); + } + + @Test + public void testEventColumnNameConverter() { + String eventId = "=foo_=eve=nt="; + byte[] valSepBytes = Bytes.toBytes(Separator.VALUES.getValue()); + byte[] maxByteArr = + Bytes.createMaxByteArray(Bytes.SIZEOF_LONG - valSepBytes.length); + byte[] ts = Bytes.add(valSepBytes, maxByteArr); + Long eventTs = Bytes.toLong(ts); + byte[] byteEventColName = + new EventColumnName(eventId, eventTs, null).getColumnQualifier(); + KeyConverter eventColumnNameConverter = + new EventColumnNameConverter(); + EventColumnName eventColName = + eventColumnNameConverter.decode(byteEventColName); + assertEquals(eventId, eventColName.getId()); + assertEquals(eventTs, eventColName.getTimestamp()); + assertNull(eventColName.getInfoKey()); + + String infoKey = "f=oo_event_in=fo=_key"; + byteEventColName = + new EventColumnName(eventId, eventTs, infoKey).getColumnQualifier(); + eventColName = eventColumnNameConverter.decode(byteEventColName); + assertEquals(eventId, eventColName.getId()); + assertEquals(eventTs, eventColName.getTimestamp()); + assertEquals(infoKey, eventColName.getInfoKey()); + } + + @Test + public void testLongKeyConverter() { + LongKeyConverter longKeyConverter = new LongKeyConverter(); + confirmLongKeyConverter(longKeyConverter, Long.MIN_VALUE); + confirmLongKeyConverter(longKeyConverter, -1234567890L); + confirmLongKeyConverter(longKeyConverter, -128L); + confirmLongKeyConverter(longKeyConverter, -127L); + confirmLongKeyConverter(longKeyConverter, -1L); + confirmLongKeyConverter(longKeyConverter, 0L); + confirmLongKeyConverter(longKeyConverter, 1L); + confirmLongKeyConverter(longKeyConverter, 127L); + confirmLongKeyConverter(longKeyConverter, 128L); + confirmLongKeyConverter(longKeyConverter, 1234567890L); + confirmLongKeyConverter(longKeyConverter, Long.MAX_VALUE); + } + + private void confirmLongKeyConverter(LongKeyConverter longKeyConverter, + Long testValue) { + Long decoded = longKeyConverter.decode(longKeyConverter.encode(testValue)); + assertEquals(testValue, decoded); + } + + @Test + public void testStringKeyConverter() { + StringKeyConverter stringKeyConverter = new StringKeyConverter(); + String phrase = "QuackAttack now!"; + + for (int i = 0; i < phrase.length(); i++) { + String sub = phrase.substring(i, phrase.length()); + confirmStrignKeyConverter(stringKeyConverter, sub); + confirmStrignKeyConverter(stringKeyConverter, sub + sub); + } + } + + private void confirmStrignKeyConverter(StringKeyConverter stringKeyConverter, + String testValue) { + String decoded = + stringKeyConverter.decode(stringKeyConverter.encode(testValue)); + assertEquals(testValue, decoded); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java new file mode 100644 index 00000000000..47702383ee2 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java @@ -0,0 +1,276 @@ +/** + * 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.timelineservice.storage.common; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKeyPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKeyPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKeyPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationRowKey; +import org.junit.Test; + + +/** + * Class to test the row key structures for various tables. + * + */ +public class TestRowKeys { + + private final static String QUALIFIER_SEP = Separator.QUALIFIERS.getValue(); + private final static byte[] QUALIFIER_SEP_BYTES = Bytes + .toBytes(QUALIFIER_SEP); + private final static String CLUSTER = "cl" + QUALIFIER_SEP + "uster"; + private final static String USER = QUALIFIER_SEP + "user"; + private final static String SUB_APP_USER = QUALIFIER_SEP + "subAppUser"; + private final static String FLOW_NAME = "dummy_" + QUALIFIER_SEP + "flow" + + QUALIFIER_SEP; + private final static Long FLOW_RUN_ID; + private final static String APPLICATION_ID; + static { + long runid = Long.MAX_VALUE - 900L; + byte[] longMaxByteArr = Bytes.toBytes(Long.MAX_VALUE); + byte[] byteArr = Bytes.toBytes(runid); + int sepByteLen = QUALIFIER_SEP_BYTES.length; + if (sepByteLen <= byteArr.length) { + for (int i = 0; i < sepByteLen; i++) { + byteArr[i] = (byte) (longMaxByteArr[i] - QUALIFIER_SEP_BYTES[i]); + } + } + FLOW_RUN_ID = Bytes.toLong(byteArr); + long clusterTs = System.currentTimeMillis(); + byteArr = Bytes.toBytes(clusterTs); + if (sepByteLen <= byteArr.length) { + for (int i = 0; i < sepByteLen; i++) { + byteArr[byteArr.length - sepByteLen + i] = + (byte) (longMaxByteArr[byteArr.length - sepByteLen + i] - + QUALIFIER_SEP_BYTES[i]); + } + } + clusterTs = Bytes.toLong(byteArr); + int seqId = 222; + APPLICATION_ID = ApplicationId.newInstance(clusterTs, seqId).toString(); + } + + private static void verifyRowPrefixBytes(byte[] byteRowKeyPrefix) { + int sepLen = QUALIFIER_SEP_BYTES.length; + for (int i = 0; i < sepLen; i++) { + assertTrue( + "Row key prefix not encoded properly.", + byteRowKeyPrefix[byteRowKeyPrefix.length - sepLen + i] == + QUALIFIER_SEP_BYTES[i]); + } + } + + @Test + public void testApplicationRowKey() { + byte[] byteRowKey = + new ApplicationRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, + APPLICATION_ID).getRowKey(); + ApplicationRowKey rowKey = ApplicationRowKey.parseRowKey(byteRowKey); + assertEquals(CLUSTER, rowKey.getClusterId()); + assertEquals(USER, rowKey.getUserId()); + assertEquals(FLOW_NAME, rowKey.getFlowName()); + assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId()); + assertEquals(APPLICATION_ID, rowKey.getAppId()); + + byte[] byteRowKeyPrefix = + new ApplicationRowKeyPrefix(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID) + .getRowKeyPrefix(); + byte[][] splits = + Separator.QUALIFIERS.split(byteRowKeyPrefix, + new int[] {Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, + Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, + Separator.VARIABLE_SIZE}); + assertEquals(5, splits.length); + assertEquals(0, splits[4].length); + assertEquals(FLOW_NAME, + Separator.QUALIFIERS.decode(Bytes.toString(splits[2]))); + assertEquals(FLOW_RUN_ID, + (Long) LongConverter.invertLong(Bytes.toLong(splits[3]))); + verifyRowPrefixBytes(byteRowKeyPrefix); + + byteRowKeyPrefix = + new ApplicationRowKeyPrefix(CLUSTER, USER, FLOW_NAME).getRowKeyPrefix(); + splits = + Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] { + Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, + Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE }); + assertEquals(4, splits.length); + assertEquals(0, splits[3].length); + assertEquals(FLOW_NAME, + Separator.QUALIFIERS.decode(Bytes.toString(splits[2]))); + verifyRowPrefixBytes(byteRowKeyPrefix); + } + + /** + * Tests the converters indirectly through the public methods of the + * corresponding rowkey. + */ + @Test + public void testAppToFlowRowKey() { + byte[] byteRowKey = new AppToFlowRowKey(APPLICATION_ID).getRowKey(); + AppToFlowRowKey rowKey = AppToFlowRowKey.parseRowKey(byteRowKey); + assertEquals(APPLICATION_ID, rowKey.getAppId()); + } + + @Test + public void testEntityRowKey() { + TimelineEntity entity = new TimelineEntity(); + entity.setId("!ent!ity!!id!"); + entity.setType("entity!Type"); + entity.setIdPrefix(54321); + + byte[] byteRowKey = + new EntityRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, APPLICATION_ID, + entity.getType(), entity.getIdPrefix(), + entity.getId()).getRowKey(); + EntityRowKey rowKey = EntityRowKey.parseRowKey(byteRowKey); + assertEquals(CLUSTER, rowKey.getClusterId()); + assertEquals(USER, rowKey.getUserId()); + assertEquals(FLOW_NAME, rowKey.getFlowName()); + assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId()); + assertEquals(APPLICATION_ID, rowKey.getAppId()); + assertEquals(entity.getType(), rowKey.getEntityType()); + assertEquals(entity.getIdPrefix(), rowKey.getEntityIdPrefix().longValue()); + assertEquals(entity.getId(), rowKey.getEntityId()); + + byte[] byteRowKeyPrefix = + new EntityRowKeyPrefix(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, + APPLICATION_ID, entity.getType(), null, null) + .getRowKeyPrefix(); + byte[][] splits = + Separator.QUALIFIERS.split( + byteRowKeyPrefix, + new int[] {Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, + Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, + AppIdKeyConverter.getKeySize(), Separator.VARIABLE_SIZE, + Bytes.SIZEOF_LONG, Separator.VARIABLE_SIZE }); + assertEquals(7, splits.length); + assertEquals(APPLICATION_ID, new AppIdKeyConverter().decode(splits[4])); + assertEquals(entity.getType(), + Separator.QUALIFIERS.decode(Bytes.toString(splits[5]))); + verifyRowPrefixBytes(byteRowKeyPrefix); + + byteRowKeyPrefix = + new EntityRowKeyPrefix(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, + APPLICATION_ID).getRowKeyPrefix(); + splits = + Separator.QUALIFIERS.split( + byteRowKeyPrefix, + new int[] {Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, + Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, + AppIdKeyConverter.getKeySize(), Separator.VARIABLE_SIZE}); + assertEquals(6, splits.length); + assertEquals(0, splits[5].length); + AppIdKeyConverter appIdKeyConverter = new AppIdKeyConverter(); + assertEquals(APPLICATION_ID, appIdKeyConverter.decode(splits[4])); + verifyRowPrefixBytes(byteRowKeyPrefix); + } + + @Test + public void testFlowActivityRowKey() { + Long ts = 1459900830000L; + Long dayTimestamp = HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(ts); + byte[] byteRowKey = + new FlowActivityRowKey(CLUSTER, ts, USER, FLOW_NAME).getRowKey(); + FlowActivityRowKey rowKey = FlowActivityRowKey.parseRowKey(byteRowKey); + assertEquals(CLUSTER, rowKey.getClusterId()); + assertEquals(dayTimestamp, rowKey.getDayTimestamp()); + assertEquals(USER, rowKey.getUserId()); + assertEquals(FLOW_NAME, rowKey.getFlowName()); + + byte[] byteRowKeyPrefix = + new FlowActivityRowKeyPrefix(CLUSTER).getRowKeyPrefix(); + byte[][] splits = + Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] { + Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE }); + assertEquals(2, splits.length); + assertEquals(0, splits[1].length); + assertEquals(CLUSTER, + Separator.QUALIFIERS.decode(Bytes.toString(splits[0]))); + verifyRowPrefixBytes(byteRowKeyPrefix); + + byteRowKeyPrefix = + new FlowActivityRowKeyPrefix(CLUSTER, ts).getRowKeyPrefix(); + splits = + Separator.QUALIFIERS.split(byteRowKeyPrefix, + new int[] {Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, + Separator.VARIABLE_SIZE}); + assertEquals(3, splits.length); + assertEquals(0, splits[2].length); + assertEquals(CLUSTER, + Separator.QUALIFIERS.decode(Bytes.toString(splits[0]))); + assertEquals(ts, + (Long) LongConverter.invertLong(Bytes.toLong(splits[1]))); + verifyRowPrefixBytes(byteRowKeyPrefix); + } + + @Test + public void testFlowRunRowKey() { + byte[] byteRowKey = + new FlowRunRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID).getRowKey(); + FlowRunRowKey rowKey = FlowRunRowKey.parseRowKey(byteRowKey); + assertEquals(CLUSTER, rowKey.getClusterId()); + assertEquals(USER, rowKey.getUserId()); + assertEquals(FLOW_NAME, rowKey.getFlowName()); + assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId()); + + byte[] byteRowKeyPrefix = + new FlowRunRowKey(CLUSTER, USER, FLOW_NAME, null).getRowKey(); + byte[][] splits = + Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] { + Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, + Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE }); + assertEquals(4, splits.length); + assertEquals(0, splits[3].length); + assertEquals(FLOW_NAME, + Separator.QUALIFIERS.decode(Bytes.toString(splits[2]))); + verifyRowPrefixBytes(byteRowKeyPrefix); + } + + @Test + public void testSubAppRowKey() { + TimelineEntity entity = new TimelineEntity(); + entity.setId("entity1"); + entity.setType("DAG"); + entity.setIdPrefix(54321); + + byte[] byteRowKey = + new SubApplicationRowKey(SUB_APP_USER, CLUSTER, + entity.getType(), entity.getIdPrefix(), + entity.getId(), USER).getRowKey(); + SubApplicationRowKey rowKey = SubApplicationRowKey.parseRowKey(byteRowKey); + assertEquals(CLUSTER, rowKey.getClusterId()); + assertEquals(SUB_APP_USER, rowKey.getSubAppUserId()); + assertEquals(entity.getType(), rowKey.getEntityType()); + assertEquals(entity.getIdPrefix(), rowKey.getEntityIdPrefix().longValue()); + assertEquals(entity.getId(), rowKey.getEntityId()); + assertEquals(USER, rowKey.getUserId()); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeysAsString.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeysAsString.java new file mode 100644 index 00000000000..148cf567c45 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeysAsString.java @@ -0,0 +1,144 @@ +/** + * 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.timelineservice.storage.common; + +import static org.junit.Assert.assertEquals; + +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKey; +import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationRowKey; +import org.junit.Test; + +/** + * Test for row key as string. + */ +public class TestRowKeysAsString { + + private final static String CLUSTER = + "cl" + TimelineReaderUtils.DEFAULT_DELIMITER_CHAR + "uster" + + TimelineReaderUtils.DEFAULT_ESCAPE_CHAR; + private final static String USER = + TimelineReaderUtils.DEFAULT_ESCAPE_CHAR + "user"; + private final static String SUB_APP_USER = + TimelineReaderUtils.DEFAULT_ESCAPE_CHAR + "subAppUser"; + + private final static String FLOW_NAME = + "dummy_" + TimelineReaderUtils.DEFAULT_DELIMITER_CHAR + + TimelineReaderUtils.DEFAULT_ESCAPE_CHAR + "flow" + + TimelineReaderUtils.DEFAULT_DELIMITER_CHAR; + private final static Long FLOW_RUN_ID = System.currentTimeMillis(); + private final static String APPLICATION_ID = + ApplicationId.newInstance(System.currentTimeMillis(), 1).toString(); + + @Test(timeout = 10000) + public void testApplicationRow() { + String rowKeyAsString = new ApplicationRowKey(CLUSTER, USER, FLOW_NAME, + FLOW_RUN_ID, APPLICATION_ID).getRowKeyAsString(); + ApplicationRowKey rowKey = + ApplicationRowKey.parseRowKeyFromString(rowKeyAsString); + assertEquals(CLUSTER, rowKey.getClusterId()); + assertEquals(USER, rowKey.getUserId()); + assertEquals(FLOW_NAME, rowKey.getFlowName()); + assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId()); + assertEquals(APPLICATION_ID, rowKey.getAppId()); + } + + @Test(timeout = 10000) + public void testEntityRowKey() { + char del = TimelineReaderUtils.DEFAULT_DELIMITER_CHAR; + char esc = TimelineReaderUtils.DEFAULT_ESCAPE_CHAR; + String id = del + esc + "ent" + esc + del + "ity" + esc + del + esc + "id" + + esc + del + esc; + String type = "entity" + esc + del + esc + "Type"; + TimelineEntity entity = new TimelineEntity(); + entity.setId(id); + entity.setType(type); + entity.setIdPrefix(54321); + + String rowKeyAsString = + new EntityRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, APPLICATION_ID, + entity.getType(), entity.getIdPrefix(), entity.getId()) + .getRowKeyAsString(); + EntityRowKey rowKey = EntityRowKey.parseRowKeyFromString(rowKeyAsString); + assertEquals(CLUSTER, rowKey.getClusterId()); + assertEquals(USER, rowKey.getUserId()); + assertEquals(FLOW_NAME, rowKey.getFlowName()); + assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId()); + assertEquals(APPLICATION_ID, rowKey.getAppId()); + assertEquals(entity.getType(), rowKey.getEntityType()); + assertEquals(entity.getIdPrefix(), rowKey.getEntityIdPrefix().longValue()); + assertEquals(entity.getId(), rowKey.getEntityId()); + + } + + @Test(timeout = 10000) + public void testFlowActivityRowKey() { + Long ts = 1459900830000L; + Long dayTimestamp = HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(ts); + String rowKeyAsString = new FlowActivityRowKey(CLUSTER, ts, USER, FLOW_NAME) + .getRowKeyAsString(); + FlowActivityRowKey rowKey = + FlowActivityRowKey.parseRowKeyFromString(rowKeyAsString); + assertEquals(CLUSTER, rowKey.getClusterId()); + assertEquals(dayTimestamp, rowKey.getDayTimestamp()); + assertEquals(USER, rowKey.getUserId()); + assertEquals(FLOW_NAME, rowKey.getFlowName()); + } + + @Test(timeout = 10000) + public void testFlowRunRowKey() { + String rowKeyAsString = + new FlowRunRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID) + .getRowKeyAsString(); + FlowRunRowKey rowKey = FlowRunRowKey.parseRowKeyFromString(rowKeyAsString); + assertEquals(CLUSTER, rowKey.getClusterId()); + assertEquals(USER, rowKey.getUserId()); + assertEquals(FLOW_NAME, rowKey.getFlowName()); + assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId()); + } + + @Test(timeout = 10000) + public void testSubApplicationRowKey() { + char del = TimelineReaderUtils.DEFAULT_DELIMITER_CHAR; + char esc = TimelineReaderUtils.DEFAULT_ESCAPE_CHAR; + String id = del + esc + "ent" + esc + del + "ity" + esc + del + esc + "id" + + esc + del + esc; + String type = "entity" + esc + del + esc + "Type"; + TimelineEntity entity = new TimelineEntity(); + entity.setId(id); + entity.setType(type); + entity.setIdPrefix(54321); + + String rowKeyAsString = new SubApplicationRowKey(SUB_APP_USER, CLUSTER, + entity.getType(), entity.getIdPrefix(), entity.getId(), USER) + .getRowKeyAsString(); + SubApplicationRowKey rowKey = SubApplicationRowKey + .parseRowKeyFromString(rowKeyAsString); + assertEquals(SUB_APP_USER, rowKey.getSubAppUserId()); + assertEquals(CLUSTER, rowKey.getClusterId()); + assertEquals(entity.getType(), rowKey.getEntityType()); + assertEquals(entity.getIdPrefix(), rowKey.getEntityIdPrefix().longValue()); + assertEquals(entity.getId(), rowKey.getEntityId()); + assertEquals(USER, rowKey.getUserId()); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestSeparator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestSeparator.java new file mode 100644 index 00000000000..30046f1cecf --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestSeparator.java @@ -0,0 +1,219 @@ +/** + * 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.timelineservice.storage.common; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.Test; + +import com.google.common.collect.Iterables; + +/** + * Tests the encoding and decoding of separators. Typically used to separate + * different segments in HBase row key. + */ +public class TestSeparator { + + private static String villain = "Dr. Heinz Doofenshmirtz"; + private static String special = + ". * | ? + \t ( ) [ ] { } ^ $ \\ \" %"; + + /** + * + */ + @Test + public void testEncodeDecodeString() { + + for (Separator separator : Separator.values()) { + testEncodeDecode(separator, ""); + testEncodeDecode(separator, " "); + testEncodeDecode(separator, "!"); + testEncodeDecode(separator, "?"); + testEncodeDecode(separator, "&"); + testEncodeDecode(separator, "+"); + testEncodeDecode(separator, "\t"); + testEncodeDecode(separator, "Dr."); + testEncodeDecode(separator, "Heinz"); + testEncodeDecode(separator, "Doofenshmirtz"); + testEncodeDecode(separator, villain); + testEncodeDecode(separator, special); + + assertNull(separator.encode(null)); + + } + } + + private void testEncodeDecode(Separator separator, String token) { + String encoded = separator.encode(token); + String decoded = separator.decode(encoded); + String msg = "token:" + token + " separator:" + separator + "."; + assertEquals(msg, token, decoded); + } + + @Test + public void testEncodeDecode() { + testEncodeDecode("Dr.", Separator.QUALIFIERS); + testEncodeDecode("Heinz", Separator.QUALIFIERS, Separator.QUALIFIERS); + testEncodeDecode("Doofenshmirtz", Separator.QUALIFIERS, null, + Separator.QUALIFIERS); + testEncodeDecode("&Perry", Separator.QUALIFIERS, Separator.VALUES, null); + testEncodeDecode("the ", Separator.QUALIFIERS, Separator.SPACE); + testEncodeDecode("Platypus...", (Separator) null); + testEncodeDecode("The what now ?!?", Separator.QUALIFIERS, + Separator.VALUES, Separator.SPACE); + + } + @Test + public void testEncodedValues() { + testEncodeDecode("Double-escape %2$ and %9$ or %%2$ or %%3$, nor %%%2$" + + "= no problem!", + Separator.QUALIFIERS, Separator.VALUES, Separator.SPACE, Separator.TAB); + } + + @Test + public void testSplits() { + byte[] maxLongBytes = Bytes.toBytes(Long.MAX_VALUE); + byte[] maxIntBytes = Bytes.toBytes(Integer.MAX_VALUE); + for (Separator separator : Separator.values()) { + String str1 = "cl" + separator.getValue() + "us"; + String str2 = separator.getValue() + "rst"; + byte[] sepByteArr = Bytes.toBytes(separator.getValue()); + byte[] longVal1Arr = Bytes.add(sepByteArr, Bytes.copy(maxLongBytes, + sepByteArr.length, Bytes.SIZEOF_LONG - sepByteArr.length)); + byte[] intVal1Arr = Bytes.add(sepByteArr, Bytes.copy(maxIntBytes, + sepByteArr.length, Bytes.SIZEOF_INT - sepByteArr.length)); + byte[] arr = separator.join( + Bytes.toBytes(separator.encode(str1)), longVal1Arr, + Bytes.toBytes(separator.encode(str2)), intVal1Arr); + int[] sizes = {Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, + Separator.VARIABLE_SIZE, Bytes.SIZEOF_INT}; + byte[][] splits = separator.split(arr, sizes); + assertEquals(4, splits.length); + assertEquals(str1, separator.decode(Bytes.toString(splits[0]))); + assertEquals(Bytes.toLong(longVal1Arr), Bytes.toLong(splits[1])); + assertEquals(str2, separator.decode(Bytes.toString(splits[2]))); + assertEquals(Bytes.toInt(intVal1Arr), Bytes.toInt(splits[3])); + + longVal1Arr = Bytes.add(Bytes.copy(maxLongBytes, 0, Bytes.SIZEOF_LONG - + sepByteArr.length), sepByteArr); + intVal1Arr = Bytes.add(Bytes.copy(maxIntBytes, 0, Bytes.SIZEOF_INT - + sepByteArr.length), sepByteArr); + arr = separator.join(Bytes.toBytes(separator.encode(str1)), longVal1Arr, + Bytes.toBytes(separator.encode(str2)), intVal1Arr); + splits = separator.split(arr, sizes); + assertEquals(4, splits.length); + assertEquals(str1, separator.decode(Bytes.toString(splits[0]))); + assertEquals(Bytes.toLong(longVal1Arr), Bytes.toLong(splits[1])); + assertEquals(str2, separator.decode(Bytes.toString(splits[2]))); + assertEquals(Bytes.toInt(intVal1Arr), Bytes.toInt(splits[3])); + + longVal1Arr = Bytes.add(sepByteArr, Bytes.copy(maxLongBytes, + sepByteArr.length, 4 - sepByteArr.length), sepByteArr); + longVal1Arr = Bytes.add(longVal1Arr, Bytes.copy(maxLongBytes, 4, 3 - + sepByteArr.length), sepByteArr); + arr = separator.join(Bytes.toBytes(separator.encode(str1)), longVal1Arr, + Bytes.toBytes(separator.encode(str2)), intVal1Arr); + splits = separator.split(arr, sizes); + assertEquals(4, splits.length); + assertEquals(str1, separator.decode(Bytes.toString(splits[0]))); + assertEquals(Bytes.toLong(longVal1Arr), Bytes.toLong(splits[1])); + assertEquals(str2, separator.decode(Bytes.toString(splits[2]))); + assertEquals(Bytes.toInt(intVal1Arr), Bytes.toInt(splits[3])); + + arr = separator.join(Bytes.toBytes(separator.encode(str1)), + Bytes.toBytes(separator.encode(str2)), intVal1Arr, longVal1Arr); + int[] sizes1 = {Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, + Bytes.SIZEOF_INT, Bytes.SIZEOF_LONG}; + splits = separator.split(arr, sizes1); + assertEquals(4, splits.length); + assertEquals(str1, separator.decode(Bytes.toString(splits[0]))); + assertEquals(str2, separator.decode(Bytes.toString(splits[1]))); + assertEquals(Bytes.toInt(intVal1Arr), Bytes.toInt(splits[2])); + assertEquals(Bytes.toLong(longVal1Arr), Bytes.toLong(splits[3])); + + try { + int[] sizes2 = {Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, + Bytes.SIZEOF_INT, 7}; + splits = separator.split(arr, sizes2); + fail("Exception should have been thrown."); + } catch (IllegalArgumentException e) {} + + try { + int[] sizes2 = {Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, 2, + Bytes.SIZEOF_LONG}; + splits = separator.split(arr, sizes2); + fail("Exception should have been thrown."); + } catch (IllegalArgumentException e) {} + } + } + + /** + * Simple test to encode and decode using the same separators and confirm that + * we end up with the same as what we started with. + * + * @param token + * @param separators + */ + private static void testEncodeDecode(String token, Separator... separators) { + byte[] encoded = Separator.encode(token, separators); + String decoded = Separator.decode(encoded, separators); + assertEquals(token, decoded); + } + + @Test + public void testJoinStripped() { + List stringList = new ArrayList(0); + stringList.add("nothing"); + + String joined = Separator.VALUES.joinEncoded(stringList); + Iterable split = Separator.VALUES.splitEncoded(joined); + assertTrue(Iterables.elementsEqual(stringList, split)); + + stringList = new ArrayList(3); + stringList.add("a"); + stringList.add("b?"); + stringList.add("c"); + + joined = Separator.VALUES.joinEncoded(stringList); + split = Separator.VALUES.splitEncoded(joined); + assertTrue(Iterables.elementsEqual(stringList, split)); + + String[] stringArray1 = {"else"}; + joined = Separator.VALUES.joinEncoded(stringArray1); + split = Separator.VALUES.splitEncoded(joined); + assertTrue(Iterables.elementsEqual(Arrays.asList(stringArray1), split)); + + String[] stringArray2 = {"d", "e?", "f"}; + joined = Separator.VALUES.joinEncoded(stringArray2); + split = Separator.VALUES.splitEncoded(joined); + assertTrue(Iterables.elementsEqual(Arrays.asList(stringArray2), split)); + + List empty = new ArrayList(0); + split = Separator.VALUES.splitEncoded(null); + assertTrue(Iterables.elementsEqual(empty, split)); + + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml new file mode 100644 index 00000000000..2597c86a59f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml @@ -0,0 +1,180 @@ + + + + + hadoop-yarn-server + org.apache.hadoop + 2.9.0-SNAPSHOT + + 4.0.0 + org.apache.hadoop + hadoop-yarn-server-timelineservice + 2.9.0-SNAPSHOT + Apache Hadoop YARN Timeline Service + + + + ${project.parent.parent.basedir} + + + + + org.apache.hadoop + hadoop-common + provided + + + + org.apache.hadoop + hadoop-annotations + + + + org.apache.hadoop + hadoop-yarn-api + + + + org.apache.hadoop + hadoop-yarn-common + + + + org.apache.hadoop + hadoop-yarn-server-common + + + + org.apache.hadoop + hadoop-yarn-server-applicationhistoryservice + + + + com.google.guava + guava + + + + com.google.inject + guice + + + + commons-io + commons-io + + + + javax.servlet + servlet-api + + + + javax.xml.bind + jaxb-api + + + + com.sun.jersey + jersey-client + + + + commons-lang + commons-lang + + + + commons-logging + commons-logging + + + + org.apache.commons + commons-csv + + + + org.codehaus.jackson + jackson-core-asl + + + + org.codehaus.jackson + jackson-mapper-asl + + + + javax.ws.rs + jsr311-api + 1.1.1 + + + + + org.apache.hadoop + hadoop-common + test-jar + test + + + + junit + junit + test + + + + org.mockito + mockito-all + test + + + + + + + maven-jar-plugin + + + + test-jar + + test-compile + + + + + org.apache.maven.plugins + maven-javadoc-plugin + + + + junit + junit + 4.11 + + + + + + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/TimelineContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/TimelineContext.java new file mode 100644 index 00000000000..694b709bd76 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/TimelineContext.java @@ -0,0 +1,146 @@ +/** + * 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.timelineservice; + +/** + * Encapsulates timeline context information. + */ +public class TimelineContext { + + private String clusterId; + private String userId; + private String flowName; + private Long flowRunId; + private String appId; + + public TimelineContext() { + this(null, null, null, 0L, null); + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((appId == null) ? 0 : appId.hashCode()); + result = prime * result + ((clusterId == null) ? 0 : clusterId.hashCode()); + result = prime * result + ((flowName == null) ? 0 : flowName.hashCode()); + result = prime * result + ((flowRunId == null) ? 0 : flowRunId.hashCode()); + result = prime * result + ((userId == null) ? 0 : userId.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + TimelineContext other = (TimelineContext) obj; + if (appId == null) { + if (other.appId != null) { + return false; + } + } else if (!appId.equals(other.appId)) { + return false; + } + if (clusterId == null) { + if (other.clusterId != null) { + return false; + } + } else if (!clusterId.equals(other.clusterId)) { + return false; + } + if (flowName == null) { + if (other.flowName != null) { + return false; + } + } else if (!flowName.equals(other.flowName)) { + return false; + } + if (flowRunId == null) { + if (other.flowRunId != null) { + return false; + } + } else if (!flowRunId.equals(other.flowRunId)) { + return false; + } + if (userId == null) { + if (other.userId != null) { + return false; + } + } else if (!userId.equals(other.userId)) { + return false; + } + return true; + } + + public TimelineContext(String clusterId, String userId, String flowName, + Long flowRunId, String appId) { + this.clusterId = clusterId; + this.userId = userId; + this.flowName = flowName; + this.flowRunId = flowRunId; + this.appId = appId; + } + + public String getClusterId() { + return clusterId; + } + + public void setClusterId(String cluster) { + this.clusterId = cluster; + } + + public String getUserId() { + return userId; + } + + public void setUserId(String user) { + this.userId = user; + } + + public String getFlowName() { + return flowName; + } + + public void setFlowName(String flow) { + this.flowName = flow; + } + + public Long getFlowRunId() { + return flowRunId; + } + + public void setFlowRunId(long runId) { + this.flowRunId = runId; + } + + public String getAppId() { + return appId; + } + + public void setAppId(String app) { + this.appId = app; + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollector.java new file mode 100644 index 00000000000..38221fe98a8 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollector.java @@ -0,0 +1,141 @@ +/** + * 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.timelineservice.collector; + +import java.util.concurrent.Future; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Service that handles writes to the timeline service and writes them to the + * backing storage for a given YARN application. + * + * App-related lifecycle management is handled by this service. + */ +@Private +@Unstable +public class AppLevelTimelineCollector extends TimelineCollector { + private static final Logger LOG = + LoggerFactory.getLogger(TimelineCollector.class); + + private final ApplicationId appId; + private final String appUser; + private final TimelineCollectorContext context; + private UserGroupInformation currentUser; + private Token delegationTokenForApp; + private long tokenMaxDate = 0; + private String tokenRenewer; + private Future renewalOrRegenerationFuture; + + public AppLevelTimelineCollector(ApplicationId appId) { + this(appId, null); + } + + public AppLevelTimelineCollector(ApplicationId appId, String user) { + super(AppLevelTimelineCollector.class.getName() + " - " + appId.toString()); + Preconditions.checkNotNull(appId, "AppId shouldn't be null"); + this.appId = appId; + this.appUser = user; + context = new TimelineCollectorContext(); + } + + public UserGroupInformation getCurrentUser() { + return currentUser; + } + + public String getAppUser() { + return appUser; + } + + void setDelegationTokenAndFutureForApp( + Token token, + Future appRenewalOrRegenerationFuture, long tknMaxDate, + String renewer) { + this.delegationTokenForApp = token; + this.tokenMaxDate = tknMaxDate; + this.tokenRenewer = renewer; + this.renewalOrRegenerationFuture = appRenewalOrRegenerationFuture; + } + + void setRenewalOrRegenerationFutureForApp( + Future appRenewalOrRegenerationFuture) { + this.renewalOrRegenerationFuture = appRenewalOrRegenerationFuture; + } + + void cancelRenewalOrRegenerationFutureForApp() { + if (renewalOrRegenerationFuture != null && + !renewalOrRegenerationFuture.isDone()) { + renewalOrRegenerationFuture.cancel(true); + } + } + + long getAppDelegationTokenMaxDate() { + return tokenMaxDate; + } + + String getAppDelegationTokenRenewer() { + return tokenRenewer; + } + + @VisibleForTesting + public Token getDelegationTokenForApp() { + return this.delegationTokenForApp; + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + context.setClusterId(conf.get(YarnConfiguration.RM_CLUSTER_ID, + YarnConfiguration.DEFAULT_RM_CLUSTER_ID)); + // Set the default values, which will be updated with an RPC call to get the + // context info from NM. + // Current user usually is not the app user, but keep this field non-null + currentUser = UserGroupInformation.getCurrentUser(); + context.setUserId(currentUser.getShortUserName()); + context.setAppId(appId.toString()); + super.serviceInit(conf); + } + + @Override + protected void serviceStart() throws Exception { + super.serviceStart(); + } + + @Override + protected void serviceStop() throws Exception { + cancelRenewalOrRegenerationFutureForApp(); + super.serviceStop(); + } + + @Override + public TimelineCollectorContext getTimelineEntityContext() { + return context; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollectorWithAgg.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollectorWithAgg.java new file mode 100644 index 00000000000..d7f47c894e3 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollectorWithAgg.java @@ -0,0 +1,147 @@ +/** + * 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.timelineservice.collector; + +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.conf.Configuration; + +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +/** + * Service that handles aggregations for applications + * and makes use of {@link AppLevelTimelineCollector} class for + * writes to Timeline Service. + * + * App-related lifecycle management is handled by this service. + */ +@Private +@Unstable +public class AppLevelTimelineCollectorWithAgg + extends AppLevelTimelineCollector { + private static final Logger LOG = + LoggerFactory.getLogger(TimelineCollector.class); + + private final static int AGGREGATION_EXECUTOR_NUM_THREADS = 1; + private final static int AGGREGATION_EXECUTOR_EXEC_INTERVAL_SECS = 15; + private static Set entityTypesSkipAggregation + = initializeSkipSet(); + + private ScheduledThreadPoolExecutor appAggregationExecutor; + private AppLevelAggregator appAggregator; + + public AppLevelTimelineCollectorWithAgg(ApplicationId appId, String user) { + super(appId, user); + } + + private static Set initializeSkipSet() { + Set result = new HashSet<>(); + result.add(TimelineEntityType.YARN_APPLICATION.toString()); + result.add(TimelineEntityType.YARN_FLOW_RUN.toString()); + result.add(TimelineEntityType.YARN_FLOW_ACTIVITY.toString()); + return result; + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + super.serviceInit(conf); + } + + @Override + protected void serviceStart() throws Exception { + // Launch the aggregation thread + appAggregationExecutor = new ScheduledThreadPoolExecutor( + AppLevelTimelineCollectorWithAgg.AGGREGATION_EXECUTOR_NUM_THREADS, + new ThreadFactoryBuilder() + .setNameFormat("TimelineCollector Aggregation thread #%d") + .build()); + appAggregator = new AppLevelAggregator(); + appAggregationExecutor.scheduleAtFixedRate(appAggregator, + AppLevelTimelineCollectorWithAgg. + AGGREGATION_EXECUTOR_EXEC_INTERVAL_SECS, + AppLevelTimelineCollectorWithAgg. + AGGREGATION_EXECUTOR_EXEC_INTERVAL_SECS, + TimeUnit.SECONDS); + super.serviceStart(); + } + + @Override + protected void serviceStop() throws Exception { + appAggregationExecutor.shutdown(); + if (!appAggregationExecutor.awaitTermination(10, TimeUnit.SECONDS)) { + LOG.info("App-level aggregator shutdown timed out, shutdown now. "); + appAggregationExecutor.shutdownNow(); + } + // Perform one round of aggregation after the aggregation executor is done. + appAggregator.aggregate(); + super.serviceStop(); + } + + @Override + protected Set getEntityTypesSkipAggregation() { + return entityTypesSkipAggregation; + } + + private class AppLevelAggregator implements Runnable { + + private void aggregate() { + LOG.debug("App-level real-time aggregating"); + if (!isReadyToAggregate()) { + LOG.warn("App-level collector is not ready, skip aggregation. "); + return; + } + try { + TimelineCollectorContext currContext = getTimelineEntityContext(); + Map aggregationGroups + = getAggregationGroups(); + if (aggregationGroups == null + || aggregationGroups.isEmpty()) { + LOG.debug("App-level collector is empty, skip aggregation. "); + return; + } + TimelineEntity resultEntity = TimelineCollector.aggregateWithoutGroupId( + aggregationGroups, currContext.getAppId(), + TimelineEntityType.YARN_APPLICATION.toString()); + TimelineEntities entities = new TimelineEntities(); + entities.addEntity(resultEntity); + putEntitiesAsync(entities, getCurrentUser()); + } catch (Exception e) { + LOG.error("Error aggregating timeline metrics", e); + } + LOG.debug("App-level real-time aggregation complete"); + } + + @Override + public void run() { + aggregate(); + } + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/NodeTimelineCollectorManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/NodeTimelineCollectorManager.java new file mode 100644 index 00000000000..e8a1a5415cb --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/NodeTimelineCollectorManager.java @@ -0,0 +1,462 @@ +/** + * 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.timelineservice.collector; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.URI; +import java.util.LinkedHashSet; +import java.util.Set; +import java.util.concurrent.Future; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.http.HttpServer2; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.util.Time; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; +import org.apache.hadoop.yarn.ipc.YarnRPC; +import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier; +import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocol; +import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextResponse; +import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoRequest; +import org.apache.hadoop.yarn.server.timelineservice.security.TimelineV2DelegationTokenSecretManagerService; +import org.apache.hadoop.yarn.server.util.timeline.TimelineServerUtils; +import org.apache.hadoop.yarn.webapp.GenericExceptionHandler; +import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider; +import org.apache.hadoop.yarn.webapp.util.WebAppUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.util.concurrent.ThreadFactoryBuilder; + +/** + * Class on the NodeManager side that manages adding and removing collectors and + * their lifecycle. Also instantiates the per-node collector webapp. + */ +@Private +@Unstable +public class NodeTimelineCollectorManager extends TimelineCollectorManager { + private static final Logger LOG = + LoggerFactory.getLogger(NodeTimelineCollectorManager.class); + + // REST server for this collector manager. + private HttpServer2 timelineRestServer; + + private String timelineRestServerBindAddress; + + private volatile CollectorNodemanagerProtocol nmCollectorService; + + private TimelineV2DelegationTokenSecretManagerService tokenMgrService; + + private final boolean runningAsAuxService; + + private UserGroupInformation loginUGI; + + private ScheduledThreadPoolExecutor tokenRenewalExecutor; + + private long tokenRenewInterval; + + private static final long TIME_BEFORE_RENEW_DATE = 10 * 1000; // 10 seconds. + + private static final long TIME_BEFORE_EXPIRY = 5 * 60 * 1000; // 5 minutes. + + static final String COLLECTOR_MANAGER_ATTR_KEY = "collector.manager"; + + @VisibleForTesting + protected NodeTimelineCollectorManager() { + this(true); + } + + protected NodeTimelineCollectorManager(boolean asAuxService) { + super(NodeTimelineCollectorManager.class.getName()); + this.runningAsAuxService = asAuxService; + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + tokenMgrService = createTokenManagerService(); + addService(tokenMgrService); + this.loginUGI = UserGroupInformation.getCurrentUser(); + tokenRenewInterval = conf.getLong( + YarnConfiguration.TIMELINE_DELEGATION_TOKEN_RENEW_INTERVAL, + YarnConfiguration.DEFAULT_TIMELINE_DELEGATION_TOKEN_RENEW_INTERVAL); + super.serviceInit(conf); + } + + @Override + protected void serviceStart() throws Exception { + if (UserGroupInformation.isSecurityEnabled()) { + // Do security login for cases where collector is running outside NM. + if (!runningAsAuxService) { + try { + doSecureLogin(); + } catch(IOException ie) { + throw new YarnRuntimeException("Failed to login", ie); + } + } + this.loginUGI = UserGroupInformation.getLoginUser(); + } + tokenRenewalExecutor = new ScheduledThreadPoolExecutor( + 1, new ThreadFactoryBuilder().setNameFormat( + "App Collector Token Renewal thread").build()); + super.serviceStart(); + startWebApp(); + } + + protected TimelineV2DelegationTokenSecretManagerService + createTokenManagerService() { + return new TimelineV2DelegationTokenSecretManagerService(); + } + + @VisibleForTesting + public TimelineV2DelegationTokenSecretManagerService + getTokenManagerService() { + return tokenMgrService; + } + + private void doSecureLogin() throws IOException { + Configuration conf = getConfig(); + InetSocketAddress addr = NetUtils.createSocketAddr(conf.getTrimmed( + YarnConfiguration.TIMELINE_SERVICE_BIND_HOST, + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_BIND_HOST), 0, + YarnConfiguration.TIMELINE_SERVICE_BIND_HOST); + SecurityUtil.login(conf, YarnConfiguration.TIMELINE_SERVICE_KEYTAB, + YarnConfiguration.TIMELINE_SERVICE_PRINCIPAL, addr.getHostName()); + } + + @Override + protected void serviceStop() throws Exception { + if (timelineRestServer != null) { + timelineRestServer.stop(); + } + if (tokenRenewalExecutor != null) { + tokenRenewalExecutor.shutdownNow(); + } + super.serviceStop(); + } + + @VisibleForTesting + public Token generateTokenForAppCollector( + String user) { + Token token = tokenMgrService. + generateToken(UserGroupInformation.createRemoteUser(user), + loginUGI.getShortUserName()); + token.setService(new Text(timelineRestServerBindAddress)); + return token; + } + + @VisibleForTesting + public long renewTokenForAppCollector( + AppLevelTimelineCollector appCollector) throws IOException { + if (appCollector.getDelegationTokenForApp() != null) { + return tokenMgrService.renewToken(appCollector.getDelegationTokenForApp(), + appCollector.getAppDelegationTokenRenewer()); + } else { + LOG.info("Delegation token not available for renewal for app " + + appCollector.getTimelineEntityContext().getAppId()); + return -1; + } + } + + @VisibleForTesting + public void cancelTokenForAppCollector( + AppLevelTimelineCollector appCollector) throws IOException { + if (appCollector.getDelegationTokenForApp() != null) { + tokenMgrService.cancelToken(appCollector.getDelegationTokenForApp(), + appCollector.getAppUser()); + } + } + + private long getRenewalDelay(long renewInterval) { + return ((renewInterval > TIME_BEFORE_RENEW_DATE) ? + renewInterval - TIME_BEFORE_RENEW_DATE : renewInterval); + } + + private long getRegenerationDelay(long tokenMaxDate) { + long regenerateTime = tokenMaxDate - Time.now(); + return ((regenerateTime > TIME_BEFORE_EXPIRY) ? + regenerateTime - TIME_BEFORE_EXPIRY : regenerateTime); + } + + private org.apache.hadoop.yarn.api.records.Token generateTokenAndSetTimer( + ApplicationId appId, AppLevelTimelineCollector appCollector) + throws IOException { + Token timelineToken = + generateTokenForAppCollector(appCollector.getAppUser()); + TimelineDelegationTokenIdentifier tokenId = + timelineToken.decodeIdentifier(); + long renewalDelay = getRenewalDelay(tokenRenewInterval); + long regenerationDelay = getRegenerationDelay(tokenId.getMaxDate()); + if (renewalDelay > 0 || regenerationDelay > 0) { + boolean isTimerForRenewal = renewalDelay < regenerationDelay; + Future renewalOrRegenerationFuture = tokenRenewalExecutor.schedule( + new CollectorTokenRenewer(appId, isTimerForRenewal), + isTimerForRenewal? renewalDelay : regenerationDelay, + TimeUnit.MILLISECONDS); + appCollector.setDelegationTokenAndFutureForApp(timelineToken, + renewalOrRegenerationFuture, tokenId.getMaxDate(), + tokenId.getRenewer().toString()); + } + LOG.info("Generated a new token " + timelineToken + " for app " + appId); + return org.apache.hadoop.yarn.api.records.Token.newInstance( + timelineToken.getIdentifier(), timelineToken.getKind().toString(), + timelineToken.getPassword(), timelineToken.getService().toString()); + } + + @Override + protected void doPostPut(ApplicationId appId, TimelineCollector collector) { + try { + // Get context info from NM + updateTimelineCollectorContext(appId, collector); + // Generate token for app collector. + org.apache.hadoop.yarn.api.records.Token token = null; + if (UserGroupInformation.isSecurityEnabled() && + collector instanceof AppLevelTimelineCollector) { + AppLevelTimelineCollector appCollector = + (AppLevelTimelineCollector) collector; + token = generateTokenAndSetTimer(appId, appCollector); + } + // Report to NM if a new collector is added. + reportNewCollectorInfoToNM(appId, token); + } catch (YarnException | IOException e) { + // throw exception here as it cannot be used if failed communicate with NM + LOG.error("Failed to communicate with NM Collector Service for " + appId); + throw new YarnRuntimeException(e); + } + } + + @Override + protected void postRemove(ApplicationId appId, TimelineCollector collector) { + if (collector instanceof AppLevelTimelineCollector) { + try { + cancelTokenForAppCollector((AppLevelTimelineCollector) collector); + } catch (IOException e) { + LOG.warn("Failed to cancel token for app collector with appId " + + appId, e); + } + } + } + + /** + * Launch the REST web server for this collector manager. + */ + private void startWebApp() { + Configuration conf = getConfig(); + String initializers = conf.get("hadoop.http.filter.initializers", ""); + Set defaultInitializers = new LinkedHashSet(); + TimelineServerUtils.addTimelineAuthFilter( + initializers, defaultInitializers, tokenMgrService); + TimelineServerUtils.setTimelineFilters( + conf, initializers, defaultInitializers); + String bindAddress = conf.get(YarnConfiguration.TIMELINE_SERVICE_BIND_HOST, + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_BIND_HOST) + ":0"; + try { + HttpServer2.Builder builder = new HttpServer2.Builder() + .setName("timeline") + .setConf(conf) + .addEndpoint(URI.create( + (YarnConfiguration.useHttps(conf) ? "https://" : "http://") + + bindAddress)); + if (YarnConfiguration.useHttps(conf)) { + builder = WebAppUtils.loadSslConfiguration(builder, conf); + } + timelineRestServer = builder.build(); + + timelineRestServer.addJerseyResourcePackage( + TimelineCollectorWebService.class.getPackage().getName() + ";" + + GenericExceptionHandler.class.getPackage().getName() + ";" + + YarnJacksonJaxbJsonProvider.class.getPackage().getName(), + "/*"); + timelineRestServer.setAttribute(COLLECTOR_MANAGER_ATTR_KEY, this); + timelineRestServer.start(); + } catch (Exception e) { + String msg = "The per-node collector webapp failed to start."; + LOG.error(msg, e); + throw new YarnRuntimeException(msg, e); + } + //TODO: We need to think of the case of multiple interfaces + this.timelineRestServerBindAddress = WebAppUtils.getResolvedAddress( + timelineRestServer.getConnectorAddress(0)); + LOG.info("Instantiated the per-node collector webapp at " + + timelineRestServerBindAddress); + } + + private void reportNewCollectorInfoToNM(ApplicationId appId, + org.apache.hadoop.yarn.api.records.Token token) + throws YarnException, IOException { + ReportNewCollectorInfoRequest request = + ReportNewCollectorInfoRequest.newInstance(appId, + this.timelineRestServerBindAddress, token); + LOG.info("Report a new collector for application: " + appId + + " to the NM Collector Service."); + getNMCollectorService().reportNewCollectorInfo(request); + } + + private void updateTimelineCollectorContext( + ApplicationId appId, TimelineCollector collector) + throws YarnException, IOException { + GetTimelineCollectorContextRequest request = + GetTimelineCollectorContextRequest.newInstance(appId); + LOG.info("Get timeline collector context for " + appId); + GetTimelineCollectorContextResponse response = + getNMCollectorService().getTimelineCollectorContext(request); + String userId = response.getUserId(); + if (userId != null && !userId.isEmpty()) { + if (LOG.isDebugEnabled()) { + LOG.debug("Setting the user in the context: " + userId); + } + collector.getTimelineEntityContext().setUserId(userId); + } + String flowName = response.getFlowName(); + if (flowName != null && !flowName.isEmpty()) { + if (LOG.isDebugEnabled()) { + LOG.debug("Setting the flow name: " + flowName); + } + collector.getTimelineEntityContext().setFlowName(flowName); + } + String flowVersion = response.getFlowVersion(); + if (flowVersion != null && !flowVersion.isEmpty()) { + if (LOG.isDebugEnabled()) { + LOG.debug("Setting the flow version: " + flowVersion); + } + collector.getTimelineEntityContext().setFlowVersion(flowVersion); + } + long flowRunId = response.getFlowRunId(); + if (flowRunId != 0L) { + if (LOG.isDebugEnabled()) { + LOG.debug("Setting the flow run id: " + flowRunId); + } + collector.getTimelineEntityContext().setFlowRunId(flowRunId); + } + } + + @VisibleForTesting + protected CollectorNodemanagerProtocol getNMCollectorService() { + if (nmCollectorService == null) { + synchronized (this) { + if (nmCollectorService == null) { + Configuration conf = getConfig(); + InetSocketAddress nmCollectorServiceAddress = conf.getSocketAddr( + YarnConfiguration.NM_BIND_HOST, + YarnConfiguration.NM_COLLECTOR_SERVICE_ADDRESS, + YarnConfiguration.DEFAULT_NM_COLLECTOR_SERVICE_ADDRESS, + YarnConfiguration.DEFAULT_NM_COLLECTOR_SERVICE_PORT); + LOG.info("nmCollectorServiceAddress: " + nmCollectorServiceAddress); + final YarnRPC rpc = YarnRPC.create(conf); + + // TODO Security settings. + nmCollectorService = (CollectorNodemanagerProtocol) rpc.getProxy( + CollectorNodemanagerProtocol.class, + nmCollectorServiceAddress, conf); + } + } + } + return nmCollectorService; + } + + @VisibleForTesting + public String getRestServerBindAddress() { + return timelineRestServerBindAddress; + } + + private final class CollectorTokenRenewer implements Runnable { + private ApplicationId appId; + // Indicates whether timer is for renewal or regeneration of token. + private boolean timerForRenewal = true; + private CollectorTokenRenewer(ApplicationId applicationId, + boolean forRenewal) { + appId = applicationId; + timerForRenewal = forRenewal; + } + + private void renewToken(AppLevelTimelineCollector appCollector) + throws IOException { + long newExpirationTime = renewTokenForAppCollector(appCollector); + // Set renewal or regeneration timer based on delay. + long renewalDelay = 0; + if (newExpirationTime > 0) { + LOG.info("Renewed token for " + appId + " with new expiration " + + "timestamp = " + newExpirationTime); + renewalDelay = getRenewalDelay(newExpirationTime - Time.now()); + } + long regenerationDelay = + getRegenerationDelay(appCollector.getAppDelegationTokenMaxDate()); + if (renewalDelay > 0 || regenerationDelay > 0) { + this.timerForRenewal = renewalDelay < regenerationDelay; + Future renewalOrRegenerationFuture = tokenRenewalExecutor.schedule( + this, timerForRenewal ? renewalDelay : regenerationDelay, + TimeUnit.MILLISECONDS); + appCollector.setRenewalOrRegenerationFutureForApp( + renewalOrRegenerationFuture); + } + } + + private void regenerateToken(AppLevelTimelineCollector appCollector) + throws IOException { + org.apache.hadoop.yarn.api.records.Token token = + generateTokenAndSetTimer(appId, appCollector); + // Report to NM if a new collector is added. + try { + reportNewCollectorInfoToNM(appId, token); + } catch (YarnException e) { + LOG.warn("Unable to report regenerated token to NM for " + appId); + } + } + + @Override + public void run() { + TimelineCollector collector = get(appId); + if (collector == null) { + LOG.info("Cannot find active collector while " + (timerForRenewal ? + "renewing" : "regenerating") + " token for " + appId); + return; + } + AppLevelTimelineCollector appCollector = + (AppLevelTimelineCollector) collector; + + synchronized (collector) { + if (!collector.isStopped()) { + try { + if (timerForRenewal) { + renewToken(appCollector); + } else { + regenerateToken(appCollector); + } + } catch (Exception e) { + LOG.warn("Unable to " + (timerForRenewal ? "renew" : "regenerate") + + " token for " + appId, e); + } + } + } + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.java new file mode 100644 index 00000000000..66f9aab034b --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.java @@ -0,0 +1,238 @@ +/** + * 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.timelineservice.collector; + +import java.nio.ByteBuffer; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.util.ExitUtil; +import org.apache.hadoop.util.ShutdownHookManager; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext; +import org.apache.hadoop.yarn.server.api.ApplicationTerminationContext; +import org.apache.hadoop.yarn.server.api.AuxiliaryService; +import org.apache.hadoop.yarn.server.api.ContainerInitializationContext; +import org.apache.hadoop.yarn.server.api.ContainerTerminationContext; +import org.apache.hadoop.yarn.server.api.ContainerType; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The top-level server for the per-node timeline collector manager. Currently + * it is defined as an auxiliary service to accommodate running within another + * daemon (e.g. node manager). + */ +@Private +@Unstable +public class PerNodeTimelineCollectorsAuxService extends AuxiliaryService { + private static final Logger LOG = + LoggerFactory.getLogger(PerNodeTimelineCollectorsAuxService.class); + private static final int SHUTDOWN_HOOK_PRIORITY = 30; + + private final NodeTimelineCollectorManager collectorManager; + private long collectorLingerPeriod; + private ScheduledExecutorService scheduler; + + public PerNodeTimelineCollectorsAuxService() { + this(new NodeTimelineCollectorManager(true)); + } + + @VisibleForTesting PerNodeTimelineCollectorsAuxService( + NodeTimelineCollectorManager collectorsManager) { + super("timeline_collector"); + this.collectorManager = collectorsManager; + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + if (!YarnConfiguration.timelineServiceV2Enabled(conf)) { + throw new YarnException( + "Looks like timeline_collector is set as an auxillary service in " + + YarnConfiguration.NM_AUX_SERVICES + + ". But Timeline service v2 is not enabled," + + " so timeline_collector needs to be removed" + + " from that list of auxillary services."); + } + collectorLingerPeriod = + conf.getLong(YarnConfiguration.ATS_APP_COLLECTOR_LINGER_PERIOD_IN_MS, + YarnConfiguration.DEFAULT_ATS_APP_COLLECTOR_LINGER_PERIOD_IN_MS); + scheduler = Executors.newSingleThreadScheduledExecutor(); + collectorManager.init(conf); + super.serviceInit(conf); + } + + @Override + protected void serviceStart() throws Exception { + collectorManager.start(); + super.serviceStart(); + } + + @Override + protected void serviceStop() throws Exception { + scheduler.shutdown(); + if (!scheduler.awaitTermination(collectorLingerPeriod, + TimeUnit.MILLISECONDS)) { + LOG.warn( + "Scheduler terminated before removing the application collectors"); + } + collectorManager.stop(); + super.serviceStop(); + } + + // these methods can be used as the basis for future service methods if the + // per-node collector runs separate from the node manager + /** + * Creates and adds an app level collector for the specified application id. + * The collector is also initialized and started. If the service already + * exists, no new service is created. + * + * @param appId Application Id to be added. + * @param user Application Master container user. + * @return whether it was added successfully + */ + public boolean addApplication(ApplicationId appId, String user) { + AppLevelTimelineCollector collector = + new AppLevelTimelineCollectorWithAgg(appId, user); + return (collectorManager.putIfAbsent(appId, collector) + == collector); + } + + /** + * Removes the app level collector for the specified application id. The + * collector is also stopped as a result. If the collector does not exist, no + * change is made. + * + * @param appId Application Id to be removed. + * @return whether it was removed successfully + */ + public boolean removeApplication(ApplicationId appId) { + return collectorManager.remove(appId); + } + + /** + * Creates and adds an app level collector for the specified application id. + * The collector is also initialized and started. If the collector already + * exists, no new collector is created. + */ + @Override + public void initializeContainer(ContainerInitializationContext context) { + // intercept the event of the AM container being created and initialize the + // app level collector service + if (context.getContainerType() == ContainerType.APPLICATION_MASTER) { + ApplicationId appId = context.getContainerId(). + getApplicationAttemptId().getApplicationId(); + addApplication(appId, context.getUser()); + } + } + + /** + * Removes the app level collector for the specified application id. The + * collector is also stopped as a result. If the collector does not exist, no + * change is made. + */ + @Override + public void stopContainer(ContainerTerminationContext context) { + // intercept the event of the AM container being stopped and remove the app + // level collector service + if (context.getContainerType() == ContainerType.APPLICATION_MASTER) { + final ApplicationId appId = + context.getContainerId().getApplicationAttemptId().getApplicationId(); + scheduler.schedule(new Runnable() { + public void run() { + removeApplication(appId); + } + }, collectorLingerPeriod, TimeUnit.MILLISECONDS); + } + } + + @VisibleForTesting + boolean hasApplication(ApplicationId appId) { + return collectorManager.containsTimelineCollector(appId); + } + + @Override + public void initializeApplication(ApplicationInitializationContext context) { + } + + @Override + public void stopApplication(ApplicationTerminationContext context) { + } + + @Override + public ByteBuffer getMetaData() { + // TODO currently it is not used; we can return a more meaningful data when + // we connect it with an AM + return ByteBuffer.allocate(0); + } + + @VisibleForTesting + public static PerNodeTimelineCollectorsAuxService + launchServer(String[] args, NodeTimelineCollectorManager collectorManager, + Configuration conf) { + Thread + .setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler()); + StringUtils.startupShutdownMessage( + PerNodeTimelineCollectorsAuxService.class, args, LOG); + PerNodeTimelineCollectorsAuxService auxService = null; + try { + auxService = collectorManager == null ? + new PerNodeTimelineCollectorsAuxService( + new NodeTimelineCollectorManager(false)) : + new PerNodeTimelineCollectorsAuxService(collectorManager); + ShutdownHookManager.get().addShutdownHook(new ShutdownHook(auxService), + SHUTDOWN_HOOK_PRIORITY); + auxService.init(conf); + auxService.start(); + } catch (Throwable t) { + LOG.error("Error starting PerNodeTimelineCollectorServer", t); + ExitUtil.terminate(-1, "Error starting PerNodeTimelineCollectorServer"); + } + return auxService; + } + + private static class ShutdownHook implements Runnable { + private final PerNodeTimelineCollectorsAuxService auxService; + + public ShutdownHook(PerNodeTimelineCollectorsAuxService auxService) { + this.auxService = auxService; + } + + public void run() { + auxService.stop(); + } + } + + public static void main(String[] args) { + Configuration conf = new YarnConfiguration(); + conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); + conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f); + launchServer(args, null, conf); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java new file mode 100644 index 00000000000..8202431459d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java @@ -0,0 +1,374 @@ +/** + * 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.timelineservice.collector; + +import java.io.IOException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.service.CompositeService; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetricOperation; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineWriteResponse; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Service that handles writes to the timeline service and writes them to the + * backing storage. + * + * Classes that extend this can add their own lifecycle management or + * customization of request handling. + */ +@Private +@Unstable +public abstract class TimelineCollector extends CompositeService { + + private static final Logger LOG = + LoggerFactory.getLogger(TimelineCollector.class); + public static final String SEPARATOR = "_"; + + private TimelineWriter writer; + private ConcurrentMap aggregationGroups + = new ConcurrentHashMap<>(); + private static Set entityTypesSkipAggregation + = new HashSet<>(); + + private volatile boolean readyToAggregate = false; + + private volatile boolean isStopped = false; + + public TimelineCollector(String name) { + super(name); + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + super.serviceInit(conf); + } + + @Override + protected void serviceStart() throws Exception { + super.serviceStart(); + } + + @Override + protected void serviceStop() throws Exception { + isStopped = true; + super.serviceStop(); + } + + boolean isStopped() { + return isStopped; + } + + protected void setWriter(TimelineWriter w) { + this.writer = w; + } + + protected Map getAggregationGroups() { + return aggregationGroups; + } + + protected void setReadyToAggregate() { + readyToAggregate = true; + } + + protected boolean isReadyToAggregate() { + return readyToAggregate; + } + + /** + * Method to decide the set of timeline entity types the collector should + * skip on aggregations. Subclasses may want to override this method to + * customize their own behaviors. + * + * @return A set of strings consists of all types the collector should skip. + */ + protected Set getEntityTypesSkipAggregation() { + return entityTypesSkipAggregation; + } + + public abstract TimelineCollectorContext getTimelineEntityContext(); + + + /** + * Handles entity writes. These writes are synchronous and are written to the + * backing storage without buffering/batching. If any entity already exists, + * it results in an update of the entity. + * + * This method should be reserved for selected critical entities and events. + * For normal voluminous writes one should use the async method + * {@link #putEntitiesAsync(TimelineEntities, UserGroupInformation)}. + * + * @param entities entities to post + * @param callerUgi the caller UGI + * @return the response that contains the result of the post. + * @throws IOException if there is any exception encountered while putting + * entities. + */ + public TimelineWriteResponse putEntities(TimelineEntities entities, + UserGroupInformation callerUgi) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("putEntities(entities=" + entities + ", callerUgi=" + + callerUgi + ")"); + } + + TimelineWriteResponse response; + // synchronize on the writer object so that no other threads can + // flush the writer buffer concurrently and swallow any exception + // caused by the timeline enitites that are being put here. + synchronized (writer) { + response = writeTimelineEntities(entities, callerUgi); + flushBufferedTimelineEntities(); + } + + return response; + } + + private TimelineWriteResponse writeTimelineEntities( + TimelineEntities entities, UserGroupInformation callerUgi) + throws IOException { + // Update application metrics for aggregation + updateAggregateStatus(entities, aggregationGroups, + getEntityTypesSkipAggregation()); + + final TimelineCollectorContext context = getTimelineEntityContext(); + return writer.write(context, entities, callerUgi); + } + + /** + * Flush buffered timeline entities, if any. + * @throws IOException if there is any exception encountered while + * flushing buffered entities. + */ + private void flushBufferedTimelineEntities() throws IOException { + writer.flush(); + } + + /** + * Handles entity writes in an asynchronous manner. The method returns as soon + * as validation is done. No promises are made on how quickly it will be + * written to the backing storage or if it will always be written to the + * backing storage. Multiple writes to the same entities may be batched and + * appropriate values updated and result in fewer writes to the backing + * storage. + * + * @param entities entities to post + * @param callerUgi the caller UGI + * @throws IOException if there is any exception encounted while putting + * entities. + */ + public void putEntitiesAsync(TimelineEntities entities, + UserGroupInformation callerUgi) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("putEntitiesAsync(entities=" + entities + ", callerUgi=" + + callerUgi + ")"); + } + + writeTimelineEntities(entities, callerUgi); + } + + /** + * Aggregate all metrics in given timeline entities with no predefined states. + * + * @param entities Entities to aggregate + * @param resultEntityId Id of the result entity + * @param resultEntityType Type of the result entity + * @param needsGroupIdInResult Marks if we want the aggregation group id in + * each aggregated metrics. + * @return A timeline entity that contains all aggregated TimelineMetric. + */ + public static TimelineEntity aggregateEntities( + TimelineEntities entities, String resultEntityId, + String resultEntityType, boolean needsGroupIdInResult) { + ConcurrentMap aggregationGroups + = new ConcurrentHashMap<>(); + updateAggregateStatus(entities, aggregationGroups, null); + if (needsGroupIdInResult) { + return aggregate(aggregationGroups, resultEntityId, resultEntityType); + } else { + return aggregateWithoutGroupId( + aggregationGroups, resultEntityId, resultEntityType); + } + } + + /** + * Update the aggregation status table for a timeline collector. + * + * @param entities Entities to update + * @param aggregationGroups Aggregation status table + * @param typesToSkip Entity types that we can safely assume to skip updating + */ + static void updateAggregateStatus( + TimelineEntities entities, + ConcurrentMap aggregationGroups, + Set typesToSkip) { + for (TimelineEntity e : entities.getEntities()) { + if ((typesToSkip != null && typesToSkip.contains(e.getType())) + || e.getMetrics().isEmpty()) { + continue; + } + AggregationStatusTable aggrTable = aggregationGroups.get(e.getType()); + if (aggrTable == null) { + AggregationStatusTable table = new AggregationStatusTable(); + aggrTable = aggregationGroups.putIfAbsent(e.getType(), + table); + if (aggrTable == null) { + aggrTable = table; + } + } + aggrTable.update(e); + } + } + + /** + * Aggregate internal status and generate timeline entities for the + * aggregation results. + * + * @param aggregationGroups Aggregation status table + * @param resultEntityId Id of the result entity + * @param resultEntityType Type of the result entity + * @return A timeline entity that contains all aggregated TimelineMetric. + */ + static TimelineEntity aggregate( + Map aggregationGroups, + String resultEntityId, String resultEntityType) { + TimelineEntity result = new TimelineEntity(); + result.setId(resultEntityId); + result.setType(resultEntityType); + for (Map.Entry entry + : aggregationGroups.entrySet()) { + entry.getValue().aggregateAllTo(result, entry.getKey()); + } + return result; + } + + /** + * Aggregate internal status and generate timeline entities for the + * aggregation results. The result metrics will not have aggregation group + * information. + * + * @param aggregationGroups Aggregation status table + * @param resultEntityId Id of the result entity + * @param resultEntityType Type of the result entity + * @return A timeline entity that contains all aggregated TimelineMetric. + */ + static TimelineEntity aggregateWithoutGroupId( + Map aggregationGroups, + String resultEntityId, String resultEntityType) { + TimelineEntity result = new TimelineEntity(); + result.setId(resultEntityId); + result.setType(resultEntityType); + for (Map.Entry entry + : aggregationGroups.entrySet()) { + entry.getValue().aggregateAllTo(result, ""); + } + return result; + } + + // Note: In memory aggregation is performed in an eventually consistent + // fashion. + protected static class AggregationStatusTable { + // On aggregation, for each metric, aggregate all per-entity accumulated + // metrics. We only use the id and type for TimelineMetrics in the key set + // of this table. + private ConcurrentMap> + aggregateTable; + + public AggregationStatusTable() { + aggregateTable = new ConcurrentHashMap<>(); + } + + public void update(TimelineEntity incoming) { + String entityId = incoming.getId(); + for (TimelineMetric m : incoming.getMetrics()) { + // Skip if the metric does not need aggregation + if (m.getRealtimeAggregationOp() == TimelineMetricOperation.NOP) { + continue; + } + // Update aggregateTable + Map aggrRow = aggregateTable.get(m); + if (aggrRow == null) { + Map tempRow = new HashMap<>(); + aggrRow = aggregateTable.putIfAbsent(m, tempRow); + if (aggrRow == null) { + aggrRow = tempRow; + } + } + synchronized (aggrRow) { + aggrRow.put(entityId, m); + } + } + } + + public TimelineEntity aggregateTo(TimelineMetric metric, TimelineEntity e, + String aggregationGroupId) { + if (metric.getRealtimeAggregationOp() == TimelineMetricOperation.NOP) { + return e; + } + Map aggrRow = aggregateTable.get(metric); + if (aggrRow != null) { + TimelineMetric aggrMetric = new TimelineMetric(); + if (aggregationGroupId.length() > 0) { + aggrMetric.setId(metric.getId() + SEPARATOR + aggregationGroupId); + } else { + aggrMetric.setId(metric.getId()); + } + aggrMetric.setRealtimeAggregationOp(TimelineMetricOperation.NOP); + Map status = new HashMap<>(); + synchronized (aggrRow) { + for (TimelineMetric m : aggrRow.values()) { + TimelineMetric.aggregateTo(m, aggrMetric, status); + // getRealtimeAggregationOp returns an enum so we can directly + // compare with "!=". + if (m.getRealtimeAggregationOp() + != aggrMetric.getRealtimeAggregationOp()) { + aggrMetric.setRealtimeAggregationOp(m.getRealtimeAggregationOp()); + } + } + aggrRow.clear(); + } + Set metrics = e.getMetrics(); + metrics.remove(aggrMetric); + metrics.add(aggrMetric); + } + return e; + } + + public TimelineEntity aggregateAllTo(TimelineEntity e, + String aggregationGroupId) { + for (TimelineMetric m : aggregateTable.keySet()) { + aggregateTo(m, e, aggregationGroupId); + } + return e; + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorContext.java new file mode 100644 index 00000000000..981ee2af5e0 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorContext.java @@ -0,0 +1,76 @@ +/** + * 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.timelineservice.collector; + +import org.apache.hadoop.yarn.server.timelineservice.TimelineContext; +import org.apache.hadoop.yarn.util.timeline.TimelineUtils; + +/** + * Encapsulates context information required by collector during a put. + */ +public class TimelineCollectorContext extends TimelineContext { + private String flowVersion; + + public TimelineCollectorContext() { + this(null, null, null, null, 0L, null); + } + + public TimelineCollectorContext(String clusterId, String userId, + String flowName, String flowVersion, Long flowRunId, String appId) { + super(clusterId, userId, flowName, flowRunId, appId); + this.flowVersion = flowVersion == null ? + TimelineUtils.DEFAULT_FLOW_VERSION : flowVersion; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = super.hashCode(); + result = + prime * result + ((flowVersion == null) ? 0 : flowVersion.hashCode()); + return result + super.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (!super.equals(obj)) { + return false; + } + TimelineCollectorContext other = (TimelineCollectorContext) obj; + if (flowVersion == null) { + if (other.flowVersion != null) { + return false; + } + } else if (!flowVersion.equals(other.flowVersion)) { + return false; + } + return true; + } + + public String getFlowVersion() { + return flowVersion; + } + + public void setFlowVersion(String version) { + this.flowVersion = version; + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.java new file mode 100644 index 00000000000..c64a3adbd59 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.java @@ -0,0 +1,280 @@ +/** + * 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.timelineservice.collector; + + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.CompositeService; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Class that manages adding and removing collectors and their lifecycle. It + * provides thread safety access to the collectors inside. + * + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public class TimelineCollectorManager extends CompositeService { + private static final Logger LOG = + LoggerFactory.getLogger(TimelineCollectorManager.class); + + + private TimelineWriter writer; + private ScheduledExecutorService writerFlusher; + private int flushInterval; + private boolean writerFlusherRunning; + + @Override + protected void serviceInit(Configuration conf) throws Exception { + writer = createTimelineWriter(conf); + writer.init(conf); + // create a single dedicated thread for flushing the writer on a periodic + // basis + writerFlusher = Executors.newSingleThreadScheduledExecutor(); + flushInterval = conf.getInt( + YarnConfiguration. + TIMELINE_SERVICE_WRITER_FLUSH_INTERVAL_SECONDS, + YarnConfiguration. + DEFAULT_TIMELINE_SERVICE_WRITER_FLUSH_INTERVAL_SECONDS); + super.serviceInit(conf); + } + + private TimelineWriter createTimelineWriter(final Configuration conf) { + String timelineWriterClassName = conf.get( + YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS, + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WRITER_CLASS); + LOG.info("Using TimelineWriter: " + timelineWriterClassName); + try { + Class timelineWriterClazz = Class.forName(timelineWriterClassName); + if (TimelineWriter.class.isAssignableFrom(timelineWriterClazz)) { + return (TimelineWriter) ReflectionUtils.newInstance( + timelineWriterClazz, conf); + } else { + throw new YarnRuntimeException("Class: " + timelineWriterClassName + + " not instance of " + TimelineWriter.class.getCanonicalName()); + } + } catch (ClassNotFoundException e) { + throw new YarnRuntimeException("Could not instantiate TimelineWriter: " + + timelineWriterClassName, e); + } + } + + @Override + protected void serviceStart() throws Exception { + super.serviceStart(); + if (writer != null) { + writer.start(); + } + // schedule the flush task + writerFlusher.scheduleAtFixedRate(new WriterFlushTask(writer), + flushInterval, flushInterval, TimeUnit.SECONDS); + writerFlusherRunning = true; + } + + // access to this map is synchronized with the map itself + private final Map collectors = + Collections.synchronizedMap( + new HashMap()); + + public TimelineCollectorManager(String name) { + super(name); + } + + protected TimelineWriter getWriter() { + return writer; + } + + /** + * Put the collector into the collection if an collector mapped by id does + * not exist. + * + * @param appId Application Id for which collector needs to be put. + * @param collector timeline collector to be put. + * @throws YarnRuntimeException if there was any exception in initializing + * and starting the app level service + * @return the collector associated with id after the potential put. + */ + public TimelineCollector putIfAbsent(ApplicationId appId, + TimelineCollector collector) { + TimelineCollector collectorInTable = null; + synchronized (collectors) { + collectorInTable = collectors.get(appId); + if (collectorInTable == null) { + try { + // initialize, start, and add it to the collection so it can be + // cleaned up when the parent shuts down + collector.init(getConfig()); + collector.setWriter(writer); + collector.start(); + collectors.put(appId, collector); + LOG.info("the collector for " + appId + " was added"); + collectorInTable = collector; + postPut(appId, collectorInTable); + } catch (Exception e) { + throw new YarnRuntimeException(e); + } + } else { + LOG.info("the collector for " + appId + " already exists!"); + } + } + return collectorInTable; + } + + /** + * Callback handler for the timeline collector manager when a collector has + * been added into the collector map. + * @param appId Application id of the collector. + * @param collector The actual timeline collector that has been added. + */ + public void postPut(ApplicationId appId, TimelineCollector collector) { + doPostPut(appId, collector); + collector.setReadyToAggregate(); + } + + /** + * A template method that will be called by + * {@link #postPut(ApplicationId, TimelineCollector)}. + * @param appId Application id of the collector. + * @param collector The actual timeline collector that has been added. + */ + protected void doPostPut(ApplicationId appId, TimelineCollector collector) { + } + + /** + * Removes the collector for the specified id. The collector is also stopped + * as a result. If the collector does not exist, no change is made. + * + * @param appId Application Id to remove. + * @return whether it was removed successfully + */ + public boolean remove(ApplicationId appId) { + TimelineCollector collector = collectors.remove(appId); + if (collector == null) { + LOG.error("the collector for " + appId + " does not exist!"); + } else { + synchronized (collector) { + postRemove(appId, collector); + // stop the service to do clean up + collector.stop(); + } + LOG.info("The collector service for " + appId + " was removed"); + } + return collector != null; + } + + protected void postRemove(ApplicationId appId, TimelineCollector collector) { + + } + + /** + * Returns the collector for the specified id. + * + * @param appId Application Id for which we need to get the collector. + * @return the collector or null if it does not exist + */ + public TimelineCollector get(ApplicationId appId) { + return collectors.get(appId); + } + + /** + * Returns whether the collector for the specified id exists in this + * collection. + * @param appId Application Id. + * @return true if collector for the app id is found, false otherwise. + */ + public boolean containsTimelineCollector(ApplicationId appId) { + return collectors.containsKey(appId); + } + + @Override + protected void serviceStop() throws Exception { + if (collectors != null && collectors.size() > 1) { + synchronized (collectors) { + for (TimelineCollector c : collectors.values()) { + c.serviceStop(); + } + } + } + // stop the flusher first + if (writerFlusher != null) { + writerFlusher.shutdown(); + writerFlusherRunning = false; + if (!writerFlusher.awaitTermination(30, TimeUnit.SECONDS)) { + // in reality it should be ample time for the flusher task to finish + // even if it times out, writers may be able to handle closing in this + // situation fine + // proceed to close the writer + LOG.warn("failed to stop the flusher task in time. " + + "will still proceed to close the writer."); + } + } + if (writer != null) { + writer.close(); + } + super.serviceStop(); + } + + @VisibleForTesting + boolean writerFlusherRunning() { + return writerFlusherRunning; + } + + /** + * Task that invokes the flush operation on the timeline writer. + */ + private static class WriterFlushTask implements Runnable { + private final TimelineWriter writer; + + public WriterFlushTask(TimelineWriter writer) { + this.writer = writer; + } + + public void run() { + try { + // synchronize on the writer object to avoid flushing timeline + // entities placed on the buffer by synchronous putEntities + // requests. + synchronized (writer) { + writer.flush(); + } + } catch (Throwable th) { + // we need to handle all exceptions or subsequent execution may be + // suppressed + LOG.error("exception during timeline writer flush!", th); + } + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorWebService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorWebService.java new file mode 100644 index 00000000000..369dc0802e4 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorWebService.java @@ -0,0 +1,254 @@ +/** + * 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.timelineservice.collector; + +import javax.servlet.ServletContext; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import javax.ws.rs.Consumes; +import javax.ws.rs.GET; +import javax.ws.rs.PUT; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.WebApplicationException; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import javax.xml.bind.annotation.XmlAccessType; +import javax.xml.bind.annotation.XmlAccessorType; +import javax.xml.bind.annotation.XmlElement; +import javax.xml.bind.annotation.XmlRootElement; + +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.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationAttemptEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.ClusterEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.ContainerEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.QueueEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; +import org.apache.hadoop.yarn.api.records.timelineservice.UserEntity; +import org.apache.hadoop.yarn.webapp.ForbiddenException; +import org.apache.hadoop.yarn.webapp.NotFoundException; + +import com.google.inject.Singleton; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The main per-node REST end point for timeline service writes. It is + * essentially a container service that routes requests to the appropriate + * per-app services. + */ +@Private +@Unstable +@Singleton +@Path("/ws/v2/timeline") +public class TimelineCollectorWebService { + private static final Logger LOG = + LoggerFactory.getLogger(TimelineCollectorWebService.class); + + private @Context ServletContext context; + + /** + * Gives information about timeline collector. + */ + @XmlRootElement(name = "about") + @XmlAccessorType(XmlAccessType.NONE) + @Public + @Unstable + public static class AboutInfo { + + private String about; + + public AboutInfo() { + + } + + public AboutInfo(String abt) { + this.about = abt; + } + + @XmlElement(name = "About") + public String getAbout() { + return about; + } + + public void setAbout(String abt) { + this.about = abt; + } + + } + + /** + * Return the description of the timeline web services. + * + * @param req Servlet request. + * @param res Servlet response. + * @return description of timeline web service. + */ + @GET + @Produces({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */}) + public AboutInfo about( + @Context HttpServletRequest req, + @Context HttpServletResponse res) { + init(res); + return new AboutInfo("Timeline Collector API"); + } + + /** + * Accepts writes to the collector, and returns a response. It simply routes + * the request to the app level collector. It expects an application as a + * context. + * + * @param req Servlet request. + * @param res Servlet response. + * @param async flag indicating whether its an async put or not. "true" + * indicates, its an async call. If null, its considered false. + * @param appId Application Id to which the entities to be put belong to. If + * appId is not there or it cannot be parsed, HTTP 400 will be sent back. + * @param entities timeline entities to be put. + * @return a Response with appropriate HTTP status. + */ + @PUT + @Path("/entities") + @Consumes({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */}) + public Response putEntities( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @QueryParam("async") String async, + @QueryParam("appid") String appId, + TimelineEntities entities) { + init(res); + UserGroupInformation callerUgi = getUser(req); + if (callerUgi == null) { + String msg = "The owner of the posted timeline entities is not set"; + LOG.error(msg); + throw new ForbiddenException(msg); + } + + try { + ApplicationId appID = parseApplicationId(appId); + if (appID == null) { + return Response.status(Response.Status.BAD_REQUEST).build(); + } + NodeTimelineCollectorManager collectorManager = + (NodeTimelineCollectorManager) context.getAttribute( + NodeTimelineCollectorManager.COLLECTOR_MANAGER_ATTR_KEY); + TimelineCollector collector = collectorManager.get(appID); + if (collector == null) { + LOG.error("Application: "+ appId + " is not found"); + throw new NotFoundException(); // different exception? + } + + boolean isAsync = async != null && async.trim().equalsIgnoreCase("true"); + if (isAsync) { + collector.putEntitiesAsync( + processTimelineEntities(entities), callerUgi); + } else { + collector.putEntities(processTimelineEntities(entities), callerUgi); + } + + return Response.ok().build(); + } catch (Exception e) { + LOG.error("Error putting entities", e); + throw new WebApplicationException(e, + Response.Status.INTERNAL_SERVER_ERROR); + } + } + + private static ApplicationId parseApplicationId(String appId) { + try { + if (appId != null) { + return ApplicationId.fromString(appId.trim()); + } else { + return null; + } + } catch (Exception e) { + LOG.error("Invalid application ID: " + appId); + return null; + } + } + + private static void init(HttpServletResponse response) { + response.setContentType(null); + } + + private static UserGroupInformation getUser(HttpServletRequest req) { + String remoteUser = req.getRemoteUser(); + UserGroupInformation callerUgi = null; + if (remoteUser != null) { + callerUgi = UserGroupInformation.createRemoteUser(remoteUser); + } + return callerUgi; + } + + // The process may not be necessary according to the way we write the backend, + // but let's keep it for now in case we need to use sub-classes APIs in the + // future (e.g., aggregation). + private static TimelineEntities processTimelineEntities( + TimelineEntities entities) { + TimelineEntities entitiesToReturn = new TimelineEntities(); + for (TimelineEntity entity : entities.getEntities()) { + TimelineEntityType type = null; + try { + type = TimelineEntityType.valueOf(entity.getType()); + } catch (IllegalArgumentException e) { + type = null; + } + if (type != null) { + switch (type) { + case YARN_CLUSTER: + entitiesToReturn.addEntity(new ClusterEntity(entity)); + break; + case YARN_FLOW_RUN: + entitiesToReturn.addEntity(new FlowRunEntity(entity)); + break; + case YARN_APPLICATION: + entitiesToReturn.addEntity(new ApplicationEntity(entity)); + break; + case YARN_APPLICATION_ATTEMPT: + entitiesToReturn.addEntity(new ApplicationAttemptEntity(entity)); + break; + case YARN_CONTAINER: + entitiesToReturn.addEntity(new ContainerEntity(entity)); + break; + case YARN_QUEUE: + entitiesToReturn.addEntity(new QueueEntity(entity)); + break; + case YARN_USER: + entitiesToReturn.addEntity(new UserEntity(entity)); + break; + default: + break; + } + } else { + entitiesToReturn.addEntity(entity); + } + } + return entitiesToReturn; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/package-info.java new file mode 100644 index 00000000000..1f7dd23776b --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/package-info.java @@ -0,0 +1,29 @@ +/* + * 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.timelineservice.collector contains + * classes which can be used across collector. This package contains classes + * which are not related to storage implementations though. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.timelineservice.collector; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/package-info.java new file mode 100644 index 00000000000..58e23f02873 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/package-info.java @@ -0,0 +1,28 @@ +/* + * 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.server.timelineservice contains classes to be used + * across timeline reader and collector. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.timelineservice; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineDataToRetrieve.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineDataToRetrieve.java new file mode 100644 index 00000000000..8d09c007eba --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineDataToRetrieve.java @@ -0,0 +1,178 @@ +/** + * 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.timelineservice.reader; + +import java.util.EnumSet; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; + +/** + * Encapsulates information regarding which data to retrieve for each entity + * while querying.
+ * Data to retrieve contains the following :
+ *
    + *
  • confsToRetrieve - Used for deciding which configs to return + * in response. This is represented as a {@link TimelineFilterList} object + * containing {@link TimelinePrefixFilter} objects. These can either be + * exact config keys' or prefixes which are then compared against config + * keys' to decide configs(inside entities) to return in response. If null + * or empty, all configurations will be fetched if fieldsToRetrieve + * contains {@link Field#CONFIGS} or {@link Field#ALL}. This should not be + * confused with configFilters which is used to decide which entities to + * return instead.
  • + *
  • metricsToRetrieve - Used for deciding which metrics to return + * in response. This is represented as a {@link TimelineFilterList} object + * containing {@link TimelinePrefixFilter} objects. These can either be + * exact metric ids' or prefixes which are then compared against metric + * ids' to decide metrics(inside entities) to return in response. If null + * or empty, all metrics will be fetched if fieldsToRetrieve contains + * {@link Field#METRICS} or {@link Field#ALL}. This should not be confused + * with metricFilters which is used to decide which entities to return + * instead.
  • + *
  • fieldsToRetrieve - Specifies which fields of the entity + * object to retrieve, see {@link Field}. If null, retrieves 3 fields, + * namely entity id, entity type and entity created time. All fields will + * be returned if {@link Field#ALL} is specified.
  • + *
  • metricsLimit - If fieldsToRetrieve contains METRICS/ALL or + * metricsToRetrieve is specified, this limit defines an upper limit to the + * number of metrics to return. This parameter is ignored if METRICS are not to + * be fetched.
  • + *
  • metricsTimeStart - Metric values before this timestamp would not + * be retrieved. If null or {@literal <0}, defaults to 0.
  • + *
  • metricsTimeEnd - Metric values after this timestamp would not + * be retrieved. If null or {@literal <0}, defaults to {@link Long#MAX_VALUE}. + *
  • + *
+ */ +@Private +@Unstable +public class TimelineDataToRetrieve { + private TimelineFilterList confsToRetrieve; + private TimelineFilterList metricsToRetrieve; + private EnumSet fieldsToRetrieve; + private Integer metricsLimit; + private Long metricsTimeBegin; + private Long metricsTimeEnd; + private static final long DEFAULT_METRICS_BEGIN_TIME = 0L; + private static final long DEFAULT_METRICS_END_TIME = Long.MAX_VALUE; + + /** + * Default limit of number of metrics to return. + */ + public static final Integer DEFAULT_METRICS_LIMIT = 1; + + public TimelineDataToRetrieve() { + this(null, null, null, null, null, null); + } + + public TimelineDataToRetrieve(TimelineFilterList confs, + TimelineFilterList metrics, EnumSet fields, + Integer limitForMetrics, Long metricTimeBegin, Long metricTimeEnd) { + this.confsToRetrieve = confs; + this.metricsToRetrieve = metrics; + this.fieldsToRetrieve = fields; + if (limitForMetrics == null || limitForMetrics < 1) { + this.metricsLimit = DEFAULT_METRICS_LIMIT; + } else { + this.metricsLimit = limitForMetrics; + } + + if (this.fieldsToRetrieve == null) { + this.fieldsToRetrieve = EnumSet.noneOf(Field.class); + } + if (metricTimeBegin == null || metricTimeBegin < 0) { + this.metricsTimeBegin = DEFAULT_METRICS_BEGIN_TIME; + } else { + this.metricsTimeBegin = metricTimeBegin; + } + if (metricTimeEnd == null || metricTimeEnd < 0) { + this.metricsTimeEnd = DEFAULT_METRICS_END_TIME; + } else { + this.metricsTimeEnd = metricTimeEnd; + } + if (this.metricsTimeBegin > this.metricsTimeEnd) { + throw new IllegalArgumentException("metricstimebegin should not be " + + "greater than metricstimeend"); + } + } + + public TimelineFilterList getConfsToRetrieve() { + return confsToRetrieve; + } + + public void setConfsToRetrieve(TimelineFilterList confs) { + this.confsToRetrieve = confs; + } + + public TimelineFilterList getMetricsToRetrieve() { + return metricsToRetrieve; + } + + public void setMetricsToRetrieve(TimelineFilterList metrics) { + this.metricsToRetrieve = metrics; + } + + public EnumSet getFieldsToRetrieve() { + return fieldsToRetrieve; + } + + public void setFieldsToRetrieve(EnumSet fields) { + this.fieldsToRetrieve = fields; + } + + /** + * Adds configs and metrics fields to fieldsToRetrieve(if they are not + * present) if confsToRetrieve and metricsToRetrieve are specified. + */ + public void addFieldsBasedOnConfsAndMetricsToRetrieve() { + if (!fieldsToRetrieve.contains(Field.CONFIGS) && confsToRetrieve != null && + !confsToRetrieve.getFilterList().isEmpty()) { + fieldsToRetrieve.add(Field.CONFIGS); + } + if (!fieldsToRetrieve.contains(Field.METRICS) && + metricsToRetrieve != null && + !metricsToRetrieve.getFilterList().isEmpty()) { + fieldsToRetrieve.add(Field.METRICS); + } + } + + public Integer getMetricsLimit() { + return metricsLimit; + } + + public Long getMetricsTimeBegin() { + return this.metricsTimeBegin; + } + + public Long getMetricsTimeEnd() { + return metricsTimeEnd; + } + + public void setMetricsLimit(Integer limit) { + if (limit == null || limit < 1) { + this.metricsLimit = DEFAULT_METRICS_LIMIT; + } else { + this.metricsLimit = limit; + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityFilters.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityFilters.java new file mode 100644 index 00000000000..a415d3467cb --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityFilters.java @@ -0,0 +1,274 @@ +/** + * 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.timelineservice.reader; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValuesFilter; + +/** + * Encapsulates information regarding the filters to apply while querying. These + * filters restrict the number of entities to return.
+ * Filters contain the following :
+ *
    + *
  • limit - A limit on the number of entities to return. If null or + * {@literal < 0}, defaults to {@link #DEFAULT_LIMIT}. The maximum possible + * value for limit can be {@link Long#MAX_VALUE}.
  • + *
  • createdTimeBegin - Matched entities should not be created before + * this timestamp. If null or {@literal <=0}, defaults to 0.
  • + *
  • createdTimeEnd - Matched entities should not be created after this + * timestamp. If null or {@literal <=0}, defaults to + * {@link Long#MAX_VALUE}.
  • + *
  • relatesTo - Matched entities should or should not relate to given + * entities depending on what's specified in the filter. The entities in + * relatesTo are identified by entity type and id. This is represented as + * a {@link TimelineFilterList} object containing + * {@link TimelineKeyValuesFilter} objects, each of which contains a + * set of values for a key and the comparison operator (equals/not equals). The + * key which represents the entity type is a string and values are a set of + * entity identifiers (also string). As it is a filter list, relatesTo can be + * evaluated with logical AND/OR and we can create a hierarchy of these + * {@link TimelineKeyValuesFilter} objects. If null or empty, the relations are + * not matched.
  • + *
  • isRelatedTo - Matched entities should or should not be related + * to given entities depending on what's specified in the filter. The entities + * in isRelatedTo are identified by entity type and id. This is represented as + * a {@link TimelineFilterList} object containing + * {@link TimelineKeyValuesFilter} objects, each of which contains a + * set of values for a key and the comparison operator (equals/not equals). The + * key which represents the entity type is a string and values are a set of + * entity identifiers (also string). As it is a filter list, relatesTo can be + * evaluated with logical AND/OR and we can create a hierarchy of these + * {@link TimelineKeyValuesFilter} objects. If null or empty, the relations are + * not matched.
  • + *
  • infoFilters - Matched entities should have exact matches to + * the given info and should be either equal or not equal to given value + * depending on what's specified in the filter. This is represented as a + * {@link TimelineFilterList} object containing {@link TimelineKeyValueFilter} + * objects, each of which contains key-value pairs with a comparison operator + * (equals/not equals). The key which represents the info key is a string but + * value can be any object. As it is a filter list, info filters can be + * evaluated with logical AND/OR and we can create a hierarchy of these + * key-value pairs. If null or empty, the filter is not applied.
  • + *
  • configFilters - Matched entities should have exact matches to + * the given configurations and should be either equal or not equal to given + * value depending on what's specified in the filter. This is represented as a + * {@link TimelineFilterList} object containing {@link TimelineKeyValueFilter} + * objects, each of which contains key-value pairs with a comparison operator + * (equals/not equals). Both key (which represents config name) and value (which + * is config value) are strings. As it is a filter list, config filters can be + * evaluated with logical AND/OR and we can create a hierarchy of these + * {@link TimelineKeyValueFilter} objects. If null or empty, the filter is not + * applied.
  • + *
  • metricFilters - Matched entities should contain the given + * metrics and satisfy the specified relation with the value. This is + * represented as a {@link TimelineFilterList} object containing + * {@link TimelineCompareFilter} objects, each of which contains key-value pairs + * along with the specified relational/comparison operator represented by + * {@link TimelineCompareOp}. The key is a string and value is integer + * (Short/Integer/Long). As it is a filter list, metric filters can be evaluated + * with logical AND/OR and we can create a hierarchy of these + * {@link TimelineCompareFilter} objects. If null or empty, the filter is not + * applied.
  • + *
  • eventFilters - Matched entities should contain or not contain the + * given events. This is represented as a {@link TimelineFilterList} object + * containing {@link TimelineExistsFilter} objects, each of which contains a + * value which must or must not exist depending on comparison operator specified + * in the filter. For event filters, the value represents a event id. As it is a + * filter list, event filters can be evaluated with logical AND/OR and we can + * create a hierarchy of these {@link TimelineExistsFilter} objects. If null or + * empty, the filter is not applied.
  • + *
  • fromId - If specified, retrieve the next set of entities from the + * given fromId. The set of entities retrieved is inclusive of specified fromId. + * fromId should be taken from the value associated with FROM_ID info key in + * entity response which was sent earlier.
  • + *
+ */ +@Private +@Unstable +public final class TimelineEntityFilters { + private final long limit; + private long createdTimeBegin; + private long createdTimeEnd; + private final TimelineFilterList relatesTo; + private final TimelineFilterList isRelatedTo; + private final TimelineFilterList infoFilters; + private final TimelineFilterList configFilters; + private final TimelineFilterList metricFilters; + private final TimelineFilterList eventFilters; + private final String fromId; + private static final long DEFAULT_BEGIN_TIME = 0L; + private static final long DEFAULT_END_TIME = Long.MAX_VALUE; + + + /** + * Default limit of number of entities to return for getEntities API. + */ + public static final long DEFAULT_LIMIT = 100; + + private TimelineEntityFilters( + Long entityLimit, Long timeBegin, Long timeEnd, + TimelineFilterList entityRelatesTo, + TimelineFilterList entityIsRelatedTo, + TimelineFilterList entityInfoFilters, + TimelineFilterList entityConfigFilters, + TimelineFilterList entityMetricFilters, + TimelineFilterList entityEventFilters, String fromId) { + if (entityLimit == null || entityLimit < 0) { + this.limit = DEFAULT_LIMIT; + } else { + this.limit = entityLimit; + } + if (timeBegin == null || timeBegin < 0) { + this.createdTimeBegin = DEFAULT_BEGIN_TIME; + } else { + this.createdTimeBegin = timeBegin; + } + if (timeEnd == null || timeEnd < 0) { + this.createdTimeEnd = DEFAULT_END_TIME; + } else { + this.createdTimeEnd = timeEnd; + } + this.relatesTo = entityRelatesTo; + this.isRelatedTo = entityIsRelatedTo; + this.infoFilters = entityInfoFilters; + this.configFilters = entityConfigFilters; + this.metricFilters = entityMetricFilters; + this.eventFilters = entityEventFilters; + this.fromId = fromId; + } + + public long getLimit() { + return limit; + } + + public long getCreatedTimeBegin() { + return createdTimeBegin; + } + + public long getCreatedTimeEnd() { + return createdTimeEnd; + } + + public TimelineFilterList getRelatesTo() { + return relatesTo; + } + + public TimelineFilterList getIsRelatedTo() { + return isRelatedTo; + } + + public TimelineFilterList getInfoFilters() { + return infoFilters; + } + + public TimelineFilterList getConfigFilters() { + return configFilters; + } + + public TimelineFilterList getMetricFilters() { + return metricFilters; + } + + public TimelineFilterList getEventFilters() { + return eventFilters; + } + + public String getFromId() { + return fromId; + } + + /** + * A builder class to build an instance of TimelineEntityFilters. + */ + public static class Builder { + private Long entityLimit; + private Long createdTimeBegin; + private Long createdTimeEnd; + private TimelineFilterList relatesToFilters; + private TimelineFilterList isRelatedToFilters; + private TimelineFilterList entityInfoFilters; + private TimelineFilterList entityConfigFilters; + private TimelineFilterList entityMetricFilters; + private TimelineFilterList entityEventFilters; + private String entityFromId; + + public Builder entityLimit(Long limit) { + this.entityLimit = limit; + return this; + } + + public Builder createdTimeBegin(Long timeBegin) { + this.createdTimeBegin = timeBegin; + return this; + } + + public Builder createTimeEnd(Long timeEnd) { + this.createdTimeEnd = timeEnd; + return this; + } + + public Builder relatesTo(TimelineFilterList relatesTo) { + this.relatesToFilters = relatesTo; + return this; + } + + public Builder isRelatedTo(TimelineFilterList isRelatedTo) { + this.isRelatedToFilters = isRelatedTo; + return this; + } + + public Builder infoFilters(TimelineFilterList infoFilters) { + this.entityInfoFilters = infoFilters; + return this; + } + + public Builder configFilters(TimelineFilterList configFilters) { + this.entityConfigFilters = configFilters; + return this; + } + + public Builder metricFilters(TimelineFilterList metricFilters) { + this.entityMetricFilters = metricFilters; + return this; + } + + public Builder eventFilters(TimelineFilterList eventFilters) { + this.entityEventFilters = eventFilters; + return this; + } + + public Builder fromId(String fromId) { + this.entityFromId = fromId; + return this; + } + + public TimelineEntityFilters build() { + return new TimelineEntityFilters(entityLimit, createdTimeBegin, + createdTimeEnd, relatesToFilters, isRelatedToFilters, + entityInfoFilters, entityConfigFilters, entityMetricFilters, + entityEventFilters, entityFromId); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParseConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParseConstants.java new file mode 100644 index 00000000000..662a1029429 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParseConstants.java @@ -0,0 +1,34 @@ +/** + * 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.timelineservice.reader; + +/** + * Set of constants used while parsing filter expressions. + */ +final class TimelineParseConstants { + private TimelineParseConstants() { + } + static final String COMMA_DELIMITER = ","; + static final String COLON_DELIMITER = ":"; + static final char NOT_CHAR = '!'; + static final char SPACE_CHAR = ' '; + static final char OPENING_BRACKET_CHAR = '('; + static final char CLOSING_BRACKET_CHAR = ')'; + static final char COMMA_CHAR = ','; +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParseException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParseException.java new file mode 100644 index 00000000000..8d4a5dc8d91 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParseException.java @@ -0,0 +1,36 @@ +/** + * 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.timelineservice.reader; + +/** + * Exception thrown to indicate that a timeline filter expression cannot be + * parsed. + */ +class TimelineParseException extends Exception { + + private static final long serialVersionUID = 1L; + + public TimelineParseException() { + super(); + } + + public TimelineParseException(String message) { + super(message); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParser.java new file mode 100644 index 00000000000..6b461a01a5c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParser.java @@ -0,0 +1,37 @@ +/** + * 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.timelineservice.reader; + +import java.io.Closeable; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; + +@Private +@Unstable +interface TimelineParser extends Closeable { + /** + * Method used for parsing. + * + * @return a {@link TimelineFilterList} object. + * @throws TimelineParseException if any problem occurs while parsing. + */ + TimelineFilterList parse() throws TimelineParseException; +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForCompareExpr.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForCompareExpr.java new file mode 100644 index 00000000000..1b020d93f0c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForCompareExpr.java @@ -0,0 +1,300 @@ +/** + * 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.timelineservice.reader; + +import java.util.Deque; +import java.util.LinkedList; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList.Operator; + +/** + * Abstract class for parsing compare expressions. + * Compare expressions are of the form : + * (<key> <compareop> <value>) <op> (<key + * > <compareop> <value>) + * compareop is used to compare value of a the specified key in the backend + * storage. compareop can be : + * 1. eq - Equals + * 2. ne - Not equals (matches if key does not exist) + * 3. ene - Exists and not equals (key must exist for match to occur) + * 4. lt - Less than + * 5. gt - Greater than + * 6. le - Less than or equals + * 7. ge - Greater than or equals + * compareop's supported would depend on implementation. For instance, all + * the above compareops' will be supported for metric filters but only eq,ne and + * ene would be supported for KV filters like config/info filters. + * + * op is a logical operator and can be either AND or OR. + * + * The way values will be interpreted would also depend on implementation + * + * A typical compare expression would look as under: + * ((key1 eq val1 OR key2 ne val2) AND (key5 gt val45)) + */ +@Private +@Unstable +abstract class TimelineParserForCompareExpr implements TimelineParser { + private enum ParseState { + PARSING_KEY, + PARSING_VALUE, + PARSING_OP, + PARSING_COMPAREOP + } + // Main expression. + private final String expr; + // Expression in lower case. + private final String exprInLowerCase; + private final String exprName; + private int offset = 0; + private int kvStartOffset = 0; + private final int exprLength; + private ParseState currentParseState = ParseState.PARSING_KEY; + // Linked list implemented as a stack. + private Deque filterListStack = new LinkedList<>(); + private TimelineFilter currentFilter = null; + private TimelineFilterList filterList = null; + public TimelineParserForCompareExpr(String expression, String name) { + if (expression != null) { + expr = expression.trim(); + exprLength = expr.length(); + exprInLowerCase = expr.toLowerCase(); + } else { + expr = null; + exprInLowerCase = null; + exprLength = 0; + } + this.exprName = name; + } + + protected TimelineFilter getCurrentFilter() { + return currentFilter; + } + + protected TimelineFilter getFilterList() { + return filterList; + } + + protected abstract TimelineFilter createFilter(); + + protected abstract Object parseValue(String strValue) + throws TimelineParseException; + + protected abstract void setCompareOpToCurrentFilter( + TimelineCompareOp compareOp, boolean keyMustExistFlag) + throws TimelineParseException; + + protected abstract void setValueToCurrentFilter(Object value); + + private void handleSpaceChar() throws TimelineParseException { + if (currentParseState == ParseState.PARSING_KEY || + currentParseState == ParseState.PARSING_VALUE) { + if (kvStartOffset == offset) { + kvStartOffset++; + offset++; + return; + } + String str = expr.substring(kvStartOffset, offset); + if (currentParseState == ParseState.PARSING_KEY) { + if (currentFilter == null) { + currentFilter = createFilter(); + } + ((TimelineCompareFilter)currentFilter).setKey(str); + currentParseState = ParseState.PARSING_COMPAREOP; + } else if (currentParseState == ParseState.PARSING_VALUE) { + if (currentFilter != null) { + setValueToCurrentFilter(parseValue(str)); + } + currentParseState = ParseState.PARSING_OP; + } + } + offset++; + } + + private void handleOpeningBracketChar() throws TimelineParseException { + if (currentParseState != ParseState.PARSING_KEY) { + throw new TimelineParseException("Encountered unexpected opening " + + "bracket while parsing " + exprName + "."); + } + offset++; + kvStartOffset = offset; + filterListStack.push(filterList); + filterList = null; + } + + private void handleClosingBracketChar() throws TimelineParseException { + if (currentParseState != ParseState.PARSING_VALUE && + currentParseState != ParseState.PARSING_OP) { + throw new TimelineParseException("Encountered unexpected closing " + + "bracket while parsing " + exprName + "."); + } + if (!filterListStack.isEmpty()) { + if (currentParseState == ParseState.PARSING_VALUE) { + setValueToCurrentFilter( + parseValue(expr.substring(kvStartOffset, offset))); + currentParseState = ParseState.PARSING_OP; + } + if (currentFilter != null) { + filterList.addFilter(currentFilter); + } + // As bracket is closing, pop the filter list from top of the stack and + // combine it with current filter list. + TimelineFilterList fList = filterListStack.pop(); + if (fList != null) { + fList.addFilter(filterList); + filterList = fList; + } + currentFilter = null; + offset++; + kvStartOffset = offset; + } else { + throw new TimelineParseException("Encountered unexpected closing " + + "bracket while parsing " + exprName + "."); + } + } + + private void parseCompareOp() throws TimelineParseException { + if (offset + 2 >= exprLength) { + throw new TimelineParseException("Compare op cannot be parsed for " + + exprName + "."); + } + TimelineCompareOp compareOp = null; + boolean keyExistFlag = true; + if (expr.charAt(offset + 2) == TimelineParseConstants.SPACE_CHAR) { + if (exprInLowerCase.startsWith("eq", offset)) { + compareOp = TimelineCompareOp.EQUAL; + } else if (exprInLowerCase.startsWith("ne", offset)) { + compareOp = TimelineCompareOp.NOT_EQUAL; + keyExistFlag = false; + } else if (exprInLowerCase.startsWith("lt", offset)) { + compareOp = TimelineCompareOp.LESS_THAN; + } else if (exprInLowerCase.startsWith("le", offset)) { + compareOp = TimelineCompareOp.LESS_OR_EQUAL; + } else if (exprInLowerCase.startsWith("gt", offset)) { + compareOp = TimelineCompareOp.GREATER_THAN; + } else if (exprInLowerCase.startsWith("ge", offset)) { + compareOp = TimelineCompareOp.GREATER_OR_EQUAL; + } + offset = offset + 3; + } else if (exprInLowerCase.startsWith("ene ", offset)) { + // Not equal but key should be present. + compareOp = TimelineCompareOp.NOT_EQUAL; + offset = offset + 4; + } + if (compareOp == null) { + throw new TimelineParseException("Compare op cannot be parsed for " + + exprName + "."); + } + setCompareOpToCurrentFilter(compareOp, keyExistFlag); + kvStartOffset = offset; + currentParseState = ParseState.PARSING_VALUE; + } + + private void parseOp(boolean closingBracket) throws TimelineParseException { + Operator operator = null; + if (exprInLowerCase.startsWith("or ", offset)) { + operator = Operator.OR; + offset = offset + 3; + } else if (exprInLowerCase.startsWith("and ", offset)) { + operator = Operator.AND; + offset = offset + 4; + } + if (operator == null) { + throw new TimelineParseException("Operator cannot be parsed for " + + exprName + "."); + } + if (filterList == null) { + filterList = new TimelineFilterList(operator); + } + if (currentFilter != null) { + filterList.addFilter(currentFilter); + } + if (closingBracket || filterList.getOperator() != operator) { + filterList = new TimelineFilterList(operator, filterList); + } + currentFilter = null; + kvStartOffset = offset; + currentParseState = ParseState.PARSING_KEY; + } + + @Override + public TimelineFilterList parse() throws TimelineParseException { + if (expr == null || exprLength == 0) { + return null; + } + boolean closingBracket = false; + while (offset < exprLength) { + char offsetChar = expr.charAt(offset); + switch(offsetChar) { + case TimelineParseConstants.SPACE_CHAR: + handleSpaceChar(); + break; + case TimelineParseConstants.OPENING_BRACKET_CHAR: + handleOpeningBracketChar(); + break; + case TimelineParseConstants.CLOSING_BRACKET_CHAR: + handleClosingBracketChar(); + closingBracket = true; + break; + default: // other characters. + // Parse based on state. + if (currentParseState == ParseState.PARSING_COMPAREOP) { + parseCompareOp(); + } else if (currentParseState == ParseState.PARSING_OP) { + parseOp(closingBracket); + closingBracket = false; + } else { + // Might be a key or value. Move ahead. + offset++; + } + break; + } + } + if (!filterListStack.isEmpty()) { + filterListStack.clear(); + throw new TimelineParseException("Encountered improper brackets while " + + "parsing " + exprName + "."); + } + if (currentParseState == ParseState.PARSING_VALUE) { + setValueToCurrentFilter( + parseValue(expr.substring(kvStartOffset, offset))); + } + if (filterList == null || filterList.getFilterList().isEmpty()) { + filterList = new TimelineFilterList(currentFilter); + } else if (currentFilter != null) { + filterList.addFilter(currentFilter); + } + return filterList; + } + + @Override + public void close() { + if (filterListStack != null) { + filterListStack.clear(); + } + filterList = null; + currentFilter = null; + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForDataToRetrieve.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForDataToRetrieve.java new file mode 100644 index 00000000000..1e6039dbfd0 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForDataToRetrieve.java @@ -0,0 +1,95 @@ +/** + * 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.timelineservice.reader; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList.Operator; + +/** + * Used for parsing metrics or configs to retrieve. + */ +@Private +@Unstable +public class TimelineParserForDataToRetrieve implements TimelineParser { + private String expr; + private final int exprLength; + public TimelineParserForDataToRetrieve(String expression) { + this.expr = expression; + if (expression != null) { + this.expr = expr.trim(); + exprLength = expr.length(); + } else { + exprLength = 0; + } + } + + @Override + public TimelineFilterList parse() throws TimelineParseException { + if (expr == null || exprLength == 0) { + return null; + } + TimelineCompareOp compareOp = null; + int openingBracketIndex = + expr.indexOf(TimelineParseConstants.OPENING_BRACKET_CHAR); + if (expr.charAt(0) == TimelineParseConstants.NOT_CHAR) { + if (openingBracketIndex == -1) { + throw new TimelineParseException("Invalid config/metric to retrieve " + + "expression"); + } + if (openingBracketIndex != 1 && + expr.substring(1, openingBracketIndex + 1).trim().length() != 1) { + throw new TimelineParseException("Invalid config/metric to retrieve " + + "expression"); + } + compareOp = TimelineCompareOp.NOT_EQUAL; + } else if (openingBracketIndex <= 0) { + compareOp = TimelineCompareOp.EQUAL; + } + char lastChar = expr.charAt(exprLength - 1); + if (compareOp == TimelineCompareOp.NOT_EQUAL && + lastChar != TimelineParseConstants.CLOSING_BRACKET_CHAR) { + throw new TimelineParseException("Invalid config/metric to retrieve " + + "expression"); + } + if (openingBracketIndex != -1 && + expr.charAt(exprLength - 1) == + TimelineParseConstants.CLOSING_BRACKET_CHAR) { + expr = expr.substring(openingBracketIndex + 1, exprLength - 1).trim(); + } + if (expr.isEmpty()) { + return null; + } + Operator op = + (compareOp == TimelineCompareOp.NOT_EQUAL) ? Operator.AND : Operator.OR; + TimelineFilterList list = new TimelineFilterList(op); + String[] splits = expr.split(TimelineParseConstants.COMMA_DELIMITER); + for (String split : splits) { + list.addFilter(new TimelinePrefixFilter(compareOp, split.trim())); + } + return list; + } + + @Override + public void close() { + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForEqualityExpr.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForEqualityExpr.java new file mode 100644 index 00000000000..74517133c6d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForEqualityExpr.java @@ -0,0 +1,343 @@ +/** + * 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.timelineservice.reader; + +import java.util.Deque; +import java.util.LinkedList; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList.Operator; + +/** + * Abstract class for parsing equality expressions. This means the values in + * expression would either be equal or not equal. + * Equality expressions are of the form : + * (<value>,<value>,<value>) <op> !(<value>, + * <value>) + * + * Here, "!" means all the values should not exist/should not be equal. + * If not specified, they should exist/be equal. + * + * op is a logical operator and can be either AND or OR. + * + * The way values will be interpreted would also depend on implementation. + * + * For instance for event filters this expression may look like, + * (event1,event2) AND !(event3,event4) + * This means for an entity to match, event1 and event2 should exist. But event3 + * and event4 should not exist. + */ +@Private +@Unstable +abstract class TimelineParserForEqualityExpr implements TimelineParser { + private enum ParseState { + PARSING_VALUE, + PARSING_OP, + PARSING_COMPAREOP + } + private final String expr; + // Expression in lower case. + private final String exprInLowerCase; + // Expression name. + private final String exprName; + // Expression offset. + private int offset = 0; + // Offset used to parse values in the expression. + private int startOffset = 0; + private final int exprLength; + private ParseState currentParseState = ParseState.PARSING_COMPAREOP; + private TimelineCompareOp currentCompareOp = null; + // Used to store filter lists which can then be combined as brackets are + // closed. + private Deque filterListStack = new LinkedList<>(); + private TimelineFilter currentFilter = null; + private TimelineFilterList filterList = null; + // Delimiter used to separate values. + private final char delimiter; + public TimelineParserForEqualityExpr(String expression, String name, + char delim) { + if (expression != null) { + expr = expression.trim(); + exprLength = expr.length(); + exprInLowerCase = expr.toLowerCase(); + } else { + exprLength = 0; + expr = null; + exprInLowerCase = null; + } + exprName = name; + delimiter = delim; + } + + protected TimelineFilter getCurrentFilter() { + return currentFilter; + } + + protected TimelineFilter getFilterList() { + return filterList; + } + + /** + * Creates filter as per implementation. + * + * @return a {@link TimelineFilter} implementation. + */ + protected abstract TimelineFilter createFilter(); + + /** + * Sets compare op to the current filter as per filter implementation. + * + * @param compareOp compare op to be set. + * @throws Exception if any problem occurs. + */ + protected abstract void setCompareOpToCurrentFilter( + TimelineCompareOp compareOp) throws TimelineParseException; + + /** + * Sets value to the current filter as per filter implementation. + * + * @param value value to be set. + * @throws Exception if any problem occurs. + */ + protected abstract void setValueToCurrentFilter(String value) + throws TimelineParseException; + + private void createAndSetFilter(boolean checkIfNull) + throws TimelineParseException { + if (!checkIfNull || currentFilter == null) { + currentFilter = createFilter(); + setCompareOpToCurrentFilter(currentCompareOp); + } + setValueToCurrentFilter(expr.substring(startOffset, offset).trim()); + } + + private void handleSpaceChar() throws TimelineParseException { + if (currentParseState == ParseState.PARSING_VALUE) { + if (startOffset == offset) { + startOffset++; + } else { + createAndSetFilter(true); + currentParseState = ParseState.PARSING_OP; + } + } + offset++; + } + + private void handleDelimiter() throws TimelineParseException { + if (currentParseState == ParseState.PARSING_OP || + currentParseState == ParseState.PARSING_VALUE) { + if (currentParseState == ParseState.PARSING_VALUE) { + createAndSetFilter(false); + } + if (filterList == null) { + filterList = new TimelineFilterList(); + } + // Add parsed filter into filterlist and make it null to move on to next + // filter. + filterList.addFilter(currentFilter); + currentFilter = null; + offset++; + startOffset = offset; + currentParseState = ParseState.PARSING_VALUE; + } else { + throw new TimelineParseException("Invalid " + exprName + "expression."); + } + } + + private void handleOpeningBracketChar(boolean encounteredNot) + throws TimelineParseException { + if (currentParseState == ParseState.PARSING_COMPAREOP || + currentParseState == ParseState.PARSING_VALUE) { + offset++; + startOffset = offset; + filterListStack.push(filterList); + filterList = null; + if (currentFilter == null) { + currentFilter = createFilter(); + } + currentCompareOp = encounteredNot ? + TimelineCompareOp.NOT_EQUAL : TimelineCompareOp.EQUAL; + setCompareOpToCurrentFilter(currentCompareOp); + currentParseState = ParseState.PARSING_VALUE; + } else { + throw new TimelineParseException("Encountered unexpected opening " + + "bracket while parsing " + exprName + "."); + } + } + + private void handleNotChar() throws TimelineParseException { + if (currentParseState == ParseState.PARSING_COMPAREOP || + currentParseState == ParseState.PARSING_VALUE) { + offset++; + while (offset < exprLength && + expr.charAt(offset) == TimelineParseConstants.SPACE_CHAR) { + offset++; + } + if (offset == exprLength) { + throw new TimelineParseException("Invalid " + exprName + "expression"); + } + if (expr.charAt(offset) == TimelineParseConstants.OPENING_BRACKET_CHAR) { + handleOpeningBracketChar(true); + } else { + throw new TimelineParseException("Invalid " + exprName + "expression"); + } + } else { + throw new TimelineParseException("Encountered unexpected not(!) char " + + "while parsing " + exprName + "."); + } + } + + private void handleClosingBracketChar() throws TimelineParseException { + if (currentParseState != ParseState.PARSING_VALUE && + currentParseState != ParseState.PARSING_OP) { + throw new TimelineParseException("Encountered unexpected closing " + + "bracket while parsing " + exprName + "."); + } + if (!filterListStack.isEmpty()) { + if (currentParseState == ParseState.PARSING_VALUE) { + if (startOffset != offset) { + createAndSetFilter(true); + currentParseState = ParseState.PARSING_OP; + } + } + if (filterList == null) { + filterList = new TimelineFilterList(); + } + if (currentFilter != null) { + filterList.addFilter(currentFilter); + } + // As bracket is closing, pop the filter list from top of the stack and + // combine it with current filter list. + TimelineFilterList fList = filterListStack.pop(); + if (fList != null) { + fList.addFilter(filterList); + filterList = fList; + } + currentFilter = null; + offset++; + startOffset = offset; + } else { + throw new TimelineParseException("Encountered unexpected closing " + + "bracket while parsing " + exprName + "."); + } + } + + private void parseOp(boolean closingBracket) throws TimelineParseException { + Operator operator = null; + if (exprInLowerCase.startsWith("or ", offset)) { + operator = Operator.OR; + offset = offset + 3; + } else if (exprInLowerCase.startsWith("and ", offset)) { + operator = Operator.AND; + offset = offset + 4; + } + if (operator == null) { + throw new TimelineParseException("Operator cannot be parsed for " + + exprName + "."); + } + if (filterList == null) { + filterList = new TimelineFilterList(operator); + } + if (currentFilter != null) { + filterList.addFilter(currentFilter); + } + if (closingBracket || filterList.getOperator() != operator) { + filterList = new TimelineFilterList(operator, filterList); + } + currentFilter = null; + startOffset = offset; + currentParseState = ParseState.PARSING_COMPAREOP; + } + + private void parseCompareOp() throws TimelineParseException { + if (currentFilter == null) { + currentFilter = createFilter(); + } + currentCompareOp = TimelineCompareOp.EQUAL; + setCompareOpToCurrentFilter(currentCompareOp); + currentParseState = ParseState.PARSING_VALUE; + } + + @Override + public TimelineFilterList parse() throws TimelineParseException { + if (expr == null || exprLength == 0) { + return null; + } + boolean closingBracket = false; + while (offset < exprLength) { + char offsetChar = expr.charAt(offset); + switch(offsetChar) { + case TimelineParseConstants.NOT_CHAR: + handleNotChar(); + break; + case TimelineParseConstants.SPACE_CHAR: + handleSpaceChar(); + break; + case TimelineParseConstants.OPENING_BRACKET_CHAR: + handleOpeningBracketChar(false); + break; + case TimelineParseConstants.CLOSING_BRACKET_CHAR: + handleClosingBracketChar(); + closingBracket = true; + break; + default: // other characters. + if (offsetChar == delimiter) { + handleDelimiter(); + } else if (currentParseState == ParseState.PARSING_COMPAREOP) { + parseCompareOp(); + } else if (currentParseState == ParseState.PARSING_OP) { + parseOp(closingBracket); + closingBracket = false; + } else { + offset++; + } + break; + } + } + if (!filterListStack.isEmpty()) { + filterListStack.clear(); + throw new TimelineParseException("Encountered improper brackets while " + + "parsing " + exprName + "."); + } + if (currentParseState == ParseState.PARSING_VALUE) { + if (startOffset != offset) { + createAndSetFilter(true); + } + } + if (filterList == null || filterList.getFilterList().isEmpty()) { + filterList = new TimelineFilterList(currentFilter); + } else if (currentFilter != null) { + filterList.addFilter(currentFilter); + } + return filterList; + } + + @Override + public void close() { + if (filterListStack != null) { + filterListStack.clear(); + } + currentFilter = null; + filterList = null; + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForExistFilters.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForExistFilters.java new file mode 100644 index 00000000000..8048c6eb855 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForExistFilters.java @@ -0,0 +1,51 @@ +/** + * 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.timelineservice.reader; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilter; + +/** + * Used for parsing existence filters such as event filters. These filters + * check for existence of a value. For example, in case of event filters, they + * check if an event exists or not and accordingly return an entity. + */ +@Private +@Unstable +class TimelineParserForExistFilters extends TimelineParserForEqualityExpr { + + public TimelineParserForExistFilters(String expression, char delimiter) { + super(expression, "Event Filter", delimiter); + } + + protected TimelineFilter createFilter() { + return new TimelineExistsFilter(); + } + + protected void setValueToCurrentFilter(String value) { + ((TimelineExistsFilter)getCurrentFilter()).setValue(value); + } + + protected void setCompareOpToCurrentFilter(TimelineCompareOp compareOp) { + ((TimelineExistsFilter)getCurrentFilter()).setCompareOp(compareOp); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForKVFilters.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForKVFilters.java new file mode 100644 index 00000000000..ec68bec1b34 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForKVFilters.java @@ -0,0 +1,78 @@ +/** + * 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.timelineservice.reader; + +import java.io.IOException; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter; + +/** + * Used for parsing key-value filters such as config and info filters. + */ +@Private +@Unstable +class TimelineParserForKVFilters extends TimelineParserForCompareExpr { + // Indicates if value has to be interpreted as a string. + private final boolean valueAsString; + public TimelineParserForKVFilters(String expression, boolean valAsStr) { + super(expression, "Config/Info Filter"); + this.valueAsString = valAsStr; + } + + protected TimelineFilter createFilter() { + return new TimelineKeyValueFilter(); + } + + protected Object parseValue(String strValue) { + if (!valueAsString) { + try { + return GenericObjectMapper.OBJECT_READER.readValue(strValue); + } catch (IOException e) { + return strValue; + } + } else { + return strValue; + } + } + + @Override + protected void setCompareOpToCurrentFilter(TimelineCompareOp compareOp, + boolean keyMustExistFlag) throws TimelineParseException { + if (compareOp != TimelineCompareOp.EQUAL && + compareOp != TimelineCompareOp.NOT_EQUAL) { + throw new TimelineParseException("TimelineCompareOp for kv-filter " + + "should be EQUAL or NOT_EQUAL"); + } + ((TimelineKeyValueFilter)getCurrentFilter()).setCompareOp( + compareOp, keyMustExistFlag); + } + + @Override + protected void setValueToCurrentFilter(Object value) { + TimelineFilter currentFilter = getCurrentFilter(); + if (currentFilter != null) { + ((TimelineKeyValueFilter)currentFilter).setValue(value); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForNumericFilters.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForNumericFilters.java new file mode 100644 index 00000000000..7c14a9f8df0 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForNumericFilters.java @@ -0,0 +1,72 @@ +/** + * 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.timelineservice.reader; + +import java.io.IOException; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils; + +/** + * Used for parsing numerical filters such as metric filters. + */ +@Private +@Unstable +class TimelineParserForNumericFilters extends TimelineParserForCompareExpr { + + public TimelineParserForNumericFilters(String expression) { + super(expression, "Metric Filter"); + } + + protected TimelineFilter createFilter() { + return new TimelineCompareFilter(); + } + + @Override + protected void setCompareOpToCurrentFilter(TimelineCompareOp compareOp, + boolean keyMustExistFlag) { + ((TimelineCompareFilter)getCurrentFilter()).setCompareOp( + compareOp, keyMustExistFlag); + } + + protected Object parseValue(String strValue) throws TimelineParseException { + Object value = null; + try { + value = GenericObjectMapper.OBJECT_READER.readValue(strValue); + } catch (IOException e) { + throw new TimelineParseException("Value cannot be parsed."); + } + if (value == null || !(TimelineStorageUtils.isIntegralValue(value))) { + throw new TimelineParseException("Value is not a number."); + } + return value; + } + + protected void setValueToCurrentFilter(Object value) { + TimelineFilter currentFilter = getCurrentFilter(); + if (currentFilter != null) { + ((TimelineCompareFilter)currentFilter).setValue(value); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForRelationFilters.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForRelationFilters.java new file mode 100644 index 00000000000..cde11e468a9 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForRelationFilters.java @@ -0,0 +1,71 @@ +/** + * 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.timelineservice.reader; + +import java.util.HashSet; +import java.util.Set; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValuesFilter; + +/** + * Used for parsing relation filters. + */ +@Private +@Unstable +class TimelineParserForRelationFilters extends + TimelineParserForEqualityExpr { + private final String valueDelimiter; + public TimelineParserForRelationFilters(String expression, char valuesDelim, + String valueDelim) { + super(expression, "Relation Filter", valuesDelim); + valueDelimiter = valueDelim; + } + + @Override + protected TimelineFilter createFilter() { + return new TimelineKeyValuesFilter(); + } + + @Override + protected void setCompareOpToCurrentFilter(TimelineCompareOp compareOp) { + ((TimelineKeyValuesFilter)getCurrentFilter()).setCompareOp(compareOp); + } + + @Override + protected void setValueToCurrentFilter(String value) + throws TimelineParseException { + if (value != null) { + String[] pairStrs = value.split(valueDelimiter); + if (pairStrs.length < 2) { + throw new TimelineParseException("Invalid relation filter expression"); + } + String key = pairStrs[0].trim(); + Set values = new HashSet(); + for (int i = 1; i < pairStrs.length; i++) { + values.add(pairStrs[i].trim()); + } + ((TimelineKeyValuesFilter)getCurrentFilter()). + setKeyAndValues(key, values); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderContext.java new file mode 100644 index 00000000000..67c3d297e11 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderContext.java @@ -0,0 +1,133 @@ +/** + * 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.timelineservice.reader; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.server.timelineservice.TimelineContext; + +/** + * Encapsulates fields necessary to make a query in timeline reader. + */ +@Private +@Unstable +public class TimelineReaderContext extends TimelineContext { + + private String entityType; + private String entityId; + private Long entityIdPrefix; + private String doAsUser; + public TimelineReaderContext(String clusterId, String userId, String flowName, + Long flowRunId, String appId, String entityType, String entityId) { + super(clusterId, userId, flowName, flowRunId, appId); + this.entityType = entityType; + this.entityId = entityId; + } + + public TimelineReaderContext(String clusterId, String userId, String flowName, + Long flowRunId, String appId, String entityType, Long entityIdPrefix, + String entityId) { + this(clusterId, userId, flowName, flowRunId, appId, entityType, entityId); + this.entityIdPrefix = entityIdPrefix; + } + + public TimelineReaderContext(String clusterId, String userId, String flowName, + Long flowRunId, String appId, String entityType, Long entityIdPrefix, + String entityId, String doasUser) { + this(clusterId, userId, flowName, flowRunId, appId, entityType, entityId); + this.entityIdPrefix = entityIdPrefix; + this.doAsUser = doasUser; + } + + public TimelineReaderContext(TimelineReaderContext other) { + this(other.getClusterId(), other.getUserId(), other.getFlowName(), + other.getFlowRunId(), other.getAppId(), other.getEntityType(), + other.getEntityIdPrefix(), other.getEntityId(), other.getDoAsUser()); + } + + @Override + public int hashCode() { + final int prime = 31; + int result = super.hashCode(); + result = prime * result + + ((entityIdPrefix == null) ? 0 : entityIdPrefix.hashCode()); + result = prime * result + ((entityId == null) ? 0 : entityId.hashCode()); + result = + prime * result + ((entityType == null) ? 0 : entityType.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (!super.equals(obj)) { + return false; + } + TimelineReaderContext other = (TimelineReaderContext) obj; + if (entityId == null) { + if (other.entityId != null) { + return false; + } + } else if (!entityId.equals(other.entityId)) { + return false; + } + if (entityType == null) { + if (other.entityType != null) { + return false; + } + } else if (!entityType.equals(other.entityType)) { + return false; + } + return true; + } + + public String getEntityType() { + return entityType; + } + + public void setEntityType(String type) { + this.entityType = type; + } + + public String getEntityId() { + return entityId; + } + + public void setEntityId(String id) { + this.entityId = id; + } + + public Long getEntityIdPrefix() { + return entityIdPrefix; + } + + public void setEntityIdPrefix(Long entityIdPrefix) { + this.entityIdPrefix = entityIdPrefix; + } + + public String getDoAsUser() { + return doAsUser; + } + + public void setDoAsUser(String doAsUser) { + this.doAsUser = doAsUser; + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderManager.java new file mode 100644 index 00000000000..67e5849ed6a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderManager.java @@ -0,0 +1,201 @@ +/** + * 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.timelineservice.reader; + +import java.io.IOException; +import java.util.Set; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader; + +/** + * This class wraps over the timeline reader store implementation. It does some + * non trivial manipulation of the timeline data before or after getting + * it from the backend store. + */ +@Private +@Unstable +public class TimelineReaderManager extends AbstractService { + + private TimelineReader reader; + + public TimelineReaderManager(TimelineReader timelineReader) { + super(TimelineReaderManager.class.getName()); + this.reader = timelineReader; + } + + /** + * Gets cluster ID from config yarn.resourcemanager.cluster-id + * if not supplied by client. + * @param clusterId + * @param conf + * @return clusterId + */ + private static String getClusterID(String clusterId, Configuration conf) { + if (clusterId == null || clusterId.isEmpty()) { + return conf.get( + YarnConfiguration.RM_CLUSTER_ID, + YarnConfiguration.DEFAULT_RM_CLUSTER_ID); + } + return clusterId; + } + + private static TimelineEntityType getTimelineEntityType(String entityType) { + if (entityType == null) { + return null; + } + try { + return TimelineEntityType.valueOf(entityType); + } catch (IllegalArgumentException e) { + return null; + } + } + + /** + * Fill UID in the info field of entity based on the query(identified by + * entity type). + * @param entityType Entity type of query. + * @param entity Timeline Entity. + * @param context Context defining the query. + */ + private static void fillUID(TimelineEntityType entityType, + TimelineEntity entity, TimelineReaderContext context) { + if (entityType != null) { + switch(entityType) { + case YARN_FLOW_ACTIVITY: + FlowActivityEntity activityEntity = (FlowActivityEntity)entity; + context.setUserId(activityEntity.getUser()); + context.setFlowName(activityEntity.getFlowName()); + entity.setUID(TimelineReaderUtils.UID_KEY, + TimelineUIDConverter.FLOW_UID.encodeUID(context)); + return; + case YARN_FLOW_RUN: + FlowRunEntity runEntity = (FlowRunEntity)entity; + context.setFlowRunId(runEntity.getRunId()); + entity.setUID(TimelineReaderUtils.UID_KEY, + TimelineUIDConverter.FLOWRUN_UID.encodeUID(context)); + return; + case YARN_APPLICATION: + context.setAppId(entity.getId()); + entity.setUID(TimelineReaderUtils.UID_KEY, + TimelineUIDConverter.APPLICATION_UID.encodeUID(context)); + return; + default: + break; + } + } + context.setEntityType(entity.getType()); + context.setEntityIdPrefix(entity.getIdPrefix()); + context.setEntityId(entity.getId()); + if (context.getDoAsUser() != null) { + entity.setUID(TimelineReaderUtils.UID_KEY, + TimelineUIDConverter.SUB_APPLICATION_ENTITY_UID.encodeUID(context)); + } else { + entity.setUID(TimelineReaderUtils.UID_KEY, + TimelineUIDConverter.GENERIC_ENTITY_UID.encodeUID(context)); + } + } + + /** + * Get a set of entities matching given predicates by making a call to + * backend storage implementation. The meaning of each argument has been + * documented in detail with {@link TimelineReader#getEntities}.If cluster ID + * has not been supplied by the client, fills the cluster id from config + * before making a call to backend storage. After fetching entities from + * backend, fills the appropriate UID based on entity type for each entity. + * + * @param context Timeline context within the scope of which entities have to + * be fetched. + * @param filters Filters which limit the number of entities to be returned. + * @param dataToRetrieve Data to carry in each entity fetched. + * @return a set of TimelineEntity objects. + * @throws IOException if any problem occurs while getting entities. + * @see TimelineReader#getEntities + */ + public Set getEntities(TimelineReaderContext context, + TimelineEntityFilters filters, TimelineDataToRetrieve dataToRetrieve) + throws IOException { + context.setClusterId(getClusterID(context.getClusterId(), getConfig())); + Set entities = reader.getEntities( + new TimelineReaderContext(context), filters, dataToRetrieve); + if (entities != null) { + TimelineEntityType type = getTimelineEntityType(context.getEntityType()); + for (TimelineEntity entity : entities) { + fillUID(type, entity, context); + } + } + return entities; + } + + /** + * Get single timeline entity by making a call to backend storage + * implementation. The meaning of each argument in detail has been + * documented with {@link TimelineReader#getEntity}. If cluster ID has not + * been supplied by the client, fills the cluster id from config before making + * a call to backend storage. After fetching entity from backend, fills the + * appropriate UID based on entity type. + * + * @param context Timeline context within the scope of which entity has to be + * fetched. + * @param dataToRetrieve Data to carry in the entity fetched. + * @return A TimelineEntity object if found, null otherwise. + * @throws IOException if any problem occurs while getting entity. + * @see TimelineReader#getEntity + */ + public TimelineEntity getEntity(TimelineReaderContext context, + TimelineDataToRetrieve dataToRetrieve) throws IOException { + context.setClusterId( + getClusterID(context.getClusterId(), getConfig())); + TimelineEntity entity = reader.getEntity( + new TimelineReaderContext(context), dataToRetrieve); + if (entity != null) { + TimelineEntityType type = getTimelineEntityType(context.getEntityType()); + fillUID(type, entity, context); + } + return entity; + } + + /** + * Gets a list of available timeline entity types for an application. This can + * be done by making a call to the backend storage implementation. The meaning + * of each argument in detail is the same as {@link TimelineReader#getEntity}. + * If cluster ID has not been supplied by the client, fills the cluster id + * from config before making a call to backend storage. + * + * @param context Timeline context within the scope of which entity types + * have to be fetched. Entity type field of this context should + * be null. + * @return A set which contains available timeline entity types, represented + * as strings if found, empty otherwise. + * @throws IOException if any problem occurs while getting entity types. + */ + public Set getEntityTypes(TimelineReaderContext context) + throws IOException{ + context.setClusterId(getClusterID(context.getClusterId(), getConfig())); + return reader.getEntityTypes(new TimelineReaderContext(context)); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java new file mode 100644 index 00000000000..5c049eaac90 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java @@ -0,0 +1,231 @@ +/** + * 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.timelineservice.reader; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.URI; +import java.util.LinkedHashSet; +import java.util.Set; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.http.HttpServer2; +import org.apache.hadoop.security.HttpCrossOriginFilterInitializer; +import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.service.CompositeService; +import org.apache.hadoop.util.ExitUtil; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hadoop.util.ShutdownHookManager; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; +import org.apache.hadoop.yarn.server.timelineservice.reader.security.TimelineReaderAuthenticationFilterInitializer; +import org.apache.hadoop.yarn.server.timelineservice.reader.security.TimelineReaderWhitelistAuthorizationFilterInitializer; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader; +import org.apache.hadoop.yarn.server.util.timeline.TimelineServerUtils; +import org.apache.hadoop.yarn.webapp.GenericExceptionHandler; +import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider; +import org.apache.hadoop.yarn.webapp.util.WebAppUtils; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Main class for Timeline Reader. */ +@Private +@Unstable +public class TimelineReaderServer extends CompositeService { + private static final Logger LOG = + LoggerFactory.getLogger(TimelineReaderServer.class); + private static final int SHUTDOWN_HOOK_PRIORITY = 30; + static final String TIMELINE_READER_MANAGER_ATTR = + "timeline.reader.manager"; + + private HttpServer2 readerWebServer; + private TimelineReaderManager timelineReaderManager; + + public TimelineReaderServer() { + super(TimelineReaderServer.class.getName()); + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + if (!YarnConfiguration.timelineServiceV2Enabled(conf)) { + throw new YarnException("timeline service v.2 is not enabled"); + } + InetSocketAddress bindAddr = conf.getSocketAddr( + YarnConfiguration.TIMELINE_SERVICE_ADDRESS, + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ADDRESS, + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_PORT); + // Login from keytab if security is enabled. + try { + SecurityUtil.login(conf, YarnConfiguration.TIMELINE_SERVICE_KEYTAB, + YarnConfiguration.TIMELINE_SERVICE_PRINCIPAL, bindAddr.getHostName()); + } catch(IOException e) { + throw new YarnRuntimeException("Failed to login from keytab", e); + } + + TimelineReader timelineReaderStore = createTimelineReaderStore(conf); + timelineReaderStore.init(conf); + addService(timelineReaderStore); + timelineReaderManager = createTimelineReaderManager(timelineReaderStore); + addService(timelineReaderManager); + super.serviceInit(conf); + } + + private TimelineReader createTimelineReaderStore(final Configuration conf) { + String timelineReaderClassName = conf.get( + YarnConfiguration.TIMELINE_SERVICE_READER_CLASS, + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_READER_CLASS); + LOG.info("Using store: " + timelineReaderClassName); + try { + Class timelineReaderClazz = Class.forName(timelineReaderClassName); + if (TimelineReader.class.isAssignableFrom(timelineReaderClazz)) { + return (TimelineReader) ReflectionUtils.newInstance( + timelineReaderClazz, conf); + } else { + throw new YarnRuntimeException("Class: " + timelineReaderClassName + + " not instance of " + TimelineReader.class.getCanonicalName()); + } + } catch (ClassNotFoundException e) { + throw new YarnRuntimeException("Could not instantiate TimelineReader: " + + timelineReaderClassName, e); + } + } + + + private TimelineReaderManager createTimelineReaderManager( + TimelineReader timelineReaderStore) { + return new TimelineReaderManager(timelineReaderStore); + } + + @Override + protected void serviceStart() throws Exception { + super.serviceStart(); + startTimelineReaderWebApp(); + } + + private void join() { + // keep the main thread that started the server up until it receives a stop + // signal + if (readerWebServer != null) { + try { + readerWebServer.join(); + } catch (InterruptedException ignore) {} + } + } + + @Override + protected void serviceStop() throws Exception { + if (readerWebServer != null) { + readerWebServer.stop(); + } + super.serviceStop(); + } + + protected void addFilters(Configuration conf) { + boolean enableCorsFilter = conf.getBoolean( + YarnConfiguration.TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED, + YarnConfiguration.TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED_DEFAULT); + // Setup CORS + if (enableCorsFilter) { + conf.setBoolean(HttpCrossOriginFilterInitializer.PREFIX + + HttpCrossOriginFilterInitializer.ENABLED_SUFFIX, true); + } + String initializers = conf.get("hadoop.http.filter.initializers", ""); + Set defaultInitializers = new LinkedHashSet(); + if (!initializers.contains( + TimelineReaderAuthenticationFilterInitializer.class.getName())) { + defaultInitializers.add( + TimelineReaderAuthenticationFilterInitializer.class.getName()); + } + + defaultInitializers.add( + TimelineReaderWhitelistAuthorizationFilterInitializer.class.getName()); + + TimelineServerUtils.setTimelineFilters( + conf, initializers, defaultInitializers); + } + + private void startTimelineReaderWebApp() { + Configuration conf = getConfig(); + addFilters(conf); + String bindAddress = WebAppUtils.getWebAppBindURL(conf, + YarnConfiguration.TIMELINE_SERVICE_BIND_HOST, + WebAppUtils.getTimelineReaderWebAppURL(conf)); + LOG.info("Instantiating TimelineReaderWebApp at " + bindAddress); + try { + HttpServer2.Builder builder = new HttpServer2.Builder() + .setName("timeline") + .setConf(conf) + .addEndpoint(URI.create("http://" + bindAddress)); + readerWebServer = builder.build(); + readerWebServer.addJerseyResourcePackage( + TimelineReaderWebServices.class.getPackage().getName() + ";" + + GenericExceptionHandler.class.getPackage().getName() + ";" + + YarnJacksonJaxbJsonProvider.class.getPackage().getName(), + "/*"); + readerWebServer.setAttribute(TIMELINE_READER_MANAGER_ATTR, + timelineReaderManager); + readerWebServer.start(); + } catch (Exception e) { + String msg = "TimelineReaderWebApp failed to start."; + LOG.error(msg, e); + throw new YarnRuntimeException(msg, e); + } + } + + @VisibleForTesting + public int getWebServerPort() { + return readerWebServer.getConnectorAddress(0).getPort(); + } + + static TimelineReaderServer startTimelineReaderServer(String[] args, + Configuration conf) { + Thread.setDefaultUncaughtExceptionHandler( + new YarnUncaughtExceptionHandler()); + StringUtils.startupShutdownMessage(TimelineReaderServer.class, + args, LOG); + TimelineReaderServer timelineReaderServer = null; + try { + timelineReaderServer = new TimelineReaderServer(); + ShutdownHookManager.get().addShutdownHook( + new CompositeServiceShutdownHook(timelineReaderServer), + SHUTDOWN_HOOK_PRIORITY); + timelineReaderServer.init(conf); + timelineReaderServer.start(); + } catch (Throwable t) { + LOG.error("Error starting TimelineReaderWebServer", t); + ExitUtil.terminate(-1, "Error starting TimelineReaderWebServer"); + } + return timelineReaderServer; + } + + public static void main(String[] args) { + Configuration conf = new YarnConfiguration(); + conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); + conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f); + TimelineReaderServer server = startTimelineReaderServer(args, conf); + server.join(); + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderUtils.java new file mode 100644 index 00000000000..4fd846857e8 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderUtils.java @@ -0,0 +1,200 @@ +/** + * 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.timelineservice.reader; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import org.apache.commons.lang.StringUtils; + +import com.google.common.annotations.VisibleForTesting; + +/** + * Set of utility methods to be used across timeline reader. + */ +public final class TimelineReaderUtils { + private TimelineReaderUtils() { + } + + /** + * Default delimiter for joining strings. + */ + @VisibleForTesting + public static final char DEFAULT_DELIMITER_CHAR = '!'; + + /** + * Default escape character used for joining strings. + */ + @VisibleForTesting + public static final char DEFAULT_ESCAPE_CHAR = '*'; + + public static final String FROMID_KEY = "FROM_ID"; + + @VisibleForTesting + public static final String UID_KEY = "UID"; + + /** + * Split the passed string along the passed delimiter character while looking + * for escape char to interpret the splitted parts correctly. For delimiter or + * escape character to be interpreted as part of the string, they have to be + * escaped by putting an escape character in front. + * @param str string to be split. + * @param delimiterChar delimiter used for splitting. + * @param escapeChar delimiter and escape character will be escaped using this + * character. + * @return a list of strings after split. + * @throws IllegalArgumentException if string is not properly escaped. + */ + static List split(final String str, final char delimiterChar, + final char escapeChar) throws IllegalArgumentException { + if (str == null) { + return null; + } + int len = str.length(); + if (len == 0) { + return Collections.emptyList(); + } + List list = new ArrayList(); + // Keeps track of offset of the passed string. + int offset = 0; + // Indicates start offset from which characters will be copied from original + // string to destination string. Resets when an escape or delimiter char is + // encountered. + int startOffset = 0; + StringBuilder builder = new StringBuilder(len); + // Iterate over the string till we reach the end. + while (offset < len) { + if (str.charAt(offset) == escapeChar) { + // An escape character must be followed by a delimiter or escape char + // but we have reached the end and have no further character to look at. + if (offset + 1 >= len) { + throw new IllegalArgumentException( + "Escape char not properly escaped."); + } + char nextChar = str.charAt(offset + 1); + // Next character must be a delimiter or an escape char. + if (nextChar != escapeChar && nextChar != delimiterChar) { + throw new IllegalArgumentException( + "Escape char or delimiter char not properly escaped."); + } + // Copy contents from the offset where last escape or delimiter char was + // encountered. + if (startOffset < offset) { + builder.append(str.substring(startOffset, offset)); + } + builder.append(nextChar); + offset += 2; + // Reset the start offset as an escape char has been encountered. + startOffset = offset; + continue; + } else if (str.charAt(offset) == delimiterChar) { + // A delimiter has been encountered without an escape character. + // String needs to be split here. Copy remaining chars and add the + // string to list. + builder.append(str.substring(startOffset, offset)); + list.add(builder.toString().trim()); + // Reset the start offset as a delimiter has been encountered. + startOffset = ++offset; + builder = new StringBuilder(len - offset); + continue; + } + offset++; + } + // Copy rest of the characters. + if (!str.isEmpty()) { + builder.append(str.substring(startOffset)); + } + // Add the last part of delimited string to list. + list.add(builder.toString().trim()); + return list; + } + + private static String escapeString(final String str, final char delimiterChar, + final char escapeChar) { + if (str == null) { + return null; + } + int len = str.length(); + if (len == 0) { + return ""; + } + StringBuilder builder = new StringBuilder(); + // Keeps track of offset of the passed string. + int offset = 0; + // Indicates start offset from which characters will be copied from original + // string to destination string. Resets when an escape or delimiter char is + // encountered. + int startOffset = 0; + // Iterate over the string till we reach the end. + while (offset < len) { + char charAtOffset = str.charAt(offset); + if (charAtOffset == escapeChar || charAtOffset == delimiterChar) { + // If an escape or delimiter character is encountered, copy characters + // from the offset where escape or delimiter was last encountered. + if (startOffset < offset) { + builder.append(str.substring(startOffset, offset)); + } + // Append escape char before delimiter/escape char. + builder.append(escapeChar).append(charAtOffset); + // Reset start offset for copying characters when next escape/delimiter + // char is encountered. + startOffset = offset + 1; + } + offset++; + } + // Copy remaining characters. + builder.append(str.substring(startOffset)); + return builder.toString(); + } + + /** + * Join different strings in the passed string array delimited by passed + * delimiter with delimiter and escape character escaped using passed escape + * char. + * @param strs strings to be joined. + * @param delimiterChar delimiter used to join strings. + * @param escapeChar escape character used to escape delimiter and escape + * char. + * @return a single string joined using delimiter and properly escaped. + */ + static String joinAndEscapeStrings(final String[] strs, + final char delimiterChar, final char escapeChar) { + int len = strs.length; + // Escape each string in string array. + for (int index = 0; index < len; index++) { + if (strs[index] == null) { + return null; + } + strs[index] = escapeString(strs[index], delimiterChar, escapeChar); + } + // Join the strings after they have been escaped. + return StringUtils.join(strs, delimiterChar); + } + + public static List split(final String str) + throws IllegalArgumentException { + return split(str, DEFAULT_DELIMITER_CHAR, DEFAULT_ESCAPE_CHAR); + } + + public static String joinAndEscapeStrings(final String[] strs) { + return joinAndEscapeStrings(strs, DEFAULT_DELIMITER_CHAR, + DEFAULT_ESCAPE_CHAR); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java new file mode 100644 index 00000000000..caff3ed163d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java @@ -0,0 +1,3405 @@ +/** + * 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.timelineservice.reader; + +import java.text.DateFormat; +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.Collections; +import java.util.Date; +import java.util.Locale; +import java.util.Set; +import java.util.TimeZone; + +import javax.servlet.ServletContext; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import javax.ws.rs.GET; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.WebApplicationException; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.util.Time; +import org.apache.hadoop.yarn.api.records.timeline.TimelineAbout; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; +import org.apache.hadoop.yarn.util.timeline.TimelineUtils; +import org.apache.hadoop.yarn.webapp.BadRequestException; +import org.apache.hadoop.yarn.webapp.NotFoundException; + +import com.google.common.annotations.VisibleForTesting; +import com.google.inject.Singleton; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** REST end point for Timeline Reader. */ +@Private +@Unstable +@Singleton +@Path("/ws/v2/timeline") +public class TimelineReaderWebServices { + private static final Logger LOG = + LoggerFactory.getLogger(TimelineReaderWebServices.class); + + @Context private ServletContext ctxt; + + private static final String QUERY_STRING_SEP = "?"; + private static final String RANGE_DELIMITER = "-"; + private static final String DATE_PATTERN = "yyyyMMdd"; + + @VisibleForTesting + static final ThreadLocal DATE_FORMAT = + new ThreadLocal() { + @Override + protected DateFormat initialValue() { + SimpleDateFormat format = + new SimpleDateFormat(DATE_PATTERN, Locale.ENGLISH); + format.setTimeZone(TimeZone.getTimeZone("GMT")); + format.setLenient(false); + return format; + } + }; + + private void init(HttpServletResponse response) { + response.setContentType(null); + } + + private static final class DateRange { + private Long dateStart; + private Long dateEnd; + private DateRange(Long start, Long end) { + this.dateStart = start; + this.dateEnd = end; + } + } + + private static long parseDate(String strDate) throws ParseException { + Date date = DATE_FORMAT.get().parse(strDate); + return date.getTime(); + } + + /** + * Parses date range which can be a single date or in the format + * "[startdate]-[enddate]" where either of start or end date may not exist. + * @param dateRange + * @return a {@link DateRange} object. + * @throws IllegalArgumentException + */ + private static DateRange parseDateRange(String dateRange) + throws IllegalArgumentException { + if (dateRange == null || dateRange.isEmpty()) { + return new DateRange(null, null); + } + // Split date range around "-" fetching two components indicating start and + // end date. + String[] dates = dateRange.split(RANGE_DELIMITER, 2); + Long start = null; + Long end = null; + try { + String startDate = dates[0].trim(); + if (!startDate.isEmpty()) { + // Start date is not in yyyyMMdd format. + if (startDate.length() != DATE_PATTERN.length()) { + throw new IllegalArgumentException("Invalid date range " + dateRange); + } + // Parse start date which exists before "-" in date range. + // If "-" does not exist in date range, this effectively + // gives single date. + start = parseDate(startDate); + } + if (dates.length > 1) { + String endDate = dates[1].trim(); + if (!endDate.isEmpty()) { + // End date is not in yyyyMMdd format. + if (endDate.length() != DATE_PATTERN.length()) { + throw new IllegalArgumentException( + "Invalid date range " + dateRange); + } + // Parse end date which exists after "-" in date range. + end = parseDate(endDate); + } + } else { + // Its a single date(without "-" in date range), so set + // end equal to start. + end = start; + } + if (start != null && end != null) { + if (start > end) { + throw new IllegalArgumentException("Invalid date range " + dateRange); + } + } + return new DateRange(start, end); + } catch (ParseException e) { + // Date could not be parsed. + throw new IllegalArgumentException("Invalid date range " + dateRange); + } + } + + private TimelineReaderManager getTimelineReaderManager() { + return (TimelineReaderManager) + ctxt.getAttribute(TimelineReaderServer.TIMELINE_READER_MANAGER_ATTR); + } + + private static void handleException(Exception e, String url, long startTime, + String invalidNumMsg) throws BadRequestException, + WebApplicationException { + long endTime = Time.monotonicNow(); + LOG.info("Processed URL " + url + " but encountered exception (Took " + + (endTime - startTime) + " ms.)"); + if (e instanceof NumberFormatException) { + throw new BadRequestException(invalidNumMsg + " is not a numeric value."); + } else if (e instanceof IllegalArgumentException) { + throw new BadRequestException(e.getMessage() == null ? + "Requested Invalid Field." : e.getMessage()); + } else if (e instanceof NotFoundException) { + throw (NotFoundException)e; + } else if (e instanceof TimelineParseException) { + throw new BadRequestException(e.getMessage() == null ? + "Filter Parsing failed." : e.getMessage()); + } else if (e instanceof BadRequestException) { + throw (BadRequestException)e; + } else { + LOG.error("Error while processing REST request", e); + throw new WebApplicationException(e, + Response.Status.INTERNAL_SERVER_ERROR); + } + } + + /** + * Return the description of the timeline reader web services. + * + * @param req Servlet request. + * @param res Servlet response. + * + * @return information about the cluster including timeline version. + */ + @GET + @Produces(MediaType.APPLICATION_JSON) + public TimelineAbout about( + @Context HttpServletRequest req, + @Context HttpServletResponse res) { + init(res); + return TimelineUtils.createTimelineAbout("Timeline Reader API"); + } + + /** + * Return a single entity for a given entity type and UID which is a delimited + * string containing clusterid, userid, flow name, flowrun id and app id. + * + * @param req Servlet request. + * @param res Servlet response. + * @param uId a delimited string containing clusterid, userid, flow name, + * flowrun id and app id which are extracted from UID and then used to + * query backend(Mandatory path param). + * @param entityType Type of entities(Mandatory path param). + * @param limit If specified, defines the number of entities to return. The + * maximum possible value for limit can be {@link Long#MAX_VALUE}. If it + * is not specified or has a value less than 0, then limit will be + * considered as 100. (Optional query param). + * @param createdTimeStart If specified, matched entities should not be + * created before this timestamp(Optional query param). + * @param createdTimeEnd If specified, matched entities should not be created + * after this timestamp(Optional query param). + * @param relatesTo If specified, matched entities should relate to given + * entities associated with a entity type. relatesto is a comma separated + * list in the format [entitytype]:[entityid1]:[entityid2]... (Optional + * query param). + * @param isRelatedTo If specified, matched entities should be related to + * given entities associated with a entity type. relatesto is a comma + * separated list in the format [entitytype]:[entityid1]:[entityid2]... + * (Optional query param). + * @param infofilters If specified, matched entities should have exact matches + * to the given info represented as key-value pairs. This is represented + * as infofilters=info1:value1,info2:value2... (Optional query param). + * @param conffilters If specified, matched entities should have exact matches + * to the given configs represented as key-value pairs. This is + * represented as conffilters=conf1:value1,conf2:value2... (Optional query + * param). + * @param metricfilters If specified, matched entities should contain the + * given metrics. This is represented as + * metricfilters=metricid1, metricid2... (Optional query param). + * @param eventfilters If specified, matched entities should contain the given + * events. This is represented as eventfilters=eventid1, eventid2... + * @param confsToRetrieve If specified, defines which configurations to + * retrieve and send back in response. These configs will be retrieved + * irrespective of whether configs are specified in fields to retrieve or + * not. + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. These metrics will be retrieved + * irrespective of whether metrics are specified in fields to retrieve or + * not. + * @param fields Specifies which fields of the entity object to retrieve, see + * {@link Field}. All fields will be retrieved if fields=ALL. If not + * specified, 3 fields i.e. entity type, id and created time is returned + * (Optional query param). + * @param metricsLimit If specified, defines the number of metrics to return. + * Considered only if fields contains METRICS/ALL or metricsToRetrieve is + * specified. Ignored otherwise. The maximum possible value for + * metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified + * or has a value less than 1, and metrics have to be retrieved, then + * metricsLimit will be considered as 1 i.e. latest single value of + * metric(s) will be returned. (Optional query param). + * @param metricsTimeStart If specified, returned metrics for the entities + * would not contain metric values before this timestamp(Optional query + * param). + * @param metricsTimeEnd If specified, returned metrics for the entities would + * not contain metric values after this timestamp(Optional query param). + * @param fromId If specified, retrieve the next set of entities from the + * given fromId. The set of entities retrieved is inclusive of specified + * fromId. fromId should be taken from the value associated with FROM_ID + * info key in entity response which was sent earlier. + * + * @return If successful, a HTTP 200(OK) response having a JSON representing + * a set of TimelineEntity instances of the given entity type + * is returned.
+ * On failures,
+ * If any problem occurs in parsing request or UID is incorrect, + * HTTP 400(Bad Request) is returned.
+ * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/app-uid/{uid}/entities/{entitytype}") + @Produces(MediaType.APPLICATION_JSON) + public Set getEntities( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("uid") String uId, + @PathParam("entitytype") String entityType, + @QueryParam("limit") String limit, + @QueryParam("createdtimestart") String createdTimeStart, + @QueryParam("createdtimeend") String createdTimeEnd, + @QueryParam("relatesto") String relatesTo, + @QueryParam("isrelatedto") String isRelatedTo, + @QueryParam("infofilters") String infofilters, + @QueryParam("conffilters") String conffilters, + @QueryParam("metricfilters") String metricfilters, + @QueryParam("eventfilters") String eventfilters, + @QueryParam("confstoretrieve") String confsToRetrieve, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("metricslimit") String metricsLimit, + @QueryParam("metricstimestart") String metricsTimeStart, + @QueryParam("metricstimeend") String metricsTimeEnd, + @QueryParam("fromid") String fromId) { + String url = req.getRequestURI() + + (req.getQueryString() == null ? "" : + QUERY_STRING_SEP + req.getQueryString()); + UserGroupInformation callerUGI = + TimelineReaderWebServicesUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderWebServicesUtils.getUserName(callerUGI)); + long startTime = Time.monotonicNow(); + init(res); + TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); + Set entities = null; + try { + TimelineReaderContext context = + TimelineUIDConverter.APPLICATION_UID.decodeUID(uId); + if (context == null) { + throw new BadRequestException("Incorrect UID " + uId); + } + context.setEntityType( + TimelineReaderWebServicesUtils.parseStr(entityType)); + entities = timelineReaderManager.getEntities(context, + TimelineReaderWebServicesUtils.createTimelineEntityFilters( + limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo, + infofilters, conffilters, metricfilters, eventfilters, + fromId), + TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( + confsToRetrieve, metricsToRetrieve, fields, metricsLimit, + metricsTimeStart, metricsTimeEnd)); + } catch (Exception e) { + handleException(e, url, startTime, + "createdTime start/end or limit or flowrunid"); + } + long endTime = Time.monotonicNow(); + if (entities == null) { + entities = Collections.emptySet(); + } + LOG.info("Processed URL " + url + + " (Took " + (endTime - startTime) + " ms.)"); + return entities; + } + + /** + * Return a set of entities that match the given parameters. Cluster ID is not + * provided by client so default cluster ID has to be taken. If userid, flow + * name and flowrun id which are optional query parameters are not specified, + * they will be queried based on app id and default cluster id from the flow + * context information stored in underlying storage implementation. If number + * of matching entities are more than the limit, most recent entities till the + * limit is reached, will be returned. + * + * @param req Servlet request. + * @param res Servlet response. + * @param appId Application id to which the entities to be queried belong to( + * Mandatory path param). + * @param entityType Type of entities(Mandatory path param). + * @param userId User id which should match for the entities(Optional query + * param) + * @param flowName Flow name which should match for the entities(Optional + * query param). + * @param flowRunId Run id which should match for the entities(Optional query + * param). + * @param limit If specified, defines the number of entities to return. The + * maximum possible value for limit can be {@link Long#MAX_VALUE}. If it + * is not specified or has a value less than 0, then limit will be + * considered as 100. (Optional query param). + * @param createdTimeStart If specified, matched entities should not be + * created before this timestamp(Optional query param). + * @param createdTimeEnd If specified, matched entities should not be created + * after this timestamp(Optional query param). + * @param relatesTo If specified, matched entities should relate to given + * entities associated with a entity type. relatesto is a comma separated + * list in the format [entitytype]:[entityid1]:[entityid2]... (Optional + * query param). + * @param isRelatedTo If specified, matched entities should be related to + * given entities associated with a entity type. relatesto is a comma + * separated list in the format [entitytype]:[entityid1]:[entityid2]... + * (Optional query param). + * @param infofilters If specified, matched entities should have exact matches + * to the given info represented as key-value pairs. This is represented + * as infofilters=info1:value1,info2:value2... (Optional query param). + * @param conffilters If specified, matched entities should have exact matches + * to the given configs represented as key-value pairs. This is + * represented as conffilters=conf1:value1,conf2:value2... (Optional query + * param). + * @param metricfilters If specified, matched entities should contain the + * given metrics. This is represented as + * metricfilters=metricid1, metricid2... (Optional query param). + * @param eventfilters If specified, matched entities should contain the given + * events. This is represented as eventfilters=eventid1, eventid2... + * @param confsToRetrieve If specified, defines which configurations to + * retrieve and send back in response. These configs will be retrieved + * irrespective of whether configs are specified in fields to retrieve or + * not. + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. These metrics will be retrieved + * irrespective of whether metrics are specified in fields to retrieve or + * not. + * @param fields Specifies which fields of the entity object to retrieve, see + * {@link Field}. All fields will be retrieved if fields=ALL. If not + * specified, 3 fields i.e. entity type, id, created time is returned + * (Optional query param). + * @param metricsLimit If specified, defines the number of metrics to return. + * Considered only if fields contains METRICS/ALL or metricsToRetrieve is + * specified. Ignored otherwise. The maximum possible value for + * metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified + * or has a value less than 1, and metrics have to be retrieved, then + * metricsLimit will be considered as 1 i.e. latest single value of + * metric(s) will be returned. (Optional query param). + * @param metricsTimeStart If specified, returned metrics for the entities + * would not contain metric values before this timestamp(Optional query + * param). + * @param metricsTimeEnd If specified, returned metrics for the entities would + * not contain metric values after this timestamp(Optional query param). + * @param fromId If specified, retrieve the next set of entities from the + * given fromId. The set of entities retrieved is inclusive of specified + * fromId. fromId should be taken from the value associated with FROM_ID + * info key in entity response which was sent earlier. + * + * @return If successful, a HTTP 200(OK) response having a JSON representing + * a set of TimelineEntity instances of the given entity type + * is returned.
+ * On failures,
+ * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned.
+ * If flow context information cannot be retrieved, HTTP 404(Not Found) + * is returned.
+ * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/apps/{appid}/entities/{entitytype}") + @Produces(MediaType.APPLICATION_JSON) + public Set getEntities( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("appid") String appId, + @PathParam("entitytype") String entityType, + @QueryParam("userid") String userId, + @QueryParam("flowname") String flowName, + @QueryParam("flowrunid") String flowRunId, + @QueryParam("limit") String limit, + @QueryParam("createdtimestart") String createdTimeStart, + @QueryParam("createdtimeend") String createdTimeEnd, + @QueryParam("relatesto") String relatesTo, + @QueryParam("isrelatedto") String isRelatedTo, + @QueryParam("infofilters") String infofilters, + @QueryParam("conffilters") String conffilters, + @QueryParam("metricfilters") String metricfilters, + @QueryParam("eventfilters") String eventfilters, + @QueryParam("confstoretrieve") String confsToRetrieve, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("metricslimit") String metricsLimit, + @QueryParam("metricstimestart") String metricsTimeStart, + @QueryParam("metricstimeend") String metricsTimeEnd, + @QueryParam("fromid") String fromId) { + return getEntities(req, res, null, appId, entityType, userId, flowName, + flowRunId, limit, createdTimeStart, createdTimeEnd, relatesTo, + isRelatedTo, infofilters, conffilters, metricfilters, eventfilters, + confsToRetrieve, metricsToRetrieve, fields, metricsLimit, + metricsTimeStart, metricsTimeEnd, fromId); + } + + /** + * Return a set of entities that match the given parameters. If userid, flow + * name and flowrun id which are optional query parameters are not specified, + * they will be queried based on app id and cluster id from the flow context + * information stored in underlying storage implementation. If number of + * matching entities are more than the limit, most recent entities till the + * limit is reached, will be returned. + * + * @param req Servlet request. + * @param res Servlet response. + * @param clusterId Cluster id to which the entities to be queried belong to( + * Mandatory path param). + * @param appId Application id to which the entities to be queried belong to( + * Mandatory path param). + * @param entityType Type of entities(Mandatory path param). + * @param userId User id which should match for the entities(Optional query + * param) + * @param flowName Flow name which should match for the entities(Optional + * query param). + * @param flowRunId Run id which should match for the entities(Optional query + * param). + * @param limit If specified, defines the number of entities to return. The + * maximum possible value for limit can be {@link Long#MAX_VALUE}. If it + * is not specified or has a value less than 0, then limit will be + * considered as 100. (Optional query param). + * @param createdTimeStart If specified, matched entities should not be + * created before this timestamp(Optional query param). + * @param createdTimeEnd If specified, matched entities should not be created + * after this timestamp(Optional query param). + * @param relatesTo If specified, matched entities should relate to given + * entities associated with a entity type. relatesto is a comma separated + * list in the format [entitytype]:[entityid1]:[entityid2]... (Optional + * query param). + * @param isRelatedTo If specified, matched entities should be related to + * given entities associated with a entity type. relatesto is a comma + * separated list in the format [entitytype]:[entityid1]:[entityid2]... + * (Optional query param). + * @param infofilters If specified, matched entities should have exact matches + * to the given info represented as key-value pairs. This is represented + * as infofilters=info1:value1,info2:value2... (Optional query param). + * @param conffilters If specified, matched entities should have exact matches + * to the given configs represented as key-value pairs. This is + * represented as conffilters=conf1:value1,conf2:value2... (Optional query + * param). + * @param metricfilters If specified, matched entities should contain the + * given metrics. This is represented as + * metricfilters=metricid1, metricid2... (Optional query param). + * @param eventfilters If specified, matched entities should contain the given + * events. This is represented as eventfilters=eventid1, eventid2... + * @param confsToRetrieve If specified, defines which configurations to + * retrieve and send back in response. These configs will be retrieved + * irrespective of whether configs are specified in fields to retrieve or + * not. + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. These metrics will be retrieved + * irrespective of whether metrics are specified in fields to retrieve or + * not. + * @param fields Specifies which fields of the entity object to retrieve, see + * {@link Field}. All fields will be retrieved if fields=ALL. If not + * specified, 3 fields i.e. entity type, id, created time is returned + * (Optional query param). + * @param metricsLimit If specified, defines the number of metrics to return. + * Considered only if fields contains METRICS/ALL or metricsToRetrieve is + * specified. Ignored otherwise. The maximum possible value for + * metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified + * or has a value less than 1, and metrics have to be retrieved, then + * metricsLimit will be considered as 1 i.e. latest single value of + * metric(s) will be returned. (Optional query param). + * @param metricsTimeStart If specified, returned metrics for the entities + * would not contain metric values before this timestamp(Optional query + * param). + * @param metricsTimeEnd If specified, returned metrics for the entities would + * not contain metric values after this timestamp(Optional query param). + * @param fromId If specified, retrieve the next set of entities from the + * given fromId. The set of entities retrieved is inclusive of specified + * fromId. fromId should be taken from the value associated with FROM_ID + * info key in entity response which was sent earlier. + * + * @return If successful, a HTTP 200(OK) response having a JSON representing + * a set of TimelineEntity instances of the given entity type + * is returned.
+ * On failures,
+ * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned.
+ * If flow context information cannot be retrieved, HTTP 404(Not Found) + * is returned.
+ * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/clusters/{clusterid}/apps/{appid}/entities/{entitytype}") + @Produces(MediaType.APPLICATION_JSON) + public Set getEntities( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("clusterid") String clusterId, + @PathParam("appid") String appId, + @PathParam("entitytype") String entityType, + @QueryParam("userid") String userId, + @QueryParam("flowname") String flowName, + @QueryParam("flowrunid") String flowRunId, + @QueryParam("limit") String limit, + @QueryParam("createdtimestart") String createdTimeStart, + @QueryParam("createdtimeend") String createdTimeEnd, + @QueryParam("relatesto") String relatesTo, + @QueryParam("isrelatedto") String isRelatedTo, + @QueryParam("infofilters") String infofilters, + @QueryParam("conffilters") String conffilters, + @QueryParam("metricfilters") String metricfilters, + @QueryParam("eventfilters") String eventfilters, + @QueryParam("confstoretrieve") String confsToRetrieve, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("metricslimit") String metricsLimit, + @QueryParam("metricstimestart") String metricsTimeStart, + @QueryParam("metricstimeend") String metricsTimeEnd, + @QueryParam("fromid") String fromId) { + String url = req.getRequestURI() + + (req.getQueryString() == null ? "" : + QUERY_STRING_SEP + req.getQueryString()); + UserGroupInformation callerUGI = + TimelineReaderWebServicesUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderWebServicesUtils.getUserName(callerUGI)); + long startTime = Time.monotonicNow(); + init(res); + TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); + Set entities = null; + try { + TimelineReaderContext context = TimelineReaderWebServicesUtils + .createTimelineReaderContext(clusterId, userId, flowName, flowRunId, + appId, entityType, null, null); + entities = timelineReaderManager.getEntities(context, + TimelineReaderWebServicesUtils.createTimelineEntityFilters( + limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo, + infofilters, conffilters, metricfilters, eventfilters, + fromId), + TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( + confsToRetrieve, metricsToRetrieve, fields, metricsLimit, + metricsTimeStart, metricsTimeEnd)); + } catch (Exception e) { + handleException(e, url, startTime, + "createdTime start/end or limit or flowrunid"); + } + long endTime = Time.monotonicNow(); + if (entities == null) { + entities = Collections.emptySet(); + } + LOG.info("Processed URL " + url + + " (Took " + (endTime - startTime) + " ms.)"); + return entities; + } + + /** + * Return a single entity for given UID which is a delimited string containing + * clusterid, userid, flow name, flowrun id, app id, entity type and entityid. + * + * @param req Servlet request. + * @param res Servlet response. + * @param uId a delimited string containing clusterid, userid, flow name, + * flowrun id, app id, entity type and entity id which are extracted from + * UID and then used to query backend(Mandatory path param). + * @param confsToRetrieve If specified, defines which configurations to + * retrieve and send back in response. These configs will be retrieved + * irrespective of whether configs are specified in fields to retrieve or + * not. + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. These metrics will be retrieved + * irrespective of whether metrics are specified in fields to retrieve or + * not. + * @param fields Specifies which fields of the entity object to retrieve, see + * {@link Field}. All fields will be retrieved if fields=ALL. If not + * specified, 3 fields i.e. entity type, id, created time is returned + * (Optional query param). + * @param metricsLimit If specified, defines the number of metrics to return. + * Considered only if fields contains METRICS/ALL or metricsToRetrieve is + * specified. Ignored otherwise. The maximum possible value for + * metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified + * or has a value less than 1, and metrics have to be retrieved, then + * metricsLimit will be considered as 1 i.e. latest single value of + * metric(s) will be returned. (Optional query param). + * @param metricsTimeStart If specified, returned metrics for the entity would + * not contain metric values before this timestamp(Optional query param). + * @param metricsTimeEnd If specified, returned metrics for the entity would + * not contain metric values after this timestamp(Optional query param). + * + * @return If successful, a HTTP 200(OK) response having a JSON representing a + * TimelineEntity instance is returned.
+ * On failures,
+ * If any problem occurs in parsing request or UID is incorrect, + * HTTP 400(Bad Request) is returned.
+ * If entity for the given entity id cannot be found, HTTP 404(Not Found) + * is returned.
+ * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/entity-uid/{uid}/") + @Produces(MediaType.APPLICATION_JSON) + public TimelineEntity getEntity( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("uid") String uId, + @QueryParam("confstoretrieve") String confsToRetrieve, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("metricslimit") String metricsLimit, + @QueryParam("metricstimestart") String metricsTimeStart, + @QueryParam("metricstimeend") String metricsTimeEnd) { + String url = req.getRequestURI() + + (req.getQueryString() == null ? "" : + QUERY_STRING_SEP + req.getQueryString()); + UserGroupInformation callerUGI = + TimelineReaderWebServicesUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderWebServicesUtils.getUserName(callerUGI)); + long startTime = Time.monotonicNow(); + init(res); + TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); + TimelineEntity entity = null; + try { + TimelineReaderContext context = + TimelineUIDConverter.GENERIC_ENTITY_UID.decodeUID(uId); + if (context == null) { + throw new BadRequestException("Incorrect UID " + uId); + } + entity = timelineReaderManager.getEntity(context, + TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( + confsToRetrieve, metricsToRetrieve, fields, metricsLimit, + metricsTimeStart, metricsTimeEnd)); + } catch (Exception e) { + handleException(e, url, startTime, "flowrunid"); + } + long endTime = Time.monotonicNow(); + if (entity == null) { + LOG.info("Processed URL " + url + " but entity not found" + " (Took " + + (endTime - startTime) + " ms.)"); + throw new NotFoundException("Timeline entity with uid: " + uId + + "is not found"); + } + LOG.info("Processed URL " + url + + " (Took " + (endTime - startTime) + " ms.)"); + return entity; + } + + /** + * Return a single entity of the given entity type and Id. Cluster ID is not + * provided by client so default cluster ID has to be taken. If userid, flow + * name and flowrun id which are optional query parameters are not specified, + * they will be queried based on app id and default cluster id from the flow + * context information stored in underlying storage implementation. + * + * @param req Servlet request. + * @param res Servlet response. + * @param appId Application id to which the entity to be queried belongs to( + * Mandatory path param). + * @param entityType Type of entity(Mandatory path param). + * @param entityId Id of the entity to be fetched(Mandatory path param). + * @param userId User id which should match for the entity(Optional query + * param). + * @param flowName Flow name which should match for the entity(Optional query + * param). + * @param flowRunId Run id which should match for the entity(Optional query + * param). + * @param confsToRetrieve If specified, defines which configurations to + * retrieve and send back in response. These configs will be retrieved + * irrespective of whether configs are specified in fields to retrieve or + * not. + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. These metrics will be retrieved + * irrespective of whether metrics are specified in fields to retrieve or + * not. + * @param fields Specifies which fields of the entity object to retrieve, see + * {@link Field}. All fields will be retrieved if fields=ALL. If not + * specified, 3 fields i.e. entity type, id, created time is returned + * (Optional query param). + * @param metricsLimit If specified, defines the number of metrics to return. + * Considered only if fields contains METRICS/ALL or metricsToRetrieve is + * specified. Ignored otherwise. The maximum possible value for + * metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified + * or has a value less than 1, and metrics have to be retrieved, then + * metricsLimit will be considered as 1 i.e. latest single value of + * metric(s) will be returned. (Optional query param). + * @param metricsTimeStart If specified, returned metrics for the entity would + * not contain metric values before this timestamp(Optional query param). + * @param metricsTimeEnd If specified, returned metrics for the entity would + * not contain metric values after this timestamp(Optional query param). + * @param entityIdPrefix Defines the id prefix for the entity to be fetched. + * If specified, then entity retrieval will be faster. + * + * @return If successful, a HTTP 200(OK) response having a JSON representing a + * TimelineEntity instance is returned.
+ * On failures,
+ * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned.
+ * If flow context information cannot be retrieved or entity for the given + * entity id cannot be found, HTTP 404(Not Found) is returned.
+ * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/apps/{appid}/entities/{entitytype}/{entityid}/") + @Produces(MediaType.APPLICATION_JSON) + public TimelineEntity getEntity( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("appid") String appId, + @PathParam("entitytype") String entityType, + @PathParam("entityid") String entityId, + @QueryParam("userid") String userId, + @QueryParam("flowname") String flowName, + @QueryParam("flowrunid") String flowRunId, + @QueryParam("confstoretrieve") String confsToRetrieve, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("metricslimit") String metricsLimit, + @QueryParam("metricstimestart") String metricsTimeStart, + @QueryParam("metricstimeend") String metricsTimeEnd, + @QueryParam("entityidprefix") String entityIdPrefix) { + return getEntity(req, res, null, appId, entityType, entityId, userId, + flowName, flowRunId, confsToRetrieve, metricsToRetrieve, fields, + metricsLimit, metricsTimeStart, metricsTimeEnd, entityIdPrefix); + } + + /** + * Return a single entity of the given entity type and Id. If userid, flowname + * and flowrun id which are optional query parameters are not specified, they + * will be queried based on app id and cluster id from the flow context + * information stored in underlying storage implementation. + * + * @param req Servlet request. + * @param res Servlet response. + * @param clusterId Cluster id to which the entity to be queried belongs to( + * Mandatory path param). + * @param appId Application id to which the entity to be queried belongs to( + * Mandatory path param). + * @param entityType Type of entity(Mandatory path param). + * @param entityId Id of the entity to be fetched(Mandatory path param). + * @param userId User id which should match for the entity(Optional query + * param). + * @param flowName Flow name which should match for the entity(Optional query + * param). + * @param flowRunId Run id which should match for the entity(Optional query + * param). + * @param confsToRetrieve If specified, defines which configurations to + * retrieve and send back in response. These configs will be retrieved + * irrespective of whether configs are specified in fields to retrieve or + * not. + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. These metrics will be retrieved + * irrespective of whether metrics are specified in fields to retrieve or + * not. + * @param fields Specifies which fields of the entity object to retrieve, see + * {@link Field}. All fields will be retrieved if fields=ALL. If not + * specified, 3 fields i.e. entity type, id and created time is returned + * (Optional query param). + * @param metricsLimit If specified, defines the number of metrics to return. + * Considered only if fields contains METRICS/ALL or metricsToRetrieve is + * specified. Ignored otherwise. The maximum possible value for + * metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified + * or has a value less than 1, and metrics have to be retrieved, then + * metricsLimit will be considered as 1 i.e. latest single value of + * metric(s) will be returned. (Optional query param). + * @param metricsTimeStart If specified, returned metrics for the entity would + * not contain metric values before this timestamp(Optional query param). + * @param metricsTimeEnd If specified, returned metrics for the entity would + * not contain metric values after this timestamp(Optional query param). + * @param entityIdPrefix Defines the id prefix for the entity to be fetched. + * If specified, then entity retrieval will be faster. + * + * @return If successful, a HTTP 200(OK) response having a JSON representing a + * TimelineEntity instance is returned.
+ * On failures,
+ * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned.
+ * If flow context information cannot be retrieved or entity for the given + * entity id cannot be found, HTTP 404(Not Found) is returned.
+ * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/clusters/{clusterid}/apps/{appid}/entities/{entitytype}/{entityid}/") + @Produces(MediaType.APPLICATION_JSON) + public TimelineEntity getEntity( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("clusterid") String clusterId, + @PathParam("appid") String appId, + @PathParam("entitytype") String entityType, + @PathParam("entityid") String entityId, + @QueryParam("userid") String userId, + @QueryParam("flowname") String flowName, + @QueryParam("flowrunid") String flowRunId, + @QueryParam("confstoretrieve") String confsToRetrieve, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("metricslimit") String metricsLimit, + @QueryParam("metricstimestart") String metricsTimeStart, + @QueryParam("metricstimeend") String metricsTimeEnd, + @QueryParam("entityidprefix") String entityIdPrefix) { + String url = req.getRequestURI() + + (req.getQueryString() == null ? "" : + QUERY_STRING_SEP + req.getQueryString()); + UserGroupInformation callerUGI = + TimelineReaderWebServicesUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderWebServicesUtils.getUserName(callerUGI)); + long startTime = Time.monotonicNow(); + init(res); + TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); + TimelineEntity entity = null; + try { + entity = timelineReaderManager.getEntity( + TimelineReaderWebServicesUtils.createTimelineReaderContext( + clusterId, userId, flowName, flowRunId, appId, entityType, + entityIdPrefix, entityId), + TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( + confsToRetrieve, metricsToRetrieve, fields, metricsLimit, + metricsTimeStart, metricsTimeEnd)); + } catch (Exception e) { + handleException(e, url, startTime, "flowrunid"); + } + long endTime = Time.monotonicNow(); + if (entity == null) { + LOG.info("Processed URL " + url + " but entity not found" + " (Took " + + (endTime - startTime) + " ms.)"); + throw new NotFoundException("Timeline entity {id: " + entityId + + ", type: " + entityType + " } is not found"); + } + LOG.info("Processed URL " + url + + " (Took " + (endTime - startTime) + " ms.)"); + return entity; + } + + /** + * Return a single flow run for given UID which is a delimited string + * containing clusterid, userid, flow name and flowrun id. + * + * @param req Servlet request. + * @param res Servlet response. + * @param uId a delimited string containing clusterid, userid, flow name and + * flowrun id which are extracted from UID and then used to query backend + * (Mandatory path param). + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. + * + * @return If successful, a HTTP 200(OK) response having a JSON representing a + * FlowRunEntity instance is returned. By default, all + * metrics for the flow run will be returned.
+ * On failures,
+ * If any problem occurs in parsing request or UID is incorrect, + * HTTP 400(Bad Request) is returned.
+ * If flow run for the given flow run id cannot be found, HTTP 404 + * (Not Found) is returned.
+ * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/run-uid/{uid}/") + @Produces(MediaType.APPLICATION_JSON) + public TimelineEntity getFlowRun( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("uid") String uId, + @QueryParam("metricstoretrieve") String metricsToRetrieve) { + String url = req.getRequestURI() + + (req.getQueryString() == null ? "" : + QUERY_STRING_SEP + req.getQueryString()); + UserGroupInformation callerUGI = + TimelineReaderWebServicesUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderWebServicesUtils.getUserName(callerUGI)); + long startTime = Time.monotonicNow(); + init(res); + TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); + TimelineEntity entity = null; + try { + TimelineReaderContext context = + TimelineUIDConverter.FLOWRUN_UID.decodeUID(uId); + if (context == null) { + throw new BadRequestException("Incorrect UID " + uId); + } + context.setEntityType(TimelineEntityType.YARN_FLOW_RUN.toString()); + entity = timelineReaderManager.getEntity(context, + TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( + null, metricsToRetrieve, null, null, null, null)); + } catch (Exception e) { + handleException(e, url, startTime, "flowrunid"); + } + long endTime = Time.monotonicNow(); + if (entity == null) { + LOG.info("Processed URL " + url + " but flowrun not found (Took " + + (endTime - startTime) + " ms.)"); + throw new NotFoundException("Flowrun with uid: " + uId + "is not found"); + } + LOG.info("Processed URL " + url + + " (Took " + (endTime - startTime) + " ms.)"); + return entity; + } + + /** + * Return a single flow run for the given user, flow name and run id. + * Cluster ID is not provided by client so default cluster ID has to be taken. + * + * @param req Servlet request. + * @param res Servlet response. + * @param userId User id representing the user who ran the flow run(Mandatory + * path param). + * @param flowName Flow name to which the flow run to be queried belongs to( + * Mandatory path param). + * @param flowRunId Id of the flow run to be queried(Mandatory path param). + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. + * + * @return If successful, a HTTP 200(OK) response having a JSON representing a + * FlowRunEntity instance is returned. By default, all + * metrics for the flow run will be returned.
+ * On failures,
+ * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned.
+ * If flow run for the given flow run id cannot be found, HTTP 404 + * (Not Found) is returned.
+ * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/users/{userid}/flows/{flowname}/runs/{flowrunid}/") + @Produces(MediaType.APPLICATION_JSON) + public TimelineEntity getFlowRun( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("userid") String userId, + @PathParam("flowname") String flowName, + @PathParam("flowrunid") String flowRunId, + @QueryParam("metricstoretrieve") String metricsToRetrieve) { + return getFlowRun(req, res, null, userId, flowName, flowRunId, + metricsToRetrieve); + } + + /** + * Return a single flow run for the given user, cluster, flow name and run id. + * + * @param req Servlet request. + * @param res Servlet response. + * @param clusterId Cluster id to which the flow run to be queried belong to( + * Mandatory path param). + * @param userId User id representing the user who ran the flow run(Mandatory + * path param). + * @param flowName Flow name to which the flow run to be queried belongs to( + * Mandatory path param). + * @param flowRunId Id of the flow run to be queried(Mandatory path param). + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. + * + * @return If successful, a HTTP 200(OK) response having a JSON representing a + * FlowRunEntity instance is returned. By default, all + * metrics for the flow run will be returned.
+ * On failures,
+ * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned.
+ * If flow run for the given flow run id cannot be found, HTTP 404 + * (Not Found) is returned.
+ * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/clusters/{clusterid}/users/{userid}/flows/{flowname}/" + + "runs/{flowrunid}/") + @Produces(MediaType.APPLICATION_JSON) + public TimelineEntity getFlowRun( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("clusterid") String clusterId, + @PathParam("userid") String userId, + @PathParam("flowname") String flowName, + @PathParam("flowrunid") String flowRunId, + @QueryParam("metricstoretrieve") String metricsToRetrieve) { + String url = req.getRequestURI() + + (req.getQueryString() == null ? "" : + QUERY_STRING_SEP + req.getQueryString()); + UserGroupInformation callerUGI = + TimelineReaderWebServicesUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderWebServicesUtils.getUserName(callerUGI)); + long startTime = Time.monotonicNow(); + init(res); + TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); + TimelineEntity entity = null; + try { + entity = timelineReaderManager.getEntity( + TimelineReaderWebServicesUtils.createTimelineReaderContext( + clusterId, userId, flowName, flowRunId, null, + TimelineEntityType.YARN_FLOW_RUN.toString(), null, null), + TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( + null, metricsToRetrieve, null, null, null, null)); + } catch (Exception e) { + handleException(e, url, startTime, "flowrunid"); + } + long endTime = Time.monotonicNow(); + if (entity == null) { + LOG.info("Processed URL " + url + " but flowrun not found (Took " + + (endTime - startTime) + " ms.)"); + throw new NotFoundException("Flow run {flow name: " + + TimelineReaderWebServicesUtils.parseStr(flowName) + ", run id: " + + TimelineReaderWebServicesUtils.parseLongStr(flowRunId) + + " } is not found"); + } + LOG.info("Processed URL " + url + + " (Took " + (endTime - startTime) + " ms.)"); + return entity; + } + + /** + * Return a list of flow runs for given UID which is a delimited string + * containing clusterid, userid and flow name. + * + * @param req Servlet request. + * @param res Servlet response. + * @param uId a delimited string containing clusterid, userid, and flow name + * which are extracted from UID and then used to query backend(Mandatory + * path param). + * @param limit If specified, defines the number of flow runs to return. The + * maximum possible value for limit can be {@link Long#MAX_VALUE}. If it + * is not specified or has a value less than 0, then limit will be + * considered as 100. (Optional query param). + * @param createdTimeStart If specified, matched flow runs should not be + * created before this timestamp(Optional query param). + * @param createdTimeEnd If specified, matched flow runs should not be created + * after this timestamp(Optional query param). + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. These metrics will be retrieved + * irrespective of whether metrics are specified in fields to retrieve or + * not. + * @param fields Specifies which fields to retrieve, see {@link Field}. All + * fields will be retrieved if fields=ALL. Amongst all the fields, only + * METRICS makes sense for flow runs hence only ALL or METRICS are + * supported as fields for fetching flow runs. Other fields will lead to + * HTTP 400 (Bad Request) response. (Optional query param). + * @param fromId If specified, retrieve the next set of flow run entities + * from the given fromId. The set of entities retrieved is inclusive of + * specified fromId. fromId should be taken from the value associated + * with FROM_ID info key in entity response which was sent earlier. + * + * @return If successful, a HTTP 200(OK) response having a JSON representing a + * set of FlowRunEntity instances for the given flow are + * returned.
+ * On failures,
+ * If any problem occurs in parsing request or UID is incorrect, + * HTTP 400(Bad Request) is returned.
+ * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/flow-uid/{uid}/runs/") + @Produces(MediaType.APPLICATION_JSON) + public Set getFlowRuns( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("uid") String uId, + @QueryParam("limit") String limit, + @QueryParam("createdtimestart") String createdTimeStart, + @QueryParam("createdtimeend") String createdTimeEnd, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("fromid") String fromId) { + String url = req.getRequestURI() + + (req.getQueryString() == null ? "" : + QUERY_STRING_SEP + req.getQueryString()); + UserGroupInformation callerUGI = + TimelineReaderWebServicesUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderWebServicesUtils.getUserName(callerUGI)); + long startTime = Time.monotonicNow(); + init(res); + TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); + Set entities = null; + try { + TimelineReaderContext context = + TimelineUIDConverter.FLOW_UID.decodeUID(uId); + if (context == null) { + throw new BadRequestException("Incorrect UID " + uId); + } + context.setEntityType(TimelineEntityType.YARN_FLOW_RUN.toString()); + entities = timelineReaderManager.getEntities(context, + TimelineReaderWebServicesUtils.createTimelineEntityFilters( + limit, createdTimeStart, createdTimeEnd, null, null, null, + null, null, null, fromId), + TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( + null, metricsToRetrieve, fields, null, null, null)); + } catch (Exception e) { + handleException(e, url, startTime, + "createdTime start/end or limit or fromId"); + } + long endTime = Time.monotonicNow(); + if (entities == null) { + entities = Collections.emptySet(); + } + LOG.info("Processed URL " + url + + " (Took " + (endTime - startTime) + " ms.)"); + return entities; + } + + /** + * Return a set of flows runs for the given user and flow name. + * Cluster ID is not provided by client so default cluster ID has to be taken. + * + * @param req Servlet request. + * @param res Servlet response. + * @param userId User id representing the user who ran the flow runs( + * Mandatory path param) + * @param flowName Flow name to which the flow runs to be queried belongs to( + * Mandatory path param). + * @param limit If specified, defines the number of flow runs to return. The + * maximum possible value for limit can be {@link Long#MAX_VALUE}. If it + * is not specified or has a value less than 0, then limit will be + * considered as 100. (Optional query param). + * @param createdTimeStart If specified, matched flow runs should not be + * created before this timestamp(Optional query param). + * @param createdTimeEnd If specified, matched flow runs should not be created + * after this timestamp(Optional query param). + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. These metrics will be retrieved + * irrespective of whether metrics are specified in fields to retrieve or + * not. + * @param fields Specifies which fields to retrieve, see {@link Field}. All + * fields will be retrieved if fields=ALL. Amongst all the fields, only + * METRICS makes sense for flow runs hence only ALL or METRICS are + * supported as fields for fetching flow runs. Other fields will lead to + * HTTP 400 (Bad Request) response. (Optional query param). + * @param fromId If specified, retrieve the next set of flow run entities + * from the given fromId. The set of entities retrieved is inclusive of + * specified fromId. fromId should be taken from the value associated + * with FROM_ID info key in entity response which was sent earlier. + * + * @return If successful, a HTTP 200(OK) response having a JSON representing a + * set of FlowRunEntity instances for the given flow are + * returned.
+ * On failures,
+ * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned.
+ * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/users/{userid}/flows/{flowname}/runs/") + @Produces(MediaType.APPLICATION_JSON) + public Set getFlowRuns( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("userid") String userId, + @PathParam("flowname") String flowName, + @QueryParam("limit") String limit, + @QueryParam("createdtimestart") String createdTimeStart, + @QueryParam("createdtimeend") String createdTimeEnd, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("fromid") String fromId) { + return getFlowRuns(req, res, null, userId, flowName, limit, + createdTimeStart, createdTimeEnd, metricsToRetrieve, fields, fromId); + } + + /** + * Return a set of flows runs for the given cluster, user and flow name. + * + * @param req Servlet request. + * @param res Servlet response. + * @param clusterId Cluster id to which the flow runs to be queried belong to( + * Mandatory path param). + * @param userId User id representing the user who ran the flow runs( + * Mandatory path param) + * @param flowName Flow name to which the flow runs to be queried belongs to( + * Mandatory path param). + * @param limit If specified, defines the number of flow runs to return. The + * maximum possible value for limit can be {@link Long#MAX_VALUE}. If it + * is not specified or has a value less than 0, then limit will be + * considered as 100. (Optional query param). + * @param createdTimeStart If specified, matched flow runs should not be + * created before this timestamp(Optional query param). + * @param createdTimeEnd If specified, matched flow runs should not be created + * after this timestamp(Optional query param). + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. These metrics will be retrieved + * irrespective of whether metrics are specified in fields to retrieve or + * not. + * @param fields Specifies which fields to retrieve, see {@link Field}. All + * fields will be retrieved if fields=ALL. Amongst all the fields, only + * METRICS makes sense for flow runs hence only ALL or METRICS are + * supported as fields for fetching flow runs. Other fields will lead to + * HTTP 400 (Bad Request) response. (Optional query param). + * @param fromId If specified, retrieve the next set of flow run entities + * from the given fromId. The set of entities retrieved is inclusive of + * specified fromId. fromId should be taken from the value associated + * with FROM_ID info key in entity response which was sent earlier. + * + * @return If successful, a HTTP 200(OK) response having a JSON representing a + * set of FlowRunEntity instances for the given flow are + * returned.
+ * On failures,
+ * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned.
+ * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/clusters/{clusterid}/users/{userid}/flows/{flowname}/runs/") + @Produces(MediaType.APPLICATION_JSON) + public Set getFlowRuns( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("clusterid") String clusterId, + @PathParam("userid") String userId, + @PathParam("flowname") String flowName, + @QueryParam("limit") String limit, + @QueryParam("createdtimestart") String createdTimeStart, + @QueryParam("createdtimeend") String createdTimeEnd, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("fromid") String fromId) { + String url = req.getRequestURI() + + (req.getQueryString() == null ? "" : + QUERY_STRING_SEP + req.getQueryString()); + UserGroupInformation callerUGI = + TimelineReaderWebServicesUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderWebServicesUtils.getUserName(callerUGI)); + long startTime = Time.monotonicNow(); + init(res); + TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); + Set entities = null; + try { + entities = timelineReaderManager.getEntities( + TimelineReaderWebServicesUtils.createTimelineReaderContext( + clusterId, userId, flowName, null, null, + TimelineEntityType.YARN_FLOW_RUN.toString(), null, null), + TimelineReaderWebServicesUtils.createTimelineEntityFilters( + limit, createdTimeStart, createdTimeEnd, null, null, null, + null, null, null, fromId), + TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( + null, metricsToRetrieve, fields, null, null, null)); + } catch (Exception e) { + handleException(e, url, startTime, + "createdTime start/end or limit or fromId"); + } + long endTime = Time.monotonicNow(); + if (entities == null) { + entities = Collections.emptySet(); + } + LOG.info("Processed URL " + url + + " (Took " + (endTime - startTime) + " ms.)"); + return entities; + } + + /** + * Return a list of active flows. Cluster ID is not provided by client so + * default cluster ID has to be taken. + * + * @param req Servlet request. + * @param res Servlet response. + * @param limit If specified, defines the number of flows to return. The + * maximum possible value for limit can be {@link Long#MAX_VALUE}. If it + * is not specified or has a value less than 0, then limit will be + * considered as 100. (Optional query param). + * @param dateRange If specified is given as "[startdate]-[enddate]"(i.e. + * start and end date separated by "-") or single date. Dates are + * interpreted in yyyyMMdd format and are assumed to be in GMT(Optional + * query param). + * If a single date is specified, all flows active on that date are + * returned. If both startdate and enddate is given, all flows active + * between start and end date will be returned. If only startdate is + * given, flows active on and after startdate are returned. If only + * enddate is given, flows active on and before enddate are returned. + * For example : + * "daterange=20150711" returns flows active on 20150711. + * "daterange=20150711-20150714" returns flows active between these + * 2 dates. + * "daterange=20150711-" returns flows active on and after 20150711. + * "daterange=-20150711" returns flows active on and before 20150711. + * @param fromId If specified, retrieve the next set of flows from the given + * fromId. The set of flows retrieved is inclusive of specified fromId. + * fromId should be taken from the value associated with FROM_ID info key + * in flow entity response which was sent earlier. + * + * @return If successful, a HTTP 200(OK) response having a JSON representing a + * set of FlowActivityEntity instances are returned.
+ * On failures,
+ * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned.
+ * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned.
+ */ + @GET + @Path("/flows/") + @Produces(MediaType.APPLICATION_JSON) + public Set getFlows( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @QueryParam("limit") String limit, + @QueryParam("daterange") String dateRange, + @QueryParam("fromid") String fromId) { + return getFlows(req, res, null, limit, dateRange, fromId); + } + + /** + * Return a list of active flows for a given cluster id. + * + * @param req Servlet request. + * @param res Servlet response. + * @param clusterId Cluster id to which the flows to be queried belong to( + * Mandatory path param). + * @param limit If specified, defines the number of flows to return. The + * maximum possible value for limit can be {@link Long#MAX_VALUE}. If it + * is not specified or has a value less than 0, then limit will be + * considered as 100. (Optional query param). + * @param dateRange If specified is given as "[startdate]-[enddate]"(i.e. + * start and end date separated by "-") or single date. Dates are + * interpreted in yyyyMMdd format and are assumed to be in GMT(Optional + * query param). + * If a single date is specified, all flows active on that date are + * returned. If both startdate and enddate is given, all flows active + * between start and end date will be returned. If only startdate is + * given, flows active on and after startdate are returned. If only + * enddate is given, flows active on and before enddate are returned. + * For example : + * "daterange=20150711" returns flows active on 20150711. + * "daterange=20150711-20150714" returns flows active between these + * 2 dates. + * "daterange=20150711-" returns flows active on and after 20150711. + * "daterange=-20150711" returns flows active on and before 20150711. + * @param fromId If specified, retrieve the next set of flows from the given + * fromId. The set of flows retrieved is inclusive of specified fromId. + * fromId should be taken from the value associated with FROM_ID info key + * in flow entity response which was sent earlier. + * + * @return If successful, a HTTP 200(OK) response having a JSON representing a + * set of FlowActivityEntity instances are returned.
+ * On failures,
+ * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned.
+ * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/clusters/{clusterid}/flows/") + @Produces(MediaType.APPLICATION_JSON) + public Set getFlows( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("clusterid") String clusterId, + @QueryParam("limit") String limit, + @QueryParam("daterange") String dateRange, + @QueryParam("fromid") String fromId) { + String url = req.getRequestURI() + + (req.getQueryString() == null ? "" : + QUERY_STRING_SEP + req.getQueryString()); + UserGroupInformation callerUGI = + TimelineReaderWebServicesUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderWebServicesUtils.getUserName(callerUGI)); + long startTime = Time.monotonicNow(); + init(res); + TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); + Set entities = null; + try { + DateRange range = parseDateRange(dateRange); + TimelineEntityFilters entityFilters = + TimelineReaderWebServicesUtils.createTimelineEntityFilters( + limit, range.dateStart, range.dateEnd, + null, null, null, null, null, null, fromId); + entities = timelineReaderManager.getEntities( + TimelineReaderWebServicesUtils.createTimelineReaderContext( + clusterId, null, null, null, null, + TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), null, null), + entityFilters, TimelineReaderWebServicesUtils. + createTimelineDataToRetrieve(null, null, null, null, null, null)); + } catch (Exception e) { + handleException(e, url, startTime, "limit"); + } + long endTime = Time.monotonicNow(); + if (entities == null) { + entities = Collections.emptySet(); + } + LOG.info("Processed URL " + url + + " (Took " + (endTime - startTime) + " ms.)"); + return entities; + } + + /** + * Return a single app for given UID which is a delimited string containing + * clusterid, userid, flow name, flowrun id and app id. + * + * @param req Servlet request. + * @param res Servlet response. + * @param uId a delimited string containing clusterid, userid, flow name, flow + * run id and app id which are extracted from UID and then used to query + * backend(Mandatory path param). + * @param confsToRetrieve If specified, defines which configurations to + * retrieve and send back in response. These configs will be retrieved + * irrespective of whether configs are specified in fields to retrieve or + * not. + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. These metrics will be retrieved + * irrespective of whether metrics are specified in fields to retrieve or + * not. + * @param fields Specifies which fields of the app entity object to retrieve, + * see {@link Field}. All fields will be retrieved if fields=ALL. If not + * specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION), + * app id and app created time is returned(Optional query param). + * @param metricsLimit If specified, defines the number of metrics to return. + * Considered only if fields contains METRICS/ALL or metricsToRetrieve is + * specified. Ignored otherwise. The maximum possible value for + * metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified + * or has a value less than 1, and metrics have to be retrieved, then + * metricsLimit will be considered as 1 i.e. latest single value of + * metric(s) will be returned. (Optional query param). + * @param metricsTimeStart If specified, returned metrics for the apps would + * not contain metric values before this timestamp(Optional query param). + * @param metricsTimeEnd If specified, returned metrics for the apps would + * not contain metric values after this timestamp(Optional query param). + * + * @return If successful, a HTTP 200(OK) response having a JSON representing a + * TimelineEntity instance is returned.
+ * On failures,
+ * If any problem occurs in parsing request or UID is incorrect, + * HTTP 400(Bad Request) is returned.
+ * If app for the given app id cannot be found, HTTP 404(Not Found) is + * returned.
+ * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/app-uid/{uid}/") + @Produces(MediaType.APPLICATION_JSON) + public TimelineEntity getApp( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("uid") String uId, + @QueryParam("confstoretrieve") String confsToRetrieve, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("metricslimit") String metricsLimit, + @QueryParam("metricstimestart") String metricsTimeStart, + @QueryParam("metricstimeend") String metricsTimeEnd) { + String url = req.getRequestURI() + + (req.getQueryString() == null ? "" : + QUERY_STRING_SEP + req.getQueryString()); + UserGroupInformation callerUGI = + TimelineReaderWebServicesUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderWebServicesUtils.getUserName(callerUGI)); + long startTime = Time.monotonicNow(); + init(res); + TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); + TimelineEntity entity = null; + try { + TimelineReaderContext context = + TimelineUIDConverter.APPLICATION_UID.decodeUID(uId); + if (context == null) { + throw new BadRequestException("Incorrect UID " + uId); + } + context.setEntityType(TimelineEntityType.YARN_APPLICATION.toString()); + entity = timelineReaderManager.getEntity(context, + TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( + confsToRetrieve, metricsToRetrieve, fields, metricsLimit, + metricsTimeStart, metricsTimeEnd)); + } catch (Exception e) { + handleException(e, url, startTime, "flowrunid"); + } + long endTime = Time.monotonicNow(); + if (entity == null) { + LOG.info("Processed URL " + url + " but app not found" + " (Took " + + (endTime - startTime) + " ms.)"); + throw new NotFoundException("App with uid " + uId + " not found"); + } + LOG.info("Processed URL " + url + + " (Took " + (endTime - startTime) + " ms.)"); + return entity; + } + + /** + * Return a single app for given app id. Cluster ID is not provided by client + * client so default cluster ID has to be taken. If userid, flow name and flow + * run id which are optional query parameters are not specified, they will be + * queried based on app id and cluster id from the flow context information + * stored in underlying storage implementation. + * + * @param req Servlet request. + * @param res Servlet response. + * @param appId Application id to be queried(Mandatory path param). + * @param flowName Flow name which should match for the app(Optional query + * param). + * @param flowRunId Run id which should match for the app(Optional query + * param). + * @param userId User id which should match for the app(Optional query param). + * @param confsToRetrieve If specified, defines which configurations to + * retrieve and send back in response. These configs will be retrieved + * irrespective of whether configs are specified in fields to retrieve or + * not. + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. These metrics will be retrieved + * irrespective of whether metrics are specified in fields to retrieve or + * not. + * @param fields Specifies which fields of the app entity object to retrieve, + * see {@link Field}. All fields will be retrieved if fields=ALL. If not + * specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION), + * app id and app created time is returned(Optional query param). + * @param metricsLimit If specified, defines the number of metrics to return. + * Considered only if fields contains METRICS/ALL or metricsToRetrieve is + * specified. Ignored otherwise. The maximum possible value for + * metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified + * or has a value less than 1, and metrics have to be retrieved, then + * metricsLimit will be considered as 1 i.e. latest single value of + * metric(s) will be returned. (Optional query param). + * @param metricsTimeStart If specified, returned metrics for the app would + * not contain metric values before this timestamp(Optional query param). + * @param metricsTimeEnd If specified, returned metrics for the app would + * not contain metric values after this timestamp(Optional query param). + * + * @return If successful, a HTTP 200(OK) response having a JSON representing a + * TimelineEntity instance is returned.
+ * On failures,
+ * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned.
+ * If flow context information cannot be retrieved or app for the given + * app id cannot be found, HTTP 404(Not Found) is returned.
+ * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/apps/{appid}/") + @Produces(MediaType.APPLICATION_JSON) + public TimelineEntity getApp( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("appid") String appId, + @QueryParam("flowname") String flowName, + @QueryParam("flowrunid") String flowRunId, + @QueryParam("userid") String userId, + @QueryParam("confstoretrieve") String confsToRetrieve, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("metricslimit") String metricsLimit, + @QueryParam("metricstimestart") String metricsTimeStart, + @QueryParam("metricstimeend") String metricsTimeEnd) { + return getApp(req, res, null, appId, flowName, flowRunId, userId, + confsToRetrieve, metricsToRetrieve, fields, metricsLimit, + metricsTimeStart, metricsTimeEnd); + } + + /** + * Return a single app for given cluster id and app id. If userid, flow name + * and flowrun id which are optional query parameters are not specified, they + * will be queried based on app id and cluster id from the flow context + * information stored in underlying storage implementation. + * + * @param req Servlet request. + * @param res Servlet response. + * @param clusterId Cluster id to which the app to be queried belong to( + * Mandatory path param). + * @param appId Application id to be queried(Mandatory path param). + * @param flowName Flow name which should match for the app(Optional query + * param). + * @param flowRunId Run id which should match for the app(Optional query + * param). + * @param userId User id which should match for the app(Optional query param). + * @param confsToRetrieve If specified, defines which configurations to + * retrieve and send back in response. These configs will be retrieved + * irrespective of whether configs are specified in fields to retrieve or + * not. + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. These metrics will be retrieved + * irrespective of whether metrics are specified in fields to retrieve or + * not. + * @param fields Specifies which fields of the app entity object to retrieve, + * see {@link Field}. All fields will be retrieved if fields=ALL. If not + * specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION), + * app id and app created time is returned(Optional query param). + * @param metricsLimit If specified, defines the number of metrics to return. + * Considered only if fields contains METRICS/ALL or metricsToRetrieve is + * specified. Ignored otherwise. The maximum possible value for + * metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified + * or has a value less than 1, and metrics have to be retrieved, then + * metricsLimit will be considered as 1 i.e. latest single value of + * metric(s) will be returned. (Optional query param). + * @param metricsTimeStart If specified, returned metrics for the app would + * not contain metric values before this timestamp(Optional query param). + * @param metricsTimeEnd If specified, returned metrics for the app would + * not contain metric values after this timestamp(Optional query param). + * + * @return If successful, a HTTP 200(OK) response having a JSON representing a + * TimelineEntity instance is returned.
+ * On failures,
+ * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned.
+ * If flow context information cannot be retrieved or app for the given + * app id cannot be found, HTTP 404(Not Found) is returned.
+ * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/clusters/{clusterid}/apps/{appid}/") + @Produces(MediaType.APPLICATION_JSON) + public TimelineEntity getApp( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("clusterid") String clusterId, + @PathParam("appid") String appId, + @QueryParam("flowname") String flowName, + @QueryParam("flowrunid") String flowRunId, + @QueryParam("userid") String userId, + @QueryParam("confstoretrieve") String confsToRetrieve, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("metricslimit") String metricsLimit, + @QueryParam("metricstimestart") String metricsTimeStart, + @QueryParam("metricstimeend") String metricsTimeEnd) { + String url = req.getRequestURI() + + (req.getQueryString() == null ? "" : + QUERY_STRING_SEP + req.getQueryString()); + UserGroupInformation callerUGI = + TimelineReaderWebServicesUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderWebServicesUtils.getUserName(callerUGI)); + long startTime = Time.monotonicNow(); + init(res); + TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); + TimelineEntity entity = null; + try { + entity = timelineReaderManager.getEntity( + TimelineReaderWebServicesUtils.createTimelineReaderContext( + clusterId, userId, flowName, flowRunId, appId, + TimelineEntityType.YARN_APPLICATION.toString(), null, null), + TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( + confsToRetrieve, metricsToRetrieve, fields, metricsLimit, + metricsTimeStart, metricsTimeEnd)); + } catch (Exception e) { + handleException(e, url, startTime, "flowrunid"); + } + long endTime = Time.monotonicNow(); + if (entity == null) { + LOG.info("Processed URL " + url + " but app not found" + " (Took " + + (endTime - startTime) + " ms.)"); + throw new NotFoundException("App " + appId + " not found"); + } + LOG.info("Processed URL " + url + + " (Took " + (endTime - startTime) + " ms.)"); + return entity; + } + + /** + * Return a list of apps for given UID which is a delimited string containing + * clusterid, userid, flow name and flowrun id. If number of matching apps are + * more than the limit, most recent apps till the limit is reached, will be + * returned. + * + * @param req Servlet request. + * @param res Servlet response. + * @param uId a delimited string containing clusterid, userid, flow name and + * flowrun id which are extracted from UID and then used to query backend. + * (Mandatory path param). + * @param limit If specified, defines the number of apps to return. The + * maximum possible value for limit can be {@link Long#MAX_VALUE}. If it + * is not specified or has a value less than 0, then limit will be + * considered as 100. (Optional query param). + * @param createdTimeStart If specified, matched apps should not be created + * before this timestamp(Optional query param). + * @param createdTimeEnd If specified, matched apps should not be created + * after this timestamp(Optional query param). + * @param relatesTo If specified, matched apps should relate to given + * entities associated with a entity type. relatesto is a comma separated + * list in the format [entitytype]:[entityid1]:[entityid2]... (Optional + * query param). + * @param isRelatedTo If specified, matched apps should be related to given + * entities associated with a entity type. relatesto is a comma separated + * list in the format [entitytype]:[entityid1]:[entityid2]... (Optional + * query param). + * @param infofilters If specified, matched apps should have exact matches + * to the given info represented as key-value pairs. This is represented + * as infofilters=info1:value1,info2:value2... (Optional query param). + * @param conffilters If specified, matched apps should have exact matches + * to the given configs represented as key-value pairs. This is + * represented as conffilters=conf1:value1,conf2:value2... (Optional query + * param). + * @param metricfilters If specified, matched apps should contain the given + * metrics. This is represented as metricfilters=metricid1, metricid2... + * (Optional query param). + * @param eventfilters If specified, matched apps should contain the given + * events. This is represented as eventfilters=eventid1, eventid2... + * @param confsToRetrieve If specified, defines which configurations to + * retrieve and send back in response. These configs will be retrieved + * irrespective of whether configs are specified in fields to retrieve or + * not. + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. These metrics will be retrieved + * irrespective of whether metrics are specified in fields to retrieve or + * not. + * @param fields Specifies which fields of the app entity object to retrieve, + * see {@link Field}. All fields will be retrieved if fields=ALL. If not + * specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION), + * app id and app created time is returned(Optional query param). + * @param metricsLimit If specified, defines the number of metrics to return. + * Considered only if fields contains METRICS/ALL or metricsToRetrieve is + * specified. Ignored otherwise. The maximum possible value for + * metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified + * or has a value less than 1, and metrics have to be retrieved, then + * metricsLimit will be considered as 1 i.e. latest single value of + * metric(s) will be returned. (Optional query param). + * @param metricsTimeStart If specified, returned metrics for the apps would + * not contain metric values before this timestamp(Optional query param). + * @param metricsTimeEnd If specified, returned metrics for the apps would + * not contain metric values after this timestamp(Optional query param). + * @param fromId If specified, retrieve the next set of applications + * from the given fromId. The set of applications retrieved is inclusive + * of specified fromId. fromId should be taken from the value associated + * with FROM_ID info key in entity response which was sent earlier. + * + * @return If successful, a HTTP 200(OK) response having a JSON representing + * a set of TimelineEntity instances representing apps is + * returned.
+ * On failures,
+ * If any problem occurs in parsing request or UID is incorrect, + * HTTP 400(Bad Request) is returned.
+ * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/run-uid/{uid}/apps") + @Produces(MediaType.APPLICATION_JSON) + public Set getFlowRunApps( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("uid") String uId, + @QueryParam("limit") String limit, + @QueryParam("createdtimestart") String createdTimeStart, + @QueryParam("createdtimeend") String createdTimeEnd, + @QueryParam("relatesto") String relatesTo, + @QueryParam("isrelatedto") String isRelatedTo, + @QueryParam("infofilters") String infofilters, + @QueryParam("conffilters") String conffilters, + @QueryParam("metricfilters") String metricfilters, + @QueryParam("eventfilters") String eventfilters, + @QueryParam("confstoretrieve") String confsToRetrieve, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("metricslimit") String metricsLimit, + @QueryParam("metricstimestart") String metricsTimeStart, + @QueryParam("metricstimeend") String metricsTimeEnd, + @QueryParam("fromid") String fromId) { + String url = req.getRequestURI() + + (req.getQueryString() == null ? "" : + QUERY_STRING_SEP + req.getQueryString()); + UserGroupInformation callerUGI = + TimelineReaderWebServicesUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderWebServicesUtils.getUserName(callerUGI)); + long startTime = Time.monotonicNow(); + init(res); + TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); + Set entities = null; + try { + TimelineReaderContext context = + TimelineUIDConverter.FLOWRUN_UID.decodeUID(uId); + if (context == null) { + throw new BadRequestException("Incorrect UID " + uId); + } + context.setEntityType(TimelineEntityType.YARN_APPLICATION.toString()); + entities = timelineReaderManager.getEntities(context, + TimelineReaderWebServicesUtils.createTimelineEntityFilters( + limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo, + infofilters, conffilters, metricfilters, eventfilters, + fromId), + TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( + confsToRetrieve, metricsToRetrieve, fields, metricsLimit, + metricsTimeStart, metricsTimeEnd)); + } catch (Exception e) { + handleException(e, url, startTime, + "createdTime start/end or limit or flowrunid"); + } + long endTime = Time.monotonicNow(); + if (entities == null) { + entities = Collections.emptySet(); + } + LOG.info("Processed URL " + url + + " (Took " + (endTime - startTime) + " ms.)"); + return entities; + } + + /** + * Return a list of apps for given user, flow name and flow run id. Cluster ID + * is not provided by client so default cluster ID has to be taken. If number + * of matching apps are more than the limit, most recent apps till the limit + * is reached, will be returned. + * + * @param req Servlet request. + * @param res Servlet response. + * @param userId User id which should match for the apps(Mandatory path param) + * @param flowName Flow name which should match for the apps(Mandatory path + * param). + * @param flowRunId Run id which should match for the apps(Mandatory path + * param). + * @param limit If specified, defines the number of apps to return. The + * maximum possible value for limit can be {@link Long#MAX_VALUE}. If it + * is not specified or has a value less than 0, then limit will be + * considered as 100. (Optional query param). + * @param createdTimeStart If specified, matched apps should not be created + * before this timestamp(Optional query param). + * @param createdTimeEnd If specified, matched apps should not be created + * after this timestamp(Optional query param). + * @param relatesTo If specified, matched apps should relate to given + * entities associated with a entity type. relatesto is a comma separated + * list in the format [entitytype]:[entityid1]:[entityid2]... (Optional + * query param). + * @param isRelatedTo If specified, matched apps should be related to given + * entities associated with a entity type. relatesto is a comma separated + * list in the format [entitytype]:[entityid1]:[entityid2]... (Optional + * query param). + * @param infofilters If specified, matched apps should have exact matches + * to the given info represented as key-value pairs. This is represented + * as infofilters=info1:value1,info2:value2... (Optional query param). + * @param conffilters If specified, matched apps should have exact matches + * to the given configs represented as key-value pairs. This is + * represented as conffilters=conf1:value1,conf2:value2... (Optional query + * param). + * @param metricfilters If specified, matched apps should contain the given + * metrics. This is represented as metricfilters=metricid1, metricid2... + * (Optional query param). + * @param eventfilters If specified, matched apps should contain the given + * events. This is represented as eventfilters=eventid1, eventid2... + * @param confsToRetrieve If specified, defines which configurations to + * retrieve and send back in response. These configs will be retrieved + * irrespective of whether configs are specified in fields to retrieve or + * not. + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. These metrics will be retrieved + * irrespective of whether metrics are specified in fields to retrieve or + * not. + * @param fields Specifies which fields of the app entity object to retrieve, + * see {@link Field}. All fields will be retrieved if fields=ALL. If not + * specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION), + * app id and app created time is returned(Optional query param). + * @param metricsLimit If specified, defines the number of metrics to return. + * Considered only if fields contains METRICS/ALL or metricsToRetrieve is + * specified. Ignored otherwise. The maximum possible value for + * metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified + * or has a value less than 1, and metrics have to be retrieved, then + * metricsLimit will be considered as 1 i.e. latest single value of + * metric(s) will be returned. (Optional query param). + * @param metricsTimeStart If specified, returned metrics for the apps would + * not contain metric values before this timestamp(Optional query param). + * @param metricsTimeEnd If specified, returned metrics for the apps would + * not contain metric values after this timestamp(Optional query param). + * @param fromId If specified, retrieve the next set of applications + * from the given fromId. The set of applications retrieved is inclusive + * of specified fromId. fromId should be taken from the value associated + * with FROM_ID info key in entity response which was sent earlier. + * + * @return If successful, a HTTP 200(OK) response having a JSON representing + * a set of TimelineEntity instances representing apps is + * returned.
+ * On failures,
+ * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned.
+ * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/users/{userid}/flows/{flowname}/runs/{flowrunid}/apps/") + @Produces(MediaType.APPLICATION_JSON) + public Set getFlowRunApps( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("userid") String userId, + @PathParam("flowname") String flowName, + @PathParam("flowrunid") String flowRunId, + @QueryParam("limit") String limit, + @QueryParam("createdtimestart") String createdTimeStart, + @QueryParam("createdtimeend") String createdTimeEnd, + @QueryParam("relatesto") String relatesTo, + @QueryParam("isrelatedto") String isRelatedTo, + @QueryParam("infofilters") String infofilters, + @QueryParam("conffilters") String conffilters, + @QueryParam("metricfilters") String metricfilters, + @QueryParam("eventfilters") String eventfilters, + @QueryParam("confstoretrieve") String confsToRetrieve, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("metricslimit") String metricsLimit, + @QueryParam("metricstimestart") String metricsTimeStart, + @QueryParam("metricstimeend") String metricsTimeEnd, + @QueryParam("fromid") String fromId) { + return getEntities(req, res, null, null, + TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName, + flowRunId, limit, createdTimeStart, createdTimeEnd, relatesTo, + isRelatedTo, infofilters, conffilters, metricfilters, eventfilters, + confsToRetrieve, metricsToRetrieve, fields, metricsLimit, + metricsTimeStart, metricsTimeEnd, fromId); + } + + /** + * Return a list of apps for a given user, cluster id, flow name and flow run + * id. If number of matching apps are more than the limit, most recent apps + * till the limit is reached, will be returned. + * + * @param req Servlet request. + * @param res Servlet response. + * @param clusterId Cluster id to which the apps to be queried belong to + * (Mandatory path param). + * @param userId User id which should match for the apps(Mandatory path param) + * @param flowName Flow name which should match for the apps(Mandatory path + * param). + * @param flowRunId Run id which should match for the apps(Mandatory path + * param). + * @param limit If specified, defines the number of apps to return. The + * maximum possible value for limit can be {@link Long#MAX_VALUE}. If it + * is not specified or has a value less than 0, then limit will be + * considered as 100. (Optional query param). + * @param createdTimeStart If specified, matched apps should not be created + * before this timestamp(Optional query param). + * @param createdTimeEnd If specified, matched apps should not be created + * after this timestamp(Optional query param). + * @param relatesTo If specified, matched apps should relate to given + * entities associated with a entity type. relatesto is a comma separated + * list in the format [entitytype]:[entityid1]:[entityid2]... (Optional + * query param). + * @param isRelatedTo If specified, matched apps should be related to given + * entities associated with a entity type. relatesto is a comma separated + * list in the format [entitytype]:[entityid1]:[entityid2]... (Optional + * query param). + * @param infofilters If specified, matched apps should have exact matches + * to the given info represented as key-value pairs. This is represented + * as infofilters=info1:value1,info2:value2... (Optional query param). + * @param conffilters If specified, matched apps should have exact matches + * to the given configs represented as key-value pairs. This is + * represented as conffilters=conf1:value1,conf2:value2... (Optional query + * param). + * @param metricfilters If specified, matched apps should contain the given + * metrics. This is represented as metricfilters=metricid1, metricid2... + * (Optional query param). + * @param eventfilters If specified, matched apps should contain the given + * events. This is represented as eventfilters=eventid1, eventid2... + * @param confsToRetrieve If specified, defines which configurations to + * retrieve and send back in response. These configs will be retrieved + * irrespective of whether configs are specified in fields to retrieve or + * not. + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. These metrics will be retrieved + * irrespective of whether metrics are specified in fields to retrieve or + * not. + * @param fields Specifies which fields of the app entity object to retrieve, + * see {@link Field}. All fields will be retrieved if fields=ALL. If not + * specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION), + * app id and app created time is returned(Optional query param). + * @param metricsLimit If specified, defines the number of metrics to return. + * Considered only if fields contains METRICS/ALL or metricsToRetrieve is + * specified. Ignored otherwise. The maximum possible value for + * metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified + * or has a value less than 1, and metrics have to be retrieved, then + * metricsLimit will be considered as 1 i.e. latest single value of + * metric(s) will be returned. (Optional query param). + * @param metricsTimeStart If specified, returned metrics for the apps would + * not contain metric values before this timestamp(Optional query param). + * @param metricsTimeEnd If specified, returned metrics for the apps would + * not contain metric values after this timestamp(Optional query param). + * @param fromId If specified, retrieve the next set of applications + * from the given fromId. The set of applications retrieved is inclusive + * of specified fromId. fromId should be taken from the value associated + * with FROM_ID info key in entity response which was sent earlier. + * + * @return If successful, a HTTP 200(OK) response having a JSON representing + * a set of TimelineEntity instances representing apps is + * returned.
+ * On failures,
+ * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned.
+ * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/clusters/{clusterid}/users/{userid}/flows/{flowname}/runs/" + + "{flowrunid}/apps/") + @Produces(MediaType.APPLICATION_JSON) + public Set getFlowRunApps( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("clusterid") String clusterId, + @PathParam("userid") String userId, + @PathParam("flowname") String flowName, + @PathParam("flowrunid") String flowRunId, + @QueryParam("limit") String limit, + @QueryParam("createdtimestart") String createdTimeStart, + @QueryParam("createdtimeend") String createdTimeEnd, + @QueryParam("relatesto") String relatesTo, + @QueryParam("isrelatedto") String isRelatedTo, + @QueryParam("infofilters") String infofilters, + @QueryParam("conffilters") String conffilters, + @QueryParam("metricfilters") String metricfilters, + @QueryParam("eventfilters") String eventfilters, + @QueryParam("confstoretrieve") String confsToRetrieve, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("metricslimit") String metricsLimit, + @QueryParam("metricstimestart") String metricsTimeStart, + @QueryParam("metricstimeend") String metricsTimeEnd, + @QueryParam("fromid") String fromId) { + return getEntities(req, res, clusterId, null, + TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName, + flowRunId, limit, createdTimeStart, createdTimeEnd, relatesTo, + isRelatedTo, infofilters, conffilters, metricfilters, eventfilters, + confsToRetrieve, metricsToRetrieve, fields, metricsLimit, + metricsTimeStart, metricsTimeEnd, fromId); + } + + /** + * Return a list of apps for given user and flow name. Cluster ID is not + * provided by client so default cluster ID has to be taken. If number of + * matching apps are more than the limit, most recent apps till the limit is + * reached, will be returned. + * + * @param req Servlet request. + * @param res Servlet response. + * @param userId User id which should match for the apps(Mandatory path param) + * @param flowName Flow name which should match for the apps(Mandatory path + * param). + * @param limit If specified, defines the number of apps to return. The + * maximum possible value for limit can be {@link Long#MAX_VALUE}. If it + * is not specified or has a value less than 0, then limit will be + * considered as 100. (Optional query param). + * @param createdTimeStart If specified, matched apps should not be created + * before this timestamp(Optional query param). + * @param createdTimeEnd If specified, matched apps should not be created + * after this timestamp(Optional query param). + * @param relatesTo If specified, matched apps should relate to given + * entities associated with a entity type. relatesto is a comma separated + * list in the format [entitytype]:[entityid1]:[entityid2]... (Optional + * query param). + * @param isRelatedTo If specified, matched apps should be related to given + * entities associated with a entity type. relatesto is a comma separated + * list in the format [entitytype]:[entityid1]:[entityid2]... (Optional + * query param). + * @param infofilters If specified, matched apps should have exact matches + * to the given info represented as key-value pairs. This is represented + * as infofilters=info1:value1,info2:value2... (Optional query param). + * @param conffilters If specified, matched apps should have exact matches + * to the given configs represented as key-value pairs. This is + * represented as conffilters=conf1:value1,conf2:value2... (Optional query + * param). + * @param metricfilters If specified, matched apps should contain the given + * metrics. This is represented as metricfilters=metricid1, metricid2... + * (Optional query param). + * @param eventfilters If specified, matched apps should contain the given + * events. This is represented as eventfilters=eventid1, eventid2... + * @param confsToRetrieve If specified, defines which configurations to + * retrieve and send back in response. These configs will be retrieved + * irrespective of whether configs are specified in fields to retrieve or + * not. + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. These metrics will be retrieved + * irrespective of whether metrics are specified in fields to retrieve or + * not. + * @param fields Specifies which fields of the app entity object to retrieve, + * see {@link Field}. All fields will be retrieved if fields=ALL. If not + * specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION), + * app id and app created time is returned(Optional query param). + * @param metricsLimit If specified, defines the number of metrics to return. + * Considered only if fields contains METRICS/ALL or metricsToRetrieve is + * specified. Ignored otherwise. The maximum possible value for + * metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified + * or has a value less than 1, and metrics have to be retrieved, then + * metricsLimit will be considered as 1 i.e. latest single value of + * metric(s) will be returned. (Optional query param). + * @param metricsTimeStart If specified, returned metrics for the apps would + * not contain metric values before this timestamp(Optional query param). + * @param metricsTimeEnd If specified, returned metrics for the apps would + * not contain metric values after this timestamp(Optional query param). + * @param fromId If specified, retrieve the next set of applications + * from the given fromId. The set of applications retrieved is inclusive + * of specified fromId. fromId should be taken from the value associated + * with FROM_ID info key in entity response which was sent earlier. + * + * @return If successful, a HTTP 200(OK) response having a JSON representing + * a set of TimelineEntity instances representing apps is + * returned.
+ * On failures,
+ * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned.
+ * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/users/{userid}/flows/{flowname}/apps/") + @Produces(MediaType.APPLICATION_JSON) + public Set getFlowApps( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("userid") String userId, + @PathParam("flowname") String flowName, + @QueryParam("limit") String limit, + @QueryParam("createdtimestart") String createdTimeStart, + @QueryParam("createdtimeend") String createdTimeEnd, + @QueryParam("relatesto") String relatesTo, + @QueryParam("isrelatedto") String isRelatedTo, + @QueryParam("infofilters") String infofilters, + @QueryParam("conffilters") String conffilters, + @QueryParam("metricfilters") String metricfilters, + @QueryParam("eventfilters") String eventfilters, + @QueryParam("confstoretrieve") String confsToRetrieve, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("metricslimit") String metricsLimit, + @QueryParam("metricstimestart") String metricsTimeStart, + @QueryParam("metricstimeend") String metricsTimeEnd, + @QueryParam("fromid") String fromId) { + return getEntities(req, res, null, null, + TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName, + null, limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo, + infofilters, conffilters, metricfilters, eventfilters, + confsToRetrieve, metricsToRetrieve, fields, metricsLimit, + metricsTimeStart, metricsTimeEnd, fromId); + } + + /** + * Return a list of apps for a given user, cluster id and flow name. If number + * of matching apps are more than the limit, most recent apps till the limit + * is reached, will be returned. + * + * @param req Servlet request. + * @param res Servlet response. + * @param clusterId Cluster id to which the apps to be queried belong to + * (Mandatory path param). + * @param userId User id which should match for the apps(Mandatory path param) + * @param flowName Flow name which should match for the apps(Mandatory path + * param). + * @param limit If specified, defines the number of apps to return. The + * maximum possible value for limit can be {@link Long#MAX_VALUE}. If it + * is not specified or has a value less than 0, then limit will be + * considered as 100. (Optional query param). + * @param createdTimeStart If specified, matched apps should not be created + * before this timestamp(Optional query param). + * @param createdTimeEnd If specified, matched apps should not be created + * after this timestamp(Optional query param). + * @param relatesTo If specified, matched apps should relate to given + * entities associated with a entity type. relatesto is a comma separated + * list in the format [entitytype]:[entityid1]:[entityid2]... (Optional + * query param). + * @param isRelatedTo If specified, matched apps should be related to given + * entities associated with a entity type. relatesto is a comma separated + * list in the format [entitytype]:[entityid1]:[entityid2]... (Optional + * query param). + * @param infofilters If specified, matched apps should have exact matches + * to the given info represented as key-value pairs. This is represented + * as infofilters=info1:value1,info2:value2... (Optional query param). + * @param conffilters If specified, matched apps should have exact matches + * to the given configs represented as key-value pairs. This is + * represented as conffilters=conf1:value1,conf2:value2... (Optional query + * param). + * @param metricfilters If specified, matched apps should contain the given + * metrics. This is represented as metricfilters=metricid1, metricid2... + * (Optional query param). + * @param eventfilters If specified, matched apps should contain the given + * events. This is represented as eventfilters=eventid1, eventid2... + * @param confsToRetrieve If specified, defines which configurations to + * retrieve and send back in response. These configs will be retrieved + * irrespective of whether configs are specified in fields to retrieve or + * not. + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. These metrics will be retrieved + * irrespective of whether metrics are specified in fields to retrieve or + * not. + * @param fields Specifies which fields of the app entity object to retrieve, + * see {@link Field}. All fields will be retrieved if fields=ALL. If not + * specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION), + * app id and app created time is returned(Optional query param). + * @param metricsLimit If specified, defines the number of metrics to return. + * Considered only if fields contains METRICS/ALL or metricsToRetrieve is + * specified. Ignored otherwise. The maximum possible value for + * metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified + * or has a value less than 1, and metrics have to be retrieved, then + * metricsLimit will be considered as 1 i.e. latest single value of + * metric(s) will be returned. (Optional query param). + * @param metricsTimeStart If specified, returned metrics for the apps would + * not contain metric values before this timestamp(Optional query param). + * @param metricsTimeEnd If specified, returned metrics for the apps would + * not contain metric values after this timestamp(Optional query param). + * @param fromId If specified, retrieve the next set of applications + * from the given fromId. The set of applications retrieved is inclusive + * of specified fromId. fromId should be taken from the value associated + * with FROM_ID info key in entity response which was sent earlier. + * + * @return If successful, a HTTP 200(OK) response having a JSON representing + * a set of TimelineEntity instances representing apps is + * returned.
+ * On failures,
+ * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned.
+ * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/clusters/{clusterid}/users/{userid}/flows/{flowname}/apps/") + @Produces(MediaType.APPLICATION_JSON) + public Set getFlowApps( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("clusterid") String clusterId, + @PathParam("userid") String userId, + @PathParam("flowname") String flowName, + @QueryParam("limit") String limit, + @QueryParam("createdtimestart") String createdTimeStart, + @QueryParam("createdtimeend") String createdTimeEnd, + @QueryParam("relatesto") String relatesTo, + @QueryParam("isrelatedto") String isRelatedTo, + @QueryParam("infofilters") String infofilters, + @QueryParam("conffilters") String conffilters, + @QueryParam("metricfilters") String metricfilters, + @QueryParam("eventfilters") String eventfilters, + @QueryParam("confstoretrieve") String confsToRetrieve, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("metricslimit") String metricsLimit, + @QueryParam("metricstimestart") String metricsTimeStart, + @QueryParam("metricstimeend") String metricsTimeEnd, + @QueryParam("fromid") String fromId) { + return getEntities(req, res, clusterId, null, + TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName, + null, limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo, + infofilters, conffilters, metricfilters, eventfilters, + confsToRetrieve, metricsToRetrieve, fields, metricsLimit, + metricsTimeStart, metricsTimeEnd, fromId); + } + + /** + * Return a set of application-attempt entities for a given applicationId. + * Cluster ID is not provided by client so default cluster ID has to be taken. + * If userid, flow name and flowrun id which are optional query parameters are + * not specified, they will be queried based on app id and default cluster id + * from the flow context information stored in underlying storage + * implementation. If number of matching entities are more than the limit, + * most recent entities till the limit is reached, will be returned. + * + * @param req Servlet request. + * @param res Servlet response. + * @param appId Application id to which the entities to be queried belong to( + * Mandatory path param). + * @param userId User id which should match for the entities(Optional query + * param) + * @param flowName Flow name which should match for the entities(Optional + * query param). + * @param flowRunId Run id which should match for the entities(Optional query + * param). + * @param limit If specified, defines the number of entities to return. The + * maximum possible value for limit can be {@link Long#MAX_VALUE}. If + * it is not specified or has a value less than 0, then limit will be + * considered as 100. (Optional query param). + * @param createdTimeStart If specified, matched entities should not be + * created before this timestamp(Optional query param). + * @param createdTimeEnd If specified, matched entities should not be created + * after this timestamp(Optional query param). + * @param relatesTo If specified, matched entities should relate to given + * entities associated with a entity type. relatesto is a comma + * separated list in the format + * [entitytype]:[entityid1]:[entityid2]... (Optional query param). + * @param isRelatedTo If specified, matched entities should be related to + * given entities associated with a entity type. relatesto is a comma + * separated list in the format + * [entitytype]:[entityid1]:[entityid2]... (Optional query param). + * @param infofilters If specified, matched entities should have exact matches + * to the given info represented as key-value pairs. This is + * represented as infofilters=info1:value1,info2:value2... (Optional + * query param). + * @param conffilters If specified, matched entities should have exact matches + * to the given configs represented as key-value pairs. This is + * represented as conffilters=conf1:value1,conf2:value2... (Optional + * query param). + * @param metricfilters If specified, matched entities should contain the + * given metrics. This is represented as metricfilters=metricid1, + * metricid2... (Optional query param). + * @param eventfilters If specified, matched entities should contain the given + * events. This is represented as eventfilters=eventid1, eventid2... + * @param confsToRetrieve If specified, defines which configurations to + * retrieve and send back in response. These configs will be + * retrieved irrespective of whether configs are specified in fields + * to retrieve or not. + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. These metrics will be retrieved + * irrespective of whether metrics are specified in fields to + * retrieve or not. + * @param fields Specifies which fields of the entity object to retrieve, see + * {@link Field}. All fields will be retrieved if fields=ALL. If not + * specified, 3 fields i.e. entity type, id, created time is returned + * (Optional query param). + * @param metricsLimit If specified, defines the number of metrics to return. + * Considered only if fields contains METRICS/ALL or + * metricsToRetrieve is specified. Ignored otherwise. The maximum + * possible value for metricsLimit can be {@link Integer#MAX_VALUE}. + * If it is not specified or has a value less than 1, and metrics + * have to be retrieved, then metricsLimit will be considered as 1 + * i.e. latest single value of metric(s) will be returned. (Optional + * query param). + * @param metricsTimeStart If specified, returned metrics for the app attempts + * would not contain metric values before this timestamp(Optional + * query param). + * @param metricsTimeEnd If specified, returned metrics for the app attempts + * would not contain metric values after this timestamp(Optional + * query param). + * @param fromId If specified, retrieve the next set of application-attempt + * entities from the given fromId. The set of application-attempt + * entities retrieved is inclusive of specified fromId. fromId should + * be taken from the value associated with FROM_ID info key in + * entity response which was sent earlier. + * + * @return If successful, a HTTP 200(OK) response having a JSON representing a + * set of TimelineEntity instances of the app-attempt + * entity type is returned.
+ * On failures,
+ * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned.
+ * If flow context information cannot be retrieved, HTTP 404(Not + * Found) is returned.
+ * For all other errors while retrieving data, HTTP 500(Internal + * Server Error) is returned. + */ + @GET + @Path("/apps/{appid}/appattempts") + @Produces(MediaType.APPLICATION_JSON) + public Set getAppAttempts(@Context HttpServletRequest req, + @Context HttpServletResponse res, @PathParam("appid") String appId, + @QueryParam("userid") String userId, + @QueryParam("flowname") String flowName, + @QueryParam("flowrunid") String flowRunId, + @QueryParam("limit") String limit, + @QueryParam("createdtimestart") String createdTimeStart, + @QueryParam("createdtimeend") String createdTimeEnd, + @QueryParam("relatesto") String relatesTo, + @QueryParam("isrelatedto") String isRelatedTo, + @QueryParam("infofilters") String infofilters, + @QueryParam("conffilters") String conffilters, + @QueryParam("metricfilters") String metricfilters, + @QueryParam("eventfilters") String eventfilters, + @QueryParam("confstoretrieve") String confsToRetrieve, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("metricslimit") String metricsLimit, + @QueryParam("metricstimestart") String metricsTimeStart, + @QueryParam("metricstimeend") String metricsTimeEnd, + @QueryParam("fromid") String fromId) { + + return getAppAttempts(req, res, null, appId, userId, flowName, flowRunId, + limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo, + infofilters, conffilters, metricfilters, eventfilters, confsToRetrieve, + metricsToRetrieve, fields, metricsLimit, metricsTimeStart, + metricsTimeEnd, fromId); + } + + /** + * Return a set of application-attempt entities for a given applicationId. If + * userid, flow name and flowrun id which are optional query parameters are + * not specified, they will be queried based on app id and cluster id from the + * flow context information stored in underlying storage implementation. If + * number of matching entities are more than the limit, most recent entities + * till the limit is reached, will be returned. + * + * @param req Servlet request. + * @param res Servlet response. + * @param clusterId Cluster id to which the entities to be queried belong to( + * Mandatory path param). + * @param appId Application id to which the entities to be queried belong to( + * Mandatory path param). + * @param userId User id which should match for the entities(Optional query + * param) + * @param flowName Flow name which should match for the entities(Optional + * query param). + * @param flowRunId Run id which should match for the entities(Optional query + * param). + * @param limit If specified, defines the number of entities to return. The + * maximum possible value for limit can be {@link Long#MAX_VALUE}. If + * it is not specified or has a value less than 0, then limit will be + * considered as 100. (Optional query param). + * @param createdTimeStart If specified, matched entities should not be + * created before this timestamp(Optional query param). + * @param createdTimeEnd If specified, matched entities should not be created + * after this timestamp(Optional query param). + * @param relatesTo If specified, matched entities should relate to given + * entities associated with a entity type. relatesto is a comma + * separated list in the format + * [entitytype]:[entityid1]:[entityid2]... (Optional query param). + * @param isRelatedTo If specified, matched entities should be related to + * given entities associated with a entity type. relatesto is a comma + * separated list in the format + * [entitytype]:[entityid1]:[entityid2]... (Optional query param). + * @param infofilters If specified, matched entities should have exact matches + * to the given info represented as key-value pairs. This is + * represented as infofilters=info1:value1,info2:value2... (Optional + * query param). + * @param conffilters If specified, matched entities should have exact matches + * to the given configs represented as key-value pairs. This is + * represented as conffilters=conf1:value1,conf2:value2... (Optional + * query param). + * @param metricfilters If specified, matched entities should contain the + * given metrics. This is represented as metricfilters=metricid1, + * metricid2... (Optional query param). + * @param eventfilters If specified, matched entities should contain the given + * events. This is represented as eventfilters=eventid1, eventid2... + * @param confsToRetrieve If specified, defines which configurations to + * retrieve and send back in response. These configs will be + * retrieved irrespective of whether configs are specified in fields + * to retrieve or not. + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. These metrics will be retrieved + * irrespective of whether metrics are specified in fields to + * retrieve or not. + * @param fields Specifies which fields of the entity object to retrieve, see + * {@link Field}. All fields will be retrieved if fields=ALL. If not + * specified, 3 fields i.e. entity type, id, created time is returned + * (Optional query param). + * @param metricsLimit If specified, defines the number of metrics to return. + * Considered only if fields contains METRICS/ALL or + * metricsToRetrieve is specified. Ignored otherwise. The maximum + * possible value for metricsLimit can be {@link Integer#MAX_VALUE}. + * If it is not specified or has a value less than 1, and metrics + * have to be retrieved, then metricsLimit will be considered as 1 + * i.e. latest single value of metric(s) will be returned. (Optional + * query param). + * @param metricsTimeStart If specified, returned metrics for the app attempts + * would not contain metric values before this timestamp(Optional + * query param). + * @param metricsTimeEnd If specified, returned metrics for the app attempts + * would not contain metric values after this timestamp(Optional + * query param). + * @param fromId If specified, retrieve the next set of application-attempt + * entities from the given fromId. The set of application-attempt + * entities retrieved is inclusive of specified fromId. fromId should + * be taken from the value associated with FROM_ID info key in + * entity response which was sent earlier. + * + * @return If successful, a HTTP 200(OK) response having a JSON representing a + * set of TimelineEntity instances of the app-attempts + * entity type is returned.
+ * On failures,
+ * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned.
+ * If flow context information cannot be retrieved, HTTP 404(Not + * Found) is returned.
+ * For all other errors while retrieving data, HTTP 500(Internal + * Server Error) is returned. + */ + @GET + @Path("/clusters/{clusterid}/apps/{appid}/appattempts") + @Produces(MediaType.APPLICATION_JSON) + public Set getAppAttempts(@Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("clusterid") String clusterId, + @PathParam("appid") String appId, @QueryParam("userid") String userId, + @QueryParam("flowname") String flowName, + @QueryParam("flowrunid") String flowRunId, + @QueryParam("limit") String limit, + @QueryParam("createdtimestart") String createdTimeStart, + @QueryParam("createdtimeend") String createdTimeEnd, + @QueryParam("relatesto") String relatesTo, + @QueryParam("isrelatedto") String isRelatedTo, + @QueryParam("infofilters") String infofilters, + @QueryParam("conffilters") String conffilters, + @QueryParam("metricfilters") String metricfilters, + @QueryParam("eventfilters") String eventfilters, + @QueryParam("confstoretrieve") String confsToRetrieve, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("metricslimit") String metricsLimit, + @QueryParam("metricstimestart") String metricsTimeStart, + @QueryParam("metricstimeend") String metricsTimeEnd, + @QueryParam("fromid") String fromId) { + + return getEntities(req, res, clusterId, appId, + TimelineEntityType.YARN_APPLICATION_ATTEMPT.toString(), userId, + flowName, flowRunId, limit, createdTimeStart, createdTimeEnd, relatesTo, + isRelatedTo, infofilters, conffilters, metricfilters, eventfilters, + confsToRetrieve, metricsToRetrieve, fields, metricsLimit, + metricsTimeStart, metricsTimeEnd, fromId); + } + + /** + * Return a single application-attempt entity for the given attempt Id. + * Cluster ID is not provided by client so default cluster ID has to be taken. + * If userid, flow name and flowrun id which are optional query parameters are + * not specified, they will be queried based on app id and default cluster id + * from the flow context information stored in underlying storage + * implementation. + * + * @param req Servlet request. + * @param res Servlet response. + * @param appId Application id to which the entity to be queried belongs to( + * Mandatory path param). + * @param appAttemptId Application Attempt Id to which the containers belong + * to( Mandatory path param). + * @param userId User id which should match for the entity(Optional query + * param). + * @param flowName Flow name which should match for the entity(Optional query + * param). + * @param flowRunId Run id which should match for the entity(Optional query + * param). + * @param confsToRetrieve If specified, defines which configurations to + * retrieve and send back in response. These configs will be + * retrieved irrespective of whether configs are specified in fields + * to retrieve or not. + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. These metrics will be retrieved + * irrespective of whether metrics are specified in fields to + * retrieve or not. + * @param fields Specifies which fields of the entity object to retrieve, see + * {@link Field}. All fields will be retrieved if fields=ALL. If not + * specified, 3 fields i.e. entity type, id, created time is returned + * (Optional query param). + * @param metricsLimit If specified, defines the number of metrics to return. + * Considered only if fields contains METRICS/ALL or + * metricsToRetrieve is specified. Ignored otherwise. The maximum + * possible value for metricsLimit can be {@link Integer#MAX_VALUE}. + * If it is not specified or has a value less than 1, and metrics + * have to be retrieved, then metricsLimit will be considered as 1 + * i.e. latest single value of metric(s) will be returned. (Optional + * query param). + * @param metricsTimeStart If specified, returned metrics for the app attempt + * would not contain metric values before this timestamp(Optional + * query param). + * @param metricsTimeEnd If specified, returned metrics for the app attempt + * would not contain metric values after this timestamp(Optional + * query param). + * @param entityIdPrefix Defines the id prefix for the entity to be fetched. + * If specified, then entity retrieval will be faster. + * + * @return If successful, a HTTP 200(OK) response having a JSON representing a + * TimelineEntity instance is returned.
+ * On failures,
+ * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned.
+ * If flow context information cannot be retrieved or entity for the + * given entity id cannot be found, HTTP 404(Not Found) is + * returned.
+ * For all other errors while retrieving data, HTTP 500(Internal + * Server Error) is returned. + */ + @GET + @Path("/apps/{appid}/appattempts/{appattemptid}") + @Produces(MediaType.APPLICATION_JSON) + public TimelineEntity getAppAttempt(@Context HttpServletRequest req, + @Context HttpServletResponse res, @PathParam("appid") String appId, + @PathParam("appattemptid") String appAttemptId, + @QueryParam("userid") String userId, + @QueryParam("flowname") String flowName, + @QueryParam("flowrunid") String flowRunId, + @QueryParam("confstoretrieve") String confsToRetrieve, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("metricslimit") String metricsLimit, + @QueryParam("metricstimestart") String metricsTimeStart, + @QueryParam("metricstimeend") String metricsTimeEnd, + @QueryParam("entityidprefix") String entityIdPrefix) { + return getAppAttempt(req, res, null, appId, appAttemptId, userId, flowName, + flowRunId, confsToRetrieve, metricsToRetrieve, fields, metricsLimit, + metricsTimeStart, metricsTimeEnd, entityIdPrefix); + } + + /** + * Return a single application attempt entity of the given entity Id. If + * userid, flowname and flowrun id which are optional query parameters are not + * specified, they will be queried based on app id and cluster id from the + * flow context information stored in underlying storage implementation. + * + * @param req Servlet request. + * @param res Servlet response. + * @param clusterId Cluster id to which the entity to be queried belongs to( + * Mandatory path param). + * @param appId Application id to which the entity to be queried belongs to( + * Mandatory path param). + * @param appAttemptId Application Attempt Id to which the containers belong + * to( Mandatory path param). + * @param userId User id which should match for the entity(Optional query + * param). + * @param flowName Flow name which should match for the entity(Optional query + * param). + * @param flowRunId Run id which should match for the entity(Optional query + * param). + * @param confsToRetrieve If specified, defines which configurations to + * retrieve and send back in response. These configs will be + * retrieved irrespective of whether configs are specified in fields + * to retrieve or not. + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. These metrics will be retrieved + * irrespective of whether metrics are specified in fields to + * retrieve or not. + * @param fields Specifies which fields of the entity object to retrieve, see + * {@link Field}. All fields will be retrieved if fields=ALL. If not + * specified, 3 fields i.e. entity type, id and created time is + * returned (Optional query param). + * @param metricsLimit If specified, defines the number of metrics to return. + * Considered only if fields contains METRICS/ALL or + * metricsToRetrieve is specified. Ignored otherwise. The maximum + * possible value for metricsLimit can be {@link Integer#MAX_VALUE}. + * If it is not specified or has a value less than 1, and metrics + * have to be retrieved, then metricsLimit will be considered as 1 + * i.e. latest single value of metric(s) will be returned. (Optional + * query param). + * @param metricsTimeStart If specified, returned metrics for the app attempt + * would not contain metric values before this timestamp(Optional + * query param). + * @param metricsTimeEnd If specified, returned metrics for the app attempt + * would not contain metric values after this timestamp(Optional + * query param). + * @param entityIdPrefix Defines the id prefix for the entity to be fetched. + * If specified, then entity retrieval will be faster. + * + * @return If successful, a HTTP 200(OK) response having a JSON representing a + * TimelineEntity instance is returned.
+ * On failures,
+ * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned.
+ * If flow context information cannot be retrieved or entity for the + * given entity id cannot be found, HTTP 404(Not Found) is + * returned.
+ * For all other errors while retrieving data, HTTP 500(Internal + * Server Error) is returned. + */ + @GET + @Path("/clusters/{clusterid}/apps/{appid}/appattempts/{appattemptid}") + @Produces(MediaType.APPLICATION_JSON) + public TimelineEntity getAppAttempt(@Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("clusterid") String clusterId, + @PathParam("appid") String appId, + @PathParam("appattemptid") String appAttemptId, + @QueryParam("userid") String userId, + @QueryParam("flowname") String flowName, + @QueryParam("flowrunid") String flowRunId, + @QueryParam("confstoretrieve") String confsToRetrieve, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("metricslimit") String metricsLimit, + @QueryParam("metricstimestart") String metricsTimeStart, + @QueryParam("metricstimeend") String metricsTimeEnd, + @QueryParam("entityidprefix") String entityIdPrefix) { + return getEntity(req, res, clusterId, appId, + TimelineEntityType.YARN_APPLICATION_ATTEMPT.toString(), appAttemptId, + userId, flowName, flowRunId, confsToRetrieve, metricsToRetrieve, fields, + metricsLimit, metricsTimeStart, metricsTimeEnd, entityIdPrefix); + } + + /** + * Return a set of container entities belonging to given application attempt + * id. Cluster ID is not provided by client so default cluster ID has to be + * taken. If userid, flow name and flowrun id which are optional query + * parameters are not specified, they will be queried based on app id and + * default cluster id from the flow context information stored in underlying + * storage implementation. If number of matching entities are more than the + * limit, most recent entities till the limit is reached, will be returned. + * + * @param req Servlet request. + * @param res Servlet response. + * @param appId Application id to which the entities to be queried belong to( + * Mandatory path param). + * @param appattemptId Application Attempt Id to which the containers belong + * to( Mandatory path param). + * @param userId User id which should match for the entities(Optional query + * param) + * @param flowName Flow name which should match for the entities(Optional + * query param). + * @param flowRunId Run id which should match for the entities(Optional query + * param). + * @param limit If specified, defines the number of entities to return. The + * maximum possible value for limit can be {@link Long#MAX_VALUE}. If + * it is not specified or has a value less than 0, then limit will be + * considered as 100. (Optional query param). + * @param createdTimeStart If specified, matched entities should not be + * created before this timestamp(Optional query param). + * @param createdTimeEnd If specified, matched entities should not be created + * after this timestamp(Optional query param). + * @param relatesTo If specified, matched entities should relate to given + * entities associated with a entity type. relatesto is a comma + * separated list in the format + * [entitytype]:[entityid1]:[entityid2]... (Optional query param). + * @param isRelatedTo If specified, matched entities should be related to + * given entities associated with a entity type. relatesto is a comma + * separated list in the format + * [entitytype]:[entityid1]:[entityid2]... (Optional query param). + * @param infofilters If specified, matched entities should have exact matches + * to the given info represented as key-value pairs. This is + * represented as infofilters=info1:value1,info2:value2... (Optional + * query param). + * @param conffilters If specified, matched entities should have exact matches + * to the given configs represented as key-value pairs. This is + * represented as conffilters=conf1:value1,conf2:value2... (Optional + * query param). + * @param metricfilters If specified, matched entities should contain the + * given metrics. This is represented as metricfilters=metricid1, + * metricid2... (Optional query param). + * @param eventfilters If specified, matched entities should contain the given + * events. This is represented as eventfilters=eventid1, eventid2... + * @param confsToRetrieve If specified, defines which configurations to + * retrieve and send back in response. These configs will be + * retrieved irrespective of whether configs are specified in fields + * to retrieve or not. + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. These metrics will be retrieved + * irrespective of whether metrics are specified in fields to + * retrieve or not. + * @param fields Specifies which fields of the entity object to retrieve, see + * {@link Field}. All fields will be retrieved if fields=ALL. If not + * specified, 3 fields i.e. entity type, id, created time is returned + * (Optional query param). + * @param metricsLimit If specified, defines the number of metrics to return. + * Considered only if fields contains METRICS/ALL or + * metricsToRetrieve is specified. Ignored otherwise. The maximum + * possible value for metricsLimit can be {@link Integer#MAX_VALUE}. + * If it is not specified or has a value less than 1, and metrics + * have to be retrieved, then metricsLimit will be considered as 1 + * i.e. latest single value of metric(s) will be returned. (Optional + * query param). + * @param metricsTimeStart If specified, returned metrics for the containers + * would not contain metric values before this timestamp(Optional + * query param). + * @param metricsTimeEnd If specified, returned metrics for the containers + * would not contain metric values after this timestamp(Optional + * query param). + * @param fromId If specified, retrieve the next set of container + * entities from the given fromId. The set of container + * entities retrieved is inclusive of specified fromId. fromId should + * be taken from the value associated with FROM_ID info key in + * entity response which was sent earlier. + * + * @return If successful, a HTTP 200(OK) response having a JSON representing a + * set of TimelineEntity instances of the containers + * belongs to given application attempt id.
+ * On failures,
+ * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned.
+ * If flow context information cannot be retrieved, HTTP 404(Not + * Found) is returned.
+ * For all other errors while retrieving data, HTTP 500(Internal + * Server Error) is returned. + */ + @GET + @Path("/apps/{appid}/appattempts/{appattemptid}/containers") + @Produces(MediaType.APPLICATION_JSON) + public Set getContainers(@Context HttpServletRequest req, + @Context HttpServletResponse res, @PathParam("appid") String appId, + @PathParam("appattemptid") String appattemptId, + @QueryParam("userid") String userId, + @QueryParam("flowname") String flowName, + @QueryParam("flowrunid") String flowRunId, + @QueryParam("limit") String limit, + @QueryParam("createdtimestart") String createdTimeStart, + @QueryParam("createdtimeend") String createdTimeEnd, + @QueryParam("relatesto") String relatesTo, + @QueryParam("isrelatedto") String isRelatedTo, + @QueryParam("infofilters") String infofilters, + @QueryParam("conffilters") String conffilters, + @QueryParam("metricfilters") String metricfilters, + @QueryParam("eventfilters") String eventfilters, + @QueryParam("confstoretrieve") String confsToRetrieve, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("metricslimit") String metricsLimit, + @QueryParam("metricstimestart") String metricsTimeStart, + @QueryParam("metricstimeend") String metricsTimeEnd, + @QueryParam("fromid") String fromId) { + return getContainers(req, res, null, appId, appattemptId, userId, flowName, + flowRunId, limit, createdTimeStart, createdTimeEnd, relatesTo, + isRelatedTo, infofilters, conffilters, metricfilters, eventfilters, + confsToRetrieve, metricsToRetrieve, fields, metricsLimit, + metricsTimeStart, metricsTimeEnd, fromId); + } + + /** + * Return a set of container entities belonging to given application attempt + * id. If userid, flow name and flowrun id which are optional query parameters + * are not specified, they will be queried based on app id and cluster id from + * the flow context information stored in underlying storage implementation. + * If number of matching entities are more than the limit, most recent + * entities till the limit is reached, will be returned. + * + * @param req Servlet request. + * @param res Servlet response. + * @param clusterId Cluster id to which the entities to be queried belong to( + * Mandatory path param). + * @param appId Application id to which the entities to be queried belong to( + * Mandatory path param). + * @param appattemptId Application Attempt Id to which the containers belong + * to( Mandatory path param). + * @param userId User id which should match for the entities(Optional query + * param) + * @param flowName Flow name which should match for the entities(Optional + * query param). + * @param flowRunId Run id which should match for the entities(Optional query + * param). + * @param limit If specified, defines the number of entities to return. The + * maximum possible value for limit can be {@link Long#MAX_VALUE}. If + * it is not specified or has a value less than 0, then limit will be + * considered as 100. (Optional query param). + * @param createdTimeStart If specified, matched entities should not be + * created before this timestamp(Optional query param). + * @param createdTimeEnd If specified, matched entities should not be created + * after this timestamp(Optional query param). + * @param relatesTo If specified, matched entities should relate to given + * entities associated with a entity type. relatesto is a comma + * separated list in the format + * [entitytype]:[entityid1]:[entityid2]... (Optional query param). + * @param isRelatedTo If specified, matched entities should be related to + * given entities associated with a entity type. relatesto is a comma + * separated list in the format + * [entitytype]:[entityid1]:[entityid2]... (Optional query param). + * @param infofilters If specified, matched entities should have exact matches + * to the given info represented as key-value pairs. This is + * represented as infofilters=info1:value1,info2:value2... (Optional + * query param). + * @param conffilters If specified, matched entities should have exact matches + * to the given configs represented as key-value pairs. This is + * represented as conffilters=conf1:value1,conf2:value2... (Optional + * query param). + * @param metricfilters If specified, matched entities should contain the + * given metrics. This is represented as metricfilters=metricid1, + * metricid2... (Optional query param). + * @param eventfilters If specified, matched entities should contain the given + * events. This is represented as eventfilters=eventid1, eventid2... + * @param confsToRetrieve If specified, defines which configurations to + * retrieve and send back in response. These configs will be + * retrieved irrespective of whether configs are specified in fields + * to retrieve or not. + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. These metrics will be retrieved + * irrespective of whether metrics are specified in fields to + * retrieve or not. + * @param fields Specifies which fields of the entity object to retrieve, see + * {@link Field}. All fields will be retrieved if fields=ALL. If not + * specified, 3 fields i.e. entity type, id, created time is returned + * (Optional query param). + * @param metricsLimit If specified, defines the number of metrics to return. + * Considered only if fields contains METRICS/ALL or + * metricsToRetrieve is specified. Ignored otherwise. The maximum + * possible value for metricsLimit can be {@link Integer#MAX_VALUE}. + * If it is not specified or has a value less than 1, and metrics + * have to be retrieved, then metricsLimit will be considered as 1 + * i.e. latest single value of metric(s) will be returned. (Optional + * query param). + * @param metricsTimeStart If specified, returned metrics for the containers + * would not contain metric values before this timestamp(Optional + * query param). + * @param metricsTimeEnd If specified, returned metrics for the containers + * would not contain metric values after this timestamp(Optional + * query param). + * @param fromId If specified, retrieve the next set of container + * entities from the given fromId. The set of container + * entities retrieved is inclusive of specified fromId. fromId should + * be taken from the value associated with FROM_ID info key in + * entity response which was sent earlier. + * + * @return If successful, a HTTP 200(OK) response having a JSON representing a + * set of TimelineEntity instances of the containers + * belongs to given application attempt id.
+ * On failures,
+ * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned.
+ * If flow context information cannot be retrieved, HTTP 404(Not + * Found) is returned.
+ * For all other errors while retrieving data, HTTP 500(Internal + * Server Error) is returned. + */ + @GET + @Path("/clusters/{clusterid}/apps/{appid}/appattempts/{appattemptid}/containers") + @Produces(MediaType.APPLICATION_JSON) + public Set getContainers(@Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("clusterid") String clusterId, + @PathParam("appid") String appId, + @PathParam("appattemptid") String appattemptId, + @QueryParam("userid") String userId, + @QueryParam("flowname") String flowName, + @QueryParam("flowrunid") String flowRunId, + @QueryParam("limit") String limit, + @QueryParam("createdtimestart") String createdTimeStart, + @QueryParam("createdtimeend") String createdTimeEnd, + @QueryParam("relatesto") String relatesTo, + @QueryParam("isrelatedto") String isRelatedTo, + @QueryParam("infofilters") String infofilters, + @QueryParam("conffilters") String conffilters, + @QueryParam("metricfilters") String metricfilters, + @QueryParam("eventfilters") String eventfilters, + @QueryParam("confstoretrieve") String confsToRetrieve, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("metricslimit") String metricsLimit, + @QueryParam("metricstimestart") String metricsTimeStart, + @QueryParam("metricstimeend") String metricsTimeEnd, + @QueryParam("fromid") String fromId) { + + String entityType = TimelineEntityType.YARN_CONTAINER.toString(); + String parentEntityType = + TimelineEntityType.YARN_APPLICATION_ATTEMPT.toString(); + String jsonFormatString = "{\"type\":\"" + parentEntityType + "\",\"id\":\"" + + appattemptId + "\"}"; + String containerFilters = + "SYSTEM_INFO_PARENT_ENTITY eq " + jsonFormatString; + String infofilter; + if (infofilters != null) { + infofilter = containerFilters + " AND " + infofilters; + } else { + infofilter = containerFilters; + } + return getEntities(req, res, clusterId, appId, entityType, userId, flowName, + flowRunId, limit, createdTimeStart, createdTimeEnd, relatesTo, + isRelatedTo, infofilter, conffilters, metricfilters, eventfilters, + confsToRetrieve, metricsToRetrieve, fields, metricsLimit, + metricsTimeStart, metricsTimeEnd, fromId); + } + + /** + * Return a single container entity for the given container Id. Cluster ID is + * not provided by client so default cluster ID has to be taken. If userid, + * flow name and flowrun id which are optional query parameters are not + * specified, they will be queried based on app id and default cluster id from + * the flow context information stored in underlying storage implementation. + * + * @param req Servlet request. + * @param res Servlet response. + * @param appId Application id to which the entity to be queried belongs to( + * Mandatory path param). + * @param containerId Container Id to which the entity to be queried belongs + * to( Mandatory path param). + * @param userId User id which should match for the entity(Optional query + * param). + * @param flowName Flow name which should match for the entity(Optional query + * param). + * @param flowRunId Run id which should match for the entity(Optional query + * param). + * @param confsToRetrieve If specified, defines which configurations to + * retrieve and send back in response. These configs will be + * retrieved irrespective of whether configs are specified in fields + * to retrieve or not. + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. These metrics will be retrieved + * irrespective of whether metrics are specified in fields to + * retrieve or not. + * @param fields Specifies which fields of the entity object to retrieve, see + * {@link Field}. All fields will be retrieved if fields=ALL. If not + * specified, 3 fields i.e. entity type, id, created time is returned + * (Optional query param). + * @param metricsLimit If specified, defines the number of metrics to return. + * Considered only if fields contains METRICS/ALL or + * metricsToRetrieve is specified. Ignored otherwise. The maximum + * possible value for metricsLimit can be {@link Integer#MAX_VALUE}. + * If it is not specified or has a value less than 1, and metrics + * have to be retrieved, then metricsLimit will be considered as 1 + * i.e. latest single value of metric(s) will be returned. (Optional + * query param). + * @param metricsTimeStart If specified, returned metrics for the container + * would not contain metric values before this timestamp(Optional + * query param). + * @param metricsTimeEnd If specified, returned metrics for the container + * would not contain metric values after this timestamp(Optional + * query param). + * @param entityIdPrefix Defines the id prefix for the entity to be fetched. + * If specified, then entity retrieval will be faster. + * + * @return If successful, a HTTP 200(OK) response having a JSON representing + * TimelineEntity instance is returned.
+ * On failures,
+ * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned.
+ * If flow context information cannot be retrieved or entity for the + * given entity id cannot be found, HTTP 404(Not Found) is + * returned.
+ * For all other errors while retrieving data, HTTP 500(Internal + * Server Error) is returned. + */ + @GET + @Path("/apps/{appid}/containers/{containerid}") + @Produces(MediaType.APPLICATION_JSON) + public TimelineEntity getContainer(@Context HttpServletRequest req, + @Context HttpServletResponse res, @PathParam("appid") String appId, + @PathParam("containerid") String containerId, + @QueryParam("userid") String userId, + @QueryParam("flowname") String flowName, + @QueryParam("flowrunid") String flowRunId, + @QueryParam("confstoretrieve") String confsToRetrieve, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("metricslimit") String metricsLimit, + @QueryParam("metricstimestart") String metricsTimeStart, + @QueryParam("metricstimeend") String metricsTimeEnd, + @QueryParam("entityidprefix") String entityIdPrefix) { + return getContainer(req, res, null, appId, containerId, userId, flowName, + flowRunId, confsToRetrieve, metricsToRetrieve, fields, metricsLimit, + entityIdPrefix, metricsTimeStart, metricsTimeEnd); + } + + /** + * Return a single container entity for the given container Id. If userid, + * flowname and flowrun id which are optional query parameters are not + * specified, they will be queried based on app id and cluster id from the + * flow context information stored in underlying storage implementation. + * + * @param req Servlet request. + * @param res Servlet response. + * @param clusterId Cluster id to which the entity to be queried belongs to( + * Mandatory path param). + * @param appId Application id to which the entity to be queried belongs to( + * Mandatory path param). + * @param containerId Container Id to which the entity to be queried belongs + * to( Mandatory path param). + * @param userId User id which should match for the entity(Optional query + * param). + * @param flowName Flow name which should match for the entity(Optional query + * param). + * @param flowRunId Run id which should match for the entity(Optional query + * param). + * @param confsToRetrieve If specified, defines which configurations to + * retrieve and send back in response. These configs will be + * retrieved irrespective of whether configs are specified in fields + * to retrieve or not. + * @param metricsToRetrieve If specified, defines which metrics to retrieve + * and send back in response. These metrics will be retrieved + * irrespective of whether metrics are specified in fields to + * retrieve or not. + * @param fields Specifies which fields of the entity object to retrieve, see + * {@link Field}. All fields will be retrieved if fields=ALL. If not + * specified, 3 fields i.e. entity type, id and created time is + * returned (Optional query param). + * @param metricsLimit If specified, defines the number of metrics to return. + * Considered only if fields contains METRICS/ALL or + * metricsToRetrieve is specified. Ignored otherwise. The maximum + * possible value for metricsLimit can be {@link Integer#MAX_VALUE}. + * If it is not specified or has a value less than 1, and metrics + * have to be retrieved, then metricsLimit will be considered as 1 + * i.e. latest single value of metric(s) will be returned. (Optional + * query param). + * @param metricsTimeStart If specified, returned metrics for the container + * would not contain metric values before this timestamp(Optional + * query param). + * @param metricsTimeEnd If specified, returned metrics for the container + * would not contain metric values after this timestamp(Optional + * query param). + * @param entityIdPrefix Defines the id prefix for the entity to be fetched. + * If specified, then entity retrieval will be faster. + * + * @return If successful, a HTTP 200(OK) response having a JSON representing a + * TimelineEntity instance is returned.
+ * On failures,
+ * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned.
+ * If flow context information cannot be retrieved or entity for the + * given entity id cannot be found, HTTP 404(Not Found) is + * returned.
+ * For all other errors while retrieving data, HTTP 500(Internal + * Server Error) is returned. + */ + @GET + @Path("/clusters/{clusterid}/apps/{appid}/containers/{containerid}") + @Produces(MediaType.APPLICATION_JSON) + public TimelineEntity getContainer(@Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("clusterid") String clusterId, + @PathParam("appid") String appId, + @PathParam("containerid") String containerId, + @QueryParam("userid") String userId, + @QueryParam("flowname") String flowName, + @QueryParam("flowrunid") String flowRunId, + @QueryParam("confstoretrieve") String confsToRetrieve, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("metricslimit") String metricsLimit, + @QueryParam("metricstimestart") String metricsTimeStart, + @QueryParam("metricstimeend") String metricsTimeEnd, + @QueryParam("entityidprefix") String entityIdPrefix) { + return getEntity(req, res, clusterId, appId, + TimelineEntityType.YARN_CONTAINER.toString(), containerId, userId, + flowName, flowRunId, confsToRetrieve, metricsToRetrieve, fields, + metricsLimit, metricsTimeStart, metricsTimeEnd, entityIdPrefix); + } + + /** + * Returns a set of available entity types for a given app id. Cluster ID is + * not provided by client so default cluster ID has to be taken. If userid, + * flow name and flow run id which are optional query parameters are not + * specified, they will be queried based on app id and cluster id from the + * flow context information stored in underlying storage implementation. + * + * @param req Servlet request. + * @param res Servlet response. + * @param appId Application id to be queried(Mandatory path param). + * @param flowName Flow name which should match for the app(Optional query + * param). + * @param flowRunId Run id which should match for the app(Optional query + * param). + * @param userId User id which should match for the app(Optional query param). + * + * @return If successful, a HTTP 200(OK) response having a JSON representing a + * list contains all timeline entity types is returned.
+ * On failures,
+ * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned.
+ * If flow context information cannot be retrieved or app for the given + * app id cannot be found, HTTP 404(Not Found) is returned.
+ * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/apps/{appid}/entity-types") + @Produces(MediaType.APPLICATION_JSON) + public Set getEntityTypes( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("appid") String appId, + @QueryParam("flowname") String flowName, + @QueryParam("flowrunid") String flowRunId, + @QueryParam("userid") String userId) { + return getEntityTypes(req, res, null, appId, flowName, flowRunId, userId); + } + + /** + * Returns a set of available entity types for a given app id. If userid, + * flow name and flow run id which are optional query parameters are not + * specified, they will be queried based on app id and cluster id from the + * flow context information stored in underlying storage implementation. + * + * @param req Servlet request. + * @param res Servlet response. + * @param clusterId Cluster id to which the app to be queried belong to( + * Mandatory path param). + * @param appId Application id to be queried(Mandatory path param). + * @param flowName Flow name which should match for the app(Optional query + * param). + * @param flowRunId Run id which should match for the app(Optional query + * param). + * @param userId User id which should match for the app(Optional query param). + * + * @return If successful, a HTTP 200(OK) response having a JSON representing a + * list contains all timeline entity types is returned.
+ * On failures,
+ * If any problem occurs in parsing request, HTTP 400(Bad Request) is + * returned.
+ * If flow context information cannot be retrieved or app for the given + * app id cannot be found, HTTP 404(Not Found) is returned.
+ * For all other errors while retrieving data, HTTP 500(Internal Server + * Error) is returned. + */ + @GET + @Path("/clusters/{clusterid}/apps/{appid}/entity-types") + @Produces(MediaType.APPLICATION_JSON) + public Set getEntityTypes( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("clusterid") String clusterId, + @PathParam("appid") String appId, + @QueryParam("flowname") String flowName, + @QueryParam("flowrunid") String flowRunId, + @QueryParam("userid") String userId) { + String url = req.getRequestURI() + + (req.getQueryString() == null ? "" : + QUERY_STRING_SEP + req.getQueryString()); + UserGroupInformation callerUGI = + TimelineReaderWebServicesUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderWebServicesUtils.getUserName(callerUGI)); + long startTime = Time.monotonicNow(); + init(res); + TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); + Set results = null; + try { + results = timelineReaderManager.getEntityTypes( + TimelineReaderWebServicesUtils.createTimelineReaderContext( + clusterId, userId, flowName, flowRunId, appId, + null, null, null)); + } catch (Exception e) { + handleException(e, url, startTime, "flowrunid"); + } + long endTime = Time.monotonicNow(); + LOG.info("Processed URL " + url + + " (Took " + (endTime - startTime) + " ms.)"); + return results; + } + + @GET + @Path("/users/{userid}/entities/{entitytype}") + @Produces(MediaType.APPLICATION_JSON) + public Set getSubAppEntities( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("userid") String userId, + @PathParam("entitytype") String entityType, + @QueryParam("limit") String limit, + @QueryParam("createdtimestart") String createdTimeStart, + @QueryParam("createdtimeend") String createdTimeEnd, + @QueryParam("relatesto") String relatesTo, + @QueryParam("isrelatedto") String isRelatedTo, + @QueryParam("infofilters") String infofilters, + @QueryParam("conffilters") String conffilters, + @QueryParam("metricfilters") String metricfilters, + @QueryParam("eventfilters") String eventfilters, + @QueryParam("confstoretrieve") String confsToRetrieve, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("metricslimit") String metricsLimit, + @QueryParam("metricstimestart") String metricsTimeStart, + @QueryParam("metricstimeend") String metricsTimeEnd, + @QueryParam("fromid") String fromId) { + return getSubAppEntities(req, res, null, userId, entityType, limit, + createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo, infofilters, + conffilters, metricfilters, eventfilters, confsToRetrieve, + metricsToRetrieve, fields, metricsLimit, metricsTimeStart, + metricsTimeEnd, fromId); + } + + @GET + @Path("/clusters/{clusterid}/users/{userid}/entities/{entitytype}") + @Produces(MediaType.APPLICATION_JSON) + public Set getSubAppEntities( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("clusterid") String clusterId, + @PathParam("userid") String userId, + @PathParam("entitytype") String entityType, + @QueryParam("limit") String limit, + @QueryParam("createdtimestart") String createdTimeStart, + @QueryParam("createdtimeend") String createdTimeEnd, + @QueryParam("relatesto") String relatesTo, + @QueryParam("isrelatedto") String isRelatedTo, + @QueryParam("infofilters") String infofilters, + @QueryParam("conffilters") String conffilters, + @QueryParam("metricfilters") String metricfilters, + @QueryParam("eventfilters") String eventfilters, + @QueryParam("confstoretrieve") String confsToRetrieve, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("metricslimit") String metricsLimit, + @QueryParam("metricstimestart") String metricsTimeStart, + @QueryParam("metricstimeend") String metricsTimeEnd, + @QueryParam("fromid") String fromId) { + String url = req.getRequestURI() + + (req.getQueryString() == null ? "" : + QUERY_STRING_SEP + req.getQueryString()); + UserGroupInformation callerUGI = + TimelineReaderWebServicesUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderWebServicesUtils.getUserName(callerUGI)); + long startTime = Time.monotonicNow(); + init(res); + TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); + Set entities = null; + try { + TimelineReaderContext context = + TimelineReaderWebServicesUtils.createTimelineReaderContext(clusterId, + null, null, null, null, entityType, null, null, userId); + entities = timelineReaderManager.getEntities(context, + TimelineReaderWebServicesUtils.createTimelineEntityFilters( + limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo, + infofilters, conffilters, metricfilters, eventfilters, + fromId), + TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( + confsToRetrieve, metricsToRetrieve, fields, metricsLimit, + metricsTimeStart, metricsTimeEnd)); + } catch (Exception e) { + handleException(e, url, startTime, + "createdTime start/end or limit"); + } + long endTime = Time.monotonicNow(); + if (entities == null) { + entities = Collections.emptySet(); + } + LOG.info("Processed URL " + url + + " (Took " + (endTime - startTime) + " ms.)"); + return entities; + } + + @GET + @Path("/users/{userid}/entities/{entitytype}/{entityid}") + @Produces(MediaType.APPLICATION_JSON) + public Set getSubAppEntities(@Context HttpServletRequest req, + @Context HttpServletResponse res, @PathParam("userid") String userId, + @PathParam("entitytype") String entityType, + @PathParam("entityid") String entityId, + @QueryParam("confstoretrieve") String confsToRetrieve, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("metricslimit") String metricsLimit, + @QueryParam("metricstimestart") String metricsTimeStart, + @QueryParam("metricstimeend") String metricsTimeEnd, + @QueryParam("entityidprefix") String entityIdPrefix) { + return getSubAppEntities(req, res, null, userId, entityType, entityId, + confsToRetrieve, metricsToRetrieve, fields, metricsLimit, + metricsTimeStart, metricsTimeEnd, entityIdPrefix); + } + + @GET + @Path("/clusters/{clusterid}/users/{userid}/entities/{entitytype}/{entityid}") + @Produces(MediaType.APPLICATION_JSON) + public Set getSubAppEntities(@Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("clusterid") String clusterId, + @PathParam("userid") String userId, + @PathParam("entitytype") String entityType, + @PathParam("entityid") String entityId, + @QueryParam("confstoretrieve") String confsToRetrieve, + @QueryParam("metricstoretrieve") String metricsToRetrieve, + @QueryParam("fields") String fields, + @QueryParam("metricslimit") String metricsLimit, + @QueryParam("metricstimestart") String metricsTimeStart, + @QueryParam("metricstimeend") String metricsTimeEnd, + @QueryParam("entityidprefix") String entityIdPrefix) { + String url = req.getRequestURI() + (req.getQueryString() == null ? "" + : QUERY_STRING_SEP + req.getQueryString()); + UserGroupInformation callerUGI = + TimelineReaderWebServicesUtils.getUser(req); + LOG.info("Received URL " + url + " from user " + + TimelineReaderWebServicesUtils.getUserName(callerUGI)); + long startTime = Time.monotonicNow(); + init(res); + TimelineReaderManager timelineReaderManager = getTimelineReaderManager(); + Set entities = null; + try { + TimelineReaderContext context = TimelineReaderWebServicesUtils + .createTimelineReaderContext(clusterId, null, null, null, null, + entityType, entityIdPrefix, entityId, userId); + entities = timelineReaderManager.getEntities(context, + new TimelineEntityFilters.Builder().build(), + TimelineReaderWebServicesUtils.createTimelineDataToRetrieve( + confsToRetrieve, metricsToRetrieve, fields, metricsLimit, + metricsTimeStart, metricsTimeEnd)); + } catch (Exception e) { + handleException(e, url, startTime, ""); + } + long endTime = Time.monotonicNow(); + if (entities == null) { + entities = Collections.emptySet(); + } + LOG.info( + "Processed URL " + url + " (Took " + (endTime - startTime) + " ms.)"); + return entities; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java new file mode 100644 index 00000000000..f83c1ac346f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java @@ -0,0 +1,315 @@ +/** + * 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.timelineservice.reader; + +import java.security.Principal; +import java.util.EnumSet; + +import javax.servlet.http.HttpServletRequest; + +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; + +/** + * Set of utility methods to be used by timeline reader web services. + */ +public final class TimelineReaderWebServicesUtils { + + private TimelineReaderWebServicesUtils() { + } + + /** + * Parse the passed context information represented as strings and convert + * into a {@link TimelineReaderContext} object. + * @param clusterId Cluster Id. + * @param userId User Id. + * @param flowName Flow Name. + * @param flowRunId Run id for the flow. + * @param appId App Id. + * @param entityType Entity Type. + * @param entityId Entity Id. + * @return a {@link TimelineReaderContext} object. + */ + static TimelineReaderContext createTimelineReaderContext(String clusterId, + String userId, String flowName, String flowRunId, String appId, + String entityType, String entityIdPrefix, String entityId) { + return new TimelineReaderContext(parseStr(clusterId), parseStr(userId), + parseStr(flowName), parseLongStr(flowRunId), parseStr(appId), + parseStr(entityType), parseLongStr(entityIdPrefix), parseStr(entityId)); + } + + static TimelineReaderContext createTimelineReaderContext(String clusterId, + String userId, String flowName, String flowRunId, String appId, + String entityType, String entityIdPrefix, String entityId, + String doAsUser) { + return new TimelineReaderContext(parseStr(clusterId), parseStr(userId), + parseStr(flowName), parseLongStr(flowRunId), parseStr(appId), + parseStr(entityType), parseLongStr(entityIdPrefix), parseStr(entityId), + parseStr(doAsUser)); + } + + /** + * Parse the passed filters represented as strings and convert them into a + * {@link TimelineEntityFilters} object. + * @param limit Limit to number of entities to return. + * @param createdTimeStart Created time start for the entities to return. + * @param createdTimeEnd Created time end for the entities to return. + * @param relatesTo Entities to return must match relatesTo. + * @param isRelatedTo Entities to return must match isRelatedTo. + * @param infofilters Entities to return must match these info filters. + * @param conffilters Entities to return must match these metric filters. + * @param metricfilters Entities to return must match these metric filters. + * @param eventfilters Entities to return must match these event filters. + * @return a {@link TimelineEntityFilters} object. + * @throws TimelineParseException if any problem occurs during parsing. + */ + static TimelineEntityFilters createTimelineEntityFilters(String limit, + String createdTimeStart, String createdTimeEnd, String relatesTo, + String isRelatedTo, String infofilters, String conffilters, + String metricfilters, String eventfilters, + String fromid) throws TimelineParseException { + return createTimelineEntityFilters( + limit, parseLongStr(createdTimeStart), + parseLongStr(createdTimeEnd), + relatesTo, isRelatedTo, infofilters, + conffilters, metricfilters, eventfilters, fromid); + } + + /** + * Parse the passed filters represented as strings and convert them into a + * {@link TimelineEntityFilters} object. + * @param limit Limit to number of entities to return. + * @param createdTimeStart Created time start for the entities to return. + * @param createdTimeEnd Created time end for the entities to return. + * @param relatesTo Entities to return must match relatesTo. + * @param isRelatedTo Entities to return must match isRelatedTo. + * @param infofilters Entities to return must match these info filters. + * @param conffilters Entities to return must match these metric filters. + * @param metricfilters Entities to return must match these metric filters. + * @param eventfilters Entities to return must match these event filters. + * @return a {@link TimelineEntityFilters} object. + * @throws TimelineParseException if any problem occurs during parsing. + */ + static TimelineEntityFilters createTimelineEntityFilters(String limit, + Long createdTimeStart, Long createdTimeEnd, String relatesTo, + String isRelatedTo, String infofilters, String conffilters, + String metricfilters, String eventfilters, + String fromid) throws TimelineParseException { + return new TimelineEntityFilters.Builder() + .entityLimit(parseLongStr(limit)) + .createdTimeBegin(createdTimeStart) + .createTimeEnd(createdTimeEnd) + .relatesTo(parseRelationFilters(relatesTo)) + .isRelatedTo(parseRelationFilters(isRelatedTo)) + .infoFilters(parseKVFilters(infofilters, false)) + .configFilters(parseKVFilters(conffilters, true)) + .metricFilters(parseMetricFilters(metricfilters)) + .eventFilters(parseEventFilters(eventfilters)) + .fromId(parseStr(fromid)).build(); + } + + /** + * Parse the passed fields represented as strings and convert them into a + * {@link TimelineDataToRetrieve} object. + * @param confs confs to retrieve. + * @param metrics metrics to retrieve. + * @param fields fields to retrieve. + * @param metricsLimit upper limit on number of metrics to return. + * @return a {@link TimelineDataToRetrieve} object. + * @throws TimelineParseException if any problem occurs during parsing. + */ + static TimelineDataToRetrieve createTimelineDataToRetrieve(String confs, + String metrics, String fields, String metricsLimit, + String metricsTimeBegin, String metricsTimeEnd) + throws TimelineParseException { + return new TimelineDataToRetrieve(parseDataToRetrieve(confs), + parseDataToRetrieve(metrics), parseFieldsStr(fields, + TimelineParseConstants.COMMA_DELIMITER), parseIntStr(metricsLimit), + parseLongStr(metricsTimeBegin), parseLongStr(metricsTimeEnd)); + } + + /** + * Parse a delimited string and convert it into a set of strings. For + * instance, if delimiter is ",", then the string should be represented as + * "value1,value2,value3". + * @param str delimited string. + * @param delimiter string is delimited by this delimiter. + * @return set of strings. + */ + static TimelineFilterList parseEventFilters(String expr) + throws TimelineParseException { + return parseFilters(new TimelineParserForExistFilters(expr, + TimelineParseConstants.COMMA_CHAR)); + } + + /** + * Parse relation filters. + * @param expr Relation filter expression + * @return a {@link TimelineFilterList} object. + * + * @throws Exception if any problem occurs. + */ + static TimelineFilterList parseRelationFilters(String expr) + throws TimelineParseException { + return parseFilters(new TimelineParserForRelationFilters(expr, + TimelineParseConstants.COMMA_CHAR, + TimelineParseConstants.COLON_DELIMITER)); + } + + private static TimelineFilterList parseFilters(TimelineParser parser) + throws TimelineParseException { + try { + return parser.parse(); + } finally { + IOUtils.closeQuietly(parser); + } + } + + /** + * Parses config and info filters. + * + * @param expr Expression to be parsed. + * @param valueAsString true, if value has to be interpreted as string, false + * otherwise. It is true for config filters and false for info filters. + * @return a {@link TimelineFilterList} object. + * @throws TimelineParseException if any problem occurs during parsing. + */ + static TimelineFilterList parseKVFilters(String expr, boolean valueAsString) + throws TimelineParseException { + return parseFilters(new TimelineParserForKVFilters(expr, valueAsString)); + } + + /** + * Interprets passed string as set of fields delimited by passed delimiter. + * For instance, if delimiter is ",", then the passed string should be + * represented as "METRICS,CONFIGS" where the delimited parts of the string + * present in {@link Field}. + * @param str passed string. + * @param delimiter string delimiter. + * @return a set of {@link Field}. + */ + static EnumSet parseFieldsStr(String str, String delimiter) { + if (str == null) { + return null; + } + String[] strs = str.split(delimiter); + EnumSet fieldList = EnumSet.noneOf(Field.class); + for (String s : strs) { + fieldList.add(Field.valueOf(s.trim().toUpperCase())); + } + return fieldList; + } + + /** + * Parses metric filters. + * + * @param expr Metric filter expression to be parsed. + * @return a {@link TimelineFilterList} object. + * @throws TimelineParseException if any problem occurs during parsing. + */ + static TimelineFilterList parseMetricFilters(String expr) + throws TimelineParseException { + return parseFilters(new TimelineParserForNumericFilters(expr)); + } + + /** + * Interpret passed string as a long. + * @param str Passed string. + * @return long representation if string is not null, null otherwise. + */ + static Long parseLongStr(String str) { + return str == null ? null : Long.parseLong(str.trim()); + } + + /** + * Interpret passed string as a integer. + * @param str Passed string. + * @return integer representation if string is not null, null otherwise. + */ + static Integer parseIntStr(String str) { + return str == null ? null : Integer.parseInt(str.trim()); + } + + /** + * Trims the passed string if its not null. + * @param str Passed string. + * @return trimmed string if string is not null, null otherwise. + */ + static String parseStr(String str) { + return StringUtils.trimToNull(str); + } + + /** + * Get UGI based on the remote user in the HTTP request. + * + * @param req HTTP request. + * @return UGI. + */ + public static UserGroupInformation getUser(HttpServletRequest req) { + return getCallerUserGroupInformation(req, false); + } + + /** + * Get UGI from the HTTP request. + * + * @param hsr HTTP request. + * @param usePrincipal if true, use principal name else use remote user name + * @return UGI. + */ + public static UserGroupInformation getCallerUserGroupInformation( + HttpServletRequest hsr, boolean usePrincipal) { + + String remoteUser = hsr.getRemoteUser(); + if (usePrincipal) { + Principal princ = hsr.getUserPrincipal(); + remoteUser = princ == null ? null : princ.getName(); + } + + UserGroupInformation callerUGI = null; + if (remoteUser != null) { + callerUGI = UserGroupInformation.createRemoteUser(remoteUser); + } + + return callerUGI; + } + + /** + * Get username from caller UGI. + * @param callerUGI caller UGI. + * @return username. + */ + static String getUserName(UserGroupInformation callerUGI) { + return ((callerUGI != null) ? callerUGI.getUserName().trim() : ""); + } + + /** + * Parses confstoretrieve and metricstoretrieve. + * @param str String representing confs/metrics to retrieve expression. + * + * @return a {@link TimelineFilterList} object. + * @throws TimelineParseException if any problem occurs during parsing. + */ + static TimelineFilterList parseDataToRetrieve(String expr) + throws TimelineParseException { + return parseFilters(new TimelineParserForDataToRetrieve(expr)); + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineUIDConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineUIDConverter.java new file mode 100644 index 00000000000..b875828719b --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineUIDConverter.java @@ -0,0 +1,275 @@ +/** + * 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.timelineservice.reader; + +import java.util.List; + +/** + * Used for encoding/decoding UID which will be used for query by UI. + */ +enum TimelineUIDConverter { + // Flow UID should contain cluster, user and flow name. + FLOW_UID { + @Override + String encodeUID(TimelineReaderContext context) { + if (context == null) { + return null; + } + if (context.getClusterId() == null || context.getUserId() == null || + context.getFlowName() == null) { + return null; + } + String[] flowNameTupleArr = {context.getClusterId(), context.getUserId(), + context.getFlowName()}; + return joinAndEscapeUIDParts(flowNameTupleArr); + } + + @Override + TimelineReaderContext decodeUID(String uId) throws Exception { + if (uId == null) { + return null; + } + List flowNameTupleList = splitUID(uId); + // Should have 3 parts i.e. cluster, user and flow name. + if (flowNameTupleList.size() != 3) { + return null; + } + return new TimelineReaderContext(flowNameTupleList.get(0), + flowNameTupleList.get(1), flowNameTupleList.get(2), null, + null, null, null); + } + }, + + // Flowrun UID should contain cluster, user, flow name and flowrun id. + FLOWRUN_UID{ + @Override + String encodeUID(TimelineReaderContext context) { + if (context == null) { + return null; + } + if (context.getClusterId() == null || context.getUserId() == null || + context.getFlowName() == null || context.getFlowRunId() == null) { + return null; + } + String[] flowRunTupleArr = {context.getClusterId(), context.getUserId(), + context.getFlowName(), context.getFlowRunId().toString()}; + return joinAndEscapeUIDParts(flowRunTupleArr); + } + + @Override + TimelineReaderContext decodeUID(String uId) throws Exception { + if (uId == null) { + return null; + } + List flowRunTupleList = splitUID(uId); + // Should have 4 parts i.e. cluster, user, flow name and flowrun id. + if (flowRunTupleList.size() != 4) { + return null; + } + return new TimelineReaderContext(flowRunTupleList.get(0), + flowRunTupleList.get(1), flowRunTupleList.get(2), + Long.parseLong(flowRunTupleList.get(3)), null, null, null); + } + }, + + // Application UID should contain cluster, user, flow name, flowrun id + // and app id OR cluster and app id(i.e.without flow context info). + APPLICATION_UID{ + @Override + String encodeUID(TimelineReaderContext context) { + if (context == null) { + return null; + } + if (context.getClusterId() == null || context.getAppId() == null) { + return null; + } + if (context.getUserId() != null && context.getFlowName() != null && + context.getFlowRunId() != null) { + // Flow information exists. + String[] appTupleArr = {context.getClusterId(), context.getUserId(), + context.getFlowName(), context.getFlowRunId().toString(), + context.getAppId()}; + return joinAndEscapeUIDParts(appTupleArr); + } else { + // Only cluster and app information exists. Flow info does not exist. + String[] appTupleArr = {context.getClusterId(), context.getAppId()}; + return joinAndEscapeUIDParts(appTupleArr); + } + } + + @Override + TimelineReaderContext decodeUID(String uId) throws Exception { + if (uId == null) { + return null; + } + List appTupleList = splitUID(uId); + // Should have 5 parts i.e. cluster, user, flow name, flowrun id + // and app id OR should have 2 parts i.e. cluster and app id. + if (appTupleList.size() == 5) { + // Flow information exists. + return new TimelineReaderContext(appTupleList.get(0), + appTupleList.get(1), appTupleList.get(2), + Long.parseLong(appTupleList.get(3)), appTupleList.get(4), + null, null); + } else if (appTupleList.size() == 2) { + // Flow information does not exist. + return new TimelineReaderContext(appTupleList.get(0), null, null, null, + appTupleList.get(1), null, null); + } else { + return null; + } + } + }, + + // Sub Application Entity UID should contain cluster, user, entity type and + // entity id + SUB_APPLICATION_ENTITY_UID { + @Override + String encodeUID(TimelineReaderContext context) { + if (context == null) { + return null; + } + if (context.getClusterId() == null || context.getDoAsUser() == null + || context.getEntityType() == null || context.getEntityId() == null) { + return null; + } + String[] entityTupleArr = {context.getClusterId(), context.getDoAsUser(), + context.getEntityType(), context.getEntityIdPrefix().toString(), + context.getEntityId()}; + return joinAndEscapeUIDParts(entityTupleArr); + } + + @Override + TimelineReaderContext decodeUID(String uId) throws Exception { + if (uId == null) { + return null; + } + List entityTupleList = splitUID(uId); + if (entityTupleList.size() == 5) { + // Flow information exists. + return new TimelineReaderContext(entityTupleList.get(0), null, null, + null, null, entityTupleList.get(2), + Long.parseLong(entityTupleList.get(3)), entityTupleList.get(4), + entityTupleList.get(1)); + } + return null; + } + }, + + // Generic Entity UID should contain cluster, user, flow name, flowrun id, + // app id, entity type and entity id OR should contain cluster, appid, entity + // type and entity id(i.e.without flow context info). + GENERIC_ENTITY_UID { + @Override + String encodeUID(TimelineReaderContext context) { + if (context == null) { + return null; + } + if (context.getClusterId() == null || context.getAppId() == null || + context.getEntityType() == null || context.getEntityId() == null) { + return null; + } + if (context.getUserId() != null && context.getFlowName() != null && + context.getFlowRunId() != null) { + // Flow information exists. + String[] entityTupleArr = {context.getClusterId(), context.getUserId(), + context.getFlowName(), context.getFlowRunId().toString(), + context.getAppId(), context.getEntityType(), + context.getEntityIdPrefix().toString(), context.getEntityId() }; + return joinAndEscapeUIDParts(entityTupleArr); + } else { + // Only entity and app information exists. Flow info does not exist. + String[] entityTupleArr = {context.getClusterId(), context.getAppId(), + context.getEntityType(), context.getEntityIdPrefix().toString(), + context.getEntityId() }; + return joinAndEscapeUIDParts(entityTupleArr); + } + } + + @Override + TimelineReaderContext decodeUID(String uId) throws Exception { + if (uId == null) { + return null; + } + List entityTupleList = splitUID(uId); + // Should have 8 parts i.e. cluster, user, flow name, flowrun id, app id, + // entity type and entity id OR should have 5 parts i.e. cluster, app id, + // entity type and entity id. + if (entityTupleList.size() == 8) { + // Flow information exists. + return new TimelineReaderContext(entityTupleList.get(0), + entityTupleList.get(1), entityTupleList.get(2), + Long.parseLong(entityTupleList.get(3)), entityTupleList.get(4), + entityTupleList.get(5), Long.parseLong(entityTupleList.get(6)), + entityTupleList.get(7)); + } else if (entityTupleList.size() == 5) { + // Flow information does not exist. + return new TimelineReaderContext(entityTupleList.get(0), null, null, + null, entityTupleList.get(1), entityTupleList.get(2), + Long.parseLong(entityTupleList.get(3)), entityTupleList.get(4)); + } else { + return null; + } + } + }; + + /** + * Split UID using {@link TimelineReaderUtils#DEFAULT_DELIMITER_CHAR} and + * {@link TimelineReaderUtils#DEFAULT_ESCAPE_CHAR}. + * @param uid UID to be splitted. + * @return a list of different parts of UID split across delimiter. + * @throws IllegalArgumentException if UID is not properly escaped. + */ + private static List splitUID(String uid) + throws IllegalArgumentException { + return TimelineReaderUtils.split(uid); + } + + /** + * Join different parts of UID delimited by + * {@link TimelineReaderUtils#DEFAULT_DELIMITER_CHAR} with delimiter and + * escape character escaped using + * {@link TimelineReaderUtils#DEFAULT_ESCAPE_CHAR} if UID parts contain them. + * @param parts an array of UID parts to be joined. + * @return a string joined using the delimiter with escape and delimiter + * characters escaped if they are part of the string parts to be + * joined. Returns null if one of the parts is null. + */ + private static String joinAndEscapeUIDParts(String[] parts) { + return TimelineReaderUtils.joinAndEscapeStrings(parts); + } + + /** + * Encodes UID depending on UID implementation. + * + * @param context Reader context. + * @return UID represented as a string. + */ + abstract String encodeUID(TimelineReaderContext context); + + /** + * Decodes UID depending on UID implementation. + * + * @param uId UID to be decoded. + * @return a {@link TimelineReaderContext} object if UID passed can be + * decoded, null otherwise. + * @throws Exception if any problem occurs while decoding. + */ + abstract TimelineReaderContext decodeUID(String uId) throws Exception; +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineCompareFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineCompareFilter.java new file mode 100644 index 00000000000..1127f4a7f26 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineCompareFilter.java @@ -0,0 +1,147 @@ +/** + * 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.timelineservice.reader.filter; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; + +/** + * Filter class which represents filter to be applied based on key-value pair + * and the relation between them represented by different relational operators. + */ +@Private +@Unstable +public class TimelineCompareFilter extends TimelineFilter { + + private TimelineCompareOp compareOp; + private String key; + private Object value; + // If comparison operator is NOT_EQUAL, this flag decides if we should return + // the entity if key does not exist. + private boolean keyMustExist = true; + + public TimelineCompareFilter() { + } + + public TimelineCompareFilter(TimelineCompareOp op, String key, Object val, + boolean keyMustExistFlag) { + this.compareOp = op; + this.key = key; + this.value = val; + if (op == TimelineCompareOp.NOT_EQUAL) { + this.keyMustExist = keyMustExistFlag; + } else { + this.keyMustExist = true; + } + } + + public TimelineCompareFilter(TimelineCompareOp op, String key, Object val) { + this(op, key, val, true); + } + + @Override + public TimelineFilterType getFilterType() { + return TimelineFilterType.COMPARE; + } + + public TimelineCompareOp getCompareOp() { + return compareOp; + } + + public String getKey() { + return key; + } + + public void setKey(String keyToBeSet) { + key = keyToBeSet; + } + + public Object getValue() { + return value; + } + + public void setCompareOp(TimelineCompareOp timelineCompareOp, + boolean keyExistFlag) { + this.compareOp = timelineCompareOp; + if (timelineCompareOp == TimelineCompareOp.NOT_EQUAL) { + this.keyMustExist = keyExistFlag; + } + } + + public void setValue(Object val) { + value = val; + } + + public boolean getKeyMustExist() { + return keyMustExist; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((compareOp == null) ? 0 : compareOp.hashCode()); + result = prime * result + ((key == null) ? 0 : key.hashCode()); + result = prime * result + (keyMustExist ? 1231 : 1237); + result = prime * result + ((value == null) ? 0 : value.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + TimelineCompareFilter other = (TimelineCompareFilter) obj; + if (compareOp != other.compareOp) { + return false; + } + if (key == null) { + if (other.key != null) { + return false; + } + } else if (!key.equals(other.key)) { + return false; + } + if (keyMustExist != other.keyMustExist) { + return false; + } + if (value == null) { + if (other.value != null) { + return false; + } + } else if (!value.equals(other.value)) { + return false; + } + return true; + } + + @Override + public String toString() { + return String.format("%s (%s, %s:%s:%b)", + this.getClass().getSimpleName(), this.compareOp.name(), + this.key, this.value, this.keyMustExist); + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineCompareOp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineCompareOp.java new file mode 100644 index 00000000000..461a7d8571e --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineCompareOp.java @@ -0,0 +1,36 @@ +/** + * 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.timelineservice.reader.filter; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; + +/** + * Comparison Operators. + */ +@Private +@Unstable +public enum TimelineCompareOp { + LESS_THAN, + LESS_OR_EQUAL, + EQUAL, + NOT_EQUAL, + GREATER_OR_EQUAL, + GREATER_THAN +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineExistsFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineExistsFilter.java new file mode 100644 index 00000000000..b4c8e25d92b --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineExistsFilter.java @@ -0,0 +1,107 @@ +/** + * 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.timelineservice.reader.filter; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; + +/** + * Filter class which represents filter to be applied based on existence of a + * value. + */ +@Private +@Unstable +public class TimelineExistsFilter extends TimelineFilter { + + private TimelineCompareOp compareOp; + private String value; + + public TimelineExistsFilter() { + } + + public TimelineExistsFilter(TimelineCompareOp op, String value) { + this.value = value; + if (op != TimelineCompareOp.EQUAL && op != TimelineCompareOp.NOT_EQUAL) { + throw new IllegalArgumentException("CompareOp for exists filter should " + + "be EQUAL or NOT_EQUAL"); + } + this.compareOp = op; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((compareOp == null) ? 0 : compareOp.hashCode()); + result = prime * result + ((value == null) ? 0 : value.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + TimelineExistsFilter other = (TimelineExistsFilter) obj; + if (compareOp != other.compareOp) { + return false; + } + if (value == null) { + if (other.value != null) { + return false; + } + } else if (!value.equals(other.value)) { + return false; + } + return true; + } + + @Override + public TimelineFilterType getFilterType() { + return TimelineFilterType.EXISTS; + } + + public void setValue(String val) { + value = val; + } + + public String getValue() { + return value; + } + + public void setCompareOp(TimelineCompareOp op) { + compareOp = op; + } + + public TimelineCompareOp getCompareOp() { + return compareOp; + } + + @Override + public String toString() { + return String.format("%s (%s %s)", + this.getClass().getSimpleName(), this.compareOp.name(), this.value); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilter.java new file mode 100644 index 00000000000..5e84976437a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilter.java @@ -0,0 +1,68 @@ +/** + * 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.timelineservice.reader.filter; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; + +/** + * Abstract base class extended to implement timeline filters. + */ +@Private +@Unstable +public abstract class TimelineFilter { + + /** + * Lists the different filter types. + */ + @Private + @Unstable + public enum TimelineFilterType { + /** + * Combines multiple filters. + */ + LIST, + /** + * Filter which is used for key-value comparison. + */ + COMPARE, + /** + * Filter which is used for checking key-value equality. + */ + KEY_VALUE, + /** + * Filter which is used for checking key-multiple values equality. + */ + KEY_VALUES, + /** + * Filter which matches prefix for a config or a metric. + */ + PREFIX, + /** + * Filter which checks existence of a value. + */ + EXISTS + } + + public abstract TimelineFilterType getFilterType(); + + public String toString() { + return this.getClass().getSimpleName(); + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilterList.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilterList.java new file mode 100644 index 00000000000..b4c7ad24dba --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilterList.java @@ -0,0 +1,141 @@ +/** + * 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.timelineservice.reader.filter; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; + +/** + * Implementation of {@link TimelineFilter} that represents an ordered list of + * timeline filters which will then be evaluated with a specified boolean + * operator {@link Operator#AND} or {@link Operator#OR}. Since you can use + * timeline filter lists as children of timeline filter lists, you can create a + * hierarchy of filters to be evaluated. + */ +@Private +@Unstable +public class TimelineFilterList extends TimelineFilter { + /** + * Specifies how filters in the filter list will be evaluated. AND means all + * the filters should match and OR means atleast one should match. + */ + @Private + @Unstable + public static enum Operator { + AND, + OR + } + + private Operator operator; + private List filterList = new ArrayList(); + + public TimelineFilterList(TimelineFilter...filters) { + this(Operator.AND, filters); + } + + public TimelineFilterList() { + this(Operator.AND); + } + + public TimelineFilterList(Operator op) { + this.operator = op; + } + + public TimelineFilterList(Operator op, TimelineFilter...filters) { + this.operator = op; + this.filterList = new ArrayList(Arrays.asList(filters)); + } + + @Override + public TimelineFilterType getFilterType() { + return TimelineFilterType.LIST; + } + + /** + * Get the filter list. + * + * @return filterList + */ + public List getFilterList() { + return filterList; + } + + /** + * Get the operator. + * + * @return operator + */ + public Operator getOperator() { + return operator; + } + + public void setOperator(Operator op) { + operator = op; + } + + public void addFilter(TimelineFilter filter) { + filterList.add(filter); + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = + prime * result + ((filterList == null) ? 0 : filterList.hashCode()); + result = + prime * result + ((operator == null) ? 0 : operator.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + TimelineFilterList other = (TimelineFilterList) obj; + if (operator != other.operator) { + return false; + } + if (filterList == null) { + if (other.filterList != null) { + return false; + } + } else if (!filterList.equals(other.filterList)) { + return false; + } + return true; + } + + @Override + public String toString() { + return String.format("TimelineFilterList %s (%d): %s", + this.operator, this.filterList.size(), this.filterList.toString()); + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValueFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValueFilter.java new file mode 100644 index 00000000000..8bc8584dced --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValueFilter.java @@ -0,0 +1,61 @@ +/** + * 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.timelineservice.reader.filter; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; + +/** + * Filter class which represents filter to be applied based on key-value pair + * being equal or not to the values in back-end store. + */ +@Private +@Unstable +public class TimelineKeyValueFilter extends TimelineCompareFilter { + public TimelineKeyValueFilter() { + } + + public TimelineKeyValueFilter(TimelineCompareOp op, String key, Object val, + boolean keyMustExistFlag) { + super(op, key, val, keyMustExistFlag); + if (op != TimelineCompareOp.EQUAL && op != TimelineCompareOp.NOT_EQUAL) { + throw new IllegalArgumentException("TimelineCompareOp for equality" + + " filter should be EQUAL or NOT_EQUAL"); + } + } + + public TimelineKeyValueFilter(TimelineCompareOp op, String key, Object val) { + this(op, key, val, true); + } + + @Override + public TimelineFilterType getFilterType() { + return TimelineFilterType.KEY_VALUE; + } + + public void setCompareOp(TimelineCompareOp timelineCompareOp, + boolean keyExistFlag) { + if (timelineCompareOp != TimelineCompareOp.EQUAL && + timelineCompareOp != TimelineCompareOp.NOT_EQUAL) { + throw new IllegalArgumentException("TimelineCompareOp for equality" + + " filter should be EQUAL or NOT_EQUAL"); + } + super.setCompareOp(timelineCompareOp, keyExistFlag); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValuesFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValuesFilter.java new file mode 100644 index 00000000000..fe4f6b2455e --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValuesFilter.java @@ -0,0 +1,126 @@ +/** + * 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.timelineservice.reader.filter; + +import java.util.Set; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; + +/** + * Filter class which represents filter to be applied based on multiple values + * for a key and these values being equal or not equal to values in back-end + * store. + */ +@Private +@Unstable +public class TimelineKeyValuesFilter extends TimelineFilter { + private TimelineCompareOp compareOp; + private String key; + private Set values; + + public TimelineKeyValuesFilter() { + } + + public TimelineKeyValuesFilter(TimelineCompareOp op, String key, + Set values) { + if (op != TimelineCompareOp.EQUAL && op != TimelineCompareOp.NOT_EQUAL) { + throw new IllegalArgumentException("TimelineCompareOp for multi value " + + "equality filter should be EQUAL or NOT_EQUAL"); + } + this.compareOp = op; + this.key = key; + this.values = values; + } + + @Override + public TimelineFilterType getFilterType() { + return TimelineFilterType.KEY_VALUES; + } + + public String getKey() { + return key; + } + + public Set getValues() { + return values; + } + + public void setKeyAndValues(String keyForValues, Set vals) { + key = keyForValues; + values = vals; + } + + public void setCompareOp(TimelineCompareOp op) { + compareOp = op; + } + + public TimelineCompareOp getCompareOp() { + return compareOp; + } + + @Override + public String toString() { + return String.format("%s (%s, %s:%s)", + this.getClass().getSimpleName(), this.compareOp.name(), + this.key, (values == null) ? "" : values.toString()); + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((compareOp == null) ? 0 : compareOp.hashCode()); + result = prime * result + ((key == null) ? 0 : key.hashCode()); + result = prime * result + ((values == null) ? 0 : values.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + TimelineKeyValuesFilter other = (TimelineKeyValuesFilter) obj; + if (compareOp != other.compareOp) { + return false; + } + if (key == null) { + if (other.key != null) { + return false; + } + } else if (!key.equals(other.key)) { + return false; + } + if (values == null) { + if (other.values != null) { + return false; + } + } else if (!values.equals(other.values)) { + return false; + } + return true; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelinePrefixFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelinePrefixFilter.java new file mode 100644 index 00000000000..bbdc96010e6 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelinePrefixFilter.java @@ -0,0 +1,99 @@ +/** + * 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.timelineservice.reader.filter; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; + +/** + * Filter class which represents filter to be applied based on prefixes. + * Prefixes can either match or not match. + */ +@Private +@Unstable +public class TimelinePrefixFilter extends TimelineFilter { + + private TimelineCompareOp compareOp; + private String prefix; + + public TimelinePrefixFilter() { + } + + public TimelinePrefixFilter(TimelineCompareOp op, String prefix) { + this.prefix = prefix; + if (op != TimelineCompareOp.EQUAL && op != TimelineCompareOp.NOT_EQUAL) { + throw new IllegalArgumentException("CompareOp for prefix filter should " + + "be EQUAL or NOT_EQUAL"); + } + this.compareOp = op; + } + + @Override + public TimelineFilterType getFilterType() { + return TimelineFilterType.PREFIX; + } + + public String getPrefix() { + return prefix; + } + + public TimelineCompareOp getCompareOp() { + return compareOp; + } + + @Override + public String toString() { + return String.format("%s (%s %s)", + this.getClass().getSimpleName(), this.compareOp.name(), this.prefix); + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((compareOp == null) ? 0 : compareOp.hashCode()); + result = prime * result + ((prefix == null) ? 0 : prefix.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + TimelinePrefixFilter other = (TimelinePrefixFilter) obj; + if (compareOp != other.compareOp) { + return false; + } + if (prefix == null) { + if (other.prefix != null) { + return false; + } + } else if (!prefix.equals(other.prefix)){ + return false; + } + return true; + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/package-info.java new file mode 100644 index 00000000000..f7c07057d7c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/package-info.java @@ -0,0 +1,28 @@ +/* + * 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.server.timelineservice.reader.filter stores + * timeline filter implementations. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.timelineservice.reader.filter; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/package-info.java new file mode 100644 index 00000000000..116509a08bc --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/package-info.java @@ -0,0 +1,29 @@ +/* + * 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.timelineservice.reader contains classes + * which can be used across reader. This package contains classes which are + * not related to storage implementations. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.timelineservice.reader; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderAuthenticationFilterInitializer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderAuthenticationFilterInitializer.java new file mode 100644 index 00000000000..e1c93be4bc1 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderAuthenticationFilterInitializer.java @@ -0,0 +1,53 @@ +/** + * 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.timelineservice.reader.security; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.http.FilterContainer; +import org.apache.hadoop.security.AuthenticationWithProxyUserFilter; +import org.apache.hadoop.yarn.server.timeline.security.TimelineAuthenticationFilterInitializer; + +/** + * Filter initializer to initialize {@link AuthenticationWithProxyUserFilter} + * for ATSv2 timeline reader server with timeline service specific + * configurations. + */ +public class TimelineReaderAuthenticationFilterInitializer extends + TimelineAuthenticationFilterInitializer{ + + /** + * Initializes {@link AuthenticationWithProxyUserFilter} + *

+ * Propagates to {@link AuthenticationWithProxyUserFilter} configuration all + * YARN configuration properties prefixed with + * {@link TimelineAuthenticationFilterInitializer#PREFIX}. + * + * @param container + * The filter container + * @param conf + * Configuration for run-time parameters + */ + @Override + public void initFilter(FilterContainer container, Configuration conf) { + setAuthFilterConfig(conf); + container.addGlobalFilter("Timeline Reader Authentication Filter", + AuthenticationWithProxyUserFilter.class.getName(), + getFilterConfig()); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderWhitelistAuthorizationFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderWhitelistAuthorizationFilter.java new file mode 100644 index 00000000000..5b2e70325ed --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderWhitelistAuthorizationFilter.java @@ -0,0 +1,123 @@ +/** + * 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.timelineservice.reader.security; + +import java.io.IOException; + +import javax.servlet.Filter; +import javax.servlet.FilterChain; +import javax.servlet.FilterConfig; +import javax.servlet.ServletException; +import javax.servlet.ServletRequest; +import javax.servlet.ServletResponse; +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.core.Response; +import javax.ws.rs.core.Response.Status; + +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.authorize.AccessControlList; +import org.apache.hadoop.security.authorize.AuthorizationException; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.webapp.ForbiddenException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderWebServicesUtils; + +/** + * Filter to check if a particular user is allowed to read ATSv2 data. + */ + +public class TimelineReaderWhitelistAuthorizationFilter implements Filter { + + public static final String EMPTY_STRING = ""; + + private static final Logger LOG = + LoggerFactory.getLogger(TimelineReaderWhitelistAuthorizationFilter.class); + + private boolean isWhitelistReadAuthEnabled = false; + + private AccessControlList allowedUsersAclList; + private AccessControlList adminAclList; + + @Override + public void destroy() { + // NOTHING + } + + @Override + public void doFilter(ServletRequest request, ServletResponse response, + FilterChain chain) throws IOException, ServletException { + if (isWhitelistReadAuthEnabled) { + UserGroupInformation callerUGI = TimelineReaderWebServicesUtils + .getCallerUserGroupInformation((HttpServletRequest) request, true); + if (callerUGI == null) { + String msg = "Unable to obtain user name, user not authenticated"; + throw new AuthorizationException(msg); + } + if (!(adminAclList.isUserAllowed(callerUGI) + || allowedUsersAclList.isUserAllowed(callerUGI))) { + String userName = callerUGI.getShortUserName(); + String msg = "User " + userName + + " is not allowed to read TimelineService V2 data."; + Response.status(Status.FORBIDDEN).entity(msg).build(); + throw new ForbiddenException("user " + userName + + " is not allowed to read TimelineService V2 data"); + } + } + if (chain != null) { + chain.doFilter(request, response); + } + } + + @Override + public void init(FilterConfig conf) throws ServletException { + String isWhitelistReadAuthEnabledStr = conf + .getInitParameter(YarnConfiguration.TIMELINE_SERVICE_READ_AUTH_ENABLED); + if (isWhitelistReadAuthEnabledStr == null) { + isWhitelistReadAuthEnabled = + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_READ_AUTH_ENABLED; + } else { + isWhitelistReadAuthEnabled = + Boolean.valueOf(isWhitelistReadAuthEnabledStr); + } + + if (isWhitelistReadAuthEnabled) { + String listAllowedUsers = conf.getInitParameter( + YarnConfiguration.TIMELINE_SERVICE_READ_ALLOWED_USERS); + if (StringUtils.isEmpty(listAllowedUsers)) { + listAllowedUsers = + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_READ_ALLOWED_USERS; + } + LOG.info("listAllowedUsers=" + listAllowedUsers); + allowedUsersAclList = new AccessControlList(listAllowedUsers); + LOG.info("allowedUsersAclList=" + allowedUsersAclList.getUsers()); + // also allow admins + String adminAclListStr = + conf.getInitParameter(YarnConfiguration.YARN_ADMIN_ACL); + if (StringUtils.isEmpty(adminAclListStr)) { + adminAclListStr = + TimelineReaderWhitelistAuthorizationFilter.EMPTY_STRING; + LOG.info("adminAclList not set, hence setting it to \"\""); + } + adminAclList = new AccessControlList(adminAclListStr); + LOG.info("adminAclList=" + adminAclList.getUsers()); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderWhitelistAuthorizationFilterInitializer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderWhitelistAuthorizationFilterInitializer.java new file mode 100644 index 00000000000..a970731272a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderWhitelistAuthorizationFilterInitializer.java @@ -0,0 +1,66 @@ +/** + * 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.timelineservice.reader.security; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.http.FilterContainer; +import org.apache.hadoop.http.FilterInitializer; +import org.apache.hadoop.yarn.conf.YarnConfiguration; + +/** + * Filter initializer to initialize + * {@link TimelineReaderWhitelistAuthorizationFilter} for ATSv2 timeline reader + * with timeline service specific configurations. + */ +public class TimelineReaderWhitelistAuthorizationFilterInitializer + extends FilterInitializer { + + /** + * Initializes {@link TimelineReaderWhitelistAuthorizationFilter}. + * + * @param container The filter container + * @param conf Configuration for run-time parameters + */ + @Override + public void initFilter(FilterContainer container, Configuration conf) { + Map params = new HashMap(); + String isWhitelistReadAuthEnabled = Boolean.toString( + conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_READ_AUTH_ENABLED, + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_READ_AUTH_ENABLED)); + params.put(YarnConfiguration.TIMELINE_SERVICE_READ_AUTH_ENABLED, + isWhitelistReadAuthEnabled); + params.put(YarnConfiguration.TIMELINE_SERVICE_READ_ALLOWED_USERS, + conf.get(YarnConfiguration.TIMELINE_SERVICE_READ_ALLOWED_USERS, + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_READ_ALLOWED_USERS)); + + params.put(YarnConfiguration.YARN_ADMIN_ACL, + conf.get(YarnConfiguration.YARN_ADMIN_ACL, + // using a default of "" + // instead of DEFAULT_YARN_ADMIN_ACL + // The reason being, DEFAULT_YARN_ADMIN_ACL is set to all users + // and we do not wish to allow everyone by default if + // read auth is enabled and YARN_ADMIN_ACL is unset + TimelineReaderWhitelistAuthorizationFilter.EMPTY_STRING)); + container.addGlobalFilter("Timeline Reader Whitelist Authorization Filter", + TimelineReaderWhitelistAuthorizationFilter.class.getName(), params); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/package-info.java new file mode 100644 index 00000000000..5888c989548 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/package-info.java @@ -0,0 +1,25 @@ +/** + * 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.server.timelineservice.reader.security contains + * classes to be used to support SPNEGO authentication for timeline reader. + */ +@InterfaceAudience.Private +package org.apache.hadoop.yarn.server.timelineservice.reader.security; +import org.apache.hadoop.classification.InterfaceAudience; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/security/CollectorNodemanagerSecurityInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/security/CollectorNodemanagerSecurityInfo.java new file mode 100644 index 00000000000..0eb5ee5aa52 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/security/CollectorNodemanagerSecurityInfo.java @@ -0,0 +1,69 @@ +/** +* 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.timelineservice.security; + +import java.lang.annotation.Annotation; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Evolving; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.KerberosInfo; +import org.apache.hadoop.security.SecurityInfo; +import org.apache.hadoop.security.token.TokenInfo; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocolPB; + +/** + * SecurityInfo implementation for CollectorNodemanager protocol. + */ +@Public +@Evolving +public class CollectorNodemanagerSecurityInfo extends SecurityInfo { + + @Override + public KerberosInfo getKerberosInfo(Class protocol, Configuration conf) { + if (!protocol + .equals(CollectorNodemanagerProtocolPB.class)) { + return null; + } + return new KerberosInfo() { + + @Override + public Class annotationType() { + return null; + } + + @Override + public String serverPrincipal() { + return YarnConfiguration.NM_PRINCIPAL; + } + + @Override + public String clientPrincipal() { + return null; + } + }; + } + + @Override + public TokenInfo getTokenInfo(Class protocol, Configuration conf) { + return null; + } +} + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/security/TimelineV2DelegationTokenSecretManagerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/security/TimelineV2DelegationTokenSecretManagerService.java new file mode 100644 index 00000000000..1a8bf091eb5 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/security/TimelineV2DelegationTokenSecretManagerService.java @@ -0,0 +1,126 @@ +/** + * 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.timelineservice.security; + +import java.io.IOException; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager; +import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier; +import org.apache.hadoop.yarn.server.timeline.security.TimelineDelgationTokenSecretManagerService; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The service wrapper of {@link TimelineV2DelegationTokenSecretManager}. + */ +public class TimelineV2DelegationTokenSecretManagerService extends + TimelineDelgationTokenSecretManagerService { + + public TimelineV2DelegationTokenSecretManagerService() { + super(TimelineV2DelegationTokenSecretManagerService.class.getName()); + } + + @Override + protected AbstractDelegationTokenSecretManager + + createTimelineDelegationTokenSecretManager(long secretKeyInterval, + long tokenMaxLifetime, long tokenRenewInterval, + long tokenRemovalScanInterval) { + return new TimelineV2DelegationTokenSecretManager(secretKeyInterval, + tokenMaxLifetime, tokenRenewInterval, tokenRemovalScanInterval); + } + + public Token generateToken( + UserGroupInformation ugi, String renewer) { + return ((TimelineV2DelegationTokenSecretManager) + getTimelineDelegationTokenSecretManager()).generateToken(ugi, renewer); + } + + public long renewToken(Token token, + String renewer) throws IOException { + return getTimelineDelegationTokenSecretManager().renewToken(token, renewer); + } + + public void cancelToken(Token token, + String canceller) throws IOException { + getTimelineDelegationTokenSecretManager().cancelToken(token, canceller); + } + + /** + * Delegation token secret manager for ATSv2. + */ + @Private + @Unstable + public static class TimelineV2DelegationTokenSecretManager extends + AbstractDelegationTokenSecretManager { + + private static final Logger LOG = + LoggerFactory.getLogger(TimelineV2DelegationTokenSecretManager.class); + + /** + * Create a timeline v2 secret manager. + * @param delegationKeyUpdateInterval the number of milliseconds for rolling + * new secret keys. + * @param delegationTokenMaxLifetime the maximum lifetime of the delegation + * tokens in milliseconds + * @param delegationTokenRenewInterval how often the tokens must be renewed + * in milliseconds + * @param delegationTokenRemoverScanInterval how often the tokens are + * scanned for expired tokens in milliseconds + */ + public TimelineV2DelegationTokenSecretManager( + long delegationKeyUpdateInterval, long delegationTokenMaxLifetime, + long delegationTokenRenewInterval, + long delegationTokenRemoverScanInterval) { + super(delegationKeyUpdateInterval, delegationTokenMaxLifetime, + delegationTokenRenewInterval, delegationTokenRemoverScanInterval); + } + + public Token generateToken( + UserGroupInformation ugi, String renewer) { + Text realUser = null; + if (ugi.getRealUser() != null) { + realUser = new Text(ugi.getRealUser().getUserName()); + } + TimelineDelegationTokenIdentifier identifier = createIdentifier(); + identifier.setOwner(new Text(ugi.getUserName())); + identifier.setRenewer(new Text(renewer)); + identifier.setRealUser(realUser); + byte[] password = createPassword(identifier); + return new Token(identifier.getBytes(), + password, identifier.getKind(), null); + } + + @Override + public TimelineDelegationTokenIdentifier createIdentifier() { + return new TimelineDelegationTokenIdentifier(); + } + + @Override + protected void logExpireToken(TimelineDelegationTokenIdentifier ident) + throws IOException { + LOG.info("Token " + ident + " expired."); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/security/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/security/package-info.java new file mode 100644 index 00000000000..825009204c0 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/security/package-info.java @@ -0,0 +1,25 @@ +/** + * 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.server.timelineservice.security contains classes + * to be used to generate delegation tokens for ATSv2. + */ +@InterfaceAudience.Private +package org.apache.hadoop.yarn.server.timelineservice.security; +import org.apache.hadoop.classification.InterfaceAudience; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.java new file mode 100644 index 00000000000..a0ee2bea3ae --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.java @@ -0,0 +1,432 @@ +/** + * 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.timelineservice.storage; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStreamReader; +import java.nio.charset.Charset; +import java.util.Comparator; +import java.util.EnumSet; +import java.util.HashSet; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.TreeMap; +import java.util.TreeSet; + +import org.apache.commons.csv.CSVFormat; +import org.apache.commons.csv.CSVParser; +import org.apache.commons.csv.CSVRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils; +import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider; +import org.codehaus.jackson.JsonGenerationException; +import org.codehaus.jackson.map.JsonMappingException; +import org.codehaus.jackson.map.ObjectMapper; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * File System based implementation for TimelineReader. This implementation may + * not provide a complete implementation of all the necessary features. This + * implementation is provided solely for basic testing purposes, and should not + * be used in a non-test situation. + */ +public class FileSystemTimelineReaderImpl extends AbstractService + implements TimelineReader { + + private static final Logger LOG = + LoggerFactory.getLogger(FileSystemTimelineReaderImpl.class); + + private String rootPath; + private static final String ENTITIES_DIR = "entities"; + + /** Default extension for output files. */ + private static final String TIMELINE_SERVICE_STORAGE_EXTENSION = ".thist"; + + @VisibleForTesting + /** Default extension for output files. */ + static final String APP_FLOW_MAPPING_FILE = "app_flow_mapping.csv"; + + /** Config param for timeline service file system storage root. */ + public static final String TIMELINE_SERVICE_STORAGE_DIR_ROOT = + YarnConfiguration.TIMELINE_SERVICE_PREFIX + "fs-writer.root-dir"; + + /** Default value for storage location on local disk. */ + private static final String STORAGE_DIR_ROOT = "timeline_service_data"; + + private final CSVFormat csvFormat = + CSVFormat.DEFAULT.withHeader("APP", "USER", "FLOW", "FLOWRUN"); + + public FileSystemTimelineReaderImpl() { + super(FileSystemTimelineReaderImpl.class.getName()); + } + + @VisibleForTesting + String getRootPath() { + return rootPath; + } + + private static ObjectMapper mapper; + + static { + mapper = new ObjectMapper(); + YarnJacksonJaxbJsonProvider.configObjectMapper(mapper); + } + + /** + * Deserialize a POJO object from a JSON string. + * + * @param Describes the type of class to be returned. + * @param clazz class to be deserialized. + * @param jsonString JSON string to deserialize. + * @return An object based on class type. Used typically for + * TimelineEntity object. + * @throws IOException if the underlying input source has problems during + * parsing. + * @throws JsonMappingException if parser has problems parsing content. + * @throws JsonGenerationException if there is a problem in JSON writing. + */ + public static T getTimelineRecordFromJSON( + String jsonString, Class clazz) + throws JsonGenerationException, JsonMappingException, IOException { + return mapper.readValue(jsonString, clazz); + } + + private static void fillFields(TimelineEntity finalEntity, + TimelineEntity real, EnumSet fields) { + if (fields.contains(Field.ALL)) { + fields = EnumSet.allOf(Field.class); + } + for (Field field : fields) { + switch(field) { + case CONFIGS: + finalEntity.setConfigs(real.getConfigs()); + break; + case METRICS: + finalEntity.setMetrics(real.getMetrics()); + break; + case INFO: + finalEntity.setInfo(real.getInfo()); + break; + case IS_RELATED_TO: + finalEntity.setIsRelatedToEntities(real.getIsRelatedToEntities()); + break; + case RELATES_TO: + finalEntity.setIsRelatedToEntities(real.getIsRelatedToEntities()); + break; + case EVENTS: + finalEntity.setEvents(real.getEvents()); + break; + default: + continue; + } + } + } + + private String getFlowRunPath(String userId, String clusterId, + String flowName, Long flowRunId, String appId) throws IOException { + if (userId != null && flowName != null && flowRunId != null) { + return userId + File.separator + flowName + File.separator + flowRunId; + } + if (clusterId == null || appId == null) { + throw new IOException("Unable to get flow info"); + } + String appFlowMappingFile = rootPath + File.separator + ENTITIES_DIR + + File.separator + clusterId + File.separator + APP_FLOW_MAPPING_FILE; + try (BufferedReader reader = + new BufferedReader(new InputStreamReader( + new FileInputStream( + appFlowMappingFile), Charset.forName("UTF-8"))); + CSVParser parser = new CSVParser(reader, csvFormat)) { + for (CSVRecord record : parser.getRecords()) { + if (record.size() < 4) { + continue; + } + String applicationId = record.get("APP"); + if (applicationId != null && !applicationId.trim().isEmpty() && + !applicationId.trim().equals(appId)) { + continue; + } + return record.get(1).trim() + File.separator + record.get(2).trim() + + File.separator + record.get(3).trim(); + } + parser.close(); + } + throw new IOException("Unable to get flow info"); + } + + private static TimelineEntity createEntityToBeReturned(TimelineEntity entity, + EnumSet fieldsToRetrieve) { + TimelineEntity entityToBeReturned = new TimelineEntity(); + entityToBeReturned.setIdentifier(entity.getIdentifier()); + entityToBeReturned.setCreatedTime(entity.getCreatedTime()); + if (fieldsToRetrieve != null) { + fillFields(entityToBeReturned, entity, fieldsToRetrieve); + } + return entityToBeReturned; + } + + private static boolean isTimeInRange(Long time, Long timeBegin, + Long timeEnd) { + return (time >= timeBegin) && (time <= timeEnd); + } + + private static void mergeEntities(TimelineEntity entity1, + TimelineEntity entity2) { + // Ideally created time wont change except in the case of issue from client. + if (entity2.getCreatedTime() != null && entity2.getCreatedTime() > 0) { + entity1.setCreatedTime(entity2.getCreatedTime()); + } + for (Entry configEntry : entity2.getConfigs().entrySet()) { + entity1.addConfig(configEntry.getKey(), configEntry.getValue()); + } + for (Entry infoEntry : entity2.getInfo().entrySet()) { + entity1.addInfo(infoEntry.getKey(), infoEntry.getValue()); + } + for (Entry> isRelatedToEntry : + entity2.getIsRelatedToEntities().entrySet()) { + String type = isRelatedToEntry.getKey(); + for (String entityId : isRelatedToEntry.getValue()) { + entity1.addIsRelatedToEntity(type, entityId); + } + } + for (Entry> relatesToEntry : + entity2.getRelatesToEntities().entrySet()) { + String type = relatesToEntry.getKey(); + for (String entityId : relatesToEntry.getValue()) { + entity1.addRelatesToEntity(type, entityId); + } + } + for (TimelineEvent event : entity2.getEvents()) { + entity1.addEvent(event); + } + for (TimelineMetric metric2 : entity2.getMetrics()) { + boolean found = false; + for (TimelineMetric metric1 : entity1.getMetrics()) { + if (metric1.getId().equals(metric2.getId())) { + metric1.addValues(metric2.getValues()); + found = true; + break; + } + } + if (!found) { + entity1.addMetric(metric2); + } + } + } + + private static TimelineEntity readEntityFromFile(BufferedReader reader) + throws IOException { + TimelineEntity entity = + getTimelineRecordFromJSON(reader.readLine(), TimelineEntity.class); + String entityStr = ""; + while ((entityStr = reader.readLine()) != null) { + if (entityStr.trim().isEmpty()) { + continue; + } + TimelineEntity anotherEntity = + getTimelineRecordFromJSON(entityStr, TimelineEntity.class); + if (!entity.getId().equals(anotherEntity.getId()) || + !entity.getType().equals(anotherEntity.getType())) { + continue; + } + mergeEntities(entity, anotherEntity); + } + return entity; + } + + private Set getEntities(File dir, String entityType, + TimelineEntityFilters filters, TimelineDataToRetrieve dataToRetrieve) + throws IOException { + // First sort the selected entities based on created/start time. + Map> sortedEntities = + new TreeMap<>( + new Comparator() { + @Override + public int compare(Long l1, Long l2) { + return l2.compareTo(l1); + } + } + ); + if (dir != null) { + File[] files = dir.listFiles(); + if (files != null) { + for (File entityFile : files) { + if (!entityFile.getName() + .contains(TIMELINE_SERVICE_STORAGE_EXTENSION)) { + continue; + } + try (BufferedReader reader = new BufferedReader( + new InputStreamReader(new FileInputStream(entityFile), + Charset.forName("UTF-8")))) { + TimelineEntity entity = readEntityFromFile(reader); + if (!entity.getType().equals(entityType)) { + continue; + } + if (!isTimeInRange(entity.getCreatedTime(), + filters.getCreatedTimeBegin(), + filters.getCreatedTimeEnd())) { + continue; + } + if (filters.getRelatesTo() != null && + !filters.getRelatesTo().getFilterList().isEmpty() && + !TimelineStorageUtils.matchRelatesTo(entity, + filters.getRelatesTo())) { + continue; + } + if (filters.getIsRelatedTo() != null && + !filters.getIsRelatedTo().getFilterList().isEmpty() && + !TimelineStorageUtils.matchIsRelatedTo(entity, + filters.getIsRelatedTo())) { + continue; + } + if (filters.getInfoFilters() != null && + !filters.getInfoFilters().getFilterList().isEmpty() && + !TimelineStorageUtils.matchInfoFilters(entity, + filters.getInfoFilters())) { + continue; + } + if (filters.getConfigFilters() != null && + !filters.getConfigFilters().getFilterList().isEmpty() && + !TimelineStorageUtils.matchConfigFilters(entity, + filters.getConfigFilters())) { + continue; + } + if (filters.getMetricFilters() != null && + !filters.getMetricFilters().getFilterList().isEmpty() && + !TimelineStorageUtils.matchMetricFilters(entity, + filters.getMetricFilters())) { + continue; + } + if (filters.getEventFilters() != null && + !filters.getEventFilters().getFilterList().isEmpty() && + !TimelineStorageUtils.matchEventFilters(entity, + filters.getEventFilters())) { + continue; + } + TimelineEntity entityToBeReturned = createEntityToBeReturned( + entity, dataToRetrieve.getFieldsToRetrieve()); + Set entitiesCreatedAtSameTime = + sortedEntities.get(entityToBeReturned.getCreatedTime()); + if (entitiesCreatedAtSameTime == null) { + entitiesCreatedAtSameTime = new HashSet(); + } + entitiesCreatedAtSameTime.add(entityToBeReturned); + sortedEntities.put(entityToBeReturned.getCreatedTime(), + entitiesCreatedAtSameTime); + } + } + } + } + + Set entities = new HashSet(); + long entitiesAdded = 0; + for (Set entitySet : sortedEntities.values()) { + for (TimelineEntity entity : entitySet) { + entities.add(entity); + ++entitiesAdded; + if (entitiesAdded >= filters.getLimit()) { + return entities; + } + } + } + return entities; + } + + @Override + public void serviceInit(Configuration conf) throws Exception { + rootPath = conf.get(TIMELINE_SERVICE_STORAGE_DIR_ROOT, + conf.get("hadoop.tmp.dir") + File.separator + STORAGE_DIR_ROOT); + super.serviceInit(conf); + } + + @Override + public TimelineEntity getEntity(TimelineReaderContext context, + TimelineDataToRetrieve dataToRetrieve) throws IOException { + String flowRunPath = getFlowRunPath(context.getUserId(), + context.getClusterId(), context.getFlowName(), context.getFlowRunId(), + context.getAppId()); + File dir = new File(new File(rootPath, ENTITIES_DIR), + context.getClusterId() + File.separator + flowRunPath + File.separator + + context.getAppId() + File.separator + context.getEntityType()); + File entityFile = new File( + dir, context.getEntityId() + TIMELINE_SERVICE_STORAGE_EXTENSION); + try (BufferedReader reader = + new BufferedReader(new InputStreamReader( + new FileInputStream(entityFile), Charset.forName("UTF-8")))) { + TimelineEntity entity = readEntityFromFile(reader); + return createEntityToBeReturned( + entity, dataToRetrieve.getFieldsToRetrieve()); + } catch (FileNotFoundException e) { + LOG.info("Cannot find entity {id:" + context.getEntityId() + " , type:" + + context.getEntityType() + "}. Will send HTTP 404 in response."); + return null; + } + } + + @Override + public Set getEntities(TimelineReaderContext context, + TimelineEntityFilters filters, TimelineDataToRetrieve dataToRetrieve) + throws IOException { + String flowRunPath = getFlowRunPath(context.getUserId(), + context.getClusterId(), context.getFlowName(), context.getFlowRunId(), + context.getAppId()); + File dir = + new File(new File(rootPath, ENTITIES_DIR), + context.getClusterId() + File.separator + flowRunPath + + File.separator + context.getAppId() + File.separator + + context.getEntityType()); + return getEntities(dir, context.getEntityType(), filters, dataToRetrieve); + } + + @Override public Set getEntityTypes(TimelineReaderContext context) + throws IOException { + Set result = new TreeSet<>(); + String flowRunPath = getFlowRunPath(context.getUserId(), + context.getClusterId(), context.getFlowName(), context.getFlowRunId(), + context.getAppId()); + File dir = new File(new File(rootPath, ENTITIES_DIR), + context.getClusterId() + File.separator + flowRunPath + + File.separator + context.getAppId()); + File[] fileList = dir.listFiles(); + if (fileList != null) { + for (File f : fileList) { + if (f.isDirectory()) { + result.add(f.getName()); + } + } + } + return result; + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineWriterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineWriterImpl.java new file mode 100644 index 00000000000..ee4197000bf --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineWriterImpl.java @@ -0,0 +1,169 @@ +/** + * 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.timelineservice.storage; + +import java.io.BufferedWriter; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.io.PrintWriter; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineWriteResponse; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineWriteResponse.TimelineWriteError; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext; +import org.apache.hadoop.yarn.util.timeline.TimelineUtils; + +import com.google.common.annotations.VisibleForTesting; + +/** + * This implements a local file based backend for storing application timeline + * information. This implementation may not provide a complete implementation of + * all the necessary features. This implementation is provided solely for basic + * testing purposes, and should not be used in a non-test situation. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public class FileSystemTimelineWriterImpl extends AbstractService + implements TimelineWriter { + + private String outputRoot; + + /** Config param for timeline service storage tmp root for FILE YARN-3264. */ + public static final String TIMELINE_SERVICE_STORAGE_DIR_ROOT + = YarnConfiguration.TIMELINE_SERVICE_PREFIX + "fs-writer.root-dir"; + + public static final String ENTITIES_DIR = "entities"; + + /** Default extension for output files. */ + public static final String TIMELINE_SERVICE_STORAGE_EXTENSION = ".thist"; + + /** default value for storage location on local disk. */ + private static final String STORAGE_DIR_ROOT = "timeline_service_data"; + + FileSystemTimelineWriterImpl() { + super((FileSystemTimelineWriterImpl.class.getName())); + } + + @Override + public TimelineWriteResponse write(TimelineCollectorContext context, + TimelineEntities entities, UserGroupInformation callerUgi) + throws IOException { + TimelineWriteResponse response = new TimelineWriteResponse(); + String clusterId = context.getClusterId(); + String userId = context.getUserId(); + String flowName = context.getFlowName(); + String flowVersion = context.getFlowVersion(); + long flowRunId = context.getFlowRunId(); + String appId = context.getAppId(); + + for (TimelineEntity entity : entities.getEntities()) { + write(clusterId, userId, flowName, flowVersion, flowRunId, appId, entity, + response); + } + return response; + } + + private synchronized void write(String clusterId, String userId, + String flowName, String flowVersion, long flowRun, String appId, + TimelineEntity entity, TimelineWriteResponse response) + throws IOException { + PrintWriter out = null; + try { + String dir = mkdirs(outputRoot, ENTITIES_DIR, clusterId, userId, + escape(flowName), escape(flowVersion), String.valueOf(flowRun), appId, + entity.getType()); + String fileName = dir + entity.getId() + + TIMELINE_SERVICE_STORAGE_EXTENSION; + out = + new PrintWriter(new BufferedWriter(new OutputStreamWriter( + new FileOutputStream(fileName, true), "UTF-8"))); + out.println(TimelineUtils.dumpTimelineRecordtoJSON(entity)); + out.write("\n"); + } catch (IOException ioe) { + TimelineWriteError error = new TimelineWriteError(); + error.setEntityId(entity.getId()); + error.setEntityType(entity.getType()); + /* + * TODO: set an appropriate error code after PoC could possibly be: + * error.setErrorCode(TimelineWriteError.IO_EXCEPTION); + */ + response.addError(error); + } finally { + if (out != null) { + out.close(); + } + } + } + + @Override + public TimelineWriteResponse aggregate(TimelineEntity data, + TimelineAggregationTrack track) throws IOException { + return null; + + } + + @VisibleForTesting + String getOutputRoot() { + return outputRoot; + } + + @Override + public void serviceInit(Configuration conf) throws Exception { + outputRoot = conf.get(TIMELINE_SERVICE_STORAGE_DIR_ROOT, + conf.get("hadoop.tmp.dir") + File.separator + STORAGE_DIR_ROOT); + } + + @Override + public void serviceStart() throws Exception { + mkdirs(outputRoot, ENTITIES_DIR); + } + + @Override + public void flush() throws IOException { + // no op + } + + private static String mkdirs(String... dirStrs) throws IOException { + StringBuilder path = new StringBuilder(); + for (String dirStr : dirStrs) { + path.append(dirStr).append(File.separatorChar); + File dir = new File(path.toString()); + if (!dir.exists()) { + if (!dir.mkdirs()) { + throw new IOException("Could not create directories for " + dir); + } + } + } + return path.toString(); + } + + // specifically escape the separator character + private static String escape(String str) { + return str.replace(File.separatorChar, '_'); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/OfflineAggregationWriter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/OfflineAggregationWriter.java new file mode 100644 index 00000000000..1484f22a8c7 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/OfflineAggregationWriter.java @@ -0,0 +1,67 @@ +/** + * 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.timelineservice.storage; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineWriteResponse; +import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.OfflineAggregationInfo; + +import java.io.IOException; + +/** + * YARN timeline service v2 offline aggregation storage interface. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public abstract class OfflineAggregationWriter extends AbstractService { + + /** + * Construct the offline writer. + * + * @param name service name + */ + public OfflineAggregationWriter(String name) { + super(name); + } + + /** + * Persist aggregated timeline entities to the offline store based on which + * track this entity is to be rolled up to. The tracks along which + * aggregations are to be done are given by {@link OfflineAggregationInfo}. + * + * @param context a {@link TimelineCollectorContext} object that describes the + * context information of the aggregated data. Depends on the + * type of the aggregation, some fields of this context maybe + * empty or null. + * @param entities {@link TimelineEntities} to be persisted. + * @param info an {@link OfflineAggregationInfo} object that describes the + * detail of the aggregation. Current supported option is + * {@link OfflineAggregationInfo#FLOW_AGGREGATION}. + * @return a {@link TimelineWriteResponse} object. + * @throws IOException if any problem occurs while writing aggregated + * entities. + */ + abstract TimelineWriteResponse writeAggregatedEntity( + TimelineCollectorContext context, TimelineEntities entities, + OfflineAggregationInfo info) throws IOException; +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineAggregationTrack.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineAggregationTrack.java new file mode 100644 index 00000000000..6a1e0867f18 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineAggregationTrack.java @@ -0,0 +1,28 @@ +/** + * 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.timelineservice.storage; + +/** + * specifies the tracks along which an entity + * info is to be aggregated on. + * + */ +public enum TimelineAggregationTrack { + APP, FLOW, USER, QUEUE +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineReader.java new file mode 100644 index 00000000000..16d623ab46f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineReader.java @@ -0,0 +1,195 @@ +/** + * 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.timelineservice.storage; + +import java.io.IOException; + +import java.util.Set; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.service.Service; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext; + +/** ATSv2 reader interface. */ +@Private +@Unstable +public interface TimelineReader extends Service { + + /** + * Possible fields to retrieve for {@link #getEntities} and + * {@link #getEntity}. + */ + public enum Field { + ALL, + EVENTS, + INFO, + METRICS, + CONFIGS, + RELATES_TO, + IS_RELATED_TO + } + + /** + *

The API to fetch the single entity given the identifier(depending on + * the entity type) in the scope of the given context.

+ * @param context Context which defines the scope in which query has to be + * made. Use getters of {@link TimelineReaderContext} to fetch context + * fields. Context contains the following :
+ *
    + *
  • entityType - Entity type(mandatory).
  • + *
  • clusterId - Identifies the cluster(mandatory).
  • + *
  • userId - Identifies the user.
  • + *
  • flowName - Context flow name.
  • + *
  • flowRunId - Context flow run id.
  • + *
  • appId - Context app id.
  • + *
  • entityId - Entity id.
  • + *
+ * Fields in context which are mandatory depends on entity type. Entity + * type is always mandatory. In addition to entity type, below is the list + * of context fields which are mandatory, based on entity type.
+ *
    + *
  • If entity type is YARN_FLOW_RUN (i.e. query to fetch a specific flow + * run), clusterId, userId, flowName and flowRunId are mandatory.
  • + *
  • If entity type is YARN_APPLICATION (i.e. query to fetch a specific + * app), query is within the scope of clusterId, userId, flowName, + * flowRunId and appId. But out of this, only clusterId and appId are + * mandatory. If only clusterId and appId are supplied, backend storage + * must fetch the flow context information i.e. userId, flowName and + * flowRunId first and based on that, fetch the app. If flow context + * information is also given, app can be directly fetched. + *
  • + *
  • For other entity types (i.e. query to fetch generic entity), query + * is within the scope of clusterId, userId, flowName, flowRunId, appId, + * entityType and entityId. But out of this, only clusterId, appId, + * entityType and entityId are mandatory. If flow context information is + * not supplied, backend storage must fetch the flow context information + * i.e. userId, flowName and flowRunId first and based on that, fetch the + * entity. If flow context information is also given, entity can be + * directly queried. + *
  • + *
+ * @param dataToRetrieve Specifies which data to retrieve for the entity. Use + * getters of TimelineDataToRetrieve class to fetch dataToRetrieve + * fields. All the dataToRetrieve fields are optional. Refer to + * {@link TimelineDataToRetrieve} for details. + * @return A TimelineEntity instance or null. The entity will + * contain the metadata plus the given fields to retrieve.
+ * If entityType is YARN_FLOW_RUN, entity returned is of type + * FlowRunEntity.
+ * For all other entity types, entity returned is of type + * TimelineEntity. + * @throws IOException if there is an exception encountered while fetching + * entity from backend storage. + */ + TimelineEntity getEntity(TimelineReaderContext context, + TimelineDataToRetrieve dataToRetrieve) throws IOException; + + /** + *

The API to search for a set of entities of the given entity type in + * the scope of the given context which matches the given predicates. The + * predicates include the created time window, limit to number of entities to + * be returned, and the entities can be filtered by checking whether they + * contain the given info/configs entries in the form of key/value pairs, + * given metrics in the form of metricsIds and its relation with metric + * values, given events in the form of the Ids, and whether they relate to/are + * related to other entities. For those parameters which have multiple + * entries, the qualified entity needs to meet all or them.

+ * + * @param context Context which defines the scope in which query has to be + * made. Use getters of {@link TimelineReaderContext} to fetch context + * fields. Context contains the following :
+ *
    + *
  • entityType - Entity type(mandatory).
  • + *
  • clusterId - Identifies the cluster(mandatory).
  • + *
  • userId - Identifies the user.
  • + *
  • flowName - Context flow name.
  • + *
  • flowRunId - Context flow run id.
  • + *
  • appId - Context app id.
  • + *
+ * Although entityIdPrefix and entityId are also part of context, + * it has no meaning for getEntities.
+ * Fields in context which are mandatory depends on entity type. Entity + * type is always mandatory. In addition to entity type, below is the list + * of context fields which are mandatory, based on entity type.
+ *
    + *
  • If entity type is YARN_FLOW_ACTIVITY (i.e. query to fetch flows), + * only clusterId is mandatory. + *
  • + *
  • If entity type is YARN_FLOW_RUN (i.e. query to fetch flow runs), + * clusterId, userId and flowName are mandatory.
  • + *
  • If entity type is YARN_APPLICATION (i.e. query to fetch apps), we + * can either get all apps within the context of flow name or within the + * context of flow run. If apps are queried within the scope of flow name, + * clusterId, userId and flowName are supplied. If they are queried within + * the scope of flow run, clusterId, userId, flowName and flowRunId are + * supplied.
  • + *
  • For other entity types (i.e. query to fetch generic entities), query + * is within the scope of clusterId, userId, flowName, flowRunId, appId and + * entityType. But out of this, only clusterId, appId and entityType are + * mandatory. If flow context information is not supplied, backend storage + * must fetch the flow context information i.e. userId, flowName and + * flowRunId first and based on that, fetch the entities. If flow context + * information is also given, entities can be directly queried. + *
  • + *
+ * @param filters Specifies filters which restrict the number of entities + * to return. Use getters of TimelineEntityFilters class to fetch + * various filters. All the filters are optional. Refer to + * {@link TimelineEntityFilters} for details. + * @param dataToRetrieve Specifies which data to retrieve for each entity. Use + * getters of TimelineDataToRetrieve class to fetch dataToRetrieve + * fields. All the dataToRetrieve fields are optional. Refer to + * {@link TimelineDataToRetrieve} for details. + * @return A set of TimelineEntity instances of the given entity + * type in the given context scope which matches the given predicates + * ordered by enitityIdPrefix(for generic entities only). + * Each entity will only contain + * the metadata(id, type , idPrefix and created time) plus the given + * fields to retrieve. + *
+ * If entityType is YARN_FLOW_ACTIVITY, entities returned are of type + * FlowActivityEntity.
+ * If entityType is YARN_FLOW_RUN, entities returned are of type + * FlowRunEntity.
+ * For all other entity types, entities returned are of type + * TimelineEntity. + * @throws IOException if there is an exception encountered while fetching + * entity from backend storage. + */ + Set getEntities( + TimelineReaderContext context, + TimelineEntityFilters filters, + TimelineDataToRetrieve dataToRetrieve) throws IOException; + + /** + * The API to list all available entity types of the given context. + * + * @param context A context defines the scope of this query. The incoming + * context should contain at least the cluster id and application id. + * + * @return A set of entity types available in the given context. + * + * @throws IOException if an exception occurred while listing from backend + * storage. + */ + Set getEntityTypes(TimelineReaderContext context) throws IOException; +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineWriter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineWriter.java new file mode 100644 index 00000000000..12bc1cb3f0e --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineWriter.java @@ -0,0 +1,83 @@ +/** + * 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.timelineservice.storage; + +import java.io.IOException; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.service.Service; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineWriteResponse; +import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext; + +/** + * This interface is for storing application timeline information. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public interface TimelineWriter extends Service { + + /** + * Stores the entire information in {@link TimelineEntities} to the timeline + * store. Any errors occurring for individual write request objects will be + * reported in the response. + * + * @param context a {@link TimelineCollectorContext} + * @param data a {@link TimelineEntities} object. + * @param callerUgi {@link UserGroupInformation}. + * @return a {@link TimelineWriteResponse} object. + * @throws IOException if there is any exception encountered while storing or + * writing entities to the back end storage. + */ + TimelineWriteResponse write(TimelineCollectorContext context, + TimelineEntities data, UserGroupInformation callerUgi) throws IOException; + + /** + * Aggregates the entity information to the timeline store based on which + * track this entity is to be rolled up to The tracks along which aggregations + * are to be done are given by {@link TimelineAggregationTrack} + * + * Any errors occurring for individual write request objects will be reported + * in the response. + * + * @param data + * a {@link TimelineEntity} object + * a {@link TimelineAggregationTrack} enum + * value. + * @param track Specifies the track or dimension along which aggregation would + * occur. Includes USER, FLOW, QUEUE, etc. + * @return a {@link TimelineWriteResponse} object. + * @throws IOException if there is any exception encountered while aggregating + * entities to the backend storage. + */ + TimelineWriteResponse aggregate(TimelineEntity data, + TimelineAggregationTrack track) throws IOException; + + /** + * Flushes the data to the backend storage. Whatever may be buffered will be + * written to the storage when the method returns. This may be a potentially + * time-consuming operation, and should be used judiciously. + * + * @throws IOException if there is any exception encountered while flushing + * entities to the backend storage. + */ + void flush() throws IOException; +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/OfflineAggregationInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/OfflineAggregationInfo.java new file mode 100644 index 00000000000..3dc5f514b2e --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/OfflineAggregationInfo.java @@ -0,0 +1,115 @@ +/** + * 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.timelineservice.storage.common; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext; + +import java.sql.PreparedStatement; +import java.sql.SQLException; + +/** + * Class to carry the offline aggregation information for storage level + * implementations. There are currently two predefined aggregation info + * instances that represent flow and user level offline aggregations. Depend on + * its implementation, a storage class may use an OfflineAggregationInfo object + * to decide behaviors dynamically. + */ +public final class OfflineAggregationInfo { + /** + * Default flow level aggregation table name. + */ + @VisibleForTesting + public static final String FLOW_AGGREGATION_TABLE_NAME + = "yarn_timeline_flow_aggregation"; + /** + * Default user level aggregation table name. + */ + public static final String USER_AGGREGATION_TABLE_NAME + = "yarn_timeline_user_aggregation"; + + // These lists are not taking effects in table creations. + private static final String[] FLOW_AGGREGATION_PK_LIST = { + "user", "cluster", "flow_name" + }; + private static final String[] USER_AGGREGATION_PK_LIST = { + "user", "cluster" + }; + + private final String tableName; + private final String[] primaryKeyList; + private final PrimaryKeyStringSetter primaryKeyStringSetter; + + private OfflineAggregationInfo(String table, String[] pkList, + PrimaryKeyStringSetter formatter) { + tableName = table; + primaryKeyList = pkList; + primaryKeyStringSetter = formatter; + } + + private interface PrimaryKeyStringSetter { + int setValues(PreparedStatement ps, TimelineCollectorContext context, + String[] extraInfo, int startPos) throws SQLException; + } + + public String getTableName() { + return tableName; + } + + public String[] getPrimaryKeyList() { + return primaryKeyList.clone(); + } + + public int setStringsForPrimaryKey(PreparedStatement ps, + TimelineCollectorContext context, String[] extraInfo, int startPos) + throws SQLException { + return primaryKeyStringSetter.setValues(ps, context, extraInfo, startPos); + } + + public static final OfflineAggregationInfo FLOW_AGGREGATION = + new OfflineAggregationInfo(FLOW_AGGREGATION_TABLE_NAME, + FLOW_AGGREGATION_PK_LIST, + new PrimaryKeyStringSetter() { + @Override + public int setValues(PreparedStatement ps, + TimelineCollectorContext context, String[] extraInfo, + int startPos) throws SQLException { + int idx = startPos; + ps.setString(idx++, context.getUserId()); + ps.setString(idx++, context.getClusterId()); + ps.setString(idx++, context.getFlowName()); + return idx; + } + }); + + public static final OfflineAggregationInfo USER_AGGREGATION = + new OfflineAggregationInfo(USER_AGGREGATION_TABLE_NAME, + USER_AGGREGATION_PK_LIST, + new PrimaryKeyStringSetter() { + @Override + public int setValues(PreparedStatement ps, + TimelineCollectorContext context, String[] extraInfo, + int startPos) throws SQLException { + int idx = startPos; + ps.setString(idx++, context.getUserId()); + ps.setString(idx++, context.getClusterId()); + return idx; + } + }); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineEntityFiltersType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineEntityFiltersType.java new file mode 100644 index 00000000000..4099e92da1a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineEntityFiltersType.java @@ -0,0 +1,71 @@ +/** + * 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.timelineservice.storage.common; + +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilter.TimelineFilterType; + +/** + * Used to define which filter to match. + */ +enum TimelineEntityFiltersType { + CONFIG { + boolean isValidFilter(TimelineFilterType filterType) { + return filterType == TimelineFilterType.LIST || + filterType == TimelineFilterType.KEY_VALUE; + } + }, + INFO { + boolean isValidFilter(TimelineFilterType filterType) { + return filterType == TimelineFilterType.LIST || + filterType == TimelineFilterType.KEY_VALUE; + } + }, + METRIC { + boolean isValidFilter(TimelineFilterType filterType) { + return filterType == TimelineFilterType.LIST || + filterType == TimelineFilterType.COMPARE; + } + }, + EVENT { + boolean isValidFilter(TimelineFilterType filterType) { + return filterType == TimelineFilterType.LIST || + filterType == TimelineFilterType.EXISTS; + } + }, + IS_RELATED_TO { + boolean isValidFilter(TimelineFilterType filterType) { + return filterType == TimelineFilterType.LIST || + filterType == TimelineFilterType.KEY_VALUES; + } + }, + RELATES_TO { + boolean isValidFilter(TimelineFilterType filterType) { + return filterType == TimelineFilterType.LIST || + filterType == TimelineFilterType.KEY_VALUES; + } + }; + + /** + * Checks whether filter type is valid for the filter being matched. + * + * @param filterType filter type. + * @return true, if its a valid filter, false otherwise. + */ + abstract boolean isValidFilter(TimelineFilterType filterType); +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineStorageUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineStorageUtils.java new file mode 100644 index 00000000000..7f7d6405ae9 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineStorageUtils.java @@ -0,0 +1,375 @@ +/** + * 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.timelineservice.storage.common; + +import java.io.IOException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilter.TimelineFilterType; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValuesFilter; + +/** + * A bunch of utility functions used across TimelineReader and TimelineWriter. + */ +@Public +@Unstable +public final class TimelineStorageUtils { + private TimelineStorageUtils() { + } + + /** + * Matches key-values filter. Used for relatesTo/isRelatedTo filters. + * + * @param entity entity which holds relatesTo/isRelatedTo relations which we + * will match against. + * @param keyValuesFilter key-values filter. + * @param entityFiltersType type of filters we are trying to match. + * @return true, if filter matches, false otherwise. + */ + private static boolean matchKeyValuesFilter(TimelineEntity entity, + TimelineKeyValuesFilter keyValuesFilter, + TimelineEntityFiltersType entityFiltersType) { + Map> relations = null; + if (entityFiltersType == TimelineEntityFiltersType.IS_RELATED_TO) { + relations = entity.getIsRelatedToEntities(); + } else if (entityFiltersType == TimelineEntityFiltersType.RELATES_TO) { + relations = entity.getRelatesToEntities(); + } + if (relations == null) { + return false; + } + Set ids = relations.get(keyValuesFilter.getKey()); + if (ids == null) { + return false; + } + boolean matched = false; + for (Object id : keyValuesFilter.getValues()) { + // Matches if id is found amongst the relationships for an entity and + // filter's compare op is EQUAL. + // If compare op is NOT_EQUAL, for a match to occur, id should not be + // found amongst relationships for an entity. + matched = !(ids.contains(id) ^ + keyValuesFilter.getCompareOp() == TimelineCompareOp.EQUAL); + if (!matched) { + return false; + } + } + return true; + } + + /** + * Matches relatesto. + * + * @param entity entity which holds relatesto relations. + * @param relatesTo the relations for filtering. + * @return true, if filter matches, false otherwise. + * @throws IOException if an unsupported filter for matching relations is + * being matched. + */ + public static boolean matchRelatesTo(TimelineEntity entity, + TimelineFilterList relatesTo) throws IOException { + return matchFilters( + entity, relatesTo, TimelineEntityFiltersType.RELATES_TO); + } + + /** + * Matches isrelatedto. + * + * @param entity entity which holds isRelatedTo relations. + * @param isRelatedTo the relations for filtering. + * @return true, if filter matches, false otherwise. + * @throws IOException if an unsupported filter for matching relations is + * being matched. + */ + public static boolean matchIsRelatedTo(TimelineEntity entity, + TimelineFilterList isRelatedTo) throws IOException { + return matchFilters( + entity, isRelatedTo, TimelineEntityFiltersType.IS_RELATED_TO); + } + + /** + * Matches key-value filter. Used for config and info filters. + * + * @param entity entity which holds the config/info which we will match + * against. + * @param kvFilter a key-value filter. + * @param entityFiltersType type of filters we are trying to match. + * @return true, if filter matches, false otherwise. + */ + private static boolean matchKeyValueFilter(TimelineEntity entity, + TimelineKeyValueFilter kvFilter, + TimelineEntityFiltersType entityFiltersType) { + Map map = null; + // Supported only for config and info filters. + if (entityFiltersType == TimelineEntityFiltersType.CONFIG) { + map = entity.getConfigs(); + } else if (entityFiltersType == TimelineEntityFiltersType.INFO) { + map = entity.getInfo(); + } + if (map == null) { + return false; + } + Object value = map.get(kvFilter.getKey()); + if (value == null) { + return false; + } + // Matches if filter's value is equal to the value of the key and filter's + // compare op is EQUAL. + // If compare op is NOT_EQUAL, for a match to occur, value should not be + // equal to the value of the key. + return !(value.equals(kvFilter.getValue()) ^ + kvFilter.getCompareOp() == TimelineCompareOp.EQUAL); + } + + /** + * Matches config filters. + * + * @param entity entity which holds a map of config key-value pairs. + * @param configFilters list of info filters. + * @return a boolean flag to indicate if both match. + * @throws IOException if an unsupported filter for matching config filters is + * being matched. + */ + public static boolean matchConfigFilters(TimelineEntity entity, + TimelineFilterList configFilters) throws IOException { + return + matchFilters(entity, configFilters, TimelineEntityFiltersType.CONFIG); + } + + /** + * Matches info filters. + * + * @param entity entity which holds a map of info key-value pairs. + * @param infoFilters list of info filters. + * @return a boolean flag to indicate if both match. + * @throws IOException if an unsupported filter for matching info filters is + * being matched. + */ + public static boolean matchInfoFilters(TimelineEntity entity, + TimelineFilterList infoFilters) throws IOException { + return matchFilters(entity, infoFilters, TimelineEntityFiltersType.INFO); + } + + /** + * Matches exists filter. Used for event filters. + * + * @param entity entity which holds the events which we will match against. + * @param existsFilter exists filter. + * @param entityFiltersType type of filters we are trying to match. + * @return true, if filter matches, false otherwise. + */ + private static boolean matchExistsFilter(TimelineEntity entity, + TimelineExistsFilter existsFilter, + TimelineEntityFiltersType entityFiltersType) { + // Currently exists filter is only supported for event filters. + if (entityFiltersType != TimelineEntityFiltersType.EVENT) { + return false; + } + Set eventIds = new HashSet(); + for (TimelineEvent event : entity.getEvents()) { + eventIds.add(event.getId()); + } + // Matches if filter's value is contained in the list of events filter's + // compare op is EQUAL. + // If compare op is NOT_EQUAL, for a match to occur, value should not be + // contained in the list of events. + return !(eventIds.contains(existsFilter.getValue()) ^ + existsFilter.getCompareOp() == TimelineCompareOp.EQUAL); + } + + /** + * Matches event filters. + * + * @param entity entity which holds a set of event objects. + * @param eventFilters the set of event Ids for filtering. + * @return a boolean flag to indicate if both match. + * @throws IOException if an unsupported filter for matching event filters is + * being matched. + */ + public static boolean matchEventFilters(TimelineEntity entity, + TimelineFilterList eventFilters) throws IOException { + return matchFilters(entity, eventFilters, TimelineEntityFiltersType.EVENT); + } + + /** + * Compare two values based on comparison operator. + * + * @param compareOp comparison operator. + * @param val1 value 1. + * @param val2 value 2. + * @return true, if relation matches, false otherwise + */ + private static boolean compareValues(TimelineCompareOp compareOp, + long val1, long val2) { + switch (compareOp) { + case LESS_THAN: + return val1 < val2; + case LESS_OR_EQUAL: + return val1 <= val2; + case EQUAL: + return val1 == val2; + case NOT_EQUAL: + return val1 != val2; + case GREATER_OR_EQUAL: + return val1 >= val2; + case GREATER_THAN: + return val1 > val2; + default: + throw new RuntimeException("Unknown TimelineCompareOp " + + compareOp.name()); + } + } + + /** + * Matches compare filter. Used for metric filters. + * + * @param entity entity which holds the metrics which we will match against. + * @param compareFilter compare filter. + * @param entityFiltersType type of filters we are trying to match. + * @return true, if filter matches, false otherwise. + * @throws IOException if metric filters holds non integral values. + */ + private static boolean matchCompareFilter(TimelineEntity entity, + TimelineCompareFilter compareFilter, + TimelineEntityFiltersType entityFiltersType) throws IOException { + // Currently exists filter is only supported for metric filters. + if (entityFiltersType != TimelineEntityFiltersType.METRIC) { + return false; + } + // We expect only integral values(short/int/long) for metric filters. + if (!isIntegralValue(compareFilter.getValue())) { + throw new IOException("Metric filters has non integral values"); + } + Map metricMap = + new HashMap(); + for (TimelineMetric metric : entity.getMetrics()) { + metricMap.put(metric.getId(), metric); + } + TimelineMetric metric = metricMap.get(compareFilter.getKey()); + if (metric == null) { + return false; + } + // We will be using the latest value of metric to compare. + return compareValues(compareFilter.getCompareOp(), + metric.getValuesJAXB().firstEntry().getValue().longValue(), + ((Number)compareFilter.getValue()).longValue()); + } + + /** + * Matches metric filters. + * + * @param entity entity which holds a set of metric objects. + * @param metricFilters list of metric filters. + * @return a boolean flag to indicate if both match. + * @throws IOException if an unsupported filter for matching metric filters is + * being matched. + */ + public static boolean matchMetricFilters(TimelineEntity entity, + TimelineFilterList metricFilters) throws IOException { + return matchFilters( + entity, metricFilters, TimelineEntityFiltersType.METRIC); + } + + /** + * Common routine to match different filters. Iterates over a filter list and + * calls routines based on filter type. + * + * @param entity Timeline entity. + * @param filters filter list. + * @param entityFiltersType type of filters which are being matched. + * @return a boolean flag to indicate if filter matches. + * @throws IOException if an unsupported filter for matching this specific + * filter is being matched. + */ + private static boolean matchFilters(TimelineEntity entity, + TimelineFilterList filters, TimelineEntityFiltersType entityFiltersType) + throws IOException { + if (filters == null || filters.getFilterList().isEmpty()) { + return false; + } + TimelineFilterList.Operator operator = filters.getOperator(); + for (TimelineFilter filter : filters.getFilterList()) { + TimelineFilterType filterType = filter.getFilterType(); + if (!entityFiltersType.isValidFilter(filterType)) { + throw new IOException("Unsupported filter " + filterType); + } + boolean matched = false; + switch (filterType) { + case LIST: + matched = matchFilters(entity, (TimelineFilterList)filter, + entityFiltersType); + break; + case COMPARE: + matched = matchCompareFilter(entity, (TimelineCompareFilter)filter, + entityFiltersType); + break; + case EXISTS: + matched = matchExistsFilter(entity, (TimelineExistsFilter)filter, + entityFiltersType); + break; + case KEY_VALUE: + matched = matchKeyValueFilter(entity, (TimelineKeyValueFilter)filter, + entityFiltersType); + break; + case KEY_VALUES: + matched = matchKeyValuesFilter(entity, (TimelineKeyValuesFilter)filter, + entityFiltersType); + break; + default: + throw new IOException("Unsupported filter " + filterType); + } + if (!matched) { + if(operator == TimelineFilterList.Operator.AND) { + return false; + } + } else { + if(operator == TimelineFilterList.Operator.OR) { + return true; + } + } + } + return operator == TimelineFilterList.Operator.AND; + } + + /** + * Checks if passed object is of integral type(Short/Integer/Long). + * + * @param obj Object to be checked. + * @return true if object passed is of type Short or Integer or Long, false + * otherwise. + */ + public static boolean isIntegralValue(Object obj) { + return (obj instanceof Short) || (obj instanceof Integer) || + (obj instanceof Long); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/package-info.java new file mode 100644 index 00000000000..0df5b8af842 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/package-info.java @@ -0,0 +1,28 @@ +/* + * 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.timelineservice.storage.common contains + * a set of utility classes used across backend storage reader and writer. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/package-info.java new file mode 100644 index 00000000000..e78db2a1ef5 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/package-info.java @@ -0,0 +1,28 @@ +/* + * 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.timelineservice.storage contains + * classes which define and implement reading and writing to backend storage. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.timelineservice.storage; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/resources/META-INF/services/org.apache.hadoop.security.SecurityInfo b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/resources/META-INF/services/org.apache.hadoop.security.SecurityInfo new file mode 100644 index 00000000000..4389219ab74 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/resources/META-INF/services/org.apache.hadoop.security.SecurityInfo @@ -0,0 +1,14 @@ +# +# Licensed 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. +# +org.apache.hadoop.yarn.server.timelineservice.security.CollectorNodemanagerSecurityInfo diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestNMTimelineCollectorManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestNMTimelineCollectorManager.java new file mode 100644 index 00000000000..e076395433e --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestNMTimelineCollectorManager.java @@ -0,0 +1,173 @@ + +/** + * 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.timelineservice.collector; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocol; +import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextResponse; +import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +/** + * Tests node level timeline collector manager. + */ +public class TestNMTimelineCollectorManager { + private NodeTimelineCollectorManager collectorManager; + + @Before + public void setup() throws Exception { + collectorManager = createCollectorManager(); + Configuration conf = new YarnConfiguration(); + conf.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS, + FileSystemTimelineWriterImpl.class, TimelineWriter.class); + collectorManager.init(conf); + collectorManager.start(); + } + + @After + public void tearDown() throws Exception { + if (collectorManager != null) { + collectorManager.stop(); + } + } + + @Test + public void testStartingWriterFlusher() throws Exception { + assertTrue(collectorManager.writerFlusherRunning()); + } + + @Test + public void testStartWebApp() throws Exception { + assertNotNull(collectorManager.getRestServerBindAddress()); + String address = collectorManager.getRestServerBindAddress(); + String[] parts = address.split(":"); + assertEquals(2, parts.length); + assertNotNull(parts[0]); + assertTrue(Integer.valueOf(parts[1]) > 0); + } + + @Test(timeout=60000) + public void testMultithreadedAdd() throws Exception { + final int numApps = 5; + List> tasks = new ArrayList>(); + for (int i = 0; i < numApps; i++) { + final ApplicationId appId = ApplicationId.newInstance(0L, i); + Callable task = new Callable() { + public Boolean call() { + AppLevelTimelineCollector collector = + new AppLevelTimelineCollectorWithAgg(appId, "user"); + return (collectorManager.putIfAbsent(appId, collector) == collector); + } + }; + tasks.add(task); + } + ExecutorService executor = Executors.newFixedThreadPool(numApps); + try { + List> futures = executor.invokeAll(tasks); + for (Future future: futures) { + assertTrue(future.get()); + } + } finally { + executor.shutdownNow(); + } + // check the keys + for (int i = 0; i < numApps; i++) { + final ApplicationId appId = ApplicationId.newInstance(0L, i); + assertTrue(collectorManager.containsTimelineCollector(appId)); + } + } + + @Test + public void testMultithreadedAddAndRemove() throws Exception { + final int numApps = 5; + List> tasks = new ArrayList>(); + for (int i = 0; i < numApps; i++) { + final ApplicationId appId = ApplicationId.newInstance(0L, i); + Callable task = new Callable() { + public Boolean call() { + AppLevelTimelineCollector collector = + new AppLevelTimelineCollectorWithAgg(appId, "user"); + boolean successPut = + (collectorManager.putIfAbsent(appId, collector) == collector); + return successPut && collectorManager.remove(appId); + } + }; + tasks.add(task); + } + ExecutorService executor = Executors.newFixedThreadPool(numApps); + try { + List> futures = executor.invokeAll(tasks); + for (Future future: futures) { + assertTrue(future.get()); + } + } finally { + executor.shutdownNow(); + } + // check the keys + for (int i = 0; i < numApps; i++) { + final ApplicationId appId = ApplicationId.newInstance(0L, i); + assertFalse(collectorManager.containsTimelineCollector(appId)); + } + } + + private NodeTimelineCollectorManager createCollectorManager() { + final NodeTimelineCollectorManager cm = + spy(new NodeTimelineCollectorManager()); + doReturn(new Configuration()).when(cm).getConfig(); + CollectorNodemanagerProtocol nmCollectorService = + mock(CollectorNodemanagerProtocol.class); + GetTimelineCollectorContextResponse response = + GetTimelineCollectorContextResponse.newInstance(null, null, null, 0L); + try { + when(nmCollectorService.getTimelineCollectorContext(any( + GetTimelineCollectorContextRequest.class))).thenReturn(response); + } catch (YarnException | IOException e) { + fail(); + } + doReturn(nmCollectorService).when(cm).getNMCollectorService(); + return cm; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestPerNodeTimelineCollectorsAuxService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestPerNodeTimelineCollectorsAuxService.java new file mode 100644 index 00000000000..a1d4aa98143 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestPerNodeTimelineCollectorsAuxService.java @@ -0,0 +1,207 @@ +/** + * 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.timelineservice.collector; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.util.ExitUtil; +import org.apache.hadoop.util.Shell; +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.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocol; +import org.apache.hadoop.yarn.server.api.ContainerInitializationContext; +import org.apache.hadoop.yarn.server.api.ContainerTerminationContext; +import org.apache.hadoop.yarn.server.api.ContainerType; +import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextRequest; +import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextResponse; +import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter; +import org.junit.After; +import org.junit.Test; + +/** + * Tests the NodeManager auxiliary service started for node level timeline + * collector. + */ +public class TestPerNodeTimelineCollectorsAuxService { + private ApplicationAttemptId appAttemptId; + private PerNodeTimelineCollectorsAuxService auxService; + private Configuration conf; + + public TestPerNodeTimelineCollectorsAuxService() { + ApplicationId appId = + ApplicationId.newInstance(System.currentTimeMillis(), 1); + appAttemptId = ApplicationAttemptId.newInstance(appId, 1); + conf = new YarnConfiguration(); + // enable timeline service v.2 + conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); + conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f); + conf.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS, + FileSystemTimelineWriterImpl.class, TimelineWriter.class); + } + + @After + public void tearDown() throws Shell.ExitCodeException { + if (auxService != null) { + auxService.stop(); + } + } + + @Test + public void testAddApplication() throws Exception { + auxService = createCollectorAndAddApplication(); + // auxService should have a single app + assertTrue(auxService.hasApplication(appAttemptId.getApplicationId())); + auxService.close(); + } + + @Test + public void testAddApplicationNonAMContainer() throws Exception { + auxService = createCollector(); + + ContainerId containerId = getContainerId(2L); // not an AM + ContainerInitializationContext context = + mock(ContainerInitializationContext.class); + when(context.getContainerId()).thenReturn(containerId); + auxService.initializeContainer(context); + // auxService should not have that app + assertFalse(auxService.hasApplication(appAttemptId.getApplicationId())); + } + + @Test + public void testRemoveApplication() throws Exception { + auxService = createCollectorAndAddApplication(); + // auxService should have a single app + assertTrue(auxService.hasApplication(appAttemptId.getApplicationId())); + + ContainerId containerId = getAMContainerId(); + ContainerTerminationContext context = + mock(ContainerTerminationContext.class); + when(context.getContainerId()).thenReturn(containerId); + when(context.getContainerType()).thenReturn( + ContainerType.APPLICATION_MASTER); + auxService.stopContainer(context); + // auxService should have the app's collector and need to remove only after + // a configured period + assertTrue(auxService.hasApplication(appAttemptId.getApplicationId())); + for (int i = 0; i < 4; i++) { + Thread.sleep(500L); + if (!auxService.hasApplication(appAttemptId.getApplicationId())) { + break; + } + } + + // auxService should not have that app + assertFalse(auxService.hasApplication(appAttemptId.getApplicationId())); + auxService.close(); + } + + @Test + public void testRemoveApplicationNonAMContainer() throws Exception { + auxService = createCollectorAndAddApplication(); + // auxService should have a single app + assertTrue(auxService.hasApplication(appAttemptId.getApplicationId())); + + ContainerId containerId = getContainerId(2L); // not an AM + ContainerTerminationContext context = + mock(ContainerTerminationContext.class); + when(context.getContainerId()).thenReturn(containerId); + auxService.stopContainer(context); + // auxService should still have that app + assertTrue(auxService.hasApplication(appAttemptId.getApplicationId())); + auxService.close(); + } + + @Test(timeout = 60000) + public void testLaunch() throws Exception { + ExitUtil.disableSystemExit(); + try { + auxService = + PerNodeTimelineCollectorsAuxService.launchServer(new String[0], + createCollectorManager(), conf); + } catch (ExitUtil.ExitException e) { + assertEquals(0, e.status); + ExitUtil.resetFirstExitException(); + fail(); + } + } + + private PerNodeTimelineCollectorsAuxService + createCollectorAndAddApplication() { + PerNodeTimelineCollectorsAuxService service = createCollector(); + // create an AM container + ContainerId containerId = getAMContainerId(); + ContainerInitializationContext context = + mock(ContainerInitializationContext.class); + when(context.getContainerId()).thenReturn(containerId); + when(context.getContainerType()).thenReturn( + ContainerType.APPLICATION_MASTER); + service.initializeContainer(context); + return service; + } + + private PerNodeTimelineCollectorsAuxService createCollector() { + NodeTimelineCollectorManager collectorManager = createCollectorManager(); + PerNodeTimelineCollectorsAuxService service = + spy(new PerNodeTimelineCollectorsAuxService(collectorManager)); + service.init(conf); + service.start(); + return service; + } + + private NodeTimelineCollectorManager createCollectorManager() { + NodeTimelineCollectorManager collectorManager = + spy(new NodeTimelineCollectorManager()); + doReturn(new Configuration()).when(collectorManager).getConfig(); + CollectorNodemanagerProtocol nmCollectorService = + mock(CollectorNodemanagerProtocol.class); + GetTimelineCollectorContextResponse response = + GetTimelineCollectorContextResponse.newInstance(null, null, null, 0L); + try { + when(nmCollectorService.getTimelineCollectorContext(any( + GetTimelineCollectorContextRequest.class))).thenReturn(response); + } catch (YarnException | IOException e) { + fail(); + } + doReturn(nmCollectorService).when(collectorManager).getNMCollectorService(); + return collectorManager; + } + + private ContainerId getAMContainerId() { + return getContainerId(1L); + } + + private ContainerId getContainerId(long id) { + return ContainerId.newContainerId(appAttemptId, id); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.java new file mode 100644 index 00000000000..7c4d989ce04 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.java @@ -0,0 +1,282 @@ +/** + * 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.timelineservice.collector; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetricOperation; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollector.AggregationStatusTable; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter; +import org.junit.Test; + +import com.google.common.collect.Sets; + +import java.io.IOException; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +/** + * Tests functionality of timeline collector. + */ +public class TestTimelineCollector { + + private TimelineEntities generateTestEntities(int groups, int entities) { + TimelineEntities te = new TimelineEntities(); + for (int j = 0; j < groups; j++) { + for (int i = 0; i < entities; i++) { + TimelineEntity entity = new TimelineEntity(); + String containerId = "container_1000178881110_2002_" + i; + entity.setId(containerId); + String entityType = "TEST_" + j; + entity.setType(entityType); + long cTime = 1425016501000L; + entity.setCreatedTime(cTime); + + // add metrics + Set metrics = new HashSet<>(); + TimelineMetric m1 = new TimelineMetric(); + m1.setId("HDFS_BYTES_WRITE"); + m1.setRealtimeAggregationOp(TimelineMetricOperation.SUM); + long ts = System.currentTimeMillis(); + m1.addValue(ts - 20000, 100L); + metrics.add(m1); + + TimelineMetric m2 = new TimelineMetric(); + m2.setId("VCORES_USED"); + m2.setRealtimeAggregationOp(TimelineMetricOperation.SUM); + m2.addValue(ts - 20000, 3L); + metrics.add(m2); + + // m3 should not show up in the aggregation + TimelineMetric m3 = new TimelineMetric(); + m3.setId("UNRELATED_VALUES"); + m3.addValue(ts - 20000, 3L); + metrics.add(m3); + + TimelineMetric m4 = new TimelineMetric(); + m4.setId("TXN_FINISH_TIME"); + m4.setRealtimeAggregationOp(TimelineMetricOperation.MAX); + m4.addValue(ts - 20000, i); + metrics.add(m4); + + entity.addMetrics(metrics); + te.addEntity(entity); + } + } + + return te; + } + + @Test + public void testAggregation() throws Exception { + // Test aggregation with multiple groups. + int groups = 3; + int n = 50; + TimelineEntities testEntities = generateTestEntities(groups, n); + TimelineEntity resultEntity = TimelineCollector.aggregateEntities( + testEntities, "test_result", "TEST_AGGR", true); + assertEquals(resultEntity.getMetrics().size(), groups * 3); + + for (int i = 0; i < groups; i++) { + Set metrics = resultEntity.getMetrics(); + for (TimelineMetric m : metrics) { + if (m.getId().startsWith("HDFS_BYTES_WRITE")) { + assertEquals(100 * n, m.getSingleDataValue().intValue()); + } else if (m.getId().startsWith("VCORES_USED")) { + assertEquals(3 * n, m.getSingleDataValue().intValue()); + } else if (m.getId().startsWith("TXN_FINISH_TIME")) { + assertEquals(n - 1, m.getSingleDataValue()); + } else { + fail("Unrecognized metric! " + m.getId()); + } + } + } + + // Test aggregation with a single group. + TimelineEntities testEntities1 = generateTestEntities(1, n); + TimelineEntity resultEntity1 = TimelineCollector.aggregateEntities( + testEntities1, "test_result", "TEST_AGGR", false); + assertEquals(resultEntity1.getMetrics().size(), 3); + + Set metrics = resultEntity1.getMetrics(); + for (TimelineMetric m : metrics) { + if (m.getId().equals("HDFS_BYTES_WRITE")) { + assertEquals(100 * n, m.getSingleDataValue().intValue()); + } else if (m.getId().equals("VCORES_USED")) { + assertEquals(3 * n, m.getSingleDataValue().intValue()); + } else if (m.getId().equals("TXN_FINISH_TIME")) { + assertEquals(n - 1, m.getSingleDataValue()); + } else { + fail("Unrecognized metric! " + m.getId()); + } + } + + } + + /** + * Test TimelineCollector's interaction with TimelineWriter upon + * putEntity() calls. + */ + @Test + public void testPutEntity() throws IOException { + TimelineWriter writer = mock(TimelineWriter.class); + TimelineCollector collector = new TimelineCollectorForTest(writer); + + TimelineEntities entities = generateTestEntities(1, 1); + collector.putEntities( + entities, UserGroupInformation.createRemoteUser("test-user")); + + verify(writer, times(1)).write(any(TimelineCollectorContext.class), + any(TimelineEntities.class), any(UserGroupInformation.class)); + verify(writer, times(1)).flush(); + } + + /** + * Test TimelineCollector's interaction with TimelineWriter upon + * putEntityAsync() calls. + */ + @Test + public void testPutEntityAsync() throws IOException { + TimelineWriter writer = mock(TimelineWriter.class); + TimelineCollector collector = new TimelineCollectorForTest(writer); + + TimelineEntities entities = generateTestEntities(1, 1); + collector.putEntitiesAsync( + entities, UserGroupInformation.createRemoteUser("test-user")); + + verify(writer, times(1)).write(any(TimelineCollectorContext.class), + any(TimelineEntities.class), any(UserGroupInformation.class)); + verify(writer, never()).flush(); + } + + private static class TimelineCollectorForTest extends TimelineCollector { + private final TimelineCollectorContext context = + new TimelineCollectorContext(); + + TimelineCollectorForTest(TimelineWriter writer) { + super("TimelineCollectorForTest"); + setWriter(writer); + } + + @Override + public TimelineCollectorContext getTimelineEntityContext() { + return context; + } + } + + private static TimelineEntity createEntity(String id, String type) { + TimelineEntity entity = new TimelineEntity(); + entity.setId(id); + entity.setType(type); + return entity; + } + + private static TimelineMetric createDummyMetric(long ts, Long value) { + TimelineMetric metric = new TimelineMetric(); + metric.setId("dummy_metric"); + metric.addValue(ts, value); + metric.setRealtimeAggregationOp(TimelineMetricOperation.SUM); + return metric; + } + + @Test + public void testClearPreviousEntitiesOnAggregation() throws Exception { + final long ts = System.currentTimeMillis(); + TimelineCollector collector = new TimelineCollector("") { + @Override + public TimelineCollectorContext getTimelineEntityContext() { + return new TimelineCollectorContext("cluster", "user", "flow", "1", + 1L, ApplicationId.newInstance(ts, 1).toString()); + } + }; + collector.init(new Configuration()); + collector.setWriter(mock(TimelineWriter.class)); + + // Put 5 entities with different metric values. + TimelineEntities entities = new TimelineEntities(); + for (int i = 1; i <=5; i++) { + TimelineEntity entity = createEntity("e" + i, "type"); + entity.addMetric(createDummyMetric(ts + i, Long.valueOf(i * 50))); + entities.addEntity(entity); + } + collector.putEntities(entities, UserGroupInformation.getCurrentUser()); + + TimelineCollectorContext currContext = collector.getTimelineEntityContext(); + // Aggregate the entities. + Map aggregationGroups + = collector.getAggregationGroups(); + assertEquals(Sets.newHashSet("type"), aggregationGroups.keySet()); + TimelineEntity aggregatedEntity = TimelineCollector. + aggregateWithoutGroupId(aggregationGroups, currContext.getAppId(), + TimelineEntityType.YARN_APPLICATION.toString()); + TimelineMetric aggregatedMetric = + aggregatedEntity.getMetrics().iterator().next(); + assertEquals(750L, aggregatedMetric.getValues().values().iterator().next()); + assertEquals(TimelineMetricOperation.SUM, + aggregatedMetric.getRealtimeAggregationOp()); + + // Aggregate entities. + aggregatedEntity = TimelineCollector. + aggregateWithoutGroupId(aggregationGroups, currContext.getAppId(), + TimelineEntityType.YARN_APPLICATION.toString()); + aggregatedMetric = aggregatedEntity.getMetrics().iterator().next(); + // No values aggregated as no metrics put for an entity between this + // aggregation and the previous one. + assertTrue(aggregatedMetric.getValues().isEmpty()); + assertEquals(TimelineMetricOperation.NOP, + aggregatedMetric.getRealtimeAggregationOp()); + + // Put 3 entities. + entities = new TimelineEntities(); + for (int i = 1; i <=3; i++) { + TimelineEntity entity = createEntity("e" + i, "type"); + entity.addMetric(createDummyMetric(System.currentTimeMillis() + i, 50L)); + entities.addEntity(entity); + } + aggregationGroups = collector.getAggregationGroups(); + collector.putEntities(entities, UserGroupInformation.getCurrentUser()); + + // Aggregate entities. + aggregatedEntity = TimelineCollector. + aggregateWithoutGroupId(aggregationGroups, currContext.getAppId(), + TimelineEntityType.YARN_APPLICATION.toString()); + // Last 3 entities picked up for aggregation. + aggregatedMetric = aggregatedEntity.getMetrics().iterator().next(); + assertEquals(150L, aggregatedMetric.getValues().values().iterator().next()); + assertEquals(TimelineMetricOperation.SUM, + aggregatedMetric.getRealtimeAggregationOp()); + + collector.close(); + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollectorManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollectorManager.java new file mode 100644 index 00000000000..f8e83998311 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollectorManager.java @@ -0,0 +1,74 @@ +/** + * 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.timelineservice.collector; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; +import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter; +import org.junit.Test; + +/** + * Unit tests for TimelineCollectorManager. + */ +public class TestTimelineCollectorManager{ + + @Test(timeout = 60000, expected = YarnRuntimeException.class) + public void testTimelineCollectorManagerWithInvalidTimelineWriter() { + Configuration conf = new YarnConfiguration(); + conf.set(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS, + Object.class.getName()); + runTimelineCollectorManagerWithConfig(conf); + } + + @Test(timeout = 60000, expected = YarnRuntimeException.class) + public void testTimelineCollectorManagerWithNonexistentTimelineWriter() { + String nonexistentTimelineWriterClass = "org.apache.org.yarn.server." + + "timelineservice.storage.XXXXXXXX"; + Configuration conf = new YarnConfiguration(); + conf.set(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS, + nonexistentTimelineWriterClass); + runTimelineCollectorManagerWithConfig(conf); + } + + @Test(timeout = 60000) + public void testTimelineCollectorManagerWithFileSystemWriter() { + Configuration conf = new YarnConfiguration(); + conf.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS, + FileSystemTimelineWriterImpl.class, TimelineWriter.class); + runTimelineCollectorManagerWithConfig(conf); + } + + /** + * Run a TimelineCollectorManager with a given configuration. + * @param conf configuration to run TimelineCollectorManager with + */ + private static void runTimelineCollectorManagerWithConfig( + final Configuration conf) { + TimelineCollectorManager collectorManager = + new TimelineCollectorManager("testTimelineCollectorManager"); + try { + collectorManager.init(conf); + collectorManager.start(); + } finally { + collectorManager.stop(); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderServer.java new file mode 100644 index 00000000000..2d470a5bb2b --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderServer.java @@ -0,0 +1,103 @@ +/** + * 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.timelineservice.reader; + +import static org.junit.Assert.assertEquals; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.Service.STATE; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; +import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineReaderImpl; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader; +import org.junit.Test; + +/** + * Tests {@link TimelineReaderServer} start and stop. + */ +public class TestTimelineReaderServer { + + @Test(timeout = 60000) + public void testStartStopServer() throws Exception { + @SuppressWarnings("resource") + TimelineReaderServer server = new TimelineReaderServer(); + Configuration config = new YarnConfiguration(); + config.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); + config.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f); + config.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS, + "localhost:0"); + config.setClass(YarnConfiguration.TIMELINE_SERVICE_READER_CLASS, + FileSystemTimelineReaderImpl.class, TimelineReader.class); + try { + server.init(config); + assertEquals(STATE.INITED, server.getServiceState()); + assertEquals(2, server.getServices().size()); + + server.start(); + assertEquals(STATE.STARTED, server.getServiceState()); + + server.stop(); + assertEquals(STATE.STOPPED, server.getServiceState()); + } finally { + server.stop(); + } + } + + @Test(timeout = 60000, expected = YarnRuntimeException.class) + public void testTimelineReaderServerWithInvalidTimelineReader() { + Configuration conf = new YarnConfiguration(); + conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); + conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f); + conf.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS, + "localhost:0"); + conf.set(YarnConfiguration.TIMELINE_SERVICE_READER_CLASS, + Object.class.getName()); + runTimelineReaderServerWithConfig(conf); + } + + @Test(timeout = 60000, expected = YarnRuntimeException.class) + public void testTimelineReaderServerWithNonexistentTimelineReader() { + String nonexistentTimelineReaderClass = "org.apache.org.yarn.server." + + "timelineservice.storage.XXXXXXXX"; + Configuration conf = new YarnConfiguration(); + conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); + conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f); + conf.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS, + "localhost:0"); + conf.set(YarnConfiguration.TIMELINE_SERVICE_READER_CLASS, + nonexistentTimelineReaderClass); + runTimelineReaderServerWithConfig(conf); + } + + /** + * Run a TimelineReaderServer with a given configuration. + * @param conf configuration to run TimelineReaderServer with + */ + private static void runTimelineReaderServerWithConfig( + final Configuration conf) { + TimelineReaderServer server = new TimelineReaderServer(); + try { + server.init(conf); + server.start(); + } finally { + server.stop(); + } + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderUtils.java new file mode 100644 index 00000000000..b8ff951f60f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderUtils.java @@ -0,0 +1,58 @@ +/** + * 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.timelineservice.reader; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import java.util.List; + +import org.junit.Test; + +/** + * Tests helper methods in {@link TimelineReaderUtils}. + */ +public class TestTimelineReaderUtils { + + @Test + public void testSplitUsingEscapeAndDelimChar() throws Exception { + List list = + TimelineReaderUtils.split("*!cluster!*!b**o***!xer!oozie**", '!', '*'); + String[] arr = new String[list.size()]; + arr = list.toArray(arr); + assertArrayEquals(new String[] {"!cluster", "!b*o*!xer", "oozie*"}, arr); + list = TimelineReaderUtils.split("*!cluster!*!b**o***!xer!!", '!', '*'); + arr = new String[list.size()]; + arr = list.toArray(arr); + assertArrayEquals(new String[] {"!cluster", "!b*o*!xer", "", ""}, arr); + } + + @Test + public void testJoinAndEscapeStrings() throws Exception { + assertEquals("*!cluster!*!b**o***!xer!oozie**", + TimelineReaderUtils.joinAndEscapeStrings( + new String[] {"!cluster", "!b*o*!xer", "oozie*"}, '!', '*')); + assertEquals("*!cluster!*!b**o***!xer!!", + TimelineReaderUtils.joinAndEscapeStrings( + new String[] {"!cluster", "!b*o*!xer", "", ""}, '!', '*')); + assertNull(TimelineReaderUtils.joinAndEscapeStrings( + new String[] {"!cluster", "!b*o*!xer", null, ""}, '!', '*')); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java new file mode 100644 index 00000000000..476959cf7d4 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java @@ -0,0 +1,757 @@ +/** + * 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.timelineservice.reader; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.io.IOException; +import java.lang.reflect.UndeclaredThrowableException; +import java.net.HttpURLConnection; +import java.net.URI; +import java.net.URL; +import java.util.Set; + +import javax.ws.rs.core.MediaType; + +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.timeline.TimelineAbout; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineReaderImpl; +import org.apache.hadoop.yarn.server.timelineservice.storage.TestFileSystemTimelineReaderImpl; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader; +import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import com.sun.jersey.api.client.Client; +import com.sun.jersey.api.client.ClientResponse; +import com.sun.jersey.api.client.ClientResponse.Status; +import com.sun.jersey.api.client.GenericType; +import com.sun.jersey.api.client.config.ClientConfig; +import com.sun.jersey.api.client.config.DefaultClientConfig; +import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory; +import com.sun.jersey.client.urlconnection.URLConnectionClientHandler; + +/** + * Test TimelineReder Web Service REST API's using filesystem storage. + */ +public class TestTimelineReaderWebServices { + + private static final String ROOT_DIR = new File("target", + TestTimelineReaderWebServices.class.getSimpleName()).getAbsolutePath(); + + private int serverPort; + private TimelineReaderServer server; + + @BeforeClass + public static void setup() throws Exception { + TestFileSystemTimelineReaderImpl.initializeDataDirectory(ROOT_DIR); + } + + @AfterClass + public static void tearDown() throws Exception { + FileUtils.deleteDirectory(new File(ROOT_DIR)); + } + + @Before + public void init() throws Exception { + try { + Configuration config = new YarnConfiguration(); + config.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); + config.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f); + config.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS, + "localhost:0"); + config.set(YarnConfiguration.RM_CLUSTER_ID, "cluster1"); + config.setClass(YarnConfiguration.TIMELINE_SERVICE_READER_CLASS, + FileSystemTimelineReaderImpl.class, TimelineReader.class); + config.set(FileSystemTimelineReaderImpl.TIMELINE_SERVICE_STORAGE_DIR_ROOT, + ROOT_DIR); + server = new TimelineReaderServer(); + server.init(config); + server.start(); + serverPort = server.getWebServerPort(); + } catch (Exception e) { + Assert.fail("Web server failed to start"); + } + } + + @After + public void stop() throws Exception { + if (server != null) { + server.stop(); + server = null; + } + } + + private static TimelineEntity newEntity(String type, String id) { + TimelineEntity entity = new TimelineEntity(); + entity.setIdentifier(new TimelineEntity.Identifier(type, id)); + return entity; + } + + private static void verifyHttpResponse(Client client, URI uri, + Status expectedStatus) { + ClientResponse resp = + client.resource(uri).accept(MediaType.APPLICATION_JSON) + .type(MediaType.APPLICATION_JSON).get(ClientResponse.class); + assertNotNull(resp); + assertEquals(resp.getClientResponseStatus().getStatusCode(), + expectedStatus.getStatusCode()); + } + + private static Client createClient() { + ClientConfig cfg = new DefaultClientConfig(); + cfg.getClasses().add(YarnJacksonJaxbJsonProvider.class); + return new Client(new URLConnectionClientHandler( + new DummyURLConnectionFactory()), cfg); + } + + private static ClientResponse getResponse(Client client, URI uri) + throws Exception { + ClientResponse resp = + client.resource(uri).accept(MediaType.APPLICATION_JSON) + .type(MediaType.APPLICATION_JSON).get(ClientResponse.class); + if (resp == null || + resp.getClientResponseStatus() != ClientResponse.Status.OK) { + String msg = new String(); + if (resp != null) { + msg = String.valueOf(resp.getClientResponseStatus()); + } + throw new IOException("Incorrect response from timeline reader. " + + "Status=" + msg); + } + return resp; + } + + private static class DummyURLConnectionFactory + implements HttpURLConnectionFactory { + + @Override + public HttpURLConnection getHttpURLConnection(final URL url) + throws IOException { + try { + return (HttpURLConnection)url.openConnection(); + } catch (UndeclaredThrowableException e) { + throw new IOException(e.getCause()); + } + } + } + + @Test + public void testAbout() throws Exception { + URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/"); + Client client = createClient(); + try { + ClientResponse resp = getResponse(client, uri); + TimelineAbout about = resp.getEntity(TimelineAbout.class); + Assert.assertNotNull(about); + Assert.assertEquals("Timeline Reader API", about.getAbout()); + } finally { + client.destroy(); + } + } + + @Test + public void testGetEntityDefaultView() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/clusters/cluster1/apps/app1/entities/app/id_1"); + ClientResponse resp = getResponse(client, uri); + TimelineEntity entity = resp.getEntity(TimelineEntity.class); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entity); + assertEquals("id_1", entity.getId()); + assertEquals("app", entity.getType()); + assertEquals((Long)1425016502000L, entity.getCreatedTime()); + // Default view i.e. when no fields are specified, entity contains only + // entity id, entity type and created time. + assertEquals(0, entity.getConfigs().size()); + assertEquals(0, entity.getMetrics().size()); + } finally { + client.destroy(); + } + } + + @Test + public void testGetEntityWithUserAndFlowInfo() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/clusters/cluster1/apps/app1/entities/app/id_1?" + + "userid=user1&flowname=flow1&flowrunid=1"); + ClientResponse resp = getResponse(client, uri); + TimelineEntity entity = resp.getEntity(TimelineEntity.class); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entity); + assertEquals("id_1", entity.getId()); + assertEquals("app", entity.getType()); + assertEquals((Long)1425016502000L, entity.getCreatedTime()); + } finally { + client.destroy(); + } + } + + @Test + public void testGetEntityCustomFields() throws Exception { + Client client = createClient(); + try { + // Fields are case insensitive. + URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/clusters/cluster1/apps/app1/entities/app/id_1?" + + "fields=CONFIGS,Metrics,info"); + ClientResponse resp = getResponse(client, uri); + TimelineEntity entity = resp.getEntity(TimelineEntity.class); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entity); + assertEquals("id_1", entity.getId()); + assertEquals("app", entity.getType()); + assertEquals(3, entity.getConfigs().size()); + assertEquals(3, entity.getMetrics().size()); + assertTrue("UID should be present", + entity.getInfo().containsKey(TimelineReaderUtils.UID_KEY)); + // Includes UID. + assertEquals(3, entity.getInfo().size()); + // No events will be returned as events are not part of fields. + assertEquals(0, entity.getEvents().size()); + } finally { + client.destroy(); + } + } + + @Test + public void testGetEntityAllFields() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/clusters/cluster1/apps/app1/entities/app/id_1?" + + "fields=ALL"); + ClientResponse resp = getResponse(client, uri); + TimelineEntity entity = resp.getEntity(TimelineEntity.class); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entity); + assertEquals("id_1", entity.getId()); + assertEquals("app", entity.getType()); + assertEquals(3, entity.getConfigs().size()); + assertEquals(3, entity.getMetrics().size()); + assertTrue("UID should be present", + entity.getInfo().containsKey(TimelineReaderUtils.UID_KEY)); + // Includes UID. + assertEquals(3, entity.getInfo().size()); + assertEquals(2, entity.getEvents().size()); + } finally { + client.destroy(); + } + } + + @Test + public void testGetEntityNotPresent() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/clusters/cluster1/apps/app1/entities/app/id_10"); + verifyHttpResponse(client, uri, Status.NOT_FOUND); + } finally { + client.destroy(); + } + } + + @Test + public void testQueryWithoutCluster() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/apps/app1/entities/app/id_1"); + ClientResponse resp = getResponse(client, uri); + TimelineEntity entity = resp.getEntity(TimelineEntity.class); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entity); + assertEquals("id_1", entity.getId()); + assertEquals("app", entity.getType()); + + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/apps/app1/entities/app"); + resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + assertEquals(4, entities.size()); + } finally { + client.destroy(); + } + } + + @Test + public void testGetEntities() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/clusters/cluster1/apps/app1/entities/app"); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + assertEquals(4, entities.size()); + assertTrue("Entities id_1, id_2, id_3 and id_4 should have been" + + " present in response", + entities.contains(newEntity("app", "id_1")) && + entities.contains(newEntity("app", "id_2")) && + entities.contains(newEntity("app", "id_3")) && + entities.contains(newEntity("app", "id_4"))); + } finally { + client.destroy(); + } + } + + @Test + public void testGetEntitiesWithLimit() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/clusters/cluster1/apps/app1/entities/app?limit=2"); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + assertEquals(2, entities.size()); + // Entities returned are based on most recent created time. + assertTrue("Entities with id_1 and id_4 should have been present " + + "in response based on entity created time.", + entities.contains(newEntity("app", "id_1")) && + entities.contains(newEntity("app", "id_4"))); + + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" + + "clusters/cluster1/apps/app1/entities/app?limit=3"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + // Even though 2 entities out of 4 have same created time, one entity + // is left out due to limit + assertEquals(3, entities.size()); + } finally { + client.destroy(); + } + } + + @Test + public void testGetEntitiesBasedOnCreatedTime() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/clusters/cluster1/apps/app1/entities/app?" + + "createdtimestart=1425016502030&createdtimeend=1425016502060"); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + assertEquals(1, entities.size()); + assertTrue("Entity with id_4 should have been present in response.", + entities.contains(newEntity("app", "id_4"))); + + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" + + "clusters/cluster1/apps/app1/entities/app?createdtimeend" + + "=1425016502010"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + assertEquals(3, entities.size()); + assertFalse("Entity with id_4 should not have been present in response.", + entities.contains(newEntity("app", "id_4"))); + + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" + + "clusters/cluster1/apps/app1/entities/app?createdtimestart=" + + "1425016502010"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + assertEquals(1, entities.size()); + assertTrue("Entity with id_4 should have been present in response.", + entities.contains(newEntity("app", "id_4"))); + } finally { + client.destroy(); + } + } + + @Test + public void testGetEntitiesByRelations() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/clusters/cluster1/apps/app1/entities/app?relatesto=" + + "flow:flow1"); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + assertEquals(1, entities.size()); + assertTrue("Entity with id_1 should have been present in response.", + entities.contains(newEntity("app", "id_1"))); + + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" + + "clusters/cluster1/apps/app1/entities/app?isrelatedto=" + + "type1:tid1_2,type2:tid2_1%60"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + assertEquals(1, entities.size()); + assertTrue("Entity with id_1 should have been present in response.", + entities.contains(newEntity("app", "id_1"))); + + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" + + "clusters/cluster1/apps/app1/entities/app?isrelatedto=" + + "type1:tid1_1:tid1_2,type2:tid2_1%60"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>(){}); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + assertEquals(1, entities.size()); + assertTrue("Entity with id_1 should have been present in response.", + entities.contains(newEntity("app", "id_1"))); + } finally { + client.destroy(); + } + } + + @Test + public void testGetEntitiesByConfigFilters() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/clusters/cluster1/apps/app1/entities/app?" + + "conffilters=config_1%20eq%20123%20AND%20config_3%20eq%20abc"); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + assertEquals(1, entities.size()); + assertTrue("Entity with id_3 should have been present in response.", + entities.contains(newEntity("app", "id_3"))); + } finally { + client.destroy(); + } + } + + @Test + public void testGetEntitiesByInfoFilters() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/clusters/cluster1/apps/app1/entities/app?" + + "infofilters=info2%20eq%203.5"); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + assertEquals(1, entities.size()); + assertTrue("Entity with id_3 should have been present in response.", + entities.contains(newEntity("app", "id_3"))); + } finally { + client.destroy(); + } + } + + @Test + public void testGetEntitiesByMetricFilters() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/clusters/cluster1/apps/app1/entities/app?" + + "metricfilters=metric3%20ge%200"); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + assertEquals(2, entities.size()); + assertTrue("Entities with id_1 and id_2 should have been present" + + " in response.", + entities.contains(newEntity("app", "id_1")) && + entities.contains(newEntity("app", "id_2"))); + } finally { + client.destroy(); + } + } + + @Test + public void testGetEntitiesByEventFilters() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/clusters/cluster1/apps/app1/entities/app?" + + "eventfilters=event_2,event_4"); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + assertEquals(1, entities.size()); + assertTrue("Entity with id_3 should have been present in response.", + entities.contains(newEntity("app", "id_3"))); + } finally { + client.destroy(); + } + } + + @Test + public void testGetEntitiesNoMatch() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/clusters/cluster1/apps/app1/entities/app?" + + "metricfilters=metric7%20ge%200&isrelatedto=type1:tid1_1:tid1_2,"+ + "type2:tid2_1%60&relatesto=flow:flow1&eventfilters=event_2,event_4" + + "&infofilters=info2%20eq%203.5&createdtimestart=1425016502030&" + + "createdtimeend=1425016502060"); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>(){}); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + assertEquals(0, entities.size()); + } finally { + client.destroy(); + } + } + + @Test + public void testInvalidValuesHandling() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/clusters/cluster1/apps/app1/entities/app?flowrunid=a23b"); + verifyHttpResponse(client, uri, Status.BAD_REQUEST); + + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" + + "clusters/cluster1/apps/app1/entities/app/id_1?flowrunid=2ab15"); + verifyHttpResponse(client, uri, Status.BAD_REQUEST); + + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" + + "clusters/cluster1/apps/app1/entities/app?limit=#$561av"); + verifyHttpResponse(client, uri, Status.BAD_REQUEST); + } finally { + client.destroy(); + } + } + + @Test + public void testGetAppAttempts() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/clusters/cluster1/apps/app1/" + + "entities/YARN_APPLICATION_ATTEMPT"); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>() { + }); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + int totalEntities = entities.size(); + assertEquals(2, totalEntities); + assertTrue( + "Entity with app-attempt-2 should have been present in response.", + entities.contains( + newEntity(TimelineEntityType.YARN_APPLICATION_ATTEMPT.toString(), + "app-attempt-1"))); + assertTrue( + "Entity with app-attempt-2 should have been present in response.", + entities.contains( + newEntity(TimelineEntityType.YARN_APPLICATION_ATTEMPT.toString(), + "app-attempt-2"))); + + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/clusters/cluster1/apps/app1/appattempts"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>() { + }); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + int retrievedEntity = entities.size(); + assertEquals(2, retrievedEntity); + assertTrue( + "Entity with app-attempt-2 should have been present in response.", + entities.contains( + newEntity(TimelineEntityType.YARN_APPLICATION_ATTEMPT.toString(), + "app-attempt-1"))); + assertTrue( + "Entity with app-attempt-2 should have been present in response.", + entities.contains( + newEntity(TimelineEntityType.YARN_APPLICATION_ATTEMPT.toString(), + "app-attempt-2"))); + + assertEquals(totalEntities, retrievedEntity); + + } finally { + client.destroy(); + } + } + + @Test + public void testGetAppAttempt() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/clusters/cluster1/apps/app1/entities/" + + "YARN_APPLICATION_ATTEMPT/app-attempt-1"); + ClientResponse resp = getResponse(client, uri); + TimelineEntity entities1 = + resp.getEntity(new GenericType() { + }); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities1); + + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/clusters/cluster1/apps/app1/appattempts/app-attempt-1"); + resp = getResponse(client, uri); + TimelineEntity entities2 = + resp.getEntity(new GenericType() { + }); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities2); + + assertEquals(entities1, entities2); + + } finally { + client.destroy(); + } + } + + @Test + public void testGetContainers() throws Exception { + Client client = createClient(); + try { + // total 3 containers in a application. + URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/clusters/cluster1/apps/app1/entities/YARN_CONTAINER"); + ClientResponse resp = getResponse(client, uri); + Set entities = + resp.getEntity(new GenericType>() { + }); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + int totalEntities = entities.size(); + assertEquals(3, totalEntities); + assertTrue( + "Entity with container_1_1 should have been present in response.", + entities.contains(newEntity( + TimelineEntityType.YARN_CONTAINER.toString(), "container_1_1"))); + assertTrue( + "Entity with container_2_1 should have been present in response.", + entities.contains(newEntity( + TimelineEntityType.YARN_CONTAINER.toString(), "container_2_1"))); + assertTrue( + "Entity with container_2_2 should have been present in response.", + entities.contains(newEntity( + TimelineEntityType.YARN_CONTAINER.toString(), "container_2_2"))); + + // for app-attempt1 1 container has run + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/clusters/cluster1/apps/app1/" + + "appattempts/app-attempt-1/containers"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>() { + }); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + int retrievedEntity = entities.size(); + assertEquals(1, retrievedEntity); + assertTrue( + "Entity with container_1_1 should have been present in response.", + entities.contains(newEntity( + TimelineEntityType.YARN_CONTAINER.toString(), "container_1_1"))); + + // for app-attempt2 2 containers has run + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/clusters/cluster1/apps/app1/" + + "appattempts/app-attempt-2/containers"); + resp = getResponse(client, uri); + entities = resp.getEntity(new GenericType>() { + }); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities); + retrievedEntity += entities.size(); + assertEquals(2, entities.size()); + assertTrue( + "Entity with container_2_1 should have been present in response.", + entities.contains(newEntity( + TimelineEntityType.YARN_CONTAINER.toString(), "container_2_1"))); + assertTrue( + "Entity with container_2_2 should have been present in response.", + entities.contains(newEntity( + TimelineEntityType.YARN_CONTAINER.toString(), "container_2_2"))); + + assertEquals(totalEntities, retrievedEntity); + + } finally { + client.destroy(); + } + } + + @Test + public void testGetContainer() throws Exception { + Client client = createClient(); + try { + URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/clusters/cluster1/apps/app1/" + + "entities/YARN_CONTAINER/container_2_2"); + ClientResponse resp = getResponse(client, uri); + TimelineEntity entities1 = + resp.getEntity(new GenericType() { + }); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities1); + + uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" + + "timeline/clusters/cluster1/apps/app1/containers/container_2_2"); + resp = getResponse(client, uri); + TimelineEntity entities2 = + resp.getEntity(new GenericType() { + }); + assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType()); + assertNotNull(entities2); + + assertEquals(entities1, entities2); + + } finally { + client.destroy(); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesUtils.java new file mode 100644 index 00000000000..13e129ea607 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesUtils.java @@ -0,0 +1,926 @@ +/** + * 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.timelineservice.reader; + +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList.Operator; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValuesFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter; +import org.junit.Test; + +import com.google.common.collect.Sets; + +/** + * Tests helper methods in {@link TimelineReaderWebServicesUtils}. + */ +public class TestTimelineReaderWebServicesUtils { + private static void verifyFilterList(String expr, TimelineFilterList list, + TimelineFilterList expectedList) throws Exception { + assertNotNull(list); + assertTrue("Unexpected List received after parsing expression " + expr + + ". Expected=" + expectedList + " but Actual=" + list, + list.equals(expectedList)); + } + + @Test + public void testMetricFiltersParsing() throws Exception { + String expr = "(((key11 ne 234 AND key12 gt 23) AND " + + "(key13 lt 34 OR key14 ge 567)) OR (key21 lt 24 OR key22 le 45))"; + TimelineFilterList expectedList = new TimelineFilterList( + Operator.OR, + new TimelineFilterList( + Operator.AND, + new TimelineFilterList( + Operator.AND, + new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL, + "key11", 234, false), + new TimelineCompareFilter(TimelineCompareOp.GREATER_THAN, + "key12", 23, true) + ), + new TimelineFilterList( + Operator.OR, + new TimelineCompareFilter(TimelineCompareOp.LESS_THAN, + "key13", 34, true), + new TimelineCompareFilter(TimelineCompareOp.GREATER_OR_EQUAL, + "key14", 567, true) + ) + ), + new TimelineFilterList( + Operator.OR, + new TimelineCompareFilter(TimelineCompareOp.LESS_THAN, + "key21", 24, true), + new TimelineCompareFilter(TimelineCompareOp.LESS_OR_EQUAL, + "key22", 45, true) + ) + ); + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList); + + expr = "abc ene 234"; + expectedList = new TimelineFilterList( + new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL, + "abc", 234, true) + ); + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList); + + expr = "abc ne 234"; + expectedList = new TimelineFilterList( + new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL, + "abc", 234, false) + ); + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList); + + expr = "abc ne 234 AND def gt 23"; + expectedList = new TimelineFilterList( + new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL, + "abc", 234, false), + new TimelineCompareFilter(TimelineCompareOp.GREATER_THAN, + "def", 23, true) + ); + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList); + + expr = "(abc ne 234 AND def gt 23)"; + expectedList = new TimelineFilterList( + new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL, + "abc", 234, false), + new TimelineCompareFilter(TimelineCompareOp.GREATER_THAN, + "def", 23, true) + ); + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList); + + expr = "abc ne 234 AND def gt 23 OR rst lt 24"; + expectedList = new TimelineFilterList( + Operator.OR, + new TimelineFilterList( + new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL, + "abc", 234, false), + new TimelineCompareFilter(TimelineCompareOp.GREATER_THAN, + "def", 23, true) + ), + new TimelineCompareFilter(TimelineCompareOp.LESS_THAN, + "rst", 24, true) + ); + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList); + + expr = "abc ne 234 AND def gt 23 OR rst lt 24 OR xyz le 456"; + expectedList = new TimelineFilterList( + Operator.OR, + new TimelineFilterList( + new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL, + "abc", 234, false), + new TimelineCompareFilter(TimelineCompareOp.GREATER_THAN, + "def", 23, true) + ), + new TimelineCompareFilter(TimelineCompareOp.LESS_THAN, + "rst", 24, true), + new TimelineCompareFilter(TimelineCompareOp.LESS_OR_EQUAL, + "xyz", 456, true) + ); + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList); + + expr = "abc ne 234 AND def gt 23 OR rst lt 24 OR xyz le 456 AND pqr ge 2"; + expectedList = new TimelineFilterList( + new TimelineFilterList( + Operator.OR, + new TimelineFilterList( + new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL, + "abc", 234, false), + new TimelineCompareFilter(TimelineCompareOp.GREATER_THAN, + "def", 23, true) + ), + new TimelineCompareFilter(TimelineCompareOp.LESS_THAN, + "rst", 24, true), + new TimelineCompareFilter(TimelineCompareOp.LESS_OR_EQUAL, + "xyz", 456, true) + ), + new TimelineCompareFilter(TimelineCompareOp.GREATER_OR_EQUAL, + "pqr", 2, true) + ); + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList); + + // Test with unnecessary spaces. + expr = " abc ne 234 AND def gt 23 OR rst lt "+ + " 24 OR xyz le 456 AND pqr ge 2 "; + expectedList = new TimelineFilterList( + new TimelineFilterList( + Operator.OR, + new TimelineFilterList( + new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL, + "abc", 234, false), + new TimelineCompareFilter(TimelineCompareOp.GREATER_THAN, + "def", 23, true) + ), + new TimelineCompareFilter(TimelineCompareOp.LESS_THAN, + "rst", 24, true), + new TimelineCompareFilter(TimelineCompareOp.LESS_OR_EQUAL, + "xyz", 456, true) + ), + new TimelineCompareFilter(TimelineCompareOp.GREATER_OR_EQUAL, + "pqr", 2, true) + ); + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList); + + expr = "(((key11 ne 234 AND key12 gt 23 OR key13 lt 24 OR key14 le 456 " + + "AND key15 ge 2) AND (key16 lt 34 OR key17 ge 567)) OR (key21 lt 24 " + + "OR key22 le 45))"; + expectedList = new TimelineFilterList( + Operator.OR, + new TimelineFilterList( + Operator.AND, + new TimelineFilterList( + new TimelineFilterList( + Operator.OR, + new TimelineFilterList( + new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL, + "key11", 234, false), + new TimelineCompareFilter( + TimelineCompareOp.GREATER_THAN, "key12", 23, true) + ), + new TimelineCompareFilter(TimelineCompareOp.LESS_THAN, + "key13", 24, true), + new TimelineCompareFilter(TimelineCompareOp.LESS_OR_EQUAL, + "key14", 456, true) + ), + new TimelineCompareFilter(TimelineCompareOp.GREATER_OR_EQUAL, + "key15", 2, true) + ), + new TimelineFilterList( + Operator.OR, + new TimelineCompareFilter(TimelineCompareOp.LESS_THAN, + "key16", 34, true), + new TimelineCompareFilter(TimelineCompareOp.GREATER_OR_EQUAL, + "key17", 567, true) + ) + ), + new TimelineFilterList( + Operator.OR, + new TimelineCompareFilter(TimelineCompareOp.LESS_THAN, + "key21", 24, true), + new TimelineCompareFilter(TimelineCompareOp.LESS_OR_EQUAL, + "key22", 45, true) + ) + ); + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList); + + expr = " ( ( ( key11 ne 234 AND key12 gt " + + "23 OR key13 lt 24 OR key14 le 456 AND key15 ge 2" + + " ) AND ( key16 lt 34 OR key17 ge 567 ) ) OR " + + "( key21 lt 24 OR key22 le 45 ) ) "; + expectedList = new TimelineFilterList( + Operator.OR, + new TimelineFilterList( + Operator.AND, + new TimelineFilterList( + new TimelineFilterList( + Operator.OR, + new TimelineFilterList( + new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL, + "key11", 234, false), + new TimelineCompareFilter( + TimelineCompareOp.GREATER_THAN, "key12", 23, true) + ), + new TimelineCompareFilter(TimelineCompareOp.LESS_THAN, + "key13", 24, true), + new TimelineCompareFilter(TimelineCompareOp.LESS_OR_EQUAL, + "key14", 456, true) + ), + new TimelineCompareFilter(TimelineCompareOp.GREATER_OR_EQUAL, + "key15", 2, true) + ), + new TimelineFilterList( + Operator.OR, + new TimelineCompareFilter(TimelineCompareOp.LESS_THAN, + "key16", 34, true), + new TimelineCompareFilter(TimelineCompareOp.GREATER_OR_EQUAL, + "key17", 567, true) + ) + ), + new TimelineFilterList( + Operator.OR, + new TimelineCompareFilter(TimelineCompareOp.LESS_THAN, + "key21", 24, true), + new TimelineCompareFilter(TimelineCompareOp.LESS_OR_EQUAL, + "key22", 45, true) + ) + ); + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList); + + expr = "(((key11 ne 234 AND key12 gt 23 OR key13 lt 24 OR key14 le 456 " + + "AND key15 ge 2) AND (key16 lt 34 OR key17 ge 567)) OR (key21 lt 24 " + + "OR key22 le 45)"; + try { + TimelineReaderWebServicesUtils.parseMetricFilters(expr); + fail("Improper brackers. Exception should have been thrown."); + } catch (TimelineParseException e) {} + + expr = "(((key11 ne 234 AND key12 gt v3 OR key13 lt 24 OR key14 le 456 " + + "AND key15 ge 2) AND (key16 lt 34 OR key17 ge 567)) OR (key21 lt 24 " + + "OR key22 le 45))"; + try { + TimelineReaderWebServicesUtils.parseMetricFilters(expr); + fail("Non Numeric value. Exception should have been thrown."); + } catch (TimelineParseException e) {} + + expr = "(((key11 ne (234 AND key12 gt 3 OR key13 lt 24 OR key14 le 456 " + + "AND key15 ge 2) AND (key16 lt 34 OR key17 ge 567)) OR (key21 lt 24 " + + "OR key22 le 45))"; + try { + TimelineReaderWebServicesUtils.parseMetricFilters(expr); + fail("Unexpected opening bracket. Exception should have been thrown."); + } catch (TimelineParseException e) {} + + expr = "(((k)ey11 ne 234 AND key12 gt 3 OR key13 lt 24 OR key14 le 456 " + + "AND key15 ge 2) AND (key16 lt 34 OR key17 ge 567)) OR (key21 lt 24 " + + "OR key22 le 45))"; + try { + TimelineReaderWebServicesUtils.parseMetricFilters(expr); + fail("Unexpected closing bracket. Exception should have been thrown."); + } catch (TimelineParseException e) {} + + expr = "(((key11 rs 234 AND key12 gt 3 OR key13 lt 24 OR key14 le 456 " + + "AND key15 ge 2) AND (key16 lt 34 OR key17 ge 567)) OR (key21 lt 24 " + + "OR key22 le 45))"; + try { + TimelineReaderWebServicesUtils.parseMetricFilters(expr); + fail("Improper compare op. Exception should have been thrown."); + } catch (TimelineParseException e) {} + + expr = "(((key11 ne 234 PI key12 gt 3 OR key13 lt 24 OR key14 le 456 " + + "AND key15 ge 2) AND (key16 lt 34 OR key17 ge 567)) OR (key21 lt 24 " + + "OR key22 le 45))"; + try { + TimelineReaderWebServicesUtils.parseMetricFilters(expr); + fail("Improper op. Exception should have been thrown."); + } catch (TimelineParseException e) {} + + expr = "(((key11 ne 234 PI key12 gt 3 OR key13 lt 24 OR key14 le 456 " + + "AND key15 ge 2) AND (key16 lt 34 OR key17 ge 567)) OR (key21 lt 24 " + + "OR key22 le 45))"; + try { + TimelineReaderWebServicesUtils.parseMetricFilters(expr); + fail("Improper op. Exception should have been thrown."); + } catch (TimelineParseException e) {} + + expr = "(key11 ne 234 AND key12 gt 3)) OR (key13 lt 24 OR key14 le 456)"; + try { + TimelineReaderWebServicesUtils.parseMetricFilters(expr); + fail("Unbalanced brackets. Exception should have been thrown."); + } catch (TimelineParseException e) {} + + expr = "(key11 rne 234 AND key12 gt 3) OR (key13 lt 24 OR key14 le 456)"; + try { + TimelineReaderWebServicesUtils.parseMetricFilters(expr); + fail("Invalid compareop. Exception should have been thrown."); + } catch (TimelineParseException e) {} + + expr = "(key11 ne 234 AND key12 gt 3) OR (key13 lt 24 OR key14 le"; + try { + TimelineReaderWebServicesUtils.parseMetricFilters(expr); + fail("Compareop cant be parsed. Exception should have been thrown."); + } catch (TimelineParseException e) {} + + assertNull(TimelineReaderWebServicesUtils.parseMetricFilters(null)); + assertNull(TimelineReaderWebServicesUtils.parseMetricFilters(" ")); + } + + @Test + public void testConfigFiltersParsing() throws Exception { + String expr = "(((key11 ne 234 AND key12 eq val12) AND " + + "(key13 ene val13 OR key14 eq 567)) OR (key21 eq val_21 OR key22 eq " + + "val.22))"; + TimelineFilterList expectedList = new TimelineFilterList( + Operator.OR, + new TimelineFilterList( + Operator.AND, + new TimelineFilterList( + Operator.AND, + new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL, + "key11", "234", false), + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, + "key12", "val12", true) + ), + new TimelineFilterList( + Operator.OR, + new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL, + "key13", "val13", true), + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, + "key14", "567", true) + ) + ), + new TimelineFilterList( + Operator.OR, + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, + "key21", "val_21", true), + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, + "key22", "val.22", true) + ) + ); + verifyFilterList(expr, TimelineReaderWebServicesUtils. + parseKVFilters(expr, true), expectedList); + + expr = "abc ne 234 AND def eq 23 OR rst ene 24 OR xyz eq 456 AND pqr eq 2"; + expectedList = new TimelineFilterList( + new TimelineFilterList( + Operator.OR, + new TimelineFilterList( + new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL, + "abc", "234", false), + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, + "def", "23", true) + ), + new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL, + "rst", "24", true), + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, + "xyz", "456", true) + ), + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, + "pqr", "2", true) + ); + verifyFilterList(expr, TimelineReaderWebServicesUtils. + parseKVFilters(expr, true), expectedList); + + // Test with unnecessary spaces. + expr = " abc ne 234 AND def eq 23 OR rst ene "+ + " 24 OR xyz eq 456 AND pqr eq 2 "; + expectedList = new TimelineFilterList( + new TimelineFilterList( + Operator.OR, + new TimelineFilterList( + new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL, + "abc", "234", false), + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, + "def", "23", true) + ), + new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL, + "rst", "24", true), + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, + "xyz", "456", true) + ), + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, + "pqr", "2", true) + ); + verifyFilterList(expr, TimelineReaderWebServicesUtils. + parseKVFilters(expr, true), expectedList); + + expr = "abc gt 234 AND def eq 23 OR rst ene 24 OR xyz eq 456 AND pqr eq 2"; + try { + TimelineReaderWebServicesUtils.parseKVFilters(expr, true); + fail("Invalid compareop specified for config filters. Should be either" + + " eq,ne or ene and exception should have been thrown."); + } catch (TimelineParseException e){} + } + @Test + public void testInfoFiltersParsing() throws Exception { + String expr = "(((key11 ne 234 AND key12 eq val12) AND " + + "(key13 ene val13 OR key14 eq 567)) OR (key21 eq val_21 OR key22 eq " + + "5.0))"; + TimelineFilterList expectedList = new TimelineFilterList( + Operator.OR, + new TimelineFilterList( + Operator.AND, + new TimelineFilterList( + Operator.AND, + new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL, + "key11", 234, false), + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, + "key12", "val12", true) + ), + new TimelineFilterList( + Operator.OR, + new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL, + "key13", "val13", true), + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, + "key14", 567, true) + ) + ), + new TimelineFilterList( + Operator.OR, + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, + "key21", "val_21", true), + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, + "key22", 5.0, true) + ) + ); + verifyFilterList(expr, TimelineReaderWebServicesUtils. + parseKVFilters(expr, false), expectedList); + + expr = "abc ne 234 AND def eq 23 OR rst ene 24 OR xyz eq 456 AND pqr eq " + + "val.1234"; + expectedList = new TimelineFilterList( + new TimelineFilterList( + Operator.OR, + new TimelineFilterList( + new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL, + "abc", 234, false), + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, + "def", 23, true) + ), + new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL, + "rst", 24, true), + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, + "xyz", 456, true) + ), + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, + "pqr", "val.1234", true) + ); + verifyFilterList(expr, TimelineReaderWebServicesUtils. + parseKVFilters(expr, false), expectedList); + + // Test with unnecessary spaces. + expr = " abc ne 234 AND def eq 23 OR rst ene "+ + " 24 OR xyz eq 456 AND pqr eq 2 "; + expectedList = new TimelineFilterList( + new TimelineFilterList( + Operator.OR, + new TimelineFilterList( + new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL, + "abc", 234, false), + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, + "def", 23, true) + ), + new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL, + "rst", 24, true), + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, + "xyz", 456, true) + ), + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, + "pqr", 2, true) + ); + verifyFilterList(expr, TimelineReaderWebServicesUtils. + parseKVFilters(expr, false), expectedList); + } + + @Test + public void testEventFiltersParsing() throws Exception { + String expr = "abc,def"; + TimelineFilterList expectedList = new TimelineFilterList( + new TimelineExistsFilter(TimelineCompareOp.EQUAL, "abc"), + new TimelineExistsFilter(TimelineCompareOp.EQUAL, "def") + ); + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseEventFilters(expr), expectedList); + + expr = "(abc,def)"; + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseEventFilters(expr), expectedList); + + expr = "(abc,def) OR (rst, uvx)"; + expectedList = new TimelineFilterList(Operator.OR, + new TimelineFilterList( + new TimelineExistsFilter(TimelineCompareOp.EQUAL, "abc"), + new TimelineExistsFilter(TimelineCompareOp.EQUAL, "def") + ), + new TimelineFilterList( + new TimelineExistsFilter(TimelineCompareOp.EQUAL, "rst"), + new TimelineExistsFilter(TimelineCompareOp.EQUAL, "uvx") + ) + ); + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseEventFilters(expr), expectedList); + + expr = "!(abc,def,uvc) OR (rst, uvx)"; + expectedList = new TimelineFilterList(Operator.OR, + new TimelineFilterList( + new TimelineExistsFilter(TimelineCompareOp.NOT_EQUAL, "abc"), + new TimelineExistsFilter(TimelineCompareOp.NOT_EQUAL, "def"), + new TimelineExistsFilter(TimelineCompareOp.NOT_EQUAL, "uvc") + ), + new TimelineFilterList( + new TimelineExistsFilter(TimelineCompareOp.EQUAL, "rst"), + new TimelineExistsFilter(TimelineCompareOp.EQUAL, "uvx") + ) + ); + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseEventFilters(expr), expectedList); + + expr = "(((!(abc,def,uvc) OR (rst, uvx)) AND (!(abcdefg) OR !(ghj,tyu)))" + + " OR ((bcd,tyu) AND uvb))"; + expectedList = new TimelineFilterList(Operator.OR, + new TimelineFilterList( + new TimelineFilterList(Operator.OR, + new TimelineFilterList( + new TimelineExistsFilter( + TimelineCompareOp.NOT_EQUAL, "abc"), + new TimelineExistsFilter( + TimelineCompareOp.NOT_EQUAL, "def"), + new TimelineExistsFilter(TimelineCompareOp.NOT_EQUAL, "uvc") + ), + new TimelineFilterList( + new TimelineExistsFilter(TimelineCompareOp.EQUAL, "rst"), + new TimelineExistsFilter(TimelineCompareOp.EQUAL, "uvx") + ) + ), + new TimelineFilterList(Operator.OR, + new TimelineFilterList( + new TimelineExistsFilter( + TimelineCompareOp.NOT_EQUAL, "abcdefg") + ), + new TimelineFilterList( + new TimelineExistsFilter( + TimelineCompareOp.NOT_EQUAL, "ghj"), + new TimelineExistsFilter( + TimelineCompareOp.NOT_EQUAL, "tyu") + ) + ) + ), + new TimelineFilterList( + new TimelineFilterList( + new TimelineExistsFilter(TimelineCompareOp.EQUAL, "bcd"), + new TimelineExistsFilter(TimelineCompareOp.EQUAL, "tyu") + ), + new TimelineExistsFilter(TimelineCompareOp.EQUAL, "uvb") + ) + ); + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseEventFilters(expr), expectedList); + + expr = " ( ( ( ! ( abc , def , uvc) OR ( rst , uvx ) )" + + " AND ( ! ( abcdefg ) OR ! ( ghj, tyu) ) ) OR ( (" + + " bcd , tyu ) AND uvb ) )"; + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseEventFilters(expr), expectedList); + + expr = "(((!(abc,def,uvc) OR (rst, uvx)) AND (!(abcdefg) OR !(ghj,tyu)))" + + " OR ((bcd,tyu) AND uvb)"; + try { + TimelineReaderWebServicesUtils.parseEventFilters(expr); + fail("Improper brackets. Exception should have been thrown"); + } catch (TimelineParseException e){} + + expr = "(((!(abc,def,uvc) (OR (rst, uvx)) AND (!(abcdefg) OR !(ghj,tyu)))" + + " OR ((bcd,tyu) AND uvb))"; + try { + TimelineReaderWebServicesUtils.parseEventFilters(expr); + fail("Unexpected opening bracket. Exception should have been thrown"); + } catch (TimelineParseException e){} + + expr = "(((!(abc,def,uvc) OR) (rst, uvx)) AND (!(abcdefg) OR !(ghj,tyu)))" + + " OR ((bcd,tyu) AND uvb))"; + try { + TimelineReaderWebServicesUtils.parseEventFilters(expr); + fail("Unexpected closing bracket. Exception should have been thrown"); + } catch (TimelineParseException e){} + + expr = "(((!(abc,def,uvc) PI (rst, uvx)) AND (!(abcdefg) OR !(ghj,tyu)))" + + " OR ((bcd,tyu) AND uvb))"; + try { + TimelineReaderWebServicesUtils.parseEventFilters(expr); + fail("Invalid op. Exception should have been thrown"); + } catch (TimelineParseException e){} + + expr = "(((!(abc,def,uvc) !OR (rst, uvx)) AND (!(abcdefg) OR !(ghj,tyu)))" + + " OR ((bcd,tyu) AND uvb))"; + try { + TimelineReaderWebServicesUtils.parseEventFilters(expr); + fail("Unexpected ! char. Exception should have been thrown"); + } catch (TimelineParseException e){} + + expr = "abc,def,uvc) OR (rst, uvx)"; + try { + TimelineReaderWebServicesUtils.parseEventFilters(expr); + fail("Unexpected closing bracket. Exception should have been thrown"); + } catch (TimelineParseException e){} + + expr = "abc,def,uvc OR )rst, uvx)"; + try { + TimelineReaderWebServicesUtils.parseEventFilters(expr); + fail("Unexpected closing bracket. Exception should have been thrown"); + } catch (TimelineParseException e){} + + expr = "abc,def,uvc OR ,rst, uvx)"; + try { + TimelineReaderWebServicesUtils.parseEventFilters(expr); + fail("Unexpected delimiter. Exception should have been thrown"); + } catch (TimelineParseException e){} + + expr = "abc,def,uvc OR ! "; + try { + TimelineReaderWebServicesUtils.parseEventFilters(expr); + fail("Unexpected not char. Exception should have been thrown"); + } catch (TimelineParseException e){} + + expr = "(abc,def,uvc)) OR (rst, uvx)"; + try { + TimelineReaderWebServicesUtils.parseEventFilters(expr); + fail("Unbalanced brackets. Exception should have been thrown"); + } catch (TimelineParseException e){} + + expr = "(((! ,(abc,def,uvc) OR (rst, uvx)) AND (!(abcdefg) OR !(ghj,tyu" + + "))) OR ((bcd,tyu) AND uvb))"; + try { + TimelineReaderWebServicesUtils.parseEventFilters(expr); + fail("( should follow ! char. Exception should have been thrown"); + } catch (TimelineParseException e){} + + assertNull(TimelineReaderWebServicesUtils.parseEventFilters(null)); + assertNull(TimelineReaderWebServicesUtils.parseEventFilters(" ")); + } + + @Test + public void testRelationFiltersParsing() throws Exception { + String expr = "type1:entity11,type2:entity21:entity22"; + TimelineFilterList expectedList = new TimelineFilterList( + new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL, + "type1", Sets.newHashSet((Object)"entity11")), + new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL, + "type2", Sets.newHashSet((Object)"entity21", "entity22")) + ); + verifyFilterList(expr, TimelineReaderWebServicesUtils. + parseRelationFilters(expr), expectedList); + + expr = "(type1:entity11,type2:entity21:entity22)"; + verifyFilterList(expr, TimelineReaderWebServicesUtils. + parseRelationFilters(expr), expectedList); + + expr = "(type1:entity11,type2:entity21:entity22) OR (type3:entity31:" + + "entity32:entity33,type1:entity11:entity12)"; + expectedList = new TimelineFilterList(Operator.OR, + new TimelineFilterList( + new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL, + "type1", Sets.newHashSet((Object)"entity11")), + new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL, + "type2", Sets.newHashSet((Object)"entity21", "entity22")) + ), + new TimelineFilterList( + new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL, + "type3", Sets.newHashSet( + (Object)"entity31", "entity32", "entity33")), + new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL, + "type1", Sets.newHashSet((Object)"entity11", "entity12")) + ) + ); + verifyFilterList(expr, TimelineReaderWebServicesUtils. + parseRelationFilters(expr), expectedList); + + expr = "!(type1:entity11,type2:entity21:entity22,type5:entity51) OR " + + "(type3:entity31:entity32:entity33,type1:entity11:entity12)"; + + expectedList = new TimelineFilterList(Operator.OR, + new TimelineFilterList( + new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL, + "type1", Sets.newHashSet((Object)"entity11")), + new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL, + "type2", Sets.newHashSet((Object)"entity21", "entity22")), + new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL, + "type5", Sets.newHashSet((Object)"entity51")) + ), + new TimelineFilterList( + new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL, + "type3", Sets.newHashSet( + (Object)"entity31", "entity32", "entity33")), + new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL, + "type1", Sets.newHashSet((Object)"entity11", "entity12")) + ) + ); + verifyFilterList(expr, TimelineReaderWebServicesUtils. + parseRelationFilters(expr), expectedList); + + expr = "(((!(type1:entity11,type2:entity21:entity22,type5:entity51) OR " + + "(type3:entity31:entity32:entity33,type1:entity11:entity12)) AND "+ + "(!(type11:entity111) OR !(type4:entity43:entity44:entity47:entity49," + + "type7:entity71))) OR ((type2:entity2,type8:entity88) AND t9:e:e1))"; + expectedList = new TimelineFilterList(Operator.OR, + new TimelineFilterList( + new TimelineFilterList(Operator.OR, + new TimelineFilterList( + new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL, + "type1", Sets.newHashSet((Object)"entity11")), + new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL, + "type2", Sets.newHashSet( + (Object)"entity21", "entity22")), + new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL, + "type5", Sets.newHashSet((Object)"entity51")) + ), + new TimelineFilterList( + new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL, + "type3", Sets.newHashSet( + (Object)"entity31", "entity32", "entity33")), + new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL, + "type1", Sets.newHashSet( + (Object)"entity11", "entity12")) + ) + ), + new TimelineFilterList(Operator.OR, + new TimelineFilterList( + new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL, + "type11", Sets.newHashSet((Object)"entity111")) + ), + new TimelineFilterList( + new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL, + "type4", Sets.newHashSet((Object)"entity43", "entity44", + "entity47", "entity49")), + new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL, + "type7", Sets.newHashSet((Object)"entity71")) + ) + ) + ), + new TimelineFilterList( + new TimelineFilterList( + new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL, + "type2", Sets.newHashSet((Object)"entity2")), + new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL, + "type8", Sets.newHashSet((Object)"entity88")) + ), + new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL, "t9", + Sets.newHashSet((Object)"e", "e1")) + ) + ); + verifyFilterList(expr, TimelineReaderWebServicesUtils. + parseRelationFilters(expr), expectedList); + + expr = " ( ( ( ! ( type1:entity11 , type2:entity21:entity22" + + " , type5:entity51 ) OR ( type3:entity31:entity32:entity33 " + + " , type1:entity11:entity12)) AND (!( type11:entity111 ) OR " + + " ! ( type4:entity43:entity44:entity47:entity49 , " + + "type7:entity71 ) ) ) OR ( ( type2:entity2 , type8:entity88) " + + "AND t9:e:e1 ) ) "; + verifyFilterList(expr, TimelineReaderWebServicesUtils. + parseRelationFilters(expr), expectedList); + + expr = "(((!(type1 : entity11,type2:entity21:entity22,type5:entity51) OR " + + "(type3:entity31:entity32:entity33,type1:entity11:entity12)) AND "+ + "(!(type11:entity111) OR !(type4:entity43:entity44:entity47:entity49," + + "type7:entity71))) OR ((type2:entity2,type8:entity88) AND t9:e:e1))"; + try { + TimelineReaderWebServicesUtils.parseRelationFilters(expr); + fail("Space not allowed in relation expression. Exception should have " + + "been thrown"); + } catch (TimelineParseException e){} + } + + @Test + public void testDataToRetrieve() throws Exception { + String expr = "abc,def"; + TimelineFilterList expectedList = new TimelineFilterList(Operator.OR, + new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "abc"), + new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "def") + ); + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList); + + expr = "(abc,def)"; + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList); + + expr = " ( abc , def ) "; + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList); + + expr = " abc , def "; + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList); + + expr = "!(abc,def)"; + expectedList = new TimelineFilterList( + new TimelinePrefixFilter(TimelineCompareOp.NOT_EQUAL, "abc"), + new TimelinePrefixFilter(TimelineCompareOp.NOT_EQUAL, "def") + ); + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList); + + expr = " ! ( abc , def ) "; + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList); + + expr = "!(abc)"; + expectedList = new TimelineFilterList( + new TimelinePrefixFilter(TimelineCompareOp.NOT_EQUAL, "abc") + ); + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList); + + expr = "(abc)"; + expectedList = new TimelineFilterList(Operator.OR, + new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "abc") + ); + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList); + + expr = "abc"; + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList); + + expr = " ! ( abc , def , xyz) "; + expectedList = new TimelineFilterList( + new TimelinePrefixFilter(TimelineCompareOp.NOT_EQUAL, "abc"), + new TimelinePrefixFilter(TimelineCompareOp.NOT_EQUAL, "def"), + new TimelinePrefixFilter(TimelineCompareOp.NOT_EQUAL, "xyz") + ); + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList); + + expr = "!(abc,def,xyz)"; + verifyFilterList(expr, + TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList); + + expr = "!(abc,def,xyz"; + try { + TimelineReaderWebServicesUtils.parseDataToRetrieve(expr); + fail("No closing bracket. Exception should have been thrown"); + } catch (TimelineParseException e){} + + expr = "!abc,def,xyz"; + try { + TimelineReaderWebServicesUtils.parseDataToRetrieve(expr); + fail("NOT(!) should be followed by opening bracket. Exception should " + + "have been thrown"); + } catch (TimelineParseException e){} + + expr = "!abc,def,xyz"; + try { + TimelineReaderWebServicesUtils.parseDataToRetrieve(expr); + fail("NOT(!) should be followed by opening bracket. Exception should " + + "have been thrown"); + } catch (TimelineParseException e){} + + expr = "! r( abc,def,xyz)"; + try { + TimelineReaderWebServicesUtils.parseDataToRetrieve(expr); + fail("NOT(!) should be followed by opening bracket. Exception should " + + "have been thrown"); + } catch (TimelineParseException e){} + + assertNull(TimelineReaderWebServicesUtils.parseDataToRetrieve(null)); + assertNull(TimelineReaderWebServicesUtils.parseDataToRetrieve(" ")); + assertNull(TimelineReaderWebServicesUtils.parseDataToRetrieve("()")); + assertNull(TimelineReaderWebServicesUtils.parseDataToRetrieve("!()")); + assertNull(TimelineReaderWebServicesUtils.parseDataToRetrieve("( )")); + assertNull(TimelineReaderWebServicesUtils.parseDataToRetrieve("!( )")); + assertNull(TimelineReaderWebServicesUtils.parseDataToRetrieve("! ( )")); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWhitelistAuthorizationFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWhitelistAuthorizationFilter.java new file mode 100644 index 00000000000..b0f8b7095d5 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWhitelistAuthorizationFilter.java @@ -0,0 +1,380 @@ +/** + * 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.timelineservice.reader; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.security.Principal; +import java.security.PrivilegedExceptionAction; +import java.util.Collections; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.Map; + +import javax.servlet.FilterConfig; +import javax.servlet.ServletContext; +import javax.servlet.ServletException; +import javax.servlet.ServletResponse; +import javax.servlet.http.HttpServletRequest; + +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.timelineservice.reader.security.TimelineReaderWhitelistAuthorizationFilter; +import org.apache.hadoop.yarn.webapp.ForbiddenException; +import org.junit.Test; +import org.mockito.Mockito; + +/** + * Unit tests for {@link TimelineReaderWhitelistAuthorizationFilter}. + * + */ +public class TestTimelineReaderWhitelistAuthorizationFilter { + + final private static String GROUP1_NAME = "group1"; + final private static String GROUP2_NAME = "group2"; + final private static String GROUP3_NAME = "group3"; + final private static String[] GROUP_NAMES = + new String[] {GROUP1_NAME, GROUP2_NAME, GROUP3_NAME}; + + private static class DummyFilterConfig implements FilterConfig { + final private Map map; + + DummyFilterConfig(Map map) { + this.map = map; + } + + @Override + public String getFilterName() { + return "dummy"; + } + + @Override + public String getInitParameter(String arg0) { + return map.get(arg0); + } + + @Override + public Enumeration getInitParameterNames() { + return Collections.enumeration(map.keySet()); + } + + @Override + public ServletContext getServletContext() { + return null; + } + } + + @Test + public void checkFilterAllowedUser() throws ServletException, IOException { + Map map = new HashMap(); + map.put(YarnConfiguration.TIMELINE_SERVICE_READ_AUTH_ENABLED, "true"); + map.put(YarnConfiguration.TIMELINE_SERVICE_READ_ALLOWED_USERS, + "user1,user2"); + final TimelineReaderWhitelistAuthorizationFilter f = + new TimelineReaderWhitelistAuthorizationFilter(); + FilterConfig fc = new DummyFilterConfig(map); + f.init(fc); + final HttpServletRequest mockHsr = Mockito.mock(HttpServletRequest.class); + Mockito.when(mockHsr.getUserPrincipal()).thenReturn(new Principal() { + @Override + public String getName() { + return "user1"; + } + }); + + final ServletResponse r = Mockito.mock(ServletResponse.class); + f.doFilter(mockHsr, r, null); + } + + @Test(expected = ForbiddenException.class) + public void checkFilterNotAllowedUser() throws ServletException, IOException { + Map map = new HashMap(); + map.put(YarnConfiguration.TIMELINE_SERVICE_READ_AUTH_ENABLED, "true"); + map.put(YarnConfiguration.TIMELINE_SERVICE_READ_ALLOWED_USERS, + "user1,user2"); + final TimelineReaderWhitelistAuthorizationFilter f = + new TimelineReaderWhitelistAuthorizationFilter(); + FilterConfig fc = new DummyFilterConfig(map); + f.init(fc); + final HttpServletRequest mockHsr = Mockito.mock(HttpServletRequest.class); + Mockito.when(mockHsr.getUserPrincipal()).thenReturn(new Principal() { + @Override + public String getName() { + return "testuser1"; + } + }); + final ServletResponse r = Mockito.mock(ServletResponse.class); + f.doFilter(mockHsr, r, null); + } + + @Test + public void checkFilterAllowedUserGroup() + throws ServletException, IOException, InterruptedException { + Map map = new HashMap(); + map.put(YarnConfiguration.TIMELINE_SERVICE_READ_AUTH_ENABLED, "true"); + map.put(YarnConfiguration.TIMELINE_SERVICE_READ_ALLOWED_USERS, + "user2 group1,group2"); + final TimelineReaderWhitelistAuthorizationFilter f = + new TimelineReaderWhitelistAuthorizationFilter(); + FilterConfig fc = new DummyFilterConfig(map); + f.init(fc); + final HttpServletRequest mockHsr = Mockito.mock(HttpServletRequest.class); + Mockito.when(mockHsr.getUserPrincipal()).thenReturn(new Principal() { + @Override + public String getName() { + return "user1"; + } + }); + final ServletResponse r = Mockito.mock(ServletResponse.class); + UserGroupInformation user1 = + UserGroupInformation.createUserForTesting("user1", GROUP_NAMES); + user1.doAs(new PrivilegedExceptionAction() { + @Override + public Object run() throws Exception { + f.doFilter(mockHsr, r, null); + return null; + } + }); + } + + @Test(expected = ForbiddenException.class) + public void checkFilterNotAlloweGroup() + throws ServletException, IOException, InterruptedException { + Map map = new HashMap(); + map.put(YarnConfiguration.TIMELINE_SERVICE_READ_AUTH_ENABLED, "true"); + map.put(YarnConfiguration.TIMELINE_SERVICE_READ_ALLOWED_USERS, + " group5,group6"); + final TimelineReaderWhitelistAuthorizationFilter f = + new TimelineReaderWhitelistAuthorizationFilter(); + FilterConfig fc = new DummyFilterConfig(map); + f.init(fc); + final HttpServletRequest mockHsr = Mockito.mock(HttpServletRequest.class); + Mockito.when(mockHsr.getUserPrincipal()).thenReturn(new Principal() { + @Override + public String getName() { + return "user200"; + } + }); + final ServletResponse r = Mockito.mock(ServletResponse.class); + UserGroupInformation user1 = + UserGroupInformation.createUserForTesting("user200", GROUP_NAMES); + user1.doAs(new PrivilegedExceptionAction() { + @Override + public Object run() throws Exception { + f.doFilter(mockHsr, r, null); + return null; + } + }); + } + + @Test + public void checkFilterAllowAdmins() + throws ServletException, IOException, InterruptedException { + // check that users in admin acl list are allowed to read + Map map = new HashMap(); + map.put(YarnConfiguration.TIMELINE_SERVICE_READ_AUTH_ENABLED, "true"); + map.put(YarnConfiguration.TIMELINE_SERVICE_READ_ALLOWED_USERS, + "user3 group5,group6"); + map.put(YarnConfiguration.YARN_ADMIN_ACL, " group1,group2"); + final TimelineReaderWhitelistAuthorizationFilter f = + new TimelineReaderWhitelistAuthorizationFilter(); + FilterConfig fc = new DummyFilterConfig(map); + f.init(fc); + final HttpServletRequest mockHsr = Mockito.mock(HttpServletRequest.class); + Mockito.when(mockHsr.getUserPrincipal()).thenReturn(new Principal() { + @Override + public String getName() { + return "user90"; + } + }); + final ServletResponse r = Mockito.mock(ServletResponse.class); + UserGroupInformation user1 = + UserGroupInformation.createUserForTesting("user90", GROUP_NAMES); + user1.doAs(new PrivilegedExceptionAction() { + @Override + public Object run() throws Exception { + f.doFilter(mockHsr, r, null); + return null; + } + }); + } + + @Test + public void checkFilterAllowAdminsWhenNoUsersSet() + throws ServletException, IOException, InterruptedException { + // check that users in admin acl list are allowed to read + Map map = new HashMap(); + map.put(YarnConfiguration.TIMELINE_SERVICE_READ_AUTH_ENABLED, "true"); + map.put(YarnConfiguration.YARN_ADMIN_ACL, " group1,group2"); + final TimelineReaderWhitelistAuthorizationFilter f = + new TimelineReaderWhitelistAuthorizationFilter(); + FilterConfig fc = new DummyFilterConfig(map); + f.init(fc); + final HttpServletRequest mockHsr = Mockito.mock(HttpServletRequest.class); + Mockito.when(mockHsr.getUserPrincipal()).thenReturn(new Principal() { + @Override + public String getName() { + return "user90"; + } + }); + final ServletResponse r = Mockito.mock(ServletResponse.class); + UserGroupInformation user1 = + UserGroupInformation.createUserForTesting("user90", GROUP_NAMES); + user1.doAs(new PrivilegedExceptionAction() { + @Override + public Object run() throws Exception { + f.doFilter(mockHsr, r, null); + return null; + } + }); + } + + @Test(expected = ForbiddenException.class) + public void checkFilterAllowNoOneWhenAdminAclsEmptyAndUserAclsEmpty() + throws ServletException, IOException, InterruptedException { + // check that users in admin acl list are allowed to read + Map map = new HashMap(); + map.put(YarnConfiguration.TIMELINE_SERVICE_READ_AUTH_ENABLED, "true"); + final TimelineReaderWhitelistAuthorizationFilter f = + new TimelineReaderWhitelistAuthorizationFilter(); + FilterConfig fc = new DummyFilterConfig(map); + f.init(fc); + final HttpServletRequest mockHsr = Mockito.mock(HttpServletRequest.class); + Mockito.when(mockHsr.getUserPrincipal()).thenReturn(new Principal() { + @Override + public String getName() { + return "user88"; + } + }); + final ServletResponse r = Mockito.mock(ServletResponse.class); + UserGroupInformation user1 = + UserGroupInformation.createUserForTesting("user88", GROUP_NAMES); + user1.doAs(new PrivilegedExceptionAction() { + @Override + public Object run() throws Exception { + f.doFilter(mockHsr, r, null); + return null; + } + }); + } + + @Test + public void checkFilterReadAuthDisabledNoAclSettings() + throws ServletException, IOException, InterruptedException { + // Default settings for Read Auth Enabled (false) + // No values in admin acls or allowed read user list + Map map = new HashMap(); + final TimelineReaderWhitelistAuthorizationFilter f = + new TimelineReaderWhitelistAuthorizationFilter(); + FilterConfig fc = new DummyFilterConfig(map); + f.init(fc); + final HttpServletRequest mockHsr = Mockito.mock(HttpServletRequest.class); + Mockito.when(mockHsr.getUserPrincipal()).thenReturn(new Principal() { + @Override + public String getName() { + return "user437"; + } + }); + final ServletResponse r = Mockito.mock(ServletResponse.class); + UserGroupInformation user1 = + UserGroupInformation.createUserForTesting("user437", GROUP_NAMES); + user1.doAs(new PrivilegedExceptionAction() { + @Override + public Object run() throws Exception { + f.doFilter(mockHsr, r, null); + return null; + } + }); + } + + @Test + public void checkFilterReadAuthDisabledButAclSettingsPopulated() + throws ServletException, IOException, InterruptedException { + Map map = new HashMap(); + // Default settings for Read Auth Enabled (false) + // But some values in admin acls and allowed read user list + map.put(YarnConfiguration.YARN_ADMIN_ACL, "user1,user2 group9,group21"); + map.put(YarnConfiguration.TIMELINE_SERVICE_READ_ALLOWED_USERS, + "user27,user36 group5,group6"); + final TimelineReaderWhitelistAuthorizationFilter f = + new TimelineReaderWhitelistAuthorizationFilter(); + FilterConfig fc = new DummyFilterConfig(map); + f.init(fc); + + final HttpServletRequest mockHsr = mock(HttpServletRequest.class); + when(mockHsr.getUserPrincipal()).thenReturn(new Principal() { + @Override + public String getName() { + return "user37"; + } + }); + + final ServletResponse r = Mockito.mock(ServletResponse.class); + UserGroupInformation user1 = + // both username and group name are not part of admin and + // read allowed users + // but read auth is turned off + UserGroupInformation.createUserForTesting("user37", GROUP_NAMES); + user1.doAs(new PrivilegedExceptionAction() { + @Override + public Object run() throws Exception { + f.doFilter(mockHsr, r, null); + return null; + } + }); + + // test with username in read allowed users + Mockito.when(mockHsr.getUserPrincipal()).thenReturn(new Principal() { + @Override + public String getName() { + return "user27"; + } + }); + final ServletResponse r2 = Mockito.mock(ServletResponse.class); + UserGroupInformation user2 = + UserGroupInformation.createUserForTesting("user27", GROUP_NAMES); + user2.doAs(new PrivilegedExceptionAction() { + @Override + public Object run() throws Exception { + f.doFilter(mockHsr, r2, null); + return null; + } + }); + + // test with username in admin users + Mockito.when(mockHsr.getUserPrincipal()).thenReturn(new Principal() { + @Override + public String getName() { + return "user2"; + } + }); + final ServletResponse r3 = Mockito.mock(ServletResponse.class); + UserGroupInformation user3 = + UserGroupInformation.createUserForTesting("user2", GROUP_NAMES); + user3.doAs(new PrivilegedExceptionAction() { + @Override + public Object run() throws Exception { + f.doFilter(mockHsr, r3, null); + return null; + } + }); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineUIDConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineUIDConverter.java new file mode 100644 index 00000000000..a720a2d8e19 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineUIDConverter.java @@ -0,0 +1,109 @@ +/** + * 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.timelineservice.reader; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.fail; + +import org.junit.Test; + +/** + * Tests Timeline UIDs' encoding and decoding. + */ +public class TestTimelineUIDConverter { + + @Test + public void testUIDEncodingDecoding() throws Exception { + TimelineReaderContext context = new TimelineReaderContext( + "!cluster", "!b*o*!xer", "oozie*", null, null, null, null); + String uid = TimelineUIDConverter.FLOW_UID.encodeUID(context); + assertEquals("*!cluster!*!b**o***!xer!oozie**", uid); + assertEquals(context, TimelineUIDConverter.FLOW_UID.decodeUID(uid)); + + context = new TimelineReaderContext("!cluster*", "!b*o!!x!*er", "*oozie!", + 123L, null, null, null); + uid = TimelineUIDConverter.FLOWRUN_UID.encodeUID(context); + assertEquals("*!cluster**!*!b**o*!*!x*!**er!**oozie*!!123", uid); + assertEquals(context, TimelineUIDConverter.FLOWRUN_UID.decodeUID(uid)); + + context = new TimelineReaderContext("yarn_cluster", "root", "hive_join", + 1234L, "application_1111111111_1111", null, null); + uid = TimelineUIDConverter.APPLICATION_UID.encodeUID(context); + assertEquals( + "yarn_cluster!root!hive_join!1234!application_1111111111_1111", uid); + assertEquals(context, TimelineUIDConverter.APPLICATION_UID.decodeUID(uid)); + context = new TimelineReaderContext("yarn_cluster", null, null, null, + "application_1111111111_1111", null, null); + uid = TimelineUIDConverter.APPLICATION_UID.encodeUID(context); + assertEquals("yarn_cluster!application_1111111111_1111", uid); + assertEquals(context, TimelineUIDConverter.APPLICATION_UID.decodeUID(uid)); + + context = new TimelineReaderContext("yarn_cluster", "root", "hive_join", + 1234L, "application_1111111111_1111", "YARN_CONTAINER", 12345L, + "container_1111111111_1111_01_000001"); + uid = TimelineUIDConverter.GENERIC_ENTITY_UID.encodeUID(context); + assertEquals("yarn_cluster!root!hive_join!1234!application_1111111111_1111!" + + "YARN_CONTAINER!12345!container_1111111111_1111_01_000001", uid); + assertEquals( + context, TimelineUIDConverter.GENERIC_ENTITY_UID.decodeUID(uid)); + context = new TimelineReaderContext("yarn_cluster", null, null, null, + "application_1111111111_1111", "YARN_CONTAINER", 54321L, + "container_1111111111_1111_01_000001"); + uid = TimelineUIDConverter.GENERIC_ENTITY_UID.encodeUID(context); + assertEquals("yarn_cluster!application_1111111111_1111!YARN_CONTAINER!" + + "54321!container_1111111111_1111_01_000001", uid); + assertEquals( + context, TimelineUIDConverter.GENERIC_ENTITY_UID.decodeUID(uid)); + + context = new TimelineReaderContext("yarn_cluster", null, null, null, null, + "YARN_CONTAINER", 54321L, "container_1111111111_1111_01_000001", + "user1"); + uid = TimelineUIDConverter.SUB_APPLICATION_ENTITY_UID.encodeUID(context); + assertEquals("yarn_cluster!user1!YARN_CONTAINER!" + + "54321!container_1111111111_1111_01_000001", uid); + assertEquals(context, + TimelineUIDConverter.SUB_APPLICATION_ENTITY_UID.decodeUID(uid)); + } + + @Test + public void testUIDNotProperlyEscaped() throws Exception { + try { + TimelineUIDConverter.FLOW_UID.decodeUID("*!cluster!*!b*o***!xer!oozie**"); + fail("UID not properly escaped. Exception should have been thrown."); + } catch (IllegalArgumentException e) { + } + + try { + TimelineUIDConverter.FLOW_UID.decodeUID("*!cluster!*!b**o***!xer!oozie*"); + fail("UID not properly escaped. Exception should have been thrown."); + } catch (IllegalArgumentException e) { + } + + try { + TimelineUIDConverter.FLOW_UID.decodeUID("*!cluster!*!b**o***xer!oozie*"); + fail("UID not properly escaped. Exception should have been thrown."); + } catch (IllegalArgumentException e) { + } + assertNull(TimelineUIDConverter. + FLOW_UID.decodeUID("!cluster!*!b**o***!xer!oozie**")); + assertNull(TimelineUIDConverter. + FLOW_UID.decodeUID("*!cluster!*!b**o**!xer!oozie**")); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineReaderImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineReaderImpl.java new file mode 100644 index 00000000000..a031a879671 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineReaderImpl.java @@ -0,0 +1,851 @@ +/** + * 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.timelineservice.storage; + +import java.io.BufferedWriter; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.io.PrintWriter; +import java.util.Arrays; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import org.apache.commons.csv.CSVFormat; +import org.apache.commons.csv.CSVPrinter; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationAttemptEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.ContainerEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList.Operator; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter; +import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValuesFilter; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field; +import org.apache.hadoop.yarn.util.timeline.TimelineUtils; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +/** + * Tests filesystem implemetation for timelineservice reader. + */ +public class TestFileSystemTimelineReaderImpl { + + private static final String ROOT_DIR = new File("target", + TestFileSystemTimelineReaderImpl.class.getSimpleName()).getAbsolutePath(); + private FileSystemTimelineReaderImpl reader; + + @BeforeClass + public static void setup() throws Exception { + initializeDataDirectory(ROOT_DIR); + } + + public static void initializeDataDirectory(String rootDir) throws Exception { + loadEntityData(rootDir); + // Create app flow mapping file. + CSVFormat format = + CSVFormat.DEFAULT.withHeader("APP", "USER", "FLOW", "FLOWRUN"); + String appFlowMappingFile = rootDir + File.separator + "entities" + + File.separator + "cluster1" + File.separator + + FileSystemTimelineReaderImpl.APP_FLOW_MAPPING_FILE; + try (PrintWriter out = + new PrintWriter(new BufferedWriter( + new FileWriter(appFlowMappingFile, true))); + CSVPrinter printer = new CSVPrinter(out, format)){ + printer.printRecord("app1", "user1", "flow1", 1); + printer.printRecord("app2", "user1", "flow1,flow", 1); + printer.close(); + } + (new File(rootDir)).deleteOnExit(); + } + + @AfterClass + public static void tearDown() throws Exception { + FileUtils.deleteDirectory(new File(ROOT_DIR)); + } + + @Before + public void init() throws Exception { + reader = new FileSystemTimelineReaderImpl(); + Configuration conf = new YarnConfiguration(); + conf.set(FileSystemTimelineReaderImpl.TIMELINE_SERVICE_STORAGE_DIR_ROOT, + ROOT_DIR); + reader.init(conf); + } + + private static void writeEntityFile(TimelineEntity entity, File dir) + throws Exception { + if (!dir.exists()) { + if (!dir.mkdirs()) { + throw new IOException("Could not create directories for " + dir); + } + } + String fileName = dir.getAbsolutePath() + File.separator + entity.getId() + + ".thist"; + try (PrintWriter out = + new PrintWriter(new BufferedWriter(new FileWriter(fileName, true)))){ + out.println(TimelineUtils.dumpTimelineRecordtoJSON(entity)); + out.write("\n"); + out.close(); + } + } + + private static void loadEntityData(String rootDir) throws Exception { + File appDir = + getAppDir(rootDir, "cluster1", "user1", "flow1", "1", "app1", "app"); + TimelineEntity entity11 = new TimelineEntity(); + entity11.setId("id_1"); + entity11.setType("app"); + entity11.setCreatedTime(1425016502000L); + Map info1 = new HashMap(); + info1.put("info1", "val1"); + info1.put("info2", "val5"); + entity11.addInfo(info1); + TimelineEvent event = new TimelineEvent(); + event.setId("event_1"); + event.setTimestamp(1425016502003L); + entity11.addEvent(event); + Set metrics = new HashSet(); + TimelineMetric metric1 = new TimelineMetric(); + metric1.setId("metric1"); + metric1.setType(TimelineMetric.Type.SINGLE_VALUE); + metric1.addValue(1425016502006L, 113); + metrics.add(metric1); + TimelineMetric metric2 = new TimelineMetric(); + metric2.setId("metric2"); + metric2.setType(TimelineMetric.Type.TIME_SERIES); + metric2.addValue(1425016502016L, 34); + metrics.add(metric2); + entity11.setMetrics(metrics); + Map configs = new HashMap(); + configs.put("config_1", "127"); + entity11.setConfigs(configs); + entity11.addRelatesToEntity("flow", "flow1"); + entity11.addIsRelatedToEntity("type1", "tid1_1"); + writeEntityFile(entity11, appDir); + TimelineEntity entity12 = new TimelineEntity(); + entity12.setId("id_1"); + entity12.setType("app"); + configs.clear(); + configs.put("config_2", "23"); + configs.put("config_3", "abc"); + entity12.addConfigs(configs); + metrics.clear(); + TimelineMetric metric12 = new TimelineMetric(); + metric12.setId("metric2"); + metric12.setType(TimelineMetric.Type.TIME_SERIES); + metric12.addValue(1425016502032L, 48); + metric12.addValue(1425016502054L, 51); + metrics.add(metric12); + TimelineMetric metric3 = new TimelineMetric(); + metric3.setId("metric3"); + metric3.setType(TimelineMetric.Type.SINGLE_VALUE); + metric3.addValue(1425016502060L, 23L); + metrics.add(metric3); + entity12.setMetrics(metrics); + entity12.addIsRelatedToEntity("type1", "tid1_2"); + entity12.addIsRelatedToEntity("type2", "tid2_1`"); + TimelineEvent event15 = new TimelineEvent(); + event15.setId("event_5"); + event15.setTimestamp(1425016502017L); + entity12.addEvent(event15); + writeEntityFile(entity12, appDir); + + TimelineEntity entity2 = new TimelineEntity(); + entity2.setId("id_2"); + entity2.setType("app"); + entity2.setCreatedTime(1425016501050L); + Map info2 = new HashMap(); + info1.put("info2", 4); + entity2.addInfo(info2); + Map configs2 = new HashMap(); + configs2.put("config_1", "129"); + configs2.put("config_3", "def"); + entity2.setConfigs(configs2); + TimelineEvent event2 = new TimelineEvent(); + event2.setId("event_2"); + event2.setTimestamp(1425016501003L); + entity2.addEvent(event2); + Set metrics2 = new HashSet(); + TimelineMetric metric21 = new TimelineMetric(); + metric21.setId("metric1"); + metric21.setType(TimelineMetric.Type.SINGLE_VALUE); + metric21.addValue(1425016501006L, 300); + metrics2.add(metric21); + TimelineMetric metric22 = new TimelineMetric(); + metric22.setId("metric2"); + metric22.setType(TimelineMetric.Type.TIME_SERIES); + metric22.addValue(1425016501056L, 31); + metric22.addValue(1425016501084L, 70); + metrics2.add(metric22); + TimelineMetric metric23 = new TimelineMetric(); + metric23.setId("metric3"); + metric23.setType(TimelineMetric.Type.SINGLE_VALUE); + metric23.addValue(1425016502060L, 23L); + metrics2.add(metric23); + entity2.setMetrics(metrics2); + entity2.addRelatesToEntity("flow", "flow2"); + writeEntityFile(entity2, appDir); + + TimelineEntity entity3 = new TimelineEntity(); + entity3.setId("id_3"); + entity3.setType("app"); + entity3.setCreatedTime(1425016501050L); + Map info3 = new HashMap(); + info3.put("info2", 3.5); + info3.put("info4", 20); + entity3.addInfo(info3); + Map configs3 = new HashMap(); + configs3.put("config_1", "123"); + configs3.put("config_3", "abc"); + entity3.setConfigs(configs3); + TimelineEvent event3 = new TimelineEvent(); + event3.setId("event_2"); + event3.setTimestamp(1425016501003L); + entity3.addEvent(event3); + TimelineEvent event4 = new TimelineEvent(); + event4.setId("event_4"); + event4.setTimestamp(1425016502006L); + entity3.addEvent(event4); + Set metrics3 = new HashSet(); + TimelineMetric metric31 = new TimelineMetric(); + metric31.setId("metric1"); + metric31.setType(TimelineMetric.Type.SINGLE_VALUE); + metric31.addValue(1425016501006L, 124); + metrics3.add(metric31); + TimelineMetric metric32 = new TimelineMetric(); + metric32.setId("metric2"); + metric32.setType(TimelineMetric.Type.TIME_SERIES); + metric32.addValue(1425016501056L, 31); + metric32.addValue(1425016501084L, 74); + metrics3.add(metric32); + entity3.setMetrics(metrics3); + entity3.addIsRelatedToEntity("type1", "tid1_2"); + writeEntityFile(entity3, appDir); + + TimelineEntity entity4 = new TimelineEntity(); + entity4.setId("id_4"); + entity4.setType("app"); + entity4.setCreatedTime(1425016502050L); + TimelineEvent event44 = new TimelineEvent(); + event44.setId("event_4"); + event44.setTimestamp(1425016502003L); + entity4.addEvent(event44); + writeEntityFile(entity4, appDir); + + File attemptDir = getAppDir(rootDir, "cluster1", "user1", "flow1", "1", + "app1", TimelineEntityType.YARN_APPLICATION_ATTEMPT.toString()); + ApplicationAttemptEntity attempt1 = new ApplicationAttemptEntity(); + attempt1.setId("app-attempt-1"); + attempt1.setCreatedTime(1425017502003L); + writeEntityFile(attempt1, attemptDir); + ApplicationAttemptEntity attempt2 = new ApplicationAttemptEntity(); + attempt2.setId("app-attempt-2"); + attempt2.setCreatedTime(1425017502004L); + writeEntityFile(attempt2, attemptDir); + + File entityDir = getAppDir(rootDir, "cluster1", "user1", "flow1", "1", + "app1", TimelineEntityType.YARN_CONTAINER.toString()); + ContainerEntity containerEntity1 = new ContainerEntity(); + containerEntity1.setId("container_1_1"); + containerEntity1.setParent(attempt1.getIdentifier()); + containerEntity1.setCreatedTime(1425017502003L); + writeEntityFile(containerEntity1, entityDir); + + ContainerEntity containerEntity2 = new ContainerEntity(); + containerEntity2.setId("container_2_1"); + containerEntity2.setParent(attempt2.getIdentifier()); + containerEntity2.setCreatedTime(1425018502003L); + writeEntityFile(containerEntity2, entityDir); + + ContainerEntity containerEntity3 = new ContainerEntity(); + containerEntity3.setId("container_2_2"); + containerEntity3.setParent(attempt2.getIdentifier()); + containerEntity3.setCreatedTime(1425018502003L); + writeEntityFile(containerEntity3, entityDir); + + File appDir2 = + getAppDir(rootDir, "cluster1", "user1", "flow1,flow", "1", "app2", + "app"); + TimelineEntity entity5 = new TimelineEntity(); + entity5.setId("id_5"); + entity5.setType("app"); + entity5.setCreatedTime(1425016502050L); + writeEntityFile(entity5, appDir2); + } + + private static File getAppDir(String rootDir, String cluster, String user, + String flowName, String flowRunId, String appId, String entityName) { + return new File(rootDir + File.separator + "entities" + File.separator + + cluster + File.separator + user + File.separator + flowName + + File.separator + flowRunId + File.separator + appId + File.separator + + entityName + File.separator); + } + + @Test + public void testGetEntityDefaultView() throws Exception { + // If no fields are specified, entity is returned with default view i.e. + // only the id, type and created time. + TimelineEntity result = reader.getEntity( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", "id_1"), + new TimelineDataToRetrieve(null, null, null, null, null, null)); + Assert.assertEquals( + (new TimelineEntity.Identifier("app", "id_1")).toString(), + result.getIdentifier().toString()); + Assert.assertEquals((Long)1425016502000L, result.getCreatedTime()); + Assert.assertEquals(0, result.getConfigs().size()); + Assert.assertEquals(0, result.getMetrics().size()); + } + + @Test + public void testGetEntityByClusterAndApp() throws Exception { + // Cluster and AppId should be enough to get an entity. + TimelineEntity result = reader.getEntity( + new TimelineReaderContext("cluster1", null, null, null, "app1", "app", + "id_1"), + new TimelineDataToRetrieve(null, null, null, null, null, null)); + Assert.assertEquals( + (new TimelineEntity.Identifier("app", "id_1")).toString(), + result.getIdentifier().toString()); + Assert.assertEquals((Long)1425016502000L, result.getCreatedTime()); + Assert.assertEquals(0, result.getConfigs().size()); + Assert.assertEquals(0, result.getMetrics().size()); + } + + /** This test checks whether we can handle commas in app flow mapping csv. */ + @Test + public void testAppFlowMappingCsv() throws Exception { + // Test getting an entity by cluster and app where flow entry + // in app flow mapping csv has commas. + TimelineEntity result = reader.getEntity( + new TimelineReaderContext("cluster1", null, null, null, "app2", + "app", "id_5"), + new TimelineDataToRetrieve(null, null, null, null, null, null)); + Assert.assertEquals( + (new TimelineEntity.Identifier("app", "id_5")).toString(), + result.getIdentifier().toString()); + Assert.assertEquals((Long)1425016502050L, result.getCreatedTime()); + } + + @Test + public void testGetEntityCustomFields() throws Exception { + // Specified fields in addition to default view will be returned. + TimelineEntity result = reader.getEntity( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", "id_1"), + new TimelineDataToRetrieve(null, null, + EnumSet.of(Field.INFO, Field.CONFIGS, Field.METRICS), null, null, + null)); + Assert.assertEquals( + (new TimelineEntity.Identifier("app", "id_1")).toString(), + result.getIdentifier().toString()); + Assert.assertEquals((Long)1425016502000L, result.getCreatedTime()); + Assert.assertEquals(3, result.getConfigs().size()); + Assert.assertEquals(3, result.getMetrics().size()); + Assert.assertEquals(2, result.getInfo().size()); + // No events will be returned + Assert.assertEquals(0, result.getEvents().size()); + } + + @Test + public void testGetEntityAllFields() throws Exception { + // All fields of TimelineEntity will be returned. + TimelineEntity result = reader.getEntity( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", "id_1"), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null, + null, null)); + Assert.assertEquals( + (new TimelineEntity.Identifier("app", "id_1")).toString(), + result.getIdentifier().toString()); + Assert.assertEquals((Long)1425016502000L, result.getCreatedTime()); + Assert.assertEquals(3, result.getConfigs().size()); + Assert.assertEquals(3, result.getMetrics().size()); + // All fields including events will be returned. + Assert.assertEquals(2, result.getEvents().size()); + } + + @Test + public void testGetAllEntities() throws Exception { + Set result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), new TimelineEntityFilters.Builder().build(), + new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null, + null, null)); + // All 4 entities will be returned + Assert.assertEquals(4, result.size()); + } + + @Test + public void testGetEntitiesWithLimit() throws Exception { + Set result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters.Builder().entityLimit(2L).build(), + new TimelineDataToRetrieve()); + Assert.assertEquals(2, result.size()); + // Needs to be rewritten once hashcode and equals for + // TimelineEntity is implemented + // Entities with id_1 and id_4 should be returned, + // based on created time, descending. + for (TimelineEntity entity : result) { + if (!entity.getId().equals("id_1") && !entity.getId().equals("id_4")) { + Assert.fail("Entity not sorted by created time"); + } + } + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters.Builder().entityLimit(3L).build(), + new TimelineDataToRetrieve()); + // Even though 2 entities out of 4 have same created time, one entity + // is left out due to limit + Assert.assertEquals(3, result.size()); + } + + @Test + public void testGetEntitiesByTimeWindows() throws Exception { + // Get entities based on created time start and end time range. + Set result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters.Builder().createdTimeBegin(1425016502030L) + .createTimeEnd(1425016502060L).build(), + new TimelineDataToRetrieve()); + Assert.assertEquals(1, result.size()); + // Only one entity with ID id_4 should be returned. + for (TimelineEntity entity : result) { + if (!entity.getId().equals("id_4")) { + Assert.fail("Incorrect filtering based on created time range"); + } + } + + // Get entities if only created time end is specified. + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters.Builder().createTimeEnd(1425016502010L) + .build(), + new TimelineDataToRetrieve()); + Assert.assertEquals(3, result.size()); + for (TimelineEntity entity : result) { + if (entity.getId().equals("id_4")) { + Assert.fail("Incorrect filtering based on created time range"); + } + } + + // Get entities if only created time start is specified. + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters.Builder().createdTimeBegin(1425016502010L) + .build(), + new TimelineDataToRetrieve()); + Assert.assertEquals(1, result.size()); + for (TimelineEntity entity : result) { + if (!entity.getId().equals("id_4")) { + Assert.fail("Incorrect filtering based on created time range"); + } + } + } + + @Test + public void testGetFilteredEntities() throws Exception { + // Get entities based on info filters. + TimelineFilterList infoFilterList = new TimelineFilterList(); + infoFilterList.addFilter( + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "info2", 3.5)); + Set result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters.Builder().infoFilters(infoFilterList).build(), + new TimelineDataToRetrieve()); + Assert.assertEquals(1, result.size()); + // Only one entity with ID id_3 should be returned. + for (TimelineEntity entity : result) { + if (!entity.getId().equals("id_3")) { + Assert.fail("Incorrect filtering based on info filters"); + } + } + + // Get entities based on config filters. + TimelineFilterList confFilterList = new TimelineFilterList(); + confFilterList.addFilter( + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "config_1", "123")); + confFilterList.addFilter( + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "config_3", "abc")); + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters.Builder().configFilters(confFilterList) + .build(), + new TimelineDataToRetrieve()); + Assert.assertEquals(1, result.size()); + for (TimelineEntity entity : result) { + if (!entity.getId().equals("id_3")) { + Assert.fail("Incorrect filtering based on config filters"); + } + } + + // Get entities based on event filters. + TimelineFilterList eventFilters = new TimelineFilterList(); + eventFilters.addFilter( + new TimelineExistsFilter(TimelineCompareOp.EQUAL, "event_2")); + eventFilters.addFilter( + new TimelineExistsFilter(TimelineCompareOp.EQUAL, "event_4")); + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters.Builder().eventFilters(eventFilters).build(), + new TimelineDataToRetrieve()); + Assert.assertEquals(1, result.size()); + for (TimelineEntity entity : result) { + if (!entity.getId().equals("id_3")) { + Assert.fail("Incorrect filtering based on event filters"); + } + } + + // Get entities based on metric filters. + TimelineFilterList metricFilterList = new TimelineFilterList(); + metricFilterList.addFilter(new TimelineCompareFilter( + TimelineCompareOp.GREATER_OR_EQUAL, "metric3", 0L)); + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList) + .build(), + new TimelineDataToRetrieve()); + Assert.assertEquals(2, result.size()); + // Two entities with IDs' id_1 and id_2 should be returned. + for (TimelineEntity entity : result) { + if (!entity.getId().equals("id_1") && !entity.getId().equals("id_2")) { + Assert.fail("Incorrect filtering based on metric filters"); + } + } + + // Get entities based on complex config filters. + TimelineFilterList list1 = new TimelineFilterList(); + list1.addFilter( + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "config_1", "129")); + list1.addFilter( + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "config_3", "def")); + TimelineFilterList list2 = new TimelineFilterList(); + list2.addFilter( + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "config_2", "23")); + list2.addFilter( + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "config_3", "abc")); + TimelineFilterList confFilterList1 = + new TimelineFilterList(Operator.OR, list1, list2); + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters.Builder().configFilters(confFilterList1) + .build(), + new TimelineDataToRetrieve()); + Assert.assertEquals(2, result.size()); + for (TimelineEntity entity : result) { + if (!entity.getId().equals("id_1") && !entity.getId().equals("id_2")) { + Assert.fail("Incorrect filtering based on config filters"); + } + } + + TimelineFilterList list3 = new TimelineFilterList(); + list3.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.NOT_EQUAL, "config_1", "123")); + list3.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.NOT_EQUAL, "config_3", "abc")); + TimelineFilterList list4 = new TimelineFilterList(); + list4.addFilter( + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "config_2", "23")); + TimelineFilterList confFilterList2 = + new TimelineFilterList(Operator.OR, list3, list4); + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters.Builder().configFilters(confFilterList2) + .build(), + new TimelineDataToRetrieve()); + Assert.assertEquals(2, result.size()); + for (TimelineEntity entity : result) { + if (!entity.getId().equals("id_1") && !entity.getId().equals("id_2")) { + Assert.fail("Incorrect filtering based on config filters"); + } + } + + TimelineFilterList confFilterList3 = new TimelineFilterList(); + confFilterList3.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.NOT_EQUAL, "config_1", "127")); + confFilterList3.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.NOT_EQUAL, "config_3", "abc")); + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters.Builder().configFilters(confFilterList3) + .build(), + new TimelineDataToRetrieve()); + Assert.assertEquals(1, result.size()); + for(TimelineEntity entity : result) { + if (!entity.getId().equals("id_2")) { + Assert.fail("Incorrect filtering based on config filters"); + } + } + + TimelineFilterList confFilterList4 = new TimelineFilterList(); + confFilterList4.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "config_dummy", "dummy")); + confFilterList4.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "config_3", "def")); + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters.Builder().configFilters(confFilterList4) + .build(), + new TimelineDataToRetrieve()); + Assert.assertEquals(0, result.size()); + + TimelineFilterList confFilterList5 = new TimelineFilterList(Operator.OR); + confFilterList5.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "config_dummy", "dummy")); + confFilterList5.addFilter(new TimelineKeyValueFilter( + TimelineCompareOp.EQUAL, "config_3", "def")); + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters.Builder().configFilters(confFilterList5) + .build(), + new TimelineDataToRetrieve()); + Assert.assertEquals(1, result.size()); + for (TimelineEntity entity : result) { + if (!entity.getId().equals("id_2")) { + Assert.fail("Incorrect filtering based on config filters"); + } + } + + // Get entities based on complex metric filters. + TimelineFilterList list6 = new TimelineFilterList(); + list6.addFilter(new TimelineCompareFilter( + TimelineCompareOp.GREATER_THAN, "metric1", 200)); + list6.addFilter(new TimelineCompareFilter( + TimelineCompareOp.EQUAL, "metric3", 23)); + TimelineFilterList list7 = new TimelineFilterList(); + list7.addFilter(new TimelineCompareFilter( + TimelineCompareOp.GREATER_OR_EQUAL, "metric2", 74)); + TimelineFilterList metricFilterList1 = + new TimelineFilterList(Operator.OR, list6, list7); + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList1) + .build(), + new TimelineDataToRetrieve()); + Assert.assertEquals(2, result.size()); + // Two entities with IDs' id_2 and id_3 should be returned. + for (TimelineEntity entity : result) { + if (!entity.getId().equals("id_2") && !entity.getId().equals("id_3")) { + Assert.fail("Incorrect filtering based on metric filters"); + } + } + + TimelineFilterList metricFilterList2 = new TimelineFilterList(); + metricFilterList2.addFilter(new TimelineCompareFilter( + TimelineCompareOp.LESS_THAN, "metric2", 70)); + metricFilterList2.addFilter(new TimelineCompareFilter( + TimelineCompareOp.LESS_OR_EQUAL, "metric3", 23)); + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList2) + .build(), + new TimelineDataToRetrieve()); + Assert.assertEquals(1, result.size()); + for (TimelineEntity entity : result) { + if (!entity.getId().equals("id_1")) { + Assert.fail("Incorrect filtering based on metric filters"); + } + } + + TimelineFilterList metricFilterList3 = new TimelineFilterList(); + metricFilterList3.addFilter(new TimelineCompareFilter( + TimelineCompareOp.LESS_THAN, "dummy_metric", 30)); + metricFilterList3.addFilter(new TimelineCompareFilter( + TimelineCompareOp.LESS_OR_EQUAL, "metric3", 23)); + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList3) + .build(), + new TimelineDataToRetrieve()); + Assert.assertEquals(0, result.size()); + + TimelineFilterList metricFilterList4 = new TimelineFilterList(Operator.OR); + metricFilterList4.addFilter(new TimelineCompareFilter( + TimelineCompareOp.LESS_THAN, "dummy_metric", 30)); + metricFilterList4.addFilter(new TimelineCompareFilter( + TimelineCompareOp.LESS_OR_EQUAL, "metric3", 23)); + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList4) + .build(), + new TimelineDataToRetrieve()); + Assert.assertEquals(2, result.size()); + for (TimelineEntity entity : result) { + if (!entity.getId().equals("id_1") && !entity.getId().equals("id_2")) { + Assert.fail("Incorrect filtering based on metric filters"); + } + } + + TimelineFilterList metricFilterList5 = + new TimelineFilterList(new TimelineCompareFilter( + TimelineCompareOp.NOT_EQUAL, "metric2", 74)); + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters.Builder().metricFilters(metricFilterList5) + .build(), + new TimelineDataToRetrieve()); + Assert.assertEquals(2, result.size()); + for (TimelineEntity entity : result) { + if (!entity.getId().equals("id_1") && !entity.getId().equals("id_2")) { + Assert.fail("Incorrect filtering based on metric filters"); + } + } + + TimelineFilterList infoFilterList1 = new TimelineFilterList(); + infoFilterList1.addFilter( + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "info2", 3.5)); + infoFilterList1.addFilter( + new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL, "info4", 20)); + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters.Builder().infoFilters(infoFilterList1) + .build(), + new TimelineDataToRetrieve()); + Assert.assertEquals(0, result.size()); + + TimelineFilterList infoFilterList2 = new TimelineFilterList(Operator.OR); + infoFilterList2.addFilter( + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "info2", 3.5)); + infoFilterList2.addFilter( + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "info1", "val1")); + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters.Builder().infoFilters(infoFilterList2) + .build(), + new TimelineDataToRetrieve()); + Assert.assertEquals(2, result.size()); + for (TimelineEntity entity : result) { + if (!entity.getId().equals("id_1") && !entity.getId().equals("id_3")) { + Assert.fail("Incorrect filtering based on info filters"); + } + } + + TimelineFilterList infoFilterList3 = new TimelineFilterList(); + infoFilterList3.addFilter( + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "dummy_info", 1)); + infoFilterList3.addFilter( + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "info2", "val5")); + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters.Builder().infoFilters(infoFilterList3) + .build(), + new TimelineDataToRetrieve()); + Assert.assertEquals(0, result.size()); + + TimelineFilterList infoFilterList4 = new TimelineFilterList(Operator.OR); + infoFilterList4.addFilter( + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "dummy_info", 1)); + infoFilterList4.addFilter( + new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "info2", "val5")); + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters.Builder().infoFilters(infoFilterList4) + .build(), + new TimelineDataToRetrieve()); + Assert.assertEquals(1, result.size()); + for (TimelineEntity entity : result) { + if (!entity.getId().equals("id_1")) { + Assert.fail("Incorrect filtering based on info filters"); + } + } + } + + @Test + public void testGetEntitiesByRelations() throws Exception { + // Get entities based on relatesTo. + TimelineFilterList relatesTo = new TimelineFilterList(Operator.OR); + Set relatesToIds = + new HashSet(Arrays.asList((Object)"flow1")); + relatesTo.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "flow", relatesToIds)); + Set result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters.Builder().relatesTo(relatesTo).build(), + new TimelineDataToRetrieve()); + Assert.assertEquals(1, result.size()); + // Only one entity with ID id_1 should be returned. + for (TimelineEntity entity : result) { + if (!entity.getId().equals("id_1")) { + Assert.fail("Incorrect filtering based on relatesTo"); + } + } + + // Get entities based on isRelatedTo. + TimelineFilterList isRelatedTo = new TimelineFilterList(Operator.OR); + Set isRelatedToIds = + new HashSet(Arrays.asList((Object)"tid1_2")); + isRelatedTo.addFilter(new TimelineKeyValuesFilter( + TimelineCompareOp.EQUAL, "type1", isRelatedToIds)); + result = reader.getEntities( + new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1", + "app", null), + new TimelineEntityFilters.Builder().isRelatedTo(isRelatedTo).build(), + new TimelineDataToRetrieve()); + Assert.assertEquals(2, result.size()); + // Two entities with IDs' id_1 and id_3 should be returned. + for (TimelineEntity entity : result) { + if (!entity.getId().equals("id_1") && !entity.getId().equals("id_3")) { + Assert.fail("Incorrect filtering based on isRelatedTo"); + } + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineWriterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineWriterImpl.java new file mode 100644 index 00000000000..6466194dc12 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineWriterImpl.java @@ -0,0 +1,142 @@ +/** + * 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.timelineservice.storage; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetricOperation; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext; +import org.apache.hadoop.yarn.util.timeline.TimelineUtils; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +/** + * Tests filesystem implemetation for timelineservice writer. + */ +public class TestFileSystemTimelineWriterImpl { + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + + /** + * Unit test for PoC YARN 3264. + * + * @throws Exception + */ + @Test + public void testWriteEntityToFile() throws Exception { + TimelineEntities te = new TimelineEntities(); + TimelineEntity entity = new TimelineEntity(); + String id = "hello"; + String type = "world"; + entity.setId(id); + entity.setType(type); + entity.setCreatedTime(1425016501000L); + te.addEntity(entity); + + TimelineMetric metric = new TimelineMetric(); + String metricId = "CPU"; + metric.setId(metricId); + metric.setType(TimelineMetric.Type.SINGLE_VALUE); + metric.setRealtimeAggregationOp(TimelineMetricOperation.SUM); + metric.addValue(1425016501000L, 1234567L); + + TimelineEntity entity2 = new TimelineEntity(); + String id2 = "metric"; + String type2 = "app"; + entity2.setId(id2); + entity2.setType(type2); + entity2.setCreatedTime(1425016503000L); + entity2.addMetric(metric); + te.addEntity(entity2); + + Map aggregatedMetrics = + new HashMap(); + aggregatedMetrics.put(metricId, metric); + + FileSystemTimelineWriterImpl fsi = null; + try { + fsi = new FileSystemTimelineWriterImpl(); + Configuration conf = new YarnConfiguration(); + String outputRoot = tmpFolder.newFolder().getAbsolutePath(); + conf.set(FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_DIR_ROOT, + outputRoot); + fsi.init(conf); + fsi.start(); + fsi.write( + new TimelineCollectorContext("cluster_id", "user_id", "flow_name", + "flow_version", 12345678L, "app_id"), + te, UserGroupInformation.createRemoteUser("user_id")); + + String fileName = fsi.getOutputRoot() + File.separator + "entities" + + File.separator + "cluster_id" + File.separator + "user_id" + + File.separator + "flow_name" + File.separator + "flow_version" + + File.separator + "12345678" + File.separator + "app_id" + + File.separator + type + File.separator + id + + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION; + Path path = Paths.get(fileName); + File f = new File(fileName); + assertTrue(f.exists() && !f.isDirectory()); + List data = Files.readAllLines(path, StandardCharsets.UTF_8); + // ensure there's only one entity + 1 new line + assertTrue("data size is:" + data.size(), data.size() == 2); + String d = data.get(0); + // confirm the contents same as what was written + assertEquals(d, TimelineUtils.dumpTimelineRecordtoJSON(entity)); + + // verify aggregated metrics + String fileName2 = fsi.getOutputRoot() + File.separator + "entities" + + File.separator + "cluster_id" + File.separator + "user_id" + + File.separator + "flow_name" + File.separator + "flow_version" + + File.separator + "12345678" + File.separator + "app_id" + + File.separator + type2 + File.separator + id2 + + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION; + Path path2 = Paths.get(fileName2); + File file = new File(fileName2); + assertTrue(file.exists() && !file.isDirectory()); + List data2 = Files.readAllLines(path2, StandardCharsets.UTF_8); + // ensure there's only one entity + 1 new line + assertTrue("data size is:" + data.size(), data2.size() == 2); + String metricToString = data2.get(0); + // confirm the contents same as what was written + assertEquals(metricToString, + TimelineUtils.dumpTimelineRecordtoJSON(entity2)); + } finally { + if (fsi != null) { + fsi.close(); + } + } + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/resources/log4j.properties b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/resources/log4j.properties new file mode 100644 index 00000000000..81a3f6ad5d2 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/resources/log4j.properties @@ -0,0 +1,19 @@ +# Licensed 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. + +# log4j configuration used during build and unit tests + +log4j.rootLogger=info,stdout +log4j.threshold=ALL +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2} (%F:%M(%L)) - %m%n diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml index 71feacef608..224b34d587c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml @@ -44,5 +44,8 @@ hadoop-yarn-server-applicationhistoryservice hadoop-yarn-server-timeline-pluginstorage hadoop-yarn-server-router + hadoop-yarn-server-timelineservice + hadoop-yarn-server-timelineservice-hbase + hadoop-yarn-server-timelineservice-hbase-tests diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md index 19e9b72669c..f610cdefe48 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md @@ -74,7 +74,7 @@ Current status Future Plans 1. Future releases will introduce a next generation timeline service -which is scalable and reliable, "Timeline Server v2". +which is scalable and reliable, ["Timeline Service v2"](./TimelineServiceV2.html). 1. The expanded features of this service *may not* be available to applications using the Timeline Server v1 REST API. That includes extended data structures as well as the ability of the client to failover between Timeline Server instances. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md new file mode 100644 index 00000000000..6a0971aed5c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md @@ -0,0 +1,1567 @@ + + +The YARN Timeline Service v.2 +======================== + + + +## Overview + +### Introduction + +YARN Timeline Service v.2 is the next major iteration of Timeline Server, following v.1 and v.1.5. +V.2 is created to address two major challenges of v.1. + +#### Scalability +V.1 is limited to a single instance of writer/reader and storage, and does not scale well beyond +small clusters. V.2 uses a more scalable distributed writer architecture and a scalable backend +storage. + +YARN Timeline Service v.2 separates the collection (writes) of data from serving (reads) of data. +It uses distributed collectors, essentially one collector for each YARN application. The readers +are separate instances that are dedicated to serving queries via REST API. + +YARN Timeline Service v.2 chooses Apache HBase as the primary backing storage, as Apache HBase +scales well to a large size while maintaining good response times for reads and writes. + +#### Usability improvements +In many cases, users are interested in information at the level of "flows" or logical groups of +YARN applications. It is much more common to launch a set or series of YARN applications to +complete a logical application. Timeline Service v.2 supports the notion of flows explicitly. In +addition, it supports aggregating metrics at the flow level. + +Also, information such as configuration and metrics is treated and supported as first-class +citizens. + +The following diagrams illustrates the relationship between different YARN entities modelling flows. + +![Flow Hierarchy](./images/flow_hierarchy.png) + +###Architecture + +YARN Timeline Service v.2 uses a set of collectors (writers) to write data to the backend storage. +The collectors are distributed and co-located with the application masters to which they are +dedicated. All data that belong to that application are sent to the application level timeline +collectors with the exception of the resource manager timeline collector. + +For a given application, the application master can write data for the application to the +co-located timeline collectors (which is an NM auxiliary service in this release). In addition, +node managers of other nodes that are running the containers for the application also write data +to the timeline collector on the node that is running the application master. + +The resource manager also maintains its own timeline collector. It emits only YARN-generic +lifecycle events to keep its volume of writes reasonable. + +The timeline readers are separate daemons separate from the timeline collectors, and they are +dedicated to serving queries via REST API. + +The following diagram illustrates the design at a high level. + +![Timeline Service v.2 architecture](./images/timeline_v2.jpg) + +### Current Status and Future Plans + +YARN Timeline Service v.2 is currently in alpha ("alpha 2"). It is a work in progress, and +many things can and will change rapidly. + +A complete end-to-end flow of writes and reads is functional, with Apache HBase as the backend. +You should be able to start generating data. When enabled, all YARN-generic events are +published as well as YARN system metrics such as CPU and memory. Furthermore, some applications +including Distributed Shell and MapReduce can write per-framework data to YARN Timeline Service +v.2. + +The basic mode of accessing data is via REST. Currently there is no support for command line +access. The REST API comes with a good number of useful and flexible query patterns (see below for +more information). + +The collectors (writers) are currently embedded in the node managers as auxiliary services. The +resource manager also has its dedicated in-process collector. The reader is currently a single +instance. Currently, it is not possible to write to Timeline Service outside the context of a YARN +application (i.e. no off-cluster client). + +Starting from alpha2, Timeline Service v.2 supports simple authorization in terms of a +configurable whitelist of users and groups who can read timeline data. Cluster admins are +allowed by default to read timeline data. + +When YARN Timeline Service v.2 is disabled, one can expect no functional or performance impact +on any other existing functionality. + +The work to make it truly production-ready continues. Some key items include + +* More robust storage fault tolerance +* Support for off-cluster clients +* Better support for long-running apps +* Support for ACLs +* Offline (time-based periodic) aggregation for flows, users, and queues for reporting and +analysis +* Timeline collectors as separate instances from node managers +* Clustering of the readers +* Migration and compatibility with v.1 + + +## Deployment + +### Configurations + +New configuration parameters that are introduced with v.2 are marked bold. + +#### Basic configuration + +| Configuration Property | Description | +|:---- |:---- | +| `yarn.timeline-service.enabled` | Indicate to clients whether Timeline service is enabled or not. If enabled, the `TimelineClient` library used by applications will post entities and events to the Timeline server. Defaults to `false`. | +| `yarn.timeline-service.version` | Indicate what is the current version of the running timeline service. For example, if "yarn.timeline-service.version" is 1.5, and "yarn.timeline-service.enabled" is true, it means the cluster will and must bring up the timeline service v.1.5 (and nothing else). On the client side, if the client uses the same version of timeline service, it must succeed. If the client chooses to use a smaller version in spite of this, then depending on how robust the compatibility story is between versions, the results may vary. Defaults to `1.0f`. | +| **`yarn.timeline-service.writer.class`** | The class for the backend storage writer. Defaults to HBase storage writer. | +| **`yarn.timeline-service.reader.class`** | The class for the backend storage reader. Defaults to HBase storage reader. | +| **`yarn.system-metrics-publisher.enabled`** | The setting that controls whether yarn system metrics is published on the Timeline service or not by RM And NM. Defaults to `false`. | +| **`yarn.timeline-service.schema.prefix`** | The schema prefix for hbase tables. Defaults to "prod.". | + +#### Advanced configuration + +| Configuration Property | Description | +|:---- |:---- | +| `yarn.timeline-service.hostname` | The hostname of the Timeline service web application. Defaults to `0.0.0.0` | +| `yarn.timeline-service.address` | Address for the Timeline server to start the RPC server. Defaults to `${yarn.timeline-service.hostname}:10200`. | +| `yarn.timeline-service.webapp.address` | The http address of the Timeline service web application. Defaults to `${yarn.timeline-service.hostname}:8188`. | +| `yarn.timeline-service.webapp.https.address` | The https address of the Timeline service web application. Defaults to `${yarn.timeline-service.hostname}:8190`. | +| **`yarn.timeline-service.hbase.configuration.file`** | Optional URL to an hbase-site.xml configuration file to be used to connect to the timeline-service hbase cluster. If empty or not specified, then the HBase configuration will be loaded from the classpath. When specified the values in the specified configuration file will override those from the ones that are present on the classpath. Defaults to `null`. | +| **`yarn.timeline-service.writer.flush-interval-seconds`** | The setting that controls how often the timeline collector flushes the timeline writer. Defaults to `60`. | +| **`yarn.timeline-service.app-collector.linger-period.ms`** | Time period till which the application collector will be alive in NM, after the application master container finishes. Defaults to `1000` (1 second). | +| **`yarn.timeline-service.timeline-client.number-of-async-entities-to-merge`** | Time line V2 client tries to merge these many number of async entities (if available) and then call the REST ATS V2 API to submit. Defaults to `10`. | +| **`yarn.timeline-service.hbase.coprocessor.app-final-value-retention-milliseconds`** | The setting that controls how long the final value of a metric of a completed app is retained before merging into the flow sum. Defaults to `259200000` (3 days). This should be set in the HBase cluster. | +| **`yarn.rm.system-metrics-publisher.emit-container-events`** | The setting that controls whether yarn container metrics is published to the timeline server or not by RM. This configuration setting is for ATS V2. Defaults to `false`. | + +#### Security Configuration + + +Security can be enabled by setting `yarn.timeline-service.http-authentication.type` +to `kerberos`, after which the following configuration options are available: + + +| Configuration Property | Description | +|:---- |:---- | +| `yarn.timeline-service.http-authentication.type` | Defines authentication used for the timeline server(collector/reader) HTTP endpoint. Supported values are: `simple` / `kerberos` / #AUTHENTICATION_HANDLER_CLASSNAME#. Defaults to `simple`. | +| `yarn.timeline-service.http-authentication.simple.anonymous.allowed` | Indicates if anonymous requests are allowed by the timeline server when using 'simple' authentication. Defaults to `true`. | +| `yarn.timeline-service.http-authentication.kerberos.principal` | The Kerberos principal to be used for the Timeline Server(Collector/Reader) HTTP endpoint. | +| `yarn.timeline-service.http-authentication.kerberos.keytab` | The Kerberos keytab to be used for the Timeline Server(Collector/Reader) HTTP endpoint.. | +| `yarn.timeline-service.principal` | The Kerberos principal for the timeline reader. NM principal would be used for timeline collector as it runs as an auxiliary service inside NM. | +| `yarn.timeline-service.keytab` | The Kerberos keytab for the timeline reader. NM keytab would be used for timeline collector as it runs as an auxiliary service inside NM. | +| `yarn.timeline-service.delegation.key.update-interval` | Defaults to `86400000` (1 day). | +| `yarn.timeline-service.delegation.token.renew-interval` | Defaults to `86400000` (1 day). | +| `yarn.timeline-service.delegation.token.max-lifetime` | Defaults to `604800000` (7 days). | +| `yarn.timeline-service.read.authentication.enabled` | Enables or disables authorization checks for reading timeline service v2 data. Default is `false` which is disabled. | +| `yarn.timeline-service.read.allowed.users` | Comma separated list of user, followed by space, then comma separated list of groups. It will allow this list of users and groups to read the data and reject everyone else. Default value is set to none. If authorization is enabled, then this configuration is mandatory. | + +#### Enabling CORS support +To enable cross-origin support (CORS) for the Timeline Service v.2, please set the following configuration parameters: + +In yarn-site.xml, set yarn.timeline-service.http-cross-origin.enabled to true. + +In core-site.xml, add org.apache.hadoop.security.HttpCrossOriginFilterInitializer to hadoop.http.filter.initializers. + +For more configurations used for cross-origin support, refer to [HttpAuthentication](../../hadoop-project-dist/hadoop-common/HttpAuthentication.html#CORS). Please note that yarn.timeline-service.http-cross-origin.enabled, if set to true, overrides hadoop.http.cross-origin.enabled. + + + +### Enabling Timeline Service v.2 + +#### Preparing Apache HBase cluster for storage +There are a few steps to be done for preparing the storage for Timeline Service v.2: + +Step 1) [Set up the HBase cluster](#Set_up_the_HBase_cluster) + +Step 2) [Enable the coprocessor](#Enable_the_coprocessor) + +Step 3) [Create the schema for Timeline Service v.2](#Create_schema) + +Each step is explained in more detail below. + +##### Step 1) Set up the HBase cluster +The first part is to set up or pick an Apache HBase cluster to use as the storage cluster. The +version of Apache HBase that is supported with Timeline Service v.2 is 1.2.6. The 1.0.x versions +do not work with Timeline Service v.2. Later versions of HBase have not been tested with +Timeline Service. + +HBase has different deployment modes. Refer to the HBase book for understanding them and pick a +mode that is suitable for your setup. +(http://hbase.apache.org/book.html#standalone_dist) + +##### Simple deployment for HBase +If you are intent on a simple deploy profile for the Apache HBase cluster +where the data loading is light but the data needs to persist across node +comings and goings, you could consider the "Standalone HBase over HDFS" deploy mode. + +This is a useful variation on the standalone HBase setup and has all HBase daemons running inside +one JVM but rather than persisting to the local filesystem, it persists to an HDFS instance. +Writing to HDFS where data is replicated ensures that data is persisted across node +comings and goings. To configure this standalone variant, edit your `hbase-site.xml` setting +the `hbase.rootdir` to point at a directory in your HDFS instance but then set +`hbase.cluster.distributed` to false. For example: + +``` + + + hbase.rootdir + hdfs://namenode.example.org:8020/hbase + + + hbase.cluster.distributed + false + + +``` + +For more details on this mode, refer to +http://hbase.apache.org/book.html#standalone.over.hdfs . + +Once you have an Apache HBase cluster ready to use, perform the following steps. + +##### Step 2) Enable the coprocessor +In this version, the coprocessor is loaded dynamically (table coprocessor for the `flowrun` table). + +Copy the timeline service jar to HDFS from where HBase can load it. It +is needed for the `flowrun` table creation in the schema creator. The default HDFS location is `/hbase/coprocessor`. +For example, + + hadoop fs -mkdir /hbase/coprocessor + hadoop fs -put hadoop-yarn-server-timelineservice-hbase-3.0.0-alpha1-SNAPSHOT.jar + /hbase/coprocessor/hadoop-yarn-server-timelineservice.jar + + +If you want to place the jar at a different location on hdfs, there also exists a yarn +configuration setting called `yarn.timeline-service.hbase.coprocessor.jar.hdfs.location`. +For example, + +``` + + yarn.timeline-service.hbase.coprocessor.jar.hdfs.location + /custom/hdfs/path/jarName + +``` + +##### Step 3) Create the timeline service schema +Finally, run the schema creator tool to create the necessary tables: + + bin/hadoop org.apache.hadoop.yarn.server.timelineservice.storage.TimelineSchemaCreator -create + +The `TimelineSchemaCreator` tool supports a few options that may come handy especially when you +are testing. For example, you can use `-skipExistingTable` (`-s` for short) to skip existing tables +and continue to create other tables rather than failing the schema creation. When no option or '-help' +('-h' for short) is provided, the command usage is printed. By default, the tables +will have a schema prefix of "prod." + +#### Enabling Timeline Service v.2 +Following are the basic configurations to start Timeline service v.2: + +``` + + yarn.timeline-service.version + 2.0f + + + + yarn.timeline-service.enabled + true + + + + yarn.nodemanager.aux-services + mapreduce_shuffle,timeline_collector + + + + yarn.nodemanager.aux-services.timeline_collector.class + org.apache.hadoop.yarn.server.timelineservice.collector.PerNodeTimelineCollectorsAuxService + + + + The setting that controls whether yarn system metrics is + published on the Timeline service or not by RM And NM. + yarn.system-metrics-publisher.enabled + true + + + + The setting that controls whether yarn container events are + published to the timeline service or not by RM. This configuration setting + is for ATS V2. + yarn.rm.system-metrics-publisher.emit-container-events + true + +``` + +In addition, you may want to set the YARN cluster name to a reasonably unique value in case you +are using multiple clusters to store data in the same Apache HBase storage: + +``` + + yarn.resourcemanager.cluster-id + my_research_test_cluster + +``` + +Also, add the `hbase-site.xml` configuration file to the client Hadoop cluster configuration so +that it can write data to the Apache HBase cluster you are using, or set +`yarn.timeline-service.hbase.configuration.file` to the file URL pointing to +`hbase-site.xml` for the same. For example: + +``` + + Optional URL to an hbase-site.xml configuration file to be + used to connect to the timeline-service hbase cluster. If empty or not + specified, then the HBase configuration will be loaded from the classpath. + When specified the values in the specified configuration file will override + those from the ones that are present on the classpath. + + yarn.timeline-service.hbase.configuration.file + file:/etc/hbase/hbase-ats-dc1/hbase-site.xml + +``` + +#### Running Timeline Service v.2 +Restart the resource manager as well as the node managers to pick up the new configuration. The +collectors start within the resource manager and the node managers in an embedded manner. + +The Timeline Service reader is a separate YARN daemon, and it can be started using the following +syntax: + + $ yarn-daemon.sh start timelinereader + +#### Enabling MapReduce to write to Timeline Service v.2 +To write MapReduce framework data to Timeline Service v.2, enable the following configuration in +`mapred-site.xml`: + +``` + + mapreduce.job.emit-timeline-data + true + +``` + +### Upgrade from alpha1 to alpha2 +If you are currently running Timeline Service v2 alpha1 version, we recommend the following: + +- Clear existing data in tables (truncate tables) since the row key for AppToFlow has changed. + +- The coprocessor is now a dynamically loaded table level coprocessor in alpha2. We +recommend dropping the table, replacing the coprocessor jar on hdfs with the alpha2 one, +restarting the Region servers and recreating the `flowrun` table. + +### Publishing application specific data + +This section is for YARN application developers that want to integrate with Timeline Service v.2. + +Developers need to use the `TimelineV2Client` API to publish per-framework data to the +Timeline Service v.2. The entity/object API for v.2 is different than v.1 as +the object model is significantly changed. The v.2 timeline entity class is +`org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity`. + +Timeline Service v.2 `putEntities` methods come in 2 varieties: `putEntities` and +`putEntitiesAsync`. The former is a blocking operation which must be used for writing more +critical data (e.g. lifecycle events). The latter is a non-blocking operation. Note that neither +has a return value. + +Creating a `TimelineV2Client` involves passing in the application id to the static method +`TimelineV2Client.createTimelineClient`. + +For example: + + + // Create and start the Timeline client v.2 + TimelineV2Client timelineClient = + TimelineV2Client.createTimelineClient(appId); + timelineClient.init(conf); + timelineClient.start(); + + try { + TimelineEntity myEntity = new TimelineEntity(); + myEntity.setType("MY_APPLICATION"); + myEntity.setId("MyApp1"); + // Compose other entity info + + // Blocking write + timelineClient.putEntities(myEntity); + + TimelineEntity myEntity2 = new TimelineEntity(); + // Compose other info + + // Non-blocking write + timelineClient.putEntitiesAsync(myEntity2); + + } catch (IOException | YarnException e) { + // Handle the exception + } finally { + // Stop the Timeline client + timelineClient.stop(); + } + +As evidenced above, you need to specify the YARN application id to be able to write to the Timeline +Service v.2. Note that currently you need to be on the cluster to be able to write to the Timeline +Service. For example, an application master or code in the container can write to the Timeline +Service, while an off-cluster MapReduce job submitter cannot. + +After creating the timeline v2 client, user also needs to set the timeline collector info which contains the collector address and collector token(only in secure mode) for the application. If `AMRMClient` is used then by registering the timeline client by calling `AMRMClient#registerTimelineV2Client` is sufficient. + + amRMClient.registerTimelineV2Client(timelineClient); + +Else address needs to be retrieved from the AM allocate response and need to be set in timeline client explicitly. + + timelineClient.setTimelineCollectorInfo(response.getCollectorInfo()); + +You can create and publish your own entities, events, and metrics as with previous versions. + +TimelineEntity objects have the following fields to hold timeline data: + +* events: A set of TimelineEvents, ordered by the timestamp of the events in descending +order. Each event contains one id and a map to store related information and is +associated with one timestamp. +* configs: A map from a string (config name) to a string (config value) representing all +configs associated with the entity. Users can post the whole config or a part of it in the +configs field. Supported for application and generic entities. Supported for application and +generic entities. +* metrics: A set of metrics related to this entity. There are two types of metrics: single +value metric and time series metric. Each metric item contains metric name (id), value, and what +kind of aggregation operation should be performed in this metric (no­op by default). Supported for +flow run, application and generic entities. +* info: A map from a string (info key name) to an object (info value) to hold up related +information for this entity. Supported for application and generic entities. +* isrelatedtoEntities and relatestoEntities: It is also possible to represent relationships between +entities. Each entity contains a relatestoEntities and isrelatedtoEntities fields to represent +relationships to other entities. Both fields are represented by a map from a string (the name of +the relationship) to a timeline entity. In this way relationships among entities can be +represented as a DAG. + +Note that when posting timeline metrics, one may choose how each metric should be aggregated +through the `TimelineMetric#setRealtimeAggregationOp()` method. The word "aggregate" here means +applying one of the `TimelineMetricOperation` for a set of entities. Timeline service v2 provides +built-in application level aggregation, which means aggregating metrics from different timeline +entities within one YARN application. Right now, there are two kinds of operations supported in +`TimelineMetricOperation`: + +* `MAX`: Getting the maximum value among all `TimelineMetric` objects. +* `SUM`: Getting the sum of all `TimelineMetric` objects. + +By default, the `NOP` operation means not performing any real-time aggregation operation. + +Application frameworks must set the "flow context" whenever possible in order to take advantage +of the flow support Timeline Service v.2 provides. The flow context consists of the following: + +* Flow name: a string that identifies the high-level flow (e.g. "distributed grep" or any +identifiable name that can uniquely represent the app) +* Flow run id: a monotonically-increasing sequence of numbers that distinguish different runs of +the same flow. +* (optional) Flow version: a string identifier that denotes a version of the flow. Flow version can +be used to identify changes in the flows, such as code changes or script changes. + +If the flow context is not specified, defaults are supplied for these attributes: + +* Flow name: the YARN application name (or the application id if the name is not set) +* Flow run id: the application start time in Unix time (milliseconds) +* Flow version: "1" + +You can provide the flow context via YARN application tags: + + ApplicationSubmissionContext appContext = app.getApplicationSubmissionContext(); + + // set the flow context as YARN application tags + Set tags = new HashSet<>(); + tags.add(TimelineUtils.generateFlowNameTag("distributed grep")); + tags.add(Timelineutils.generateFlowVersionTag("3df8b0d6100530080d2e0decf9e528e57c42a90a")); + tags.add(TimelineUtils.generateFlowRunIdTag(System.currentTimeMillis())); + + appContext.setApplicationTags(tags); + + +## Timeline Service v.2 REST API + +Querying Timeline Service v.2 is currently only supported via REST API; there is no API client +implemented in the YARN libraries. + +The v.2 REST API is implemented at under the path, `/ws/v2/timeline/` on the Timeline Service web +service. + +Here is an informal description of the API. + +### Root path + + GET /ws/v2/timeline/ + +Returns a JSON object describing the service instance and version information. + + { + "About":"Timeline Reader API", + "timeline-service-version":"3.0.0-alpha1-SNAPSHOT", + "timeline-service-build-version":"3.0.0-alpha1-SNAPSHOT from fb0acd08e6f0b030d82eeb7cbfa5404376313e60 by sjlee source checksum be6cba0e42417d53be16459e1685e7", + "timeline-service-version-built-on":"2016-04-11T23:15Z", + "hadoop-version":"3.0.0-alpha1-SNAPSHOT", + "hadoop-build-version":"3.0.0-alpha1-SNAPSHOT from fb0acd08e6f0b030d82eeb7cbfa5404376313e60 by sjlee source checksum ee968fd0aedcc7384230ee3ca216e790", + "hadoop-version-built-on":"2016-04-11T23:14Z" + } + +The following shows the supported queries on the REST API. + +### Query Flows + +With Query Flows API, you can retrieve a list of active flows that had runs most recently. +If the REST endpoint without the cluster name is used, the cluster specified by the configuration +`yarn.resourcemanager.cluster-id` in `yarn-site.xml` is taken. If none of the flows match the +predicates, an empty list will be returned. + +#### HTTP request: + + GET /ws/v2/timeline/clusters/{cluster name}/flows/ + + or + + GET /ws/v2/timeline/flows/ + +#### Query Parameters Supported: + +1. `limit` - If specified, defines the number of flows to return. The maximum possible value for limit + is maximum value of Long. If it is not specified or has a value less than 0, then limit will be + considered as 100. +1. `daterange` - If specified is given as "[startdate]-[enddate]"(i.e. start and end date separated by + "-") or single date. Dates are interpreted in the yyyyMMdd format and are assumed to be in UTC. + If a single date is specified, all flows active on that date are returned. If both startdate and enddate + is given, all flows active between start and end date will be returned. If only startdate is given, flows + active on and after startdate are returned. If only enddate is given, flows active on and before enddate + are returned.
+ _For example_ :
+ "daterange=20150711" returns flows active on 20150711.
+ "daterange=20150711-20150714" returns flows active between these 2 dates.
+ "daterange=20150711-" returns flows active on and after 20150711.
+ "daterange=-20150711" returns flows active on and before 20150711.
+1. `fromid` - If specified, retrieve the next set of flows from the given fromid. The set of entities retrieved is inclusive of specified fromid. + fromid should be taken from the value associated with FROM_ID info key in flow entity response which was sent earlier. + +#### Example JSON Response: + + [ + { + "metrics": [], + "events": [], + "id": "test-cluster/1460419200000/sjlee@ds-date", + "type": "YARN_FLOW_ACTIVITY", + "createdtime": 0, + "flowruns": [ + { + "metrics": [], + "events": [], + "id": "sjlee@ds-date/1460420305659", + "type": "YARN_FLOW_RUN", + "createdtime": 0, + "info": { + "SYSTEM_INFO_FLOW_VERSION": "1", + "SYSTEM_INFO_FLOW_RUN_ID": 1460420305659, + "SYSTEM_INFO_FLOW_NAME": "ds-date", + "SYSTEM_INFO_USER": "sjlee" + }, + "isrelatedto": {}, + "relatesto": {} + }, + { + "metrics": [], + "events": [], + "id": "sjlee@ds-date/1460420587974", + "type": "YARN_FLOW_RUN", + "createdtime": 0, + "info": { + "SYSTEM_INFO_FLOW_VERSION": "1", + "SYSTEM_INFO_FLOW_RUN_ID": 1460420587974, + "SYSTEM_INFO_FLOW_NAME": "ds-date", + "SYSTEM_INFO_USER": "sjlee" + }, + "isrelatedto": {}, + "relatesto": {} + } + ], + "info": { + "SYSTEM_INFO_CLUSTER": "test-cluster", + "UID": "test-cluster!sjlee!ds-date", + "FROM_ID": "test-cluster!1460419200000!sjlee!ds-date", + "SYSTEM_INFO_FLOW_NAME": "ds-date", + "SYSTEM_INFO_DATE": 1460419200000, + "SYSTEM_INFO_USER": "sjlee" + }, + "isrelatedto": {}, + "relatesto": {} + } + ] + +#### Response Codes + +1. If successful, a HTTP 200 (OK) response is returned. +1. If any problem occurs in parsing request, HTTP 400 (Bad Request) is returned. +1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned. + + +### Query Flow Runs + +With Query Flow Runs API, you can drill further down to get the runs (specific instances) of a +given flow. This returns the most recent runs that belong to the given flow. If the REST +endpoint without the cluster name is used, the cluster specified by the configuration +`yarn.resourcemanager.cluster-id` in `yarn-site.xml` is taken. If none of the flow runs match the +predicates, an empty list will be returned. + +#### HTTP request: + + GET /ws/v2/timeline/clusters/{cluster name}/users/{user name}/flows/{flow name}/runs/ + + or + + GET /ws/v2/timeline/users/{user name}/flows/{flow name}/runs/ + +#### Query Parameters Supported: + +1. `limit` - If specified, defines the number of flows to return. The maximum possible value for limit + is maximum value of Long. If it is not specified or has a value less than 0, then limit will be + considered as 100. +1. `createdtimestart` - If specified, then only flow runs started after this timestamp are returned. +1. `createdtimeend` - If specified, then only flow runs started before this timestamp are returned. +1. `metricstoretrieve` - If specified, defines which metrics to retrieve or which ones not to retrieve and send back in response. + metricstoretrieve can be an expression of the form :
+ (<metricprefix>,<metricprefix>,<metricprefix>,<metricprefix>...)
+ This specifies a comma separated list of metric id prefixes. Only metrics matching any of the prefixes will be retrieved. Brackets are optional for the + simple expression. Alternatively, expressions can be of the form :
+ !(<metricprefix>,<metricprefix>,<metricprefix>,<metricprefix>...)
+ This specifies a comma separated list of metric id prefixes. Only metrics not matching any of the prefixes will be retrieved.
+ If metricstoretrieve is specified, metrics will be retrieved irrespective of whether `METRICS` is specified in fields query param + or not. Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `fields` - Specifies which fields to retrieve. For querying flow runs, only `ALL` or `METRICS` are valid fields. + Other fields will lead to HTTP 400 (Bad Request) response. If not specified, in response, id, type, createdtime and info fields + will be returned. +1. `fromid` - If specified, retrieve the next set of flow run entities from the given fromid. The set of entities retrieved is inclusive of specified fromid. + fromid should be taken from the value associated with FROM_ID info key in flow entity response which was sent earlier. + +#### Example JSON Response: + + [ + { + "metrics": [], + "events": [], + "id": "sjlee@ds-date/1460420587974", + "type": "YARN_FLOW_RUN", + "createdtime": 1460420587974, + "info": { + "UID": "test-cluster!sjlee!ds-date!1460420587974", + "FROM_ID": "test-cluster!sjlee!ds-date!1460420587974", + "SYSTEM_INFO_FLOW_RUN_ID": 1460420587974, + "SYSTEM_INFO_FLOW_NAME": "ds-date", + "SYSTEM_INFO_FLOW_RUN_END_TIME": 1460420595198, + "SYSTEM_INFO_USER": "sjlee" + }, + "isrelatedto": {}, + "relatesto": {} + }, + { + "metrics": [], + "events": [], + "id": "sjlee@ds-date/1460420305659", + "type": "YARN_FLOW_RUN", + "createdtime": 1460420305659, + "info": { + "UID": "test-cluster!sjlee!ds-date!1460420305659", + "FROM_ID": "test-cluster!sjlee!ds-date!1460420305659", + "SYSTEM_INFO_FLOW_RUN_ID": 1460420305659, + "SYSTEM_INFO_FLOW_NAME": "ds-date", + "SYSTEM_INFO_FLOW_RUN_END_TIME": 1460420311966, + "SYSTEM_INFO_USER": "sjlee" + }, + "isrelatedto": {}, + "relatesto": {} + } + ] + +#### Response Codes + +1. If successful, a HTTP 200 (OK) response is returned. +1. If any problem occurs in parsing request or if an invalid field is specified in fields query param, HTTP 400 (Bad Request) is returned. +1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned. + + +### Query Flow Run + +With this API, you can query a specific flow run identified by cluster, user, flow name and run id. +If the REST endpoint without the cluster name is used, the cluster specified by the configuration +`yarn.resourcemanager.cluster-id` in `yarn-site.xml` is taken. Metrics are returned by default +while querying individual flow runs. + +#### HTTP request: + + GET /ws/v2/timeline/clusters/{cluster name}/users/{user name}/flows/{flow name}/runs/{run id} + + or + + GET /ws/v2/timeline/users/{user name}/flows/{flow name}/runs/{run id} + +#### Query Parameters Supported: + +1. `metricstoretrieve` - If specified, defines which metrics to retrieve or which ones not to retrieve and send back in response. + metricstoretrieve can be an expression of the form :
+ (<metricprefix>,<metricprefix>,<metricprefix>,<metricprefix>...)
+ This specifies a comma separated list of metric id prefixes. Only metrics matching any of the prefixes will be retrieved. Brackets are optional for the + simple expression. Alternatively, expressions can be of the form :
+ !(<metricprefix>,<metricprefix>,<metricprefix>,<metricprefix>...)
+ This specifies a comma separated list of metric id prefixes. Only metrics not matching any of the prefixes will be retrieved.
+ Please note that URL unsafe characters such as spaces will have to be suitably encoded. + +#### Example JSON Response: + + { + "metrics": [ + { + "type": "SINGLE_VALUE", + "id": "org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter:BYTES_READ", + "aggregationOp": "NOP", + "values": { + "1465246377261": 118 + } + }, + { + "type": "SINGLE_VALUE", + "id": "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter:BYTES_WRITTEN", + "aggregationOp": "NOP", + "values": { + "1465246377261": 97 + } + } + ], + "events": [], + "id": "varun@QuasiMonteCarlo/1465246348599", + "type": "YARN_FLOW_RUN", + "createdtime": 1465246348599, + "isrelatedto": {}, + "info": { + "UID":"yarn-cluster!varun!QuasiMonteCarlo!1465246348599", + "FROM_ID":"yarn-cluster!varun!QuasiMonteCarlo!1465246348599", + "SYSTEM_INFO_FLOW_RUN_END_TIME":1465246378051, + "SYSTEM_INFO_FLOW_NAME":"QuasiMonteCarlo", + "SYSTEM_INFO_USER":"varun", + "SYSTEM_INFO_FLOW_RUN_ID":1465246348599 + }, + "relatesto": {} + } + +#### Response Codes + +1. If successful, a HTTP 200(OK) response is returned. +1. If any problem occurs in parsing request, HTTP 400 (Bad Request) is returned. +1. If flow run for the given flow run id cannot be found, HTTP 404 (Not Found) is returned. +1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned. + +### Query Apps for a flow + +With this API, you can query all the YARN applications that are part of a specific flow. If the +REST endpoint without the cluster name is used, the cluster specified by the configuration +`yarn.resourcemanager.cluster-id` in `yarn-site.xml` is taken. If the number of matching +applications are more than the limit, the most recent apps up to the limit will be returned. If +none of the apps match the predicates, an empty list will be returned. + +#### HTTP request: + + GET /ws/v2/timeline/clusters/{cluster name}/users/{user name}/flows/{flow name}/apps + + or + + GET /ws/v2/timeline/users/{user name}/flows/{flow name}/apps + +#### Query Parameters Supported: + +1. `limit` - If specified, defines the number of applications to return. The maximum possible value for limit + is maximum value of Long. If it is not specified or has a value less than 0, then limit will be + considered as 100. +1. `createdtimestart` - If specified, then only applications created after this timestamp are returned. +1. `createdtimeend` - If specified, then only applications created before this timestamp are returned. +1. `relatesto` - If specified, matched applications must relate to or not relate to given entities associated with a entity type. + relatesto is represented as an expression of the form :
+ "(<entitytype>:<entityid>:<entityid>...,<entitytype>:<entityid>:<entityid>...) <op> !(<entitytype>:<entityid>:<entityid>...,<entitytype>:<entityid>:<entityid>...)".
+ If relatesto expression has entity type - entity id(s) relations specified within enclosing brackets proceeding "!", this means apps with + these relations in its relatesto field, will not be returned. For expressions or subexpressions without "!", all apps which have the specified + relations in its relatesto field, will be returned. "op" is a logical operator and can be either AND or OR. entity type can be followed by any number + of entity id(s). And we can combine any number of ANDs' and ORs' to create complex expressions. Brackets can be used to club expressions together.
+ _For example_ : relatesto can be "(((type1:id1:id2:id3,type3:id9) AND !(type2:id7:id8)) OR (type1:id4))".
+ Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `isrelatedto` - If specified, matched applications must be related to or not related to given entities associated with a entity type. isrelatedto is + represented in the same form as relatesto. +1. `infofilters` - If specified, matched applications must have exact matches to the given info key and must be either equal or not equal to + given value. The info key is a string but value can be any object. infofilters are represented as an expression of the form :
+ "(<key> <compareop> <value>) <op> (<key> <compareop> <value>)".
+ Here op can be either of AND or OR. And compareop can be either of "eq", "ne" or "ene".
+ "eq" means equals, "ne" means not equals and existence of key is not required for a match and "ene" means not equals but existence of key is + required. We can combine any number of ANDs' and ORs' to create complex expressions. Brackets can be used to club expressions together.
+ _For example_ : infofilters can be "(((infokey1 eq value1) AND (infokey2 ne value1)) OR (infokey1 ene value3))".
+ Note : If value is an object then value can be given in the form of JSON format without any space.
+ _For example_ : infofilters can be (infokey1 eq {"<key>":"<value>","<key>":"<value>"...}).
+ Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `conffilters` - If specified, matched applications must have exact matches to the given config name and must be either equal or not equal + to the given config value. Both the config name and value must be strings. conffilters are represented in the same form as infofilters. +1. `metricfilters` - If specified, matched applications must have exact matches to the given metric and satisfy the specified relation with the + metric value. Metric id must be a string and and metric value must be an integral value. metricfilters are represented as an expression of the form :
+ "(<metricid> <compareop> <metricvalue>) <op> (<metricid> <compareop> <metricvalue>)".
+ Here op can be either of AND or OR. And compareop can be either of "eq", "ne", "ene", "gt", "ge", "lt" and "le".
+ "eq" means equals, "ne" means not equals and existence of metric is not required for a match, "ene" means not equals but existence of metric is + required, "gt" means greater than, "ge" means greater than or equals, "lt" means less than and "le" means less than or equals. We can combine + any number of ANDs' and ORs' to create complex expressions. Brackets can be used to club expressions together.
+ _For example_ : metricfilters can be "(((metric1 eq 50) AND (metric2 gt 40)) OR (metric1 lt 20))".
+ This in essence is an expression equivalent to "(metric1 == 50 AND metric2 > 40) OR (metric1 < 20)"
+ Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `eventfilters` - If specified, matched applications must contain or not contain the given events depending on the expression. eventfilters is + represented as an expression of the form :
+ "(<eventid>,<eventid>) <op> !(<eventid>,<eventid>,<eventid>)".
+ Here, "!" means none of the comma-separated list of events within the enclosed brackets proceeding "!" must exist for a match to occur. + If "!" is not specified, the specified events within the enclosed brackets must exist. op is a logical operator and can be either AND or OR. + We can combine any number of ANDs' and ORs' to create complex expressions. Brackets can be used to club expressions together.
+ _For example_ : eventfilters can be "(((event1,event2) AND !(event4)) OR (event3,event7,event5))".
+ Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `metricstoretrieve` - If specified, defines which metrics to retrieve or which ones not to retrieve and send back in response. + metricstoretrieve can be an expression of the form :
+ (<metricprefix>,<metricprefix>,<metricprefix>,<metricprefix>...)
+ This specifies a comma separated list of metric id prefixes. Only metrics matching any of the prefixes will be retrieved. Brackets are optional for + the simple expression. Alternatively, expressions can be of the form:
+ !(<metricprefix>,<metricprefix>,<metricprefix>,<metricprefix>...)
+ This specifies a comma separated list of metric id prefixes. Only metrics not matching any of the prefixes will be retrieved.
+ If metricstoretrieve is specified, metrics will be retrieved irrespective of whether `METRICS` is specified in fields query param + or not. Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `confstoretrieve` - If specified, defines which configs to retrieve or which ones not to retrieve and send back in response. + confstoretrieve can be an expression of the form :
+ (<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>...)
+ This specifies a comma separated list of config name prefixes. Only configs matching any of the prefixes will be retrieved. Brackets are optional for + the simple expression. Alternatively, expressions can be of the form:
+ !(<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>...)
+ This specifies a comma separated list of config name prefixes. Only configs not matching any of the prefixes will be retrieved.
+ If confstoretrieve is specified, configs will be retrieved irrespective of whether `CONFIGS` is specified in fields query param + or not. Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `fields` - Specifies which fields to retrieve. Possible values for fields can be `EVENTS`, `INFO`, `CONFIGS`, `METRICS`, `RELATES_TO`, + `IS_RELATED_TO` and `ALL`. All fields will be retrieved if `ALL` is specified. Multiple fields can be specified as a comma-separated list. + If fields is not specified, in response, app id, type (equivalent to YARN_APPLICATION), app createdtime and UID in info field will be returned. +1. `metricslimit` - If specified, defines the number of metrics to return. Considered only if fields contains METRICS/ALL + or metricstoretrieve is specified. Ignored otherwise. The maximum possible value for metricslimit can be maximum value of + Integer. If it is not specified or has a value less than 1, and metrics have to be retrieved, then metricslimit will be + considered as 1 i.e. latest single value of metric(s) will be returned. +1. `metricsTimeStart` - If specified, then metrics for the entity after this timestamp are returned. +1. `metricsTimeEnd` - If specified, then metrics for the entity before this timestamp are returned. +1. `fromid` - If specified, retrieve the next set of application entities from the given fromid. The set of entities retrieved is inclusive of specified fromid. + fromid should be taken from the value associated with FROM_ID info key in flow entity response which was sent earlier. + +#### Example JSON Response: + + [ + { + "metrics": [ ], + "events": [ ], + "type": "YARN_APPLICATION", + "id": "application_1465246237936_0001", + "createdtime": 1465246348599, + "isrelatedto": { }, + "configs": { }, + "info": { + "UID": "yarn-cluster!application_1465246237936_0001" + "FROM_ID": "yarn-cluster!varun!QuasiMonteCarlo!1465246348599!application_1465246237936_0001", + }, + "relatesto": { } + }, + { + "metrics": [ ], + "events": [ ], + "type": "YARN_APPLICATION", + "id": "application_1464983628730_0005", + "createdtime": 1465033881959, + "isrelatedto": { }, + "configs": { }, + "info": { + "UID": "yarn-cluster!application_1464983628730_0005" + "FROM_ID": "yarn-cluster!varun!QuasiMonteCarlo!1465246348599!application_1464983628730_0005", + }, + "relatesto": { } + } + ] + +#### Response Codes + +1. If successful, a HTTP 200 (OK) response is returned. +1. If any problem occurs in parsing request, HTTP 400 (Bad Request) is returned. +1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned. + +### Query Apps for a flow run + +With this API, you can query all the YARN applications that are part of a specific flow run. If the +REST endpoint without the cluster name is used, the cluster specified by the configuration +`yarn.resourcemanager.cluster-id` in `yarn-site.xml` is taken. If number of matching applications +are more than the limit, the most recent apps up to the limit will be returned. If none of the apps +match the predicates, an empty list will be returned. + +#### HTTP request: + + GET /ws/v2/timeline/clusters/{cluster name}/users/{user name}/flows/{flow name}/runs/{run id}/apps + + or + + GET /ws/v2/timeline/users/{user name}/flows/{flow name}/runs/{run id}/apps/ + +#### Query Parameters Supported: + +1. `limit` - If specified, defines the number of applications to return. The maximum possible value for limit + is maximum value of Long. If it is not specified or has a value less than 0, then limit will be + considered as 100. +1. `createdtimestart` - If specified, then only applications created after this timestamp are returned. +1. `createdtimeend` - If specified, then only applications created before this timestamp are returned. +1. `relatesto` - If specified, matched applications must relate to or not relate to given entities associated with a entity type. + relatesto is represented as an expression of the form :
+ "(<entitytype>:<entityid>:<entityid>...,<entitytype>:<entityid>:<entityid>...) <op> !(<entitytype>:<entityid>:<entityid>...,<entitytype>:<entityid>:<entityid>...)".
+ If relatesto expression has entity type - entity id(s) relations specified within enclosing brackets proceeding "!", this means apps with + these relations in its relatesto field, will not be returned. For expressions or subexpressions without "!", all apps which have the specified + relations in its relatesto field, will be returned. "op" is a logical operator and can be either AND or OR. entity type can be followed by any number + of entity id(s). And we can combine any number of ANDs' and ORs' to create complex expressions. Brackets can be used to club expressions together.
+ _For example_ : relatesto can be "(((type1:id1:id2:id3,type3:id9) AND !(type2:id7:id8)) OR (type1:id4))".
+ Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `isrelatedto` - If specified, matched applications must be related to or not related to given entities associated with a entity type. isrelatedto is + represented in the same form as relatesto. +1. `infofilters` - If specified, matched applications must have exact matches to the given info key and must be either equal or not equal to + given value. The info key is a string but value can be any object. infofilters are represented as an expression of the form :
+ "(<key> <compareop> <value>) <op> (<key> <compareop> <value>)".
+ Here op can be either of AND or OR. And compareop can be either of "eq", "ne" or "ene".
+ "eq" means equals, "ne" means not equals and existence of key is not required for a match and "ene" means not equals but existence of key is + required. We can combine any number of ANDs' and ORs' to create complex expressions. Brackets can be used to club expressions together.
+ _For example_ : infofilters can be "(((infokey1 eq value1) AND (infokey2 ne value1)) OR (infokey1 ene value3))".
+ Note : If value is an object then value can be given in the form of JSON format without any space.
+ _For example_ : infofilters can be (infokey1 eq {"<key>":"<value>","<key>":"<value>"...}).
+ Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `conffilters` - If specified, matched applications must have exact matches to the given config name and must be either equal or not equal + to the given config value. Both the config name and value must be strings. conffilters are represented in the same form as infofilters. +1. `metricfilters` - If specified, matched applications must have exact matches to the given metric and satisfy the specified relation with the + metric value. Metric id must be a string and and metric value must be an integral value. metricfilters are represented as an expression of the form :
+ "(<metricid> <compareop> <metricvalue>) <op> (<metricid> <compareop> <metricvalue>)".
+ Here op can be either of AND or OR. And compareop can be either of "eq", "ne", "ene", "gt", "ge", "lt" and "le".
+ "eq" means equals, "ne" means not equals and existence of metric is not required for a match, "ene" means not equals but existence of metric is + required, "gt" means greater than, "ge" means greater than or equals, "lt" means less than and "le" means less than or equals. We can combine + any number of ANDs' and ORs' to create complex expressions. Brackets can be used to club expressions together.
+ _For example_ : metricfilters can be "(((metric1 eq 50) AND (metric2 gt 40)) OR (metric1 lt 20))".
+ This in essence is an expression equivalent to "(metric1 == 50 AND metric2 > 40) OR (metric1 < 20)"
+ Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `eventfilters` - If specified, matched applications must contain or not contain the given events depending on the expression. eventfilters is + represented as an expression of the form :
+ "(<eventid>,<eventid>) <op> !(<eventid>,<eventid>,<eventid>)".
+ Here, "!" means none of the comma-separated list of events within the enclosed brackets proceeding "!" must exist for a match to occur. + If "!" is not specified, the specified events within the enclosed brackets must exist. op is a logical operator and can be either AND or OR. + We can combine any number of ANDs' and ORs' to create complex expressions. Brackets can be used to club expressions together.
+ _For example_ : eventfilters can be "(((event1,event2) AND !(event4)) OR (event3,event7,event5))".
+ Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `metricstoretrieve` - If specified, defines which metrics to retrieve or which ones not to retrieve and send back in response. + metricstoretrieve can be an expression of the form :
+ (<metricprefix>,<metricprefix>,<metricprefix>,<metricprefix>...)
+ This specifies a comma separated list of metric id prefixes. Only metrics matching any of the prefixes will be retrieved. Brackets are optional for the + simple expression. Alternatively, expressions can be of the form :
+ !(<metricprefix>,<metricprefix>,<metricprefix>,<metricprefix>...)
+ This specifies a comma separated list of metric id prefixes. Only metrics not matching any of the prefixes will be retrieved.
+ If metricstoretrieve is specified, metrics will be retrieved irrespective of whether `METRICS` is specified in fields query param + or not. Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `confstoretrieve` - If specified, defines which configs to retrieve or which ones not to retrieve and send back in response. + confstoretrieve can be an expression of the form :
+ (<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>...)
+ This specifies a comma separated list of config name prefixes. Only configs matching any of the prefixes will be retrieved. Brackets are optional for the + simple expression. Alternatively, expressions can be of the form :
+ !(<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>...)
+ This specifies a comma separated list of config name prefixes. Only configs not matching any of the prefixes will be retrieved.
+ If confstoretrieve is specified, configs will be retrieved irrespective of whether `CONFIGS` is specified in fields query param + or not. Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `fields` - Specifies which fields to retrieve. Possible values for fields can be `EVENTS`, `INFO`, `CONFIGS`, `METRICS`, `RELATES_TO`, + `IS_RELATED_TO` and `ALL`. All fields will be retrieved if `ALL` is specified. Multiple fields can be specified as a comma-separated list. + If fields is not specified, in response, app id, type (equivalent to YARN_APPLICATION), app createdtime and UID in info field will be returned. +1. `metricslimit` - If specified, defines the number of metrics to return. Considered only if fields contains METRICS/ALL + or metricstoretrieve is specified. Ignored otherwise. The maximum possible value for metricslimit can be maximum value of + Integer. If it is not specified or has a value less than 1, and metrics have to be retrieved, then metricslimit will be + considered as 1 i.e. latest single value of metric(s) will be returned. +1. `metricsTimeStart` - If specified, then metrics for the entity after this timestamp are returned. +1. `metricsTimeEnd` - If specified, then metrics for the entity before this timestamp are returned. +1. `fromid` - If specified, retrieve the next set of application entities from the given fromid. The set of entities retrieved is inclusive of specified fromid. + fromid should be taken from the value associated with FROM_ID info key in flow entity response which was sent earlier. + +#### Example JSON Response: + + [ + { + "metrics": [], + "events": [], + "id": "application_1460419579913_0002", + "type": "YARN_APPLICATION", + "createdtime": 1460419580171, + "info": { + "UID": "test-cluster!sjlee!ds-date!1460419580171!application_1460419579913_0002" + "FROM_ID": "test-cluster!sjlee!ds-date!1460419580171!application_1460419579913_0002", + }, + "configs": {}, + "isrelatedto": {}, + "relatesto": {} + } + ] + +#### Response Codes + +1. If successful, a HTTP 200 (OK) response is returned. +1. If any problem occurs in parsing request, HTTP 400 (Bad Request) is returned. +1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned. + + +### Query app + +With this API, you can query a single YARN application identified by the cluster and the +application ID. If the REST endpoint without the cluster name is used, the cluster specified by the +configuration `yarn.resourcemanager.cluster-id` in `yarn-site.xml` is taken. Flow context +information i.e. user, flow name and run id are not mandatory but if specified in query param can +preclude the need for an additional operation to fetch flow context information based on cluster +and app id. + +#### HTTP request: + + GET /ws/v2/timeline/clusters/{cluster name}/apps/{app id} + + or + + GET /ws/v2/timeline/apps/{app id} + +#### Query Parameters Supported: + +1. `userid` - If specified, only applications belonging to this user will be returned. This query param must be specified along with flowname and flowrunid query params, otherwise it will be ignored. + If userid, flowname and flowrunid are not specified, we would have to fetch flow context information based on cluster and appid while executing the query. +1. `flowname` - Only applications belonging to this flowname will be returned. This query param must be specified along with userid and flowrunid query params, otherwise it will be ignored. + If userid, flowname and flowrunid are not specified, we would have to fetch flow context information based on cluster and appid while executing the query. +1. `flowrunid` - Only applications belonging to this flow run id will be returned. This query param must be specified along with userid and flowname query params, otherwise it will be ignored. + If userid, flowname and flowrunid are not specified, we would have to fetch flow context information based on cluster and appid while executing the query. +1. `metricstoretrieve` - If specified, defines which metrics to retrieve or which ones not to retrieve and send back in response. + metricstoretrieve can be an expression of the form :
+ (<metricprefix>,<metricprefix>,<metricprefix>,<metricprefix>...)
+ This specifies a comma separated list of metric id prefixes. Only metrics matching any of the prefixes will be retrieved. Brackets are optional for the + simple expression. Alternatively, expressions can be of the form :
+ !(<metricprefix>,<metricprefix>,<metricprefix>,<metricprefix>...)
+ This specifies a comma separated list of metric id prefixes. Only metrics not matching any of the prefixes will be retrieved.
+ If metricstoretrieve is specified, metrics will be retrieved irrespective of whether `METRICS` is specified in fields query param + or not. Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `confstoretrieve` - If specified, defines which configs to retrieve or which ones not to retrieve and send back in response. + confstoretrieve can be an expression of the form :
+ (<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>...)
+ This specifies a comma separated list of config name prefixes. Only configs matching any of the prefixes will be retrieved. Brackets are optional for the + simple expression. Alternatively, expressions can be of the form :
+ !(<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>...)
+ This specifies a comma separated list of config name prefixes. Only configs not matching any of the prefixes will be retrieved.
+ If confstoretrieve is specified, configs will be retrieved irrespective of whether `CONFIGS` is specified in fields query param + or not. Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `fields` - Specifies which fields to retrieve. Possible values for fields can be `EVENTS`, `INFO`, `CONFIGS`, `METRICS`, `RELATES_TO`, + `IS_RELATED_TO` and `ALL`. All fields will be retrieved if `ALL` is specified. Multiple fields can be specified as a comma-separated list. + If fields is not specified, in response, app id, type (equivalent to YARN_APPLICATION), app createdtime and UID in info field will be returned. +1. `metricslimit` - If specified, defines the number of metrics to return. Considered only if fields contains METRICS/ALL + or metricstoretrieve is specified. Ignored otherwise. The maximum possible value for metricslimit can be maximum value of + Integer. If it is not specified or has a value less than 1, and metrics have to be retrieved, then metricslimit will be + considered as 1 i.e. latest single value of metric(s) will be returned. +1. `metricsTimeStart` - If specified, then metrics for the entity after this timestamp are returned. +1. `metricsTimeEnd` - If specified, then metrics for the entity before this timestamp are returned. + +#### Example JSON Response: + + { + "metrics": [], + "events": [], + "id": "application_1460419579913_0002", + "type": "YARN_APPLICATION", + "createdtime": 1460419580171, + "info": { + "UID": "test-cluster!sjlee!ds-date!1460419580171!application_1460419579913_0002" + }, + "configs": {}, + "isrelatedto": {}, + "relatesto": {} + } + +#### Response Codes + +1. If successful, a HTTP 200(OK) response is returned. +1. If any problem occurs in parsing request, HTTP 400 (Bad Request) is returned. +1. If flow context information cannot be retrieved or application for the given app id cannot be found, HTTP 404 (Not Found) is returned. +1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned. + +### Query generic entities with in the scope of Application + +With this API, you can query generic entities identified by cluster ID, application ID and +per-framework entity type. If the REST endpoint without the cluster name is used, the cluster +specified by the configuration `yarn.resourcemanager.cluster-id` in `yarn-site.xml` is taken. Flow +context information i.e. user, flow name and run id are not mandatory but if specified in query +param can preclude the need for an additional operation to fetch flow context information based on +cluster and app id. If number of matching entities are more than the limit, the most recent +entities up to the limit will be returned. This endpoint can be used to query containers, +application attempts or any other generic entity which clients put into the backend. +For instance, we can query containers by specifying entity type as `YARN_CONTAINER` and application +attempts by specifying entity type as `YARN_APPLICATION_ATTEMPT`. +If none of the entities match the predicates, an empty list will be returned. + +#### HTTP request: + + GET /ws/v2/timeline/clusters/{cluster name}/apps/{app id}/entities/{entity type} + + or + + GET /ws/v2/timeline/apps/{app id}/entities/{entity type} + +#### Query Parameters Supported: + +1. `userid` - If specified, only entities belonging to this user will be returned. This query param must be specified along with flowname and flowrunid query params, otherwise it will be ignored. + If userid, flowname and flowrunid are not specified, we would have to fetch flow context information based on cluster and appid while executing the query. +1. `flowname` - If specified, only entities belonging to this flowname will be returned. This query param must be specified along with userid and flowrunid query params, otherwise it will be ignored. + If userid, flowname and flowrunid are not specified, we would have to fetch flow context information based on cluster and appid while executing the query. +1. `flowrunid` - If specified, only entities belonging to this flow run id will be returned. This query param must be specified along with userid and flowname query params, otherwise it will be ignored. + If userid, flowname and flowrunid are not specified, we would have to fetch flow context information based on cluster and appid while executing the query. +1. `limit` - If specified, defines the number of entities to return. The maximum possible value for limit is maximum value of Long. If it is not specified + or has a value less than 0, then limit will be considered as 100. +1. `createdtimestart` - If specified, then only entities created after this timestamp are returned. +1. `createdtimeend` - If specified, then only entities created before this timestamp are returned. +1. `relatesto` - If specified, matched entities must relate to or not relate to given entities associated with a entity type. + relatesto is represented as an expression of the form :
+ "(<entitytype>:<entityid>:<entityid>...,<entitytype>:<entityid>:<entityid>...) <op> !(<entitytype>:<entityid>:<entityid>...,<entitytype>:<entityid>:<entityid>...)".
+ If relatesto expression has entity type - entity id(s) relations specified within enclosing brackets proceeding "!", this means entities with + these relations in its relatesto field, will not be returned. For expressions or subexpressions without "!", all entities which have the specified + relations in its relatesto field, will be returned. "op" is a logical operator and can be either AND or OR. entity type can be followed by any number + of entity id(s). And we can combine any number of ANDs' and ORs' to create complex expressions. Brackets can be used to club expressions together.
+ _For example_ : relatesto can be "(((type1:id1:id2:id3,type3:id9) AND !(type2:id7:id8)) OR (type1:id4))".
+ Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `isrelatedto` - If specified, matched entities must be related to or not related to given entities associated with a entity type. isrelatedto is + represented in the same form as relatesto. +1. `infofilters` - If specified, matched entities must have exact matches to the given info key and must be either equal or not equal to + given value. The info key is a string but value can be any object. infofilters are represented as an expression of the form :
+ "(<key> <compareop> <value>) <op> (<key> <compareop> <value>)".
+ Here op can be either of AND or OR. And compareop can be either of "eq", "ne" or "ene".
+ "eq" means equals, "ne" means not equals and existence of key is not required for a match and "ene" means not equals but existence of key is + required. We can combine any number of ANDs' and ORs' to create complex expressions. Brackets can be used to club expressions together.
+ _For example_ : infofilters can be "(((infokey1 eq value1) AND (infokey2 ne value1)) OR (infokey1 ene value3))".
+ Note : If value is an object then value can be given in the form of JSON format without any space.
+ _For example_ : infofilters can be (infokey1 eq {"<key>":"<value>","<key>":"<value>"...}).
+ Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `conffilters` - If specified, matched entities must have exact matches to the given config name and must be either equal or not equal + to the given config value. Both the config name and value must be strings. conffilters are represented in the same form as infofilters. +1. `metricfilters` - If specified, matched entities must have exact matches to the given metric and satisfy the specified relation with the + metric value. Metric id must be a string and and metric value must be an integral value. metricfilters are represented as an expression of the form :
+ "(<metricid> <compareop> <metricvalue>) <op> (<metricid> <compareop> <metricvalue>)"
+ Here op can be either of AND or OR. And compareop can be either of "eq", "ne", "ene", "gt", "ge", "lt" and "le".
+ "eq" means equals, "ne" means not equals and existence of metric is not required for a match, "ene" means not equals but existence of metric is + required, "gt" means greater than, "ge" means greater than or equals, "lt" means less than and "le" means less than or equals. We can combine + any number of ANDs' and ORs' to create complex expressions. Brackets can be used to club expressions together.
+ _For example_ : metricfilters can be "(((metric1 eq 50) AND (metric2 gt 40)) OR (metric1 lt 20))".
+ This in essence is an expression equivalent to "(metric1 == 50 AND metric2 > 40) OR (metric1 < 20)"
+ Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `eventfilters` - If specified, matched entities must contain or not contain the given events depending on the expression. eventfilters is + represented as an expression of the form :
+ "(<eventid>,<eventid>) <op> !(<eventid>,<eventid>,<eventid>)".
+ Here, "!" means none of the comma-separated list of events within the enclosed brackets proceeding "!" must exist for a match to occur. + If "!" is not specified, the specified events within the enclosed brackets must exist. op is a logical operator and can be either AND or OR. + We can combine any number of ANDs' and ORs' to create complex expressions. Brackets can be used to club expressions together.
+ _For example_ : eventfilters can be "(((event1,event2) AND !(event4)) OR (event3,event7,event5))".
+ Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `metricstoretrieve` - If specified, defines which metrics to retrieve or which ones not to retrieve and send back in response. + metricstoretrieve can be an expression of the form :
+ (<metricprefix>,<metricprefix>,<metricprefix>,<metricprefix>...)
+ This specifies a comma separated list of metric id prefixes. Only metrics matching any of the prefixes will be retrieved. Brackets are optional for + the simple expression. Alternatively, expressions can be of the form:
+ !(<metricprefix>,<metricprefix>,<metricprefix>,<metricprefix>...)
+ This specifies a comma separated list of metric id prefixes. Only metrics not matching any of the prefixes will be retrieved.
+ If metricstoretrieve is specified, metrics will be retrieved irrespective of whether `METRICS` is specified in fields query param + or not. Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `confstoretrieve` - If specified, defines which configs to retrieve or which ones not to retrieve and send back in response. + confstoretrieve can be an expression of the form :
+ (<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>...)
+ This specifies a comma separated list of config name prefixes. Only configs matching any of the prefixes will be retrieved. Brackets are optional for + the simple expression. Alternatively, expressions can be of the form:
+ !(<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>...)
+ This specifies a comma separated list of config name prefixes. Only configs not matching any of the prefixes will be retrieved.
+ If confstoretrieve is specified, configs will be retrieved irrespective of whether `CONFIGS` is specified in fields query param + or not. Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `fields` - Specifies which fields to retrieve. Possible values for fields can be `EVENTS`, `INFO`, `CONFIGS`, `METRICS`, `RELATES_TO`, + `IS_RELATED_TO` and `ALL`. All fields will be retrieved if `ALL` is specified. Multiple fields can be specified as a comma-separated list. + If fields is not specified, in response, entity id, entity type, createdtime and UID in info field will be returned. +1. `metricslimit` - If specified, defines the number of metrics to return. Considered only if fields contains METRICS/ALL + or metricstoretrieve is specified. Ignored otherwise. The maximum possible value for metricslimit can be maximum value of + Integer. If it is not specified or has a value less than 1, and metrics have to be retrieved, then metricslimit will be + considered as 1 i.e. latest single value of metric(s) will be returned. +1. `metricsTimeStart` - If specified, then metrics for the entity after this timestamp are returned. +1. `metricsTimeEnd` - If specified, then metrics for the entity before this timestamp are returned. +1. `fromid` - If specified, retrieve the next set of generic entities from the given fromid. The set of entities retrieved is inclusive of specified fromid. + fromid should be taken from the value associated with FROM_ID info key in flow entity response which was sent earlier. + +#### Example JSON Response: + + [ + { + "metrics": [ ], + "events": [ ], + "type": "YARN_APPLICATION_ATTEMPT", + "id": "appattempt_1465246237936_0001_000001", + "createdtime": 1465246358873, + "isrelatedto": { }, + "configs": { }, + "info": { + "UID": "yarn-cluster!application_1465246237936_0001!YARN_APPLICATION_ATTEMPT!appattempt_1465246237936_0001_000001" + "FROM_ID": "yarn-cluster!sjlee!ds-date!1460419580171!application_1465246237936_0001!YARN_APPLICATION_ATTEMPT!0!appattempt_1465246237936_0001_000001" + }, + "relatesto": { } + }, + { + "metrics": [ ], + "events": [ ], + "type": "YARN_APPLICATION_ATTEMPT", + "id": "appattempt_1465246237936_0001_000002", + "createdtime": 1465246359045, + "isrelatedto": { }, + "configs": { }, + "info": { + "UID": "yarn-cluster!application_1465246237936_0001!YARN_APPLICATION_ATTEMPT!appattempt_1465246237936_0001_000002" + "FROM_ID": "yarn-cluster!sjlee!ds-date!1460419580171!application_1465246237936_0001!YARN_APPLICATION_ATTEMPT!0!appattempt_1465246237936_0001_000002" + }, + "relatesto": { } + } + ] + +#### Response Codes + +1. If successful, a HTTP 200(OK) response is returned. +1. If any problem occurs in parsing request, HTTP 400 (Bad Request) is returned. +1. If flow context information cannot be retrieved, HTTP 404 (Not Found) is returned. +1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned. + +### Query generic entities. + +With this API, you can query generic entities per user identified by cluster ID, doAsUser and +entity type. If the REST endpoint without the cluster name is used, the cluster +specified by the configuration `yarn.resourcemanager.cluster-id` in `yarn-site.xml` is taken. +If number of matching entities are more than the limit, the most recent +entities up to the limit will be returned. This endpoint can be used to query generic entity which +clients put into the backend. For instance, we can query user entities by specifying entity type as `TEZ_DAG_ID`. +If none of the entities match the predicates, an empty list will be returned. +**Note** : As of today, we can query only those entities which are published with doAsUser which is different from application owner. + +#### HTTP request: + + GET /ws/v2/timeline/clusters/{cluster name}/users/{userid}/entities/{entitytype} + + or + + GET /ws/v2/timeline/users/{userid}/entities/{entitytype} + +#### Query Parameters Supported: + +1. `limit` - If specified, defines the number of entities to return. The maximum possible value for limit is maximum value of Long. If it is not specified + or has a value less than 0, then limit will be considered as 100. +1. `createdtimestart` - If specified, then only entities created after this timestamp are returned. +1. `createdtimeend` - If specified, then only entities created before this timestamp are returned. +1. `relatesto` - If specified, matched entities must relate to or not relate to given entities associated with a entity type. + relatesto is represented as an expression of the form :
+ "(<entitytype>:<entityid>:<entityid>...,<entitytype>:<entityid>:<entityid>...) <op> !(<entitytype>:<entityid>:<entityid>...,<entitytype>:<entityid>:<entityid>...)".
+ If relatesto expression has entity type - entity id(s) relations specified within enclosing brackets proceeding "!", this means entities with + these relations in its relatesto field, will not be returned. For expressions or subexpressions without "!", all entities which have the specified + relations in its relatesto field, will be returned. "op" is a logical operator and can be either AND or OR. entity type can be followed by any number + of entity id(s). And we can combine any number of ANDs' and ORs' to create complex expressions. Brackets can be used to club expressions together.
+ _For example_ : relatesto can be "(((type1:id1:id2:id3,type3:id9) AND !(type2:id7:id8)) OR (type1:id4))".
+ Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `isrelatedto` - If specified, matched entities must be related to or not related to given entities associated with a entity type. isrelatedto is + represented in the same form as relatesto. +1. `infofilters` - If specified, matched entities must have exact matches to the given info key and must be either equal or not equal to + given value. The info key is a string but value can be any object. infofilters are represented as an expression of the form :
+ "(<key> <compareop> <value>) <op> (<key> <compareop> <value>)".
+ Here op can be either of AND or OR. And compareop can be either of "eq", "ne" or "ene".
+ "eq" means equals, "ne" means not equals and existence of key is not required for a match and "ene" means not equals but existence of key is + required. We can combine any number of ANDs' and ORs' to create complex expressions. Brackets can be used to club expressions together.
+ _For example_ : infofilters can be "(((infokey1 eq value1) AND (infokey2 ne value1)) OR (infokey1 ene value3))".
+ Note : If value is an object then value can be given in the form of JSON format without any space.
+ _For example_ : infofilters can be (infokey1 eq {"<key>":"<value>","<key>":"<value>"...}).
+ Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `conffilters` - If specified, matched entities must have exact matches to the given config name and must be either equal or not equal + to the given config value. Both the config name and value must be strings. conffilters are represented in the same form as infofilters. +1. `metricfilters` - If specified, matched entities must have exact matches to the given metric and satisfy the specified relation with the + metric value. Metric id must be a string and and metric value must be an integral value. metricfilters are represented as an expression of the form :
+ "(<metricid> <compareop> <metricvalue>) <op> (<metricid> <compareop> <metricvalue>)"
+ Here op can be either of AND or OR. And compareop can be either of "eq", "ne", "ene", "gt", "ge", "lt" and "le".
+ "eq" means equals, "ne" means not equals and existence of metric is not required for a match, "ene" means not equals but existence of metric is + required, "gt" means greater than, "ge" means greater than or equals, "lt" means less than and "le" means less than or equals. We can combine + any number of ANDs' and ORs' to create complex expressions. Brackets can be used to club expressions together.
+ _For example_ : metricfilters can be "(((metric1 eq 50) AND (metric2 gt 40)) OR (metric1 lt 20))".
+ This in essence is an expression equivalent to "(metric1 == 50 AND metric2 > 40) OR (metric1 < 20)"
+ Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `eventfilters` - If specified, matched entities must contain or not contain the given events depending on the expression. eventfilters is + represented as an expression of the form :
+ "(<eventid>,<eventid>) <op> !(<eventid>,<eventid>,<eventid>)".
+ Here, "!" means none of the comma-separated list of events within the enclosed brackets proceeding "!" must exist for a match to occur. + If "!" is not specified, the specified events within the enclosed brackets must exist. op is a logical operator and can be either AND or OR. + We can combine any number of ANDs' and ORs' to create complex expressions. Brackets can be used to club expressions together.
+ _For example_ : eventfilters can be "(((event1,event2) AND !(event4)) OR (event3,event7,event5))".
+ Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `metricstoretrieve` - If specified, defines which metrics to retrieve or which ones not to retrieve and send back in response. + metricstoretrieve can be an expression of the form :
+ (<metricprefix>,<metricprefix>,<metricprefix>,<metricprefix>...)
+ This specifies a comma separated list of metric id prefixes. Only metrics matching any of the prefixes will be retrieved. Brackets are optional for + the simple expression. Alternatively, expressions can be of the form:
+ !(<metricprefix>,<metricprefix>,<metricprefix>,<metricprefix>...)
+ This specifies a comma separated list of metric id prefixes. Only metrics not matching any of the prefixes will be retrieved.
+ If metricstoretrieve is specified, metrics will be retrieved irrespective of whether `METRICS` is specified in fields query param + or not. Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `confstoretrieve` - If specified, defines which configs to retrieve or which ones not to retrieve and send back in response. + confstoretrieve can be an expression of the form :
+ (<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>...)
+ This specifies a comma separated list of config name prefixes. Only configs matching any of the prefixes will be retrieved. Brackets are optional for + the simple expression. Alternatively, expressions can be of the form:
+ !(<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>...)
+ This specifies a comma separated list of config name prefixes. Only configs not matching any of the prefixes will be retrieved.
+ If confstoretrieve is specified, configs will be retrieved irrespective of whether `CONFIGS` is specified in fields query param + or not. Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `fields` - Specifies which fields to retrieve. Possible values for fields can be `EVENTS`, `INFO`, `CONFIGS`, `METRICS`, `RELATES_TO`, + `IS_RELATED_TO` and `ALL`. All fields will be retrieved if `ALL` is specified. Multiple fields can be specified as a comma-separated list. + If fields is not specified, in response, entity id, entity type, createdtime and UID in info field will be returned. +1. `metricslimit` - If specified, defines the number of metrics to return. Considered only if fields contains METRICS/ALL + or metricstoretrieve is specified. Ignored otherwise. The maximum possible value for metricslimit can be maximum value of + Integer. If it is not specified or has a value less than 1, and metrics have to be retrieved, then metricslimit will be + considered as 1 i.e. latest single value of metric(s) will be returned. +1. `metricsTimeStart` - If specified, then metrics for the entity after this timestamp are returned. +1. `metricsTimeEnd` - If specified, then metrics for the entity before this timestamp are returned. +1. `fromid` - If specified, retrieve the next set of generic entities from the given fromid. The set of entities retrieved is inclusive of specified fromid. + fromid should be taken from the value associated with FROM_ID info key in flow entity response which was sent earlier. + +#### Example JSON Response: + + [ + { + "metrics": [ ], + "events": [ ], + "type": "TEZ_DAG_ID", + "id": "dag_1465246237936_0001_000001", + "createdtime": 1465246358873, + "isrelatedto": { }, + "configs": { }, + "info": { + "UID": "yarn-cluster!sjlee!TEZ_DAG_ID!0!dag_1465246237936_0001_000001" + "FROM_ID": "sjlee!yarn-cluster!TEZ_DAG_ID!0!dag_1465246237936_0001_000001" + }, + "relatesto": { } + }, + { + "metrics": [ ], + "events": [ ], + "type": "TEZ_DAG_ID", + "id": "dag_1465246237936_0001_000002", + "createdtime": 1465246359045, + "isrelatedto": { }, + "configs": { }, + "info": { + "UID": "yarn-cluster!sjlee!TEZ_DAG_ID!0!dag_1465246237936_0001_000002!userX" + "FROM_ID": "sjlee!yarn-cluster!TEZ_DAG_ID!0!dag_1465246237936_0001_000002!userX" + }, + "relatesto": { } + } + ] + +#### Response Codes + +1. If successful, a HTTP 200(OK) response is returned. +1. If any problem occurs in parsing request, HTTP 400 (Bad Request) is returned. +1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned. + +### Query generic entity with in the scope of Application + +With this API, you can query a specific generic entity identified by cluster ID, application ID, +per-framework entity type and entity ID. If the REST endpoint without the cluster name is used, the +cluster specified by the configuration `yarn.resourcemanager.cluster-id` in `yarn-site.xml` is +taken. Flow context information i.e. user, flow name and run id are not mandatory but if specified +in query param can preclude the need for an additional operation to fetch flow context information +based on cluster and app id. This endpoint can be used to query a single container, application +attempt or any other generic entity which clients put into the backend. For instance, we can query +a specific YARN container by specifying entity type as `YARN_CONTAINER` and giving entity ID as +container ID. Similarly, application attempt can be queried by specifying entity type as +`YARN_APPLICATION_ATTEMPT` and entity ID being the application attempt ID. + +#### HTTP request: + + GET /ws/v2/timeline/clusters/{cluster name}/apps/{app id}/entities/{entity type}/{entity id} + + or + + GET /ws/v2/timeline/apps/{app id}/entities/{entity type}/{entity id} + +#### Query Parameters Supported: + +1. `userid` - If specified, entity must belong to this user. This query param must be specified along with flowname and flowrunid query params, otherwise it will be ignored. + If userid, flowname and flowrunid are not specified, we would have to fetch flow context information based on cluster and appid while executing the query. +1. `flowname` - If specified, entity must belong to this flow name. This query param must be specified along with userid and flowrunid query params, otherwise it will be ignored. + If userid, flowname and flowrunid are not specified, we would have to fetch flow context information based on cluster and appid while executing the query. +1. `flowrunid` - If specified, entity must belong to this flow run id. This query param must be specified along with userid and flowname query params, otherwise it will be ignored. + If userid, flowname and flowrunid are not specified, we would have to fetch flow context information based on cluster and appid while executing the query. +1. `metricstoretrieve` - If specified, defines which metrics to retrieve or which ones not to retrieve and send back in response. + metricstoretrieve can be an expression of the form :
+ (<metricprefix>,<metricprefix>,<metricprefix>,<metricprefix>...)
+ This specifies a comma separated list of metric id prefixes. Only metrics matching any of the prefixes will be retrieved. Brackets are optional + for the simple expression. Alternatively, expressions can be of the form:
+ !(<metricprefix>,<metricprefix>,<metricprefix>,<metricprefix>...)
+ This specifies a comma separated list of metric id prefixes. Only metrics not matching any of the prefixes will be retrieved.
+ If metricstoretrieve is specified, metrics will be retrieved irrespective of whether `METRICS` is specified in fields query param + or not. Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `confstoretrieve` - If specified, defines which configs to retrieve or which ones not to retrieve and send back in response. + confstoretrieve can be an expression of the form :
+ (<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>...)
+ This specifies a comma separated list of config name prefixes. Only configs matching any of the prefixes will be retrieved. Brackets are optional for + the simple expression. Alternatively, expressions can be of the form :
+ !(<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>...)
+ This specifies a comma separated list of config name prefixes. Only configs not matching any of the prefixes will be retrieved.
+ If confstoretrieve is specified, configs will be retrieved irrespective of whether `CONFIGS` is specified in fields query param + or not. Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `fields` - Specifies which fields to retrieve. Possible values for fields can be `EVENTS`, `INFO`, `CONFIGS`, `METRICS`, `RELATES_TO`, + `IS_RELATED_TO` and `ALL`. All fields will be retrieved if `ALL` is specified. Multiple fields can be specified as a comma-separated list. + If fields is not specified, in response, entity id, entity type, createdtime and UID in info field will be returned. +1. `metricslimit` - If specified, defines the number of metrics to return. Considered only if fields contains METRICS/ALL + or metricstoretrieve is specified. Ignored otherwise. The maximum possible value for metricslimit can be maximum value of + Integer. If it is not specified or has a value less than 1, and metrics have to be retrieved, then metricslimit will be + considered as 1 i.e. latest single value of metric(s) will be returned. +1. `metricsTimeStart` - If specified, then metrics for the entity after this timestamp are returned. +1. `metricsTimeEnd` - If specified, then metrics for the entity before this timestamp are returned. +1. `entityidprefix` Defines the id prefix for the entity to be fetched. If specified, then entity retrieval will be faster. + +#### Example JSON Response: + + { + "metrics": [ ], + "events": [ ], + "type": "YARN_APPLICATION_ATTEMPT", + "id": "appattempt_1465246237936_0001_000001", + "createdtime": 1465246358873, + "isrelatedto": { }, + "configs": { }, + "info": { + "UID": "yarn-cluster!application_1465246237936_0001!YARN_APPLICATION_ATTEMPT!0!appattempt_1465246237936_0001_000001" + "FROM_ID": "yarn-cluster!sjlee!ds-date!1460419580171!application_1465246237936_0001!YARN_APPLICATION_ATTEMPT!0!appattempt_1465246237936_0001_000001" + }, + "relatesto": { } + } + +#### Response Codes + +1. If successful, a HTTP 200 (OK) response is returned. +1. If any problem occurs in parsing request, HTTP 400 (Bad Request) is returned. +1. If flow context information cannot be retrieved or entity for the given entity id cannot be found, HTTP 404 (Not Found) is returned. +1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned. + +### Query generic entity. + +With this API, you can query generic entity per user identified by cluster ID, doAsUser and +entity type and entity ID. If the REST endpoint without the cluster name is used, the cluster +specified by the configuration `yarn.resourcemanager.cluster-id` in `yarn-site.xml` is taken. +If number of matching entities are more than the limit, the most recent +entities up to the limit will be returned. This endpoint can be used to query generic entity which +clients put into the backend. For instance, we can query user entities by specifying entity type as `TEZ_DAG_ID`. +If none of the entities match the predicates, an empty list will be returned. +**Note** : As of today, we can query only those entities which are published with doAsUser which is different from application owner. + +#### HTTP request: + + GET /ws/v2/timeline/clusters/{cluster name}/users/{userid}/entities/{entitytype}/{entityid} + + or + + GET /ws/v2/timeline/users/{userid}/entities/{entitytype}/{entityid} + +#### Query Parameters Supported: + +1. `metricstoretrieve` - If specified, defines which metrics to retrieve or which ones not to retrieve and send back in response. + metricstoretrieve can be an expression of the form :
+ (<metricprefix>,<metricprefix>,<metricprefix>,<metricprefix>...)
+ This specifies a comma separated list of metric id prefixes. Only metrics matching any of the prefixes will be retrieved. Brackets are optional for + the simple expression. Alternatively, expressions can be of the form:
+ !(<metricprefix>,<metricprefix>,<metricprefix>,<metricprefix>...)
+ This specifies a comma separated list of metric id prefixes. Only metrics not matching any of the prefixes will be retrieved.
+ If metricstoretrieve is specified, metrics will be retrieved irrespective of whether `METRICS` is specified in fields query param + or not. Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `confstoretrieve` - If specified, defines which configs to retrieve or which ones not to retrieve and send back in response. + confstoretrieve can be an expression of the form :
+ (<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>...)
+ This specifies a comma separated list of config name prefixes. Only configs matching any of the prefixes will be retrieved. Brackets are optional for + the simple expression. Alternatively, expressions can be of the form:
+ !(<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>,<config\_name\_prefix>...)
+ This specifies a comma separated list of config name prefixes. Only configs not matching any of the prefixes will be retrieved.
+ If confstoretrieve is specified, configs will be retrieved irrespective of whether `CONFIGS` is specified in fields query param + or not. Please note that URL unsafe characters such as spaces will have to be suitably encoded. +1. `fields` - Specifies which fields to retrieve. Possible values for fields can be `EVENTS`, `INFO`, `CONFIGS`, `METRICS`, `RELATES_TO`, + `IS_RELATED_TO` and `ALL`. All fields will be retrieved if `ALL` is specified. Multiple fields can be specified as a comma-separated list. + If fields is not specified, in response, entity id, entity type, createdtime and UID in info field will be returned. +1. `metricslimit` - If specified, defines the number of metrics to return. Considered only if fields contains METRICS/ALL + or metricstoretrieve is specified. Ignored otherwise. The maximum possible value for metricslimit can be maximum value of + Integer. If it is not specified or has a value less than 1, and metrics have to be retrieved, then metricslimit will be + considered as 1 i.e. latest single value of metric(s) will be returned. +1. `metricsTimeStart` - If specified, then metrics for the entity after this timestamp are returned. +1. `metricsTimeEnd` - If specified, then metrics for the entity before this timestamp are returned. +1. `fromid` - If specified, retrieve the next set of generic entities from the given fromid. The set of entities retrieved is inclusive of specified fromid. + fromid should be taken from the value associated with FROM_ID info key in flow entity response which was sent earlier. + +#### Example JSON Response: + + [ + { + "metrics": [ ], + "events": [ ], + "type": "TEZ_DAG_ID", + "id": "dag_1465246237936_0001_000001", + "createdtime": 1465246358873, + "isrelatedto": { }, + "configs": { }, + "info": { + "UID": "yarn-cluster!sjlee!TEZ_DAG_ID!0!dag_1465246237936_0001_000001!userX" + "FROM_ID": "sjlee!yarn-cluster!TEZ_DAG_ID!0!dag_1465246237936_0001_000001!userX" + }, + "relatesto": { } + } + ] + +#### Response Codes + +1. If successful, a HTTP 200(OK) response is returned. +1. If any problem occurs in parsing request, HTTP 400 (Bad Request) is returned. +1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned. + +### Query generic entity types + +With this API, you can query set of available entity types for a given app id. If the REST endpoint without the cluster name is used, the cluster specified by the configuration yarn.resourcemanager.cluster-id in yarn-site.xml is taken. If userid, flow name and flow run id which are optional query parameters are not specified, they will be queried based on app id and cluster id from the flow context information stored in underlying storage implementation. + +#### HTTP request: + + GET /ws/v2/timeline/apps/{appid}/entity-types + + or + + GET /ws/v2/timeline/clusters/{clusterid}/apps/{appid}/entity-types + +#### Query Parameters Supported: + +1. `userid` - If specified, entity must belong to this user. This query param must be specified along with flowname and flowrunid query params, otherwise it will be ignored. + If userid, flowname and flowrunid are not specified then timeline reader will fetch flow context information based on cluster and appid while executing the query. +1. `flowname` - If specified, entity must belong to this flow name. This query param must be specified along with userid and flowrunid query params, otherwise it will be ignored. + If userid, flowname and flowrunid are not specified, we would have to fetch flow context information based on cluster and appid while executing the query. +1. `flowrunid` - If specified, entity must belong to this flow run id. This query param must be specified along with userid and flowname query params, otherwise it will be ignored. + If userid, flowname and flowrunid are not specified, we would have to fetch flow context information based on cluster and appid while executing the query. + +#### Example JSON Response: + + { + YARN_APPLICATION_ATTEMPT, + YARN_CONTAINER, + MAPREDUCE_JOB, + MAPREDUCE_TASK, + MAPREDUCE_TASK_ATTEMPT + } + +#### Response Codes + +1. If successful, a HTTP 200 (OK) response is returned. +1. If any problem occurs in parsing request, HTTP 400 (Bad Request) is returned. +1. If flow context information cannot be retrieved or entity for the given entity id cannot be found, HTTP 404 (Not Found) is returned. +1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/flow_hierarchy.png b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/flow_hierarchy.png new file mode 100644 index 00000000000..fc4b06b9831 Binary files /dev/null and b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/flow_hierarchy.png differ diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/timeline_v2.jpg b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/timeline_v2.jpg new file mode 100644 index 00000000000..a356e269dce Binary files /dev/null and b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/timeline_v2.jpg differ diff --git a/hadoop-yarn-project/pom.xml b/hadoop-yarn-project/pom.xml index 9c3dd1a1567..1ada81fac26 100644 --- a/hadoop-yarn-project/pom.xml +++ b/hadoop-yarn-project/pom.xml @@ -160,12 +160,44 @@ + + + org.apache.maven.plugins + maven-antrun-plugin + + + dist + prepare-package + + run + + + + + yarnjardir="${project.build.directory}/${project.artifactId}-${project.version}/share/hadoop/yarn" + yarnlibdir="${yarnjardir}/lib/" + cd "${yarnjardir}/timelineservice/lib" + for jar in `ls` + do + if [ -f "${yarnlibdir}""${jar}" ];then + rm -rf ${jar} + echo "Removing duplicate jar $jar from share/hadoop/yarn/timelineservice/lib folder" + fi + done + + + + + + + + + -