YARN-5792. Adopt the id prefix for YARN, MR, and DS entities. Contributed by Varun Saxena.

This commit is contained in:
Sangjin Lee 2016-11-21 13:48:35 -08:00 committed by Varun Saxena
parent c92a7ab31c
commit 092fead5d9
30 changed files with 587 additions and 286 deletions

View File

@ -78,6 +78,9 @@
import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.util.TimelineServiceHelper;
import org.codehaus.jackson.map.ObjectMapper;
import org.codehaus.jackson.node.JsonNodeFactory;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.sun.jersey.api.client.ClientHandlerException; import com.sun.jersey.api.client.ClientHandlerException;
@ -1124,7 +1127,7 @@ private void processEventForTimelineServer(HistoryEvent event, JobId jobId,
private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
createTaskEntity(HistoryEvent event, long timestamp, String taskId, createTaskEntity(HistoryEvent event, long timestamp, String taskId,
String entityType, String relatedJobEntity, JobId jobId, String entityType, String relatedJobEntity, JobId jobId,
boolean setCreatedTime) { boolean setCreatedTime, long taskIdPrefix) {
org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity = org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
createBaseEntity(event, timestamp, entityType, setCreatedTime); createBaseEntity(event, timestamp, entityType, setCreatedTime);
entity.setId(taskId); entity.setId(taskId);
@ -1133,6 +1136,7 @@ private void processEventForTimelineServer(HistoryEvent event, JobId jobId,
((TaskStartedEvent)event).getTaskType().toString()); ((TaskStartedEvent)event).getTaskType().toString());
} }
entity.addIsRelatedToEntity(relatedJobEntity, jobId.toString()); entity.addIsRelatedToEntity(relatedJobEntity, jobId.toString());
entity.setIdPrefix(taskIdPrefix);
return entity; return entity;
} }
@ -1141,11 +1145,12 @@ private void processEventForTimelineServer(HistoryEvent event, JobId jobId,
private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
createTaskAttemptEntity(HistoryEvent event, long timestamp, createTaskAttemptEntity(HistoryEvent event, long timestamp,
String taskAttemptId, String entityType, String relatedTaskEntity, String taskAttemptId, String entityType, String relatedTaskEntity,
String taskId, boolean setCreatedTime) { String taskId, boolean setCreatedTime, long taskAttemptIdPrefix) {
org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity = org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
createBaseEntity(event, timestamp, entityType, setCreatedTime); createBaseEntity(event, timestamp, entityType, setCreatedTime);
entity.setId(taskAttemptId); entity.setId(taskAttemptId);
entity.addIsRelatedToEntity(relatedTaskEntity, taskId); entity.addIsRelatedToEntity(relatedTaskEntity, taskId);
entity.setIdPrefix(taskAttemptIdPrefix);
return entity; return entity;
} }
@ -1196,6 +1201,8 @@ private void processEventForNewTimelineService(HistoryEvent event,
String taskId = null; String taskId = null;
String taskAttemptId = null; String taskAttemptId = null;
boolean setCreatedTime = false; boolean setCreatedTime = false;
long taskIdPrefix = 0;
long taskAttemptIdPrefix = 0;
switch (event.getEventType()) { switch (event.getEventType()) {
// Handle job events // Handle job events
@ -1218,15 +1225,21 @@ private void processEventForNewTimelineService(HistoryEvent event,
case TASK_STARTED: case TASK_STARTED:
setCreatedTime = true; setCreatedTime = true;
taskId = ((TaskStartedEvent)event).getTaskId().toString(); taskId = ((TaskStartedEvent)event).getTaskId().toString();
taskIdPrefix = TimelineServiceHelper.
invertLong(((TaskStartedEvent)event).getStartTime());
break; break;
case TASK_FAILED: case TASK_FAILED:
taskId = ((TaskFailedEvent)event).getTaskId().toString(); taskId = ((TaskFailedEvent)event).getTaskId().toString();
taskIdPrefix = TimelineServiceHelper.
invertLong(((TaskFailedEvent)event).getStartTime());
break; break;
case TASK_UPDATED: case TASK_UPDATED:
taskId = ((TaskUpdatedEvent)event).getTaskId().toString(); taskId = ((TaskUpdatedEvent)event).getTaskId().toString();
break; break;
case TASK_FINISHED: case TASK_FINISHED:
taskId = ((TaskFinishedEvent)event).getTaskId().toString(); taskId = ((TaskFinishedEvent)event).getTaskId().toString();
taskIdPrefix = TimelineServiceHelper.
invertLong(((TaskFinishedEvent)event).getStartTime());
break; break;
case MAP_ATTEMPT_STARTED: case MAP_ATTEMPT_STARTED:
case REDUCE_ATTEMPT_STARTED: case REDUCE_ATTEMPT_STARTED:
@ -1234,6 +1247,8 @@ private void processEventForNewTimelineService(HistoryEvent event,
taskId = ((TaskAttemptStartedEvent)event).getTaskId().toString(); taskId = ((TaskAttemptStartedEvent)event).getTaskId().toString();
taskAttemptId = ((TaskAttemptStartedEvent)event). taskAttemptId = ((TaskAttemptStartedEvent)event).
getTaskAttemptId().toString(); getTaskAttemptId().toString();
taskAttemptIdPrefix = TimelineServiceHelper.
invertLong(((TaskAttemptStartedEvent)event).getStartTime());
break; break;
case CLEANUP_ATTEMPT_STARTED: case CLEANUP_ATTEMPT_STARTED:
case SETUP_ATTEMPT_STARTED: case SETUP_ATTEMPT_STARTED:
@ -1253,16 +1268,22 @@ private void processEventForNewTimelineService(HistoryEvent event,
getTaskId().toString(); getTaskId().toString();
taskAttemptId = ((TaskAttemptUnsuccessfulCompletionEvent)event). taskAttemptId = ((TaskAttemptUnsuccessfulCompletionEvent)event).
getTaskAttemptId().toString(); getTaskAttemptId().toString();
taskAttemptIdPrefix = TimelineServiceHelper.invertLong(
((TaskAttemptUnsuccessfulCompletionEvent)event).getStartTime());
break; break;
case MAP_ATTEMPT_FINISHED: case MAP_ATTEMPT_FINISHED:
taskId = ((MapAttemptFinishedEvent)event).getTaskId().toString(); taskId = ((MapAttemptFinishedEvent)event).getTaskId().toString();
taskAttemptId = ((MapAttemptFinishedEvent)event). taskAttemptId = ((MapAttemptFinishedEvent)event).
getAttemptId().toString(); getAttemptId().toString();
taskAttemptIdPrefix = TimelineServiceHelper.
invertLong(((MapAttemptFinishedEvent)event).getStartTime());
break; break;
case REDUCE_ATTEMPT_FINISHED: case REDUCE_ATTEMPT_FINISHED:
taskId = ((ReduceAttemptFinishedEvent)event).getTaskId().toString(); taskId = ((ReduceAttemptFinishedEvent)event).getTaskId().toString();
taskAttemptId = ((ReduceAttemptFinishedEvent)event). taskAttemptId = ((ReduceAttemptFinishedEvent)event).
getAttemptId().toString(); getAttemptId().toString();
taskAttemptIdPrefix = TimelineServiceHelper.
invertLong(((ReduceAttemptFinishedEvent)event).getStartTime());
break; break;
case SETUP_ATTEMPT_FINISHED: case SETUP_ATTEMPT_FINISHED:
case CLEANUP_ATTEMPT_FINISHED: case CLEANUP_ATTEMPT_FINISHED:
@ -1291,12 +1312,12 @@ private void processEventForNewTimelineService(HistoryEvent event,
// TaskEntity // TaskEntity
tEntity = createTaskEntity(event, timestamp, taskId, tEntity = createTaskEntity(event, timestamp, taskId,
MAPREDUCE_TASK_ENTITY_TYPE, MAPREDUCE_JOB_ENTITY_TYPE, MAPREDUCE_TASK_ENTITY_TYPE, MAPREDUCE_JOB_ENTITY_TYPE,
jobId, setCreatedTime); jobId, setCreatedTime, taskIdPrefix);
} else { } else {
// TaskAttemptEntity // TaskAttemptEntity
tEntity = createTaskAttemptEntity(event, timestamp, taskAttemptId, tEntity = createTaskAttemptEntity(event, timestamp, taskAttemptId,
MAPREDUCE_TASK_ATTEMPT_ENTITY_TYPE, MAPREDUCE_TASK_ENTITY_TYPE, MAPREDUCE_TASK_ATTEMPT_ENTITY_TYPE, MAPREDUCE_TASK_ENTITY_TYPE,
taskId, setCreatedTime); taskId, setCreatedTime, taskAttemptIdPrefix);
} }
} }
try { try {

View File

@ -1530,7 +1530,7 @@ private static JobCounterUpdateEvent createJobCounterUpdateEventTAKilled(
StringUtils.join( StringUtils.join(
LINE_SEPARATOR, taskAttempt.getDiagnostics()), LINE_SEPARATOR, taskAttempt.getDiagnostics()),
taskAttempt.getCounters(), taskAttempt taskAttempt.getCounters(), taskAttempt
.getProgressSplitBlock().burst()); .getProgressSplitBlock().burst(), taskAttempt.launchTime);
return tauce; return tauce;
} }
@ -1943,35 +1943,35 @@ private void logAttemptFinishedEvent(TaskAttemptStateInternal state) {
this.container == null ? -1 : this.container.getNodeId().getPort(); this.container == null ? -1 : this.container.getNodeId().getPort();
if (attemptId.getTaskId().getTaskType() == TaskType.MAP) { if (attemptId.getTaskId().getTaskType() == TaskType.MAP) {
MapAttemptFinishedEvent mfe = MapAttemptFinishedEvent mfe =
new MapAttemptFinishedEvent(TypeConverter.fromYarn(attemptId), new MapAttemptFinishedEvent(TypeConverter.fromYarn(attemptId),
TypeConverter.fromYarn(attemptId.getTaskId().getTaskType()), TypeConverter.fromYarn(attemptId.getTaskId().getTaskType()),
state.toString(), state.toString(),
this.reportedStatus.mapFinishTime, this.reportedStatus.mapFinishTime,
finishTime, finishTime,
containerHostName, containerHostName,
containerNodePort, containerNodePort,
this.nodeRackName == null ? "UNKNOWN" : this.nodeRackName, this.nodeRackName == null ? "UNKNOWN" : this.nodeRackName,
this.reportedStatus.stateString, this.reportedStatus.stateString,
getCounters(), getCounters(),
getProgressSplitBlock().burst()); getProgressSplitBlock().burst(), launchTime);
eventHandler.handle( eventHandler.handle(
new JobHistoryEvent(attemptId.getTaskId().getJobId(), mfe)); new JobHistoryEvent(attemptId.getTaskId().getJobId(), mfe));
} else { } else {
ReduceAttemptFinishedEvent rfe = ReduceAttemptFinishedEvent rfe =
new ReduceAttemptFinishedEvent(TypeConverter.fromYarn(attemptId), new ReduceAttemptFinishedEvent(TypeConverter.fromYarn(attemptId),
TypeConverter.fromYarn(attemptId.getTaskId().getTaskType()), TypeConverter.fromYarn(attemptId.getTaskId().getTaskType()),
state.toString(), state.toString(),
this.reportedStatus.shuffleFinishTime, this.reportedStatus.shuffleFinishTime,
this.reportedStatus.sortFinishTime, this.reportedStatus.sortFinishTime,
finishTime, finishTime,
containerHostName, containerHostName,
containerNodePort, containerNodePort,
this.nodeRackName == null ? "UNKNOWN" : this.nodeRackName, this.nodeRackName == null ? "UNKNOWN" : this.nodeRackName,
this.reportedStatus.stateString, this.reportedStatus.stateString,
getCounters(), getCounters(),
getProgressSplitBlock().burst()); getProgressSplitBlock().burst(), launchTime);
eventHandler.handle( eventHandler.handle(
new JobHistoryEvent(attemptId.getTaskId().getJobId(), rfe)); 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 final Set<TaskAttemptId> inProgressAttempts;
private boolean historyTaskStartGenerated = false; private boolean historyTaskStartGenerated = false;
// Launch time reported in history events.
private long launchTime;
private static final SingleArcTransition<TaskImpl, TaskEvent> private static final SingleArcTransition<TaskImpl, TaskEvent>
ATTEMPT_KILLED_TRANSITION = new AttemptKilledTransition(); ATTEMPT_KILLED_TRANSITION = new AttemptKilledTransition();
@ -705,8 +707,9 @@ private void handleTaskAttemptCompletion(TaskAttemptId attemptId,
} }
private void sendTaskStartedEvent() { private void sendTaskStartedEvent() {
launchTime = getLaunchTime();
TaskStartedEvent tse = new TaskStartedEvent( TaskStartedEvent tse = new TaskStartedEvent(
TypeConverter.fromYarn(taskId), getLaunchTime(), TypeConverter.fromYarn(taskId), launchTime,
TypeConverter.fromYarn(taskId.getTaskType()), TypeConverter.fromYarn(taskId.getTaskType()),
getSplitsAsString()); getSplitsAsString());
eventHandler eventHandler
@ -714,18 +717,19 @@ private void sendTaskStartedEvent() {
historyTaskStartGenerated = true; historyTaskStartGenerated = true;
} }
private static TaskFinishedEvent createTaskFinishedEvent(TaskImpl task, TaskStateInternal taskState) { private static TaskFinishedEvent createTaskFinishedEvent(TaskImpl task,
TaskStateInternal taskState) {
TaskFinishedEvent tfe = TaskFinishedEvent tfe =
new TaskFinishedEvent(TypeConverter.fromYarn(task.taskId), new TaskFinishedEvent(TypeConverter.fromYarn(task.taskId),
TypeConverter.fromYarn(task.successfulAttempt), TypeConverter.fromYarn(task.successfulAttempt),
task.getFinishTime(task.successfulAttempt), task.getFinishTime(task.successfulAttempt),
TypeConverter.fromYarn(task.taskId.getTaskType()), TypeConverter.fromYarn(task.taskId.getTaskType()),
taskState.toString(), taskState.toString(), task.getCounters(), task.launchTime);
task.getCounters());
return tfe; 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(); StringBuilder errorSb = new StringBuilder();
if (diag != null) { if (diag != null) {
for (String d : diag) { for (String d : diag) {
@ -740,7 +744,7 @@ private static TaskFailedEvent createTaskFailedEvent(TaskImpl task, List<String>
errorSb.toString(), errorSb.toString(),
taskState.toString(), taskState.toString(),
taId == null ? null : TypeConverter.fromYarn(taId), taId == null ? null : TypeConverter.fromYarn(taId),
task.getCounters()); task.getCounters(), task.launchTime);
return taskFailedEvent; return taskFailedEvent;
} }
@ -861,7 +865,8 @@ private TaskStateInternal recover(TaskInfo taskInfo,
TaskFailedEvent tfe = new TaskFailedEvent(taskInfo.getTaskId(), TaskFailedEvent tfe = new TaskFailedEvent(taskInfo.getTaskId(),
taskInfo.getFinishTime(), taskInfo.getTaskType(), taskInfo.getFinishTime(), taskInfo.getTaskType(),
taskInfo.getError(), taskInfo.getTaskStatus(), taskInfo.getError(), taskInfo.getTaskStatus(),
taskInfo.getFailedDueToAttemptId(), taskInfo.getCounters()); taskInfo.getFailedDueToAttemptId(), taskInfo.getCounters(),
launchTime);
eventHandler.handle(new JobHistoryEvent(taskId.getJobId(), tfe)); eventHandler.handle(new JobHistoryEvent(taskId.getJobId(), tfe));
eventHandler.handle( eventHandler.handle(
new JobTaskEvent(taskId, getExternalState(taskState))); new JobTaskEvent(taskId, getExternalState(taskState)));

View File

@ -58,7 +58,7 @@ public void testTaskAttemptFinishedEvent() throws Exception {
Counters counters = new Counters(); Counters counters = new Counters();
TaskAttemptFinishedEvent test = new TaskAttemptFinishedEvent(taskAttemptId, TaskAttemptFinishedEvent test = new TaskAttemptFinishedEvent(taskAttemptId,
TaskType.REDUCE, "TEST", 123L, "RAKNAME", "HOSTNAME", "STATUS", TaskType.REDUCE, "TEST", 123L, "RAKNAME", "HOSTNAME", "STATUS",
counters); counters, 234);
assertEquals(test.getAttemptId().toString(), taskAttemptId.toString()); assertEquals(test.getAttemptId().toString(), taskAttemptId.toString());
assertEquals(test.getCounters(), counters); assertEquals(test.getCounters(), counters);
@ -69,7 +69,7 @@ public void testTaskAttemptFinishedEvent() throws Exception {
assertEquals(test.getTaskId(), tid); assertEquals(test.getTaskId(), tid);
assertEquals(test.getTaskStatus(), "TEST"); assertEquals(test.getTaskStatus(), "TEST");
assertEquals(test.getTaskType(), TaskType.REDUCE); assertEquals(test.getTaskType(), TaskType.REDUCE);
assertEquals(234, test.getStartTime());
} }
/** /**

View File

@ -148,7 +148,7 @@ public void testFirstFlushOnCompletionEvent() throws Exception {
// First completion event, but min-queue-size for batching flushes is 10 // First completion event, but min-queue-size for batching flushes is 10
handleEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent( 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(); verify(mockWriter).flush();
} finally { } finally {
@ -184,7 +184,7 @@ public void testMaxUnflushedCompletionEvents() throws Exception {
for (int i = 0 ; i < 100 ; i++) { for (int i = 0 ; i < 100 ; i++) {
queueEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent( 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); handleNextNEvents(jheh, 9);
@ -229,7 +229,7 @@ public void testUnflushedTimer() throws Exception {
for (int i = 0 ; i < 100 ; i++) { for (int i = 0 ; i < 100 ; i++) {
queueEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent( 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); handleNextNEvents(jheh, 9);
@ -272,7 +272,7 @@ public void testBatchedFlushJobEndMultiplier() throws Exception {
for (int i = 0 ; i < 100 ; i++) { for (int i = 0 ; i < 100 ; i++) {
queueEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent( 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( queueEvent(jheh, new JobHistoryEvent(t.jobId, new JobFinishedEvent(
TypeConverter.fromYarn(t.jobId), 0, 10, 10, 0, 0, null, null, new Counters()))); TypeConverter.fromYarn(t.jobId), 0, 10, 10, 0, 0, null, null, new Counters())));

View File

@ -32,9 +32,10 @@
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
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 @InterfaceAudience.Private
@ -58,9 +59,10 @@ public class MapAttemptFinishedEvent implements HistoryEvent {
int[] cpuUsages; int[] cpuUsages;
int[] vMemKbytes; int[] vMemKbytes;
int[] physMemKbytes; 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 id Task Attempt ID
* @param taskType Type of the task * @param taskType Type of the task
* @param taskStatus Status of the task * @param taskStatus Status of the task
@ -77,12 +79,13 @@ public class MapAttemptFinishedEvent implements HistoryEvent {
* virtual memory and physical memory. * virtual memory and physical memory.
* *
* If you have no splits data, code {@code null} for this * 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 public MapAttemptFinishedEvent(TaskAttemptID id, TaskType taskType,
(TaskAttemptID id, TaskType taskType, String taskStatus, String taskStatus, long mapFinishTime, long finishTime, String hostname,
long mapFinishTime, long finishTime, String hostname, int port, int port, String rackName, String state, Counters counters,
String rackName, String state, Counters counters, int[][] allSplits) { int[][] allSplits, long startTs) {
this.attemptId = id; this.attemptId = id;
this.taskType = taskType; this.taskType = taskType;
this.taskStatus = taskStatus; this.taskStatus = taskStatus;
@ -98,6 +101,16 @@ public class MapAttemptFinishedEvent implements HistoryEvent {
this.cpuUsages = ProgressSplitsBlock.arrayGetCPUTime(allSplits); this.cpuUsages = ProgressSplitsBlock.arrayGetCPUTime(allSplits);
this.vMemKbytes = ProgressSplitsBlock.arrayGetVMemKbytes(allSplits); this.vMemKbytes = ProgressSplitsBlock.arrayGetVMemKbytes(allSplits);
this.physMemKbytes = ProgressSplitsBlock.arrayGetPhysMemKbytes(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());
} }
/** /**
@ -117,15 +130,13 @@ public class MapAttemptFinishedEvent implements HistoryEvent {
* @param counters Counters for the attempt * @param counters Counters for the attempt
*/ */
@Deprecated @Deprecated
public MapAttemptFinishedEvent public MapAttemptFinishedEvent(TaskAttemptID id, TaskType taskType,
(TaskAttemptID id, TaskType taskType, String taskStatus, String taskStatus, long mapFinishTime, long finishTime, String hostname,
long mapFinishTime, long finishTime, String hostname, String state, Counters counters) {
String state, Counters counters) {
this(id, taskType, taskStatus, mapFinishTime, finishTime, hostname, -1, "", this(id, taskType, taskStatus, mapFinishTime, finishTime, hostname, -1, "",
state, counters, null); state, counters, null);
} }
MapAttemptFinishedEvent() {} MapAttemptFinishedEvent() {}
public Object getDatum() { public Object getDatum() {
@ -175,38 +186,56 @@ public void setDatum(Object oDatum) {
this.physMemKbytes = AvroArrayUtils.fromAvro(datum.getPhysMemKbytes()); this.physMemKbytes = AvroArrayUtils.fromAvro(datum.getPhysMemKbytes());
} }
/** Get the task ID */ /** Gets the task ID. */
public TaskID getTaskId() { return attemptId.getTaskID(); } public TaskID getTaskId() {
/** Get the attempt id */ return attemptId.getTaskID();
}
/** Gets the attempt id. */
public TaskAttemptID getAttemptId() { public TaskAttemptID getAttemptId() {
return attemptId; return attemptId;
} }
/** Get the task type */ /** Gets the task type. */
public TaskType getTaskType() { public TaskType getTaskType() {
return taskType; return taskType;
} }
/** Get the task status */ /** Gets the task status. */
public String getTaskStatus() { return taskStatus.toString(); } public String getTaskStatus() { return taskStatus.toString(); }
/** Get the map phase finish time */ /** Gets the map phase finish time. */
public long getMapFinishTime() { return mapFinishTime; } public long getMapFinishTime() { return mapFinishTime; }
/** Get the attempt finish time */ /** Gets the attempt finish time. */
public long getFinishTime() { return finishTime; } 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(); } public String getHostname() { return hostname.toString(); }
/** Get the tracker rpc port */ /** Gets the tracker rpc port. */
public int getPort() { return port; } public int getPort() { return port; }
/** Get the rack name */ /** Gets the rack name. */
public String getRackName() { public String getRackName() {
return rackName == null ? null : rackName.toString(); return rackName == null ? null : rackName.toString();
} }
/**
/** Get the state string */ * Gets the attempt state string.
public String getState() { return state.toString(); } * @return map attempt state
/** Get the counters */ */
Counters getCounters() { return counters; } public String getState() {
/** Get the event type */ return state.toString();
}
/**
* Gets the counters.
* @return counters
*/
Counters getCounters() {
return counters;
}
/** Gets the event type. */
public EventType getEventType() { public EventType getEventType() {
return EventType.MAP_ATTEMPT_FINISHED; return EventType.MAP_ATTEMPT_FINISHED;
} }

View File

@ -32,6 +32,7 @@
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
import org.apache.hadoop.yarn.util.SystemClock;
/** /**
* Event to record successful completion of a reduce attempt * Event to record successful completion of a reduce attempt
@ -59,6 +60,7 @@ public class ReduceAttemptFinishedEvent implements HistoryEvent {
int[] cpuUsages; int[] cpuUsages;
int[] vMemKbytes; int[] vMemKbytes;
int[] physMemKbytes; int[] physMemKbytes;
private long startTime;
/** /**
* Create an event to record completion of a reduce attempt * Create an event to record completion of a reduce attempt
@ -76,13 +78,13 @@ public class ReduceAttemptFinishedEvent implements HistoryEvent {
* @param allSplits the "splits", or a pixelated graph of various * @param allSplits the "splits", or a pixelated graph of various
* measurable worker node state variables against progress. * measurable worker node state variables against progress.
* Currently there are four; wallclock time, CPU time, * 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 public ReduceAttemptFinishedEvent(TaskAttemptID id, TaskType taskType,
(TaskAttemptID id, TaskType taskType, String taskStatus, String taskStatus, long shuffleFinishTime, long sortFinishTime,
long shuffleFinishTime, long sortFinishTime, long finishTime, long finishTime, String hostname, int port, String rackName,
String hostname, int port, String rackName, String state, String state, Counters counters, int[][] allSplits, long startTs) {
Counters counters, int[][] allSplits) {
this.attemptId = id; this.attemptId = id;
this.taskType = taskType; this.taskType = taskType;
this.taskStatus = taskStatus; this.taskStatus = taskStatus;
@ -99,6 +101,16 @@ public class ReduceAttemptFinishedEvent implements HistoryEvent {
this.cpuUsages = ProgressSplitsBlock.arrayGetCPUTime(allSplits); this.cpuUsages = ProgressSplitsBlock.arrayGetCPUTime(allSplits);
this.vMemKbytes = ProgressSplitsBlock.arrayGetVMemKbytes(allSplits); this.vMemKbytes = ProgressSplitsBlock.arrayGetVMemKbytes(allSplits);
this.physMemKbytes = ProgressSplitsBlock.arrayGetPhysMemKbytes(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());
} }
/** /**
@ -118,13 +130,12 @@ public class ReduceAttemptFinishedEvent implements HistoryEvent {
* @param state State of the attempt * @param state State of the attempt
* @param counters Counters for the attempt * @param counters Counters for the attempt
*/ */
public ReduceAttemptFinishedEvent public ReduceAttemptFinishedEvent(TaskAttemptID id, TaskType taskType,
(TaskAttemptID id, TaskType taskType, String taskStatus, String taskStatus, long shuffleFinishTime, long sortFinishTime,
long shuffleFinishTime, long sortFinishTime, long finishTime, long finishTime, String hostname, String state, Counters counters) {
String hostname, String state, Counters counters) {
this(id, taskType, taskStatus, this(id, taskType, taskStatus,
shuffleFinishTime, sortFinishTime, finishTime, shuffleFinishTime, sortFinishTime, finishTime,
hostname, -1, "", state, counters, null); hostname, -1, "", state, counters, null);
} }
ReduceAttemptFinishedEvent() {} ReduceAttemptFinishedEvent() {}
@ -178,39 +189,55 @@ public void setDatum(Object oDatum) {
this.physMemKbytes = AvroArrayUtils.fromAvro(datum.getPhysMemKbytes()); this.physMemKbytes = AvroArrayUtils.fromAvro(datum.getPhysMemKbytes());
} }
/** Get the Task ID */ /** Gets the Task ID. */
public TaskID getTaskId() { return attemptId.getTaskID(); } public TaskID getTaskId() { return attemptId.getTaskID(); }
/** Get the attempt id */ /** Gets the attempt id. */
public TaskAttemptID getAttemptId() { public TaskAttemptID getAttemptId() {
return attemptId; return attemptId;
} }
/** Get the task type */ /** Gets the task type. */
public TaskType getTaskType() { public TaskType getTaskType() {
return taskType; return taskType;
} }
/** Get the task status */ /** Gets the task status. */
public String getTaskStatus() { return taskStatus.toString(); } 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; } 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; } public long getShuffleFinishTime() { return shuffleFinishTime; }
/** Get the finish time of the attempt */ /** Gets the finish time of the attempt. */
public long getFinishTime() { return finishTime; } 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(); } public String getHostname() { return hostname.toString(); }
/** Get the tracker rpc port */ /** Gets the tracker rpc port. */
public int getPort() { return 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() { public String getRackName() {
return rackName == null ? null : rackName.toString(); return rackName == null ? null : rackName.toString();
} }
/**
/** Get the state string */ * Gets the state string.
public String getState() { return state.toString(); } * @return reduce attempt state
/** Get the counters for the attempt */ */
Counters getCounters() { return counters; } public String getState() {
/** Get the event type */ return state.toString();
}
/**
* Gets the counters.
* @return counters
*/
Counters getCounters() {
return counters;
}
/** Gets the event type. */
public EventType getEventType() { public EventType getEventType() {
return EventType.REDUCE_ATTEMPT_FINISHED; return EventType.REDUCE_ATTEMPT_FINISHED;
} }

View File

@ -31,6 +31,7 @@
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
import org.apache.hadoop.yarn.util.SystemClock;
/** /**
* Event to record successful task completion * Event to record successful task completion
@ -50,10 +51,11 @@ public class TaskAttemptFinishedEvent implements HistoryEvent {
private String hostname; private String hostname;
private String state; private String state;
private Counters counters; private Counters counters;
private long startTime;
/** /**
* Create an event to record successful finishes for setup and cleanup * Create an event to record successful finishes for setup and cleanup
* attempts * attempts.
* @param id Attempt ID * @param id Attempt ID
* @param taskType Type of task * @param taskType Type of task
* @param taskStatus Status of task * @param taskStatus Status of task
@ -61,11 +63,12 @@ public class TaskAttemptFinishedEvent implements HistoryEvent {
* @param hostname Host where the attempt executed * @param hostname Host where the attempt executed
* @param state State string * @param state State string
* @param counters Counters for the attempt * @param counters Counters for the attempt
* @param startTs Task start time to be used for writing entity to ATSv2.
*/ */
public TaskAttemptFinishedEvent(TaskAttemptID id, public TaskAttemptFinishedEvent(TaskAttemptID id,
TaskType taskType, String taskStatus, TaskType taskType, String taskStatus,
long finishTime, String rackName, long finishTime, String rackName,
String hostname, String state, Counters counters) { String hostname, String state, Counters counters, long startTs) {
this.attemptId = id; this.attemptId = id;
this.taskType = taskType; this.taskType = taskType;
this.taskStatus = taskStatus; this.taskStatus = taskStatus;
@ -74,6 +77,14 @@ public TaskAttemptFinishedEvent(TaskAttemptID id,
this.hostname = hostname; this.hostname = hostname;
this.state = state; this.state = state;
this.counters = counters; 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() {} TaskAttemptFinishedEvent() {}
@ -107,33 +118,43 @@ public void setDatum(Object oDatum) {
this.counters = EventReader.fromAvro(datum.getCounters()); this.counters = EventReader.fromAvro(datum.getCounters());
} }
/** Get the task ID */ /** Gets the task ID. */
public TaskID getTaskId() { return attemptId.getTaskID(); } public TaskID getTaskId() { return attemptId.getTaskID(); }
/** Get the task attempt id */ /** Gets the task attempt id. */
public TaskAttemptID getAttemptId() { public TaskAttemptID getAttemptId() {
return attemptId; return attemptId;
} }
/** Get the task type */ /** Gets the task type. */
public TaskType getTaskType() { public TaskType getTaskType() {
return taskType; return taskType;
} }
/** Get the task status */ /** Gets the task status. */
public String getTaskStatus() { return taskStatus.toString(); } public String getTaskStatus() { return taskStatus.toString(); }
/** Get the attempt finish time */ /** Gets the attempt finish time. */
public long getFinishTime() { return finishTime; } 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(); } public String getHostname() { return hostname.toString(); }
/** Get the rackname where the attempt executed */ /** Gets the rackname where the attempt executed. */
public String getRackName() { public String getRackName() {
return rackName == null ? null : rackName.toString(); return rackName == null ? null : rackName.toString();
} }
/** Get the state string */ /**
* Gets the state string.
* @return task attempt state.
*/
public String getState() { return state.toString(); } public String getState() { return state.toString(); }
/** Get the counters for the attempt */ /** Gets the counters for the attempt. */
Counters getCounters() { return counters; } Counters getCounters() { return counters; }
/** Get the event type */ /** Gets the event type. */
public EventType getEventType() { public EventType getEventType() {
// Note that the task type can be setup/map/reduce/cleanup but the // Note that the task type can be setup/map/reduce/cleanup but the
// attempt-type can only be map/reduce. // attempt-type can only be map/reduce.

View File

@ -33,6 +33,7 @@
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
import org.apache.hadoop.yarn.util.SystemClock;
/** /**
* Event to record unsuccessful (Killed/Failed) completion of task attempts * Event to record unsuccessful (Killed/Failed) completion of task attempts
@ -58,10 +59,11 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
int[] cpuUsages; int[] cpuUsages;
int[] vMemKbytes; int[] vMemKbytes;
int[] physMemKbytes; int[] physMemKbytes;
private long startTime;
private static final Counters EMPTY_COUNTERS = new Counters(); 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 id Attempt ID
* @param taskType Type of the task * @param taskType Type of the task
* @param status Status of the attempt * @param status Status of the attempt
@ -75,12 +77,13 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
* measurable worker node state variables against progress. * measurable worker node state variables against progress.
* Currently there are four; wallclock time, CPU time, * 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 TaskAttemptUnsuccessfulCompletionEvent public TaskAttemptUnsuccessfulCompletionEvent
(TaskAttemptID id, TaskType taskType, (TaskAttemptID id, TaskType taskType,
String status, long finishTime, String status, long finishTime,
String hostname, int port, String rackName, String hostname, int port, String rackName,
String error, Counters counters, int[][] allSplits) { String error, Counters counters, int[][] allSplits, long startTs) {
this.attemptId = id; this.attemptId = id;
this.taskType = taskType; this.taskType = taskType;
this.status = status; this.status = status;
@ -99,6 +102,15 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
ProgressSplitsBlock.arrayGetVMemKbytes(allSplits); ProgressSplitsBlock.arrayGetVMemKbytes(allSplits);
this.physMemKbytes = this.physMemKbytes =
ProgressSplitsBlock.arrayGetPhysMemKbytes(allSplits); 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());
} }
/** /**
@ -190,39 +202,49 @@ public void setDatum(Object odatum) {
AvroArrayUtils.fromAvro(datum.getPhysMemKbytes()); AvroArrayUtils.fromAvro(datum.getPhysMemKbytes());
} }
/** Get the task id */ /** Gets the task id. */
public TaskID getTaskId() { public TaskID getTaskId() {
return attemptId.getTaskID(); return attemptId.getTaskID();
} }
/** Get the task type */ /** Gets the task type. */
public TaskType getTaskType() { public TaskType getTaskType() {
return TaskType.valueOf(taskType.toString()); return TaskType.valueOf(taskType.toString());
} }
/** Get the attempt id */ /** Gets the attempt id. */
public TaskAttemptID getTaskAttemptId() { public TaskAttemptID getTaskAttemptId() {
return attemptId; return attemptId;
} }
/** Get the finish time */ /** Gets the finish time. */
public long getFinishTime() { return finishTime; } 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; } 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; } 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() { public String getRackName() {
return rackName == null ? null : rackName.toString(); return rackName == null ? null : rackName.toString();
} }
/** Get the error string */ /** Gets the error string. */
public String getError() { return error.toString(); } public String getError() { return error.toString(); }
/** Get the task status */ /**
* Gets the task attempt status.
* @return task attempt status.
*/
public String getTaskStatus() { public String getTaskStatus() {
return status.toString(); return status.toString();
} }
/** Get the counters */ /** Gets the counters. */
Counters getCounters() { return counters; } Counters getCounters() { return counters; }
/** Get the event type */ /** Gets the event type. */
public EventType getEventType() { public EventType getEventType() {
// Note that the task type can be setup/map/reduce/cleanup but the // Note that the task type can be setup/map/reduce/cleanup but the
// attempt-type can only be map/reduce. // attempt-type can only be map/reduce.

View File

@ -32,6 +32,7 @@
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
import org.apache.hadoop.yarn.util.SystemClock;
/** /**
* Event to record the failure of a task * Event to record the failure of a task
@ -49,11 +50,12 @@ public class TaskFailedEvent implements HistoryEvent {
private String status; private String status;
private String error; private String error;
private Counters counters; private Counters counters;
private long startTime;
private static final Counters EMPTY_COUNTERS = new Counters(); 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 id Task ID
* @param finishTime Finish time of the task * @param finishTime Finish time of the task
* @param taskType Type of the task * @param taskType Type of the task
@ -61,10 +63,11 @@ public class TaskFailedEvent implements HistoryEvent {
* @param status Status * @param status Status
* @param failedDueToAttempt The attempt id due to which the task failed * @param failedDueToAttempt The attempt id due to which the task failed
* @param counters Counters for the task * @param counters Counters for the task
* @param startTs task start time.
*/ */
public TaskFailedEvent(TaskID id, long finishTime, public TaskFailedEvent(TaskID id, long finishTime,
TaskType taskType, String error, String status, TaskType taskType, String error, String status,
TaskAttemptID failedDueToAttempt, Counters counters) { TaskAttemptID failedDueToAttempt, Counters counters, long startTs) {
this.id = id; this.id = id;
this.finishTime = finishTime; this.finishTime = finishTime;
this.taskType = taskType; this.taskType = taskType;
@ -72,15 +75,23 @@ public TaskFailedEvent(TaskID id, long finishTime,
this.status = status; this.status = status;
this.failedDueToAttempt = failedDueToAttempt; this.failedDueToAttempt = failedDueToAttempt;
this.counters = counters; 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, public TaskFailedEvent(TaskID id, long finishTime,
TaskType taskType, String error, String status, TaskType taskType, String error, String status,
TaskAttemptID failedDueToAttempt) { TaskAttemptID failedDueToAttempt) {
this(id, finishTime, taskType, error, status, this(id, finishTime, taskType, error, status, failedDueToAttempt,
failedDueToAttempt, EMPTY_COUNTERS); EMPTY_COUNTERS);
} }
TaskFailedEvent() {} TaskFailedEvent() {}
public Object getDatum() { public Object getDatum() {
@ -118,27 +129,37 @@ public void setDatum(Object odatum) {
EventReader.fromAvro(datum.getCounters()); EventReader.fromAvro(datum.getCounters());
} }
/** Get the task id */ /** Gets the task id. */
public TaskID getTaskId() { return id; } public TaskID getTaskId() { return id; }
/** Get the error string */ /** Gets the error string. */
public String getError() { return error; } public String getError() { return error; }
/** Get the finish time of the attempt */ /** Gets the finish time of the attempt. */
public long getFinishTime() { public long getFinishTime() {
return finishTime; 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() { public TaskType getTaskType() {
return taskType; 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() { public TaskAttemptID getFailedAttemptID() {
return failedDueToAttempt; return failedDueToAttempt;
} }
/** Get the task status */ /**
* Gets the task status.
* @return task status
*/
public String getTaskStatus() { return status; } public String getTaskStatus() { return status; }
/** Get task counters */ /** Gets task counters. */
public Counters getCounters() { return counters; } public Counters getCounters() { return counters; }
/** Get the event type */ /** Gets the event type. */
public EventType getEventType() { public EventType getEventType() {
return EventType.TASK_FAILED; return EventType.TASK_FAILED;
} }

View File

@ -32,6 +32,7 @@
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
import org.apache.hadoop.yarn.util.SystemClock;
/** /**
* Event to record the successful completion of a task * Event to record the successful completion of a task
@ -49,27 +50,36 @@ public class TaskFinishedEvent implements HistoryEvent {
private TaskType taskType; private TaskType taskType;
private String status; private String status;
private Counters counters; 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 id Task ID
* @param attemptId Task Attempt ID of the successful attempt for this task * @param attemptId Task Attempt ID of the successful attempt for this task
* @param finishTime Finish time of the task * @param finishTime Finish time of the task
* @param taskType Type of the task * @param taskType Type of the task
* @param status Status string * @param status Status string
* @param counters Counters for the task * @param counters Counters for the task
* @param startTs task start time
*/ */
public TaskFinishedEvent(TaskID id, TaskAttemptID attemptId, long finishTime, public TaskFinishedEvent(TaskID id, TaskAttemptID attemptId, long finishTime,
TaskType taskType, TaskType taskType,
String status, Counters counters) { String status, Counters counters, long startTs) {
this.taskid = id; this.taskid = id;
this.successfulAttemptId = attemptId; this.successfulAttemptId = attemptId;
this.finishTime = finishTime; this.finishTime = finishTime;
this.taskType = taskType; this.taskType = taskType;
this.status = status; this.status = status;
this.counters = counters; 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() {} TaskFinishedEvent() {}
public Object getDatum() { public Object getDatum() {
@ -101,23 +111,33 @@ public void setDatum(Object oDatum) {
this.counters = EventReader.fromAvro(datum.getCounters()); this.counters = EventReader.fromAvro(datum.getCounters());
} }
/** Get task id */ /** Gets task id. */
public TaskID getTaskId() { return taskid; } public TaskID getTaskId() { return taskid; }
/** Get successful task attempt id */ /** Gets successful task attempt id. */
public TaskAttemptID getSuccessfulTaskAttemptId() { public TaskAttemptID getSuccessfulTaskAttemptId() {
return successfulAttemptId; return successfulAttemptId;
} }
/** Get the task finish time */ /** Gets the task finish time. */
public long getFinishTime() { return finishTime; } 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; } public Counters getCounters() { return counters; }
/** Get task type */ /** Gets task type. */
public TaskType getTaskType() { public TaskType getTaskType() {
return taskType; return taskType;
} }
/** Get task status */ /**
* Gets task status.
* @return task status
*/
public String getTaskStatus() { return status.toString(); } public String getTaskStatus() { return status.toString(); }
/** Get event type */ /** Gets event type. */
public EventType getEventType() { public EventType getEventType() {
return EventType.TASK_FINISHED; return EventType.TASK_FINISHED;
} }

View File

@ -298,10 +298,10 @@ private void checkNewTimelineEvent(ApplicationId appId,
" does not exist.", " does not exist.",
jobEventFile.exists()); jobEventFile.exists());
verifyEntity(jobEventFile, EventType.JOB_FINISHED.name(), verifyEntity(jobEventFile, EventType.JOB_FINISHED.name(),
true, false, null); true, false, null, false);
Set<String> cfgsToCheck = Sets.newHashSet("dummy_conf1", "dummy_conf2", Set<String> cfgsToCheck = Sets.newHashSet("dummy_conf1", "dummy_conf2",
"huge_dummy_conf1", "huge_dummy_conf2"); "huge_dummy_conf1", "huge_dummy_conf2");
verifyEntity(jobEventFile, null, false, true, cfgsToCheck); verifyEntity(jobEventFile, null, false, true, cfgsToCheck, false);
// for this test, we expect MR job metrics are published in YARN_APPLICATION // for this test, we expect MR job metrics are published in YARN_APPLICATION
String outputAppDir = String outputAppDir =
@ -322,8 +322,8 @@ private void checkNewTimelineEvent(ApplicationId appId,
"appEventFilePath: " + appEventFilePath + "appEventFilePath: " + appEventFilePath +
" does not exist.", " does not exist.",
appEventFile.exists()); appEventFile.exists());
verifyEntity(appEventFile, null, true, false, null); verifyEntity(appEventFile, null, true, false, null, false);
verifyEntity(appEventFile, null, false, true, cfgsToCheck); verifyEntity(appEventFile, null, false, true, cfgsToCheck, false);
// check for task event file // check for task event file
String outputDirTask = String outputDirTask =
@ -344,7 +344,7 @@ private void checkNewTimelineEvent(ApplicationId appId,
" does not exist.", " does not exist.",
taskEventFile.exists()); taskEventFile.exists());
verifyEntity(taskEventFile, EventType.TASK_FINISHED.name(), verifyEntity(taskEventFile, EventType.TASK_FINISHED.name(),
true, false, null); true, false, null, true);
// check for task attempt event file // check for task attempt event file
String outputDirTaskAttempt = String outputDirTaskAttempt =
@ -363,7 +363,7 @@ private void checkNewTimelineEvent(ApplicationId appId,
Assert.assertTrue("taskAttemptEventFileName: " + taskAttemptEventFilePath + Assert.assertTrue("taskAttemptEventFileName: " + taskAttemptEventFilePath +
" does not exist.", taskAttemptEventFile.exists()); " does not exist.", taskAttemptEventFile.exists());
verifyEntity(taskAttemptEventFile, EventType.MAP_ATTEMPT_FINISHED.name(), verifyEntity(taskAttemptEventFile, EventType.MAP_ATTEMPT_FINISHED.name(),
true, false, null); true, false, null, true);
} }
/** /**
@ -380,12 +380,13 @@ private void checkNewTimelineEvent(ApplicationId appId,
* @throws IOException * @throws IOException
*/ */
private void verifyEntity(File entityFile, String eventId, private void verifyEntity(File entityFile, String eventId,
boolean chkMetrics, boolean chkCfg, Set<String> cfgsToVerify) boolean chkMetrics, boolean chkCfg, Set<String> cfgsToVerify,
throws IOException { boolean checkIdPrefix) throws IOException {
BufferedReader reader = null; BufferedReader reader = null;
String strLine; String strLine;
try { try {
reader = new BufferedReader(new FileReader(entityFile)); reader = new BufferedReader(new FileReader(entityFile));
long idPrefix = -1;
while ((strLine = reader.readLine()) != null) { while ((strLine = reader.readLine()) != null) {
if (strLine.trim().length() > 0) { if (strLine.trim().length() > 0) {
org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
@ -394,6 +395,19 @@ private void verifyEntity(File entityFile, String eventId,
strLine.trim(), strLine.trim(),
org.apache.hadoop.yarn.api.records.timelineservice. org.apache.hadoop.yarn.api.records.timelineservice.
TimelineEntity.class); 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) { if (eventId == null) {
// Job metrics are published without any events for // Job metrics are published without any events for
// ApplicationEntity. There is also possibility that some other // ApplicationEntity. There is also possibility that some other

View File

@ -104,6 +104,8 @@
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
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.hadoop.yarn.util.timeline.TimelineUtils;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
@ -313,6 +315,17 @@ public enum DSEntity {
protected final Set<ContainerId> launchedContainers = protected final Set<ContainerId> launchedContainers =
Collections.newSetFromMap(new ConcurrentHashMap<ContainerId, Boolean>()); 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 * @param args Command line args
*/ */
@ -866,7 +879,15 @@ public void onContainersCompleted(List<ContainerStatus> completedContainers) {
+ containerStatus.getContainerId()); + containerStatus.getContainerId());
} }
if (timelineServiceV2Enabled) { if (timelineServiceV2Enabled) {
publishContainerEndEventOnTimelineServiceV2(containerStatus); Long containerStartTime =
containerStartTimes.get(containerStatus.getContainerId());
if (containerStartTime == null) {
containerStartTime = SystemClock.getInstance().getTime();
containerStartTimes.put(containerStatus.getContainerId(),
containerStartTime);
}
publishContainerEndEventOnTimelineServiceV2(containerStatus,
containerStartTime);
} else if (timelineServiceV1Enabled) { } else if (timelineServiceV1Enabled) {
publishContainerEndEvent(timelineClient, containerStatus, domainId, publishContainerEndEvent(timelineClient, containerStatus, domainId,
appSubmitterUgi); appSubmitterUgi);
@ -994,8 +1015,10 @@ public void onContainerStarted(ContainerId containerId,
containerId, container.getNodeId()); containerId, container.getNodeId());
} }
if (applicationMaster.timelineServiceV2Enabled) { if (applicationMaster.timelineServiceV2Enabled) {
applicationMaster long startTime = SystemClock.getInstance().getTime();
.publishContainerStartEventOnTimelineServiceV2(container); applicationMaster.getContainerStartTimes().put(containerId, startTime);
applicationMaster.publishContainerStartEventOnTimelineServiceV2(
container, startTime);
} else if (applicationMaster.timelineServiceV1Enabled) { } else if (applicationMaster.timelineServiceV1Enabled) {
applicationMaster.publishContainerStartEvent( applicationMaster.publishContainerStartEvent(
applicationMaster.timelineClient, container, applicationMaster.timelineClient, container,
@ -1356,24 +1379,24 @@ Thread createLaunchContainerThread(Container allocatedContainer,
} }
private void publishContainerStartEventOnTimelineServiceV2( private void publishContainerStartEventOnTimelineServiceV2(
Container container) { Container container, long startTime) {
final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
entity = entity =
new org.apache.hadoop.yarn.api.records.timelineservice. new org.apache.hadoop.yarn.api.records.timelineservice.
TimelineEntity(); TimelineEntity();
entity.setId(container.getId().toString()); entity.setId(container.getId().toString());
entity.setType(DSEntity.DS_CONTAINER.toString()); entity.setType(DSEntity.DS_CONTAINER.toString());
long ts = System.currentTimeMillis(); entity.setCreatedTime(startTime);
entity.setCreatedTime(ts);
entity.addInfo("user", appSubmitterUgi.getShortUserName()); entity.addInfo("user", appSubmitterUgi.getShortUserName());
org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent event = org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent event =
new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent(); new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent();
event.setTimestamp(ts); event.setTimestamp(startTime);
event.setId(DSEvent.DS_CONTAINER_START.toString()); event.setId(DSEvent.DS_CONTAINER_START.toString());
event.addInfo("Node", container.getNodeId().toString()); event.addInfo("Node", container.getNodeId().toString());
event.addInfo("Resources", container.getResource().toString()); event.addInfo("Resources", container.getResource().toString());
entity.addEvent(event); entity.addEvent(event);
entity.setIdPrefix(TimelineServiceHelper.invertLong(startTime));
try { try {
appSubmitterUgi.doAs(new PrivilegedExceptionAction<Object>() { appSubmitterUgi.doAs(new PrivilegedExceptionAction<Object>() {
@ -1391,7 +1414,7 @@ public TimelinePutResponse run() throws Exception {
} }
private void publishContainerEndEventOnTimelineServiceV2( private void publishContainerEndEventOnTimelineServiceV2(
final ContainerStatus container) { final ContainerStatus container, long containerStartTime) {
final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
entity = entity =
new org.apache.hadoop.yarn.api.records.timelineservice. new org.apache.hadoop.yarn.api.records.timelineservice.
@ -1407,6 +1430,7 @@ private void publishContainerEndEventOnTimelineServiceV2(
event.addInfo("State", container.getState().name()); event.addInfo("State", container.getState().name());
event.addInfo("Exit Status", container.getExitStatus()); event.addInfo("Exit Status", container.getExitStatus());
entity.addEvent(event); entity.addEvent(event);
entity.setIdPrefix(TimelineServiceHelper.invertLong(containerStartTime));
try { try {
appSubmitterUgi.doAs(new PrivilegedExceptionAction<Object>() { appSubmitterUgi.doAs(new PrivilegedExceptionAction<Object>() {
@ -1441,6 +1465,8 @@ private void publishApplicationAttemptEventOnTimelineServiceV2(
event.setId(appEvent.toString()); event.setId(appEvent.toString());
event.setTimestamp(ts); event.setTimestamp(ts);
entity.addEvent(event); entity.addEvent(event);
entity.setIdPrefix(
TimelineServiceHelper.invertLong(appAttemptID.getAttemptId()));
try { try {
appSubmitterUgi.doAs(new PrivilegedExceptionAction<Object>() { appSubmitterUgi.doAs(new PrivilegedExceptionAction<Object>() {

View File

@ -64,7 +64,9 @@
import org.apache.hadoop.yarn.api.records.YarnApplicationState; 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.TimelineDomain;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities; import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType; import org.apache.hadoop.yarn.api.records.timelineservice.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.YarnClient;
import org.apache.hadoop.yarn.client.api.impl.DirectTimelineWriter; 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.TestTimelineClient;
@ -81,6 +83,7 @@
import org.apache.hadoop.yarn.server.timeline.TimelineVersion; import org.apache.hadoop.yarn.server.timeline.TimelineVersion;
import org.apache.hadoop.yarn.server.timeline.TimelineVersionWatcher; import org.apache.hadoop.yarn.server.timeline.TimelineVersionWatcher;
import org.apache.hadoop.yarn.server.timelineservice.collector.PerNodeTimelineCollectorsAuxService; 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.FileSystemTimelineWriterImpl;
import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.util.LinuxResourceCalculatorPlugin; import org.apache.hadoop.yarn.util.LinuxResourceCalculatorPlugin;
@ -523,15 +526,31 @@ private void checkTimelineV2(boolean haveDomain, ApplicationId appId,
"appattempt_" + appId.getClusterTimestamp() + "_000" + appId.getId() "appattempt_" + appId.getClusterTimestamp() + "_000" + appId.getId()
+ "_000001" + "_000001"
+ FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION; + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION;
verifyEntityTypeFileExists(basePath, "DS_APP_ATTEMPT", File dsAppAttemptEntityFile = verifyEntityTypeFileExists(basePath,
appTimestampFileName); "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 // Verify DS_CONTAINER entities posted by the client.
String containerTimestampFileName = String containerTimestampFileName =
"container_" + appId.getClusterTimestamp() + "_000" + appId.getId() "container_" + appId.getClusterTimestamp() + "_000" + appId.getId()
+ "_01_000002.thist"; + "_01_000002.thist";
verifyEntityTypeFileExists(basePath, "DS_CONTAINER", File dsContainerEntityFile = verifyEntityTypeFileExists(basePath,
containerTimestampFileName); "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. // Verify NM posting container metrics info.
String containerMetricsTimestampFileName = String containerMetricsTimestampFileName =
@ -541,29 +560,13 @@ private void checkTimelineV2(boolean haveDomain, ApplicationId appId,
File containerEntityFile = verifyEntityTypeFileExists(basePath, File containerEntityFile = verifyEntityTypeFileExists(basePath,
TimelineEntityType.YARN_CONTAINER.toString(), TimelineEntityType.YARN_CONTAINER.toString(),
containerMetricsTimestampFileName); containerMetricsTimestampFileName);
Assert.assertEquals( verifyEntityForTimelineV2(containerEntityFile,
"Container created event needs to be published atleast once", ContainerMetricsConstants.CREATED_EVENT_TYPE, 1, 1, 0, true);
1,
getNumOfStringOccurrences(containerEntityFile,
ContainerMetricsConstants.CREATED_EVENT_TYPE));
// to avoid race condition of testcase, atleast check 4 times with sleep // to avoid race condition of testcase, atleast check 40 times with sleep
// of 500ms // of 50ms
long numOfContainerFinishedOccurrences = 0; verifyEntityForTimelineV2(containerEntityFile,
for (int i = 0; i < 4; i++) { ContainerMetricsConstants.FINISHED_EVENT_TYPE, 1, 40, 50, true);
numOfContainerFinishedOccurrences =
getNumOfStringOccurrences(containerEntityFile,
ContainerMetricsConstants.FINISHED_EVENT_TYPE);
if (numOfContainerFinishedOccurrences > 0) {
break;
} else {
Thread.sleep(500L);
}
}
Assert.assertEquals(
"Container finished event needs to be published atleast once",
1,
numOfContainerFinishedOccurrences);
// Verify RM posting Application life cycle Events are getting published // Verify RM posting Application life cycle Events are getting published
String appMetricsTimestampFileName = String appMetricsTimestampFileName =
@ -573,29 +576,14 @@ private void checkTimelineV2(boolean haveDomain, ApplicationId appId,
verifyEntityTypeFileExists(basePath, verifyEntityTypeFileExists(basePath,
TimelineEntityType.YARN_APPLICATION.toString(), TimelineEntityType.YARN_APPLICATION.toString(),
appMetricsTimestampFileName); appMetricsTimestampFileName);
Assert.assertEquals( // No need to check idprefix for app.
"Application created event should be published atleast once", verifyEntityForTimelineV2(appEntityFile,
1, ApplicationMetricsConstants.CREATED_EVENT_TYPE, 1, 1, 0, false);
getNumOfStringOccurrences(appEntityFile,
ApplicationMetricsConstants.CREATED_EVENT_TYPE));
// to avoid race condition of testcase, atleast check 4 times with sleep // to avoid race condition of testcase, atleast check 40 times with sleep
// of 500ms // of 50ms
long numOfStringOccurrences = 0; verifyEntityForTimelineV2(appEntityFile,
for (int i = 0; i < 4; i++) { ApplicationMetricsConstants.FINISHED_EVENT_TYPE, 1, 40, 50, false);
numOfStringOccurrences =
getNumOfStringOccurrences(appEntityFile,
ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
if (numOfStringOccurrences > 0) {
break;
} else {
Thread.sleep(500L);
}
}
Assert.assertEquals(
"Application finished event should be published atleast once",
1,
numOfStringOccurrences);
// Verify RM posting AppAttempt life cycle Events are getting published // Verify RM posting AppAttempt life cycle Events are getting published
String appAttemptMetricsTimestampFileName = String appAttemptMetricsTimestampFileName =
@ -606,17 +594,10 @@ private void checkTimelineV2(boolean haveDomain, ApplicationId appId,
verifyEntityTypeFileExists(basePath, verifyEntityTypeFileExists(basePath,
TimelineEntityType.YARN_APPLICATION_ATTEMPT.toString(), TimelineEntityType.YARN_APPLICATION_ATTEMPT.toString(),
appAttemptMetricsTimestampFileName); appAttemptMetricsTimestampFileName);
Assert.assertEquals( verifyEntityForTimelineV2(appAttemptEntityFile,
"AppAttempt register event should be published atleast once", AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE, 1, 1, 0, true);
1, verifyEntityForTimelineV2(appAttemptEntityFile,
getNumOfStringOccurrences(appAttemptEntityFile, AppAttemptMetricsConstants.FINISHED_EVENT_TYPE, 1, 1, 0, true);
AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE));
Assert.assertEquals(
"AppAttempt finished event should be published atleast once",
1,
getNumOfStringOccurrences(appAttemptEntityFile,
AppAttemptMetricsConstants.FINISHED_EVENT_TYPE));
} finally { } finally {
FileUtils.deleteDirectory(tmpRootFolder.getParentFile()); FileUtils.deleteDirectory(tmpRootFolder.getParentFile());
} }
@ -636,22 +617,64 @@ private File verifyEntityTypeFileExists(String basePath, String entityType,
return entityFile; return entityFile;
} }
private long getNumOfStringOccurrences(File entityFile, String searchString) /**
throws IOException { * Checks the events and idprefix published for an entity.
BufferedReader reader = null; *
String strLine; * @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; long actualCount = 0;
try { for (int i = 0; i < checkTimes; i++) {
reader = new BufferedReader(new FileReader(entityFile)); BufferedReader reader = null;
while ((strLine = reader.readLine()) != null) { String strLine = null;
if (strLine.trim().contains(searchString)) { actualCount = 0;
actualCount++; 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);
} }
} finally {
reader.close();
} }
return actualCount; Assert.assertEquals("Unexpected number of " + expectedEvent +
" event published.", numOfExpectedEvent, actualCount);
} }
/** /**

View File

@ -155,6 +155,7 @@
import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher; import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher;
import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.server.utils.YarnServerSecurityUtils; import org.apache.hadoop.yarn.server.utils.YarnServerSecurityUtils;
import org.apache.hadoop.yarn.util.SystemClock;
import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.resource.Resources;
import org.apache.hadoop.yarn.util.timeline.TimelineUtils; import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
@ -1052,10 +1053,11 @@ protected void startContainerInternal(
Credentials credentials = Credentials credentials =
YarnServerSecurityUtils.parseCredentials(launchContext); YarnServerSecurityUtils.parseCredentials(launchContext);
long containerStartTime = SystemClock.getInstance().getTime();
Container container = Container container =
new ContainerImpl(getConfig(), this.dispatcher, new ContainerImpl(getConfig(), this.dispatcher,
launchContext, credentials, metrics, containerTokenIdentifier, launchContext, credentials, metrics, containerTokenIdentifier,
context); context, containerStartTime);
ApplicationId applicationID = ApplicationId applicationID =
containerId.getApplicationAttemptId().getApplicationId(); containerId.getApplicationAttemptId().getApplicationId();
if (context.getContainers().putIfAbsent(containerId, container) != null) { if (context.getContainers().putIfAbsent(containerId, container) != null) {
@ -1112,7 +1114,7 @@ protected void startContainerInternal(
} }
this.context.getNMStateStore().storeContainer(containerId, this.context.getNMStateStore().storeContainer(containerId,
containerTokenIdentifier.getVersion(), request); containerTokenIdentifier.getVersion(), containerStartTime, request);
dispatcher.getEventHandler().handle( dispatcher.getEventHandler().handle(
new ApplicationContainerInitEvent(container)); new ApplicationContainerInitEvent(container));

View File

@ -23,12 +23,16 @@
public class ApplicationContainerFinishedEvent extends ApplicationEvent { public class ApplicationContainerFinishedEvent extends ApplicationEvent {
private ContainerStatus containerStatus; private ContainerStatus containerStatus;
// Required by NMTimelinePublisher.
private long containerStartTime;
public ApplicationContainerFinishedEvent(ContainerStatus containerStatus) { public ApplicationContainerFinishedEvent(ContainerStatus containerStatus,
long containerStartTs) {
super(containerStatus.getContainerId().getApplicationAttemptId(). super(containerStatus.getContainerId().getApplicationAttemptId().
getApplicationId(), getApplicationId(),
ApplicationEventType.APPLICATION_CONTAINER_FINISHED); ApplicationEventType.APPLICATION_CONTAINER_FINISHED);
this.containerStatus = containerStatus; this.containerStatus = containerStatus;
this.containerStartTime = containerStartTs;
} }
public ContainerId getContainerID() { public ContainerId getContainerID() {
@ -39,4 +43,7 @@ public ContainerStatus getContainerStatus() {
return containerStatus; return containerStatus;
} }
public long getContainerStartTime() {
return containerStartTime;
}
} }

View File

@ -37,6 +37,8 @@ public interface Container extends EventHandler<ContainerEvent> {
ContainerId getContainerId(); ContainerId getContainerId();
long getContainerStartTime();
Resource getResource(); Resource getResource();
ContainerTokenIdentifier getContainerTokenIdentifier(); ContainerTokenIdentifier getContainerTokenIdentifier();

View File

@ -173,11 +173,11 @@ private ReInitializationContext createContextForRollback() {
/** The NM-wide configuration - not specific to this container */ /** The NM-wide configuration - not specific to this container */
private final Configuration daemonConf; private final Configuration daemonConf;
private final long startTime;
private static final Logger LOG = private static final Logger LOG =
LoggerFactory.getLogger(ContainerImpl.class); LoggerFactory.getLogger(ContainerImpl.class);
// whether container has been recovered after a restart // whether container has been recovered after a restart
private RecoveredContainerStatus recoveredStatus = private RecoveredContainerStatus recoveredStatus =
RecoveredContainerStatus.REQUESTED; RecoveredContainerStatus.REQUESTED;
@ -190,6 +190,16 @@ public ContainerImpl(Configuration conf, Dispatcher dispatcher,
ContainerLaunchContext launchContext, Credentials creds, ContainerLaunchContext launchContext, Credentials creds,
NodeManagerMetrics metrics, NodeManagerMetrics metrics,
ContainerTokenIdentifier containerTokenIdentifier, Context context) { ContainerTokenIdentifier containerTokenIdentifier, Context context) {
this(conf, dispatcher, launchContext, creds, metrics,
containerTokenIdentifier, context, SystemClock.getInstance().getTime());
}
public ContainerImpl(Configuration conf, Dispatcher dispatcher,
ContainerLaunchContext launchContext, Credentials creds,
NodeManagerMetrics metrics,
ContainerTokenIdentifier containerTokenIdentifier, Context context,
long startTs) {
this.startTime = startTs;
this.daemonConf = conf; this.daemonConf = conf;
this.dispatcher = dispatcher; this.dispatcher = dispatcher;
this.stateStore = context.getNMStateStore(); this.stateStore = context.getNMStateStore();
@ -263,7 +273,7 @@ public ContainerImpl(Configuration conf, Dispatcher dispatcher,
ContainerTokenIdentifier containerTokenIdentifier, Context context, ContainerTokenIdentifier containerTokenIdentifier, Context context,
RecoveredContainerState rcs) { RecoveredContainerState rcs) {
this(conf, dispatcher, launchContext, creds, metrics, this(conf, dispatcher, launchContext, creds, metrics,
containerTokenIdentifier, context); containerTokenIdentifier, context, rcs.getStartTime());
this.recoveredStatus = rcs.getStatus(); this.recoveredStatus = rcs.getStatus();
this.exitCode = rcs.getExitCode(); this.exitCode = rcs.getExitCode();
this.recoveredAsKilled = rcs.getKilled(); this.recoveredAsKilled = rcs.getKilled();
@ -630,6 +640,11 @@ public ContainerId getContainerId() {
return this.containerId; return this.containerId;
} }
@Override
public long getContainerStartTime() {
return this.startTime;
}
@Override @Override
public Resource getResource() { public Resource getResource() {
return Resources.clone( return Resources.clone(
@ -694,7 +709,8 @@ private void sendFinishedEvents() {
EventHandler eventHandler = dispatcher.getEventHandler(); EventHandler eventHandler = dispatcher.getEventHandler();
ContainerStatus containerStatus = cloneAndGetContainerStatus(); ContainerStatus containerStatus = cloneAndGetContainerStatus();
eventHandler.handle(new ApplicationContainerFinishedEvent(containerStatus)); eventHandler.handle(
new ApplicationContainerFinishedEvent(containerStatus, startTime));
// Tell the scheduler the container is Done // Tell the scheduler the container is Done
eventHandler.handle(new ContainerSchedulerEvent(this, eventHandler.handle(new ContainerSchedulerEvent(this,

View File

@ -112,6 +112,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
"ContainerManager/containers/"; "ContainerManager/containers/";
private static final String CONTAINER_REQUEST_KEY_SUFFIX = "/request"; private static final String CONTAINER_REQUEST_KEY_SUFFIX = "/request";
private static final String CONTAINER_VERSION_KEY_SUFFIX = "/version"; private static final String CONTAINER_VERSION_KEY_SUFFIX = "/version";
private static final String CONTAINER_START_TIME_KEY_SUFFIX = "/starttime";
private static final String CONTAINER_DIAGS_KEY_SUFFIX = "/diagnostics"; private static final String CONTAINER_DIAGS_KEY_SUFFIX = "/diagnostics";
private static final String CONTAINER_LAUNCHED_KEY_SUFFIX = "/launched"; private static final String CONTAINER_LAUNCHED_KEY_SUFFIX = "/launched";
private static final String CONTAINER_QUEUED_KEY_SUFFIX = "/queued"; private static final String CONTAINER_QUEUED_KEY_SUFFIX = "/queued";
@ -257,6 +258,8 @@ private RecoveredContainerState loadContainerState(ContainerId containerId,
StartContainerRequestProto.parseFrom(entry.getValue())); StartContainerRequestProto.parseFrom(entry.getValue()));
} else if (suffix.equals(CONTAINER_VERSION_KEY_SUFFIX)) { } else if (suffix.equals(CONTAINER_VERSION_KEY_SUFFIX)) {
rcs.version = Integer.parseInt(asString(entry.getValue())); rcs.version = Integer.parseInt(asString(entry.getValue()));
} else if (suffix.equals(CONTAINER_START_TIME_KEY_SUFFIX)) {
rcs.setStartTime(Long.parseLong(asString(entry.getValue())));
} else if (suffix.equals(CONTAINER_DIAGS_KEY_SUFFIX)) { } else if (suffix.equals(CONTAINER_DIAGS_KEY_SUFFIX)) {
rcs.diagnostics = asString(entry.getValue()); rcs.diagnostics = asString(entry.getValue());
} else if (suffix.equals(CONTAINER_QUEUED_KEY_SUFFIX)) { } else if (suffix.equals(CONTAINER_QUEUED_KEY_SUFFIX)) {
@ -296,21 +299,23 @@ private RecoveredContainerState loadContainerState(ContainerId containerId,
@Override @Override
public void storeContainer(ContainerId containerId, int containerVersion, public void storeContainer(ContainerId containerId, int containerVersion,
StartContainerRequest startRequest) throws IOException { long startTime, StartContainerRequest startRequest) throws IOException {
String idStr = containerId.toString(); String idStr = containerId.toString();
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("storeContainer: containerId= " + idStr LOG.debug("storeContainer: containerId= " + idStr
+ ", startRequest= " + startRequest); + ", startRequest= " + startRequest);
} }
String keyRequest = CONTAINERS_KEY_PREFIX + idStr String keyRequest = getContainerKey(idStr, CONTAINER_REQUEST_KEY_SUFFIX);
+ CONTAINER_REQUEST_KEY_SUFFIX;
String keyVersion = getContainerVersionKey(idStr); String keyVersion = getContainerVersionKey(idStr);
String keyStartTime =
getContainerKey(idStr, CONTAINER_START_TIME_KEY_SUFFIX);
try { try {
WriteBatch batch = db.createWriteBatch(); WriteBatch batch = db.createWriteBatch();
try { try {
batch.put(bytes(keyRequest), batch.put(bytes(keyRequest),
((StartContainerRequestPBImpl) startRequest) ((StartContainerRequestPBImpl) startRequest).getProto().
.getProto().toByteArray()); toByteArray());
batch.put(bytes(keyStartTime), bytes(Long.toString(startTime)));
if (containerVersion != 0) { if (containerVersion != 0) {
batch.put(bytes(keyVersion), batch.put(bytes(keyVersion),
bytes(Integer.toString(containerVersion))); bytes(Integer.toString(containerVersion)));
@ -326,7 +331,11 @@ public void storeContainer(ContainerId containerId, int containerVersion,
@VisibleForTesting @VisibleForTesting
String getContainerVersionKey(String containerId) { String getContainerVersionKey(String containerId) {
return CONTAINERS_KEY_PREFIX + containerId + CONTAINER_VERSION_KEY_SUFFIX; return getContainerKey(containerId, CONTAINER_VERSION_KEY_SUFFIX);
}
private String getContainerKey(String containerId, String suffix) {
return CONTAINERS_KEY_PREFIX + containerId + suffix;
} }
@Override @Override

View File

@ -71,7 +71,7 @@ public List<RecoveredContainerState> loadContainersState()
@Override @Override
public void storeContainer(ContainerId containerId, int version, public void storeContainer(ContainerId containerId, int version,
StartContainerRequest startRequest) throws IOException { long startTime, StartContainerRequest startRequest) throws IOException {
} }
@Override @Override

View File

@ -87,6 +87,7 @@ public static class RecoveredContainerState {
int version; int version;
private RecoveredContainerType recoveryType = private RecoveredContainerType recoveryType =
RecoveredContainerType.RECOVER; RecoveredContainerType.RECOVER;
private long startTime;
public RecoveredContainerStatus getStatus() { public RecoveredContainerStatus getStatus() {
return status; return status;
@ -108,6 +109,14 @@ public int getVersion() {
return version; return version;
} }
public long getStartTime() {
return startTime;
}
public void setStartTime(long ts) {
startTime = ts;
}
public StartContainerRequest getStartRequest() { public StartContainerRequest getStartRequest() {
return startRequest; return startRequest;
} }
@ -145,6 +154,7 @@ public String toString() {
return new StringBuffer("Status: ").append(getStatus()) return new StringBuffer("Status: ").append(getStatus())
.append(", Exit code: ").append(exitCode) .append(", Exit code: ").append(exitCode)
.append(", Version: ").append(version) .append(", Version: ").append(version)
.append(", Start Time: ").append(startTime)
.append(", Killed: ").append(getKilled()) .append(", Killed: ").append(getKilled())
.append(", Diagnostics: ").append(getDiagnostics()) .append(", Diagnostics: ").append(getDiagnostics())
.append(", Capability: ").append(getCapability()) .append(", Capability: ").append(getCapability())
@ -365,11 +375,12 @@ public abstract List<RecoveredContainerState> loadContainersState()
* Record a container start request * Record a container start request
* @param containerId the container ID * @param containerId the container ID
* @param containerVersion the container Version * @param containerVersion the container Version
* @param startTime container start time
* @param startRequest the container start request * @param startRequest the container start request
* @throws IOException * @throws IOException
*/ */
public abstract void storeContainer(ContainerId containerId, public abstract void storeContainer(ContainerId containerId,
int containerVersion, StartContainerRequest startRequest) int containerVersion, long startTime, StartContainerRequest startRequest)
throws IOException; throws IOException;
/** /**

View File

@ -55,6 +55,7 @@
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEvent;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl.ContainerMetric; import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl.ContainerMetric;
import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree; import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
import org.apache.hadoop.yarn.util.TimelineServiceHelper;
import org.apache.hadoop.yarn.util.timeline.TimelineUtils; import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
@ -149,6 +150,8 @@ public void reportContainerResourceUsage(Container container, Long pmemUsage,
Math.round(cpuUsagePercentPerCore)); Math.round(cpuUsagePercentPerCore));
entity.addMetric(cpuMetric); entity.addMetric(cpuMetric);
} }
entity.setIdPrefix(TimelineServiceHelper.
invertLong(container.getContainerStartTime()));
ApplicationId appId = container.getContainerId().getApplicationAttemptId() ApplicationId appId = container.getContainerId().getApplicationAttemptId()
.getApplicationId(); .getApplicationId();
try { try {
@ -195,15 +198,17 @@ private void publishContainerCreatedEvent(ContainerEvent event) {
tEvent.setId(ContainerMetricsConstants.CREATED_EVENT_TYPE); tEvent.setId(ContainerMetricsConstants.CREATED_EVENT_TYPE);
tEvent.setTimestamp(event.getTimestamp()); tEvent.setTimestamp(event.getTimestamp());
long containerStartTime = container.getContainerStartTime();
entity.addEvent(tEvent); entity.addEvent(tEvent);
entity.setCreatedTime(event.getTimestamp()); entity.setCreatedTime(containerStartTime);
entity.setIdPrefix(TimelineServiceHelper.invertLong(containerStartTime));
dispatcher.getEventHandler().handle(new TimelinePublishEvent(entity, dispatcher.getEventHandler().handle(new TimelinePublishEvent(entity,
containerId.getApplicationAttemptId().getApplicationId())); containerId.getApplicationAttemptId().getApplicationId()));
} }
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
private void publishContainerFinishedEvent(ContainerStatus containerStatus, private void publishContainerFinishedEvent(ContainerStatus containerStatus,
long timeStamp) { long containerFinishTime, long containerStartTime) {
ContainerId containerId = containerStatus.getContainerId(); ContainerId containerId = containerStatus.getContainerId();
TimelineEntity entity = createContainerEntity(containerId); TimelineEntity entity = createContainerEntity(containerId);
@ -215,13 +220,14 @@ private void publishContainerFinishedEvent(ContainerStatus containerStatus,
entityInfo.put(ContainerMetricsConstants.STATE_INFO, entityInfo.put(ContainerMetricsConstants.STATE_INFO,
ContainerState.COMPLETE.toString()); ContainerState.COMPLETE.toString());
entityInfo.put(ContainerMetricsConstants.CONTAINER_FINISHED_TIME, entityInfo.put(ContainerMetricsConstants.CONTAINER_FINISHED_TIME,
timeStamp); containerFinishTime);
entity.setInfo(entityInfo); entity.setInfo(entityInfo);
TimelineEvent tEvent = new TimelineEvent(); TimelineEvent tEvent = new TimelineEvent();
tEvent.setId(ContainerMetricsConstants.FINISHED_EVENT_TYPE); tEvent.setId(ContainerMetricsConstants.FINISHED_EVENT_TYPE);
tEvent.setTimestamp(timeStamp); tEvent.setTimestamp(containerFinishTime);
entity.addEvent(tEvent); entity.addEvent(tEvent);
entity.setIdPrefix(TimelineServiceHelper.invertLong(containerStartTime));
dispatcher.getEventHandler().handle(new TimelinePublishEvent(entity, dispatcher.getEventHandler().handle(new TimelinePublishEvent(entity,
containerId.getApplicationAttemptId().getApplicationId())); containerId.getApplicationAttemptId().getApplicationId()));
@ -237,6 +243,8 @@ private void publishContainerLocalizationEvent(
tEvent.setId(eventType); tEvent.setId(eventType);
tEvent.setTimestamp(event.getTimestamp()); tEvent.setTimestamp(event.getTimestamp());
entity.addEvent(tEvent); entity.addEvent(tEvent);
entity.setIdPrefix(TimelineServiceHelper.
invertLong(container.getContainerStartTime()));
ApplicationId appId = ApplicationId appId =
container.getContainerId().getApplicationAttemptId().getApplicationId(); container.getContainerId().getApplicationAttemptId().getApplicationId();
@ -300,7 +308,7 @@ public void publishApplicationEvent(ApplicationEvent event) {
ApplicationContainerFinishedEvent evnt = ApplicationContainerFinishedEvent evnt =
(ApplicationContainerFinishedEvent) event; (ApplicationContainerFinishedEvent) event;
publishContainerFinishedEvent(evnt.getContainerStatus(), publishContainerFinishedEvent(evnt.getContainerStatus(),
event.getTimestamp()); event.getTimestamp(), evnt.getContainerStartTime());
break; break;
default: default:

View File

@ -601,7 +601,7 @@ public void initContainer(int containerNum) {
public void containerFinished(int containerNum) { public void containerFinished(int containerNum) {
app.handle(new ApplicationContainerFinishedEvent(containers.get( app.handle(new ApplicationContainerFinishedEvent(containers.get(
containerNum).cloneAndGetContainerStatus())); containerNum).cloneAndGetContainerStatus(), 0));
drainDispatcherEvents(); drainDispatcherEvents();
} }

View File

@ -126,10 +126,12 @@ public synchronized List<RecoveredContainerState> loadContainersState()
@Override @Override
public synchronized void storeContainer(ContainerId containerId, public synchronized void storeContainer(ContainerId containerId,
int version, StartContainerRequest startRequest) throws IOException { int version, long startTime, StartContainerRequest startRequest)
throws IOException {
RecoveredContainerState rcs = new RecoveredContainerState(); RecoveredContainerState rcs = new RecoveredContainerState();
rcs.startRequest = startRequest; rcs.startRequest = startRequest;
rcs.version = version; rcs.version = version;
rcs.setStartTime(startTime);
containerStates.put(containerId, rcs); containerStates.put(containerId, rcs);
} }

View File

@ -234,7 +234,8 @@ public void testContainerStorage() throws IOException {
StartContainerRequest containerReq = createContainerRequest(containerId); StartContainerRequest containerReq = createContainerRequest(containerId);
// store a container and verify recovered // store a container and verify recovered
stateStore.storeContainer(containerId, 0, containerReq); long containerStartTime = System.currentTimeMillis();
stateStore.storeContainer(containerId, 0, containerStartTime, containerReq);
// verify the container version key is not stored for new containers // verify the container version key is not stored for new containers
DB db = stateStore.getDB(); DB db = stateStore.getDB();
@ -246,6 +247,7 @@ public void testContainerStorage() throws IOException {
assertEquals(1, recoveredContainers.size()); assertEquals(1, recoveredContainers.size());
RecoveredContainerState rcs = recoveredContainers.get(0); RecoveredContainerState rcs = recoveredContainers.get(0);
assertEquals(0, rcs.getVersion()); assertEquals(0, rcs.getVersion());
assertEquals(containerStartTime, rcs.getStartTime());
assertEquals(RecoveredContainerStatus.REQUESTED, rcs.getStatus()); assertEquals(RecoveredContainerStatus.REQUESTED, rcs.getStatus());
assertEquals(ContainerExitStatus.INVALID, rcs.getExitCode()); assertEquals(ContainerExitStatus.INVALID, rcs.getExitCode());
assertEquals(false, rcs.getKilled()); assertEquals(false, rcs.getKilled());
@ -998,7 +1000,7 @@ public void testUnexpectedKeyDoesntThrowException() throws IOException {
StartContainerRequest containerReq = StartContainerRequest.newInstance(clc, StartContainerRequest containerReq = StartContainerRequest.newInstance(clc,
containerToken); containerToken);
stateStore.storeContainer(containerId, 0, containerReq); stateStore.storeContainer(containerId, 0, 0, containerReq);
// add a invalid key // add a invalid key
byte[] invalidKey = ("ContainerManager/containers/" byte[] invalidKey = ("ContainerManager/containers/"

View File

@ -235,4 +235,8 @@ public void sendKillEvent(int exitStatus, String description) {
public boolean isRecovering() { public boolean isRecovering() {
return false; return false;
} }
public long getContainerStartTime() {
return 0;
}
} }

View File

@ -218,8 +218,8 @@ public boolean isPmemCheckEnabled() {
Context context = mock(Context.class); Context context = mock(Context.class);
Container container = Container container =
new ContainerImpl(conf, dispatcher, launchContext, new ContainerImpl(conf, dispatcher, launchContext,
null, metrics, null, metrics, BuilderUtils.newContainerTokenIdentifier(
BuilderUtils.newContainerTokenIdentifier(containerToken), context) { containerToken), context) {
@Override @Override
public ContainerState getContainerState() { public ContainerState getContainerState() {

View File

@ -526,13 +526,6 @@ public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request)
message); message);
} }
boolean timelineV2Enabled =
YarnConfiguration.timelineServiceV2Enabled(getConfig());
if (timelineV2Enabled) {
// Check & update collectors info from request.
updateAppCollectorsMap(request);
}
// Evaluate whether a DECOMMISSIONING node is ready to be DECOMMISSIONED. // Evaluate whether a DECOMMISSIONING node is ready to be DECOMMISSIONED.
if (rmNode.getState() == NodeState.DECOMMISSIONING && if (rmNode.getState() == NodeState.DECOMMISSIONING &&
decommissioningWatcher.checkReadyToBeDecommissioned( decommissioningWatcher.checkReadyToBeDecommissioned(
@ -547,6 +540,13 @@ public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request)
NodeAction.SHUTDOWN, message); NodeAction.SHUTDOWN, message);
} }
boolean timelineV2Enabled =
YarnConfiguration.timelineServiceV2Enabled(getConfig());
if (timelineV2Enabled) {
// Check & update collectors info from request.
updateAppCollectorsMap(request);
}
// Heartbeat response // Heartbeat response
NodeHeartbeatResponse nodeHeartBeatResponse = YarnServerBuilderUtils NodeHeartbeatResponse nodeHeartBeatResponse = YarnServerBuilderUtils
.newNodeHeartbeatResponse(lastNodeHeartbeatResponse. .newNodeHeartbeatResponse(lastNodeHeartbeatResponse.

View File

@ -58,6 +58,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager; import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager;
import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollector; import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollector;
import org.apache.hadoop.yarn.util.TimelineServiceHelper;
import org.apache.hadoop.yarn.util.timeline.TimelineUtils; import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
@ -294,8 +295,8 @@ private static ApplicationEntity createApplicationEntity(
@Override @Override
public void appAttemptRegistered(RMAppAttempt appAttempt, public void appAttemptRegistered(RMAppAttempt appAttempt,
long registeredTime) { long registeredTime) {
TimelineEntity entity = ApplicationAttemptId attemptId = appAttempt.getAppAttemptId();
createAppAttemptEntity(appAttempt.getAppAttemptId()); TimelineEntity entity = createAppAttemptEntity(attemptId);
entity.setCreatedTime(registeredTime); entity.setCreatedTime(registeredTime);
TimelineEvent tEvent = new TimelineEvent(); TimelineEvent tEvent = new TimelineEvent();
@ -317,6 +318,8 @@ public void appAttemptRegistered(RMAppAttempt appAttempt,
appAttempt.getMasterContainer().getId().toString()); appAttempt.getMasterContainer().getId().toString());
} }
entity.setInfo(entityInfo); entity.setInfo(entityInfo);
entity.setIdPrefix(
TimelineServiceHelper.invertLong(attemptId.getAttemptId()));
getDispatcher().getEventHandler().handle( getDispatcher().getEventHandler().handle(
new TimelineV2PublishEvent(SystemMetricsEventType.PUBLISH_ENTITY, new TimelineV2PublishEvent(SystemMetricsEventType.PUBLISH_ENTITY,
@ -327,7 +330,7 @@ public void appAttemptRegistered(RMAppAttempt appAttempt,
@Override @Override
public void appAttemptFinished(RMAppAttempt appAttempt, public void appAttemptFinished(RMAppAttempt appAttempt,
RMAppAttemptState appAttemtpState, RMApp app, long finishedTime) { RMAppAttemptState appAttemtpState, RMApp app, long finishedTime) {
ApplicationAttemptId attemptId = appAttempt.getAppAttemptId();
ApplicationAttemptEntity entity = ApplicationAttemptEntity entity =
createAppAttemptEntity(appAttempt.getAppAttemptId()); createAppAttemptEntity(appAttempt.getAppAttemptId());
@ -346,7 +349,8 @@ public void appAttemptFinished(RMAppAttempt appAttempt,
entityInfo.put(AppAttemptMetricsConstants.STATE_INFO, RMServerUtils entityInfo.put(AppAttemptMetricsConstants.STATE_INFO, RMServerUtils
.createApplicationAttemptState(appAttemtpState).toString()); .createApplicationAttemptState(appAttemtpState).toString());
entity.setInfo(entityInfo); entity.setInfo(entityInfo);
entity.setIdPrefix(
TimelineServiceHelper.invertLong(attemptId.getAttemptId()));
getDispatcher().getEventHandler().handle( getDispatcher().getEventHandler().handle(
new TimelineV2PublishEvent(SystemMetricsEventType.PUBLISH_ENTITY, new TimelineV2PublishEvent(SystemMetricsEventType.PUBLISH_ENTITY,

View File

@ -69,6 +69,7 @@
import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineReaderImpl; 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.FileSystemTimelineWriterImpl;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter; import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
import org.apache.hadoop.yarn.util.TimelineServiceHelper;
import org.apache.hadoop.yarn.util.timeline.TimelineUtils; import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Assert; import org.junit.Assert;
@ -216,7 +217,8 @@ public void testPublishApplicationMetrics() throws Exception {
+ FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION; + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION;
File appFile = new File(outputDirApp, timelineServiceFileName); File appFile = new File(outputDirApp, timelineServiceFileName);
Assert.assertTrue(appFile.exists()); Assert.assertTrue(appFile.exists());
verifyEntity(appFile, 3, ApplicationMetricsConstants.CREATED_EVENT_TYPE, 8); verifyEntity(
appFile, 3, ApplicationMetricsConstants.CREATED_EVENT_TYPE, 8, 0);
} }
@Test(timeout = 10000) @Test(timeout = 10000)
@ -251,7 +253,7 @@ public void testPublishAppAttemptMetrics() throws Exception {
File appFile = new File(outputDirApp, timelineServiceFileName); File appFile = new File(outputDirApp, timelineServiceFileName);
Assert.assertTrue(appFile.exists()); Assert.assertTrue(appFile.exists());
verifyEntity(appFile, 2, AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE, verifyEntity(appFile, 2, AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE,
0); 0, TimelineServiceHelper.invertLong(appAttemptId.getAttemptId()));
} }
@Test(timeout = 10000) @Test(timeout = 10000)
@ -283,7 +285,7 @@ public void testPublishContainerMetrics() throws Exception {
File appFile = new File(outputDirApp, timelineServiceFileName); File appFile = new File(outputDirApp, timelineServiceFileName);
Assert.assertTrue(appFile.exists()); Assert.assertTrue(appFile.exists());
verifyEntity(appFile, 2, verifyEntity(appFile, 2,
ContainerMetricsConstants.CREATED_IN_RM_EVENT_TYPE, 0); ContainerMetricsConstants.CREATED_IN_RM_EVENT_TYPE, 0, 0);
} }
private RMApp createAppAndRegister(ApplicationId appId) { private RMApp createAppAndRegister(ApplicationId appId) {
@ -297,7 +299,8 @@ private RMApp createAppAndRegister(ApplicationId appId) {
} }
private static void verifyEntity(File entityFile, long expectedEvents, private static void verifyEntity(File entityFile, long expectedEvents,
String eventForCreatedTime, long expectedMetrics) throws IOException { String eventForCreatedTime, long expectedMetrics, long idPrefix)
throws IOException {
BufferedReader reader = null; BufferedReader reader = null;
String strLine; String strLine;
long count = 0; long count = 0;
@ -309,6 +312,7 @@ private static void verifyEntity(File entityFile, long expectedEvents,
TimelineEntity entity = FileSystemTimelineReaderImpl. TimelineEntity entity = FileSystemTimelineReaderImpl.
getTimelineRecordFromJSON(strLine.trim(), TimelineEntity.class); getTimelineRecordFromJSON(strLine.trim(), TimelineEntity.class);
metricsCount = entity.getMetrics().size(); metricsCount = entity.getMetrics().size();
assertEquals(idPrefix, entity.getIdPrefix());
for (TimelineEvent event : entity.getEvents()) { for (TimelineEvent event : entity.getEvents()) {
if (event.getId().equals(eventForCreatedTime)) { if (event.getId().equals(eventForCreatedTime)) {
assertTrue(entity.getCreatedTime() > 0); assertTrue(entity.getCreatedTime() > 0);
@ -394,6 +398,7 @@ private static RMAppAttempt createRMAppAttempt(
when(appAttempt.getTrackingUrl()).thenReturn("test tracking url"); when(appAttempt.getTrackingUrl()).thenReturn("test tracking url");
when(appAttempt.getOriginalTrackingUrl()).thenReturn( when(appAttempt.getOriginalTrackingUrl()).thenReturn(
"test original tracking url"); "test original tracking url");
when(appAttempt.getStartTime()).thenReturn(200L);
return appAttempt; return appAttempt;
} }