YARN-2928 and YARN-5355: YARN Timeline Service v.2: alpha 2 (varunsaxena)

For a list of commits and contributors, see YARN-2928 and YARN-5355 branch history.
This commit is contained in:
Varun Saxena 2017-10-17 23:40:32 +05:30
commit 54a694172d
423 changed files with 58559 additions and 2582 deletions

View File

@ -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

View File

@ -188,9 +188,38 @@
<directory>${project.build.directory}/site</directory>
<outputDirectory>/share/doc/hadoop/${hadoop.component}</outputDirectory>
</fileSet>
<!-- Copy dependecies from hadoop-yarn-server-timelineservice as well -->
<fileSet>
<directory>hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/target/lib</directory>
<outputDirectory>share/hadoop/${hadoop.component}/timelineservice/lib</outputDirectory>
</fileSet>
</fileSets>
<moduleSets>
<moduleSet>
<includes>
<include>org.apache.hadoop:hadoop-yarn-server-timelineservice</include>
<include>org.apache.hadoop:hadoop-yarn-server-timelineservice-hbase</include>
</includes>
<binaries>
<outputDirectory>share/hadoop/${hadoop.component}/timelineservice</outputDirectory>
<includeDependencies>false</includeDependencies>
<unpack>false</unpack>
</binaries>
</moduleSet>
<moduleSet>
<includes>
<include>org.apache.hadoop:hadoop-yarn-server-timelineservice-hbase-tests</include>
</includes>
<binaries>
<outputDirectory>share/hadoop/${hadoop.component}/timelineservice/test</outputDirectory>
<includeDependencies>false</includeDependencies>
<unpack>false</unpack>
</binaries>
</moduleSet>
<moduleSet>
<excludes>
<exclude>org.apache.hadoop:hadoop-yarn-server-timelineservice*</exclude>
</excludes>
<binaries>
<outputDirectory>share/hadoop/${hadoop.component}</outputDirectory>
<includeDependencies>false</includeDependencies>

View File

@ -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.</description>
</property>
<property>
<name>security.collector-nodemanager.protocol.acl</name>
<value>*</value>
<description>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.</description>
</property>
</configuration>

View File

@ -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

View File

@ -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<JobHistoryEvent> {
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<TimelineMetric> 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<String, String> 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) {

View File

@ -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")

View File

@ -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?

View File

@ -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));
}
}

View File

@ -139,6 +139,8 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
private final Set<TaskAttemptId> inProgressAttempts;
private boolean historyTaskStartGenerated = false;
// Launch time reported in history events.
private long launchTime;
private static final SingleArcTransition<TaskImpl, TaskEvent>
ATTEMPT_KILLED_TRANSITION = new AttemptKilledTransition();
@ -705,8 +707,9 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
}
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<TaskEvent> {
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<String> diag, TaskStateInternal taskState, TaskAttemptId taId) {
private static TaskFailedEvent createTaskFailedEvent(TaskImpl task,
List<String> 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<TaskEvent> {
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<TaskEvent> {
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)));

View File

@ -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<ContainerStatus> finishedContainers = response.getCompletedContainersStatuses();
List<ContainerStatus> 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);
}

View File

@ -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<TimelineMetric> getTimelineMetrics() {
return null;
}
}
}

View File

@ -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<? extends AppContext> 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<? extends AppContext> 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();
}
}

View File

@ -297,7 +297,7 @@ public class TestLocalContainerAllocator {
Resources.none(), null, 1, null,
Collections.<NMToken>emptyList(),
yarnToken,
Collections.<UpdatedContainer>emptyList());
Collections.<UpdatedContainer>emptyList(), null);
response.setApplicationPriority(Priority.newInstance(0));
return response;
}

View File

@ -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<TimelineDelegationTokenIdentifier> collectorToken =
new Token<TimelineDelegationTokenIdentifier>(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<Token<? extends TokenIdentifier>> 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<TimelineDelegationTokenIdentifier> collectorToken1 =
new Token<TimelineDelegationTokenIdentifier>(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.<ContainerStatus>emptyList(),
Collections.<Container>emptyList(),
Collections.<NodeReport>emptyList(),
Resource.newInstance(512000, 1024), null, 10, null,
Collections.<NMToken>emptyList());
response.setCollectorInfo(collectorInfo);
return response;
}
}
public static void main(String[] args) throws Exception {
TestRMContainerAllocator t = new TestRMContainerAllocator();
t.testSimple();

View File

@ -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<TimelineMetric> getTimelineMetrics() {
return null;
}
}

