YARN-5792. Adopt the id prefix for YARN, MR, and DS entities. Contributed by Varun Saxena.
This commit is contained in:
parent
c92a7ab31c
commit
092fead5d9
@ -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 {
|
||||||
|
@ -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));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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)));
|
||||||
|
@ -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());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -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())));
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
@ -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.
|
||||||
|
@ -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.
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
@ -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
|
||||||
|
@ -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>() {
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -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));
|
||||||
|
|
||||||
|
@ -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;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -37,6 +37,8 @@ public interface Container extends EventHandler<ContainerEvent> {
|
|||||||
|
|
||||||
ContainerId getContainerId();
|
ContainerId getContainerId();
|
||||||
|
|
||||||
|
long getContainerStartTime();
|
||||||
|
|
||||||
Resource getResource();
|
Resource getResource();
|
||||||
|
|
||||||
ContainerTokenIdentifier getContainerTokenIdentifier();
|
ContainerTokenIdentifier getContainerTokenIdentifier();
|
||||||
|
@ -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,
|
||||||
|
@ -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
|
||||||
|
@ -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
|
||||||
|
@ -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;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -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:
|
||||||
|
@ -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();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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/"
|
||||||
|
@ -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;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -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() {
|
||||||
|
@ -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.
|
||||||
|
@ -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,
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user