YARN-5355: YARN Timeline Service v.2: alpha 2 (varunsaxena)
This commit is contained in:
commit
32cba6c303
@ -223,4 +223,15 @@
|
|||||||
group list is separated by a blank. For e.g. "alice,bob users,wheel".
|
group list is separated by a blank. For e.g. "alice,bob users,wheel".
|
||||||
A special value of "*" means all users are allowed.</description>
|
A special value of "*" means all users are allowed.</description>
|
||||||
</property>
|
</property>
|
||||||
|
|
||||||
|
<property>
|
||||||
|
<name>security.collector-nodemanager.protocol.acl</name>
|
||||||
|
<value>*</value>
|
||||||
|
<description>ACL for CollectorNodemanagerProtocol, used by nodemanager
|
||||||
|
if timeline service v2 is enabled, for the timeline collector and nodemanager
|
||||||
|
to communicate with each other.
|
||||||
|
The ACL is a comma-separated list of user and group names. The user and
|
||||||
|
group list is separated by a blank. For e.g. "alice,bob users,wheel".
|
||||||
|
A special value of "*" means all users are allowed.</description>
|
||||||
|
</property>
|
||||||
</configuration>
|
</configuration>
|
||||||
|
@ -78,6 +78,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|||||||
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 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 +1125,7 @@ public class JobHistoryEventHandler extends AbstractService
|
|||||||
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 +1134,7 @@ public class JobHistoryEventHandler extends AbstractService
|
|||||||
((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 +1143,12 @@ public class JobHistoryEventHandler extends AbstractService
|
|||||||
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 +1199,8 @@ public class JobHistoryEventHandler extends AbstractService
|
|||||||
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 +1223,21 @@ public class JobHistoryEventHandler extends AbstractService
|
|||||||
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 +1245,8 @@ public class JobHistoryEventHandler extends AbstractService
|
|||||||
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 +1266,22 @@ public class JobHistoryEventHandler extends AbstractService
|
|||||||
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 +1310,12 @@ public class JobHistoryEventHandler extends AbstractService
|
|||||||
// 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 @@ public abstract class TaskAttemptImpl implements
|
|||||||
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 @@ public abstract class TaskAttemptImpl implements
|
|||||||
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 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|||||||
}
|
}
|
||||||
|
|
||||||
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 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|||||||
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 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|||||||
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 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|||||||
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)));
|
||||||
|
@ -848,7 +848,8 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|||||||
updateAMRMToken(response.getAMRMToken());
|
updateAMRMToken(response.getAMRMToken());
|
||||||
}
|
}
|
||||||
|
|
||||||
List<ContainerStatus> finishedContainers = response.getCompletedContainersStatuses();
|
List<ContainerStatus> finishedContainers =
|
||||||
|
response.getCompletedContainersStatuses();
|
||||||
|
|
||||||
// propagate preemption requests
|
// propagate preemption requests
|
||||||
final PreemptionMessage preemptReq = response.getPreemptionMessage();
|
final PreemptionMessage preemptReq = response.getPreemptionMessage();
|
||||||
@ -877,16 +878,13 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|||||||
|
|
||||||
handleUpdatedNodes(response);
|
handleUpdatedNodes(response);
|
||||||
handleJobPriorityChange(response);
|
handleJobPriorityChange(response);
|
||||||
// handle receiving the timeline collector address for this app
|
// Handle receiving the timeline collector address and token for this app.
|
||||||
String collectorAddr = response.getCollectorAddr();
|
|
||||||
MRAppMaster.RunningAppContext appContext =
|
MRAppMaster.RunningAppContext appContext =
|
||||||
(MRAppMaster.RunningAppContext)this.getContext();
|
(MRAppMaster.RunningAppContext)this.getContext();
|
||||||
if (collectorAddr != null && !collectorAddr.isEmpty()
|
if (appContext.getTimelineV2Client() != null) {
|
||||||
&& appContext.getTimelineV2Client() != null) {
|
appContext.getTimelineV2Client().
|
||||||
appContext.getTimelineV2Client().setTimelineServiceAddress(
|
setTimelineCollectorInfo(response.getCollectorInfo());
|
||||||
response.getCollectorAddr());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
for (ContainerStatus cont : finishedContainers) {
|
for (ContainerStatus cont : finishedContainers) {
|
||||||
processFinishedContainer(cont);
|
processFinishedContainer(cont);
|
||||||
}
|
}
|
||||||
|
@ -58,7 +58,7 @@ public class TestEvents {
|
|||||||
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 class TestEvents {
|
|||||||
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 class TestJobHistoryEventHandler {
|
|||||||
|
|
||||||
// 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 class TestJobHistoryEventHandler {
|
|||||||
|
|
||||||
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 class TestJobHistoryEventHandler {
|
|||||||
|
|
||||||
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 class TestJobHistoryEventHandler {
|
|||||||
|
|
||||||
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())));
|
||||||
|
@ -18,6 +18,7 @@
|
|||||||
|
|
||||||
package org.apache.hadoop.mapreduce.v2.app.rm;
|
package org.apache.hadoop.mapreduce.v2.app.rm;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
import static org.mockito.Matchers.any;
|
import static org.mockito.Matchers.any;
|
||||||
import static org.mockito.Matchers.anyFloat;
|
import static org.mockito.Matchers.anyFloat;
|
||||||
import static org.mockito.Matchers.anyInt;
|
import static org.mockito.Matchers.anyInt;
|
||||||
@ -27,6 +28,7 @@ import static org.mockito.Mockito.doReturn;
|
|||||||
import static org.mockito.Mockito.inOrder;
|
import static org.mockito.Mockito.inOrder;
|
||||||
import static org.mockito.Mockito.mock;
|
import static org.mockito.Mockito.mock;
|
||||||
import static org.mockito.Mockito.never;
|
import static org.mockito.Mockito.never;
|
||||||
|
import static org.mockito.Mockito.spy;
|
||||||
import static org.mockito.Mockito.times;
|
import static org.mockito.Mockito.times;
|
||||||
import static org.mockito.Mockito.verify;
|
import static org.mockito.Mockito.verify;
|
||||||
import static org.mockito.Mockito.when;
|
import static org.mockito.Mockito.when;
|
||||||
@ -99,6 +101,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRespo
|
|||||||
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
|
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
|
import org.apache.hadoop.yarn.api.records.CollectorInfo;
|
||||||
import org.apache.hadoop.yarn.api.records.Container;
|
import org.apache.hadoop.yarn.api.records.Container;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
|
import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
@ -110,6 +113,7 @@ import org.apache.hadoop.yarn.api.records.NodeReport;
|
|||||||
import org.apache.hadoop.yarn.api.records.Priority;
|
import org.apache.hadoop.yarn.api.records.Priority;
|
||||||
import org.apache.hadoop.yarn.api.records.Resource;
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||||
|
import org.apache.hadoop.yarn.client.api.TimelineV2Client;
|
||||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||||
import org.apache.hadoop.yarn.event.Dispatcher;
|
import org.apache.hadoop.yarn.event.Dispatcher;
|
||||||
import org.apache.hadoop.yarn.event.DrainDispatcher;
|
import org.apache.hadoop.yarn.event.DrainDispatcher;
|
||||||
@ -121,6 +125,7 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
|
|||||||
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
|
import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
|
||||||
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
|
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
|
||||||
|
import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
|
||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
|
||||||
import org.apache.hadoop.yarn.server.api.records.NodeAction;
|
import org.apache.hadoop.yarn.server.api.records.NodeAction;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
|
import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
|
||||||
@ -140,6 +145,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretMan
|
|||||||
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
||||||
import org.apache.hadoop.yarn.util.Clock;
|
import org.apache.hadoop.yarn.util.Clock;
|
||||||
import org.apache.hadoop.yarn.util.ControlledClock;
|
import org.apache.hadoop.yarn.util.ControlledClock;
|
||||||
|
import org.apache.hadoop.yarn.util.Records;
|
||||||
import org.apache.hadoop.yarn.util.SystemClock;
|
import org.apache.hadoop.yarn.util.SystemClock;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
@ -748,6 +754,96 @@ public class TestRMContainerAllocator {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testUpdateCollectorInfo() throws Exception {
|
||||||
|
LOG.info("Running testUpdateCollectorInfo");
|
||||||
|
Configuration conf = new Configuration();
|
||||||
|
conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
|
||||||
|
conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
|
||||||
|
ApplicationId appId = ApplicationId.newInstance(1, 1);
|
||||||
|
ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1);
|
||||||
|
JobId jobId = MRBuilderUtils.newJobId(appId, 0);
|
||||||
|
Job mockJob = mock(Job.class);
|
||||||
|
when(mockJob.getReport()).thenReturn(
|
||||||
|
MRBuilderUtils.newJobReport(jobId, "job", "user", JobState.RUNNING, 0,
|
||||||
|
0, 0, 0, 0, 0, 0, "jobfile", null, false, ""));
|
||||||
|
String localAddr = "localhost:1234";
|
||||||
|
UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
|
||||||
|
|
||||||
|
// Generate a timeline delegation token.
|
||||||
|
TimelineDelegationTokenIdentifier ident =
|
||||||
|
new TimelineDelegationTokenIdentifier(new Text(ugi.getUserName()),
|
||||||
|
new Text("renewer"), null);
|
||||||
|
ident.setSequenceNumber(1);
|
||||||
|
Token<TimelineDelegationTokenIdentifier> collectorToken =
|
||||||
|
new Token<TimelineDelegationTokenIdentifier>(ident.getBytes(),
|
||||||
|
new byte[0], TimelineDelegationTokenIdentifier.KIND_NAME,
|
||||||
|
new Text(localAddr));
|
||||||
|
org.apache.hadoop.yarn.api.records.Token token =
|
||||||
|
org.apache.hadoop.yarn.api.records.Token.newInstance(
|
||||||
|
collectorToken.getIdentifier(), collectorToken.getKind().toString(),
|
||||||
|
collectorToken.getPassword(),
|
||||||
|
collectorToken.getService().toString());
|
||||||
|
CollectorInfo collectorInfo = CollectorInfo.newInstance(localAddr, token);
|
||||||
|
// Mock scheduler to server Allocate request.
|
||||||
|
final MockSchedulerForTimelineCollector mockScheduler =
|
||||||
|
new MockSchedulerForTimelineCollector(collectorInfo);
|
||||||
|
MyContainerAllocator allocator =
|
||||||
|
new MyContainerAllocator(null, conf, attemptId, mockJob,
|
||||||
|
SystemClock.getInstance()) {
|
||||||
|
@Override
|
||||||
|
protected void register() {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected ApplicationMasterProtocol createSchedulerProxy() {
|
||||||
|
return mockScheduler;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
// Initially UGI should have no tokens.
|
||||||
|
ArrayList<Token<? extends TokenIdentifier>> tokens =
|
||||||
|
new ArrayList<>(ugi.getTokens());
|
||||||
|
assertEquals(0, tokens.size());
|
||||||
|
TimelineV2Client client = spy(TimelineV2Client.createTimelineClient(appId));
|
||||||
|
client.init(conf);
|
||||||
|
when(((RunningAppContext)allocator.getContext()).getTimelineV2Client()).
|
||||||
|
thenReturn(client);
|
||||||
|
|
||||||
|
// Send allocate request to RM and fetch collector address and token.
|
||||||
|
allocator.schedule();
|
||||||
|
verify(client).setTimelineCollectorInfo(collectorInfo);
|
||||||
|
// Verify if token has been updated in UGI.
|
||||||
|
tokens = new ArrayList<>(ugi.getTokens());
|
||||||
|
assertEquals(1, tokens.size());
|
||||||
|
assertEquals(TimelineDelegationTokenIdentifier.KIND_NAME,
|
||||||
|
tokens.get(0).getKind());
|
||||||
|
assertEquals(collectorToken.decodeIdentifier(),
|
||||||
|
tokens.get(0).decodeIdentifier());
|
||||||
|
|
||||||
|
// Generate new collector token, send allocate request to RM and fetch the
|
||||||
|
// new token.
|
||||||
|
ident.setSequenceNumber(100);
|
||||||
|
Token<TimelineDelegationTokenIdentifier> collectorToken1 =
|
||||||
|
new Token<TimelineDelegationTokenIdentifier>(ident.getBytes(),
|
||||||
|
new byte[0], TimelineDelegationTokenIdentifier.KIND_NAME,
|
||||||
|
new Text(localAddr));
|
||||||
|
token = org.apache.hadoop.yarn.api.records.Token.newInstance(
|
||||||
|
collectorToken1.getIdentifier(), collectorToken1.getKind().toString(),
|
||||||
|
collectorToken1.getPassword(), collectorToken1.getService().toString());
|
||||||
|
collectorInfo = CollectorInfo.newInstance(localAddr, token);
|
||||||
|
mockScheduler.updateCollectorInfo(collectorInfo);
|
||||||
|
allocator.schedule();
|
||||||
|
verify(client).setTimelineCollectorInfo(collectorInfo);
|
||||||
|
// Verify if new token has been updated in UGI.
|
||||||
|
tokens = new ArrayList<>(ugi.getTokens());
|
||||||
|
assertEquals(1, tokens.size());
|
||||||
|
assertEquals(TimelineDelegationTokenIdentifier.KIND_NAME,
|
||||||
|
tokens.get(0).getKind());
|
||||||
|
assertEquals(collectorToken1.decodeIdentifier(),
|
||||||
|
tokens.get(0).decodeIdentifier());
|
||||||
|
allocator.close();
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testMapReduceScheduling() throws Exception {
|
public void testMapReduceScheduling() throws Exception {
|
||||||
|
|
||||||
@ -3488,6 +3584,46 @@ public class TestRMContainerAllocator {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private final static class MockSchedulerForTimelineCollector
|
||||||
|
implements ApplicationMasterProtocol {
|
||||||
|
private CollectorInfo collectorInfo;
|
||||||
|
|
||||||
|
private MockSchedulerForTimelineCollector(CollectorInfo info) {
|
||||||
|
this.collectorInfo = info;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void updateCollectorInfo(CollectorInfo info) {
|
||||||
|
collectorInfo = info;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RegisterApplicationMasterResponse registerApplicationMaster(
|
||||||
|
RegisterApplicationMasterRequest request) throws YarnException,
|
||||||
|
IOException {
|
||||||
|
return Records.newRecord(RegisterApplicationMasterResponse.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public FinishApplicationMasterResponse finishApplicationMaster(
|
||||||
|
FinishApplicationMasterRequest request) throws YarnException,
|
||||||
|
IOException {
|
||||||
|
return FinishApplicationMasterResponse.newInstance(false);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public AllocateResponse allocate(AllocateRequest request)
|
||||||
|
throws YarnException, IOException {
|
||||||
|
AllocateResponse response = AllocateResponse.newInstance(
|
||||||
|
request.getResponseId(), Collections.<ContainerStatus>emptyList(),
|
||||||
|
Collections.<Container>emptyList(),
|
||||||
|
Collections.<NodeReport>emptyList(),
|
||||||
|
Resource.newInstance(512000, 1024), null, 10, null,
|
||||||
|
Collections.<NMToken>emptyList());
|
||||||
|
response.setCollectorInfo(collectorInfo);
|
||||||
|
return response;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public static void main(String[] args) throws Exception {
|
public static void main(String[] args) throws Exception {
|
||||||
TestRMContainerAllocator t = new TestRMContainerAllocator();
|
TestRMContainerAllocator t = new TestRMContainerAllocator();
|
||||||
t.testSimple();
|
t.testSimple();
|
||||||
|
@ -32,9 +32,10 @@ import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils;
|
|||||||
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 class MapAttemptFinishedEvent implements HistoryEvent {
|
|||||||
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.mapreduce.util.JobHistoryEventUtils;
|
|||||||
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 class ReduceAttemptFinishedEvent implements HistoryEvent {
|
|||||||
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.mapreduce.util.JobHistoryEventUtils;
|
|||||||
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 class TaskAttemptFinishedEvent implements HistoryEvent {
|
|||||||
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 class TaskAttemptFinishedEvent implements HistoryEvent {
|
|||||||
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.mapreduce.util.JobHistoryEventUtils;
|
|||||||
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 class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
|
|||||||
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.mapreduce.util.JobHistoryEventUtils;
|
|||||||
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 class TaskFailedEvent implements HistoryEvent {
|
|||||||
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 class TaskFailedEvent implements HistoryEvent {
|
|||||||
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.mapreduce.util.JobHistoryEventUtils;
|
|||||||
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 class TaskFinishedEvent implements HistoryEvent {
|
|||||||
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 @@ public class TestMRTimelineEventHandling {
|
|||||||
" 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 @@ public class TestMRTimelineEventHandling {
|
|||||||
"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 @@ public class TestMRTimelineEventHandling {
|
|||||||
" 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 @@ public class TestMRTimelineEventHandling {
|
|||||||
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 @@ public class TestMRTimelineEventHandling {
|
|||||||
* @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 @@ public class TestMRTimelineEventHandling {
|
|||||||
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
|
||||||
|
@ -37,7 +37,6 @@ import org.apache.hadoop.security.UserGroupInformation;
|
|||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
|
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
|
||||||
import org.apache.hadoop.yarn.client.api.TimelineClient;
|
import org.apache.hadoop.yarn.client.api.TimelineClient;
|
||||||
import org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl;
|
|
||||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||||
|
|
||||||
|
|
||||||
@ -54,7 +53,7 @@ class JobHistoryFileReplayMapperV1 extends
|
|||||||
|
|
||||||
public void map(IntWritable key, IntWritable val, Context context) throws IOException {
|
public void map(IntWritable key, IntWritable val, Context context) throws IOException {
|
||||||
// collect the apps it needs to process
|
// collect the apps it needs to process
|
||||||
TimelineClient tlc = new TimelineClientImpl();
|
TimelineClient tlc = TimelineClient.createTimelineClient();
|
||||||
TimelineEntityConverterV1 converter = new TimelineEntityConverterV1();
|
TimelineEntityConverterV1 converter = new TimelineEntityConverterV1();
|
||||||
JobHistoryFileReplayHelper helper = new JobHistoryFileReplayHelper(context);
|
JobHistoryFileReplayHelper helper = new JobHistoryFileReplayHelper(context);
|
||||||
int replayMode = helper.getReplayMode();
|
int replayMode = helper.getReplayMode();
|
||||||
|
@ -32,7 +32,6 @@ import org.apache.hadoop.security.UserGroupInformation;
|
|||||||
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
|
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
|
||||||
import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
|
import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
|
||||||
import org.apache.hadoop.yarn.client.api.TimelineClient;
|
import org.apache.hadoop.yarn.client.api.TimelineClient;
|
||||||
import org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Adds simple entities with random string payload, events, metrics, and
|
* Adds simple entities with random string payload, events, metrics, and
|
||||||
@ -46,7 +45,7 @@ class SimpleEntityWriterV1
|
|||||||
|
|
||||||
public void map(IntWritable key, IntWritable val, Context context)
|
public void map(IntWritable key, IntWritable val, Context context)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
TimelineClient tlc = new TimelineClientImpl();
|
TimelineClient tlc = TimelineClient.createTimelineClient();
|
||||||
Configuration conf = context.getConfiguration();
|
Configuration conf = context.getConfiguration();
|
||||||
|
|
||||||
final int kbs = conf.getInt(KBS_SENT, KBS_SENT_DEFAULT);
|
final int kbs = conf.getInt(KBS_SENT, KBS_SENT_DEFAULT);
|
||||||
|
@ -55,17 +55,15 @@ documentation.
|
|||||||
YARN Timeline Service v.2
|
YARN Timeline Service v.2
|
||||||
-------------------
|
-------------------
|
||||||
|
|
||||||
We are introducing an early preview (alpha 1) of a major revision of YARN
|
We are introducing an early preview (alpha 2) of a major revision of YARN
|
||||||
Timeline Service: v.2. YARN Timeline Service v.2 addresses two major
|
Timeline Service: v.2. YARN Timeline Service v.2 addresses two major
|
||||||
challenges: improving scalability and reliability of Timeline Service, and
|
challenges: improving scalability and reliability of Timeline Service, and
|
||||||
enhancing usability by introducing flows and aggregation.
|
enhancing usability by introducing flows and aggregation.
|
||||||
|
|
||||||
YARN Timeline Service v.2 alpha 1 is provided so that users and developers
|
YARN Timeline Service v.2 alpha 2 is provided so that users and developers
|
||||||
can test it and provide feedback and suggestions for making it a ready
|
can test it and provide feedback and suggestions for making it a ready
|
||||||
replacement for Timeline Service v.1.x. It should be used only in a test
|
replacement for Timeline Service v.1.x. It should be used only in a test
|
||||||
capacity. Most importantly, security is not enabled. Do not set up or use
|
capacity.
|
||||||
Timeline Service v.2 until security is implemented if security is a
|
|
||||||
critical requirement.
|
|
||||||
|
|
||||||
More details are available in the
|
More details are available in the
|
||||||
[YARN Timeline Service v.2](./hadoop-yarn/hadoop-yarn-site/TimelineServiceV2.html)
|
[YARN Timeline Service v.2](./hadoop-yarn/hadoop-yarn-site/TimelineServiceV2.html)
|
||||||
|
@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceStability.Evolving;
|
|||||||
import org.apache.hadoop.classification.InterfaceStability.Stable;
|
import org.apache.hadoop.classification.InterfaceStability.Stable;
|
||||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||||
import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
|
import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
|
||||||
|
import org.apache.hadoop.yarn.api.records.CollectorInfo;
|
||||||
import org.apache.hadoop.yarn.api.records.AMCommand;
|
import org.apache.hadoop.yarn.api.records.AMCommand;
|
||||||
import org.apache.hadoop.yarn.api.records.Container;
|
import org.apache.hadoop.yarn.api.records.Container;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
||||||
@ -92,21 +93,21 @@ public abstract class AllocateResponse {
|
|||||||
.preemptionMessage(preempt).nmTokens(nmTokens).build();
|
.preemptionMessage(preempt).nmTokens(nmTokens).build();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Public
|
@Private
|
||||||
@Unstable
|
@Unstable
|
||||||
public static AllocateResponse newInstance(int responseId,
|
public static AllocateResponse newInstance(int responseId,
|
||||||
List<ContainerStatus> completedContainers,
|
List<ContainerStatus> completedContainers,
|
||||||
List<Container> allocatedContainers, List<NodeReport> updatedNodes,
|
List<Container> allocatedContainers, List<NodeReport> updatedNodes,
|
||||||
Resource availResources, AMCommand command, int numClusterNodes,
|
Resource availResources, AMCommand command, int numClusterNodes,
|
||||||
PreemptionMessage preempt, List<NMToken> nmTokens,
|
PreemptionMessage preempt, List<NMToken> nmTokens,
|
||||||
List<UpdatedContainer> updatedContainers) {
|
CollectorInfo collectorInfo) {
|
||||||
return AllocateResponse.newBuilder().numClusterNodes(numClusterNodes)
|
return AllocateResponse.newBuilder().numClusterNodes(numClusterNodes)
|
||||||
.responseId(responseId)
|
.responseId(responseId)
|
||||||
.completedContainersStatuses(completedContainers)
|
.completedContainersStatuses(completedContainers)
|
||||||
.allocatedContainers(allocatedContainers).updatedNodes(updatedNodes)
|
.allocatedContainers(allocatedContainers).updatedNodes(updatedNodes)
|
||||||
.availableResources(availResources).amCommand(command)
|
.availableResources(availResources).amCommand(command)
|
||||||
.preemptionMessage(preempt).nmTokens(nmTokens)
|
.preemptionMessage(preempt).nmTokens(nmTokens)
|
||||||
.updatedContainers(updatedContainers).build();
|
.collectorInfo(collectorInfo).build();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Private
|
@Private
|
||||||
@ -133,7 +134,7 @@ public abstract class AllocateResponse {
|
|||||||
List<Container> allocatedContainers, List<NodeReport> updatedNodes,
|
List<Container> allocatedContainers, List<NodeReport> updatedNodes,
|
||||||
Resource availResources, AMCommand command, int numClusterNodes,
|
Resource availResources, AMCommand command, int numClusterNodes,
|
||||||
PreemptionMessage preempt, List<NMToken> nmTokens, Token amRMToken,
|
PreemptionMessage preempt, List<NMToken> nmTokens, Token amRMToken,
|
||||||
List<UpdatedContainer> updatedContainers, String collectorAddr) {
|
List<UpdatedContainer> updatedContainers, CollectorInfo collectorInfo) {
|
||||||
return AllocateResponse.newBuilder().numClusterNodes(numClusterNodes)
|
return AllocateResponse.newBuilder().numClusterNodes(numClusterNodes)
|
||||||
.responseId(responseId)
|
.responseId(responseId)
|
||||||
.completedContainersStatuses(completedContainers)
|
.completedContainersStatuses(completedContainers)
|
||||||
@ -141,7 +142,7 @@ public abstract class AllocateResponse {
|
|||||||
.availableResources(availResources).amCommand(command)
|
.availableResources(availResources).amCommand(command)
|
||||||
.preemptionMessage(preempt).nmTokens(nmTokens)
|
.preemptionMessage(preempt).nmTokens(nmTokens)
|
||||||
.updatedContainers(updatedContainers).amRmToken(amRMToken)
|
.updatedContainers(updatedContainers).amRmToken(amRMToken)
|
||||||
.collectorAddr(collectorAddr).build();
|
.collectorInfo(collectorInfo).build();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -333,17 +334,18 @@ public abstract class AllocateResponse {
|
|||||||
public abstract void setApplicationPriority(Priority priority);
|
public abstract void setApplicationPriority(Priority priority);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The address of collector that belong to this app
|
* The data associated with the collector that belongs to this app. Contains
|
||||||
|
* address and token alongwith identification information.
|
||||||
*
|
*
|
||||||
* @return The address of collector that belong to this attempt
|
* @return The data of collector that belong to this attempt
|
||||||
*/
|
*/
|
||||||
@Public
|
@Public
|
||||||
@Unstable
|
@Unstable
|
||||||
public abstract String getCollectorAddr();
|
public abstract CollectorInfo getCollectorInfo();
|
||||||
|
|
||||||
@Private
|
@Private
|
||||||
@Unstable
|
@Unstable
|
||||||
public abstract void setCollectorAddr(String collectorAddr);
|
public abstract void setCollectorInfo(CollectorInfo info);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get the list of container update errors to inform the
|
* Get the list of container update errors to inform the
|
||||||
@ -559,15 +561,17 @@ public abstract class AllocateResponse {
|
|||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Set the <code>collectorAddr</code> of the response.
|
* Set the <code>collectorInfo</code> of the response.
|
||||||
* @see AllocateResponse#setCollectorAddr(String)
|
* @see AllocateResponse#setCollectorInfo(CollectorInfo)
|
||||||
* @param collectorAddr <code>collectorAddr</code> of the response
|
* @param collectorInfo <code>collectorInfo</code> of the response which
|
||||||
|
* contains collector address, RM id, version and collector token.
|
||||||
* @return {@link AllocateResponseBuilder}
|
* @return {@link AllocateResponseBuilder}
|
||||||
*/
|
*/
|
||||||
@Private
|
@Private
|
||||||
@Unstable
|
@Unstable
|
||||||
public AllocateResponseBuilder collectorAddr(String collectorAddr) {
|
public AllocateResponseBuilder collectorInfo(
|
||||||
allocateResponse.setCollectorAddr(collectorAddr);
|
CollectorInfo collectorInfo) {
|
||||||
|
allocateResponse.setCollectorInfo(collectorInfo);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -16,31 +16,44 @@
|
|||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package org.apache.hadoop.yarn.server.api.records;
|
package org.apache.hadoop.yarn.api.records;
|
||||||
|
|
||||||
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
||||||
import org.apache.hadoop.yarn.util.Records;
|
import org.apache.hadoop.yarn.util.Records;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Collector info containing collector address and collector token passed from
|
||||||
|
* RM to AM in Allocate Response.
|
||||||
|
*/
|
||||||
@Private
|
@Private
|
||||||
public abstract class AppCollectorsMap {
|
@InterfaceStability.Unstable
|
||||||
|
public abstract class CollectorInfo {
|
||||||
|
|
||||||
public static AppCollectorsMap newInstance(
|
protected static final long DEFAULT_TIMESTAMP_VALUE = -1;
|
||||||
ApplicationId id, String collectorAddr) {
|
|
||||||
AppCollectorsMap appCollectorsMap =
|
public static CollectorInfo newInstance(String collectorAddr) {
|
||||||
Records.newRecord(AppCollectorsMap.class);
|
return newInstance(collectorAddr, null);
|
||||||
appCollectorsMap.setApplicationId(id);
|
|
||||||
appCollectorsMap.setCollectorAddr(collectorAddr);
|
|
||||||
return appCollectorsMap;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public abstract ApplicationId getApplicationId();
|
public static CollectorInfo newInstance(String collectorAddr, Token token) {
|
||||||
|
CollectorInfo amCollectorInfo =
|
||||||
public abstract void setApplicationId(ApplicationId id);
|
Records.newRecord(CollectorInfo.class);
|
||||||
|
amCollectorInfo.setCollectorAddr(collectorAddr);
|
||||||
|
amCollectorInfo.setCollectorToken(token);
|
||||||
|
return amCollectorInfo;
|
||||||
|
}
|
||||||
|
|
||||||
public abstract String getCollectorAddr();
|
public abstract String getCollectorAddr();
|
||||||
|
|
||||||
public abstract void setCollectorAddr(String addr);
|
public abstract void setCollectorAddr(String addr);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get delegation token for app collector which AM will use to publish
|
||||||
|
* entities.
|
||||||
|
* @return the delegation token for app collector.
|
||||||
|
*/
|
||||||
|
public abstract Token getCollectorToken();
|
||||||
|
|
||||||
|
public abstract void setCollectorToken(Token token);
|
||||||
}
|
}
|
@ -49,4 +49,32 @@ public class ApplicationEntity extends HierarchicalTimelineEntity {
|
|||||||
public void setQueue(String queue) {
|
public void setQueue(String queue) {
|
||||||
addInfo(QUEUE_INFO_KEY, queue);
|
addInfo(QUEUE_INFO_KEY, queue);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Checks if the input TimelineEntity object is an ApplicationEntity.
|
||||||
|
*
|
||||||
|
* @param te TimelineEntity object.
|
||||||
|
* @return true if input is an ApplicationEntity, false otherwise
|
||||||
|
*/
|
||||||
|
public static boolean isApplicationEntity(TimelineEntity te) {
|
||||||
|
return (te == null ? false
|
||||||
|
: te.getType().equals(TimelineEntityType.YARN_APPLICATION.toString()));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param te TimelineEntity object.
|
||||||
|
* @param eventId event with this id needs to be fetched
|
||||||
|
* @return TimelineEvent if TimelineEntity contains the desired event.
|
||||||
|
*/
|
||||||
|
public static TimelineEvent getApplicationEvent(TimelineEntity te,
|
||||||
|
String eventId) {
|
||||||
|
if (isApplicationEntity(te)) {
|
||||||
|
for (TimelineEvent event : te.getEvents()) {
|
||||||
|
if (event.getId().equals(eventId)) {
|
||||||
|
return event;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -55,6 +55,7 @@ import com.fasterxml.jackson.annotation.JsonSetter;
|
|||||||
@InterfaceStability.Unstable
|
@InterfaceStability.Unstable
|
||||||
public class TimelineEntity implements Comparable<TimelineEntity> {
|
public class TimelineEntity implements Comparable<TimelineEntity> {
|
||||||
protected final static String SYSTEM_INFO_KEY_PREFIX = "SYSTEM_INFO_";
|
protected final static String SYSTEM_INFO_KEY_PREFIX = "SYSTEM_INFO_";
|
||||||
|
public final static long DEFAULT_ENTITY_PREFIX = 0L;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Identifier of timeline entity(entity id + entity type).
|
* Identifier of timeline entity(entity id + entity type).
|
||||||
@ -146,6 +147,7 @@ public class TimelineEntity implements Comparable<TimelineEntity> {
|
|||||||
private HashMap<String, Set<String>> isRelatedToEntities = new HashMap<>();
|
private HashMap<String, Set<String>> isRelatedToEntities = new HashMap<>();
|
||||||
private HashMap<String, Set<String>> relatesToEntities = new HashMap<>();
|
private HashMap<String, Set<String>> relatesToEntities = new HashMap<>();
|
||||||
private Long createdTime;
|
private Long createdTime;
|
||||||
|
private long idPrefix;
|
||||||
|
|
||||||
public TimelineEntity() {
|
public TimelineEntity() {
|
||||||
identifier = new Identifier();
|
identifier = new Identifier();
|
||||||
@ -548,20 +550,10 @@ public class TimelineEntity implements Comparable<TimelineEntity> {
|
|||||||
public int compareTo(TimelineEntity other) {
|
public int compareTo(TimelineEntity other) {
|
||||||
int comparison = getType().compareTo(other.getType());
|
int comparison = getType().compareTo(other.getType());
|
||||||
if (comparison == 0) {
|
if (comparison == 0) {
|
||||||
if (getCreatedTime() == null) {
|
if (getIdPrefix() > other.getIdPrefix()) {
|
||||||
if (other.getCreatedTime() == null) {
|
// Descending order by entity id prefix
|
||||||
return getId().compareTo(other.getId());
|
|
||||||
} else {
|
|
||||||
return 1;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
if (other.getCreatedTime() == null) {
|
|
||||||
return -1;
|
return -1;
|
||||||
}
|
} else if (getIdPrefix() < other.getIdPrefix()) {
|
||||||
if (getCreatedTime() > other.getCreatedTime()) {
|
|
||||||
// Order by created time desc
|
|
||||||
return -1;
|
|
||||||
} else if (getCreatedTime() < other.getCreatedTime()) {
|
|
||||||
return 1;
|
return 1;
|
||||||
} else {
|
} else {
|
||||||
return getId().compareTo(other.getId());
|
return getId().compareTo(other.getId());
|
||||||
@ -582,4 +574,38 @@ public class TimelineEntity implements Comparable<TimelineEntity> {
|
|||||||
return real.toString();
|
return real.toString();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@XmlElement(name = "idprefix")
|
||||||
|
public long getIdPrefix() {
|
||||||
|
if (real == null) {
|
||||||
|
return idPrefix;
|
||||||
|
} else {
|
||||||
|
return real.getIdPrefix();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets idPrefix for an entity.
|
||||||
|
* <p>
|
||||||
|
* <b>Note</b>: Entities will be stored in the order of idPrefix specified.
|
||||||
|
* If users decide to set idPrefix for an entity, they <b>MUST</b> provide
|
||||||
|
* the same prefix for every update of this entity.
|
||||||
|
* </p>
|
||||||
|
* Example: <blockquote><pre>
|
||||||
|
* TimelineEntity entity = new TimelineEntity();
|
||||||
|
* entity.setIdPrefix(value);
|
||||||
|
* </pre></blockquote>
|
||||||
|
* Users can use {@link TimelineServiceHelper#invertLong(long)} to invert
|
||||||
|
* the prefix if necessary.
|
||||||
|
*
|
||||||
|
* @param entityIdPrefix prefix for an entity.
|
||||||
|
*/
|
||||||
|
@JsonSetter("idprefix")
|
||||||
|
public void setIdPrefix(long entityIdPrefix) {
|
||||||
|
if (real == null) {
|
||||||
|
this.idPrefix = entityIdPrefix;
|
||||||
|
} else {
|
||||||
|
real.setIdPrefix(entityIdPrefix);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
@ -1782,6 +1782,10 @@ public class YarnConfiguration extends Configuration {
|
|||||||
YARN_SECURITY_SERVICE_AUTHORIZATION_APPLICATIONHISTORY_PROTOCOL =
|
YARN_SECURITY_SERVICE_AUTHORIZATION_APPLICATIONHISTORY_PROTOCOL =
|
||||||
"security.applicationhistory.protocol.acl";
|
"security.applicationhistory.protocol.acl";
|
||||||
|
|
||||||
|
public static final String
|
||||||
|
YARN_SECURITY_SERVICE_AUTHORIZATION_COLLECTOR_NODEMANAGER_PROTOCOL =
|
||||||
|
"security.collector-nodemanager.protocol.acl";
|
||||||
|
|
||||||
/** No. of milliseconds to wait between sending a SIGTERM and SIGKILL
|
/** No. of milliseconds to wait between sending a SIGTERM and SIGKILL
|
||||||
* to a running container */
|
* to a running container */
|
||||||
public static final String NM_SLEEP_DELAY_BEFORE_SIGKILL_MS =
|
public static final String NM_SLEEP_DELAY_BEFORE_SIGKILL_MS =
|
||||||
@ -2099,7 +2103,7 @@ public class YarnConfiguration extends Configuration {
|
|||||||
TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_PREFIX + "with-user-dir";
|
TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_PREFIX + "with-user-dir";
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Settings for timeline service v2.0
|
* Settings for timeline service v2.0.
|
||||||
*/
|
*/
|
||||||
public static final String TIMELINE_SERVICE_WRITER_CLASS =
|
public static final String TIMELINE_SERVICE_WRITER_CLASS =
|
||||||
TIMELINE_SERVICE_PREFIX + "writer.class";
|
TIMELINE_SERVICE_PREFIX + "writer.class";
|
||||||
@ -2112,9 +2116,20 @@ public class YarnConfiguration extends Configuration {
|
|||||||
TIMELINE_SERVICE_PREFIX + "reader.class";
|
TIMELINE_SERVICE_PREFIX + "reader.class";
|
||||||
|
|
||||||
public static final String DEFAULT_TIMELINE_SERVICE_READER_CLASS =
|
public static final String DEFAULT_TIMELINE_SERVICE_READER_CLASS =
|
||||||
"org.apache.hadoop.yarn.server.timelineservice" +
|
"org.apache.hadoop.yarn.server.timelineservice.storage" +
|
||||||
".storage.HBaseTimelineReaderImpl";
|
".HBaseTimelineReaderImpl";
|
||||||
|
|
||||||
|
/**
|
||||||
|
* default schema prefix for hbase tables.
|
||||||
|
*/
|
||||||
|
public static final String DEFAULT_TIMELINE_SERVICE_HBASE_SCHEMA_PREFIX =
|
||||||
|
"prod.";
|
||||||
|
|
||||||
|
/**
|
||||||
|
* config param name to override schema prefix.
|
||||||
|
*/
|
||||||
|
public static final String TIMELINE_SERVICE_HBASE_SCHEMA_PREFIX_NAME =
|
||||||
|
TIMELINE_SERVICE_PREFIX + "hbase-schema.prefix";
|
||||||
|
|
||||||
/** The setting that controls how often the timeline collector flushes the
|
/** The setting that controls how often the timeline collector flushes the
|
||||||
* timeline writer.
|
* timeline writer.
|
||||||
@ -2134,6 +2149,58 @@ public class YarnConfiguration extends Configuration {
|
|||||||
TIMELINE_SERVICE_PREFIX
|
TIMELINE_SERVICE_PREFIX
|
||||||
+ "hbase.coprocessor.app-final-value-retention-milliseconds";
|
+ "hbase.coprocessor.app-final-value-retention-milliseconds";
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The name of the setting for the location of the coprocessor
|
||||||
|
* jar on hdfs.
|
||||||
|
*/
|
||||||
|
public static final String FLOW_RUN_COPROCESSOR_JAR_HDFS_LOCATION =
|
||||||
|
TIMELINE_SERVICE_PREFIX
|
||||||
|
+ "hbase.coprocessor.jar.hdfs.location";
|
||||||
|
|
||||||
|
/** default hdfs location for flowrun coprocessor jar. */
|
||||||
|
public static final String DEFAULT_HDFS_LOCATION_FLOW_RUN_COPROCESSOR_JAR =
|
||||||
|
"/hbase/coprocessor/hadoop-yarn-server-timelineservice.jar";
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The name for setting that points to an optional HBase configuration
|
||||||
|
* (hbase-site.xml file) with settings that will override the ones found on
|
||||||
|
* the classpath.
|
||||||
|
*/
|
||||||
|
public static final String TIMELINE_SERVICE_HBASE_CONFIGURATION_FILE =
|
||||||
|
TIMELINE_SERVICE_PREFIX
|
||||||
|
+ "hbase.configuration.file";
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The name for setting that enables or disables authentication checks
|
||||||
|
* for reading timeline service v2 data.
|
||||||
|
*/
|
||||||
|
public static final String TIMELINE_SERVICE_READ_AUTH_ENABLED =
|
||||||
|
TIMELINE_SERVICE_PREFIX + "read.authentication.enabled";
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The default setting for authentication checks for reading timeline
|
||||||
|
* service v2 data.
|
||||||
|
*/
|
||||||
|
public static final Boolean DEFAULT_TIMELINE_SERVICE_READ_AUTH_ENABLED =
|
||||||
|
false;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The name for setting that lists the users and groups who are allowed
|
||||||
|
* to read timeline service v2 data. It is a comma separated list of
|
||||||
|
* user, followed by space, then comma separated list of groups.
|
||||||
|
* It will allow this list of users and groups to read the data
|
||||||
|
* and reject everyone else.
|
||||||
|
*/
|
||||||
|
public static final String TIMELINE_SERVICE_READ_ALLOWED_USERS =
|
||||||
|
TIMELINE_SERVICE_PREFIX + "read.allowed.users";
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The default value for list of the users who are allowed to read
|
||||||
|
* timeline service v2 data.
|
||||||
|
*/
|
||||||
|
public static final String DEFAULT_TIMELINE_SERVICE_READ_ALLOWED_USERS =
|
||||||
|
"";
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The setting that controls how long the final value of a metric of a
|
* The setting that controls how long the final value of a metric of a
|
||||||
* completed app is retained before merging into the flow sum. Up to this time
|
* completed app is retained before merging into the flow sum. Up to this time
|
||||||
@ -2154,6 +2221,8 @@ public class YarnConfiguration extends Configuration {
|
|||||||
|
|
||||||
public static final int DEFAULT_NUMBER_OF_ASYNC_ENTITIES_TO_MERGE = 10;
|
public static final int DEFAULT_NUMBER_OF_ASYNC_ENTITIES_TO_MERGE = 10;
|
||||||
|
|
||||||
|
/** default version for any flow. */
|
||||||
|
public static final String DEFAULT_FLOW_VERSION = "1";
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The time period for which timeline v2 client will wait for draining
|
* The time period for which timeline v2 client will wait for draining
|
||||||
|
@ -46,4 +46,12 @@ public final class TimelineServiceHelper {
|
|||||||
(HashMap<E, V>) originalMap : new HashMap<E, V>(originalMap);
|
(HashMap<E, V>) originalMap : new HashMap<E, V>(originalMap);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Inverts the given key.
|
||||||
|
* @param key value to be inverted .
|
||||||
|
* @return inverted long
|
||||||
|
*/
|
||||||
|
public static long invertLong(long key) {
|
||||||
|
return Long.MAX_VALUE - key;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -613,3 +613,8 @@ message StringBytesMapProto {
|
|||||||
optional string key = 1;
|
optional string key = 1;
|
||||||
optional bytes value = 2;
|
optional bytes value = 2;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
message CollectorInfoProto {
|
||||||
|
optional string collector_addr = 1;
|
||||||
|
optional hadoop.common.TokenProto collector_token = 2;
|
||||||
|
}
|
||||||
|
@ -112,7 +112,7 @@ message AllocateResponseProto {
|
|||||||
repeated NMTokenProto nm_tokens = 9;
|
repeated NMTokenProto nm_tokens = 9;
|
||||||
optional hadoop.common.TokenProto am_rm_token = 12;
|
optional hadoop.common.TokenProto am_rm_token = 12;
|
||||||
optional PriorityProto application_priority = 13;
|
optional PriorityProto application_priority = 13;
|
||||||
optional string collector_addr = 14;
|
optional CollectorInfoProto collector_info = 14;
|
||||||
repeated UpdateContainerErrorProto update_errors = 15;
|
repeated UpdateContainerErrorProto update_errors = 15;
|
||||||
repeated UpdatedContainerProto updated_containers = 16;
|
repeated UpdatedContainerProto updated_containers = 16;
|
||||||
}
|
}
|
||||||
|
@ -0,0 +1,71 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.yarn.api.records.timelineservice;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Various tests for the ApplicationEntity class.
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class TestApplicationEntity {
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIsApplicationEntity() {
|
||||||
|
TimelineEntity te = new TimelineEntity();
|
||||||
|
te.setType(TimelineEntityType.YARN_APPLICATION.toString());
|
||||||
|
assertTrue(ApplicationEntity.isApplicationEntity(te));
|
||||||
|
|
||||||
|
te = null;
|
||||||
|
assertEquals(false, ApplicationEntity.isApplicationEntity(te));
|
||||||
|
|
||||||
|
te = new TimelineEntity();
|
||||||
|
te.setType(TimelineEntityType.YARN_CLUSTER.toString());
|
||||||
|
assertEquals(false, ApplicationEntity.isApplicationEntity(te));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGetApplicationEvent() {
|
||||||
|
TimelineEntity te = null;
|
||||||
|
TimelineEvent tEvent = ApplicationEntity.getApplicationEvent(te,
|
||||||
|
"no event");
|
||||||
|
assertEquals(null, tEvent);
|
||||||
|
|
||||||
|
te = new TimelineEntity();
|
||||||
|
te.setType(TimelineEntityType.YARN_APPLICATION.toString());
|
||||||
|
TimelineEvent event = new TimelineEvent();
|
||||||
|
event.setId("start_event");
|
||||||
|
event.setTimestamp(System.currentTimeMillis());
|
||||||
|
te.addEvent(event);
|
||||||
|
tEvent = ApplicationEntity.getApplicationEvent(te, "start_event");
|
||||||
|
assertEquals(event, tEvent);
|
||||||
|
|
||||||
|
te = new TimelineEntity();
|
||||||
|
te.setType(TimelineEntityType.YARN_CLUSTER.toString());
|
||||||
|
event = new TimelineEvent();
|
||||||
|
event.setId("start_event_cluster");
|
||||||
|
event.setTimestamp(System.currentTimeMillis());
|
||||||
|
te.addEvent(event);
|
||||||
|
tEvent = ApplicationEntity.getApplicationEvent(te, "start_event_cluster");
|
||||||
|
assertEquals(null, tEvent);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
@ -66,6 +66,8 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
|
|||||||
configurationPropsToSkipCompare
|
configurationPropsToSkipCompare
|
||||||
.add(YarnConfiguration
|
.add(YarnConfiguration
|
||||||
.YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCETRACKER_PROTOCOL);
|
.YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCETRACKER_PROTOCOL);
|
||||||
|
configurationPropsToSkipCompare.add(YarnConfiguration
|
||||||
|
.YARN_SECURITY_SERVICE_AUTHORIZATION_COLLECTOR_NODEMANAGER_PROTOCOL);
|
||||||
configurationPropsToSkipCompare.add(YarnConfiguration.CURATOR_LEADER_ELECTOR);
|
configurationPropsToSkipCompare.add(YarnConfiguration.CURATOR_LEADER_ELECTOR);
|
||||||
|
|
||||||
// Federation default configs to be ignored
|
// Federation default configs to be ignored
|
||||||
|
@ -104,6 +104,8 @@ import org.apache.hadoop.yarn.client.api.async.impl.NMClientAsyncImpl;
|
|||||||
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 class ApplicationMaster {
|
|||||||
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 class ApplicationMaster {
|
|||||||
+ 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 class ApplicationMaster {
|
|||||||
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 @@ public class ApplicationMaster {
|
|||||||
}
|
}
|
||||||
|
|
||||||
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 class ApplicationMaster {
|
|||||||
}
|
}
|
||||||
|
|
||||||
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 @@ public class ApplicationMaster {
|
|||||||
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 @@ public class ApplicationMaster {
|
|||||||
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.FinalApplicationStatus;
|
|||||||
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.PluginStoreTestUtils;
|
|||||||
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 @@ public class TestDistributedShell {
|
|||||||
"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 @@ public class TestDistributedShell {
|
|||||||
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 @@ public class TestDistributedShell {
|
|||||||
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 @@ public class TestDistributedShell {
|
|||||||
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 @@ public class TestDistributedShell {
|
|||||||
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);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -68,8 +68,6 @@ extends AMRMClientAsync<T> {
|
|||||||
|
|
||||||
private volatile boolean keepRunning;
|
private volatile boolean keepRunning;
|
||||||
private volatile float progress;
|
private volatile float progress;
|
||||||
|
|
||||||
private volatile String collectorAddr;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*
|
*
|
||||||
@ -325,17 +323,16 @@ extends AMRMClientAsync<T> {
|
|||||||
}
|
}
|
||||||
|
|
||||||
AllocateResponse response = (AllocateResponse) object;
|
AllocateResponse response = (AllocateResponse) object;
|
||||||
String collectorAddress = response.getCollectorAddr();
|
String collectorAddress = null;
|
||||||
|
if (response.getCollectorInfo() != null) {
|
||||||
|
collectorAddress = response.getCollectorInfo().getCollectorAddr();
|
||||||
|
}
|
||||||
|
|
||||||
TimelineV2Client timelineClient =
|
TimelineV2Client timelineClient =
|
||||||
client.getRegisteredTimelineV2Client();
|
client.getRegisteredTimelineV2Client();
|
||||||
if (timelineClient != null && collectorAddress != null
|
if (timelineClient != null && response.getCollectorInfo() != null) {
|
||||||
&& !collectorAddress.isEmpty()) {
|
timelineClient.
|
||||||
if (collectorAddr == null
|
setTimelineCollectorInfo(response.getCollectorInfo());
|
||||||
|| !collectorAddr.equals(collectorAddress)) {
|
|
||||||
collectorAddr = collectorAddress;
|
|
||||||
timelineClient.setTimelineServiceAddress(collectorAddress);
|
|
||||||
LOG.info("collectorAddress " + collectorAddress);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
List<NodeReport> updatedNodes = response.getUpdatedNodes();
|
List<NodeReport> updatedNodes = response.getUpdatedNodes();
|
||||||
|
@ -0,0 +1,72 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.yarn.client;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.hadoop.ipc.RPC;
|
||||||
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
|
import org.apache.hadoop.security.token.Token;
|
||||||
|
import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||||
|
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
|
||||||
|
import org.junit.After;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test Base for Application Master Service Protocol.
|
||||||
|
*/
|
||||||
|
public abstract class ApplicationMasterServiceProtoTestBase
|
||||||
|
extends ProtocolHATestBase {
|
||||||
|
|
||||||
|
private ApplicationMasterProtocol amClient;
|
||||||
|
private ApplicationAttemptId attemptId;
|
||||||
|
|
||||||
|
protected void startupHAAndSetupClient() throws Exception {
|
||||||
|
attemptId = this.cluster.createFakeApplicationAttemptId();
|
||||||
|
|
||||||
|
Token<AMRMTokenIdentifier> appToken =
|
||||||
|
this.cluster.getResourceManager().getRMContext()
|
||||||
|
.getAMRMTokenSecretManager().createAndGetAMRMToken(attemptId);
|
||||||
|
appToken.setService(ClientRMProxy.getAMRMTokenService(this.conf));
|
||||||
|
UserGroupInformation.setLoginUser(UserGroupInformation
|
||||||
|
.createRemoteUser(UserGroupInformation.getCurrentUser().getUserName()));
|
||||||
|
UserGroupInformation.getCurrentUser().addToken(appToken);
|
||||||
|
syncToken(appToken);
|
||||||
|
amClient = ClientRMProxy
|
||||||
|
.createRMProxy(this.conf, ApplicationMasterProtocol.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void shutDown() {
|
||||||
|
if(this.amClient != null) {
|
||||||
|
RPC.stopProxy(this.amClient);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected ApplicationMasterProtocol getAMClient() {
|
||||||
|
return amClient;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void syncToken(Token<AMRMTokenIdentifier> token) throws IOException {
|
||||||
|
for (int i = 0; i < this.cluster.getNumOfResourceManager(); i++) {
|
||||||
|
this.cluster.getResourceManager(i).getRMContext()
|
||||||
|
.getAMRMTokenSecretManager().addPersistedPassword(token);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -25,6 +25,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest
|
|||||||
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
|
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
|
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
|
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
|
||||||
|
import org.apache.hadoop.yarn.api.records.CollectorInfo;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
|
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assert.assertFalse;
|
import static org.junit.Assert.assertFalse;
|
||||||
@ -804,11 +805,20 @@ public abstract class ProtocolHATestBase extends ClientBaseWithFixes {
|
|||||||
}
|
}
|
||||||
|
|
||||||
public AllocateResponse createFakeAllocateResponse() {
|
public AllocateResponse createFakeAllocateResponse() {
|
||||||
return AllocateResponse.newInstance(-1,
|
if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) {
|
||||||
new ArrayList<ContainerStatus>(),
|
return AllocateResponse.newInstance(-1,
|
||||||
new ArrayList<Container>(), new ArrayList<NodeReport>(),
|
new ArrayList<ContainerStatus>(), new ArrayList<Container>(),
|
||||||
Resource.newInstance(1024, 2), null, 1,
|
new ArrayList<NodeReport>(), Resource.newInstance(1024, 2), null, 1,
|
||||||
null, new ArrayList<NMToken>());
|
null, new ArrayList<NMToken>(), CollectorInfo.newInstance(
|
||||||
|
"host:port", Token.newInstance(new byte[] {0}, "TIMELINE",
|
||||||
|
new byte[] {0}, "rm")));
|
||||||
|
} else {
|
||||||
|
return AllocateResponse.newInstance(-1,
|
||||||
|
new ArrayList<ContainerStatus>(),
|
||||||
|
new ArrayList<Container>(), new ArrayList<NodeReport>(),
|
||||||
|
Resource.newInstance(1024, 2), null, 1,
|
||||||
|
null, new ArrayList<NMToken>());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -0,0 +1,71 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.yarn.client;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
|
||||||
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
|
||||||
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||||
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||||
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.HATestUtil;
|
||||||
|
import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl;
|
||||||
|
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests Application Master Protocol with timeline service v2 enabled.
|
||||||
|
*/
|
||||||
|
public class TestApplicationMasterServiceProtocolForTimelineV2
|
||||||
|
extends ApplicationMasterServiceProtoTestBase {
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void initialize() throws Exception {
|
||||||
|
HATestUtil.setRpcAddressForRM(RM1_NODE_ID, RM1_PORT_BASE + 200, conf);
|
||||||
|
HATestUtil.setRpcAddressForRM(RM2_NODE_ID, RM2_PORT_BASE + 200, conf);
|
||||||
|
conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
|
||||||
|
conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
|
||||||
|
conf.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS,
|
||||||
|
FileSystemTimelineWriterImpl.class, TimelineWriter.class);
|
||||||
|
startHACluster(0, false, false, true);
|
||||||
|
super.startupHAAndSetupClient();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(timeout = 15000)
|
||||||
|
public void testAllocateForTimelineV2OnHA()
|
||||||
|
throws YarnException, IOException {
|
||||||
|
AllocateRequest request = AllocateRequest.newInstance(0, 50f,
|
||||||
|
new ArrayList<ResourceRequest>(),
|
||||||
|
new ArrayList<ContainerId>(),
|
||||||
|
ResourceBlacklistRequest.newInstance(new ArrayList<String>(),
|
||||||
|
new ArrayList<String>()));
|
||||||
|
AllocateResponse response = getAMClient().allocate(request);
|
||||||
|
Assert.assertEquals(response, this.cluster.createFakeAllocateResponse());
|
||||||
|
Assert.assertNotNull(response.getCollectorInfo());
|
||||||
|
Assert.assertEquals("host:port",
|
||||||
|
response.getCollectorInfo().getCollectorAddr());
|
||||||
|
Assert.assertNotNull(response.getCollectorInfo().getCollectorToken());
|
||||||
|
}
|
||||||
|
}
|
@ -23,10 +23,6 @@ import java.util.ArrayList;
|
|||||||
|
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
|
|
||||||
import org.apache.hadoop.ipc.RPC;
|
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
|
||||||
import org.apache.hadoop.security.token.Token;
|
|
||||||
import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
|
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
|
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
|
||||||
@ -34,45 +30,20 @@ import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRespons
|
|||||||
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
|
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
|
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
|
||||||
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
|
import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
|
||||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||||
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
|
|
||||||
import org.junit.After;
|
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
|
||||||
public class TestApplicationMasterServiceProtocolOnHA
|
public class TestApplicationMasterServiceProtocolOnHA
|
||||||
extends ProtocolHATestBase {
|
extends ApplicationMasterServiceProtoTestBase {
|
||||||
private ApplicationMasterProtocol amClient;
|
|
||||||
private ApplicationAttemptId attemptId ;
|
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void initialize() throws Exception {
|
public void initialize() throws Exception {
|
||||||
startHACluster(0, false, false, true);
|
startHACluster(0, false, false, true);
|
||||||
attemptId = this.cluster.createFakeApplicationAttemptId();
|
super.startupHAAndSetupClient();
|
||||||
|
|
||||||
Token<AMRMTokenIdentifier> appToken =
|
|
||||||
this.cluster.getResourceManager().getRMContext()
|
|
||||||
.getAMRMTokenSecretManager().createAndGetAMRMToken(attemptId);
|
|
||||||
appToken.setService(ClientRMProxy.getAMRMTokenService(this.conf));
|
|
||||||
UserGroupInformation.setLoginUser(UserGroupInformation
|
|
||||||
.createRemoteUser(UserGroupInformation.getCurrentUser().getUserName()));
|
|
||||||
UserGroupInformation.getCurrentUser().addToken(appToken);
|
|
||||||
syncToken(appToken);
|
|
||||||
|
|
||||||
amClient = ClientRMProxy
|
|
||||||
.createRMProxy(this.conf, ApplicationMasterProtocol.class);
|
|
||||||
}
|
|
||||||
|
|
||||||
@After
|
|
||||||
public void shutDown() {
|
|
||||||
if(this.amClient != null) {
|
|
||||||
RPC.stopProxy(this.amClient);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout = 15000)
|
@Test(timeout = 15000)
|
||||||
@ -81,7 +52,7 @@ public class TestApplicationMasterServiceProtocolOnHA
|
|||||||
RegisterApplicationMasterRequest request =
|
RegisterApplicationMasterRequest request =
|
||||||
RegisterApplicationMasterRequest.newInstance("localhost", 0, "");
|
RegisterApplicationMasterRequest.newInstance("localhost", 0, "");
|
||||||
RegisterApplicationMasterResponse response =
|
RegisterApplicationMasterResponse response =
|
||||||
amClient.registerApplicationMaster(request);
|
getAMClient().registerApplicationMaster(request);
|
||||||
Assert.assertEquals(response,
|
Assert.assertEquals(response,
|
||||||
this.cluster.createFakeRegisterApplicationMasterResponse());
|
this.cluster.createFakeRegisterApplicationMasterResponse());
|
||||||
}
|
}
|
||||||
@ -93,7 +64,7 @@ public class TestApplicationMasterServiceProtocolOnHA
|
|||||||
FinishApplicationMasterRequest.newInstance(
|
FinishApplicationMasterRequest.newInstance(
|
||||||
FinalApplicationStatus.SUCCEEDED, "", "");
|
FinalApplicationStatus.SUCCEEDED, "", "");
|
||||||
FinishApplicationMasterResponse response =
|
FinishApplicationMasterResponse response =
|
||||||
amClient.finishApplicationMaster(request);
|
getAMClient().finishApplicationMaster(request);
|
||||||
Assert.assertEquals(response,
|
Assert.assertEquals(response,
|
||||||
this.cluster.createFakeFinishApplicationMasterResponse());
|
this.cluster.createFakeFinishApplicationMasterResponse());
|
||||||
}
|
}
|
||||||
@ -105,14 +76,7 @@ public class TestApplicationMasterServiceProtocolOnHA
|
|||||||
new ArrayList<ContainerId>(),
|
new ArrayList<ContainerId>(),
|
||||||
ResourceBlacklistRequest.newInstance(new ArrayList<String>(),
|
ResourceBlacklistRequest.newInstance(new ArrayList<String>(),
|
||||||
new ArrayList<String>()));
|
new ArrayList<String>()));
|
||||||
AllocateResponse response = amClient.allocate(request);
|
AllocateResponse response = getAMClient().allocate(request);
|
||||||
Assert.assertEquals(response, this.cluster.createFakeAllocateResponse());
|
Assert.assertEquals(response, this.cluster.createFakeAllocateResponse());
|
||||||
}
|
}
|
||||||
|
|
||||||
private void syncToken(Token<AMRMTokenIdentifier> token) throws IOException {
|
|
||||||
for (int i = 0; i < this.cluster.getNumOfResourceManager(); i++) {
|
|
||||||
this.cluster.getResourceManager(i).getRMContext()
|
|
||||||
.getAMRMTokenSecretManager().addPersistedPassword(token);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -426,7 +426,7 @@ public class TestAMRMClientAsync {
|
|||||||
}
|
}
|
||||||
AllocateResponse response =
|
AllocateResponse response =
|
||||||
AllocateResponse.newInstance(0, completed, allocated,
|
AllocateResponse.newInstance(0, completed, allocated,
|
||||||
new ArrayList<NodeReport>(), null, null, 1, null, nmTokens,
|
new ArrayList<NodeReport>(), null, null, 1, null, nmTokens, null,
|
||||||
updatedContainers);
|
updatedContainers);
|
||||||
return response;
|
return response;
|
||||||
}
|
}
|
||||||
|
@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
|
|||||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||||
import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
|
import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
||||||
|
import org.apache.hadoop.yarn.api.records.CollectorInfo;
|
||||||
import org.apache.hadoop.yarn.api.records.AMCommand;
|
import org.apache.hadoop.yarn.api.records.AMCommand;
|
||||||
import org.apache.hadoop.yarn.api.records.Container;
|
import org.apache.hadoop.yarn.api.records.Container;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
||||||
@ -38,6 +39,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
|
|||||||
import org.apache.hadoop.yarn.api.records.Token;
|
import org.apache.hadoop.yarn.api.records.Token;
|
||||||
import org.apache.hadoop.yarn.api.records.UpdateContainerError;
|
import org.apache.hadoop.yarn.api.records.UpdateContainerError;
|
||||||
import org.apache.hadoop.yarn.api.records.UpdatedContainer;
|
import org.apache.hadoop.yarn.api.records.UpdatedContainer;
|
||||||
|
import org.apache.hadoop.yarn.api.records.impl.pb.CollectorInfoPBImpl;
|
||||||
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
|
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
|
||||||
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
|
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
|
||||||
import org.apache.hadoop.yarn.api.records.impl.pb.NMTokenPBImpl;
|
import org.apache.hadoop.yarn.api.records.impl.pb.NMTokenPBImpl;
|
||||||
@ -48,6 +50,7 @@ import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
|
|||||||
import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
|
import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
|
||||||
import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
|
import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
|
||||||
import org.apache.hadoop.yarn.api.records.impl.pb.UpdatedContainerPBImpl;
|
import org.apache.hadoop.yarn.api.records.impl.pb.UpdatedContainerPBImpl;
|
||||||
|
import org.apache.hadoop.yarn.proto.YarnProtos.CollectorInfoProto;
|
||||||
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
|
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
|
||||||
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
|
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
|
||||||
import org.apache.hadoop.yarn.proto.YarnProtos.NodeReportProto;
|
import org.apache.hadoop.yarn.proto.YarnProtos.NodeReportProto;
|
||||||
@ -80,6 +83,7 @@ public class AllocateResponsePBImpl extends AllocateResponse {
|
|||||||
private PreemptionMessage preempt;
|
private PreemptionMessage preempt;
|
||||||
private Token amrmToken = null;
|
private Token amrmToken = null;
|
||||||
private Priority appPriority = null;
|
private Priority appPriority = null;
|
||||||
|
private CollectorInfo collectorInfo = null;
|
||||||
|
|
||||||
public AllocateResponsePBImpl() {
|
public AllocateResponsePBImpl() {
|
||||||
builder = AllocateResponseProto.newBuilder();
|
builder = AllocateResponseProto.newBuilder();
|
||||||
@ -162,6 +166,9 @@ public class AllocateResponsePBImpl extends AllocateResponse {
|
|||||||
if (this.amrmToken != null) {
|
if (this.amrmToken != null) {
|
||||||
builder.setAmRmToken(convertToProtoFormat(this.amrmToken));
|
builder.setAmRmToken(convertToProtoFormat(this.amrmToken));
|
||||||
}
|
}
|
||||||
|
if (this.collectorInfo != null) {
|
||||||
|
builder.setCollectorInfo(convertToProtoFormat(this.collectorInfo));
|
||||||
|
}
|
||||||
if (this.appPriority != null) {
|
if (this.appPriority != null) {
|
||||||
builder.setApplicationPriority(convertToProtoFormat(this.appPriority));
|
builder.setApplicationPriority(convertToProtoFormat(this.appPriority));
|
||||||
}
|
}
|
||||||
@ -398,19 +405,25 @@ public class AllocateResponsePBImpl extends AllocateResponse {
|
|||||||
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized String getCollectorAddr() {
|
public synchronized CollectorInfo getCollectorInfo() {
|
||||||
AllocateResponseProtoOrBuilder p = viaProto ? proto : builder;
|
AllocateResponseProtoOrBuilder p = viaProto ? proto : builder;
|
||||||
return p.getCollectorAddr();
|
if (this.collectorInfo != null) {
|
||||||
|
return this.collectorInfo;
|
||||||
|
}
|
||||||
|
if (!p.hasCollectorInfo()) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
this.collectorInfo = convertFromProtoFormat(p.getCollectorInfo());
|
||||||
|
return this.collectorInfo;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized void setCollectorAddr(String collectorAddr) {
|
public synchronized void setCollectorInfo(CollectorInfo info) {
|
||||||
maybeInitBuilder();
|
maybeInitBuilder();
|
||||||
if (collectorAddr == null) {
|
if (info == null) {
|
||||||
builder.clearCollectorAddr();
|
builder.clearCollectorInfo();
|
||||||
return;
|
|
||||||
}
|
}
|
||||||
builder.setCollectorAddr(collectorAddr);
|
this.collectorInfo = info;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -718,6 +731,16 @@ public class AllocateResponsePBImpl extends AllocateResponse {
|
|||||||
return ((NodeReportPBImpl)t).getProto();
|
return ((NodeReportPBImpl)t).getProto();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private synchronized CollectorInfoPBImpl convertFromProtoFormat(
|
||||||
|
CollectorInfoProto p) {
|
||||||
|
return new CollectorInfoPBImpl(p);
|
||||||
|
}
|
||||||
|
|
||||||
|
private synchronized CollectorInfoProto convertToProtoFormat(
|
||||||
|
CollectorInfo t) {
|
||||||
|
return ((CollectorInfoPBImpl)t).getProto();
|
||||||
|
}
|
||||||
|
|
||||||
private synchronized ContainerPBImpl convertFromProtoFormat(
|
private synchronized ContainerPBImpl convertFromProtoFormat(
|
||||||
ContainerProto p) {
|
ContainerProto p) {
|
||||||
return new ContainerPBImpl(p);
|
return new ContainerPBImpl(p);
|
||||||
|
@ -0,0 +1,152 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.yarn.api.records.impl.pb;
|
||||||
|
|
||||||
|
import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
|
||||||
|
import org.apache.hadoop.yarn.api.records.CollectorInfo;
|
||||||
|
import org.apache.hadoop.yarn.api.records.Token;
|
||||||
|
import org.apache.hadoop.yarn.proto.YarnProtos.CollectorInfoProto;
|
||||||
|
import org.apache.hadoop.yarn.proto.YarnProtos.CollectorInfoProtoOrBuilder;
|
||||||
|
|
||||||
|
import com.google.protobuf.TextFormat;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Protocol record implementation of {@link CollectorInfo}.
|
||||||
|
*/
|
||||||
|
public class CollectorInfoPBImpl extends CollectorInfo {
|
||||||
|
|
||||||
|
private CollectorInfoProto proto = CollectorInfoProto.getDefaultInstance();
|
||||||
|
|
||||||
|
private CollectorInfoProto.Builder builder = null;
|
||||||
|
private boolean viaProto = false;
|
||||||
|
|
||||||
|
private String collectorAddr = null;
|
||||||
|
private Token collectorToken = null;
|
||||||
|
|
||||||
|
|
||||||
|
public CollectorInfoPBImpl() {
|
||||||
|
builder = CollectorInfoProto.newBuilder();
|
||||||
|
}
|
||||||
|
|
||||||
|
public CollectorInfoPBImpl(CollectorInfoProto proto) {
|
||||||
|
this.proto = proto;
|
||||||
|
viaProto = true;
|
||||||
|
}
|
||||||
|
|
||||||
|
public CollectorInfoProto getProto() {
|
||||||
|
mergeLocalToProto();
|
||||||
|
proto = viaProto ? proto : builder.build();
|
||||||
|
viaProto = true;
|
||||||
|
return proto;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return getProto().hashCode();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void maybeInitBuilder() {
|
||||||
|
if (viaProto || builder == null) {
|
||||||
|
builder = CollectorInfoProto.newBuilder(proto);
|
||||||
|
}
|
||||||
|
viaProto = false;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void mergeLocalToProto() {
|
||||||
|
if (viaProto) {
|
||||||
|
maybeInitBuilder();
|
||||||
|
}
|
||||||
|
mergeLocalToBuilder();
|
||||||
|
proto = builder.build();
|
||||||
|
viaProto = true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object other) {
|
||||||
|
if (other == null) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (other.getClass().isAssignableFrom(this.getClass())) {
|
||||||
|
return this.getProto().equals(this.getClass().cast(other).getProto());
|
||||||
|
}
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return TextFormat.shortDebugString(getProto());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getCollectorAddr() {
|
||||||
|
CollectorInfoProtoOrBuilder p = viaProto ? proto : builder;
|
||||||
|
if (this.collectorAddr == null && p.hasCollectorAddr()) {
|
||||||
|
this.collectorAddr = p.getCollectorAddr();
|
||||||
|
}
|
||||||
|
return this.collectorAddr;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setCollectorAddr(String addr) {
|
||||||
|
maybeInitBuilder();
|
||||||
|
if (collectorAddr == null) {
|
||||||
|
builder.clearCollectorAddr();
|
||||||
|
}
|
||||||
|
this.collectorAddr = addr;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Token getCollectorToken() {
|
||||||
|
CollectorInfoProtoOrBuilder p = viaProto ? proto : builder;
|
||||||
|
if (this.collectorToken != null) {
|
||||||
|
return this.collectorToken;
|
||||||
|
}
|
||||||
|
if (!p.hasCollectorToken()) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
this.collectorToken = convertFromProtoFormat(p.getCollectorToken());
|
||||||
|
return this.collectorToken;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setCollectorToken(Token token) {
|
||||||
|
maybeInitBuilder();
|
||||||
|
if (token == null) {
|
||||||
|
builder.clearCollectorToken();
|
||||||
|
}
|
||||||
|
this.collectorToken = token;
|
||||||
|
}
|
||||||
|
|
||||||
|
private TokenPBImpl convertFromProtoFormat(TokenProto p) {
|
||||||
|
return new TokenPBImpl(p);
|
||||||
|
}
|
||||||
|
|
||||||
|
private TokenProto convertToProtoFormat(Token t) {
|
||||||
|
return ((TokenPBImpl) t).getProto();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void mergeLocalToBuilder() {
|
||||||
|
if (this.collectorAddr != null) {
|
||||||
|
builder.setCollectorAddr(this.collectorAddr);
|
||||||
|
}
|
||||||
|
if (this.collectorToken != null) {
|
||||||
|
builder.setCollectorToken(convertToProtoFormat(this.collectorToken));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -23,6 +23,7 @@ import java.io.IOException;
|
|||||||
import org.apache.hadoop.classification.InterfaceAudience.Public;
|
import org.apache.hadoop.classification.InterfaceAudience.Public;
|
||||||
import org.apache.hadoop.service.CompositeService;
|
import org.apache.hadoop.service.CompositeService;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
|
import org.apache.hadoop.yarn.api.records.CollectorInfo;
|
||||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
|
||||||
import org.apache.hadoop.yarn.client.api.impl.TimelineV2ClientImpl;
|
import org.apache.hadoop.yarn.client.api.impl.TimelineV2ClientImpl;
|
||||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||||
@ -83,10 +84,13 @@ public abstract class TimelineV2Client extends CompositeService {
|
|||||||
|
|
||||||
/**
|
/**
|
||||||
* <p>
|
* <p>
|
||||||
* Update the timeline service address where the request will be sent to.
|
* Update collector info received in AllocateResponse which contains the
|
||||||
|
* timeline service address where the request will be sent to and the timeline
|
||||||
|
* delegation token which will be used to send the request.
|
||||||
* </p>
|
* </p>
|
||||||
*
|
*
|
||||||
* @param address the timeline service address
|
* @param collectorInfo Collector info which contains the timeline service
|
||||||
|
* address and timeline delegation token.
|
||||||
*/
|
*/
|
||||||
public abstract void setTimelineServiceAddress(String address);
|
public abstract void setTimelineCollectorInfo(CollectorInfo collectorInfo);
|
||||||
}
|
}
|
||||||
|
@ -19,7 +19,11 @@
|
|||||||
package org.apache.hadoop.yarn.client.api.impl;
|
package org.apache.hadoop.yarn.client.api.impl;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.InterruptedIOException;
|
||||||
|
import java.lang.reflect.UndeclaredThrowableException;
|
||||||
|
import java.net.InetSocketAddress;
|
||||||
import java.net.URI;
|
import java.net.URI;
|
||||||
|
import java.security.PrivilegedExceptionAction;
|
||||||
import java.util.concurrent.BlockingQueue;
|
import java.util.concurrent.BlockingQueue;
|
||||||
import java.util.concurrent.Callable;
|
import java.util.concurrent.Callable;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
@ -36,15 +40,22 @@ import org.apache.commons.logging.Log;
|
|||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.io.Text;
|
||||||
|
import org.apache.hadoop.net.NetUtils;
|
||||||
|
import org.apache.hadoop.security.SecurityUtil;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
|
import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
|
import org.apache.hadoop.yarn.api.records.CollectorInfo;
|
||||||
|
import org.apache.hadoop.yarn.api.records.Token;
|
||||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
|
||||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
|
||||||
import org.apache.hadoop.yarn.client.api.TimelineV2Client;
|
import org.apache.hadoop.yarn.client.api.TimelineV2Client;
|
||||||
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.client.TimelineDelegationTokenIdentifier;
|
||||||
|
|
||||||
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import com.sun.jersey.api.client.ClientResponse;
|
import com.sun.jersey.api.client.ClientResponse;
|
||||||
import com.sun.jersey.core.util.MultivaluedMapImpl;
|
import com.sun.jersey.core.util.MultivaluedMapImpl;
|
||||||
|
|
||||||
@ -59,6 +70,8 @@ public class TimelineV2ClientImpl extends TimelineV2Client {
|
|||||||
|
|
||||||
private TimelineEntityDispatcher entityDispatcher;
|
private TimelineEntityDispatcher entityDispatcher;
|
||||||
private volatile String timelineServiceAddress;
|
private volatile String timelineServiceAddress;
|
||||||
|
@VisibleForTesting
|
||||||
|
volatile Token currentTimelineToken = null;
|
||||||
|
|
||||||
// Retry parameters for identifying new timeline service
|
// Retry parameters for identifying new timeline service
|
||||||
// TODO consider to merge with connection retry
|
// TODO consider to merge with connection retry
|
||||||
@ -69,6 +82,8 @@ public class TimelineV2ClientImpl extends TimelineV2Client {
|
|||||||
|
|
||||||
private ApplicationId contextAppId;
|
private ApplicationId contextAppId;
|
||||||
|
|
||||||
|
private UserGroupInformation authUgi;
|
||||||
|
|
||||||
public TimelineV2ClientImpl(ApplicationId appId) {
|
public TimelineV2ClientImpl(ApplicationId appId) {
|
||||||
super(TimelineV2ClientImpl.class.getName());
|
super(TimelineV2ClientImpl.class.getName());
|
||||||
this.contextAppId = appId;
|
this.contextAppId = appId;
|
||||||
@ -88,7 +103,6 @@ public class TimelineV2ClientImpl extends TimelineV2Client {
|
|||||||
UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
|
UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
|
||||||
UserGroupInformation realUgi = ugi.getRealUser();
|
UserGroupInformation realUgi = ugi.getRealUser();
|
||||||
String doAsUser = null;
|
String doAsUser = null;
|
||||||
UserGroupInformation authUgi = null;
|
|
||||||
if (realUgi != null) {
|
if (realUgi != null) {
|
||||||
authUgi = realUgi;
|
authUgi = realUgi;
|
||||||
doAsUser = ugi.getShortUserName();
|
doAsUser = ugi.getShortUserName();
|
||||||
@ -96,7 +110,6 @@ public class TimelineV2ClientImpl extends TimelineV2Client {
|
|||||||
authUgi = ugi;
|
authUgi = ugi;
|
||||||
doAsUser = null;
|
doAsUser = null;
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO need to add/cleanup filter retry later for ATSV2. similar to V1
|
// TODO need to add/cleanup filter retry later for ATSV2. similar to V1
|
||||||
DelegationTokenAuthenticatedURL.Token token =
|
DelegationTokenAuthenticatedURL.Token token =
|
||||||
new DelegationTokenAuthenticatedURL.Token();
|
new DelegationTokenAuthenticatedURL.Token();
|
||||||
@ -140,8 +153,72 @@ public class TimelineV2ClientImpl extends TimelineV2Client {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void setTimelineServiceAddress(String address) {
|
public void setTimelineCollectorInfo(CollectorInfo collectorInfo) {
|
||||||
this.timelineServiceAddress = address;
|
if (collectorInfo == null) {
|
||||||
|
LOG.warn("Not setting collector info as it is null.");
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
// First update the token so that it is available when collector address is
|
||||||
|
// used.
|
||||||
|
if (collectorInfo.getCollectorToken() != null) {
|
||||||
|
// Use collector address to update token service if its not available.
|
||||||
|
setTimelineDelegationToken(
|
||||||
|
collectorInfo.getCollectorToken(), collectorInfo.getCollectorAddr());
|
||||||
|
}
|
||||||
|
// Update timeline service address.
|
||||||
|
if (collectorInfo.getCollectorAddr() != null &&
|
||||||
|
!collectorInfo.getCollectorAddr().isEmpty() &&
|
||||||
|
!collectorInfo.getCollectorAddr().equals(timelineServiceAddress)) {
|
||||||
|
this.timelineServiceAddress = collectorInfo.getCollectorAddr();
|
||||||
|
LOG.info("Updated timeline service address to " + timelineServiceAddress);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void setTimelineDelegationToken(Token delegationToken,
|
||||||
|
String collectorAddr) {
|
||||||
|
// Checks below are to ensure that an invalid token is not updated in UGI.
|
||||||
|
// This is required because timeline token is set via a public API.
|
||||||
|
if (!delegationToken.getKind().equals(
|
||||||
|
TimelineDelegationTokenIdentifier.KIND_NAME.toString())) {
|
||||||
|
LOG.warn("Timeline token to be updated should be of kind " +
|
||||||
|
TimelineDelegationTokenIdentifier.KIND_NAME);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
if (collectorAddr == null || collectorAddr.isEmpty()) {
|
||||||
|
collectorAddr = timelineServiceAddress;
|
||||||
|
}
|
||||||
|
// Token need not be updated if both address and token service do not exist.
|
||||||
|
String service = delegationToken.getService();
|
||||||
|
if ((service == null || service.isEmpty()) &&
|
||||||
|
(collectorAddr == null || collectorAddr.isEmpty())) {
|
||||||
|
LOG.warn("Timeline token does not have service and timeline service " +
|
||||||
|
"address is not yet set. Not updating the token");
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
// No need to update a duplicate token.
|
||||||
|
if (currentTimelineToken != null &&
|
||||||
|
currentTimelineToken.equals(delegationToken)) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
currentTimelineToken = delegationToken;
|
||||||
|
// Convert the token, sanitize the token service and add it to UGI.
|
||||||
|
org.apache.hadoop.security.token.
|
||||||
|
Token<TimelineDelegationTokenIdentifier> timelineToken =
|
||||||
|
new org.apache.hadoop.security.token.
|
||||||
|
Token<TimelineDelegationTokenIdentifier>(
|
||||||
|
delegationToken.getIdentifier().array(),
|
||||||
|
delegationToken.getPassword().array(),
|
||||||
|
new Text(delegationToken.getKind()),
|
||||||
|
service == null ? new Text() : new Text(service));
|
||||||
|
// Prefer timeline service address over service coming in the token for
|
||||||
|
// updating the token service.
|
||||||
|
InetSocketAddress serviceAddr =
|
||||||
|
(collectorAddr != null && !collectorAddr.isEmpty()) ?
|
||||||
|
NetUtils.createSocketAddr(collectorAddr) :
|
||||||
|
SecurityUtil.getTokenServiceAddr(timelineToken);
|
||||||
|
SecurityUtil.setTokenService(timelineToken, serviceAddr);
|
||||||
|
authUgi.addToken(timelineToken);
|
||||||
|
LOG.info("Updated timeline delegation token " + timelineToken);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Private
|
@Private
|
||||||
@ -192,19 +269,33 @@ public class TimelineV2ClientImpl extends TimelineV2Client {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private ClientResponse doPutObjects(URI base, String path,
|
||||||
|
MultivaluedMap<String, String> params, Object obj) {
|
||||||
|
return connector.getClient().resource(base).path(path).queryParams(params)
|
||||||
|
.accept(MediaType.APPLICATION_JSON).type(MediaType.APPLICATION_JSON)
|
||||||
|
.put(ClientResponse.class, obj);
|
||||||
|
}
|
||||||
|
|
||||||
protected void putObjects(URI base, String path,
|
protected void putObjects(URI base, String path,
|
||||||
MultivaluedMap<String, String> params, Object obj)
|
MultivaluedMap<String, String> params, Object obj)
|
||||||
throws IOException, YarnException {
|
throws IOException, YarnException {
|
||||||
ClientResponse resp;
|
ClientResponse resp = null;
|
||||||
try {
|
try {
|
||||||
resp = connector.getClient().resource(base).path(path).queryParams(params)
|
resp = authUgi.doAs(new PrivilegedExceptionAction<ClientResponse>() {
|
||||||
.accept(MediaType.APPLICATION_JSON).type(MediaType.APPLICATION_JSON)
|
@Override
|
||||||
.put(ClientResponse.class, obj);
|
public ClientResponse run() throws Exception {
|
||||||
} catch (RuntimeException re) {
|
return doPutObjects(base, path, params, obj);
|
||||||
// runtime exception is expected if the client cannot connect the server
|
}
|
||||||
String msg = "Failed to get the response from the timeline server.";
|
});
|
||||||
LOG.error(msg, re);
|
} catch (UndeclaredThrowableException ue) {
|
||||||
throw new IOException(re);
|
Throwable cause = ue.getCause();
|
||||||
|
if (cause instanceof IOException) {
|
||||||
|
throw (IOException)cause;
|
||||||
|
} else {
|
||||||
|
throw new IOException(cause);
|
||||||
|
}
|
||||||
|
} catch (InterruptedException ie) {
|
||||||
|
throw (IOException) new InterruptedIOException().initCause(ie);
|
||||||
}
|
}
|
||||||
if (resp == null || resp.getStatusInfo()
|
if (resp == null || resp.getStatusInfo()
|
||||||
.getStatusCode() != ClientResponse.Status.OK.getStatusCode()) {
|
.getStatusCode() != ClientResponse.Status.OK.getStatusCode()) {
|
||||||
|
@ -2341,6 +2341,39 @@
|
|||||||
<value>259200000</value>
|
<value>259200000</value>
|
||||||
</property>
|
</property>
|
||||||
|
|
||||||
|
<property>
|
||||||
|
<description>
|
||||||
|
The default hdfs location for flowrun coprocessor jar.
|
||||||
|
</description>
|
||||||
|
<name>yarn.timeline-service.hbase.coprocessor.jar.hdfs.location
|
||||||
|
</name>
|
||||||
|
<value>/hbase/coprocessor/hadoop-yarn-server-timelineservice.jar</value>
|
||||||
|
</property>
|
||||||
|
|
||||||
|
<property>
|
||||||
|
<description>
|
||||||
|
The value of this parameter sets the prefix for all tables that are part of
|
||||||
|
timeline service in the hbase storage schema. It can be set to "dev."
|
||||||
|
or "staging." if it is to be used for development or staging instances.
|
||||||
|
This way the data in production tables stays in a separate set of tables
|
||||||
|
prefixed by "prod.".
|
||||||
|
</description>
|
||||||
|
<name>yarn.timeline-service.hbase-schema.prefix</name>
|
||||||
|
<value>prod.</value>
|
||||||
|
</property>
|
||||||
|
|
||||||
|
<property>
|
||||||
|
<description> Optional URL to an hbase-site.xml configuration file to be
|
||||||
|
used to connect to the timeline-service hbase cluster. If empty or not
|
||||||
|
specified, then the HBase configuration will be loaded from the classpath.
|
||||||
|
When specified the values in the specified configuration file will override
|
||||||
|
those from the ones that are present on the classpath.
|
||||||
|
</description>
|
||||||
|
<name>yarn.timeline-service.hbase.configuration.file
|
||||||
|
</name>
|
||||||
|
<value></value>
|
||||||
|
</property>
|
||||||
|
|
||||||
<!-- Shared Cache Configuration -->
|
<!-- Shared Cache Configuration -->
|
||||||
|
|
||||||
<property>
|
<property>
|
||||||
@ -3145,6 +3178,17 @@
|
|||||||
<value>64</value>
|
<value>64</value>
|
||||||
</property>
|
</property>
|
||||||
|
|
||||||
|
<property>
|
||||||
|
<description>
|
||||||
|
Flag to enable cross-origin (CORS) support for timeline service v1.x or
|
||||||
|
Timeline Reader in timeline service v2. For timeline service v2, also add
|
||||||
|
org.apache.hadoop.security.HttpCrossOriginFilterInitializer to the
|
||||||
|
configuration hadoop.http.filter.initializers in core-site.xml.
|
||||||
|
</description>
|
||||||
|
<name>yarn.timeline-service.http-cross-origin.enabled</name>
|
||||||
|
<value>false</value>
|
||||||
|
</property>
|
||||||
|
|
||||||
<property>
|
<property>
|
||||||
<description>
|
<description>
|
||||||
Flag to enable cross-origin (CORS) support for timeline service v1.x or
|
Flag to enable cross-origin (CORS) support for timeline service v1.x or
|
||||||
|
@ -101,6 +101,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersRequestP
|
|||||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersResponsePBImpl;
|
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersResponsePBImpl;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationRequestPBImpl;
|
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationRequestPBImpl;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationResponsePBImpl;
|
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationResponsePBImpl;
|
||||||
|
import org.apache.hadoop.yarn.api.records.CollectorInfo;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
@ -406,6 +407,7 @@ public class TestPBImplRecords extends BasePBImplRecordsTest {
|
|||||||
generateByNewInstance(CommitResponse.class);
|
generateByNewInstance(CommitResponse.class);
|
||||||
generateByNewInstance(ApplicationTimeout.class);
|
generateByNewInstance(ApplicationTimeout.class);
|
||||||
generateByNewInstance(QueueConfigurations.class);
|
generateByNewInstance(QueueConfigurations.class);
|
||||||
|
generateByNewInstance(CollectorInfo.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -18,6 +18,11 @@
|
|||||||
|
|
||||||
package org.apache.hadoop.yarn.client.api.impl;
|
package org.apache.hadoop.yarn.client.api.impl;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertNotEquals;
|
||||||
|
import static org.junit.Assert.assertNotNull;
|
||||||
|
import static org.junit.Assert.assertNull;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.net.URI;
|
import java.net.URI;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
@ -27,11 +32,16 @@ import javax.ws.rs.core.MultivaluedMap;
|
|||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.io.Text;
|
||||||
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
|
import org.apache.hadoop.yarn.api.records.CollectorInfo;
|
||||||
|
import org.apache.hadoop.yarn.api.records.Token;
|
||||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
|
||||||
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
|
||||||
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.client.TimelineDelegationTokenIdentifier;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
@ -151,7 +161,7 @@ public class TestTimelineClientV2Impl {
|
|||||||
maxRetries);
|
maxRetries);
|
||||||
c.init(conf);
|
c.init(conf);
|
||||||
c.start();
|
c.start();
|
||||||
c.setTimelineServiceAddress("localhost:12345");
|
c.setTimelineCollectorInfo(CollectorInfo.newInstance("localhost:12345"));
|
||||||
try {
|
try {
|
||||||
c.putEntities(new TimelineEntity());
|
c.putEntities(new TimelineEntity());
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
@ -310,6 +320,50 @@ public class TestTimelineClientV2Impl {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSetTimelineToken() throws Exception {
|
||||||
|
UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
|
||||||
|
assertEquals(0, ugi.getTokens().size());
|
||||||
|
assertNull("Timeline token in v2 client should not be set",
|
||||||
|
client.currentTimelineToken);
|
||||||
|
|
||||||
|
Token token = Token.newInstance(
|
||||||
|
new byte[0], "kind", new byte[0], "service");
|
||||||
|
client.setTimelineCollectorInfo(CollectorInfo.newInstance(null, token));
|
||||||
|
assertNull("Timeline token in v2 client should not be set as token kind " +
|
||||||
|
"is unexepcted.", client.currentTimelineToken);
|
||||||
|
assertEquals(0, ugi.getTokens().size());
|
||||||
|
|
||||||
|
token = Token.newInstance(new byte[0], TimelineDelegationTokenIdentifier.
|
||||||
|
KIND_NAME.toString(), new byte[0], null);
|
||||||
|
client.setTimelineCollectorInfo(CollectorInfo.newInstance(null, token));
|
||||||
|
assertNull("Timeline token in v2 client should not be set as serice is " +
|
||||||
|
"not set.", client.currentTimelineToken);
|
||||||
|
assertEquals(0, ugi.getTokens().size());
|
||||||
|
|
||||||
|
TimelineDelegationTokenIdentifier ident =
|
||||||
|
new TimelineDelegationTokenIdentifier(new Text(ugi.getUserName()),
|
||||||
|
new Text("renewer"), null);
|
||||||
|
ident.setSequenceNumber(1);
|
||||||
|
token = Token.newInstance(ident.getBytes(),
|
||||||
|
TimelineDelegationTokenIdentifier.KIND_NAME.toString(), new byte[0],
|
||||||
|
"localhost:1234");
|
||||||
|
client.setTimelineCollectorInfo(CollectorInfo.newInstance(null, token));
|
||||||
|
assertEquals(1, ugi.getTokens().size());
|
||||||
|
assertNotNull("Timeline token should be set in v2 client.",
|
||||||
|
client.currentTimelineToken);
|
||||||
|
assertEquals(token, client.currentTimelineToken);
|
||||||
|
|
||||||
|
ident.setSequenceNumber(20);
|
||||||
|
Token newToken = Token.newInstance(ident.getBytes(),
|
||||||
|
TimelineDelegationTokenIdentifier.KIND_NAME.toString(), new byte[0],
|
||||||
|
"localhost:1234");
|
||||||
|
client.setTimelineCollectorInfo(CollectorInfo.newInstance(null, newToken));
|
||||||
|
assertEquals(1, ugi.getTokens().size());
|
||||||
|
assertNotEquals(token, client.currentTimelineToken);
|
||||||
|
assertEquals(newToken, client.currentTimelineToken);
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testAfterStop() throws Exception {
|
public void testAfterStop() throws Exception {
|
||||||
client.setSleepBeforeReturn(true);
|
client.setSleepBeforeReturn(true);
|
||||||
|
@ -20,14 +20,14 @@ package org.apache.hadoop.yarn.server.applicationhistoryservice;
|
|||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.net.InetSocketAddress;
|
import java.net.InetSocketAddress;
|
||||||
import java.util.ArrayList;
|
import java.util.LinkedHashSet;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.http.HttpServer2;
|
import org.apache.hadoop.http.HttpServer2;
|
||||||
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
||||||
import org.apache.hadoop.metrics2.source.JvmMetrics;
|
import org.apache.hadoop.metrics2.source.JvmMetrics;
|
||||||
import org.apache.hadoop.security.AuthenticationFilterInitializer;
|
|
||||||
import org.apache.hadoop.security.HttpCrossOriginFilterInitializer;
|
import org.apache.hadoop.security.HttpCrossOriginFilterInitializer;
|
||||||
import org.apache.hadoop.security.SecurityUtil;
|
import org.apache.hadoop.security.SecurityUtil;
|
||||||
import org.apache.hadoop.service.CompositeService;
|
import org.apache.hadoop.service.CompositeService;
|
||||||
@ -47,10 +47,9 @@ import org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore;
|
|||||||
import org.apache.hadoop.yarn.server.timeline.TimelineDataManager;
|
import org.apache.hadoop.yarn.server.timeline.TimelineDataManager;
|
||||||
import org.apache.hadoop.yarn.server.timeline.TimelineStore;
|
import org.apache.hadoop.yarn.server.timeline.TimelineStore;
|
||||||
import org.apache.hadoop.yarn.server.timeline.security.TimelineACLsManager;
|
import org.apache.hadoop.yarn.server.timeline.security.TimelineACLsManager;
|
||||||
import org.apache.hadoop.yarn.server.timeline.security.TimelineAuthenticationFilter;
|
import org.apache.hadoop.yarn.server.timeline.security.TimelineV1DelegationTokenSecretManagerService;
|
||||||
import org.apache.hadoop.yarn.server.timeline.security.TimelineAuthenticationFilterInitializer;
|
|
||||||
import org.apache.hadoop.yarn.server.timeline.security.TimelineDelegationTokenSecretManagerService;
|
|
||||||
import org.apache.hadoop.yarn.server.timeline.webapp.CrossOriginFilterInitializer;
|
import org.apache.hadoop.yarn.server.timeline.webapp.CrossOriginFilterInitializer;
|
||||||
|
import org.apache.hadoop.yarn.server.util.timeline.TimelineServerUtils;
|
||||||
import org.apache.hadoop.yarn.webapp.WebApp;
|
import org.apache.hadoop.yarn.webapp.WebApp;
|
||||||
import org.apache.hadoop.yarn.webapp.WebApps;
|
import org.apache.hadoop.yarn.webapp.WebApps;
|
||||||
import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
|
import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
|
||||||
@ -75,7 +74,7 @@ public class ApplicationHistoryServer extends CompositeService {
|
|||||||
private ApplicationACLsManager aclsManager;
|
private ApplicationACLsManager aclsManager;
|
||||||
private ApplicationHistoryManager historyManager;
|
private ApplicationHistoryManager historyManager;
|
||||||
private TimelineStore timelineStore;
|
private TimelineStore timelineStore;
|
||||||
private TimelineDelegationTokenSecretManagerService secretManagerService;
|
private TimelineV1DelegationTokenSecretManagerService secretManagerService;
|
||||||
private TimelineDataManager timelineDataManager;
|
private TimelineDataManager timelineDataManager;
|
||||||
private WebApp webApp;
|
private WebApp webApp;
|
||||||
private JvmPauseMonitor pauseMonitor;
|
private JvmPauseMonitor pauseMonitor;
|
||||||
@ -223,9 +222,9 @@ public class ApplicationHistoryServer extends CompositeService {
|
|||||||
TimelineStore.class), conf);
|
TimelineStore.class), conf);
|
||||||
}
|
}
|
||||||
|
|
||||||
private TimelineDelegationTokenSecretManagerService
|
private TimelineV1DelegationTokenSecretManagerService
|
||||||
createTimelineDelegationTokenSecretManagerService(Configuration conf) {
|
createTimelineDelegationTokenSecretManagerService(Configuration conf) {
|
||||||
return new TimelineDelegationTokenSecretManagerService();
|
return new TimelineV1DelegationTokenSecretManagerService();
|
||||||
}
|
}
|
||||||
|
|
||||||
private TimelineDataManager createTimelineDataManager(Configuration conf) {
|
private TimelineDataManager createTimelineDataManager(Configuration conf) {
|
||||||
@ -237,63 +236,33 @@ public class ApplicationHistoryServer extends CompositeService {
|
|||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
private void startWebApp() {
|
private void startWebApp() {
|
||||||
Configuration conf = getConfig();
|
Configuration conf = getConfig();
|
||||||
TimelineAuthenticationFilter.setTimelineDelegationTokenSecretManager(
|
|
||||||
secretManagerService.getTimelineDelegationTokenSecretManager());
|
|
||||||
// Always load pseudo authentication filter to parse "user.name" in an URL
|
// Always load pseudo authentication filter to parse "user.name" in an URL
|
||||||
// to identify a HTTP request's user in insecure mode.
|
// to identify a HTTP request's user in insecure mode.
|
||||||
// When Kerberos authentication type is set (i.e., secure mode is turned on),
|
// When Kerberos authentication type is set (i.e., secure mode is turned on),
|
||||||
// the customized filter will be loaded by the timeline server to do Kerberos
|
// the customized filter will be loaded by the timeline server to do Kerberos
|
||||||
// + DT authentication.
|
// + DT authentication.
|
||||||
String initializers = conf.get("hadoop.http.filter.initializers");
|
String initializers = conf.get("hadoop.http.filter.initializers", "");
|
||||||
boolean modifiedInitializers = false;
|
Set<String> defaultInitializers = new LinkedHashSet<String>();
|
||||||
|
// Add CORS filter
|
||||||
initializers =
|
|
||||||
initializers == null || initializers.length() == 0 ? "" : initializers;
|
|
||||||
|
|
||||||
if (!initializers.contains(CrossOriginFilterInitializer.class.getName())) {
|
if (!initializers.contains(CrossOriginFilterInitializer.class.getName())) {
|
||||||
if(conf.getBoolean(YarnConfiguration
|
if(conf.getBoolean(YarnConfiguration.
|
||||||
.TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED, YarnConfiguration
|
TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED,
|
||||||
.TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED_DEFAULT)) {
|
YarnConfiguration.
|
||||||
if (initializers.contains(HttpCrossOriginFilterInitializer.class.getName())) {
|
TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED_DEFAULT)) {
|
||||||
initializers =
|
if (initializers.contains(
|
||||||
initializers.replaceAll(HttpCrossOriginFilterInitializer.class.getName(),
|
HttpCrossOriginFilterInitializer.class.getName())) {
|
||||||
|
initializers = initializers.replaceAll(
|
||||||
|
HttpCrossOriginFilterInitializer.class.getName(),
|
||||||
CrossOriginFilterInitializer.class.getName());
|
CrossOriginFilterInitializer.class.getName());
|
||||||
|
} else {
|
||||||
|
defaultInitializers.add(CrossOriginFilterInitializer.class.getName());
|
||||||
}
|
}
|
||||||
else {
|
|
||||||
if (initializers.length() != 0) {
|
|
||||||
initializers += ",";
|
|
||||||
}
|
|
||||||
initializers += CrossOriginFilterInitializer.class.getName();
|
|
||||||
}
|
|
||||||
modifiedInitializers = true;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
TimelineServerUtils.addTimelineAuthFilter(
|
||||||
if (!initializers.contains(TimelineAuthenticationFilterInitializer.class
|
initializers, defaultInitializers, secretManagerService);
|
||||||
.getName())) {
|
TimelineServerUtils.setTimelineFilters(
|
||||||
if (initializers.length() != 0) {
|
conf, initializers, defaultInitializers);
|
||||||
initializers += ",";
|
|
||||||
}
|
|
||||||
initializers += TimelineAuthenticationFilterInitializer.class.getName();
|
|
||||||
modifiedInitializers = true;
|
|
||||||
}
|
|
||||||
|
|
||||||
String[] parts = initializers.split(",");
|
|
||||||
ArrayList<String> target = new ArrayList<String>();
|
|
||||||
for (String filterInitializer : parts) {
|
|
||||||
filterInitializer = filterInitializer.trim();
|
|
||||||
if (filterInitializer.equals(AuthenticationFilterInitializer.class
|
|
||||||
.getName())) {
|
|
||||||
modifiedInitializers = true;
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
target.add(filterInitializer);
|
|
||||||
}
|
|
||||||
String actualInitializers =
|
|
||||||
org.apache.commons.lang.StringUtils.join(target, ",");
|
|
||||||
if (modifiedInitializers) {
|
|
||||||
conf.set("hadoop.http.filter.initializers", actualInitializers);
|
|
||||||
}
|
|
||||||
String bindAddress = WebAppUtils.getWebAppBindURL(conf,
|
String bindAddress = WebAppUtils.getWebAppBindURL(conf,
|
||||||
YarnConfiguration.TIMELINE_SERVICE_BIND_HOST,
|
YarnConfiguration.TIMELINE_SERVICE_BIND_HOST,
|
||||||
WebAppUtils.getAHSWebAppURLWithoutScheme(conf));
|
WebAppUtils.getAHSWebAppURLWithoutScheme(conf));
|
||||||
|
@ -26,7 +26,6 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
|||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
|
import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
|
||||||
import org.apache.hadoop.security.token.delegation.DelegationKey;
|
import org.apache.hadoop.security.token.delegation.DelegationKey;
|
||||||
import org.apache.hadoop.service.AbstractService;
|
|
||||||
import org.apache.hadoop.util.ReflectionUtils;
|
import org.apache.hadoop.util.ReflectionUtils;
|
||||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||||
import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
|
import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
|
||||||
@ -37,18 +36,16 @@ import org.slf4j.Logger;
|
|||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The service wrapper of {@link TimelineDelegationTokenSecretManager}
|
* The service wrapper of {@link TimelineV1DelegationTokenSecretManager}.
|
||||||
*/
|
*/
|
||||||
@Private
|
@Private
|
||||||
@Unstable
|
@Unstable
|
||||||
public class TimelineDelegationTokenSecretManagerService extends
|
public class TimelineV1DelegationTokenSecretManagerService extends
|
||||||
AbstractService {
|
TimelineDelgationTokenSecretManagerService {
|
||||||
|
|
||||||
private TimelineDelegationTokenSecretManager secretManager = null;
|
|
||||||
private TimelineStateStore stateStore = null;
|
private TimelineStateStore stateStore = null;
|
||||||
|
|
||||||
public TimelineDelegationTokenSecretManagerService() {
|
public TimelineV1DelegationTokenSecretManagerService() {
|
||||||
super(TimelineDelegationTokenSecretManagerService.class.getName());
|
super(TimelineV1DelegationTokenSecretManagerService.class.getName());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -58,19 +55,7 @@ public class TimelineDelegationTokenSecretManagerService extends
|
|||||||
stateStore = createStateStore(conf);
|
stateStore = createStateStore(conf);
|
||||||
stateStore.init(conf);
|
stateStore.init(conf);
|
||||||
}
|
}
|
||||||
|
super.serviceInit(conf);
|
||||||
long secretKeyInterval =
|
|
||||||
conf.getLong(YarnConfiguration.TIMELINE_DELEGATION_KEY_UPDATE_INTERVAL,
|
|
||||||
YarnConfiguration.DEFAULT_TIMELINE_DELEGATION_KEY_UPDATE_INTERVAL);
|
|
||||||
long tokenMaxLifetime =
|
|
||||||
conf.getLong(YarnConfiguration.TIMELINE_DELEGATION_TOKEN_MAX_LIFETIME,
|
|
||||||
YarnConfiguration.DEFAULT_TIMELINE_DELEGATION_TOKEN_MAX_LIFETIME);
|
|
||||||
long tokenRenewInterval =
|
|
||||||
conf.getLong(YarnConfiguration.TIMELINE_DELEGATION_TOKEN_RENEW_INTERVAL,
|
|
||||||
YarnConfiguration.DEFAULT_TIMELINE_DELEGATION_TOKEN_RENEW_INTERVAL);
|
|
||||||
secretManager = new TimelineDelegationTokenSecretManager(secretKeyInterval,
|
|
||||||
tokenMaxLifetime, tokenRenewInterval, 3600000, stateStore);
|
|
||||||
super.init(conf);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -78,10 +63,9 @@ public class TimelineDelegationTokenSecretManagerService extends
|
|||||||
if (stateStore != null) {
|
if (stateStore != null) {
|
||||||
stateStore.start();
|
stateStore.start();
|
||||||
TimelineServiceState state = stateStore.loadState();
|
TimelineServiceState state = stateStore.loadState();
|
||||||
secretManager.recover(state);
|
((TimelineV1DelegationTokenSecretManager)
|
||||||
|
getTimelineDelegationTokenSecretManager()).recover(state);
|
||||||
}
|
}
|
||||||
|
|
||||||
secretManager.startThreads();
|
|
||||||
super.serviceStart();
|
super.serviceStart();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -90,9 +74,18 @@ public class TimelineDelegationTokenSecretManagerService extends
|
|||||||
if (stateStore != null) {
|
if (stateStore != null) {
|
||||||
stateStore.stop();
|
stateStore.stop();
|
||||||
}
|
}
|
||||||
|
super.serviceStop();
|
||||||
|
}
|
||||||
|
|
||||||
secretManager.stopThreads();
|
@Override
|
||||||
super.stop();
|
protected AbstractDelegationTokenSecretManager
|
||||||
|
<TimelineDelegationTokenIdentifier>
|
||||||
|
createTimelineDelegationTokenSecretManager(long secretKeyInterval,
|
||||||
|
long tokenMaxLifetime, long tokenRenewInterval,
|
||||||
|
long tokenRemovalScanInterval) {
|
||||||
|
return new TimelineV1DelegationTokenSecretManager(secretKeyInterval,
|
||||||
|
tokenMaxLifetime, tokenRenewInterval, tokenRemovalScanInterval,
|
||||||
|
stateStore);
|
||||||
}
|
}
|
||||||
|
|
||||||
protected TimelineStateStore createStateStore(
|
protected TimelineStateStore createStateStore(
|
||||||
@ -104,27 +97,20 @@ public class TimelineDelegationTokenSecretManagerService extends
|
|||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Ge the instance of {link #TimelineDelegationTokenSecretManager}
|
* Delegation token secret manager for ATSv1 and ATSv1.5.
|
||||||
*
|
|
||||||
* @return the instance of {link #TimelineDelegationTokenSecretManager}
|
|
||||||
*/
|
*/
|
||||||
public TimelineDelegationTokenSecretManager
|
|
||||||
getTimelineDelegationTokenSecretManager() {
|
|
||||||
return secretManager;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Private
|
@Private
|
||||||
@Unstable
|
@Unstable
|
||||||
public static class TimelineDelegationTokenSecretManager extends
|
public static class TimelineV1DelegationTokenSecretManager extends
|
||||||
AbstractDelegationTokenSecretManager<TimelineDelegationTokenIdentifier> {
|
AbstractDelegationTokenSecretManager<TimelineDelegationTokenIdentifier> {
|
||||||
|
|
||||||
public static final Logger LOG =
|
public static final Logger LOG =
|
||||||
LoggerFactory.getLogger(TimelineDelegationTokenSecretManager.class);
|
LoggerFactory.getLogger(TimelineV1DelegationTokenSecretManager.class);
|
||||||
|
|
||||||
private TimelineStateStore stateStore;
|
private TimelineStateStore stateStore;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a timeline secret manager
|
* Create a timeline v1 secret manager.
|
||||||
* @param delegationKeyUpdateInterval the number of milliseconds for rolling
|
* @param delegationKeyUpdateInterval the number of milliseconds for rolling
|
||||||
* new secret keys.
|
* new secret keys.
|
||||||
* @param delegationTokenMaxLifetime the maximum lifetime of the delegation
|
* @param delegationTokenMaxLifetime the maximum lifetime of the delegation
|
||||||
@ -135,7 +121,7 @@ public class TimelineDelegationTokenSecretManagerService extends
|
|||||||
* scanned for expired tokens in milliseconds
|
* scanned for expired tokens in milliseconds
|
||||||
* @param stateStore timeline service state store
|
* @param stateStore timeline service state store
|
||||||
*/
|
*/
|
||||||
public TimelineDelegationTokenSecretManager(
|
public TimelineV1DelegationTokenSecretManager(
|
||||||
long delegationKeyUpdateInterval,
|
long delegationKeyUpdateInterval,
|
||||||
long delegationTokenMaxLifetime,
|
long delegationTokenMaxLifetime,
|
||||||
long delegationTokenRenewInterval,
|
long delegationTokenRenewInterval,
|
||||||
@ -236,5 +222,4 @@ public class TimelineDelegationTokenSecretManagerService extends
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
@ -55,27 +55,31 @@ import org.junit.Test;
|
|||||||
import org.junit.runner.RunWith;
|
import org.junit.runner.RunWith;
|
||||||
import org.junit.runners.Parameterized;
|
import org.junit.runners.Parameterized;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test cases for authentication via TimelineAuthenticationFilter while
|
||||||
|
* publishing entities for ATSv1.
|
||||||
|
*/
|
||||||
@RunWith(Parameterized.class)
|
@RunWith(Parameterized.class)
|
||||||
public class TestTimelineAuthenticationFilter {
|
public class TestTimelineAuthenticationFilterForV1 {
|
||||||
|
|
||||||
private static final String FOO_USER = "foo";
|
private static final String FOO_USER = "foo";
|
||||||
private static final String BAR_USER = "bar";
|
private static final String BAR_USER = "bar";
|
||||||
private static final String HTTP_USER = "HTTP";
|
private static final String HTTP_USER = "HTTP";
|
||||||
|
|
||||||
private static final File testRootDir = new File(
|
private static final File TEST_ROOT_DIR = new File(
|
||||||
System.getProperty("test.build.dir", "target/test-dir"),
|
System.getProperty("test.build.dir", "target/test-dir"),
|
||||||
TestTimelineAuthenticationFilter.class.getName() + "-root");
|
TestTimelineAuthenticationFilterForV1.class.getName() + "-root");
|
||||||
private static File httpSpnegoKeytabFile = new File(
|
private static File httpSpnegoKeytabFile = new File(
|
||||||
KerberosTestUtils.getKeytabFile());
|
KerberosTestUtils.getKeytabFile());
|
||||||
private static String httpSpnegoPrincipal =
|
private static String httpSpnegoPrincipal =
|
||||||
KerberosTestUtils.getServerPrincipal();
|
KerberosTestUtils.getServerPrincipal();
|
||||||
private static final String BASEDIR =
|
private static final String BASEDIR =
|
||||||
System.getProperty("test.build.dir", "target/test-dir") + "/"
|
System.getProperty("test.build.dir", "target/test-dir") + "/"
|
||||||
+ TestTimelineAuthenticationFilter.class.getSimpleName();
|
+ TestTimelineAuthenticationFilterForV1.class.getSimpleName();
|
||||||
|
|
||||||
@Parameterized.Parameters
|
@Parameterized.Parameters
|
||||||
public static Collection<Object[]> withSsl() {
|
public static Collection<Object[]> withSsl() {
|
||||||
return Arrays.asList(new Object[][] { { false }, { true } });
|
return Arrays.asList(new Object[][] {{false}, {true}});
|
||||||
}
|
}
|
||||||
|
|
||||||
private static MiniKdc testMiniKDC;
|
private static MiniKdc testMiniKDC;
|
||||||
@ -85,14 +89,14 @@ public class TestTimelineAuthenticationFilter {
|
|||||||
private static Configuration conf;
|
private static Configuration conf;
|
||||||
private static boolean withSsl;
|
private static boolean withSsl;
|
||||||
|
|
||||||
public TestTimelineAuthenticationFilter(boolean withSsl) {
|
public TestTimelineAuthenticationFilterForV1(boolean withSsl) {
|
||||||
TestTimelineAuthenticationFilter.withSsl = withSsl;
|
TestTimelineAuthenticationFilterForV1.withSsl = withSsl;
|
||||||
}
|
}
|
||||||
|
|
||||||
@BeforeClass
|
@BeforeClass
|
||||||
public static void setup() {
|
public static void setup() {
|
||||||
try {
|
try {
|
||||||
testMiniKDC = new MiniKdc(MiniKdc.createConf(), testRootDir);
|
testMiniKDC = new MiniKdc(MiniKdc.createConf(), TEST_ROOT_DIR);
|
||||||
testMiniKDC.start();
|
testMiniKDC.start();
|
||||||
testMiniKDC.createPrincipal(
|
testMiniKDC.createPrincipal(
|
||||||
httpSpnegoKeytabFile, HTTP_USER + "/localhost");
|
httpSpnegoKeytabFile, HTTP_USER + "/localhost");
|
||||||
@ -111,11 +115,11 @@ public class TestTimelineAuthenticationFilter {
|
|||||||
KerberosAuthenticationHandler.KEYTAB,
|
KerberosAuthenticationHandler.KEYTAB,
|
||||||
httpSpnegoKeytabFile.getAbsolutePath());
|
httpSpnegoKeytabFile.getAbsolutePath());
|
||||||
conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
|
conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
|
||||||
"kerberos");
|
"kerberos");
|
||||||
conf.set(YarnConfiguration.TIMELINE_SERVICE_PRINCIPAL,
|
conf.set(YarnConfiguration.TIMELINE_SERVICE_PRINCIPAL,
|
||||||
httpSpnegoPrincipal);
|
httpSpnegoPrincipal);
|
||||||
conf.set(YarnConfiguration.TIMELINE_SERVICE_KEYTAB,
|
conf.set(YarnConfiguration.TIMELINE_SERVICE_KEYTAB,
|
||||||
httpSpnegoKeytabFile.getAbsolutePath());
|
httpSpnegoKeytabFile.getAbsolutePath());
|
||||||
conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
|
conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
|
||||||
conf.setClass(YarnConfiguration.TIMELINE_SERVICE_STORE,
|
conf.setClass(YarnConfiguration.TIMELINE_SERVICE_STORE,
|
||||||
MemoryTimelineStore.class, TimelineStore.class);
|
MemoryTimelineStore.class, TimelineStore.class);
|
||||||
@ -136,8 +140,8 @@ public class TestTimelineAuthenticationFilter {
|
|||||||
FileUtil.fullyDelete(base);
|
FileUtil.fullyDelete(base);
|
||||||
base.mkdirs();
|
base.mkdirs();
|
||||||
keystoresDir = new File(BASEDIR).getAbsolutePath();
|
keystoresDir = new File(BASEDIR).getAbsolutePath();
|
||||||
sslConfDir =
|
sslConfDir = KeyStoreTestUtil.getClasspathDir(
|
||||||
KeyStoreTestUtil.getClasspathDir(TestTimelineAuthenticationFilter.class);
|
TestTimelineAuthenticationFilterForV1.class);
|
||||||
KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, conf, false);
|
KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, conf, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -145,6 +149,7 @@ public class TestTimelineAuthenticationFilter {
|
|||||||
testTimelineServer.init(conf);
|
testTimelineServer.init(conf);
|
||||||
testTimelineServer.start();
|
testTimelineServer.start();
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
|
e.printStackTrace();
|
||||||
assertTrue("Couldn't setup TimelineServer", false);
|
assertTrue("Couldn't setup TimelineServer", false);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -181,14 +186,14 @@ public class TestTimelineAuthenticationFilter {
|
|||||||
TimelineClient client = createTimelineClientForUGI();
|
TimelineClient client = createTimelineClientForUGI();
|
||||||
TimelineEntity entityToStore = new TimelineEntity();
|
TimelineEntity entityToStore = new TimelineEntity();
|
||||||
entityToStore.setEntityType(
|
entityToStore.setEntityType(
|
||||||
TestTimelineAuthenticationFilter.class.getName());
|
TestTimelineAuthenticationFilterForV1.class.getName());
|
||||||
entityToStore.setEntityId("entity1");
|
entityToStore.setEntityId("entity1");
|
||||||
entityToStore.setStartTime(0L);
|
entityToStore.setStartTime(0L);
|
||||||
TimelinePutResponse putResponse = client.putEntities(entityToStore);
|
TimelinePutResponse putResponse = client.putEntities(entityToStore);
|
||||||
Assert.assertEquals(0, putResponse.getErrors().size());
|
Assert.assertEquals(0, putResponse.getErrors().size());
|
||||||
TimelineEntity entityToRead =
|
TimelineEntity entityToRead =
|
||||||
testTimelineServer.getTimelineStore().getEntity(
|
testTimelineServer.getTimelineStore().getEntity("entity1",
|
||||||
"entity1", TestTimelineAuthenticationFilter.class.getName(), null);
|
TestTimelineAuthenticationFilterForV1.class.getName(), null);
|
||||||
Assert.assertNotNull(entityToRead);
|
Assert.assertNotNull(entityToRead);
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
@ -202,13 +207,14 @@ public class TestTimelineAuthenticationFilter {
|
|||||||
public Void call() throws Exception {
|
public Void call() throws Exception {
|
||||||
TimelineClient client = createTimelineClientForUGI();
|
TimelineClient client = createTimelineClientForUGI();
|
||||||
TimelineDomain domainToStore = new TimelineDomain();
|
TimelineDomain domainToStore = new TimelineDomain();
|
||||||
domainToStore.setId(TestTimelineAuthenticationFilter.class.getName());
|
domainToStore.setId(
|
||||||
|
TestTimelineAuthenticationFilterForV1.class.getName());
|
||||||
domainToStore.setReaders("*");
|
domainToStore.setReaders("*");
|
||||||
domainToStore.setWriters("*");
|
domainToStore.setWriters("*");
|
||||||
client.putDomain(domainToStore);
|
client.putDomain(domainToStore);
|
||||||
TimelineDomain domainToRead =
|
TimelineDomain domainToRead =
|
||||||
testTimelineServer.getTimelineStore().getDomain(
|
testTimelineServer.getTimelineStore().getDomain(
|
||||||
TestTimelineAuthenticationFilter.class.getName());
|
TestTimelineAuthenticationFilterForV1.class.getName());
|
||||||
Assert.assertNotNull(domainToRead);
|
Assert.assertNotNull(domainToRead);
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
@ -218,22 +224,24 @@ public class TestTimelineAuthenticationFilter {
|
|||||||
@Test
|
@Test
|
||||||
public void testDelegationTokenOperations() throws Exception {
|
public void testDelegationTokenOperations() throws Exception {
|
||||||
TimelineClient httpUserClient =
|
TimelineClient httpUserClient =
|
||||||
KerberosTestUtils.doAs(HTTP_USER + "/localhost", new Callable<TimelineClient>() {
|
KerberosTestUtils.doAs(HTTP_USER + "/localhost",
|
||||||
@Override
|
new Callable<TimelineClient>() {
|
||||||
public TimelineClient call() throws Exception {
|
@Override
|
||||||
return createTimelineClientForUGI();
|
public TimelineClient call() throws Exception {
|
||||||
}
|
return createTimelineClientForUGI();
|
||||||
});
|
}
|
||||||
|
});
|
||||||
UserGroupInformation httpUser =
|
UserGroupInformation httpUser =
|
||||||
KerberosTestUtils.doAs(HTTP_USER + "/localhost", new Callable<UserGroupInformation>() {
|
KerberosTestUtils.doAs(HTTP_USER + "/localhost",
|
||||||
@Override
|
new Callable<UserGroupInformation>() {
|
||||||
public UserGroupInformation call() throws Exception {
|
@Override
|
||||||
return UserGroupInformation.getCurrentUser();
|
public UserGroupInformation call() throws Exception {
|
||||||
}
|
return UserGroupInformation.getCurrentUser();
|
||||||
});
|
}
|
||||||
|
});
|
||||||
// Let HTTP user to get the delegation for itself
|
// Let HTTP user to get the delegation for itself
|
||||||
Token<TimelineDelegationTokenIdentifier> token =
|
Token<TimelineDelegationTokenIdentifier> token =
|
||||||
httpUserClient.getDelegationToken(httpUser.getShortUserName());
|
httpUserClient.getDelegationToken(httpUser.getShortUserName());
|
||||||
Assert.assertNotNull(token);
|
Assert.assertNotNull(token);
|
||||||
TimelineDelegationTokenIdentifier tDT = token.decodeIdentifier();
|
TimelineDelegationTokenIdentifier tDT = token.decodeIdentifier();
|
||||||
Assert.assertNotNull(tDT);
|
Assert.assertNotNull(tDT);
|
||||||
@ -317,7 +325,8 @@ public class TestTimelineAuthenticationFilter {
|
|||||||
barUserClient.getDelegationToken(httpUser.getShortUserName());
|
barUserClient.getDelegationToken(httpUser.getShortUserName());
|
||||||
Assert.fail();
|
Assert.fail();
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
Assert.assertTrue(e.getCause() instanceof AuthorizationException || e.getCause() instanceof AuthenticationException);
|
Assert.assertTrue(e.getCause() instanceof AuthorizationException ||
|
||||||
|
e.getCause() instanceof AuthenticationException);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
@ -24,6 +24,7 @@ import java.util.Set;
|
|||||||
|
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
import org.apache.hadoop.yarn.api.records.NodeLabel;
|
import org.apache.hadoop.yarn.api.records.NodeLabel;
|
||||||
|
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
||||||
import org.apache.hadoop.yarn.server.api.records.MasterKey;
|
import org.apache.hadoop.yarn.server.api.records.MasterKey;
|
||||||
import org.apache.hadoop.yarn.server.api.records.NodeStatus;
|
import org.apache.hadoop.yarn.server.api.records.NodeStatus;
|
||||||
import org.apache.hadoop.yarn.util.Records;
|
import org.apache.hadoop.yarn.util.Records;
|
||||||
@ -47,7 +48,7 @@ public abstract class NodeHeartbeatRequest {
|
|||||||
public static NodeHeartbeatRequest newInstance(NodeStatus nodeStatus,
|
public static NodeHeartbeatRequest newInstance(NodeStatus nodeStatus,
|
||||||
MasterKey lastKnownContainerTokenMasterKey,
|
MasterKey lastKnownContainerTokenMasterKey,
|
||||||
MasterKey lastKnownNMTokenMasterKey, Set<NodeLabel> nodeLabels,
|
MasterKey lastKnownNMTokenMasterKey, Set<NodeLabel> nodeLabels,
|
||||||
Map<ApplicationId, String> registeredCollectors) {
|
Map<ApplicationId, AppCollectorData> registeringCollectors) {
|
||||||
NodeHeartbeatRequest nodeHeartbeatRequest =
|
NodeHeartbeatRequest nodeHeartbeatRequest =
|
||||||
Records.newRecord(NodeHeartbeatRequest.class);
|
Records.newRecord(NodeHeartbeatRequest.class);
|
||||||
nodeHeartbeatRequest.setNodeStatus(nodeStatus);
|
nodeHeartbeatRequest.setNodeStatus(nodeStatus);
|
||||||
@ -56,7 +57,7 @@ public abstract class NodeHeartbeatRequest {
|
|||||||
nodeHeartbeatRequest
|
nodeHeartbeatRequest
|
||||||
.setLastKnownNMTokenMasterKey(lastKnownNMTokenMasterKey);
|
.setLastKnownNMTokenMasterKey(lastKnownNMTokenMasterKey);
|
||||||
nodeHeartbeatRequest.setNodeLabels(nodeLabels);
|
nodeHeartbeatRequest.setNodeLabels(nodeLabels);
|
||||||
nodeHeartbeatRequest.setRegisteredCollectors(registeredCollectors);
|
nodeHeartbeatRequest.setRegisteringCollectors(registeringCollectors);
|
||||||
return nodeHeartbeatRequest;
|
return nodeHeartbeatRequest;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -79,7 +80,9 @@ public abstract class NodeHeartbeatRequest {
|
|||||||
List<LogAggregationReport> logAggregationReportsForApps);
|
List<LogAggregationReport> logAggregationReportsForApps);
|
||||||
|
|
||||||
// This tells RM registered collectors' address info on this node
|
// This tells RM registered collectors' address info on this node
|
||||||
public abstract Map<ApplicationId, String> getRegisteredCollectors();
|
public abstract Map<ApplicationId, AppCollectorData>
|
||||||
public abstract void setRegisteredCollectors(Map<ApplicationId,
|
getRegisteringCollectors();
|
||||||
String> appCollectorsMap);
|
|
||||||
|
public abstract void setRegisteringCollectors(Map<ApplicationId,
|
||||||
|
AppCollectorData> appCollectorsMap);
|
||||||
}
|
}
|
||||||
|
@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|||||||
import org.apache.hadoop.yarn.api.records.Container;
|
import org.apache.hadoop.yarn.api.records.Container;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
import org.apache.hadoop.yarn.api.records.Resource;
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
|
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
||||||
import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
|
import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
|
||||||
import org.apache.hadoop.yarn.server.api.records.MasterKey;
|
import org.apache.hadoop.yarn.server.api.records.MasterKey;
|
||||||
import org.apache.hadoop.yarn.server.api.records.NodeAction;
|
import org.apache.hadoop.yarn.server.api.records.NodeAction;
|
||||||
@ -47,10 +48,9 @@ public abstract class NodeHeartbeatResponse {
|
|||||||
public abstract List<ApplicationId> getApplicationsToCleanup();
|
public abstract List<ApplicationId> getApplicationsToCleanup();
|
||||||
|
|
||||||
// This tells NM the collectors' address info of related apps
|
// This tells NM the collectors' address info of related apps
|
||||||
public abstract Map<ApplicationId, String> getAppCollectorsMap();
|
public abstract Map<ApplicationId, AppCollectorData> getAppCollectors();
|
||||||
|
public abstract void setAppCollectors(
|
||||||
public abstract void setAppCollectorsMap(
|
Map<ApplicationId, AppCollectorData> appCollectorsMap);
|
||||||
Map<ApplicationId, String> appCollectorsMap);
|
|
||||||
|
|
||||||
public abstract void setResponseId(int responseId);
|
public abstract void setResponseId(int responseId);
|
||||||
|
|
||||||
|
@ -22,14 +22,15 @@ import java.util.Arrays;
|
|||||||
|
|
||||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
import org.apache.hadoop.yarn.server.api.records.AppCollectorsMap;
|
import org.apache.hadoop.yarn.api.records.Token;
|
||||||
|
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
||||||
import org.apache.hadoop.yarn.util.Records;
|
import org.apache.hadoop.yarn.util.Records;
|
||||||
|
|
||||||
@Private
|
@Private
|
||||||
public abstract class ReportNewCollectorInfoRequest {
|
public abstract class ReportNewCollectorInfoRequest {
|
||||||
|
|
||||||
public static ReportNewCollectorInfoRequest newInstance(
|
public static ReportNewCollectorInfoRequest newInstance(
|
||||||
List<AppCollectorsMap> appCollectorsList) {
|
List<AppCollectorData> appCollectorsList) {
|
||||||
ReportNewCollectorInfoRequest request =
|
ReportNewCollectorInfoRequest request =
|
||||||
Records.newRecord(ReportNewCollectorInfoRequest.class);
|
Records.newRecord(ReportNewCollectorInfoRequest.class);
|
||||||
request.setAppCollectorsList(appCollectorsList);
|
request.setAppCollectorsList(appCollectorsList);
|
||||||
@ -37,17 +38,17 @@ public abstract class ReportNewCollectorInfoRequest {
|
|||||||
}
|
}
|
||||||
|
|
||||||
public static ReportNewCollectorInfoRequest newInstance(
|
public static ReportNewCollectorInfoRequest newInstance(
|
||||||
ApplicationId id, String collectorAddr) {
|
ApplicationId id, String collectorAddr, Token token) {
|
||||||
ReportNewCollectorInfoRequest request =
|
ReportNewCollectorInfoRequest request =
|
||||||
Records.newRecord(ReportNewCollectorInfoRequest.class);
|
Records.newRecord(ReportNewCollectorInfoRequest.class);
|
||||||
request.setAppCollectorsList(
|
request.setAppCollectorsList(
|
||||||
Arrays.asList(AppCollectorsMap.newInstance(id, collectorAddr)));
|
Arrays.asList(AppCollectorData.newInstance(id, collectorAddr, token)));
|
||||||
return request;
|
return request;
|
||||||
}
|
}
|
||||||
|
|
||||||
public abstract List<AppCollectorsMap> getAppCollectorsList();
|
public abstract List<AppCollectorData> getAppCollectorsList();
|
||||||
|
|
||||||
public abstract void setAppCollectorsList(
|
public abstract void setAppCollectorsList(
|
||||||
List<AppCollectorsMap> appCollectorsList);
|
List<AppCollectorData> appCollectorsList);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -26,16 +26,20 @@ import java.util.List;
|
|||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
|
import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
|
||||||
|
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
import org.apache.hadoop.yarn.api.records.NodeLabel;
|
import org.apache.hadoop.yarn.api.records.NodeLabel;
|
||||||
|
import org.apache.hadoop.yarn.api.records.Token;
|
||||||
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
|
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
|
||||||
import org.apache.hadoop.yarn.api.records.impl.pb.NodeLabelPBImpl;
|
import org.apache.hadoop.yarn.api.records.impl.pb.NodeLabelPBImpl;
|
||||||
|
import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
|
||||||
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
|
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
|
||||||
import org.apache.hadoop.yarn.proto.YarnProtos.NodeLabelProto;
|
import org.apache.hadoop.yarn.proto.YarnProtos.NodeLabelProto;
|
||||||
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto;
|
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto;
|
||||||
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeStatusProto;
|
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeStatusProto;
|
||||||
|
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorDataProto;
|
||||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.LogAggregationReportProto;
|
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.LogAggregationReportProto;
|
||||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorsMapProto;
|
|
||||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatRequestProto;
|
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatRequestProto;
|
||||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatRequestProtoOrBuilder;
|
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatRequestProtoOrBuilder;
|
||||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeLabelsProto;
|
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeLabelsProto;
|
||||||
@ -58,7 +62,7 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
|
|||||||
private Set<NodeLabel> labels = null;
|
private Set<NodeLabel> labels = null;
|
||||||
private List<LogAggregationReport> logAggregationReportsForApps = null;
|
private List<LogAggregationReport> logAggregationReportsForApps = null;
|
||||||
|
|
||||||
private Map<ApplicationId, String> registeredCollectors = null;
|
private Map<ApplicationId, AppCollectorData> registeringCollectors = null;
|
||||||
|
|
||||||
public NodeHeartbeatRequestPBImpl() {
|
public NodeHeartbeatRequestPBImpl() {
|
||||||
builder = NodeHeartbeatRequestProto.newBuilder();
|
builder = NodeHeartbeatRequestProto.newBuilder();
|
||||||
@ -114,8 +118,8 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
|
|||||||
if (this.logAggregationReportsForApps != null) {
|
if (this.logAggregationReportsForApps != null) {
|
||||||
addLogAggregationStatusForAppsToProto();
|
addLogAggregationStatusForAppsToProto();
|
||||||
}
|
}
|
||||||
if (this.registeredCollectors != null) {
|
if (this.registeringCollectors != null) {
|
||||||
addRegisteredCollectorsToProto();
|
addRegisteringCollectorsToProto();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -158,14 +162,23 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
|
|||||||
return ((LogAggregationReportPBImpl) value).getProto();
|
return ((LogAggregationReportPBImpl) value).getProto();
|
||||||
}
|
}
|
||||||
|
|
||||||
private void addRegisteredCollectorsToProto() {
|
private void addRegisteringCollectorsToProto() {
|
||||||
maybeInitBuilder();
|
maybeInitBuilder();
|
||||||
builder.clearRegisteredCollectors();
|
builder.clearRegisteringCollectors();
|
||||||
for (Map.Entry<ApplicationId, String> entry :
|
for (Map.Entry<ApplicationId, AppCollectorData> entry :
|
||||||
registeredCollectors.entrySet()) {
|
registeringCollectors.entrySet()) {
|
||||||
builder.addRegisteredCollectors(AppCollectorsMapProto.newBuilder()
|
AppCollectorData data = entry.getValue();
|
||||||
.setAppId(convertToProtoFormat(entry.getKey()))
|
AppCollectorDataProto.Builder appCollectorDataBuilder =
|
||||||
.setAppCollectorAddr(entry.getValue()));
|
AppCollectorDataProto.newBuilder()
|
||||||
|
.setAppId(convertToProtoFormat(entry.getKey()))
|
||||||
|
.setAppCollectorAddr(data.getCollectorAddr())
|
||||||
|
.setRmIdentifier(data.getRMIdentifier())
|
||||||
|
.setVersion(data.getVersion());
|
||||||
|
if (data.getCollectorToken() != null) {
|
||||||
|
appCollectorDataBuilder.setAppCollectorToken(
|
||||||
|
convertToProtoFormat(data.getCollectorToken()));
|
||||||
|
}
|
||||||
|
builder.addRegisteringCollectors(appCollectorDataBuilder);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -251,35 +264,42 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Map<ApplicationId, String> getRegisteredCollectors() {
|
public Map<ApplicationId, AppCollectorData> getRegisteringCollectors() {
|
||||||
if (this.registeredCollectors != null) {
|
if (this.registeringCollectors != null) {
|
||||||
return this.registeredCollectors;
|
return this.registeringCollectors;
|
||||||
}
|
}
|
||||||
initRegisteredCollectors();
|
initRegisteredCollectors();
|
||||||
return registeredCollectors;
|
return registeringCollectors;
|
||||||
}
|
}
|
||||||
|
|
||||||
private void initRegisteredCollectors() {
|
private void initRegisteredCollectors() {
|
||||||
NodeHeartbeatRequestProtoOrBuilder p = viaProto ? proto : builder;
|
NodeHeartbeatRequestProtoOrBuilder p = viaProto ? proto : builder;
|
||||||
List<AppCollectorsMapProto> list = p.getRegisteredCollectorsList();
|
List<AppCollectorDataProto> list = p.getRegisteringCollectorsList();
|
||||||
if (!list.isEmpty()) {
|
if (!list.isEmpty()) {
|
||||||
this.registeredCollectors = new HashMap<>();
|
this.registeringCollectors = new HashMap<>();
|
||||||
for (AppCollectorsMapProto c : list) {
|
for (AppCollectorDataProto c : list) {
|
||||||
ApplicationId appId = convertFromProtoFormat(c.getAppId());
|
ApplicationId appId = convertFromProtoFormat(c.getAppId());
|
||||||
this.registeredCollectors.put(appId, c.getAppCollectorAddr());
|
Token collectorToken = null;
|
||||||
|
if (c.hasAppCollectorToken()){
|
||||||
|
collectorToken = convertFromProtoFormat(c.getAppCollectorToken());
|
||||||
|
}
|
||||||
|
AppCollectorData data = AppCollectorData.newInstance(appId,
|
||||||
|
c.getAppCollectorAddr(), c.getRmIdentifier(), c.getVersion(),
|
||||||
|
collectorToken);
|
||||||
|
this.registeringCollectors.put(appId, data);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void setRegisteredCollectors(
|
public void setRegisteringCollectors(
|
||||||
Map<ApplicationId, String> registeredCollectors) {
|
Map<ApplicationId, AppCollectorData> registeredCollectors) {
|
||||||
if (registeredCollectors == null || registeredCollectors.isEmpty()) {
|
if (registeredCollectors == null || registeredCollectors.isEmpty()) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
maybeInitBuilder();
|
maybeInitBuilder();
|
||||||
this.registeredCollectors = new HashMap<ApplicationId, String>();
|
this.registeringCollectors = new HashMap<>();
|
||||||
this.registeredCollectors.putAll(registeredCollectors);
|
this.registeringCollectors.putAll(registeredCollectors);
|
||||||
}
|
}
|
||||||
|
|
||||||
private NodeStatusPBImpl convertFromProtoFormat(NodeStatusProto p) {
|
private NodeStatusPBImpl convertFromProtoFormat(NodeStatusProto p) {
|
||||||
@ -306,6 +326,14 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
|
|||||||
return ((MasterKeyPBImpl)t).getProto();
|
return ((MasterKeyPBImpl)t).getProto();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private TokenPBImpl convertFromProtoFormat(TokenProto p) {
|
||||||
|
return new TokenPBImpl(p);
|
||||||
|
}
|
||||||
|
|
||||||
|
private TokenProto convertToProtoFormat(Token t) {
|
||||||
|
return ((TokenPBImpl) t).getProto();
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Set<NodeLabel> getNodeLabels() {
|
public Set<NodeLabel> getNodeLabels() {
|
||||||
initNodeLabels();
|
initNodeLabels();
|
||||||
|
@ -26,26 +26,30 @@ import java.util.Iterator;
|
|||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
|
import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SignalContainerRequestPBImpl;
|
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SignalContainerRequestPBImpl;
|
||||||
|
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
import org.apache.hadoop.yarn.api.records.Container;
|
import org.apache.hadoop.yarn.api.records.Container;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
import org.apache.hadoop.yarn.api.records.Resource;
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
|
import org.apache.hadoop.yarn.api.records.Token;
|
||||||
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
|
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
|
||||||
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
|
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
|
||||||
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
|
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
|
||||||
import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
|
import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
|
||||||
import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
|
import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
|
||||||
|
import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
|
||||||
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
|
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
|
||||||
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
|
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
|
||||||
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
|
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
|
||||||
import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
|
import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
|
||||||
|
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorDataProto;
|
||||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ContainerQueuingLimitProto;
|
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ContainerQueuingLimitProto;
|
||||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.SignalContainerRequestProto;
|
import org.apache.hadoop.yarn.proto.YarnServiceProtos.SignalContainerRequestProto;
|
||||||
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto;
|
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto;
|
||||||
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeActionProto;
|
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeActionProto;
|
||||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorsMapProto;
|
|
||||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatResponseProto;
|
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatResponseProto;
|
||||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatResponseProtoOrBuilder;
|
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatResponseProtoOrBuilder;
|
||||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.SystemCredentialsForAppsProto;
|
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.SystemCredentialsForAppsProto;
|
||||||
@ -70,7 +74,7 @@ public class NodeHeartbeatResponsePBImpl extends NodeHeartbeatResponse {
|
|||||||
private List<ApplicationId> applicationsToCleanup = null;
|
private List<ApplicationId> applicationsToCleanup = null;
|
||||||
private Map<ApplicationId, ByteBuffer> systemCredentials = null;
|
private Map<ApplicationId, ByteBuffer> systemCredentials = null;
|
||||||
private Resource resource = null;
|
private Resource resource = null;
|
||||||
private Map<ApplicationId, String> appCollectorsMap = null;
|
private Map<ApplicationId, AppCollectorData> appCollectorsMap = null;
|
||||||
|
|
||||||
private MasterKey containerTokenMasterKey = null;
|
private MasterKey containerTokenMasterKey = null;
|
||||||
private MasterKey nmTokenMasterKey = null;
|
private MasterKey nmTokenMasterKey = null;
|
||||||
@ -146,11 +150,21 @@ public class NodeHeartbeatResponsePBImpl extends NodeHeartbeatResponse {
|
|||||||
|
|
||||||
private void addAppCollectorsMapToProto() {
|
private void addAppCollectorsMapToProto() {
|
||||||
maybeInitBuilder();
|
maybeInitBuilder();
|
||||||
builder.clearAppCollectorsMap();
|
builder.clearAppCollectors();
|
||||||
for (Map.Entry<ApplicationId, String> entry : appCollectorsMap.entrySet()) {
|
for (Map.Entry<ApplicationId, AppCollectorData> entry
|
||||||
builder.addAppCollectorsMap(AppCollectorsMapProto.newBuilder()
|
: appCollectorsMap.entrySet()) {
|
||||||
.setAppId(convertToProtoFormat(entry.getKey()))
|
AppCollectorData data = entry.getValue();
|
||||||
.setAppCollectorAddr(entry.getValue()));
|
AppCollectorDataProto.Builder appCollectorDataBuilder =
|
||||||
|
AppCollectorDataProto.newBuilder()
|
||||||
|
.setAppId(convertToProtoFormat(entry.getKey()))
|
||||||
|
.setAppCollectorAddr(data.getCollectorAddr())
|
||||||
|
.setRmIdentifier(data.getRMIdentifier())
|
||||||
|
.setVersion(data.getVersion());
|
||||||
|
if (data.getCollectorToken() != null) {
|
||||||
|
appCollectorDataBuilder.setAppCollectorToken(
|
||||||
|
convertToProtoFormat(data.getCollectorToken()));
|
||||||
|
}
|
||||||
|
builder.addAppCollectors(appCollectorDataBuilder);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -568,7 +582,7 @@ public class NodeHeartbeatResponsePBImpl extends NodeHeartbeatResponse {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Map<ApplicationId, String> getAppCollectorsMap() {
|
public Map<ApplicationId, AppCollectorData> getAppCollectors() {
|
||||||
if (this.appCollectorsMap != null) {
|
if (this.appCollectorsMap != null) {
|
||||||
return this.appCollectorsMap;
|
return this.appCollectorsMap;
|
||||||
}
|
}
|
||||||
@ -589,12 +603,19 @@ public class NodeHeartbeatResponsePBImpl extends NodeHeartbeatResponse {
|
|||||||
|
|
||||||
private void initAppCollectorsMap() {
|
private void initAppCollectorsMap() {
|
||||||
NodeHeartbeatResponseProtoOrBuilder p = viaProto ? proto : builder;
|
NodeHeartbeatResponseProtoOrBuilder p = viaProto ? proto : builder;
|
||||||
List<AppCollectorsMapProto> list = p.getAppCollectorsMapList();
|
List<AppCollectorDataProto> list = p.getAppCollectorsList();
|
||||||
if (!list.isEmpty()) {
|
if (!list.isEmpty()) {
|
||||||
this.appCollectorsMap = new HashMap<>();
|
this.appCollectorsMap = new HashMap<>();
|
||||||
for (AppCollectorsMapProto c : list) {
|
for (AppCollectorDataProto c : list) {
|
||||||
ApplicationId appId = convertFromProtoFormat(c.getAppId());
|
ApplicationId appId = convertFromProtoFormat(c.getAppId());
|
||||||
this.appCollectorsMap.put(appId, c.getAppCollectorAddr());
|
Token collectorToken = null;
|
||||||
|
if (c.hasAppCollectorToken()){
|
||||||
|
collectorToken = convertFromProtoFormat(c.getAppCollectorToken());
|
||||||
|
}
|
||||||
|
AppCollectorData data = AppCollectorData.newInstance(appId,
|
||||||
|
c.getAppCollectorAddr(), c.getRmIdentifier(), c.getVersion(),
|
||||||
|
collectorToken);
|
||||||
|
this.appCollectorsMap.put(appId, data);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -611,14 +632,14 @@ public class NodeHeartbeatResponsePBImpl extends NodeHeartbeatResponse {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void setAppCollectorsMap(
|
public void setAppCollectors(
|
||||||
Map<ApplicationId, String> appCollectorsMap) {
|
Map<ApplicationId, AppCollectorData> appCollectors) {
|
||||||
if (appCollectorsMap == null || appCollectorsMap.isEmpty()) {
|
if (appCollectors == null || appCollectors.isEmpty()) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
maybeInitBuilder();
|
maybeInitBuilder();
|
||||||
this.appCollectorsMap = new HashMap<ApplicationId, String>();
|
this.appCollectorsMap = new HashMap<>();
|
||||||
this.appCollectorsMap.putAll(appCollectorsMap);
|
this.appCollectorsMap.putAll(appCollectors);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -773,5 +794,13 @@ public class NodeHeartbeatResponsePBImpl extends NodeHeartbeatResponse {
|
|||||||
SignalContainerRequest t) {
|
SignalContainerRequest t) {
|
||||||
return ((SignalContainerRequestPBImpl)t).getProto();
|
return ((SignalContainerRequestPBImpl)t).getProto();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private TokenProto convertToProtoFormat(Token t) {
|
||||||
|
return ((TokenPBImpl) t).getProto();
|
||||||
|
}
|
||||||
|
|
||||||
|
private TokenPBImpl convertFromProtoFormat(TokenProto p) {
|
||||||
|
return new TokenPBImpl(p);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -20,12 +20,12 @@ package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;
|
|||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorsMapProto;
|
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
||||||
|
import org.apache.hadoop.yarn.server.api.records.impl.pb.AppCollectorDataPBImpl;
|
||||||
|
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorDataProto;
|
||||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewCollectorInfoRequestProto;
|
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewCollectorInfoRequestProto;
|
||||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewCollectorInfoRequestProtoOrBuilder;
|
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewCollectorInfoRequestProtoOrBuilder;
|
||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoRequest;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoRequest;
|
||||||
import org.apache.hadoop.yarn.server.api.records.AppCollectorsMap;
|
|
||||||
import org.apache.hadoop.yarn.server.api.records.impl.pb.AppCollectorsMapPBImpl;
|
|
||||||
|
|
||||||
public class ReportNewCollectorInfoRequestPBImpl extends
|
public class ReportNewCollectorInfoRequestPBImpl extends
|
||||||
ReportNewCollectorInfoRequest {
|
ReportNewCollectorInfoRequest {
|
||||||
@ -36,7 +36,7 @@ public class ReportNewCollectorInfoRequestPBImpl extends
|
|||||||
private ReportNewCollectorInfoRequestProto.Builder builder = null;
|
private ReportNewCollectorInfoRequestProto.Builder builder = null;
|
||||||
private boolean viaProto = false;
|
private boolean viaProto = false;
|
||||||
|
|
||||||
private List<AppCollectorsMap> collectorsList = null;
|
private List<AppCollectorData> collectorsList = null;
|
||||||
|
|
||||||
public ReportNewCollectorInfoRequestPBImpl() {
|
public ReportNewCollectorInfoRequestPBImpl() {
|
||||||
builder = ReportNewCollectorInfoRequestProto.newBuilder();
|
builder = ReportNewCollectorInfoRequestProto.newBuilder();
|
||||||
@ -96,9 +96,9 @@ public class ReportNewCollectorInfoRequestPBImpl extends
|
|||||||
private void addLocalCollectorsToProto() {
|
private void addLocalCollectorsToProto() {
|
||||||
maybeInitBuilder();
|
maybeInitBuilder();
|
||||||
builder.clearAppCollectors();
|
builder.clearAppCollectors();
|
||||||
List<AppCollectorsMapProto> protoList =
|
List<AppCollectorDataProto> protoList =
|
||||||
new ArrayList<AppCollectorsMapProto>();
|
new ArrayList<AppCollectorDataProto>();
|
||||||
for (AppCollectorsMap m : this.collectorsList) {
|
for (AppCollectorData m : this.collectorsList) {
|
||||||
protoList.add(convertToProtoFormat(m));
|
protoList.add(convertToProtoFormat(m));
|
||||||
}
|
}
|
||||||
builder.addAllAppCollectors(protoList);
|
builder.addAllAppCollectors(protoList);
|
||||||
@ -106,16 +106,16 @@ public class ReportNewCollectorInfoRequestPBImpl extends
|
|||||||
|
|
||||||
private void initLocalCollectorsList() {
|
private void initLocalCollectorsList() {
|
||||||
ReportNewCollectorInfoRequestProtoOrBuilder p = viaProto ? proto : builder;
|
ReportNewCollectorInfoRequestProtoOrBuilder p = viaProto ? proto : builder;
|
||||||
List<AppCollectorsMapProto> list =
|
List<AppCollectorDataProto> list =
|
||||||
p.getAppCollectorsList();
|
p.getAppCollectorsList();
|
||||||
this.collectorsList = new ArrayList<AppCollectorsMap>();
|
this.collectorsList = new ArrayList<AppCollectorData>();
|
||||||
for (AppCollectorsMapProto m : list) {
|
for (AppCollectorDataProto m : list) {
|
||||||
this.collectorsList.add(convertFromProtoFormat(m));
|
this.collectorsList.add(convertFromProtoFormat(m));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<AppCollectorsMap> getAppCollectorsList() {
|
public List<AppCollectorData> getAppCollectorsList() {
|
||||||
if (this.collectorsList == null) {
|
if (this.collectorsList == null) {
|
||||||
initLocalCollectorsList();
|
initLocalCollectorsList();
|
||||||
}
|
}
|
||||||
@ -123,7 +123,7 @@ public class ReportNewCollectorInfoRequestPBImpl extends
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void setAppCollectorsList(List<AppCollectorsMap> appCollectorsList) {
|
public void setAppCollectorsList(List<AppCollectorData> appCollectorsList) {
|
||||||
maybeInitBuilder();
|
maybeInitBuilder();
|
||||||
if (appCollectorsList == null) {
|
if (appCollectorsList == null) {
|
||||||
builder.clearAppCollectors();
|
builder.clearAppCollectors();
|
||||||
@ -131,14 +131,14 @@ public class ReportNewCollectorInfoRequestPBImpl extends
|
|||||||
this.collectorsList = appCollectorsList;
|
this.collectorsList = appCollectorsList;
|
||||||
}
|
}
|
||||||
|
|
||||||
private AppCollectorsMapPBImpl convertFromProtoFormat(
|
private AppCollectorDataPBImpl convertFromProtoFormat(
|
||||||
AppCollectorsMapProto p) {
|
AppCollectorDataProto p) {
|
||||||
return new AppCollectorsMapPBImpl(p);
|
return new AppCollectorDataPBImpl(p);
|
||||||
}
|
}
|
||||||
|
|
||||||
private AppCollectorsMapProto convertToProtoFormat(
|
private AppCollectorDataProto convertToProtoFormat(
|
||||||
AppCollectorsMap m) {
|
AppCollectorData m) {
|
||||||
return ((AppCollectorsMapPBImpl) m).getProto();
|
return ((AppCollectorDataPBImpl) m).getProto();
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -0,0 +1,125 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.yarn.server.api.records;
|
||||||
|
|
||||||
|
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||||
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
|
import org.apache.hadoop.yarn.api.records.Token;
|
||||||
|
import org.apache.hadoop.yarn.util.Records;
|
||||||
|
|
||||||
|
|
||||||
|
@Private
|
||||||
|
@InterfaceStability.Unstable
|
||||||
|
public abstract class AppCollectorData {
|
||||||
|
|
||||||
|
protected static final long DEFAULT_TIMESTAMP_VALUE = -1;
|
||||||
|
|
||||||
|
public static AppCollectorData newInstance(
|
||||||
|
ApplicationId id, String collectorAddr, long rmIdentifier, long version,
|
||||||
|
Token token) {
|
||||||
|
AppCollectorData appCollectorData =
|
||||||
|
Records.newRecord(AppCollectorData.class);
|
||||||
|
appCollectorData.setApplicationId(id);
|
||||||
|
appCollectorData.setCollectorAddr(collectorAddr);
|
||||||
|
appCollectorData.setRMIdentifier(rmIdentifier);
|
||||||
|
appCollectorData.setVersion(version);
|
||||||
|
appCollectorData.setCollectorToken(token);
|
||||||
|
return appCollectorData;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static AppCollectorData newInstance(
|
||||||
|
ApplicationId id, String collectorAddr, long rmIdentifier, long version) {
|
||||||
|
return newInstance(id, collectorAddr, rmIdentifier, version, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static AppCollectorData newInstance(ApplicationId id,
|
||||||
|
String collectorAddr, Token token) {
|
||||||
|
return newInstance(id, collectorAddr, DEFAULT_TIMESTAMP_VALUE,
|
||||||
|
DEFAULT_TIMESTAMP_VALUE, token);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static AppCollectorData newInstance(ApplicationId id,
|
||||||
|
String collectorAddr) {
|
||||||
|
return newInstance(id, collectorAddr, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns if a collector data item happens before another one. Null data
|
||||||
|
* items happens before any other non-null items. Non-null data items A
|
||||||
|
* happens before another non-null item B when A's rmIdentifier is less than
|
||||||
|
* B's rmIdentifier. Or A's version is less than B's if they have the same
|
||||||
|
* rmIdentifier.
|
||||||
|
*
|
||||||
|
* @param dataA first collector data item.
|
||||||
|
* @param dataB second collector data item.
|
||||||
|
* @return true if dataA happens before dataB.
|
||||||
|
*/
|
||||||
|
public static boolean happensBefore(AppCollectorData dataA,
|
||||||
|
AppCollectorData dataB) {
|
||||||
|
if (dataA == null && dataB == null) {
|
||||||
|
return false;
|
||||||
|
} else if (dataA == null || dataB == null) {
|
||||||
|
return dataA == null;
|
||||||
|
}
|
||||||
|
|
||||||
|
return
|
||||||
|
(dataA.getRMIdentifier() < dataB.getRMIdentifier())
|
||||||
|
|| ((dataA.getRMIdentifier() == dataB.getRMIdentifier())
|
||||||
|
&& (dataA.getVersion() < dataB.getVersion()));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns if the collector data has been stamped by the RM with a RM cluster
|
||||||
|
* timestamp and a version number.
|
||||||
|
*
|
||||||
|
* @return true if RM has already assigned a timestamp for this collector.
|
||||||
|
* Otherwise, it means the RM has not recognized the existence of this
|
||||||
|
* collector.
|
||||||
|
*/
|
||||||
|
public boolean isStamped() {
|
||||||
|
return (getRMIdentifier() != DEFAULT_TIMESTAMP_VALUE)
|
||||||
|
|| (getVersion() != DEFAULT_TIMESTAMP_VALUE);
|
||||||
|
}
|
||||||
|
|
||||||
|
public abstract ApplicationId getApplicationId();
|
||||||
|
|
||||||
|
public abstract void setApplicationId(ApplicationId id);
|
||||||
|
|
||||||
|
public abstract String getCollectorAddr();
|
||||||
|
|
||||||
|
public abstract void setCollectorAddr(String addr);
|
||||||
|
|
||||||
|
public abstract long getRMIdentifier();
|
||||||
|
|
||||||
|
public abstract void setRMIdentifier(long rmId);
|
||||||
|
|
||||||
|
public abstract long getVersion();
|
||||||
|
|
||||||
|
public abstract void setVersion(long version);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get delegation token for app collector which AM will use to publish
|
||||||
|
* entities.
|
||||||
|
* @return the delegation token for app collector.
|
||||||
|
*/
|
||||||
|
public abstract Token getCollectorToken();
|
||||||
|
|
||||||
|
public abstract void setCollectorToken(Token token);
|
||||||
|
}
|
@ -19,39 +19,43 @@ package org.apache.hadoop.yarn.server.api.records.impl.pb;
|
|||||||
|
|
||||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||||
|
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
|
import org.apache.hadoop.yarn.api.records.Token;
|
||||||
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
|
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
|
||||||
import org.apache.hadoop.yarn.server.api.records.AppCollectorsMap;
|
import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
|
||||||
|
|
||||||
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
|
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
|
||||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorsMapProto;
|
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorDataProto;
|
||||||
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorsMapProtoOrBuilder;
|
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorDataProtoOrBuilder;
|
||||||
|
|
||||||
import com.google.protobuf.TextFormat;
|
import com.google.protobuf.TextFormat;
|
||||||
|
|
||||||
@Private
|
@Private
|
||||||
@Unstable
|
@Unstable
|
||||||
public class AppCollectorsMapPBImpl extends AppCollectorsMap {
|
public class AppCollectorDataPBImpl extends AppCollectorData {
|
||||||
|
|
||||||
private AppCollectorsMapProto proto =
|
private AppCollectorDataProto proto =
|
||||||
AppCollectorsMapProto.getDefaultInstance();
|
AppCollectorDataProto.getDefaultInstance();
|
||||||
|
|
||||||
private AppCollectorsMapProto.Builder builder = null;
|
private AppCollectorDataProto.Builder builder = null;
|
||||||
private boolean viaProto = false;
|
private boolean viaProto = false;
|
||||||
|
|
||||||
private ApplicationId appId = null;
|
private ApplicationId appId = null;
|
||||||
private String collectorAddr = null;
|
private String collectorAddr = null;
|
||||||
|
private Long rmIdentifier = null;
|
||||||
|
private Long version = null;
|
||||||
|
private Token collectorToken = null;
|
||||||
|
|
||||||
public AppCollectorsMapPBImpl() {
|
public AppCollectorDataPBImpl() {
|
||||||
builder = AppCollectorsMapProto.newBuilder();
|
builder = AppCollectorDataProto.newBuilder();
|
||||||
}
|
}
|
||||||
|
|
||||||
public AppCollectorsMapPBImpl(AppCollectorsMapProto proto) {
|
public AppCollectorDataPBImpl(AppCollectorDataProto proto) {
|
||||||
this.proto = proto;
|
this.proto = proto;
|
||||||
viaProto = true;
|
viaProto = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
public AppCollectorsMapProto getProto() {
|
public AppCollectorDataProto getProto() {
|
||||||
mergeLocalToProto();
|
mergeLocalToProto();
|
||||||
proto = viaProto ? proto : builder.build();
|
proto = viaProto ? proto : builder.build();
|
||||||
viaProto = true;
|
viaProto = true;
|
||||||
@ -81,7 +85,7 @@ public class AppCollectorsMapPBImpl extends AppCollectorsMap {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ApplicationId getApplicationId() {
|
public ApplicationId getApplicationId() {
|
||||||
AppCollectorsMapProtoOrBuilder p = viaProto ? proto : builder;
|
AppCollectorDataProtoOrBuilder p = viaProto ? proto : builder;
|
||||||
if (this.appId == null && p.hasAppId()) {
|
if (this.appId == null && p.hasAppId()) {
|
||||||
this.appId = convertFromProtoFormat(p.getAppId());
|
this.appId = convertFromProtoFormat(p.getAppId());
|
||||||
}
|
}
|
||||||
@ -90,7 +94,7 @@ public class AppCollectorsMapPBImpl extends AppCollectorsMap {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String getCollectorAddr() {
|
public String getCollectorAddr() {
|
||||||
AppCollectorsMapProtoOrBuilder p = viaProto ? proto : builder;
|
AppCollectorDataProtoOrBuilder p = viaProto ? proto : builder;
|
||||||
if (this.collectorAddr == null
|
if (this.collectorAddr == null
|
||||||
&& p.hasAppCollectorAddr()) {
|
&& p.hasAppCollectorAddr()) {
|
||||||
this.collectorAddr = p.getAppCollectorAddr();
|
this.collectorAddr = p.getAppCollectorAddr();
|
||||||
@ -116,6 +120,68 @@ public class AppCollectorsMapPBImpl extends AppCollectorsMap {
|
|||||||
this.collectorAddr = collectorAddr;
|
this.collectorAddr = collectorAddr;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getRMIdentifier() {
|
||||||
|
AppCollectorDataProtoOrBuilder p = viaProto ? proto : builder;
|
||||||
|
if (this.rmIdentifier == null && p.hasRmIdentifier()) {
|
||||||
|
this.rmIdentifier = p.getRmIdentifier();
|
||||||
|
}
|
||||||
|
if (this.rmIdentifier != null) {
|
||||||
|
return this.rmIdentifier;
|
||||||
|
} else {
|
||||||
|
return AppCollectorData.DEFAULT_TIMESTAMP_VALUE;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setRMIdentifier(long rmId) {
|
||||||
|
maybeInitBuilder();
|
||||||
|
this.rmIdentifier = rmId;
|
||||||
|
builder.setRmIdentifier(rmId);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getVersion() {
|
||||||
|
AppCollectorDataProtoOrBuilder p = viaProto ? proto : builder;
|
||||||
|
if (this.version == null && p.hasRmIdentifier()) {
|
||||||
|
this.version = p.getRmIdentifier();
|
||||||
|
}
|
||||||
|
if (this.version != null) {
|
||||||
|
return this.version;
|
||||||
|
} else {
|
||||||
|
return AppCollectorData.DEFAULT_TIMESTAMP_VALUE;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setVersion(long version) {
|
||||||
|
maybeInitBuilder();
|
||||||
|
this.version = version;
|
||||||
|
builder.setVersion(version);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Token getCollectorToken() {
|
||||||
|
AppCollectorDataProtoOrBuilder p = viaProto ? proto : builder;
|
||||||
|
if (this.collectorToken != null) {
|
||||||
|
return this.collectorToken;
|
||||||
|
}
|
||||||
|
if (!p.hasAppCollectorToken()) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
this.collectorToken = new TokenPBImpl(p.getAppCollectorToken());
|
||||||
|
return this.collectorToken;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setCollectorToken(Token token) {
|
||||||
|
maybeInitBuilder();
|
||||||
|
if (token == null) {
|
||||||
|
builder.clearAppCollectorToken();
|
||||||
|
}
|
||||||
|
this.collectorToken = token;
|
||||||
|
}
|
||||||
|
|
||||||
private ApplicationIdPBImpl convertFromProtoFormat(ApplicationIdProto p) {
|
private ApplicationIdPBImpl convertFromProtoFormat(ApplicationIdProto p) {
|
||||||
return new ApplicationIdPBImpl(p);
|
return new ApplicationIdPBImpl(p);
|
||||||
}
|
}
|
||||||
@ -126,7 +192,7 @@ public class AppCollectorsMapPBImpl extends AppCollectorsMap {
|
|||||||
|
|
||||||
private void maybeInitBuilder() {
|
private void maybeInitBuilder() {
|
||||||
if (viaProto || builder == null) {
|
if (viaProto || builder == null) {
|
||||||
builder = AppCollectorsMapProto.newBuilder(proto);
|
builder = AppCollectorDataProto.newBuilder(proto);
|
||||||
}
|
}
|
||||||
viaProto = false;
|
viaProto = false;
|
||||||
}
|
}
|
||||||
@ -147,6 +213,15 @@ public class AppCollectorsMapPBImpl extends AppCollectorsMap {
|
|||||||
if (this.collectorAddr != null) {
|
if (this.collectorAddr != null) {
|
||||||
builder.setAppCollectorAddr(this.collectorAddr);
|
builder.setAppCollectorAddr(this.collectorAddr);
|
||||||
}
|
}
|
||||||
|
if (this.rmIdentifier != null) {
|
||||||
|
builder.setRmIdentifier(this.rmIdentifier);
|
||||||
|
}
|
||||||
|
if (this.version != null) {
|
||||||
|
builder.setVersion(this.version);
|
||||||
|
}
|
||||||
|
if (this.collectorToken != null) {
|
||||||
|
builder.setAppCollectorToken(
|
||||||
|
((TokenPBImpl)this.collectorToken).getProto());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
@ -0,0 +1,19 @@
|
|||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
/** Server records PB implementations. */
|
||||||
|
package org.apache.hadoop.yarn.server.api.records.impl.pb;
|
@ -23,27 +23,33 @@ import javax.servlet.ServletException;
|
|||||||
|
|
||||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||||
|
import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
|
||||||
import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationFilter;
|
import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationFilter;
|
||||||
import org.apache.hadoop.yarn.server.timeline.security.TimelineDelegationTokenSecretManagerService.TimelineDelegationTokenSecretManager;
|
import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Timeline authentication filter provides delegation token support for ATSv1
|
||||||
|
* and ATSv2.
|
||||||
|
*/
|
||||||
@Private
|
@Private
|
||||||
@Unstable
|
@Unstable
|
||||||
public class TimelineAuthenticationFilter
|
public class TimelineAuthenticationFilter
|
||||||
extends DelegationTokenAuthenticationFilter {
|
extends DelegationTokenAuthenticationFilter {
|
||||||
|
|
||||||
private static TimelineDelegationTokenSecretManager secretManager;
|
private static AbstractDelegationTokenSecretManager
|
||||||
|
<TimelineDelegationTokenIdentifier> secretManager;
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void init(FilterConfig filterConfig) throws ServletException {
|
public void init(FilterConfig filterConfig) throws ServletException {
|
||||||
filterConfig.getServletContext().setAttribute(
|
filterConfig.getServletContext().setAttribute(
|
||||||
DelegationTokenAuthenticationFilter.DELEGATION_TOKEN_SECRET_MANAGER_ATTR,
|
DelegationTokenAuthenticationFilter.
|
||||||
secretManager);
|
DELEGATION_TOKEN_SECRET_MANAGER_ATTR, secretManager);
|
||||||
super.init(filterConfig);
|
super.init(filterConfig);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void setTimelineDelegationTokenSecretManager(
|
public static void setTimelineDelegationTokenSecretManager(
|
||||||
TimelineDelegationTokenSecretManager secretManager) {
|
AbstractDelegationTokenSecretManager
|
||||||
TimelineAuthenticationFilter.secretManager = secretManager;
|
<TimelineDelegationTokenIdentifier> secretMgr) {
|
||||||
|
TimelineAuthenticationFilter.secretManager = secretMgr;
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
@ -51,30 +51,19 @@ import java.util.Map;
|
|||||||
public class TimelineAuthenticationFilterInitializer extends FilterInitializer {
|
public class TimelineAuthenticationFilterInitializer extends FilterInitializer {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The configuration prefix of timeline HTTP authentication
|
* The configuration prefix of timeline HTTP authentication.
|
||||||
*/
|
*/
|
||||||
public static final String PREFIX = "yarn.timeline-service.http-authentication.";
|
public static final String PREFIX =
|
||||||
|
"yarn.timeline-service.http-authentication.";
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
Map<String, String> filterConfig;
|
Map<String, String> filterConfig;
|
||||||
|
|
||||||
/**
|
protected void setAuthFilterConfig(Configuration conf) {
|
||||||
* Initializes {@link TimelineAuthenticationFilter}
|
|
||||||
* <p>
|
|
||||||
* Propagates to {@link TimelineAuthenticationFilter} configuration all YARN
|
|
||||||
* configuration properties prefixed with {@value #PREFIX}
|
|
||||||
*
|
|
||||||
* @param container
|
|
||||||
* The filter container
|
|
||||||
* @param conf
|
|
||||||
* Configuration for run-time parameters
|
|
||||||
*/
|
|
||||||
@Override
|
|
||||||
public void initFilter(FilterContainer container, Configuration conf) {
|
|
||||||
filterConfig = new HashMap<String, String>();
|
filterConfig = new HashMap<String, String>();
|
||||||
|
|
||||||
// setting the cookie path to root '/' so it is used for all resources.
|
// setting the cookie path to root '/' so it is used for all resources.
|
||||||
filterConfig.put(TimelineAuthenticationFilter.COOKIE_PATH, "/");
|
filterConfig.put(AuthenticationFilter.COOKIE_PATH, "/");
|
||||||
|
|
||||||
for (Map.Entry<String, String> entry : conf) {
|
for (Map.Entry<String, String> entry : conf) {
|
||||||
String name = entry.getKey();
|
String name = entry.getKey();
|
||||||
@ -95,6 +84,41 @@ public class TimelineAuthenticationFilterInitializer extends FilterInitializer {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Resolve _HOST into bind address
|
||||||
|
String bindAddress = conf.get(HttpServer2.BIND_ADDRESS);
|
||||||
|
String principal =
|
||||||
|
filterConfig.get(KerberosAuthenticationHandler.PRINCIPAL);
|
||||||
|
if (principal != null) {
|
||||||
|
try {
|
||||||
|
principal = SecurityUtil.getServerPrincipal(principal, bindAddress);
|
||||||
|
} catch (IOException ex) {
|
||||||
|
throw new RuntimeException("Could not resolve Kerberos principal " +
|
||||||
|
"name: " + ex.toString(), ex);
|
||||||
|
}
|
||||||
|
filterConfig.put(KerberosAuthenticationHandler.PRINCIPAL,
|
||||||
|
principal);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected Map<String, String> getFilterConfig() {
|
||||||
|
return filterConfig;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Initializes {@link TimelineAuthenticationFilter}.
|
||||||
|
* <p>
|
||||||
|
* Propagates to {@link TimelineAuthenticationFilter} configuration all YARN
|
||||||
|
* configuration properties prefixed with {@value #PREFIX}.
|
||||||
|
*
|
||||||
|
* @param container
|
||||||
|
* The filter container.
|
||||||
|
* @param conf
|
||||||
|
* Configuration for run-time parameters.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void initFilter(FilterContainer container, Configuration conf) {
|
||||||
|
setAuthFilterConfig(conf);
|
||||||
|
|
||||||
String authType = filterConfig.get(AuthenticationFilter.AUTH_TYPE);
|
String authType = filterConfig.get(AuthenticationFilter.AUTH_TYPE);
|
||||||
if (authType.equals(PseudoAuthenticationHandler.TYPE)) {
|
if (authType.equals(PseudoAuthenticationHandler.TYPE)) {
|
||||||
filterConfig.put(AuthenticationFilter.AUTH_TYPE,
|
filterConfig.put(AuthenticationFilter.AUTH_TYPE,
|
||||||
@ -102,23 +126,7 @@ public class TimelineAuthenticationFilterInitializer extends FilterInitializer {
|
|||||||
} else if (authType.equals(KerberosAuthenticationHandler.TYPE)) {
|
} else if (authType.equals(KerberosAuthenticationHandler.TYPE)) {
|
||||||
filterConfig.put(AuthenticationFilter.AUTH_TYPE,
|
filterConfig.put(AuthenticationFilter.AUTH_TYPE,
|
||||||
KerberosDelegationTokenAuthenticationHandler.class.getName());
|
KerberosDelegationTokenAuthenticationHandler.class.getName());
|
||||||
|
|
||||||
// Resolve _HOST into bind address
|
|
||||||
String bindAddress = conf.get(HttpServer2.BIND_ADDRESS);
|
|
||||||
String principal =
|
|
||||||
filterConfig.get(KerberosAuthenticationHandler.PRINCIPAL);
|
|
||||||
if (principal != null) {
|
|
||||||
try {
|
|
||||||
principal = SecurityUtil.getServerPrincipal(principal, bindAddress);
|
|
||||||
} catch (IOException ex) {
|
|
||||||
throw new RuntimeException(
|
|
||||||
"Could not resolve Kerberos principal name: " + ex.toString(), ex);
|
|
||||||
}
|
|
||||||
filterConfig.put(KerberosAuthenticationHandler.PRINCIPAL,
|
|
||||||
principal);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
filterConfig.put(DelegationTokenAuthenticationHandler.TOKEN_KIND,
|
filterConfig.put(DelegationTokenAuthenticationHandler.TOKEN_KIND,
|
||||||
TimelineDelegationTokenIdentifier.KIND_NAME.toString());
|
TimelineDelegationTokenIdentifier.KIND_NAME.toString());
|
||||||
|
|
@ -0,0 +1,83 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.yarn.server.timeline.security;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
|
||||||
|
import org.apache.hadoop.service.AbstractService;
|
||||||
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||||
|
import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Abstract implementation of delegation token manager service for different
|
||||||
|
* versions of timeline service.
|
||||||
|
*/
|
||||||
|
public abstract class TimelineDelgationTokenSecretManagerService extends
|
||||||
|
AbstractService {
|
||||||
|
|
||||||
|
public TimelineDelgationTokenSecretManagerService(String name) {
|
||||||
|
super(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static long delegationTokenRemovalScanInterval = 3600000L;
|
||||||
|
|
||||||
|
private AbstractDelegationTokenSecretManager
|
||||||
|
<TimelineDelegationTokenIdentifier> secretManager = null;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void serviceInit(Configuration conf) throws Exception {
|
||||||
|
long secretKeyInterval =
|
||||||
|
conf.getLong(YarnConfiguration.TIMELINE_DELEGATION_KEY_UPDATE_INTERVAL,
|
||||||
|
YarnConfiguration.DEFAULT_TIMELINE_DELEGATION_KEY_UPDATE_INTERVAL);
|
||||||
|
long tokenMaxLifetime =
|
||||||
|
conf.getLong(YarnConfiguration.TIMELINE_DELEGATION_TOKEN_MAX_LIFETIME,
|
||||||
|
YarnConfiguration.DEFAULT_TIMELINE_DELEGATION_TOKEN_MAX_LIFETIME);
|
||||||
|
long tokenRenewInterval =
|
||||||
|
conf.getLong(YarnConfiguration.TIMELINE_DELEGATION_TOKEN_RENEW_INTERVAL,
|
||||||
|
YarnConfiguration.DEFAULT_TIMELINE_DELEGATION_TOKEN_RENEW_INTERVAL);
|
||||||
|
secretManager = createTimelineDelegationTokenSecretManager(
|
||||||
|
secretKeyInterval, tokenMaxLifetime, tokenRenewInterval,
|
||||||
|
delegationTokenRemovalScanInterval);
|
||||||
|
super.init(conf);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected abstract
|
||||||
|
AbstractDelegationTokenSecretManager<TimelineDelegationTokenIdentifier>
|
||||||
|
createTimelineDelegationTokenSecretManager(long secretKeyInterval,
|
||||||
|
long tokenMaxLifetime, long tokenRenewInterval,
|
||||||
|
long tokenRemovalScanInterval);
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void serviceStart() throws Exception {
|
||||||
|
secretManager.startThreads();
|
||||||
|
super.serviceStart();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void serviceStop() throws Exception {
|
||||||
|
secretManager.stopThreads();
|
||||||
|
super.stop();
|
||||||
|
}
|
||||||
|
|
||||||
|
public AbstractDelegationTokenSecretManager
|
||||||
|
<TimelineDelegationTokenIdentifier>
|
||||||
|
getTimelineDelegationTokenSecretManager() {
|
||||||
|
return secretManager;
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,26 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Package org.apache.hadoop.server.timeline.security contains classes related
|
||||||
|
* to timeline authentication filters and abstract delegation token service for
|
||||||
|
* ATSv1 and ATSv2.
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
package org.apache.hadoop.yarn.server.timeline.security;
|
||||||
|
import org.apache.hadoop.classification.InterfaceAudience;
|
@ -0,0 +1,92 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.yarn.server.util.timeline;
|
||||||
|
|
||||||
|
import java.util.LinkedHashSet;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.security.AuthenticationFilterInitializer;
|
||||||
|
import org.apache.hadoop.yarn.server.timeline.security.TimelineAuthenticationFilter;
|
||||||
|
import org.apache.hadoop.yarn.server.timeline.security.TimelineAuthenticationFilterInitializer;
|
||||||
|
import org.apache.hadoop.yarn.server.timeline.security.TimelineDelgationTokenSecretManagerService;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set of utility methods to be used across timeline reader and collector.
|
||||||
|
*/
|
||||||
|
public final class TimelineServerUtils {
|
||||||
|
private static final Log LOG = LogFactory.getLog(TimelineServerUtils.class);
|
||||||
|
|
||||||
|
private TimelineServerUtils() {
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets filter initializers configuration based on existing configuration and
|
||||||
|
* default filters added by timeline service(such as timeline auth filter and
|
||||||
|
* CORS filter).
|
||||||
|
* @param conf Configuration object.
|
||||||
|
* @param configuredInitializers Comma separated list of filter initializers.
|
||||||
|
* @param defaultInitializers Set of initializers added by default by timeline
|
||||||
|
* service.
|
||||||
|
*/
|
||||||
|
public static void setTimelineFilters(Configuration conf,
|
||||||
|
String configuredInitializers, Set<String> defaultInitializers) {
|
||||||
|
String[] parts = configuredInitializers.split(",");
|
||||||
|
Set<String> target = new LinkedHashSet<String>();
|
||||||
|
for (String filterInitializer : parts) {
|
||||||
|
filterInitializer = filterInitializer.trim();
|
||||||
|
if (filterInitializer.equals(
|
||||||
|
AuthenticationFilterInitializer.class.getName()) ||
|
||||||
|
filterInitializer.isEmpty()) {
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
target.add(filterInitializer);
|
||||||
|
}
|
||||||
|
target.addAll(defaultInitializers);
|
||||||
|
String actualInitializers =
|
||||||
|
org.apache.commons.lang.StringUtils.join(target, ",");
|
||||||
|
LOG.info("Filter initializers set for timeline service: " +
|
||||||
|
actualInitializers);
|
||||||
|
conf.set("hadoop.http.filter.initializers", actualInitializers);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Adds timeline authentication filter to the set of default filter
|
||||||
|
* initializers and assigns the delegation token manager service to it.
|
||||||
|
* @param initializers Comma separated list of filter initializers.
|
||||||
|
* @param defaultInitializers Set of initializers added by default by timeline
|
||||||
|
* service.
|
||||||
|
* @param delegationTokenMgrService Delegation token manager service.
|
||||||
|
* This will be used by timeline authentication filter to assign
|
||||||
|
* delegation tokens.
|
||||||
|
*/
|
||||||
|
public static void addTimelineAuthFilter(String initializers,
|
||||||
|
Set<String> defaultInitializers,
|
||||||
|
TimelineDelgationTokenSecretManagerService delegationTokenMgrService) {
|
||||||
|
TimelineAuthenticationFilter.setTimelineDelegationTokenSecretManager(
|
||||||
|
delegationTokenMgrService.getTimelineDelegationTokenSecretManager());
|
||||||
|
if (!initializers.contains(
|
||||||
|
TimelineAuthenticationFilterInitializer.class.getName())) {
|
||||||
|
defaultInitializers.add(
|
||||||
|
TimelineAuthenticationFilterInitializer.class.getName());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,25 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Package org.apache.hadoop.server.util.timeline contains utility classes used
|
||||||
|
* by ATSv1 and ATSv2 on the server side.
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
package org.apache.hadoop.yarn.server.util.timeline;
|
||||||
|
import org.apache.hadoop.classification.InterfaceAudience;
|
@ -22,6 +22,7 @@ option java_generic_services = true;
|
|||||||
option java_generate_equals_and_hash = true;
|
option java_generate_equals_and_hash = true;
|
||||||
package hadoop.yarn;
|
package hadoop.yarn;
|
||||||
|
|
||||||
|
import "Security.proto";
|
||||||
import "yarn_protos.proto";
|
import "yarn_protos.proto";
|
||||||
import "yarn_server_common_protos.proto";
|
import "yarn_server_common_protos.proto";
|
||||||
import "yarn_service_protos.proto";
|
import "yarn_service_protos.proto";
|
||||||
@ -90,7 +91,7 @@ message NodeHeartbeatRequestProto {
|
|||||||
optional MasterKeyProto last_known_nm_token_master_key = 3;
|
optional MasterKeyProto last_known_nm_token_master_key = 3;
|
||||||
optional NodeLabelsProto nodeLabels = 4;
|
optional NodeLabelsProto nodeLabels = 4;
|
||||||
repeated LogAggregationReportProto log_aggregation_reports_for_apps = 5;
|
repeated LogAggregationReportProto log_aggregation_reports_for_apps = 5;
|
||||||
repeated AppCollectorsMapProto registered_collectors = 6;
|
repeated AppCollectorDataProto registering_collectors = 6;
|
||||||
}
|
}
|
||||||
|
|
||||||
message LogAggregationReportProto {
|
message LogAggregationReportProto {
|
||||||
@ -116,7 +117,7 @@ message NodeHeartbeatResponseProto {
|
|||||||
repeated SignalContainerRequestProto containers_to_signal = 13;
|
repeated SignalContainerRequestProto containers_to_signal = 13;
|
||||||
optional ResourceProto resource = 14;
|
optional ResourceProto resource = 14;
|
||||||
optional ContainerQueuingLimitProto container_queuing_limit = 15;
|
optional ContainerQueuingLimitProto container_queuing_limit = 15;
|
||||||
repeated AppCollectorsMapProto app_collectors_map = 16;
|
repeated AppCollectorDataProto app_collectors = 16;
|
||||||
// to be used in place of containers_to_decrease
|
// to be used in place of containers_to_decrease
|
||||||
repeated ContainerProto containers_to_update = 17;
|
repeated ContainerProto containers_to_update = 17;
|
||||||
}
|
}
|
||||||
@ -134,16 +135,19 @@ message SystemCredentialsForAppsProto {
|
|||||||
////////////////////////////////////////////////////////////////////////
|
////////////////////////////////////////////////////////////////////////
|
||||||
////// From collector_nodemanager_protocol ////////////////////////////
|
////// From collector_nodemanager_protocol ////////////////////////////
|
||||||
////////////////////////////////////////////////////////////////////////
|
////////////////////////////////////////////////////////////////////////
|
||||||
message AppCollectorsMapProto {
|
message AppCollectorDataProto {
|
||||||
optional ApplicationIdProto appId = 1;
|
optional ApplicationIdProto app_id = 1;
|
||||||
optional string appCollectorAddr = 2;
|
optional string app_collector_addr = 2;
|
||||||
|
optional int64 rm_identifier = 3 [default = -1];
|
||||||
|
optional int64 version = 4 [default = -1];
|
||||||
|
optional hadoop.common.TokenProto app_collector_token = 5;
|
||||||
}
|
}
|
||||||
|
|
||||||
//////////////////////////////////////////////////////
|
//////////////////////////////////////////////////////
|
||||||
/////// collector_nodemanager_protocol //////////////
|
/////// collector_nodemanager_protocol //////////////
|
||||||
//////////////////////////////////////////////////////
|
//////////////////////////////////////////////////////
|
||||||
message ReportNewCollectorInfoRequestProto {
|
message ReportNewCollectorInfoRequestProto {
|
||||||
repeated AppCollectorsMapProto app_collectors = 1;
|
repeated AppCollectorDataProto app_collectors = 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
message ReportNewCollectorInfoResponseProto {
|
message ReportNewCollectorInfoResponseProto {
|
||||||
|
@ -30,6 +30,7 @@ import org.apache.hadoop.ipc.RPC;
|
|||||||
import org.apache.hadoop.ipc.Server;
|
import org.apache.hadoop.ipc.Server;
|
||||||
import org.apache.hadoop.net.NetUtils;
|
import org.apache.hadoop.net.NetUtils;
|
||||||
import org.apache.hadoop.security.SecurityUtil;
|
import org.apache.hadoop.security.SecurityUtil;
|
||||||
|
import org.apache.hadoop.util.Time;
|
||||||
import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
|
import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
|
||||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
|
||||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
|
import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
|
||||||
@ -72,12 +73,13 @@ import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC;
|
|||||||
import org.apache.hadoop.yarn.ipc.RPCUtil;
|
import org.apache.hadoop.yarn.ipc.RPCUtil;
|
||||||
import org.apache.hadoop.yarn.ipc.YarnRPC;
|
import org.apache.hadoop.yarn.ipc.YarnRPC;
|
||||||
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
|
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
|
||||||
|
import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
|
||||||
import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocol;
|
import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocol;
|
||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextRequest;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextRequest;
|
||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextResponse;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextResponse;
|
||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoRequest;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoRequest;
|
||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoResponse;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoResponse;
|
||||||
import org.apache.hadoop.yarn.server.api.records.AppCollectorsMap;
|
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
||||||
import org.apache.hadoop.yarn.util.Records;
|
import org.apache.hadoop.yarn.util.Records;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
@ -93,6 +95,21 @@ public class TestRPC {
|
|||||||
"collectors' number in ReportNewCollectorInfoRequest is not ONE.";
|
"collectors' number in ReportNewCollectorInfoRequest is not ONE.";
|
||||||
|
|
||||||
public static final String DEFAULT_COLLECTOR_ADDR = "localhost:0";
|
public static final String DEFAULT_COLLECTOR_ADDR = "localhost:0";
|
||||||
|
private static final Token DEFAULT_COLLECTOR_TOKEN;
|
||||||
|
static {
|
||||||
|
TimelineDelegationTokenIdentifier identifier =
|
||||||
|
new TimelineDelegationTokenIdentifier();
|
||||||
|
identifier.setOwner(new Text("user"));
|
||||||
|
identifier.setRenewer(new Text("user"));
|
||||||
|
identifier.setRealUser(new Text("user"));
|
||||||
|
long now = Time.now();
|
||||||
|
identifier.setIssueDate(now);
|
||||||
|
identifier.setMaxDate(now + 1000L);
|
||||||
|
identifier.setMasterKeyId(500);
|
||||||
|
identifier.setSequenceNumber(5);
|
||||||
|
DEFAULT_COLLECTOR_TOKEN = Token.newInstance(identifier.getBytes(),
|
||||||
|
identifier.getKind().toString(), identifier.getBytes(), "localhost:0");
|
||||||
|
}
|
||||||
|
|
||||||
public static final ApplicationId DEFAULT_APP_ID =
|
public static final ApplicationId DEFAULT_APP_ID =
|
||||||
ApplicationId.newInstance(0, 0);
|
ApplicationId.newInstance(0, 0);
|
||||||
@ -173,7 +190,16 @@ public class TestRPC {
|
|||||||
try {
|
try {
|
||||||
ReportNewCollectorInfoRequest request =
|
ReportNewCollectorInfoRequest request =
|
||||||
ReportNewCollectorInfoRequest.newInstance(
|
ReportNewCollectorInfoRequest.newInstance(
|
||||||
DEFAULT_APP_ID, DEFAULT_COLLECTOR_ADDR);
|
DEFAULT_APP_ID, DEFAULT_COLLECTOR_ADDR, null);
|
||||||
|
proxy.reportNewCollectorInfo(request);
|
||||||
|
} catch (YarnException e) {
|
||||||
|
Assert.fail("RPC call failured is not expected here.");
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
ReportNewCollectorInfoRequest request =
|
||||||
|
ReportNewCollectorInfoRequest.newInstance(
|
||||||
|
DEFAULT_APP_ID, DEFAULT_COLLECTOR_ADDR, DEFAULT_COLLECTOR_TOKEN);
|
||||||
proxy.reportNewCollectorInfo(request);
|
proxy.reportNewCollectorInfo(request);
|
||||||
} catch (YarnException e) {
|
} catch (YarnException e) {
|
||||||
Assert.fail("RPC call failured is not expected here.");
|
Assert.fail("RPC call failured is not expected here.");
|
||||||
@ -429,14 +455,16 @@ public class TestRPC {
|
|||||||
public ReportNewCollectorInfoResponse reportNewCollectorInfo(
|
public ReportNewCollectorInfoResponse reportNewCollectorInfo(
|
||||||
ReportNewCollectorInfoRequest request)
|
ReportNewCollectorInfoRequest request)
|
||||||
throws YarnException, IOException {
|
throws YarnException, IOException {
|
||||||
List<AppCollectorsMap> appCollectors = request.getAppCollectorsList();
|
List<AppCollectorData> appCollectors = request.getAppCollectorsList();
|
||||||
if (appCollectors.size() == 1) {
|
if (appCollectors.size() == 1) {
|
||||||
// check default appID and collectorAddr
|
// check default appID and collectorAddr
|
||||||
AppCollectorsMap appCollector = appCollectors.get(0);
|
AppCollectorData appCollector = appCollectors.get(0);
|
||||||
Assert.assertEquals(appCollector.getApplicationId(),
|
Assert.assertEquals(appCollector.getApplicationId(),
|
||||||
DEFAULT_APP_ID);
|
DEFAULT_APP_ID);
|
||||||
Assert.assertEquals(appCollector.getCollectorAddr(),
|
Assert.assertEquals(appCollector.getCollectorAddr(),
|
||||||
DEFAULT_COLLECTOR_ADDR);
|
DEFAULT_COLLECTOR_ADDR);
|
||||||
|
Assert.assertTrue(appCollector.getCollectorToken() == null ||
|
||||||
|
appCollector.getCollectorToken().equals(DEFAULT_COLLECTOR_TOKEN));
|
||||||
} else {
|
} else {
|
||||||
throw new YarnException(ILLEGAL_NUMBER_MESSAGE);
|
throw new YarnException(ILLEGAL_NUMBER_MESSAGE);
|
||||||
}
|
}
|
||||||
|
@ -37,6 +37,7 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
|||||||
import org.apache.hadoop.yarn.api.records.NodeId;
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||||
import org.apache.hadoop.yarn.api.records.NodeLabel;
|
import org.apache.hadoop.yarn.api.records.NodeLabel;
|
||||||
import org.apache.hadoop.yarn.api.records.Resource;
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
|
import org.apache.hadoop.yarn.api.records.Token;
|
||||||
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl;
|
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl;
|
||||||
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
|
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
|
||||||
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
|
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
|
||||||
@ -48,6 +49,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.NodeHeartbeatRe
|
|||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RegisterNodeManagerRequestPBImpl;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RegisterNodeManagerRequestPBImpl;
|
||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RegisterNodeManagerResponsePBImpl;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RegisterNodeManagerResponsePBImpl;
|
||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.UnRegisterNodeManagerRequestPBImpl;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.UnRegisterNodeManagerRequestPBImpl;
|
||||||
|
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
||||||
import org.apache.hadoop.yarn.server.api.records.MasterKey;
|
import org.apache.hadoop.yarn.server.api.records.MasterKey;
|
||||||
import org.apache.hadoop.yarn.server.api.records.NodeAction;
|
import org.apache.hadoop.yarn.server.api.records.NodeAction;
|
||||||
import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
|
import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
|
||||||
@ -109,14 +111,14 @@ public class TestYarnServerApiClasses {
|
|||||||
original.setLastKnownNMTokenMasterKey(getMasterKey());
|
original.setLastKnownNMTokenMasterKey(getMasterKey());
|
||||||
original.setNodeStatus(getNodeStatus());
|
original.setNodeStatus(getNodeStatus());
|
||||||
original.setNodeLabels(getValidNodeLabels());
|
original.setNodeLabels(getValidNodeLabels());
|
||||||
Map<ApplicationId, String> collectors = getCollectors();
|
Map<ApplicationId, AppCollectorData> collectors = getCollectors(false);
|
||||||
original.setRegisteredCollectors(collectors);
|
original.setRegisteringCollectors(collectors);
|
||||||
NodeHeartbeatRequestPBImpl copy = new NodeHeartbeatRequestPBImpl(
|
NodeHeartbeatRequestPBImpl copy = new NodeHeartbeatRequestPBImpl(
|
||||||
original.getProto());
|
original.getProto());
|
||||||
assertEquals(1, copy.getLastKnownContainerTokenMasterKey().getKeyId());
|
assertEquals(1, copy.getLastKnownContainerTokenMasterKey().getKeyId());
|
||||||
assertEquals(1, copy.getLastKnownNMTokenMasterKey().getKeyId());
|
assertEquals(1, copy.getLastKnownNMTokenMasterKey().getKeyId());
|
||||||
assertEquals("localhost", copy.getNodeStatus().getNodeId().getHost());
|
assertEquals("localhost", copy.getNodeStatus().getNodeId().getHost());
|
||||||
assertEquals(collectors, copy.getRegisteredCollectors());
|
assertEquals(collectors, copy.getRegisteringCollectors());
|
||||||
// check labels are coming with valid values
|
// check labels are coming with valid values
|
||||||
Assert.assertTrue(original.getNodeLabels()
|
Assert.assertTrue(original.getNodeLabels()
|
||||||
.containsAll(copy.getNodeLabels()));
|
.containsAll(copy.getNodeLabels()));
|
||||||
@ -128,6 +130,16 @@ public class TestYarnServerApiClasses {
|
|||||||
Assert.assertEquals(0, copy.getNodeLabels().size());
|
Assert.assertEquals(0, copy.getNodeLabels().size());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testNodeHBRequestPBImplWithNullCollectorToken() {
|
||||||
|
NodeHeartbeatRequestPBImpl original = new NodeHeartbeatRequestPBImpl();
|
||||||
|
Map<ApplicationId, AppCollectorData> collectors = getCollectors(true);
|
||||||
|
original.setRegisteringCollectors(collectors);
|
||||||
|
NodeHeartbeatRequestPBImpl copy = new NodeHeartbeatRequestPBImpl(
|
||||||
|
original.getProto());
|
||||||
|
assertEquals(collectors, copy.getRegisteringCollectors());
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test NodeHeartbeatRequestPBImpl.
|
* Test NodeHeartbeatRequestPBImpl.
|
||||||
*/
|
*/
|
||||||
@ -153,8 +165,8 @@ public class TestYarnServerApiClasses {
|
|||||||
original.setNextHeartBeatInterval(1000);
|
original.setNextHeartBeatInterval(1000);
|
||||||
original.setNodeAction(NodeAction.NORMAL);
|
original.setNodeAction(NodeAction.NORMAL);
|
||||||
original.setResponseId(100);
|
original.setResponseId(100);
|
||||||
Map<ApplicationId, String> collectors = getCollectors();
|
Map<ApplicationId, AppCollectorData> collectors = getCollectors(false);
|
||||||
original.setAppCollectorsMap(collectors);
|
original.setAppCollectors(collectors);
|
||||||
|
|
||||||
NodeHeartbeatResponsePBImpl copy = new NodeHeartbeatResponsePBImpl(
|
NodeHeartbeatResponsePBImpl copy = new NodeHeartbeatResponsePBImpl(
|
||||||
original.getProto());
|
original.getProto());
|
||||||
@ -164,7 +176,7 @@ public class TestYarnServerApiClasses {
|
|||||||
assertEquals(1, copy.getContainerTokenMasterKey().getKeyId());
|
assertEquals(1, copy.getContainerTokenMasterKey().getKeyId());
|
||||||
assertEquals(1, copy.getNMTokenMasterKey().getKeyId());
|
assertEquals(1, copy.getNMTokenMasterKey().getKeyId());
|
||||||
assertEquals("testDiagnosticMessage", copy.getDiagnosticsMessage());
|
assertEquals("testDiagnosticMessage", copy.getDiagnosticsMessage());
|
||||||
assertEquals(collectors, copy.getAppCollectorsMap());
|
assertEquals(collectors, copy.getAppCollectors());
|
||||||
assertEquals(false, copy.getAreNodeLabelsAcceptedByRM());
|
assertEquals(false, copy.getAreNodeLabelsAcceptedByRM());
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -177,6 +189,16 @@ public class TestYarnServerApiClasses {
|
|||||||
assertTrue(copy.getAreNodeLabelsAcceptedByRM());
|
assertTrue(copy.getAreNodeLabelsAcceptedByRM());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testNodeHBResponsePBImplWithNullCollectorToken() {
|
||||||
|
NodeHeartbeatResponsePBImpl original = new NodeHeartbeatResponsePBImpl();
|
||||||
|
Map<ApplicationId, AppCollectorData> collectors = getCollectors(true);
|
||||||
|
original.setAppCollectors(collectors);
|
||||||
|
NodeHeartbeatResponsePBImpl copy = new NodeHeartbeatResponsePBImpl(
|
||||||
|
original.getProto());
|
||||||
|
assertEquals(collectors, copy.getAppCollectors());
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testNodeHeartbeatResponsePBImplWithDecreasedContainers() {
|
public void testNodeHeartbeatResponsePBImplWithDecreasedContainers() {
|
||||||
NodeHeartbeatResponsePBImpl original = new NodeHeartbeatResponsePBImpl();
|
NodeHeartbeatResponsePBImpl original = new NodeHeartbeatResponsePBImpl();
|
||||||
@ -347,12 +369,18 @@ public class TestYarnServerApiClasses {
|
|||||||
return nodeLabels;
|
return nodeLabels;
|
||||||
}
|
}
|
||||||
|
|
||||||
private Map<ApplicationId, String> getCollectors() {
|
private Map<ApplicationId, AppCollectorData> getCollectors(
|
||||||
|
boolean hasNullCollectorToken) {
|
||||||
ApplicationId appID = ApplicationId.newInstance(1L, 1);
|
ApplicationId appID = ApplicationId.newInstance(1L, 1);
|
||||||
String collectorAddr = "localhost:0";
|
String collectorAddr = "localhost:0";
|
||||||
Map<ApplicationId, String> collectorMap =
|
AppCollectorData data = AppCollectorData.newInstance(appID, collectorAddr);
|
||||||
new HashMap<ApplicationId, String>();
|
if (!hasNullCollectorToken) {
|
||||||
collectorMap.put(appID, collectorAddr);
|
data.setCollectorToken(
|
||||||
|
Token.newInstance(new byte[0], "kind", new byte[0], "s"));
|
||||||
|
}
|
||||||
|
Map<ApplicationId, AppCollectorData> collectorMap =
|
||||||
|
new HashMap<>();
|
||||||
|
collectorMap.put(appID, data);
|
||||||
return collectorMap;
|
return collectorMap;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -27,7 +27,9 @@ import static org.apache.hadoop.yarn.server.timeline.security.TimelineAuthentica
|
|||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.mockito.Mockito;
|
import org.mockito.Mockito;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests {@link TimelineAuthenticationFilterInitializer}.
|
||||||
|
*/
|
||||||
public class TestTimelineAuthenticationFilterInitializer {
|
public class TestTimelineAuthenticationFilterInitializer {
|
||||||
|
|
||||||
@Test
|
@Test
|
@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|||||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
import org.apache.hadoop.yarn.api.records.NodeId;
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
|
||||||
|
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
||||||
import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
|
import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManager;
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManager;
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
|
||||||
@ -65,11 +66,18 @@ public interface Context {
|
|||||||
Map<ApplicationId, Credentials> getSystemCredentialsForApps();
|
Map<ApplicationId, Credentials> getSystemCredentialsForApps();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get the registered collectors that located on this NM.
|
* Get the list of collectors that are registering with the RM from this node.
|
||||||
* @return registered collectors, or null if the timeline service v.2 is not
|
* @return registering collectors, or null if the timeline service v.2 is not
|
||||||
* enabled
|
* enabled
|
||||||
*/
|
*/
|
||||||
Map<ApplicationId, String> getRegisteredCollectors();
|
ConcurrentMap<ApplicationId, AppCollectorData> getRegisteringCollectors();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the list of collectors registered with the RM and known by this node.
|
||||||
|
* @return known collectors, or null if the timeline service v.2 is not
|
||||||
|
* enabled.
|
||||||
|
*/
|
||||||
|
ConcurrentMap<ApplicationId, AppCollectorData> getKnownCollectors();
|
||||||
|
|
||||||
ConcurrentMap<ContainerId, Container> getContainers();
|
ConcurrentMap<ContainerId, Container> getContainers();
|
||||||
|
|
||||||
|
@ -57,11 +57,13 @@ import org.apache.hadoop.yarn.event.EventHandler;
|
|||||||
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
||||||
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
|
||||||
|
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
||||||
import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
|
import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManager;
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManager;
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.collectormanager.NMCollectorService;
|
import org.apache.hadoop.yarn.server.nodemanager.collectormanager.NMCollectorService;
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
|
||||||
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
|
import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.nodelabels.ConfigurationNodeLabelsProvider;
|
import org.apache.hadoop.yarn.server.nodemanager.nodelabels.ConfigurationNodeLabelsProvider;
|
||||||
@ -464,8 +466,14 @@ public class NodeManager extends CompositeService
|
|||||||
if (!rmWorkPreservingRestartEnabled) {
|
if (!rmWorkPreservingRestartEnabled) {
|
||||||
LOG.info("Cleaning up running containers on resync");
|
LOG.info("Cleaning up running containers on resync");
|
||||||
containerManager.cleanupContainersOnNMResync();
|
containerManager.cleanupContainersOnNMResync();
|
||||||
|
// Clear all known collectors for resync.
|
||||||
|
if (context.getKnownCollectors() != null) {
|
||||||
|
context.getKnownCollectors().clear();
|
||||||
|
}
|
||||||
} else {
|
} else {
|
||||||
LOG.info("Preserving containers on resync");
|
LOG.info("Preserving containers on resync");
|
||||||
|
// Re-register known timeline collectors.
|
||||||
|
reregisterCollectors();
|
||||||
}
|
}
|
||||||
((NodeStatusUpdaterImpl) nodeStatusUpdater)
|
((NodeStatusUpdaterImpl) nodeStatusUpdater)
|
||||||
.rebootNodeStatusUpdaterAndRegisterWithRM();
|
.rebootNodeStatusUpdaterAndRegisterWithRM();
|
||||||
@ -477,6 +485,38 @@ public class NodeManager extends CompositeService
|
|||||||
}.start();
|
}.start();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Reregisters all collectors known by this node to the RM. This method is
|
||||||
|
* called when the RM needs to resync with the node.
|
||||||
|
*/
|
||||||
|
protected void reregisterCollectors() {
|
||||||
|
Map<ApplicationId, AppCollectorData> knownCollectors
|
||||||
|
= context.getKnownCollectors();
|
||||||
|
if (knownCollectors == null) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
ConcurrentMap<ApplicationId, AppCollectorData> registeringCollectors
|
||||||
|
= context.getRegisteringCollectors();
|
||||||
|
for (Map.Entry<ApplicationId, AppCollectorData> entry
|
||||||
|
: knownCollectors.entrySet()) {
|
||||||
|
Application app = context.getApplications().get(entry.getKey());
|
||||||
|
if ((app != null)
|
||||||
|
&& !ApplicationState.FINISHED.equals(app.getApplicationState())) {
|
||||||
|
registeringCollectors.putIfAbsent(entry.getKey(), entry.getValue());
|
||||||
|
AppCollectorData data = entry.getValue();
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug(entry.getKey() + " : " + data.getCollectorAddr() + "@<"
|
||||||
|
+ data.getRMIdentifier() + ", " + data.getVersion() + ">");
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("Remove collector data for done app " + entry.getKey());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
knownCollectors.clear();
|
||||||
|
}
|
||||||
|
|
||||||
public static class NMContext implements Context {
|
public static class NMContext implements Context {
|
||||||
|
|
||||||
private NodeId nodeId = null;
|
private NodeId nodeId = null;
|
||||||
@ -492,7 +532,10 @@ public class NodeManager extends CompositeService
|
|||||||
protected final ConcurrentMap<ContainerId, Container> containers =
|
protected final ConcurrentMap<ContainerId, Container> containers =
|
||||||
new ConcurrentSkipListMap<ContainerId, Container>();
|
new ConcurrentSkipListMap<ContainerId, Container>();
|
||||||
|
|
||||||
private Map<ApplicationId, String> registeredCollectors;
|
private ConcurrentMap<ApplicationId, AppCollectorData>
|
||||||
|
registeringCollectors;
|
||||||
|
|
||||||
|
private ConcurrentMap<ApplicationId, AppCollectorData> knownCollectors;
|
||||||
|
|
||||||
protected final ConcurrentMap<ContainerId,
|
protected final ConcurrentMap<ContainerId,
|
||||||
org.apache.hadoop.yarn.api.records.Container> increasedContainers =
|
org.apache.hadoop.yarn.api.records.Container> increasedContainers =
|
||||||
@ -526,7 +569,8 @@ public class NodeManager extends CompositeService
|
|||||||
NMStateStoreService stateStore, boolean isDistSchedulingEnabled,
|
NMStateStoreService stateStore, boolean isDistSchedulingEnabled,
|
||||||
Configuration conf) {
|
Configuration conf) {
|
||||||
if (YarnConfiguration.timelineServiceV2Enabled(conf)) {
|
if (YarnConfiguration.timelineServiceV2Enabled(conf)) {
|
||||||
this.registeredCollectors = new ConcurrentHashMap<>();
|
this.registeringCollectors = new ConcurrentHashMap<>();
|
||||||
|
this.knownCollectors = new ConcurrentHashMap<>();
|
||||||
}
|
}
|
||||||
this.containerTokenSecretManager = containerTokenSecretManager;
|
this.containerTokenSecretManager = containerTokenSecretManager;
|
||||||
this.nmTokenSecretManager = nmTokenSecretManager;
|
this.nmTokenSecretManager = nmTokenSecretManager;
|
||||||
@ -681,18 +725,14 @@ public class NodeManager extends CompositeService
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Map<ApplicationId, String> getRegisteredCollectors() {
|
public ConcurrentMap<ApplicationId, AppCollectorData>
|
||||||
return this.registeredCollectors;
|
getRegisteringCollectors() {
|
||||||
|
return this.registeringCollectors;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void addRegisteredCollectors(
|
@Override
|
||||||
Map<ApplicationId, String> newRegisteredCollectors) {
|
public ConcurrentMap<ApplicationId, AppCollectorData> getKnownCollectors() {
|
||||||
if (registeredCollectors != null) {
|
return this.knownCollectors;
|
||||||
this.registeredCollectors.putAll(newRegisteredCollectors);
|
|
||||||
} else {
|
|
||||||
LOG.warn("collectors are added when the registered collectors are " +
|
|
||||||
"initialized");
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -71,7 +71,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
|
|||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
|
||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
|
||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.UnRegisterNodeManagerRequest;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.UnRegisterNodeManagerRequest;
|
||||||
|
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
||||||
import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
|
import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
|
||||||
import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
|
import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
|
||||||
import org.apache.hadoop.yarn.server.api.records.MasterKey;
|
import org.apache.hadoop.yarn.server.api.records.MasterKey;
|
||||||
@ -760,7 +760,6 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
|
|||||||
}
|
}
|
||||||
|
|
||||||
protected void startStatusUpdater() {
|
protected void startStatusUpdater() {
|
||||||
|
|
||||||
statusUpdaterRunnable = new StatusUpdaterRunnable();
|
statusUpdaterRunnable = new StatusUpdaterRunnable();
|
||||||
statusUpdater =
|
statusUpdater =
|
||||||
new Thread(statusUpdaterRunnable, "Node Status Updater");
|
new Thread(statusUpdaterRunnable, "Node Status Updater");
|
||||||
@ -1043,7 +1042,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
|
|||||||
.getNMTokenSecretManager().getCurrentKey(),
|
.getNMTokenSecretManager().getCurrentKey(),
|
||||||
nodeLabelsForHeartbeat,
|
nodeLabelsForHeartbeat,
|
||||||
NodeStatusUpdaterImpl.this.context
|
NodeStatusUpdaterImpl.this.context
|
||||||
.getRegisteredCollectors());
|
.getRegisteringCollectors());
|
||||||
|
|
||||||
if (logAggregationEnabled) {
|
if (logAggregationEnabled) {
|
||||||
// pull log aggregation status for application running in this NM
|
// pull log aggregation status for application running in this NM
|
||||||
@ -1134,7 +1133,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (YarnConfiguration.timelineServiceV2Enabled(context.getConf())) {
|
if (YarnConfiguration.timelineServiceV2Enabled(context.getConf())) {
|
||||||
updateTimelineClientsAddress(response);
|
updateTimelineCollectorData(response);
|
||||||
}
|
}
|
||||||
|
|
||||||
} catch (ConnectException e) {
|
} catch (ConnectException e) {
|
||||||
@ -1164,40 +1163,48 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void updateTimelineClientsAddress(
|
private void updateTimelineCollectorData(
|
||||||
NodeHeartbeatResponse response) {
|
NodeHeartbeatResponse response) {
|
||||||
Map<ApplicationId, String> knownCollectorsMap =
|
Map<ApplicationId, AppCollectorData> incomingCollectorsMap =
|
||||||
response.getAppCollectorsMap();
|
response.getAppCollectors();
|
||||||
if (knownCollectorsMap == null) {
|
if (incomingCollectorsMap == null) {
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug("No collectors to update RM");
|
LOG.debug("No collectors to update RM");
|
||||||
}
|
}
|
||||||
} else {
|
return;
|
||||||
Set<Map.Entry<ApplicationId, String>> rmKnownCollectors =
|
}
|
||||||
knownCollectorsMap.entrySet();
|
Map<ApplicationId, AppCollectorData> knownCollectors =
|
||||||
for (Map.Entry<ApplicationId, String> entry : rmKnownCollectors) {
|
context.getKnownCollectors();
|
||||||
ApplicationId appId = entry.getKey();
|
for (Map.Entry<ApplicationId, AppCollectorData> entry
|
||||||
String collectorAddr = entry.getValue();
|
: incomingCollectorsMap.entrySet()) {
|
||||||
|
ApplicationId appId = entry.getKey();
|
||||||
|
AppCollectorData collectorData = entry.getValue();
|
||||||
|
|
||||||
// Only handle applications running on local node.
|
// Only handle applications running on local node.
|
||||||
// Not include apps with timeline collectors running in local
|
Application application = context.getApplications().get(appId);
|
||||||
Application application = context.getApplications().get(appId);
|
if (application != null) {
|
||||||
// TODO this logic could be problematic if the collector address
|
// Update collector data if the newly received data happens after
|
||||||
// gets updated due to NM restart or collector service failure
|
// the known data (updates the known data).
|
||||||
if (application != null &&
|
AppCollectorData existingData = knownCollectors.get(appId);
|
||||||
!context.getRegisteredCollectors().containsKey(appId)) {
|
if (AppCollectorData.happensBefore(existingData, collectorData)) {
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug("Sync a new collector address: " + collectorAddr +
|
LOG.debug("Sync a new collector address: "
|
||||||
" for application: " + appId + " from RM.");
|
+ collectorData.getCollectorAddr()
|
||||||
|
+ " for application: " + appId + " from RM.");
|
||||||
}
|
}
|
||||||
|
// Update information for clients.
|
||||||
NMTimelinePublisher nmTimelinePublisher =
|
NMTimelinePublisher nmTimelinePublisher =
|
||||||
context.getNMTimelinePublisher();
|
context.getNMTimelinePublisher();
|
||||||
if (nmTimelinePublisher != null) {
|
if (nmTimelinePublisher != null) {
|
||||||
nmTimelinePublisher.setTimelineServiceAddress(
|
nmTimelinePublisher.setTimelineServiceAddress(
|
||||||
application.getAppId(), collectorAddr);
|
application.getAppId(), collectorData.getCollectorAddr());
|
||||||
}
|
}
|
||||||
|
// Update information for the node manager itself.
|
||||||
|
knownCollectors.put(appId, collectorData);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
// Remove the registering collector data
|
||||||
|
context.getRegisteringCollectors().remove(entry.getKey());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -26,6 +26,7 @@ import org.slf4j.Logger;
|
|||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
||||||
import org.apache.hadoop.ipc.Server;
|
import org.apache.hadoop.ipc.Server;
|
||||||
import org.apache.hadoop.service.CompositeService;
|
import org.apache.hadoop.service.CompositeService;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
@ -37,10 +38,10 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorCon
|
|||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextResponse;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextResponse;
|
||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoRequest;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoRequest;
|
||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoResponse;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoResponse;
|
||||||
import org.apache.hadoop.yarn.server.api.records.AppCollectorsMap;
|
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
|
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
|
||||||
|
import org.apache.hadoop.yarn.server.nodemanager.security.authorize.NMPolicyProvider;
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher;
|
import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -74,16 +75,21 @@ public class NMCollectorService extends CompositeService implements
|
|||||||
|
|
||||||
Configuration serverConf = new Configuration(conf);
|
Configuration serverConf = new Configuration(conf);
|
||||||
|
|
||||||
// TODO Security settings.
|
|
||||||
YarnRPC rpc = YarnRPC.create(conf);
|
YarnRPC rpc = YarnRPC.create(conf);
|
||||||
|
|
||||||
|
// Kerberos based authentication to be used for CollectorNodemanager
|
||||||
|
// protocol if security is enabled.
|
||||||
server =
|
server =
|
||||||
rpc.getServer(CollectorNodemanagerProtocol.class, this,
|
rpc.getServer(CollectorNodemanagerProtocol.class, this,
|
||||||
collectorServerAddress, serverConf,
|
collectorServerAddress, serverConf, null,
|
||||||
this.context.getNMTokenSecretManager(),
|
|
||||||
conf.getInt(YarnConfiguration.NM_COLLECTOR_SERVICE_THREAD_COUNT,
|
conf.getInt(YarnConfiguration.NM_COLLECTOR_SERVICE_THREAD_COUNT,
|
||||||
YarnConfiguration.DEFAULT_NM_COLLECTOR_SERVICE_THREAD_COUNT));
|
YarnConfiguration.DEFAULT_NM_COLLECTOR_SERVICE_THREAD_COUNT));
|
||||||
|
|
||||||
|
if (conf.getBoolean(
|
||||||
|
CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, false)) {
|
||||||
|
server.refreshServiceAcl(conf, new NMPolicyProvider());
|
||||||
|
}
|
||||||
|
|
||||||
server.start();
|
server.start();
|
||||||
collectorServerAddress = conf.updateConnectAddr(
|
collectorServerAddress = conf.updateConnectAddr(
|
||||||
YarnConfiguration.NM_BIND_HOST,
|
YarnConfiguration.NM_BIND_HOST,
|
||||||
@ -95,7 +101,6 @@ public class NMCollectorService extends CompositeService implements
|
|||||||
LOG.info("NMCollectorService started at " + collectorServerAddress);
|
LOG.info("NMCollectorService started at " + collectorServerAddress);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void serviceStop() throws Exception {
|
public void serviceStop() throws Exception {
|
||||||
if (server != null) {
|
if (server != null) {
|
||||||
@ -108,23 +113,31 @@ public class NMCollectorService extends CompositeService implements
|
|||||||
@Override
|
@Override
|
||||||
public ReportNewCollectorInfoResponse reportNewCollectorInfo(
|
public ReportNewCollectorInfoResponse reportNewCollectorInfo(
|
||||||
ReportNewCollectorInfoRequest request) throws YarnException, IOException {
|
ReportNewCollectorInfoRequest request) throws YarnException, IOException {
|
||||||
List<AppCollectorsMap> newCollectorsList = request.getAppCollectorsList();
|
List<AppCollectorData> newCollectorsList = request.getAppCollectorsList();
|
||||||
if (newCollectorsList != null && !newCollectorsList.isEmpty()) {
|
if (newCollectorsList != null && !newCollectorsList.isEmpty()) {
|
||||||
Map<ApplicationId, String> newCollectorsMap =
|
Map<ApplicationId, AppCollectorData> newCollectorsMap =
|
||||||
new HashMap<ApplicationId, String>();
|
new HashMap<>();
|
||||||
for (AppCollectorsMap collector : newCollectorsList) {
|
for (AppCollectorData collector : newCollectorsList) {
|
||||||
ApplicationId appId = collector.getApplicationId();
|
ApplicationId appId = collector.getApplicationId();
|
||||||
String collectorAddr = collector.getCollectorAddr();
|
newCollectorsMap.put(appId, collector);
|
||||||
newCollectorsMap.put(appId, collectorAddr);
|
|
||||||
// set registered collector address to TimelineClient.
|
// set registered collector address to TimelineClient.
|
||||||
|
// TODO: Do we need to do this after we received confirmation from
|
||||||
|
// the RM?
|
||||||
NMTimelinePublisher nmTimelinePublisher =
|
NMTimelinePublisher nmTimelinePublisher =
|
||||||
context.getNMTimelinePublisher();
|
context.getNMTimelinePublisher();
|
||||||
if (nmTimelinePublisher != null) {
|
if (nmTimelinePublisher != null) {
|
||||||
nmTimelinePublisher.setTimelineServiceAddress(appId, collectorAddr);
|
nmTimelinePublisher.setTimelineServiceAddress(appId,
|
||||||
|
collector.getCollectorAddr());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
((NodeManager.NMContext)context).addRegisteredCollectors(
|
Map<ApplicationId, AppCollectorData> registeringCollectors
|
||||||
newCollectorsMap);
|
= context.getRegisteringCollectors();
|
||||||
|
if (registeringCollectors != null) {
|
||||||
|
registeringCollectors.putAll(newCollectorsMap);
|
||||||
|
} else {
|
||||||
|
LOG.warn("collectors are added when the registered collectors are " +
|
||||||
|
"initialized");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
return ReportNewCollectorInfoResponse.newInstance();
|
return ReportNewCollectorInfoResponse.newInstance();
|
||||||
|
@ -244,7 +244,8 @@ public class AuxServices extends AbstractService
|
|||||||
for (AuxiliaryService serv : serviceMap.values()) {
|
for (AuxiliaryService serv : serviceMap.values()) {
|
||||||
try {
|
try {
|
||||||
serv.initializeContainer(new ContainerInitializationContext(
|
serv.initializeContainer(new ContainerInitializationContext(
|
||||||
event.getUser(), event.getContainer().getContainerId(),
|
event.getContainer().getUser(),
|
||||||
|
event.getContainer().getContainerId(),
|
||||||
event.getContainer().getResource(), event.getContainer()
|
event.getContainer().getResource(), event.getContainer()
|
||||||
.getContainerTokenIdentifier().getContainerType()));
|
.getContainerTokenIdentifier().getContainerType()));
|
||||||
} catch (Throwable th) {
|
} catch (Throwable th) {
|
||||||
|
@ -155,6 +155,7 @@ import org.apache.hadoop.yarn.server.nodemanager.security.authorize.NMPolicyProv
|
|||||||
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;
|
||||||
|
|
||||||
@ -401,6 +402,16 @@ public class ContainerManagerImpl extends CompositeService implements
|
|||||||
LOG.debug(
|
LOG.debug(
|
||||||
"Recovering Flow context: " + fc + " for an application " + appId);
|
"Recovering Flow context: " + fc + " for an application " + appId);
|
||||||
}
|
}
|
||||||
|
} else {
|
||||||
|
// in upgrade situations, where there is no prior existing flow context,
|
||||||
|
// default would be used.
|
||||||
|
fc = new FlowContext(TimelineUtils.generateDefaultFlowName(null, appId),
|
||||||
|
YarnConfiguration.DEFAULT_FLOW_VERSION, appId.getClusterTimestamp());
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug(
|
||||||
|
"No prior existing flow context found. Using default Flow context: "
|
||||||
|
+ fc + " for an application " + appId);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
LOG.info("Recovering application " + appId);
|
LOG.info("Recovering application " + appId);
|
||||||
@ -1052,10 +1063,11 @@ public class ContainerManagerImpl extends CompositeService implements
|
|||||||
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 +1124,7 @@ public class ContainerManagerImpl extends CompositeService implements
|
|||||||
}
|
}
|
||||||
|
|
||||||
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 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
|||||||
|
|
||||||
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 class ApplicationContainerFinishedEvent extends ApplicationEvent {
|
|||||||
return containerStatus;
|
return containerStatus;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public long getContainerStartTime() {
|
||||||
|
return containerStartTime;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -45,6 +45,7 @@ import org.apache.hadoop.yarn.event.Dispatcher;
|
|||||||
import org.apache.hadoop.yarn.proto.YarnProtos;
|
import org.apache.hadoop.yarn.proto.YarnProtos;
|
||||||
import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.ContainerManagerApplicationProto;
|
import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.ContainerManagerApplicationProto;
|
||||||
import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.FlowContextProto;
|
import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.FlowContextProto;
|
||||||
|
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEvent;
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEvent;
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEventType;
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEventType;
|
||||||
@ -558,6 +559,29 @@ public class ApplicationImpl implements Application {
|
|||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
static class AppCompletelyDoneTransition implements
|
static class AppCompletelyDoneTransition implements
|
||||||
SingleArcTransition<ApplicationImpl, ApplicationEvent> {
|
SingleArcTransition<ApplicationImpl, ApplicationEvent> {
|
||||||
|
|
||||||
|
private void updateCollectorStatus(ApplicationImpl app) {
|
||||||
|
// Remove collectors info for finished apps.
|
||||||
|
// TODO check we remove related collectors info in failure cases
|
||||||
|
// (YARN-3038)
|
||||||
|
Map<ApplicationId, AppCollectorData> registeringCollectors
|
||||||
|
= app.context.getRegisteringCollectors();
|
||||||
|
if (registeringCollectors != null) {
|
||||||
|
registeringCollectors.remove(app.getAppId());
|
||||||
|
}
|
||||||
|
Map<ApplicationId, AppCollectorData> knownCollectors =
|
||||||
|
app.context.getKnownCollectors();
|
||||||
|
if (knownCollectors != null) {
|
||||||
|
knownCollectors.remove(app.getAppId());
|
||||||
|
}
|
||||||
|
// stop timelineClient when application get finished.
|
||||||
|
NMTimelinePublisher nmTimelinePublisher =
|
||||||
|
app.context.getNMTimelinePublisher();
|
||||||
|
if (nmTimelinePublisher != null) {
|
||||||
|
nmTimelinePublisher.stopTimelineClient(app.getAppId());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void transition(ApplicationImpl app, ApplicationEvent event) {
|
public void transition(ApplicationImpl app, ApplicationEvent event) {
|
||||||
|
|
||||||
@ -566,20 +590,7 @@ public class ApplicationImpl implements Application {
|
|||||||
new LogHandlerAppFinishedEvent(app.appId));
|
new LogHandlerAppFinishedEvent(app.appId));
|
||||||
|
|
||||||
app.context.getNMTokenSecretManager().appFinished(app.getAppId());
|
app.context.getNMTokenSecretManager().appFinished(app.getAppId());
|
||||||
// Remove collectors info for finished apps.
|
updateCollectorStatus(app);
|
||||||
// TODO check we remove related collectors info in failure cases
|
|
||||||
// (YARN-3038)
|
|
||||||
Map<ApplicationId, String> registeredCollectors =
|
|
||||||
app.context.getRegisteredCollectors();
|
|
||||||
if (registeredCollectors != null) {
|
|
||||||
registeredCollectors.remove(app.getAppId());
|
|
||||||
}
|
|
||||||
// stop timelineClient when application get finished.
|
|
||||||
NMTimelinePublisher nmTimelinePublisher =
|
|
||||||
app.context.getNMTimelinePublisher();
|
|
||||||
if (nmTimelinePublisher != null) {
|
|
||||||
nmTimelinePublisher.stopTimelineClient(app.getAppId());
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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 @@ public class ContainerImpl implements Container {
|
|||||||
|
|
||||||
/** 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 class ContainerImpl implements Container {
|
|||||||
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 class ContainerImpl implements Container {
|
|||||||
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 class ContainerImpl implements Container {
|
|||||||
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 @@ public class ContainerImpl implements Container {
|
|||||||
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 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
|
|||||||
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 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
|
|||||||
|
|
||||||
@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 class NMLeveldbStateStoreService extends NMStateStoreService {
|
|||||||
|
|
||||||
@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 class NMNullStateStoreService extends NMStateStoreService {
|
|||||||
|
|
||||||
@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 abstract class NMStateStoreService extends AbstractService {
|
|||||||
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 abstract class NMStateStoreService extends AbstractService {
|
|||||||
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 abstract class NMStateStoreService extends AbstractService {
|
|||||||
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 class NMStateStoreService extends AbstractService {
|
|||||||
* 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;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -23,6 +23,7 @@ import org.apache.hadoop.security.authorize.PolicyProvider;
|
|||||||
import org.apache.hadoop.security.authorize.Service;
|
import org.apache.hadoop.security.authorize.Service;
|
||||||
import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
|
import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
|
||||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||||
|
import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocolPB;
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocolPB;
|
import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocolPB;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -32,18 +33,21 @@ import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocolPB;
|
|||||||
@InterfaceStability.Unstable
|
@InterfaceStability.Unstable
|
||||||
public class NMPolicyProvider extends PolicyProvider {
|
public class NMPolicyProvider extends PolicyProvider {
|
||||||
|
|
||||||
private static final Service[] nodeManagerServices =
|
private static final Service[] NODE_MANAGER_SERVICES =
|
||||||
new Service[] {
|
new Service[] {
|
||||||
new Service(
|
new Service(YarnConfiguration.
|
||||||
YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_CONTAINER_MANAGEMENT_PROTOCOL,
|
YARN_SECURITY_SERVICE_AUTHORIZATION_CONTAINER_MANAGEMENT_PROTOCOL,
|
||||||
ContainerManagementProtocolPB.class),
|
ContainerManagementProtocolPB.class),
|
||||||
new Service(YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCE_LOCALIZER,
|
new Service(YarnConfiguration.
|
||||||
LocalizationProtocolPB.class)
|
YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCE_LOCALIZER,
|
||||||
};
|
LocalizationProtocolPB.class),
|
||||||
|
new Service(YarnConfiguration.
|
||||||
|
YARN_SECURITY_SERVICE_AUTHORIZATION_COLLECTOR_NODEMANAGER_PROTOCOL,
|
||||||
|
CollectorNodemanagerProtocolPB.class)
|
||||||
|
};
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Service[] getServices() {
|
public Service[] getServices() {
|
||||||
return nodeManagerServices;
|
return NODE_MANAGER_SERVICES;
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -19,6 +19,7 @@
|
|||||||
package org.apache.hadoop.yarn.server.nodemanager.timelineservice;
|
package org.apache.hadoop.yarn.server.nodemanager.timelineservice;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.security.PrivilegedExceptionAction;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
@ -26,8 +27,10 @@ import org.slf4j.Logger;
|
|||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.apache.hadoop.service.CompositeService;
|
import org.apache.hadoop.service.CompositeService;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
|
import org.apache.hadoop.yarn.api.records.CollectorInfo;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerState;
|
import org.apache.hadoop.yarn.api.records.ContainerState;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
||||||
@ -55,6 +58,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.even
|
|||||||
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;
|
||||||
@ -77,6 +81,8 @@ public class NMTimelinePublisher extends CompositeService {
|
|||||||
|
|
||||||
private String httpAddress;
|
private String httpAddress;
|
||||||
|
|
||||||
|
private UserGroupInformation nmLoginUGI;
|
||||||
|
|
||||||
private final Map<ApplicationId, TimelineV2Client> appToClientMap;
|
private final Map<ApplicationId, TimelineV2Client> appToClientMap;
|
||||||
|
|
||||||
public NMTimelinePublisher(Context context) {
|
public NMTimelinePublisher(Context context) {
|
||||||
@ -91,6 +97,9 @@ public class NMTimelinePublisher extends CompositeService {
|
|||||||
dispatcher.register(NMTimelineEventType.class,
|
dispatcher.register(NMTimelineEventType.class,
|
||||||
new ForwardingEventHandler());
|
new ForwardingEventHandler());
|
||||||
addIfService(dispatcher);
|
addIfService(dispatcher);
|
||||||
|
this.nmLoginUGI = UserGroupInformation.isSecurityEnabled() ?
|
||||||
|
UserGroupInformation.getLoginUser() :
|
||||||
|
UserGroupInformation.getCurrentUser();
|
||||||
super.serviceInit(conf);
|
super.serviceInit(conf);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -149,6 +158,8 @@ public class NMTimelinePublisher extends CompositeService {
|
|||||||
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 +206,17 @@ public class NMTimelinePublisher extends CompositeService {
|
|||||||
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 +228,14 @@ public class NMTimelinePublisher extends CompositeService {
|
|||||||
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 +251,8 @@ public class NMTimelinePublisher extends CompositeService {
|
|||||||
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 +316,7 @@ public class NMTimelinePublisher extends CompositeService {
|
|||||||
ApplicationContainerFinishedEvent evnt =
|
ApplicationContainerFinishedEvent evnt =
|
||||||
(ApplicationContainerFinishedEvent) event;
|
(ApplicationContainerFinishedEvent) event;
|
||||||
publishContainerFinishedEvent(evnt.getContainerStatus(),
|
publishContainerFinishedEvent(evnt.getContainerStatus(),
|
||||||
event.getTimestamp());
|
event.getTimestamp(), evnt.getContainerStartTime());
|
||||||
break;
|
break;
|
||||||
|
|
||||||
default:
|
default:
|
||||||
@ -391,11 +407,23 @@ public class NMTimelinePublisher extends CompositeService {
|
|||||||
|
|
||||||
public void createTimelineClient(ApplicationId appId) {
|
public void createTimelineClient(ApplicationId appId) {
|
||||||
if (!appToClientMap.containsKey(appId)) {
|
if (!appToClientMap.containsKey(appId)) {
|
||||||
TimelineV2Client timelineClient =
|
try {
|
||||||
TimelineV2Client.createTimelineClient(appId);
|
TimelineV2Client timelineClient =
|
||||||
timelineClient.init(getConfig());
|
nmLoginUGI.doAs(new PrivilegedExceptionAction<TimelineV2Client>() {
|
||||||
timelineClient.start();
|
@Override
|
||||||
appToClientMap.put(appId, timelineClient);
|
public TimelineV2Client run() throws Exception {
|
||||||
|
TimelineV2Client timelineClient =
|
||||||
|
TimelineV2Client.createTimelineClient(appId);
|
||||||
|
timelineClient.init(getConfig());
|
||||||
|
timelineClient.start();
|
||||||
|
return timelineClient;
|
||||||
|
}
|
||||||
|
});
|
||||||
|
appToClientMap.put(appId, timelineClient);
|
||||||
|
} catch (IOException | InterruptedException | RuntimeException |
|
||||||
|
Error e) {
|
||||||
|
LOG.warn("Unable to create timeline client for app " + appId, e);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -410,11 +438,11 @@ public class NMTimelinePublisher extends CompositeService {
|
|||||||
String collectorAddr) {
|
String collectorAddr) {
|
||||||
TimelineV2Client client = appToClientMap.get(appId);
|
TimelineV2Client client = appToClientMap.get(appId);
|
||||||
if (client != null) {
|
if (client != null) {
|
||||||
client.setTimelineServiceAddress(collectorAddr);
|
client.setTimelineCollectorInfo(CollectorInfo.newInstance(collectorAddr));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private TimelineV2Client getTimelineClient(ApplicationId appId) {
|
private TimelineV2Client getTimelineClient(ApplicationId appId) {
|
||||||
return appToClientMap.get(appId);
|
return appToClientMap.get(appId);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -59,6 +59,7 @@ import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
|||||||
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.server.api.protocolrecords.LogAggregationReport;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
|
||||||
|
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
||||||
import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
|
import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
|
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
||||||
@ -659,7 +660,13 @@ public abstract class BaseAMRMProxyTest {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Map<ApplicationId, String> getRegisteredCollectors() {
|
public ConcurrentMap<ApplicationId, AppCollectorData>
|
||||||
|
getRegisteringCollectors() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ConcurrentMap<ApplicationId, AppCollectorData> getKnownCollectors() {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -601,7 +601,7 @@ public class TestApplication {
|
|||||||
|
|
||||||
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 class NMMemoryStateStoreService extends NMStateStoreService {
|
|||||||
|
|
||||||
@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 class TestNMLeveldbStateStoreService {
|
|||||||
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 class TestNMLeveldbStateStoreService {
|
|||||||
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 class TestNMLeveldbStateStoreService {
|
|||||||
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 class MockContainer implements Container {
|
|||||||
public boolean isRecovering() {
|
public boolean isRecovering() {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public long getContainerStartTime() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -218,8 +218,8 @@ public class TestNMWebServer {
|
|||||||
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() {
|
||||||
|
@ -45,8 +45,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest
|
|||||||
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
|
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
|
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
|
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
|
||||||
|
|
||||||
|
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
import org.apache.hadoop.yarn.api.records.Container;
|
import org.apache.hadoop.yarn.api.records.Container;
|
||||||
@ -61,6 +59,7 @@ import org.apache.hadoop.yarn.ipc.YarnRPC;
|
|||||||
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
|
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
|
import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
|
||||||
@ -254,7 +253,7 @@ public class ApplicationMasterService extends AbstractService implements
|
|||||||
|
|
||||||
// Remove collector address when app get finished.
|
// Remove collector address when app get finished.
|
||||||
if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) {
|
if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) {
|
||||||
rmApp.removeCollectorAddr();
|
((RMAppImpl) rmApp).removeCollectorData();
|
||||||
}
|
}
|
||||||
// checking whether the app exits in RMStateStore at first not to throw
|
// checking whether the app exits in RMStateStore at first not to throw
|
||||||
// ApplicationDoesNotExistInCacheException before and after
|
// ApplicationDoesNotExistInCacheException before and after
|
||||||
|
@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterReque
|
|||||||
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
|
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
||||||
|
import org.apache.hadoop.yarn.api.records.CollectorInfo;
|
||||||
import org.apache.hadoop.yarn.api.records.Container;
|
import org.apache.hadoop.yarn.api.records.Container;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
|
import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
|
||||||
@ -293,9 +294,10 @@ final class DefaultAMSProcessor implements ApplicationMasterServiceProcessor {
|
|||||||
// add collector address for this application
|
// add collector address for this application
|
||||||
if (YarnConfiguration.timelineServiceV2Enabled(
|
if (YarnConfiguration.timelineServiceV2Enabled(
|
||||||
getRmContext().getYarnConfiguration())) {
|
getRmContext().getYarnConfiguration())) {
|
||||||
response.setCollectorAddr(
|
CollectorInfo collectorInfo = app.getCollectorInfo();
|
||||||
getRmContext().getRMApps().get(appAttemptId.getApplicationId())
|
if (collectorInfo != null) {
|
||||||
.getCollectorAddr());
|
response.setCollectorInfo(collectorInfo);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// add preemption to the allocateResponse message (if any)
|
// add preemption to the allocateResponse message (if any)
|
||||||
|
@ -27,6 +27,7 @@ import java.util.List;
|
|||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.ConcurrentMap;
|
import java.util.concurrent.ConcurrentMap;
|
||||||
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||||
import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
|
import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
|
||||||
import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
|
import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
|
||||||
@ -63,12 +64,14 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequ
|
|||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
|
||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.UnRegisterNodeManagerRequest;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.UnRegisterNodeManagerRequest;
|
||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.UnRegisterNodeManagerResponse;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.UnRegisterNodeManagerResponse;
|
||||||
|
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
||||||
import org.apache.hadoop.yarn.server.api.records.MasterKey;
|
import org.apache.hadoop.yarn.server.api.records.MasterKey;
|
||||||
import org.apache.hadoop.yarn.server.api.records.NodeAction;
|
import org.apache.hadoop.yarn.server.api.records.NodeAction;
|
||||||
import org.apache.hadoop.yarn.server.api.records.NodeStatus;
|
import org.apache.hadoop.yarn.server.api.records.NodeStatus;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NodeLabelsUtils;
|
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NodeLabelsUtils;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.resource.DynamicResourceConfiguration;
|
import org.apache.hadoop.yarn.server.resourcemanager.resource.DynamicResourceConfiguration;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
||||||
@ -118,6 +121,8 @@ public class ResourceTrackerService extends AbstractService implements
|
|||||||
private boolean isDelegatedCentralizedNodeLabelsConf;
|
private boolean isDelegatedCentralizedNodeLabelsConf;
|
||||||
private DynamicResourceConfiguration drConf;
|
private DynamicResourceConfiguration drConf;
|
||||||
|
|
||||||
|
private final AtomicLong timelineCollectorVersion = new AtomicLong(0);
|
||||||
|
|
||||||
public ResourceTrackerService(RMContext rmContext,
|
public ResourceTrackerService(RMContext rmContext,
|
||||||
NodesListManager nodesListManager,
|
NodesListManager nodesListManager,
|
||||||
NMLivelinessMonitor nmLivelinessMonitor,
|
NMLivelinessMonitor nmLivelinessMonitor,
|
||||||
@ -521,16 +526,6 @@ public class ResourceTrackerService extends AbstractService implements
|
|||||||
message);
|
message);
|
||||||
}
|
}
|
||||||
|
|
||||||
boolean timelineV2Enabled =
|
|
||||||
YarnConfiguration.timelineServiceV2Enabled(getConfig());
|
|
||||||
if (timelineV2Enabled) {
|
|
||||||
// Check & update collectors info from request.
|
|
||||||
// TODO make sure it won't have race condition issue for AM failed over
|
|
||||||
// case that the older registration could possible override the newer
|
|
||||||
// one.
|
|
||||||
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(
|
||||||
@ -545,6 +540,13 @@ public class ResourceTrackerService extends AbstractService implements
|
|||||||
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.
|
||||||
@ -613,44 +615,66 @@ public class ResourceTrackerService extends AbstractService implements
|
|||||||
|
|
||||||
private void setAppCollectorsMapToResponse(
|
private void setAppCollectorsMapToResponse(
|
||||||
List<ApplicationId> runningApps, NodeHeartbeatResponse response) {
|
List<ApplicationId> runningApps, NodeHeartbeatResponse response) {
|
||||||
Map<ApplicationId, String> liveAppCollectorsMap = new
|
Map<ApplicationId, AppCollectorData> liveAppCollectorsMap = new
|
||||||
HashMap<ApplicationId, String>();
|
HashMap<>();
|
||||||
Map<ApplicationId, RMApp> rmApps = rmContext.getRMApps();
|
Map<ApplicationId, RMApp> rmApps = rmContext.getRMApps();
|
||||||
// Set collectors for all running apps on this node.
|
// Set collectors for all running apps on this node.
|
||||||
for (ApplicationId appId : runningApps) {
|
for (ApplicationId appId : runningApps) {
|
||||||
String appCollectorAddr = rmApps.get(appId).getCollectorAddr();
|
RMApp app = rmApps.get(appId);
|
||||||
if (appCollectorAddr != null) {
|
if (app != null) {
|
||||||
liveAppCollectorsMap.put(appId, appCollectorAddr);
|
AppCollectorData appCollectorData = rmApps.get(appId)
|
||||||
} else {
|
.getCollectorData();
|
||||||
if (LOG.isDebugEnabled()) {
|
if (appCollectorData != null) {
|
||||||
LOG.debug("Collector for applicaton: " + appId +
|
liveAppCollectorsMap.put(appId, appCollectorData);
|
||||||
" hasn't registered yet!");
|
} else {
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("Collector for applicaton: " + appId +
|
||||||
|
" hasn't registered yet!");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
response.setAppCollectorsMap(liveAppCollectorsMap);
|
response.setAppCollectors(liveAppCollectorsMap);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void updateAppCollectorsMap(NodeHeartbeatRequest request) {
|
private void updateAppCollectorsMap(NodeHeartbeatRequest request) {
|
||||||
Map<ApplicationId, String> registeredCollectorsMap =
|
Map<ApplicationId, AppCollectorData> registeringCollectorsMap =
|
||||||
request.getRegisteredCollectors();
|
request.getRegisteringCollectors();
|
||||||
if (registeredCollectorsMap != null
|
if (registeringCollectorsMap != null
|
||||||
&& !registeredCollectorsMap.isEmpty()) {
|
&& !registeringCollectorsMap.isEmpty()) {
|
||||||
Map<ApplicationId, RMApp> rmApps = rmContext.getRMApps();
|
Map<ApplicationId, RMApp> rmApps = rmContext.getRMApps();
|
||||||
for (Map.Entry<ApplicationId, String> entry:
|
for (Map.Entry<ApplicationId, AppCollectorData> entry:
|
||||||
registeredCollectorsMap.entrySet()) {
|
registeringCollectorsMap.entrySet()) {
|
||||||
ApplicationId appId = entry.getKey();
|
ApplicationId appId = entry.getKey();
|
||||||
String collectorAddr = entry.getValue();
|
AppCollectorData collectorData = entry.getValue();
|
||||||
if (collectorAddr != null && !collectorAddr.isEmpty()) {
|
if (collectorData != null) {
|
||||||
|
if (!collectorData.isStamped()) {
|
||||||
|
// Stamp the collector if we have not done so
|
||||||
|
collectorData.setRMIdentifier(
|
||||||
|
ResourceManager.getClusterTimeStamp());
|
||||||
|
collectorData.setVersion(
|
||||||
|
timelineCollectorVersion.getAndIncrement());
|
||||||
|
}
|
||||||
RMApp rmApp = rmApps.get(appId);
|
RMApp rmApp = rmApps.get(appId);
|
||||||
if (rmApp == null) {
|
if (rmApp == null) {
|
||||||
LOG.warn("Cannot update collector info because application ID: " +
|
LOG.warn("Cannot update collector info because application ID: " +
|
||||||
appId + " is not found in RMContext!");
|
appId + " is not found in RMContext!");
|
||||||
} else {
|
} else {
|
||||||
String previousCollectorAddr = rmApp.getCollectorAddr();
|
synchronized (rmApp) {
|
||||||
if (previousCollectorAddr == null
|
AppCollectorData previousCollectorData = rmApp.getCollectorData();
|
||||||
|| !previousCollectorAddr.equals(collectorAddr)) {
|
if (AppCollectorData.happensBefore(previousCollectorData,
|
||||||
rmApp.setCollectorAddr(collectorAddr);
|
collectorData)) {
|
||||||
|
// Sending collector update event.
|
||||||
|
// Note: RM has to store the newly received collector data
|
||||||
|
// synchronously. Otherwise, the RM may send out stale collector
|
||||||
|
// data before this update is done, and the RM then crashes, the
|
||||||
|
// newly updated collector data will get lost.
|
||||||
|
LOG.info("Update collector information for application " + appId
|
||||||
|
+ " with new address: " + collectorData.getCollectorAddr()
|
||||||
|
+ " timestamp: " + collectorData.getRMIdentifier()
|
||||||
|
+ ", " + collectorData.getVersion());
|
||||||
|
((RMAppImpl) rmApp).setCollectorData(collectorData);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -58,6 +58,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptS
|
|||||||
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 @@ public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
|
|||||||
@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 class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
|
|||||||
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 class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
|
|||||||
@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 class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
|
|||||||
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,
|
||||||
|
@ -23,6 +23,8 @@ import java.util.List;
|
|||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.ipc.CallerContext;
|
import org.apache.hadoop.ipc.CallerContext;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
|
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||||
@ -30,6 +32,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|||||||
import org.apache.hadoop.yarn.api.records.ApplicationReport;
|
import org.apache.hadoop.yarn.api.records.ApplicationReport;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
|
import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
|
||||||
|
import org.apache.hadoop.yarn.api.records.CollectorInfo;
|
||||||
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
||||||
import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
|
import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
|
||||||
import org.apache.hadoop.yarn.api.records.NodeId;
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||||
@ -39,6 +42,7 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|||||||
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
|
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
|
||||||
import org.apache.hadoop.yarn.event.EventHandler;
|
import org.apache.hadoop.yarn.event.EventHandler;
|
||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
|
||||||
|
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
||||||
|
|
||||||
@ -180,28 +184,27 @@ public interface RMApp extends EventHandler<RMAppEvent> {
|
|||||||
String getTrackingUrl();
|
String getTrackingUrl();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The collector address for the application. It should be used only if the
|
* The timeline collector information for the application. It should be used
|
||||||
* timeline service v.2 is enabled.
|
* only if the timeline service v.2 is enabled.
|
||||||
*
|
*
|
||||||
* @return the address for the application's collector, or null if the
|
* @return the data for the application's collector, including collector
|
||||||
* timeline service v.2 is not enabled.
|
* address, RM ID, version and collector token. Return null if the timeline
|
||||||
|
* service v.2 is not enabled.
|
||||||
*/
|
*/
|
||||||
String getCollectorAddr();
|
@InterfaceAudience.Private
|
||||||
|
@InterfaceStability.Unstable
|
||||||
|
AppCollectorData getCollectorData();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Set collector address for the application. It should be used only if the
|
* The timeline collector information to be sent to AM. It should be used
|
||||||
* timeline service v.2 is enabled.
|
* only if the timeline service v.2 is enabled.
|
||||||
*
|
*
|
||||||
* @param collectorAddr the address of collector
|
* @return collector info, including collector address and collector token.
|
||||||
|
* Return null if the timeline service v.2 is not enabled.
|
||||||
*/
|
*/
|
||||||
void setCollectorAddr(String collectorAddr);
|
@InterfaceAudience.Private
|
||||||
|
@InterfaceStability.Unstable
|
||||||
/**
|
CollectorInfo getCollectorInfo();
|
||||||
* Remove collector address when application is finished or killed. It should
|
|
||||||
* be used only if the timeline service v.2 is enabled.
|
|
||||||
*/
|
|
||||||
void removeCollectorAddr();
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The original tracking url for the application master.
|
* The original tracking url for the application master.
|
||||||
* @return the original tracking url for the application master.
|
* @return the original tracking url for the application master.
|
||||||
|
@ -30,9 +30,6 @@ public enum RMAppEventType {
|
|||||||
// Source: Scheduler
|
// Source: Scheduler
|
||||||
APP_ACCEPTED,
|
APP_ACCEPTED,
|
||||||
|
|
||||||
// TODO add source later
|
|
||||||
COLLECTOR_UPDATE,
|
|
||||||
|
|
||||||
// Source: RMAppAttempt
|
// Source: RMAppAttempt
|
||||||
ATTEMPT_REGISTERED,
|
ATTEMPT_REGISTERED,
|
||||||
ATTEMPT_UNREGISTERED,
|
ATTEMPT_UNREGISTERED,
|
||||||
|
@ -55,6 +55,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
|
|||||||
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationTimeout;
|
import org.apache.hadoop.yarn.api.records.ApplicationTimeout;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
|
import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
|
||||||
|
import org.apache.hadoop.yarn.api.records.CollectorInfo;
|
||||||
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
||||||
import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
|
import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
|
||||||
import org.apache.hadoop.yarn.api.records.NodeId;
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||||
@ -72,6 +73,7 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|||||||
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
|
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
|
||||||
import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier;
|
import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier;
|
||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
|
||||||
|
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService;
|
import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEvent;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEventType;
|
import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEventType;
|
||||||
@ -165,7 +167,8 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|||||||
private long storedFinishTime = 0;
|
private long storedFinishTime = 0;
|
||||||
private int firstAttemptIdInStateStore = 1;
|
private int firstAttemptIdInStateStore = 1;
|
||||||
private int nextAttemptId = 1;
|
private int nextAttemptId = 1;
|
||||||
private volatile String collectorAddr;
|
private AppCollectorData collectorData;
|
||||||
|
private CollectorInfo collectorInfo;
|
||||||
// This field isn't protected by readlock now.
|
// This field isn't protected by readlock now.
|
||||||
private volatile RMAppAttempt currentAttempt;
|
private volatile RMAppAttempt currentAttempt;
|
||||||
private String queue;
|
private String queue;
|
||||||
@ -529,7 +532,7 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|||||||
*/
|
*/
|
||||||
public void startTimelineCollector() {
|
public void startTimelineCollector() {
|
||||||
AppLevelTimelineCollector collector =
|
AppLevelTimelineCollector collector =
|
||||||
new AppLevelTimelineCollector(applicationId);
|
new AppLevelTimelineCollector(applicationId, user);
|
||||||
rmContext.getRMTimelineCollectorManager().putIfAbsent(
|
rmContext.getRMTimelineCollectorManager().putIfAbsent(
|
||||||
applicationId, collector);
|
applicationId, collector);
|
||||||
}
|
}
|
||||||
@ -611,18 +614,22 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String getCollectorAddr() {
|
public AppCollectorData getCollectorData() {
|
||||||
return this.collectorAddr;
|
return this.collectorData;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
public void setCollectorData(AppCollectorData incomingData) {
|
||||||
public void setCollectorAddr(String collectorAddress) {
|
this.collectorData = incomingData;
|
||||||
this.collectorAddr = collectorAddress;
|
this.collectorInfo = CollectorInfo.newInstance(
|
||||||
|
incomingData.getCollectorAddr(), incomingData.getCollectorToken());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
public CollectorInfo getCollectorInfo() {
|
||||||
public void removeCollectorAddr() {
|
return this.collectorInfo;
|
||||||
this.collectorAddr = null;
|
}
|
||||||
|
|
||||||
|
public void removeCollectorData() {
|
||||||
|
this.collectorData = null;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -24,6 +24,7 @@ import java.util.Collections;
|
|||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
|
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
@ -39,6 +40,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
|
|||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
|
||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
|
||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
|
||||||
|
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
||||||
import org.apache.hadoop.yarn.server.api.records.MasterKey;
|
import org.apache.hadoop.yarn.server.api.records.MasterKey;
|
||||||
import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
|
import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
|
||||||
import org.apache.hadoop.yarn.server.api.records.NodeStatus;
|
import org.apache.hadoop.yarn.server.api.records.NodeStatus;
|
||||||
@ -60,6 +62,8 @@ public class MockNM {
|
|||||||
private String version;
|
private String version;
|
||||||
private Map<ContainerId, ContainerStatus> containerStats =
|
private Map<ContainerId, ContainerStatus> containerStats =
|
||||||
new HashMap<ContainerId, ContainerStatus>();
|
new HashMap<ContainerId, ContainerStatus>();
|
||||||
|
private Map<ApplicationId, AppCollectorData> registeringCollectors
|
||||||
|
= new ConcurrentHashMap<>();
|
||||||
|
|
||||||
public MockNM(String nodeIdStr, int memory, ResourceTrackerService resourceTracker) {
|
public MockNM(String nodeIdStr, int memory, ResourceTrackerService resourceTracker) {
|
||||||
// scale vcores based on the requested memory
|
// scale vcores based on the requested memory
|
||||||
@ -117,6 +121,15 @@ public class MockNM {
|
|||||||
true, ++responseId);
|
true, ++responseId);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void addRegisteringCollector(ApplicationId appId,
|
||||||
|
AppCollectorData data) {
|
||||||
|
this.registeringCollectors.put(appId, data);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Map<ApplicationId, AppCollectorData> getRegisteringCollectors() {
|
||||||
|
return this.registeringCollectors;
|
||||||
|
}
|
||||||
|
|
||||||
public RegisterNodeManagerResponse registerNode() throws Exception {
|
public RegisterNodeManagerResponse registerNode() throws Exception {
|
||||||
return registerNode(null, null);
|
return registerNode(null, null);
|
||||||
}
|
}
|
||||||
@ -229,6 +242,9 @@ public class MockNM {
|
|||||||
req.setNodeStatus(status);
|
req.setNodeStatus(status);
|
||||||
req.setLastKnownContainerTokenMasterKey(this.currentContainerTokenMasterKey);
|
req.setLastKnownContainerTokenMasterKey(this.currentContainerTokenMasterKey);
|
||||||
req.setLastKnownNMTokenMasterKey(this.currentNMTokenMasterKey);
|
req.setLastKnownNMTokenMasterKey(this.currentNMTokenMasterKey);
|
||||||
|
|
||||||
|
req.setRegisteringCollectors(this.registeringCollectors);
|
||||||
|
|
||||||
NodeHeartbeatResponse heartbeatResponse =
|
NodeHeartbeatResponse heartbeatResponse =
|
||||||
resourceTracker.nodeHeartbeat(req);
|
resourceTracker.nodeHeartbeat(req);
|
||||||
|
|
||||||
|
@ -0,0 +1,126 @@
|
|||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.yarn.server.resourcemanager;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||||
|
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
||||||
|
import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl;
|
||||||
|
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test if the new active RM could recover collector status on a state
|
||||||
|
* transition.
|
||||||
|
*/
|
||||||
|
public class TestRMHATimelineCollectors extends RMHATestBase {
|
||||||
|
public static final Log LOG = LogFactory
|
||||||
|
.getLog(TestSubmitApplicationWithRMHA.class);
|
||||||
|
|
||||||
|
@Before
|
||||||
|
@Override
|
||||||
|
public void setup() throws Exception {
|
||||||
|
super.setup();
|
||||||
|
confForRM1.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
|
||||||
|
confForRM2.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
|
||||||
|
confForRM1.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS,
|
||||||
|
FileSystemTimelineWriterImpl.class, TimelineWriter.class);
|
||||||
|
confForRM1.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
|
||||||
|
confForRM2.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
|
||||||
|
confForRM2.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS,
|
||||||
|
FileSystemTimelineWriterImpl.class, TimelineWriter.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testRebuildCollectorDataOnFailover() throws Exception {
|
||||||
|
startRMs();
|
||||||
|
MockNM nm1
|
||||||
|
= new MockNM("127.0.0.1:1234", 15120, rm2.getResourceTrackerService());
|
||||||
|
MockNM nm2
|
||||||
|
= new MockNM("127.0.0.1:5678", 15121, rm2.getResourceTrackerService());
|
||||||
|
RMApp app1 = rm1.submitApp(1024);
|
||||||
|
String collectorAddr1 = "1.2.3.4:5";
|
||||||
|
AppCollectorData data1 = AppCollectorData.newInstance(
|
||||||
|
app1.getApplicationId(), collectorAddr1);
|
||||||
|
nm1.addRegisteringCollector(app1.getApplicationId(), data1);
|
||||||
|
|
||||||
|
String collectorAddr2 = "5.4.3.2:1";
|
||||||
|
RMApp app2 = rm1.submitApp(1024);
|
||||||
|
AppCollectorData data2 = AppCollectorData.newInstance(
|
||||||
|
app2.getApplicationId(), collectorAddr2, rm1.getStartTime(), 1);
|
||||||
|
nm1.addRegisteringCollector(app2.getApplicationId(), data2);
|
||||||
|
|
||||||
|
explicitFailover();
|
||||||
|
|
||||||
|
List<ApplicationId> runningApps = new ArrayList<>();
|
||||||
|
runningApps.add(app1.getApplicationId());
|
||||||
|
runningApps.add(app2.getApplicationId());
|
||||||
|
nm1.registerNode(runningApps);
|
||||||
|
nm2.registerNode(runningApps);
|
||||||
|
|
||||||
|
String collectorAddr12 = "1.2.3.4:56";
|
||||||
|
AppCollectorData data12 = AppCollectorData.newInstance(
|
||||||
|
app1.getApplicationId(), collectorAddr12, rm1.getStartTime(), 0);
|
||||||
|
nm2.addRegisteringCollector(app1.getApplicationId(), data12);
|
||||||
|
|
||||||
|
String collectorAddr22 = "5.4.3.2:10";
|
||||||
|
AppCollectorData data22 = AppCollectorData.newInstance(
|
||||||
|
app2.getApplicationId(), collectorAddr22, rm1.getStartTime(), 2);
|
||||||
|
nm2.addRegisteringCollector(app2.getApplicationId(), data22);
|
||||||
|
|
||||||
|
Map<ApplicationId, AppCollectorData> results1
|
||||||
|
= nm1.nodeHeartbeat(true).getAppCollectors();
|
||||||
|
assertEquals(collectorAddr1,
|
||||||
|
results1.get(app1.getApplicationId()).getCollectorAddr());
|
||||||
|
assertEquals(collectorAddr2,
|
||||||
|
results1.get(app2.getApplicationId()).getCollectorAddr());
|
||||||
|
|
||||||
|
Map<ApplicationId, AppCollectorData> results2
|
||||||
|
= nm2.nodeHeartbeat(true).getAppCollectors();
|
||||||
|
// addr of app1 should be collectorAddr1 since it's registering (no time
|
||||||
|
// stamp).
|
||||||
|
assertEquals(collectorAddr1,
|
||||||
|
results2.get(app1.getApplicationId()).getCollectorAddr());
|
||||||
|
// addr of app2 should be collectorAddr22 since its version number is
|
||||||
|
// greater.
|
||||||
|
assertEquals(collectorAddr22,
|
||||||
|
results2.get(app2.getApplicationId()).getCollectorAddr());
|
||||||
|
|
||||||
|
// Now nm1 should get updated collector list
|
||||||
|
nm1.getRegisteringCollectors().clear();
|
||||||
|
Map<ApplicationId, AppCollectorData> results12
|
||||||
|
= nm1.nodeHeartbeat(true).getAppCollectors();
|
||||||
|
assertEquals(collectorAddr1,
|
||||||
|
results12.get(app1.getApplicationId()).getCollectorAddr());
|
||||||
|
assertEquals(collectorAddr22,
|
||||||
|
results12.get(app2.getApplicationId()).getCollectorAddr());
|
||||||
|
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
@ -63,6 +63,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
|
|||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
|
||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
|
||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.UnRegisterNodeManagerRequest;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.UnRegisterNodeManagerRequest;
|
||||||
|
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
||||||
import org.apache.hadoop.yarn.server.api.records.NodeAction;
|
import org.apache.hadoop.yarn.server.api.records.NodeAction;
|
||||||
import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
|
import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
|
||||||
import org.apache.hadoop.yarn.server.api.records.NodeStatus;
|
import org.apache.hadoop.yarn.server.api.records.NodeStatus;
|
||||||
@ -70,6 +71,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NodeLabelsUtils;
|
|||||||
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
|
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
|
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
||||||
@ -1011,13 +1013,23 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|||||||
RMNodeImpl node2 =
|
RMNodeImpl node2 =
|
||||||
(RMNodeImpl) rm.getRMContext().getRMNodes().get(nm2.getNodeId());
|
(RMNodeImpl) rm.getRMContext().getRMNodes().get(nm2.getNodeId());
|
||||||
|
|
||||||
RMApp app1 = rm.submitApp(1024);
|
RMAppImpl app1 = (RMAppImpl) rm.submitApp(1024);
|
||||||
String collectorAddr1 = "1.2.3.4:5";
|
String collectorAddr1 = "1.2.3.4:5";
|
||||||
app1.setCollectorAddr(collectorAddr1);
|
app1.setCollectorData(AppCollectorData.newInstance(
|
||||||
|
app1.getApplicationId(), collectorAddr1));
|
||||||
|
|
||||||
String collectorAddr2 = "5.4.3.2:1";
|
String collectorAddr2 = "5.4.3.2:1";
|
||||||
RMApp app2 = rm.submitApp(1024);
|
RMAppImpl app2 = (RMAppImpl) rm.submitApp(1024);
|
||||||
app2.setCollectorAddr(collectorAddr2);
|
app2.setCollectorData(AppCollectorData.newInstance(
|
||||||
|
app2.getApplicationId(), collectorAddr2));
|
||||||
|
|
||||||
|
String collectorAddr3 = "5.4.3.2:2";
|
||||||
|
app2.setCollectorData(AppCollectorData.newInstance(
|
||||||
|
app2.getApplicationId(), collectorAddr3, 0, 1));
|
||||||
|
|
||||||
|
String collectorAddr4 = "5.4.3.2:3";
|
||||||
|
app2.setCollectorData(AppCollectorData.newInstance(
|
||||||
|
app2.getApplicationId(), collectorAddr4, 1, 0));
|
||||||
|
|
||||||
// Create a running container for app1 running on nm1
|
// Create a running container for app1 running on nm1
|
||||||
ContainerId runningContainerId1 = BuilderUtils.newContainerId(
|
ContainerId runningContainerId1 = BuilderUtils.newContainerId(
|
||||||
@ -1055,14 +1067,18 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
|||||||
Assert.assertEquals(app2.getApplicationId(), node2.getRunningApps().get(0));
|
Assert.assertEquals(app2.getApplicationId(), node2.getRunningApps().get(0));
|
||||||
|
|
||||||
nodeHeartbeat1 = nm1.nodeHeartbeat(true);
|
nodeHeartbeat1 = nm1.nodeHeartbeat(true);
|
||||||
Map<ApplicationId, String> map1 = nodeHeartbeat1.getAppCollectorsMap();
|
Map<ApplicationId, AppCollectorData> map1
|
||||||
|
= nodeHeartbeat1.getAppCollectors();
|
||||||
Assert.assertEquals(1, map1.size());
|
Assert.assertEquals(1, map1.size());
|
||||||
Assert.assertEquals(collectorAddr1, map1.get(app1.getApplicationId()));
|
Assert.assertEquals(collectorAddr1,
|
||||||
|
map1.get(app1.getApplicationId()).getCollectorAddr());
|
||||||
|
|
||||||
nodeHeartbeat2 = nm2.nodeHeartbeat(true);
|
nodeHeartbeat2 = nm2.nodeHeartbeat(true);
|
||||||
Map<ApplicationId, String> map2 = nodeHeartbeat2.getAppCollectorsMap();
|
Map<ApplicationId, AppCollectorData> map2
|
||||||
|
= nodeHeartbeat2.getAppCollectors();
|
||||||
Assert.assertEquals(1, map2.size());
|
Assert.assertEquals(1, map2.size());
|
||||||
Assert.assertEquals(collectorAddr2, map2.get(app2.getApplicationId()));
|
Assert.assertEquals(collectorAddr4,
|
||||||
|
map2.get(app2.getApplicationId()).getCollectorAddr());
|
||||||
}
|
}
|
||||||
|
|
||||||
private void checkRebootedNMCount(MockRM rm2, int count)
|
private void checkRebootedNMCount(MockRM rm2, int count)
|
||||||
|
@ -31,6 +31,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport;
|
|||||||
import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
|
import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
|
import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
|
||||||
|
import org.apache.hadoop.yarn.api.records.CollectorInfo;
|
||||||
import org.apache.hadoop.yarn.api.records.Container;
|
import org.apache.hadoop.yarn.api.records.Container;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
||||||
@ -43,6 +44,7 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|||||||
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
|
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
|
||||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
|
||||||
|
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics;
|
||||||
@ -97,15 +99,7 @@ public abstract class MockAsm extends MockApps {
|
|||||||
throw new UnsupportedOperationException("Not supported yet.");
|
throw new UnsupportedOperationException("Not supported yet.");
|
||||||
}
|
}
|
||||||
@Override
|
@Override
|
||||||
public String getCollectorAddr() {
|
public AppCollectorData getCollectorData() {
|
||||||
throw new UnsupportedOperationException("Not supported yet.");
|
|
||||||
}
|
|
||||||
@Override
|
|
||||||
public void setCollectorAddr(String collectorAddr) {
|
|
||||||
throw new UnsupportedOperationException("Not supported yet.");
|
|
||||||
}
|
|
||||||
@Override
|
|
||||||
public void removeCollectorAddr() {
|
|
||||||
throw new UnsupportedOperationException("Not supported yet.");
|
throw new UnsupportedOperationException("Not supported yet.");
|
||||||
}
|
}
|
||||||
@Override
|
@Override
|
||||||
@ -246,6 +240,11 @@ public abstract class MockAsm extends MockApps {
|
|||||||
public boolean isAppInCompletedStates() {
|
public boolean isAppInCompletedStates() {
|
||||||
throw new UnsupportedOperationException("Not supported yet.");
|
throw new UnsupportedOperationException("Not supported yet.");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public CollectorInfo getCollectorInfo() {
|
||||||
|
throw new UnsupportedOperationException("Not supported yet.");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static RMApp newApplication(int i) {
|
public static RMApp newApplication(int i) {
|
||||||
|
@ -69,6 +69,7 @@ import org.apache.hadoop.yarn.server.timelineservice.collector.AppLevelTimelineC
|
|||||||
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 class TestSystemMetricsPublisherForV2 {
|
|||||||
+ 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 class TestSystemMetricsPublisherForV2 {
|
|||||||
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 class TestSystemMetricsPublisherForV2 {
|
|||||||
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 @@ public class TestSystemMetricsPublisherForV2 {
|
|||||||
}
|
}
|
||||||
|
|
||||||
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 @@ public class TestSystemMetricsPublisherForV2 {
|
|||||||
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 @@ public class TestSystemMetricsPublisherForV2 {
|
|||||||
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;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|||||||
import org.apache.hadoop.yarn.api.records.ApplicationReport;
|
import org.apache.hadoop.yarn.api.records.ApplicationReport;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
|
import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
|
||||||
|
import org.apache.hadoop.yarn.api.records.CollectorInfo;
|
||||||
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
||||||
import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
|
import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
|
||||||
import org.apache.hadoop.yarn.api.records.NodeId;
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||||
@ -43,6 +44,7 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState;
|
|||||||
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
|
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
|
||||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
|
||||||
|
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
||||||
|
|
||||||
@ -305,17 +307,8 @@ public class MockRMApp implements RMApp {
|
|||||||
throw new UnsupportedOperationException("Not supported yet.");
|
throw new UnsupportedOperationException("Not supported yet.");
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getCollectorAddr() {
|
|
||||||
throw new UnsupportedOperationException("Not supported yet.");
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void removeCollectorAddr() {
|
public AppCollectorData getCollectorData() {
|
||||||
throw new UnsupportedOperationException("Not supported yet.");
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void setCollectorAddr(String collectorAddr) {
|
|
||||||
throw new UnsupportedOperationException("Not supported yet.");
|
throw new UnsupportedOperationException("Not supported yet.");
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -333,4 +326,9 @@ public class MockRMApp implements RMApp {
|
|||||||
public boolean isAppInCompletedStates() {
|
public boolean isAppInCompletedStates() {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public CollectorInfo getCollectorInfo() {
|
||||||
|
throw new UnsupportedOperationException("Not supported yet.");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -125,6 +125,17 @@
|
|||||||
<groupId>commons-logging</groupId>
|
<groupId>commons-logging</groupId>
|
||||||
<artifactId>commons-logging</artifactId>
|
<artifactId>commons-logging</artifactId>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.bouncycastle</groupId>
|
||||||
|
<artifactId>bcprov-jdk16</artifactId>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.hadoop</groupId>
|
||||||
|
<artifactId>hadoop-auth</artifactId>
|
||||||
|
<scope>test</scope>
|
||||||
|
<type>test-jar</type>
|
||||||
|
</dependency>
|
||||||
</dependencies>
|
</dependencies>
|
||||||
|
|
||||||
<build>
|
<build>
|
||||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
x
Reference in New Issue
Block a user