View File

@ -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<TimelineMetric> getTimelineMetrics();
}

View File

@ -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<TimelineMetric> getTimelineMetrics() {
Set<TimelineMetric> jobMetrics = JobHistoryEventUtils.
countersToTimelineMetric(getTotalCounters(), finishTime);
jobMetrics.addAll(JobHistoryEventUtils.
countersToTimelineMetric(getMapCounters(), finishTime, "MAP:"));
jobMetrics.addAll(JobHistoryEventUtils.
countersToTimelineMetric(getReduceCounters(), finishTime, "REDUCE:"));
return jobMetrics;
}
}

View File

@ -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<TimelineMetric> getTimelineMetrics() {
return null;
}
}

View File

@ -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<TimelineMetric> getTimelineMetrics() {
return null;
}
}

View File

@ -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<TimelineMetric> getTimelineMetrics() {
return null;
}
}

View File

@ -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<TimelineMetric> getTimelineMetrics() {
return null;
}
}

View File

@ -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<TimelineMetric> getTimelineMetrics() {
return null;
}
}

View File

@ -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<JobACL, AccessControlList> 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<JobACL, AccessControlList> 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<TimelineMetric> getTimelineMetrics() {
return null;
}
}

View File

@ -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<TimelineMetric> getTimelineMetrics() {
return null;
}
}

View File

@ -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<TimelineMetric> getTimelineMetrics() {
Set<TimelineMetric> metrics = JobHistoryEventUtils
.countersToTimelineMetric(getCounters(), finishTime);
return metrics;
}
}

View File

@ -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<TimelineMetric> getTimelineMetrics() {
return null;
}
}

View File

@ -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<TimelineMetric> getTimelineMetrics() {
Set<TimelineMetric> metrics = JobHistoryEventUtils
.countersToTimelineMetric(getCounters(), finishTime);
return metrics;
}
}

View File

@ -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<TimelineMetric> getTimelineMetrics() {
Set<TimelineMetric> metrics = JobHistoryEventUtils
.countersToTimelineMetric(getCounters(), finishTime);
return metrics;
}
}

View File

@ -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<TimelineMetric> getTimelineMetrics() {
return null;
}
}

View File

@ -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<TimelineMetric> getTimelineMetrics() {
Set<TimelineMetric> metrics = JobHistoryEventUtils
.countersToTimelineMetric(getCounters(), finishTime);
return metrics;
}
}

View File

@ -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<TimelineMetric> getTimelineMetrics() {
Set<TimelineMetric> metrics = JobHistoryEventUtils
.countersToTimelineMetric(getCounters(), finishTime);
return metrics;
}
}

View File

@ -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<TimelineMetric> getTimelineMetrics() {
Set<TimelineMetric> jobMetrics = JobHistoryEventUtils
.countersToTimelineMetric(getCounters(), finishTime);
return jobMetrics;
}
}

View File

@ -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<TimelineMetric> getTimelineMetrics() {
return null;
}
}

View File

@ -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<TimelineMetric> getTimelineMetrics() {
return null;
}
}

View File

@ -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<TimelineMetric> countersToTimelineMetric(Counters counters,
long timestamp) {
return countersToTimelineMetric(counters, timestamp, "");
}
public static Set<TimelineMetric> countersToTimelineMetric(Counters counters,
long timestamp, String groupNamePrefix) {
Set<TimelineMetric> entityMetrics = new HashSet<TimelineMetric>();
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;
}
}

View File

@ -99,6 +99,12 @@
<scope>test</scope>
<type>test-jar</type>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-yarn-server-timelineservice</artifactId>
<scope>test</scope>
<type>test-jar</type>
</dependency>
<dependency>
<groupId>org.hsqldb</groupId>
<artifactId>hsqldb</artifactId>

View File

@ -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<YarnApplicationState> appStates =
EnumSet.allOf(YarnApplicationState.class);
ApplicationId firstAppId = null;
List<ApplicationReport> 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<String> 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<String> 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<String> 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,

View File

@ -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<ApplicationId, RMApp> 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<Boolean>() {
@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 {

View File

@ -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
<IntWritable, IntWritable, Writable, Writable> {
@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;
}

View File

@ -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);

View File

@ -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();

View File

@ -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<JobFiles> 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<TimelineEntity> 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<TimelineEntity> entitySet, UserGroupInformation ugi)
throws IOException {
TimelineEntities entities = new TimelineEntities();
entities.setEntities(entitySet);
collector.putEntities(entities, ugi);
}
private void writePerEntity(AppLevelTimelineCollector collector,
List<TimelineEntity> 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());
}
}
}

View File

@ -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', ' '};
}

View File

@ -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<IntWritable,IntWritable,Writable,Writable> {
class SimpleEntityWriterV1
extends org.apache.hadoop.mapreduce.Mapper
<IntWritable, IntWritable, Writable, Writable> {
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();

View File

@ -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);
}
}
}

View File

@ -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<TimelineEntity> createTaskAndTaskAttemptEntities(JobInfo jobInfo) {
private Set<TimelineEntity>
createTaskAndTaskAttemptEntities(JobInfo jobInfo) {
Set<TimelineEntity> entities = new HashSet<>();
Map<TaskID,TaskInfo> taskInfoMap = jobInfo.getAllTasks();
Map<TaskID, TaskInfo> 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<TimelineEntity> createTaskAttemptEntities(TaskInfo taskInfo) {
Set<TimelineEntity> taskAttempts = new HashSet<TimelineEntity>();
Map<TaskAttemptID,TaskAttemptInfo> taskAttemptInfoMap =
Map<TaskAttemptID, TaskAttemptInfo> 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());

View File

@ -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<TimelineEntity> createTimelineEntities(JobInfo jobInfo,
Configuration conf) {
List<TimelineEntity> entities = new ArrayList<>();
// create the job entity
TimelineEntity job = createJobEntity(jobInfo, conf);
entities.add(job);
// create the task and task attempt entities
List<TimelineEntity> 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<String, String> 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<TimelineEntity> createTaskAndTaskAttemptEntities(
JobInfo jobInfo) {
List<TimelineEntity> entities = new ArrayList<>();
Map<TaskID, TaskInfo> 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<TimelineEntity> 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<TimelineEntity> createTaskAttemptEntities(TaskInfo taskInfo) {
Set<TimelineEntity> taskAttempts = new HashSet<TimelineEntity>();
Map<TaskAttemptID, TaskAttemptInfo> 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;
}
}

View File

@ -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 <maps>] number of mappers (default: " + NUM_MAPS_DEFAULT +
")\n" +
" [-v] timeline service version\n" +
" [-mtype <mapper type in integer>]\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 <mapper type in integer>] (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 <path>] root path of job history files (mtype=2)\n" +
SimpleEntityWriterConstants.TEST_TIMES_DEFAULT + ")\n" +
" [-d <path>] hdfs root path of job history files (mtype=2)\n" +
" [-r <replay mode>] (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;
}

View File

@ -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);

View File

@ -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,

View File

@ -48,6 +48,11 @@
<!-- Version number for xerces used by JDiff -->
<xerces.jdiff.version>2.11.0</xerces.jdiff.version>
<kafka.version>0.8.2.1</kafka.version>
<hbase.version>1.2.6</hbase.version>
<hbase-compatible-hadoop.version>2.5.1</hbase-compatible-hadoop.version>
<hadoop.assemblies.version>${project.version}</hadoop.assemblies.version>
<commons-daemon.version>1.0.13</commons-daemon.version>
@ -319,6 +324,31 @@
<type>test-jar</type>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-yarn-server-applicationhistoryservice</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-yarn-server-timelineservice</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-yarn-server-timelineservice</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-yarn-server-timelineservice-hbase</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-yarn-applications-distributedshell</artifactId>
@ -489,6 +519,11 @@
<artifactId>commons-compress</artifactId>
<version>1.4.1</version>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-csv</artifactId>
<version>1.0</version>
</dependency>
<dependency>
<groupId>xmlenc</groupId>
<artifactId>xmlenc</artifactId>
@ -1110,6 +1145,73 @@
<artifactId>mssql-jdbc</artifactId>
<version>${mssql.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-common</artifactId>
<version>${hbase.version}</version>
<exclusions>
<exclusion>
<artifactId>jdk.tools</artifactId>
<groupId>jdk.tools</groupId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-common</artifactId>
<version>${hbase.version}</version>
<scope>test</scope>
<classifier>tests</classifier>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-client</artifactId>
<version>${hbase.version}</version>
<exclusions>
<!-- exclude jdk.tools (1.7) as we're not managing it -->
<exclusion>
<groupId>jdk.tools</groupId>
<artifactId>jdk.tools</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-server</artifactId>
<version>${hbase.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-server</artifactId>
<version>${hbase.version}</version>
<scope>test</scope>
<classifier>tests</classifier>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-testing-util</artifactId>
<version>${hbase.version}</version>
<scope>test</scope>
<optional>true</optional>
<exclusions>
<exclusion>
<groupId>org.jruby</groupId>
<artifactId>jruby-complete</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdfs</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-minicluster</artifactId>
</exclusion>
<exclusion>
<artifactId>jdk.tools</artifactId>
<groupId>jdk.tools</groupId>
</exclusion>
</exclusions>
</dependency>
</dependencies>
</dependencyManagement>

View File

@ -127,6 +127,7 @@
<item name="Node Labels" href="hadoop-yarn/hadoop-yarn-site/NodeLabel.html"/>
<item name="Web Application Proxy" href="hadoop-yarn/hadoop-yarn-site/WebApplicationProxy.html"/>
<item name="Timeline Server" href="hadoop-yarn/hadoop-yarn-site/TimelineServer.html"/>
<item name="Timeline Service V.2" href="hadoop-yarn/hadoop-yarn-site/TimelineServiceV2.html"/>
<item name="Writing YARN Applications" href="hadoop-yarn/hadoop-yarn-site/WritingYarnApplications.html"/>
<item name="YARN Application Security" href="hadoop-yarn/hadoop-yarn-site/YarnApplicationSecurity.html"/>
<item name="NodeManager" href="hadoop-yarn/hadoop-yarn-site/NodeManager.html"/>
@ -147,6 +148,7 @@
<item name="Resource Manager" href="hadoop-yarn/hadoop-yarn-site/ResourceManagerRest.html"/>
<item name="Node Manager" href="hadoop-yarn/hadoop-yarn-site/NodeManagerRest.html"/>
<item name="Timeline Server" href="hadoop-yarn/hadoop-yarn-site/TimelineServer.html#Timeline_Server_REST_API_v1"/>
<item name="Timeline Service V.2" href="hadoop-yarn/hadoop-yarn-site/TimelineServiceV2.html#Timeline_Service_v.2_REST_API"/>
</menu>
<menu name="Hadoop Compatible File Systems" inherit="top">

View File

@ -69,6 +69,7 @@ function print_usage(){
echo " Use -remove-application-from-state-store <appId> 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

View File

@ -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 ^<jar^> run a jar file

View File

@ -118,6 +118,23 @@
<Bug pattern="BC_UNCONFIRMED_CAST" />
</Match>
<!-- Object cast is based on the event type -->
<Match>
<Class name="org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher" />
<Method name="publishApplicationEvent" />
<Bug pattern="BC_UNCONFIRMED_CAST" />
</Match>
<Match>
<Class name="org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher" />
<Method name="publishLocalizationEvent" />
<Bug pattern="BC_UNCONFIRMED_CAST" />
</Match>
<Match>
<Class name="org.apache.hadoop.yarn.server.resourcemanager.metrics.TimelineServiceV2Publisher$TimelineV2EventHandler" />
<Bug pattern="BC_UNCONFIRMED_CAST" />
</Match>
<!-- Ignore intentional switch fallthroughs -->
<Match>
@ -517,6 +534,16 @@
</Or>
<Bug pattern="IS2_INCONSISTENT_SYNC" />
</Match>
<!-- Ignore SQL_PREPARED_STATEMENT_GENERATED_FROM_NONCONSTANT_STRING warnings for Timeline Phoenix storage. -->
<!-- Since we're using dynamic columns, we have to generate SQL statements dynamically -->
<Match>
<Class name="org.apache.hadoop.yarn.server.timelineservice.storage.PhoenixOfflineAggregationWriterImpl" />
<Or>
<Method name="storeEntityVariableLengthFields" />
<Method name="writeAggregatedEntity" />
</Or>
<Bug pattern="SQL_PREPARED_STATEMENT_GENERATED_FROM_NONCONSTANT_STRING" />
</Match>
<!-- Following fields are used in ErrorsAndWarningsBlock, which is not a part of analysis of findbugs -->
<Match>

View File

@ -47,6 +47,10 @@
<groupId>commons-logging</groupId>
<artifactId>commons-logging</artifactId>
</dependency>
<dependency>
<groupId>javax.xml.bind</groupId>
<artifactId>jaxb-api</artifactId>
</dependency>
<!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
<dependency>

View File

@ -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<NMToken> nmTokens,
List<ContainerResourceIncrease> increasedContainers,
List<ContainerResourceDecrease> decreasedContainers) {
return newInstance(responseId, completedContainers, allocatedContainers,
updatedNodes, availResources, command, numClusterNodes, preempt,
nmTokens);
List<ContainerResourceDecrease> 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<Container> allocatedContainers, List<NodeReport> updatedNodes,
Resource availResources, AMCommand command, int numClusterNodes,
PreemptionMessage preempt, List<NMToken> nmTokens, Token amRMToken,
List<UpdatedContainer> updatedContainers) {
List<UpdatedContainer> 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 <code>applicationPriority</code> of the response.
* @see AllocateResponse#setApplicationPriority(Priority)
* @param applicationPriority
* <code>applicationPriority</code> of the response
* @return {@link AllocateResponseBuilder}
*/
@Private
@Unstable
public AllocateResponseBuilder applicationPriority(
Priority applicationPriority) {
allocateResponse.setApplicationPriority(applicationPriority);
return this;
}
/**
* Set the <code>collectorInfo</code> of the response.
* @see AllocateResponse#setCollectorInfo(CollectorInfo)
* @param collectorInfo <code>collectorInfo</code> 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 <code>updateErrors</code> of the response.
* @see AllocateResponse#setUpdateErrors(List)
* @param updateErrors <code>updateErrors</code> of the response
* @return {@link AllocateResponseBuilder}
*/
@Private
@Unstable
public AllocateResponseBuilder updateErrors(
List<UpdateContainerError> 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<ContainerResourceDecrease> getDecreasedContainers();
}

View File

@ -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);
}

View File

@ -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;
/**
* <p>
@ -231,11 +232,8 @@ public class TimelineEntity implements Comparable<TimelineEntity> {
*/
public void setRelatedEntities(
Map<String, Set<String>> relatedEntities) {
if (relatedEntities != null && !(relatedEntities instanceof HashMap)) {
this.relatedEntities = new HashMap<String, Set<String>>(relatedEntities);
} else {
this.relatedEntities = (HashMap<String, Set<String>>) relatedEntities;
}
this.relatedEntities = TimelineServiceHelper.mapCastToHashMap(
relatedEntities);
}
/**
@ -297,11 +295,8 @@ public class TimelineEntity implements Comparable<TimelineEntity> {
* a map of primary filters
*/
public void setPrimaryFilters(Map<String, Set<Object>> primaryFilters) {
if (primaryFilters != null && !(primaryFilters instanceof HashMap)) {
this.primaryFilters = new HashMap<String, Set<Object>>(primaryFilters);
} else {
this.primaryFilters = (HashMap<String, Set<Object>>) primaryFilters;
}
this.primaryFilters =
TimelineServiceHelper.mapCastToHashMap(primaryFilters);
}
/**
@ -350,11 +345,7 @@ public class TimelineEntity implements Comparable<TimelineEntity> {
* a map of other information
*/
public void setOtherInfo(Map<String, Object> otherInfo) {
if (otherInfo != null && !(otherInfo instanceof HashMap)) {
this.otherInfo = new HashMap<String, Object>(otherInfo);
} else {
this.otherInfo = (HashMap<String, Object>) otherInfo;
}
this.otherInfo = TimelineServiceHelper.mapCastToHashMap(otherInfo);
}
/**

View File

@ -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<TimelineEvent> {
* a map of of the information of the event
*/
public void setEventInfo(Map<String, Object> eventInfo) {
if (eventInfo != null && !(eventInfo instanceof HashMap)) {
this.eventInfo = new HashMap<String, Object>(eventInfo);
} else {
this.eventInfo = (HashMap<String, Object>) eventInfo;
}
this.eventInfo = TimelineServiceHelper.mapCastToHashMap(
eventInfo);
}
@Override

View File

@ -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());
}
}
}

View File

@ -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;
}
}

View File

@ -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());
}
}
}

View File

@ -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());
}
}
}

View File

@ -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<FlowRunEntity> 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<FlowRunEntity> runs) {
flowRuns.addAll(runs);
}
@XmlElement(name = "flowruns")
public NavigableSet<FlowRunEntity> getFlowRuns() {
return flowRuns;
}
public int getNumberOfRuns() {
return flowRuns.size();
}
}

View File

@ -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);
}
}

View File

@ -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<Identifier> 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<Identifier> children = (Set<Identifier>) identifiers;
return children;
}
public void setChildren(Set<Identifier> children) {
addInfo(CHILDREN_INFO_KEY, children);
}
public void addChildren(Set<Identifier> children) {
TimelineEntityType thisType = TimelineEntityType.valueOf(getType());
for (Identifier child : children) {
validateChild(child, thisType);
}
Set<Identifier> 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());
}
}
}

View File

@ -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());
}
}
}

View File

@ -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<TimelineEntity> entities = new ArrayList<>();
public TimelineEntities() {
}
@XmlElement(name = "entities")
public List<TimelineEntity> getEntities() {
return entities;
}
public void setEntities(List<TimelineEntity> timelineEntities) {
this.entities = timelineEntities;
}
public void addEntities(List<TimelineEntity> timelineEntities) {
this.entities.addAll(timelineEntities);
}
public void addEntity(TimelineEntity entity) {
entities.add(entity);
}
}

View File

@ -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<TimelineEntity> {
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<String, Object> info = new HashMap<>();
private HashMap<String, String> configs = new HashMap<>();
private Set<TimelineMetric> metrics = new HashSet<>();
// events should be sorted by timestamp in descending order
private NavigableSet<TimelineEvent> events = new TreeSet<>();
private HashMap<String, Set<String>> isRelatedToEntities = new HashMap<>();
private HashMap<String, Set<String>> relatesToEntities = new HashMap<>();
private Long createdTime;
private long idPrefix;
public TimelineEntity() {
identifier = new Identifier();
}
/**
* <p>
* The constuctor is used to construct a proxy {@link TimelineEntity} or its
* subclass object from the real entity object that carries information.
* </p>
*
* <p>
* It is usually used in the case where we want to recover class polymorphism
* after deserializing the entity from its JSON form.
* </p>
* @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<String, Object> getInfoJAXB() {
if (real == null) {
return info;
} else {
return real.getInfoJAXB();
}
}
public Map<String, Object> getInfo() {
if (real == null) {
return info;
} else {
return real.getInfo();
}
}
public void setInfo(Map<String, Object> entityInfos) {
if (real == null) {
this.info = TimelineServiceHelper.mapCastToHashMap(entityInfos);
} else {
real.setInfo(entityInfos);
}
}
public void addInfo(Map<String, Object> 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<String, String> getConfigsJAXB() {
if (real == null) {
return configs;
} else {
return real.getConfigsJAXB();
}
}
public Map<String, String> getConfigs() {
if (real == null) {
return configs;
} else {
return real.getConfigs();
}
}
public void setConfigs(Map<String, String> entityConfigs) {
if (real == null) {
this.configs = TimelineServiceHelper.mapCastToHashMap(entityConfigs);
} else {
real.setConfigs(entityConfigs);
}
}
public void addConfigs(Map<String, String> 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<TimelineMetric> getMetrics() {
if (real == null) {
return metrics;
} else {
return real.getMetrics();
}
}
public void setMetrics(Set<TimelineMetric> entityMetrics) {
if (real == null) {
this.metrics = entityMetrics;
} else {
real.setMetrics(entityMetrics);
}
}
public void addMetrics(Set<TimelineMetric> 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<TimelineEvent> getEvents() {
if (real == null) {
return events;
} else {
return real.getEvents();
}
}
public void setEvents(NavigableSet<TimelineEvent> entityEvents) {
if (real == null) {
this.events = entityEvents;
} else {
real.setEvents(entityEvents);
}
}
public void addEvents(Set<TimelineEvent> 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<String, Set<String>> getIsRelatedToEntities() {
if (real == null) {
return isRelatedToEntities;
} else {
return real.getIsRelatedToEntities();
}
}
// required by JAXB
@InterfaceAudience.Private
@XmlElement(name = "isrelatedto")
public HashMap<String, Set<String>> getIsRelatedToEntitiesJAXB() {
if (real == null) {
return isRelatedToEntities;
} else {
return real.getIsRelatedToEntitiesJAXB();
}
}
@JsonSetter("isrelatedto")
public void setIsRelatedToEntities(
Map<String, Set<String>> isRelatedTo) {
if (real == null) {
this.isRelatedToEntities =
TimelineServiceHelper.mapCastToHashMap(isRelatedTo);
} else {
real.setIsRelatedToEntities(isRelatedTo);
}
}
public void addIsRelatedToEntities(
Map<String, Set<String>> isRelatedTo) {
if (real == null) {
for (Map.Entry<String, Set<String>> entry : isRelatedTo.entrySet()) {
Set<String> 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<String> 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<String, Set<String>> getRelatesToEntitiesJAXB() {
if (real == null) {
return relatesToEntities;
} else {
return real.getRelatesToEntitiesJAXB();
}
}
public Map<String, Set<String>> getRelatesToEntities() {
if (real == null) {
return relatesToEntities;
} else {
return real.getRelatesToEntities();
}
}
public void addRelatesToEntities(Map<String, Set<String>> relatesTo) {
if (real == null) {
for (Map.Entry<String, Set<String>> entry : relatesTo.entrySet()) {
Set<String> 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<String> 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<String, Set<String>> 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.
* <p>
* <b>Note</b>: Entities will be stored in the order of idPrefix specified.
* If users decide to set idPrefix for an entity, they <b>MUST</b> provide
* the same prefix for every update of this entity.
* </p>
* Example: <blockquote><pre>
* TimelineEntity entity = new TimelineEntity();
* entity.setIdPrefix(value);
* </pre></blockquote>
* 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);
}
}
}

View File

@ -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);
}
}

View File

@ -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<TimelineEvent> {
public static final long INVALID_TIMESTAMP = 0L;
private String id;
private HashMap<String, Object> 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<String, Object> getInfoJAXB() {
return info;
}
public Map<String, Object> getInfo() {
return info;
}
public void setInfo(Map<String, Object> infos) {
this.info = TimelineServiceHelper.mapCastToHashMap(infos);
}
public void addInfo(Map<String, Object> 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);
}
}
}

View File

@ -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<Long, Number> 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<Long, Number> getValuesJAXB() {
return values;
}
public Map<Long, Number> getValues() {
return values;
}
public void setValues(Map<Long, Number> 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<Long, Number> 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<Long, Number> 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<Object, Object> state) {
TimelineMetricOperation operation
= incomingMetric.getRealtimeAggregationOp();
return operation.aggregate(incomingMetric, baseAggregatedMetric, state);
}
}

View File

@ -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;
}
}

View File

@ -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<Object, Object> 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<Object, Object> 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<Object, Object> 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<Object, Object> 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<Object, Object> 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<Object, Object> 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<Object, Object> state);
}

View File

@ -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<TimelineWriteError> errors = new ArrayList<TimelineWriteError>();
public TimelineWriteResponse() {
}
/**
* Get a list of {@link TimelineWriteError} instances.
*
* @return a list of {@link TimelineWriteError} instances
*/
@XmlElement(name = "errors")
public List<TimelineWriteError> 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<TimelineWriteError> 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<TimelineWriteError> 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;
}
}
}

View File

@ -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());
}
}
}

View File

@ -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;

View File

@ -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);

View File

@ -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 <E> key type
* @param <V> value type
* @return casted HashMap object
*/
public static <E, V> HashMap<E, V> mapCastToHashMap(
Map<E, V> originalMap) {
return originalMap == null ? null : originalMap instanceof HashMap ?
(HashMap<E, V>) originalMap : new HashMap<E, V>(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;
}
}

View File

@ -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;
}

View File

@ -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;
}

View File

@ -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);
}
}

View File

@ -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<Object, Object> 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<Long, Number> metricValues = new HashMap<Long, Number>();
metricValues.put(timestamp, value);
m.setValues(metricValues);
return m;
}
}

View File

@ -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);

View File

@ -70,6 +70,16 @@
<groupId>commons-io</groupId>
<artifactId>commons-io</artifactId>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-yarn-server-applicationhistoryservice</artifactId>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-yarn-server-timelineservice</artifactId>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
<dependency>
<groupId>org.apache.hadoop</groupId>

View File

@ -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<ContainerId> launchedContainers =
Collections.newSetFromMap(new ConcurrentHashMap<ContainerId, Boolean>());
/**
* Container start times used to set id prefix while publishing entity
* to ATSv2.
*/
private final ConcurrentMap<ContainerId, Long> containerStartTimes =
new ConcurrentHashMap<ContainerId, Long>();
private ConcurrentMap<ContainerId, Long> 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<Void>() {
@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<Object>() {
@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<Object>() {
@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<Object>() {
@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);
}
}
}

View File

@ -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<String> tags = new HashSet<String>();
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<String, String> 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<String> commands = new ArrayList<String>();
commands.add(command.toString());

View File

@ -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<String> argsList = new ArrayList<String>(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<ApplicationReport> 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<String> argsList = new ArrayList<String>(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;
}
}

View File

@ -41,7 +41,7 @@ public class TestDistributedShellWithNodeLabels {
static final int NUM_NMS = 2;
TestDistributedShell distShellTest;
@Before
public void setup() throws Exception {
distShellTest = new TestDistributedShell();

View File

@ -54,6 +54,8 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> 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<T extends AMRMClient.ContainerRequest> 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 <code>check</code> 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<Boolean> check) throws InterruptedException {
waitFor(check, 1000);

View File

@ -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;
/**
* <code>AMRMClientAsync</code> 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 <code>check</code> 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<Boolean> check) throws InterruptedException {
waitFor(check, 1000);

View File

@ -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<T> {
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<T> {
}
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<NodeReport> updatedNodes = response.getUpdatedNodes();
if (!updatedNodes.isEmpty()) {
handler.onNodesUpdated(updatedNodes);

View File

@ -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());
}

View File

@ -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<AMRMTokenIdentifier> 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<AMRMTokenIdentifier> token) throws IOException {
for (int i = 0; i < this.cluster.getNumOfResourceManager(); i++) {
this.cluster.getResourceManager(i).getRMContext()
.getAMRMTokenSecretManager().addPersistedPassword(token);
}
}
}

View File

@ -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<ContainerStatus>(),
new ArrayList<Container>(), new ArrayList<NodeReport>(),
Resource.newInstance(1024, 2), null, 1,
null, new ArrayList<NMToken>());
if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) {
return AllocateResponse.newInstance(-1,
new ArrayList<ContainerStatus>(), new ArrayList<Container>(),
new ArrayList<NodeReport>(), Resource.newInstance(1024, 2), null, 1,
null, new ArrayList<NMToken>(), null,
new ArrayList<UpdatedContainer>(),
CollectorInfo.newInstance("host:port", Token.newInstance(
new byte[] {0}, "TIMELINE", new byte[] {0}, "rm")));
} else {
return AllocateResponse.newInstance(-1,
new ArrayList<ContainerStatus>(),
new ArrayList<Container>(), new ArrayList<NodeReport>(),
Resource.newInstance(1024, 2), null, 1,
null, new ArrayList<NMToken>());
}
}
}

View File

@ -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<ResourceRequest>(),
new ArrayList<ContainerId>(),
ResourceBlacklistRequest.newInstance(new ArrayList<String>(),
new ArrayList<String>()));
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());
}
}

View File

@ -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<AMRMTokenIdentifier> 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<ContainerId>(),
ResourceBlacklistRequest.newInstance(new ArrayList<String>(),
new ArrayList<String>()));
AllocateResponse response = amClient.allocate(request);
AllocateResponse response = getAMClient().allocate(request);
Assert.assertEquals(response, this.cluster.createFakeAllocateResponse());
}
private void syncToken(Token<AMRMTokenIdentifier> token) throws IOException {
for (int i = 0; i < this.cluster.getNumOfResourceManager(); i++) {
this.cluster.getResourceManager(i).getRMContext()
.getAMRMTokenSecretManager().addPersistedPassword(token);
}
}
}

View File

@ -426,8 +426,8 @@ public class TestAMRMClientAsync {
}
AllocateResponse response =
AllocateResponse.newInstance(0, completed, allocated,
new ArrayList<NodeReport>(), null, null, 1, null, nmTokens,
updatedContainers);
new ArrayList<NodeReport>(), null, null, 1, null, nmTokens, null,
updatedContainers, null);
return response;
}

View File

@ -215,6 +215,7 @@
<exclude>src/main/resources/webapps/yarn/.keep</exclude>
<exclude>src/main/resources/webapps/applicationhistory/.keep</exclude>
<exclude>src/main/resources/webapps/sharedcache/.keep</exclude>
<exclude>src/main/resources/webapps/timeline/.keep</exclude>
<exclude>src/main/resources/webapps/cluster/.keep</exclude>
<exclude>src/main/resources/webapps/test/.keep</exclude>
<exclude>src/main/resources/webapps/proxy/.keep</exclude>

Some files were not shown because too many files have changed in this diff Show More