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

This commit is contained in:
Varun Saxena 2017-08-30 11:41:10 +05:30
commit 32cba6c303
200 changed files with 11762 additions and 3305 deletions

View File

@ -223,4 +223,15 @@
group list is separated by a blank. For e.g. "alice,bob users,wheel".
A special value of "*" means all users are allowed.</description>
</property>
<property>
<name>security.collector-nodemanager.protocol.acl</name>
<value>*</value>
<description>ACL for CollectorNodemanagerProtocol, used by nodemanager
if timeline service v2 is enabled, for the timeline collector and nodemanager
to communicate with each other.
The ACL is a comma-separated list of user and group names. The user and
group list is separated by a blank. For e.g. "alice,bob users,wheel".
A special value of "*" means all users are allowed.</description>
</property>
</configuration>

View File

@ -78,6 +78,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.util.TimelineServiceHelper;
import com.google.common.annotations.VisibleForTesting;
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
createTaskEntity(HistoryEvent event, long timestamp, String taskId,
String entityType, String relatedJobEntity, JobId jobId,
boolean setCreatedTime) {
boolean setCreatedTime, long taskIdPrefix) {
org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
createBaseEntity(event, timestamp, entityType, setCreatedTime);
entity.setId(taskId);
@ -1133,6 +1134,7 @@ public class JobHistoryEventHandler extends AbstractService
((TaskStartedEvent)event).getTaskType().toString());
}
entity.addIsRelatedToEntity(relatedJobEntity, jobId.toString());
entity.setIdPrefix(taskIdPrefix);
return entity;
}
@ -1141,11 +1143,12 @@ public class JobHistoryEventHandler extends AbstractService
private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
createTaskAttemptEntity(HistoryEvent event, long timestamp,
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 =
createBaseEntity(event, timestamp, entityType, setCreatedTime);
entity.setId(taskAttemptId);
entity.addIsRelatedToEntity(relatedTaskEntity, taskId);
entity.setIdPrefix(taskAttemptIdPrefix);
return entity;
}
@ -1196,6 +1199,8 @@ public class JobHistoryEventHandler extends AbstractService
String taskId = null;
String taskAttemptId = null;
boolean setCreatedTime = false;
long taskIdPrefix = 0;
long taskAttemptIdPrefix = 0;
switch (event.getEventType()) {
// Handle job events
@ -1218,15 +1223,21 @@ public class JobHistoryEventHandler extends AbstractService
case TASK_STARTED:
setCreatedTime = true;
taskId = ((TaskStartedEvent)event).getTaskId().toString();
taskIdPrefix = TimelineServiceHelper.
invertLong(((TaskStartedEvent)event).getStartTime());
break;
case TASK_FAILED:
taskId = ((TaskFailedEvent)event).getTaskId().toString();
taskIdPrefix = TimelineServiceHelper.
invertLong(((TaskFailedEvent)event).getStartTime());
break;
case TASK_UPDATED:
taskId = ((TaskUpdatedEvent)event).getTaskId().toString();
break;
case TASK_FINISHED:
taskId = ((TaskFinishedEvent)event).getTaskId().toString();
taskIdPrefix = TimelineServiceHelper.
invertLong(((TaskFinishedEvent)event).getStartTime());
break;
case MAP_ATTEMPT_STARTED:
case REDUCE_ATTEMPT_STARTED:
@ -1234,6 +1245,8 @@ public class JobHistoryEventHandler extends AbstractService
taskId = ((TaskAttemptStartedEvent)event).getTaskId().toString();
taskAttemptId = ((TaskAttemptStartedEvent)event).
getTaskAttemptId().toString();
taskAttemptIdPrefix = TimelineServiceHelper.
invertLong(((TaskAttemptStartedEvent)event).getStartTime());
break;
case CLEANUP_ATTEMPT_STARTED:
case SETUP_ATTEMPT_STARTED:
@ -1253,16 +1266,22 @@ public class JobHistoryEventHandler extends AbstractService
getTaskId().toString();
taskAttemptId = ((TaskAttemptUnsuccessfulCompletionEvent)event).
getTaskAttemptId().toString();
taskAttemptIdPrefix = TimelineServiceHelper.invertLong(
((TaskAttemptUnsuccessfulCompletionEvent)event).getStartTime());
break;
case MAP_ATTEMPT_FINISHED:
taskId = ((MapAttemptFinishedEvent)event).getTaskId().toString();
taskAttemptId = ((MapAttemptFinishedEvent)event).
getAttemptId().toString();
taskAttemptIdPrefix = TimelineServiceHelper.
invertLong(((MapAttemptFinishedEvent)event).getStartTime());
break;
case REDUCE_ATTEMPT_FINISHED:
taskId = ((ReduceAttemptFinishedEvent)event).getTaskId().toString();
taskAttemptId = ((ReduceAttemptFinishedEvent)event).
getAttemptId().toString();
taskAttemptIdPrefix = TimelineServiceHelper.
invertLong(((ReduceAttemptFinishedEvent)event).getStartTime());
break;
case SETUP_ATTEMPT_FINISHED:
case CLEANUP_ATTEMPT_FINISHED:
@ -1291,12 +1310,12 @@ public class JobHistoryEventHandler extends AbstractService
// TaskEntity
tEntity = createTaskEntity(event, timestamp, taskId,
MAPREDUCE_TASK_ENTITY_TYPE, MAPREDUCE_JOB_ENTITY_TYPE,
jobId, setCreatedTime);
jobId, setCreatedTime, taskIdPrefix);
} else {
// TaskAttemptEntity
tEntity = createTaskAttemptEntity(event, timestamp, taskAttemptId,
MAPREDUCE_TASK_ATTEMPT_ENTITY_TYPE, MAPREDUCE_TASK_ENTITY_TYPE,
taskId, setCreatedTime);
taskId, setCreatedTime, taskAttemptIdPrefix);
}
}
try {

View File

@ -1530,7 +1530,7 @@ public abstract class TaskAttemptImpl implements
StringUtils.join(
LINE_SEPARATOR, taskAttempt.getDiagnostics()),
taskAttempt.getCounters(), taskAttempt
.getProgressSplitBlock().burst());
.getProgressSplitBlock().burst(), taskAttempt.launchTime);
return tauce;
}
@ -1943,35 +1943,35 @@ public abstract class TaskAttemptImpl implements
this.container == null ? -1 : this.container.getNodeId().getPort();
if (attemptId.getTaskId().getTaskType() == TaskType.MAP) {
MapAttemptFinishedEvent mfe =
new MapAttemptFinishedEvent(TypeConverter.fromYarn(attemptId),
TypeConverter.fromYarn(attemptId.getTaskId().getTaskType()),
state.toString(),
this.reportedStatus.mapFinishTime,
finishTime,
containerHostName,
containerNodePort,
this.nodeRackName == null ? "UNKNOWN" : this.nodeRackName,
this.reportedStatus.stateString,
getCounters(),
getProgressSplitBlock().burst());
eventHandler.handle(
new JobHistoryEvent(attemptId.getTaskId().getJobId(), mfe));
new MapAttemptFinishedEvent(TypeConverter.fromYarn(attemptId),
TypeConverter.fromYarn(attemptId.getTaskId().getTaskType()),
state.toString(),
this.reportedStatus.mapFinishTime,
finishTime,
containerHostName,
containerNodePort,
this.nodeRackName == null ? "UNKNOWN" : this.nodeRackName,
this.reportedStatus.stateString,
getCounters(),
getProgressSplitBlock().burst(), launchTime);
eventHandler.handle(
new JobHistoryEvent(attemptId.getTaskId().getJobId(), mfe));
} else {
ReduceAttemptFinishedEvent rfe =
new ReduceAttemptFinishedEvent(TypeConverter.fromYarn(attemptId),
TypeConverter.fromYarn(attemptId.getTaskId().getTaskType()),
state.toString(),
this.reportedStatus.shuffleFinishTime,
this.reportedStatus.sortFinishTime,
finishTime,
containerHostName,
containerNodePort,
this.nodeRackName == null ? "UNKNOWN" : this.nodeRackName,
this.reportedStatus.stateString,
getCounters(),
getProgressSplitBlock().burst());
eventHandler.handle(
new JobHistoryEvent(attemptId.getTaskId().getJobId(), rfe));
ReduceAttemptFinishedEvent rfe =
new ReduceAttemptFinishedEvent(TypeConverter.fromYarn(attemptId),
TypeConverter.fromYarn(attemptId.getTaskId().getTaskType()),
state.toString(),
this.reportedStatus.shuffleFinishTime,
this.reportedStatus.sortFinishTime,
finishTime,
containerHostName,
containerNodePort,
this.nodeRackName == null ? "UNKNOWN" : this.nodeRackName,
this.reportedStatus.stateString,
getCounters(),
getProgressSplitBlock().burst(), launchTime);
eventHandler.handle(
new JobHistoryEvent(attemptId.getTaskId().getJobId(), rfe));
}
}

View File

@ -139,6 +139,8 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
private final Set<TaskAttemptId> inProgressAttempts;
private boolean historyTaskStartGenerated = false;
// Launch time reported in history events.
private long launchTime;
private static final SingleArcTransition<TaskImpl, TaskEvent>
ATTEMPT_KILLED_TRANSITION = new AttemptKilledTransition();
@ -705,8 +707,9 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
}
private void sendTaskStartedEvent() {
launchTime = getLaunchTime();
TaskStartedEvent tse = new TaskStartedEvent(
TypeConverter.fromYarn(taskId), getLaunchTime(),
TypeConverter.fromYarn(taskId), launchTime,
TypeConverter.fromYarn(taskId.getTaskType()),
getSplitsAsString());
eventHandler
@ -714,18 +717,19 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
historyTaskStartGenerated = true;
}
private static TaskFinishedEvent createTaskFinishedEvent(TaskImpl task, TaskStateInternal taskState) {
private static TaskFinishedEvent createTaskFinishedEvent(TaskImpl task,
TaskStateInternal taskState) {
TaskFinishedEvent tfe =
new TaskFinishedEvent(TypeConverter.fromYarn(task.taskId),
TypeConverter.fromYarn(task.successfulAttempt),
task.getFinishTime(task.successfulAttempt),
TypeConverter.fromYarn(task.taskId.getTaskType()),
taskState.toString(),
task.getCounters());
taskState.toString(), task.getCounters(), task.launchTime);
return tfe;
}
private static TaskFailedEvent createTaskFailedEvent(TaskImpl task, List<String> diag, TaskStateInternal taskState, TaskAttemptId taId) {
private static TaskFailedEvent createTaskFailedEvent(TaskImpl task,
List<String> diag, TaskStateInternal taskState, TaskAttemptId taId) {
StringBuilder errorSb = new StringBuilder();
if (diag != null) {
for (String d : diag) {
@ -740,7 +744,7 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
errorSb.toString(),
taskState.toString(),
taId == null ? null : TypeConverter.fromYarn(taId),
task.getCounters());
task.getCounters(), task.launchTime);
return taskFailedEvent;
}
@ -861,7 +865,8 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
TaskFailedEvent tfe = new TaskFailedEvent(taskInfo.getTaskId(),
taskInfo.getFinishTime(), taskInfo.getTaskType(),
taskInfo.getError(), taskInfo.getTaskStatus(),
taskInfo.getFailedDueToAttemptId(), taskInfo.getCounters());
taskInfo.getFailedDueToAttemptId(), taskInfo.getCounters(),
launchTime);
eventHandler.handle(new JobHistoryEvent(taskId.getJobId(), tfe));
eventHandler.handle(
new JobTaskEvent(taskId, getExternalState(taskState)));

View File

@ -848,7 +848,8 @@ public class RMContainerAllocator extends RMContainerRequestor
updateAMRMToken(response.getAMRMToken());
}
List<ContainerStatus> finishedContainers = response.getCompletedContainersStatuses();
List<ContainerStatus> finishedContainers =
response.getCompletedContainersStatuses();
// propagate preemption requests
final PreemptionMessage preemptReq = response.getPreemptionMessage();
@ -877,16 +878,13 @@ public class RMContainerAllocator extends RMContainerRequestor
handleUpdatedNodes(response);
handleJobPriorityChange(response);
// handle receiving the timeline collector address for this app
String collectorAddr = response.getCollectorAddr();
// Handle receiving the timeline collector address and token for this app.
MRAppMaster.RunningAppContext appContext =
(MRAppMaster.RunningAppContext)this.getContext();
if (collectorAddr != null && !collectorAddr.isEmpty()
&& appContext.getTimelineV2Client() != null) {
appContext.getTimelineV2Client().setTimelineServiceAddress(
response.getCollectorAddr());
if (appContext.getTimelineV2Client() != null) {
appContext.getTimelineV2Client().
setTimelineCollectorInfo(response.getCollectorInfo());
}
for (ContainerStatus cont : finishedContainers) {
processFinishedContainer(cont);
}

View File

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

View File

@ -148,7 +148,7 @@ public class TestJobHistoryEventHandler {
// First completion event, but min-queue-size for batching flushes is 10
handleEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent(
t.taskID, t.taskAttemptID, 0, TaskType.MAP, "", null)));
t.taskID, t.taskAttemptID, 0, TaskType.MAP, "", null, 0)));
verify(mockWriter).flush();
} finally {
@ -184,7 +184,7 @@ public class TestJobHistoryEventHandler {
for (int i = 0 ; i < 100 ; i++) {
queueEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent(
t.taskID, t.taskAttemptID, 0, TaskType.MAP, "", null)));
t.taskID, t.taskAttemptID, 0, TaskType.MAP, "", null, 0)));
}
handleNextNEvents(jheh, 9);
@ -229,7 +229,7 @@ public class TestJobHistoryEventHandler {
for (int i = 0 ; i < 100 ; i++) {
queueEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent(
t.taskID, t.taskAttemptID, 0, TaskType.MAP, "", null)));
t.taskID, t.taskAttemptID, 0, TaskType.MAP, "", null, 0)));
}
handleNextNEvents(jheh, 9);
@ -272,7 +272,7 @@ public class TestJobHistoryEventHandler {
for (int i = 0 ; i < 100 ; i++) {
queueEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent(
t.taskID, t.taskAttemptID, 0, TaskType.MAP, "", null)));
t.taskID, t.taskAttemptID, 0, TaskType.MAP, "", null, 0)));
}
queueEvent(jheh, new JobHistoryEvent(t.jobId, new JobFinishedEvent(
TypeConverter.fromYarn(t.jobId), 0, 10, 10, 0, 0, null, null, new Counters())));

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.mapreduce.v2.app.rm;
import static org.junit.Assert.assertEquals;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyFloat;
import static org.mockito.Matchers.anyInt;
@ -27,6 +28,7 @@ import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.inOrder;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
@ -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.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.CollectorInfo;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
import org.apache.hadoop.yarn.api.records.ContainerId;
@ -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.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.client.api.TimelineV2Client;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.event.DrainDispatcher;
@ -121,6 +125,7 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
import org.apache.hadoop.yarn.server.api.records.NodeAction;
import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
@ -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.util.Clock;
import org.apache.hadoop.yarn.util.ControlledClock;
import org.apache.hadoop.yarn.util.Records;
import org.apache.hadoop.yarn.util.SystemClock;
import org.junit.After;
import org.junit.Assert;
@ -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
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 {
TestRMContainerAllocator t = new TestRMContainerAllocator();
t.testSimple();

View File

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

View File

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

View File

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

View File

@ -33,6 +33,7 @@ import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
import org.apache.hadoop.yarn.util.SystemClock;
/**
* Event to record unsuccessful (Killed/Failed) completion of task attempts
@ -58,10 +59,11 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
int[] cpuUsages;
int[] vMemKbytes;
int[] physMemKbytes;
private long startTime;
private static final Counters EMPTY_COUNTERS = new Counters();
/**
* Create an event to record the unsuccessful completion of attempts
* Create an event to record the unsuccessful completion of attempts.
* @param id Attempt ID
* @param taskType Type of the task
* @param status Status of the attempt
@ -75,12 +77,13 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
* measurable worker node state variables against progress.
* Currently there are four; wallclock time, CPU time,
* virtual memory and physical memory.
* @param startTs Task start time to be used for writing entity to ATSv2.
*/
public TaskAttemptUnsuccessfulCompletionEvent
(TaskAttemptID id, TaskType taskType,
String status, long finishTime,
String hostname, int port, String rackName,
String error, Counters counters, int[][] allSplits) {
String error, Counters counters, int[][] allSplits, long startTs) {
this.attemptId = id;
this.taskType = taskType;
this.status = status;
@ -99,6 +102,15 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
ProgressSplitsBlock.arrayGetVMemKbytes(allSplits);
this.physMemKbytes =
ProgressSplitsBlock.arrayGetPhysMemKbytes(allSplits);
this.startTime = startTs;
}
public TaskAttemptUnsuccessfulCompletionEvent(TaskAttemptID id,
TaskType taskType, String status, long finishTime, String hostname,
int port, String rackName, String error, Counters counters,
int[][] allSplits) {
this(id, taskType, status, finishTime, hostname, port, rackName, error,
counters, allSplits, SystemClock.getInstance().getTime());
}
/**
@ -190,39 +202,49 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
AvroArrayUtils.fromAvro(datum.getPhysMemKbytes());
}
/** Get the task id */
/** Gets the task id. */
public TaskID getTaskId() {
return attemptId.getTaskID();
}
/** Get the task type */
/** Gets the task type. */
public TaskType getTaskType() {
return TaskType.valueOf(taskType.toString());
}
/** Get the attempt id */
/** Gets the attempt id. */
public TaskAttemptID getTaskAttemptId() {
return attemptId;
}
/** Get the finish time */
/** Gets the finish time. */
public long getFinishTime() { return finishTime; }
/** Get the name of the host where the attempt executed */
/**
* Gets the task attempt start time to be used while publishing to ATSv2.
* @return task attempt start time.
*/
public long getStartTime() {
return startTime;
}
/** Gets the name of the host where the attempt executed. */
public String getHostname() { return hostname; }
/** Get the rpc port for the host where the attempt executed */
/** Gets the rpc port for the host where the attempt executed. */
public int getPort() { return port; }
/** Get the rack name of the node where the attempt ran */
/** Gets the rack name of the node where the attempt ran. */
public String getRackName() {
return rackName == null ? null : rackName.toString();
}
/** Get the error string */
/** Gets the error string. */
public String getError() { return error.toString(); }
/** Get the task status */
/**
* Gets the task attempt status.
* @return task attempt status.
*/
public String getTaskStatus() {
return status.toString();
}
/** Get the counters */
/** Gets the counters. */
Counters getCounters() { return counters; }
/** Get the event type */
/** Gets the event type. */
public EventType getEventType() {
// Note that the task type can be setup/map/reduce/cleanup but the
// attempt-type can only be map/reduce.

View File

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

View File

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

View File

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

View File

@ -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.timeline.TimelineEntity;
import org.apache.hadoop.yarn.client.api.TimelineClient;
import org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl;
import org.apache.hadoop.yarn.exceptions.YarnException;
@ -54,7 +53,7 @@ class JobHistoryFileReplayMapperV1 extends
public void map(IntWritable key, IntWritable val, Context context) throws IOException {
// collect the apps it needs to process
TimelineClient tlc = new TimelineClientImpl();
TimelineClient tlc = TimelineClient.createTimelineClient();
TimelineEntityConverterV1 converter = new TimelineEntityConverterV1();
JobHistoryFileReplayHelper helper = new JobHistoryFileReplayHelper(context);
int replayMode = helper.getReplayMode();

View File

@ -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.TimelineEvent;
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
@ -46,7 +45,7 @@ class SimpleEntityWriterV1
public void map(IntWritable key, IntWritable val, Context context)
throws IOException {
TimelineClient tlc = new TimelineClientImpl();
TimelineClient tlc = TimelineClient.createTimelineClient();
Configuration conf = context.getConfiguration();
final int kbs = conf.getInt(KBS_SENT, KBS_SENT_DEFAULT);

View File

@ -55,17 +55,15 @@ documentation.
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
challenges: improving scalability and reliability of Timeline Service, and
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
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
Timeline Service v.2 until security is implemented if security is a
critical requirement.
capacity.
More details are available in the
[YARN Timeline Service v.2](./hadoop-yarn/hadoop-yarn-site/TimelineServiceV2.html)

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceStability.Evolving;
import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
import org.apache.hadoop.yarn.api.records.CollectorInfo;
import org.apache.hadoop.yarn.api.records.AMCommand;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
@ -92,21 +93,21 @@ public abstract class AllocateResponse {
.preemptionMessage(preempt).nmTokens(nmTokens).build();
}
@Public
@Private
@Unstable
public static AllocateResponse newInstance(int responseId,
List<ContainerStatus> completedContainers,
List<Container> allocatedContainers, List<NodeReport> updatedNodes,
Resource availResources, AMCommand command, int numClusterNodes,
PreemptionMessage preempt, List<NMToken> nmTokens,
List<UpdatedContainer> updatedContainers) {
CollectorInfo collectorInfo) {
return AllocateResponse.newBuilder().numClusterNodes(numClusterNodes)
.responseId(responseId)
.completedContainersStatuses(completedContainers)
.allocatedContainers(allocatedContainers).updatedNodes(updatedNodes)
.availableResources(availResources).amCommand(command)
.preemptionMessage(preempt).nmTokens(nmTokens)
.updatedContainers(updatedContainers).build();
.collectorInfo(collectorInfo).build();
}
@Private
@ -133,7 +134,7 @@ public abstract class AllocateResponse {
List<Container> allocatedContainers, List<NodeReport> updatedNodes,
Resource availResources, AMCommand command, int numClusterNodes,
PreemptionMessage preempt, List<NMToken> nmTokens, Token amRMToken,
List<UpdatedContainer> updatedContainers, String collectorAddr) {
List<UpdatedContainer> updatedContainers, CollectorInfo collectorInfo) {
return AllocateResponse.newBuilder().numClusterNodes(numClusterNodes)
.responseId(responseId)
.completedContainersStatuses(completedContainers)
@ -141,7 +142,7 @@ public abstract class AllocateResponse {
.availableResources(availResources).amCommand(command)
.preemptionMessage(preempt).nmTokens(nmTokens)
.updatedContainers(updatedContainers).amRmToken(amRMToken)
.collectorAddr(collectorAddr).build();
.collectorInfo(collectorInfo).build();
}
/**
@ -333,17 +334,18 @@ public abstract class AllocateResponse {
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
@Unstable
public abstract String getCollectorAddr();
public abstract CollectorInfo getCollectorInfo();
@Private
@Unstable
public abstract void setCollectorAddr(String collectorAddr);
public abstract void setCollectorInfo(CollectorInfo info);
/**
* 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.
* @see AllocateResponse#setCollectorAddr(String)
* @param collectorAddr <code>collectorAddr</code> of the response
* Set the <code>collectorInfo</code> of the response.
* @see AllocateResponse#setCollectorInfo(CollectorInfo)
* @param collectorInfo <code>collectorInfo</code> of the response which
* contains collector address, RM id, version and collector token.
* @return {@link AllocateResponseBuilder}
*/
@Private
@Unstable
public AllocateResponseBuilder collectorAddr(String collectorAddr) {
allocateResponse.setCollectorAddr(collectorAddr);
public AllocateResponseBuilder collectorInfo(
CollectorInfo collectorInfo) {
allocateResponse.setCollectorInfo(collectorInfo);
return this;
}

View File

@ -16,31 +16,44 @@
* 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.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.util.Records;
/**
* Collector info containing collector address and collector token passed from
* RM to AM in Allocate Response.
*/
@Private
public abstract class AppCollectorsMap {
@InterfaceStability.Unstable
public abstract class CollectorInfo {
public static AppCollectorsMap newInstance(
ApplicationId id, String collectorAddr) {
AppCollectorsMap appCollectorsMap =
Records.newRecord(AppCollectorsMap.class);
appCollectorsMap.setApplicationId(id);
appCollectorsMap.setCollectorAddr(collectorAddr);
return appCollectorsMap;
protected static final long DEFAULT_TIMESTAMP_VALUE = -1;
public static CollectorInfo newInstance(String collectorAddr) {
return newInstance(collectorAddr, null);
}
public abstract ApplicationId getApplicationId();
public abstract void setApplicationId(ApplicationId id);
public static CollectorInfo newInstance(String collectorAddr, Token token) {
CollectorInfo amCollectorInfo =
Records.newRecord(CollectorInfo.class);
amCollectorInfo.setCollectorAddr(collectorAddr);
amCollectorInfo.setCollectorToken(token);
return amCollectorInfo;
}
public abstract String getCollectorAddr();
public abstract void setCollectorAddr(String addr);
/**
* Get delegation token for app collector which AM will use to publish
* entities.
* @return the delegation token for app collector.
*/
public abstract Token getCollectorToken();
public abstract void setCollectorToken(Token token);
}

View File

@ -49,4 +49,32 @@ public class ApplicationEntity extends HierarchicalTimelineEntity {
public void setQueue(String queue) {
addInfo(QUEUE_INFO_KEY, queue);
}
/**
* Checks if the input TimelineEntity object is an ApplicationEntity.
*
* @param te TimelineEntity object.
* @return true if input is an ApplicationEntity, false otherwise
*/
public static boolean isApplicationEntity(TimelineEntity te) {
return (te == null ? false
: te.getType().equals(TimelineEntityType.YARN_APPLICATION.toString()));
}
/**
* @param te TimelineEntity object.
* @param eventId event with this id needs to be fetched
* @return TimelineEvent if TimelineEntity contains the desired event.
*/
public static TimelineEvent getApplicationEvent(TimelineEntity te,
String eventId) {
if (isApplicationEntity(te)) {
for (TimelineEvent event : te.getEvents()) {
if (event.getId().equals(eventId)) {
return event;
}
}
}
return null;
}
}

View File

@ -55,6 +55,7 @@ import com.fasterxml.jackson.annotation.JsonSetter;
@InterfaceStability.Unstable
public class TimelineEntity implements Comparable<TimelineEntity> {
protected final static String SYSTEM_INFO_KEY_PREFIX = "SYSTEM_INFO_";
public final static long DEFAULT_ENTITY_PREFIX = 0L;
/**
* Identifier of timeline entity(entity id + entity type).
@ -146,6 +147,7 @@ public class TimelineEntity implements Comparable<TimelineEntity> {
private HashMap<String, Set<String>> isRelatedToEntities = new HashMap<>();
private HashMap<String, Set<String>> relatesToEntities = new HashMap<>();
private Long createdTime;
private long idPrefix;
public TimelineEntity() {
identifier = new Identifier();
@ -548,20 +550,10 @@ public class TimelineEntity implements Comparable<TimelineEntity> {
public int compareTo(TimelineEntity other) {
int comparison = getType().compareTo(other.getType());
if (comparison == 0) {
if (getCreatedTime() == null) {
if (other.getCreatedTime() == null) {
return getId().compareTo(other.getId());
} else {
return 1;
}
}
if (other.getCreatedTime() == null) {
if (getIdPrefix() > other.getIdPrefix()) {
// Descending order by entity id prefix
return -1;
}
if (getCreatedTime() > other.getCreatedTime()) {
// Order by created time desc
return -1;
} else if (getCreatedTime() < other.getCreatedTime()) {
} else if (getIdPrefix() < other.getIdPrefix()) {
return 1;
} else {
return getId().compareTo(other.getId());
@ -582,4 +574,38 @@ public class TimelineEntity implements Comparable<TimelineEntity> {
return real.toString();
}
}
@XmlElement(name = "idprefix")
public long getIdPrefix() {
if (real == null) {
return idPrefix;
} else {
return real.getIdPrefix();
}
}
/**
* Sets idPrefix for an entity.
* <p>
* <b>Note</b>: Entities will be stored in the order of idPrefix specified.
* If users decide to set idPrefix for an entity, they <b>MUST</b> provide
* the same prefix for every update of this entity.
* </p>
* Example: <blockquote><pre>
* TimelineEntity entity = new TimelineEntity();
* entity.setIdPrefix(value);
* </pre></blockquote>
* Users can use {@link TimelineServiceHelper#invertLong(long)} to invert
* the prefix if necessary.
*
* @param entityIdPrefix prefix for an entity.
*/
@JsonSetter("idprefix")
public void setIdPrefix(long entityIdPrefix) {
if (real == null) {
this.idPrefix = entityIdPrefix;
} else {
real.setIdPrefix(entityIdPrefix);
}
}
}

View File

@ -1782,6 +1782,10 @@ public class YarnConfiguration extends Configuration {
YARN_SECURITY_SERVICE_AUTHORIZATION_APPLICATIONHISTORY_PROTOCOL =
"security.applicationhistory.protocol.acl";
public static final String
YARN_SECURITY_SERVICE_AUTHORIZATION_COLLECTOR_NODEMANAGER_PROTOCOL =
"security.collector-nodemanager.protocol.acl";
/** No. of milliseconds to wait between sending a SIGTERM and SIGKILL
* to a running container */
public static final String NM_SLEEP_DELAY_BEFORE_SIGKILL_MS =
@ -2099,7 +2103,7 @@ public class YarnConfiguration extends Configuration {
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 =
TIMELINE_SERVICE_PREFIX + "writer.class";
@ -2112,9 +2116,20 @@ public class YarnConfiguration extends Configuration {
TIMELINE_SERVICE_PREFIX + "reader.class";
public static final String DEFAULT_TIMELINE_SERVICE_READER_CLASS =
"org.apache.hadoop.yarn.server.timelineservice" +
".storage.HBaseTimelineReaderImpl";
"org.apache.hadoop.yarn.server.timelineservice.storage" +
".HBaseTimelineReaderImpl";
/**
* default schema prefix for hbase tables.
*/
public static final String DEFAULT_TIMELINE_SERVICE_HBASE_SCHEMA_PREFIX =
"prod.";
/**
* config param name to override schema prefix.
*/
public static final String TIMELINE_SERVICE_HBASE_SCHEMA_PREFIX_NAME =
TIMELINE_SERVICE_PREFIX + "hbase-schema.prefix";
/** The setting that controls how often the timeline collector flushes the
* timeline writer.
@ -2134,6 +2149,58 @@ public class YarnConfiguration extends Configuration {
TIMELINE_SERVICE_PREFIX
+ "hbase.coprocessor.app-final-value-retention-milliseconds";
/**
* The name of the setting for the location of the coprocessor
* jar on hdfs.
*/
public static final String FLOW_RUN_COPROCESSOR_JAR_HDFS_LOCATION =
TIMELINE_SERVICE_PREFIX
+ "hbase.coprocessor.jar.hdfs.location";
/** default hdfs location for flowrun coprocessor jar. */
public static final String DEFAULT_HDFS_LOCATION_FLOW_RUN_COPROCESSOR_JAR =
"/hbase/coprocessor/hadoop-yarn-server-timelineservice.jar";
/**
* The name for setting that points to an optional HBase configuration
* (hbase-site.xml file) with settings that will override the ones found on
* the classpath.
*/
public static final String TIMELINE_SERVICE_HBASE_CONFIGURATION_FILE =
TIMELINE_SERVICE_PREFIX
+ "hbase.configuration.file";
/**
* The name for setting that enables or disables authentication checks
* for reading timeline service v2 data.
*/
public static final String TIMELINE_SERVICE_READ_AUTH_ENABLED =
TIMELINE_SERVICE_PREFIX + "read.authentication.enabled";
/**
* The default setting for authentication checks for reading timeline
* service v2 data.
*/
public static final Boolean DEFAULT_TIMELINE_SERVICE_READ_AUTH_ENABLED =
false;
/**
* The name for setting that lists the users and groups who are allowed
* to read timeline service v2 data. It is a comma separated list of
* user, followed by space, then comma separated list of groups.
* It will allow this list of users and groups to read the data
* and reject everyone else.
*/
public static final String TIMELINE_SERVICE_READ_ALLOWED_USERS =
TIMELINE_SERVICE_PREFIX + "read.allowed.users";
/**
* The default value for list of the users who are allowed to read
* timeline service v2 data.
*/
public static final String DEFAULT_TIMELINE_SERVICE_READ_ALLOWED_USERS =
"";
/**
* The setting that controls how long the final value of a metric of a
* completed app is retained before merging into the flow sum. Up to this time
@ -2154,6 +2221,8 @@ public class YarnConfiguration extends Configuration {
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

View File

@ -46,4 +46,12 @@ public final class TimelineServiceHelper {
(HashMap<E, V>) originalMap : new HashMap<E, V>(originalMap);
}
/**
* Inverts the given key.
* @param key value to be inverted .
* @return inverted long
*/
public static long invertLong(long key) {
return Long.MAX_VALUE - key;
}
}

View File

@ -613,3 +613,8 @@ message StringBytesMapProto {
optional string key = 1;
optional bytes value = 2;
}
message CollectorInfoProto {
optional string collector_addr = 1;
optional hadoop.common.TokenProto collector_token = 2;
}

View File

@ -112,7 +112,7 @@ message AllocateResponseProto {
repeated NMTokenProto nm_tokens = 9;
optional hadoop.common.TokenProto am_rm_token = 12;
optional PriorityProto application_priority = 13;
optional string collector_addr = 14;
optional CollectorInfoProto collector_info = 14;
repeated UpdateContainerErrorProto update_errors = 15;
repeated UpdatedContainerProto updated_containers = 16;
}

View File

@ -0,0 +1,71 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.api.records.timelineservice;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertEquals;
import org.junit.Test;
/**
* Various tests for the ApplicationEntity class.
*
*/
public class TestApplicationEntity {
@Test
public void testIsApplicationEntity() {
TimelineEntity te = new TimelineEntity();
te.setType(TimelineEntityType.YARN_APPLICATION.toString());
assertTrue(ApplicationEntity.isApplicationEntity(te));
te = null;
assertEquals(false, ApplicationEntity.isApplicationEntity(te));
te = new TimelineEntity();
te.setType(TimelineEntityType.YARN_CLUSTER.toString());
assertEquals(false, ApplicationEntity.isApplicationEntity(te));
}
@Test
public void testGetApplicationEvent() {
TimelineEntity te = null;
TimelineEvent tEvent = ApplicationEntity.getApplicationEvent(te,
"no event");
assertEquals(null, tEvent);
te = new TimelineEntity();
te.setType(TimelineEntityType.YARN_APPLICATION.toString());
TimelineEvent event = new TimelineEvent();
event.setId("start_event");
event.setTimestamp(System.currentTimeMillis());
te.addEvent(event);
tEvent = ApplicationEntity.getApplicationEvent(te, "start_event");
assertEquals(event, tEvent);
te = new TimelineEntity();
te.setType(TimelineEntityType.YARN_CLUSTER.toString());
event = new TimelineEvent();
event.setId("start_event_cluster");
event.setTimestamp(System.currentTimeMillis());
te.addEvent(event);
tEvent = ApplicationEntity.getApplicationEvent(te, "start_event_cluster");
assertEquals(null, tEvent);
}
}

View File

@ -66,6 +66,8 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
configurationPropsToSkipCompare
.add(YarnConfiguration
.YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCETRACKER_PROTOCOL);
configurationPropsToSkipCompare.add(YarnConfiguration
.YARN_SECURITY_SERVICE_AUTHORIZATION_COLLECTOR_NODEMANAGER_PROTOCOL);
configurationPropsToSkipCompare.add(YarnConfiguration.CURATOR_LEADER_ELECTOR);
// Federation default configs to be ignored

View File

@ -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.exceptions.YarnException;
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.log4j.LogManager;
@ -313,6 +315,17 @@ public class ApplicationMaster {
protected final Set<ContainerId> launchedContainers =
Collections.newSetFromMap(new ConcurrentHashMap<ContainerId, Boolean>());
/**
* Container start times used to set id prefix while publishing entity
* to ATSv2.
*/
private final ConcurrentMap<ContainerId, Long> containerStartTimes =
new ConcurrentHashMap<ContainerId, Long>();
private ConcurrentMap<ContainerId, Long> getContainerStartTimes() {
return containerStartTimes;
}
/**
* @param args Command line args
*/
@ -866,7 +879,15 @@ public class ApplicationMaster {
+ containerStatus.getContainerId());
}
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) {
publishContainerEndEvent(timelineClient, containerStatus, domainId,
appSubmitterUgi);
@ -994,8 +1015,10 @@ public class ApplicationMaster {
containerId, container.getNodeId());
}
if (applicationMaster.timelineServiceV2Enabled) {
applicationMaster
.publishContainerStartEventOnTimelineServiceV2(container);
long startTime = SystemClock.getInstance().getTime();
applicationMaster.getContainerStartTimes().put(containerId, startTime);
applicationMaster.publishContainerStartEventOnTimelineServiceV2(
container, startTime);
} else if (applicationMaster.timelineServiceV1Enabled) {
applicationMaster.publishContainerStartEvent(
applicationMaster.timelineClient, container,
@ -1356,24 +1379,24 @@ public class ApplicationMaster {
}
private void publishContainerStartEventOnTimelineServiceV2(
Container container) {
Container container, long startTime) {
final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
entity =
new org.apache.hadoop.yarn.api.records.timelineservice.
TimelineEntity();
entity.setId(container.getId().toString());
entity.setType(DSEntity.DS_CONTAINER.toString());
long ts = System.currentTimeMillis();
entity.setCreatedTime(ts);
entity.setCreatedTime(startTime);
entity.addInfo("user", appSubmitterUgi.getShortUserName());
org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent event =
new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent();
event.setTimestamp(ts);
event.setTimestamp(startTime);
event.setId(DSEvent.DS_CONTAINER_START.toString());
event.addInfo("Node", container.getNodeId().toString());
event.addInfo("Resources", container.getResource().toString());
entity.addEvent(event);
entity.setIdPrefix(TimelineServiceHelper.invertLong(startTime));
try {
appSubmitterUgi.doAs(new PrivilegedExceptionAction<Object>() {
@ -1391,7 +1414,7 @@ public class ApplicationMaster {
}
private void publishContainerEndEventOnTimelineServiceV2(
final ContainerStatus container) {
final ContainerStatus container, long containerStartTime) {
final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
entity =
new org.apache.hadoop.yarn.api.records.timelineservice.
@ -1407,6 +1430,7 @@ public class ApplicationMaster {
event.addInfo("State", container.getState().name());
event.addInfo("Exit Status", container.getExitStatus());
entity.addEvent(event);
entity.setIdPrefix(TimelineServiceHelper.invertLong(containerStartTime));
try {
appSubmitterUgi.doAs(new PrivilegedExceptionAction<Object>() {
@ -1441,6 +1465,8 @@ public class ApplicationMaster {
event.setId(appEvent.toString());
event.setTimestamp(ts);
entity.addEvent(event);
entity.setIdPrefix(
TimelineServiceHelper.invertLong(appAttemptID.getAttemptId()));
try {
appSubmitterUgi.doAs(new PrivilegedExceptionAction<Object>() {

View File

@ -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.timeline.TimelineDomain;
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.applications.distributedshell.ApplicationMaster.DSEvent;
import org.apache.hadoop.yarn.client.api.YarnClient;
import org.apache.hadoop.yarn.client.api.impl.DirectTimelineWriter;
import org.apache.hadoop.yarn.client.api.impl.TestTimelineClient;
@ -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.TimelineVersionWatcher;
import org.apache.hadoop.yarn.server.timelineservice.collector.PerNodeTimelineCollectorsAuxService;
import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineReaderImpl;
import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl;
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.util.LinuxResourceCalculatorPlugin;
@ -523,15 +526,31 @@ public class TestDistributedShell {
"appattempt_" + appId.getClusterTimestamp() + "_000" + appId.getId()
+ "_000001"
+ FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION;
verifyEntityTypeFileExists(basePath, "DS_APP_ATTEMPT",
appTimestampFileName);
File dsAppAttemptEntityFile = verifyEntityTypeFileExists(basePath,
"DS_APP_ATTEMPT", appTimestampFileName);
// Check if required events are published and same idprefix is sent for
// on each publish.
verifyEntityForTimelineV2(dsAppAttemptEntityFile,
DSEvent.DS_APP_ATTEMPT_START.toString(), 1, 1, 0, true);
// to avoid race condition of testcase, atleast check 40 times with sleep
// of 50ms
verifyEntityForTimelineV2(dsAppAttemptEntityFile,
DSEvent.DS_APP_ATTEMPT_END.toString(), 1, 40, 50, true);
// Verify DS_CONTAINER entities posted by the client
// Verify DS_CONTAINER entities posted by the client.
String containerTimestampFileName =
"container_" + appId.getClusterTimestamp() + "_000" + appId.getId()
+ "_01_000002.thist";
verifyEntityTypeFileExists(basePath, "DS_CONTAINER",
containerTimestampFileName);
File dsContainerEntityFile = verifyEntityTypeFileExists(basePath,
"DS_CONTAINER", containerTimestampFileName);
// Check if required events are published and same idprefix is sent for
// on each publish.
verifyEntityForTimelineV2(dsContainerEntityFile,
DSEvent.DS_CONTAINER_START.toString(), 1, 1, 0, true);
// to avoid race condition of testcase, atleast check 40 times with sleep
// of 50ms
verifyEntityForTimelineV2(dsContainerEntityFile,
DSEvent.DS_CONTAINER_END.toString(), 1, 40, 50, true);
// Verify NM posting container metrics info.
String containerMetricsTimestampFileName =
@ -541,29 +560,13 @@ public class TestDistributedShell {
File containerEntityFile = verifyEntityTypeFileExists(basePath,
TimelineEntityType.YARN_CONTAINER.toString(),
containerMetricsTimestampFileName);
Assert.assertEquals(
"Container created event needs to be published atleast once",
1,
getNumOfStringOccurrences(containerEntityFile,
ContainerMetricsConstants.CREATED_EVENT_TYPE));
verifyEntityForTimelineV2(containerEntityFile,
ContainerMetricsConstants.CREATED_EVENT_TYPE, 1, 1, 0, true);
// to avoid race condition of testcase, atleast check 4 times with sleep
// of 500ms
long numOfContainerFinishedOccurrences = 0;
for (int i = 0; i < 4; i++) {
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);
// to avoid race condition of testcase, atleast check 40 times with sleep
// of 50ms
verifyEntityForTimelineV2(containerEntityFile,
ContainerMetricsConstants.FINISHED_EVENT_TYPE, 1, 40, 50, true);
// Verify RM posting Application life cycle Events are getting published
String appMetricsTimestampFileName =
@ -573,29 +576,14 @@ public class TestDistributedShell {
verifyEntityTypeFileExists(basePath,
TimelineEntityType.YARN_APPLICATION.toString(),
appMetricsTimestampFileName);
Assert.assertEquals(
"Application created event should be published atleast once",
1,
getNumOfStringOccurrences(appEntityFile,
ApplicationMetricsConstants.CREATED_EVENT_TYPE));
// No need to check idprefix for app.
verifyEntityForTimelineV2(appEntityFile,
ApplicationMetricsConstants.CREATED_EVENT_TYPE, 1, 1, 0, false);
// to avoid race condition of testcase, atleast check 4 times with sleep
// of 500ms
long numOfStringOccurrences = 0;
for (int i = 0; i < 4; i++) {
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);
// to avoid race condition of testcase, atleast check 40 times with sleep
// of 50ms
verifyEntityForTimelineV2(appEntityFile,
ApplicationMetricsConstants.FINISHED_EVENT_TYPE, 1, 40, 50, false);
// Verify RM posting AppAttempt life cycle Events are getting published
String appAttemptMetricsTimestampFileName =
@ -606,17 +594,10 @@ public class TestDistributedShell {
verifyEntityTypeFileExists(basePath,
TimelineEntityType.YARN_APPLICATION_ATTEMPT.toString(),
appAttemptMetricsTimestampFileName);
Assert.assertEquals(
"AppAttempt register event should be published atleast once",
1,
getNumOfStringOccurrences(appAttemptEntityFile,
AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE));
Assert.assertEquals(
"AppAttempt finished event should be published atleast once",
1,
getNumOfStringOccurrences(appAttemptEntityFile,
AppAttemptMetricsConstants.FINISHED_EVENT_TYPE));
verifyEntityForTimelineV2(appAttemptEntityFile,
AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE, 1, 1, 0, true);
verifyEntityForTimelineV2(appAttemptEntityFile,
AppAttemptMetricsConstants.FINISHED_EVENT_TYPE, 1, 1, 0, true);
} finally {
FileUtils.deleteDirectory(tmpRootFolder.getParentFile());
}
@ -636,22 +617,64 @@ public class TestDistributedShell {
return entityFile;
}
private long getNumOfStringOccurrences(File entityFile, String searchString)
throws IOException {
BufferedReader reader = null;
String strLine;
/**
* Checks the events and idprefix published for an entity.
*
* @param entityFile Entity file.
* @param expectedEvent Expected event Id.
* @param numOfExpectedEvent Number of expected occurences of expected event
* id.
* @param checkTimes Number of times to check.
* @param sleepTime Sleep time for each iteration.
* @param checkIdPrefix Whether to check idprefix.
* @throws IOException if entity file reading fails.
* @throws InterruptedException if sleep is interrupted.
*/
private void verifyEntityForTimelineV2(File entityFile, String expectedEvent,
long numOfExpectedEvent, int checkTimes, long sleepTime,
boolean checkIdPrefix) throws IOException, InterruptedException {
long actualCount = 0;
try {
reader = new BufferedReader(new FileReader(entityFile));
while ((strLine = reader.readLine()) != null) {
if (strLine.trim().contains(searchString)) {
actualCount++;
for (int i = 0; i < checkTimes; i++) {
BufferedReader reader = null;
String strLine = null;
actualCount = 0;
try {
reader = new BufferedReader(new FileReader(entityFile));
long idPrefix = -1;
while ((strLine = reader.readLine()) != null) {
String entityLine = strLine.trim();
if (entityLine.isEmpty()) {
continue;
}
if (entityLine.contains(expectedEvent)) {
actualCount++;
}
if (checkIdPrefix) {
TimelineEntity entity = FileSystemTimelineReaderImpl.
getTimelineRecordFromJSON(entityLine, TimelineEntity.class);
Assert.assertTrue("Entity ID prefix expected to be > 0",
entity.getIdPrefix() > 0);
if (idPrefix == -1) {
idPrefix = entity.getIdPrefix();
} else {
Assert.assertEquals("Entity ID prefix should be same across " +
"each publish of same entity",
idPrefix, entity.getIdPrefix());
}
}
}
} finally {
reader.close();
}
if (numOfExpectedEvent == actualCount) {
break;
}
if (sleepTime > 0 && i < checkTimes - 1) {
Thread.sleep(sleepTime);
}
} finally {
reader.close();
}
return actualCount;
Assert.assertEquals("Unexpected number of " + expectedEvent +
" event published.", numOfExpectedEvent, actualCount);
}
/**

View File

@ -68,8 +68,6 @@ extends AMRMClientAsync<T> {
private volatile boolean keepRunning;
private volatile float progress;
private volatile String collectorAddr;
/**
*
@ -325,17 +323,16 @@ extends AMRMClientAsync<T> {
}
AllocateResponse response = (AllocateResponse) object;
String collectorAddress = response.getCollectorAddr();
String collectorAddress = null;
if (response.getCollectorInfo() != null) {
collectorAddress = response.getCollectorInfo().getCollectorAddr();
}
TimelineV2Client timelineClient =
client.getRegisteredTimelineV2Client();
if (timelineClient != null && collectorAddress != null
&& !collectorAddress.isEmpty()) {
if (collectorAddr == null
|| !collectorAddr.equals(collectorAddress)) {
collectorAddr = collectorAddress;
timelineClient.setTimelineServiceAddress(collectorAddress);
LOG.info("collectorAddress " + collectorAddress);
}
if (timelineClient != null && response.getCollectorInfo() != null) {
timelineClient.
setTimelineCollectorInfo(response.getCollectorInfo());
}
List<NodeReport> updatedNodes = response.getUpdatedNodes();

View File

@ -0,0 +1,72 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.client;
import java.io.IOException;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
import org.junit.After;
/**
* Test Base for Application Master Service Protocol.
*/
public abstract class ApplicationMasterServiceProtoTestBase
extends ProtocolHATestBase {
private ApplicationMasterProtocol amClient;
private ApplicationAttemptId attemptId;
protected void startupHAAndSetupClient() throws Exception {
attemptId = this.cluster.createFakeApplicationAttemptId();
Token<AMRMTokenIdentifier> appToken =
this.cluster.getResourceManager().getRMContext()
.getAMRMTokenSecretManager().createAndGetAMRMToken(attemptId);
appToken.setService(ClientRMProxy.getAMRMTokenService(this.conf));
UserGroupInformation.setLoginUser(UserGroupInformation
.createRemoteUser(UserGroupInformation.getCurrentUser().getUserName()));
UserGroupInformation.getCurrentUser().addToken(appToken);
syncToken(appToken);
amClient = ClientRMProxy
.createRMProxy(this.conf, ApplicationMasterProtocol.class);
}
@After
public void shutDown() {
if(this.amClient != null) {
RPC.stopProxy(this.amClient);
}
}
protected ApplicationMasterProtocol getAMClient() {
return amClient;
}
private void syncToken(Token<AMRMTokenIdentifier> token) throws IOException {
for (int i = 0; i < this.cluster.getNumOfResourceManager(); i++) {
this.cluster.getResourceManager(i).getRMContext()
.getAMRMTokenSecretManager().addPersistedPassword(token);
}
}
}

View File

@ -25,6 +25,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
import org.apache.hadoop.yarn.api.records.CollectorInfo;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
@ -804,11 +805,20 @@ public abstract class ProtocolHATestBase extends ClientBaseWithFixes {
}
public AllocateResponse createFakeAllocateResponse() {
return AllocateResponse.newInstance(-1,
new ArrayList<ContainerStatus>(),
new ArrayList<Container>(), new ArrayList<NodeReport>(),
Resource.newInstance(1024, 2), null, 1,
null, new ArrayList<NMToken>());
if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) {
return AllocateResponse.newInstance(-1,
new ArrayList<ContainerStatus>(), new ArrayList<Container>(),
new ArrayList<NodeReport>(), Resource.newInstance(1024, 2), null, 1,
null, new ArrayList<NMToken>(), CollectorInfo.newInstance(
"host:port", Token.newInstance(new byte[] {0}, "TIMELINE",
new byte[] {0}, "rm")));
} else {
return AllocateResponse.newInstance(-1,
new ArrayList<ContainerStatus>(),
new ArrayList<Container>(), new ArrayList<NodeReport>(),
Resource.newInstance(1024, 2), null, 1,
null, new ArrayList<NMToken>());
}
}
}

View File

@ -0,0 +1,71 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.client;
import java.io.IOException;
import java.util.ArrayList;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.server.resourcemanager.HATestUtil;
import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
/**
* Tests Application Master Protocol with timeline service v2 enabled.
*/
public class TestApplicationMasterServiceProtocolForTimelineV2
extends ApplicationMasterServiceProtoTestBase {
@Before
public void initialize() throws Exception {
HATestUtil.setRpcAddressForRM(RM1_NODE_ID, RM1_PORT_BASE + 200, conf);
HATestUtil.setRpcAddressForRM(RM2_NODE_ID, RM2_PORT_BASE + 200, conf);
conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
conf.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS,
FileSystemTimelineWriterImpl.class, TimelineWriter.class);
startHACluster(0, false, false, true);
super.startupHAAndSetupClient();
}
@Test(timeout = 15000)
public void testAllocateForTimelineV2OnHA()
throws YarnException, IOException {
AllocateRequest request = AllocateRequest.newInstance(0, 50f,
new ArrayList<ResourceRequest>(),
new ArrayList<ContainerId>(),
ResourceBlacklistRequest.newInstance(new ArrayList<String>(),
new ArrayList<String>()));
AllocateResponse response = getAMClient().allocate(request);
Assert.assertEquals(response, this.cluster.createFakeAllocateResponse());
Assert.assertNotNull(response.getCollectorInfo());
Assert.assertEquals("host:port",
response.getCollectorInfo().getCollectorAddr());
Assert.assertNotNull(response.getCollectorInfo().getCollectorToken());
}
}

View File

@ -23,10 +23,6 @@ import java.util.ArrayList;
import org.junit.Assert;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
@ -34,45 +30,20 @@ import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRespons
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
public class TestApplicationMasterServiceProtocolOnHA
extends ProtocolHATestBase {
private ApplicationMasterProtocol amClient;
private ApplicationAttemptId attemptId ;
extends ApplicationMasterServiceProtoTestBase {
@Before
public void initialize() throws Exception {
startHACluster(0, false, false, true);
attemptId = this.cluster.createFakeApplicationAttemptId();
Token<AMRMTokenIdentifier> appToken =
this.cluster.getResourceManager().getRMContext()
.getAMRMTokenSecretManager().createAndGetAMRMToken(attemptId);
appToken.setService(ClientRMProxy.getAMRMTokenService(this.conf));
UserGroupInformation.setLoginUser(UserGroupInformation
.createRemoteUser(UserGroupInformation.getCurrentUser().getUserName()));
UserGroupInformation.getCurrentUser().addToken(appToken);
syncToken(appToken);
amClient = ClientRMProxy
.createRMProxy(this.conf, ApplicationMasterProtocol.class);
}
@After
public void shutDown() {
if(this.amClient != null) {
RPC.stopProxy(this.amClient);
}
super.startupHAAndSetupClient();
}
@Test(timeout = 15000)
@ -81,7 +52,7 @@ public class TestApplicationMasterServiceProtocolOnHA
RegisterApplicationMasterRequest request =
RegisterApplicationMasterRequest.newInstance("localhost", 0, "");
RegisterApplicationMasterResponse response =
amClient.registerApplicationMaster(request);
getAMClient().registerApplicationMaster(request);
Assert.assertEquals(response,
this.cluster.createFakeRegisterApplicationMasterResponse());
}
@ -93,7 +64,7 @@ public class TestApplicationMasterServiceProtocolOnHA
FinishApplicationMasterRequest.newInstance(
FinalApplicationStatus.SUCCEEDED, "", "");
FinishApplicationMasterResponse response =
amClient.finishApplicationMaster(request);
getAMClient().finishApplicationMaster(request);
Assert.assertEquals(response,
this.cluster.createFakeFinishApplicationMasterResponse());
}
@ -105,14 +76,7 @@ public class TestApplicationMasterServiceProtocolOnHA
new ArrayList<ContainerId>(),
ResourceBlacklistRequest.newInstance(new ArrayList<String>(),
new ArrayList<String>()));
AllocateResponse response = amClient.allocate(request);
AllocateResponse response = getAMClient().allocate(request);
Assert.assertEquals(response, this.cluster.createFakeAllocateResponse());
}
private void syncToken(Token<AMRMTokenIdentifier> token) throws IOException {
for (int i = 0; i < this.cluster.getNumOfResourceManager(); i++) {
this.cluster.getResourceManager(i).getRMContext()
.getAMRMTokenSecretManager().addPersistedPassword(token);
}
}
}

View File

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

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
import org.apache.hadoop.yarn.api.records.CollectorInfo;
import org.apache.hadoop.yarn.api.records.AMCommand;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.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.UpdateContainerError;
import org.apache.hadoop.yarn.api.records.UpdatedContainer;
import org.apache.hadoop.yarn.api.records.impl.pb.CollectorInfoPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.NMTokenPBImpl;
@ -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.TokenPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.UpdatedContainerPBImpl;
import org.apache.hadoop.yarn.proto.YarnProtos.CollectorInfoProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
import org.apache.hadoop.yarn.proto.YarnProtos.NodeReportProto;
@ -80,6 +83,7 @@ public class AllocateResponsePBImpl extends AllocateResponse {
private PreemptionMessage preempt;
private Token amrmToken = null;
private Priority appPriority = null;
private CollectorInfo collectorInfo = null;
public AllocateResponsePBImpl() {
builder = AllocateResponseProto.newBuilder();
@ -162,6 +166,9 @@ public class AllocateResponsePBImpl extends AllocateResponse {
if (this.amrmToken != null) {
builder.setAmRmToken(convertToProtoFormat(this.amrmToken));
}
if (this.collectorInfo != null) {
builder.setCollectorInfo(convertToProtoFormat(this.collectorInfo));
}
if (this.appPriority != null) {
builder.setApplicationPriority(convertToProtoFormat(this.appPriority));
}
@ -398,19 +405,25 @@ public class AllocateResponsePBImpl extends AllocateResponse {
@Override
public synchronized String getCollectorAddr() {
public synchronized CollectorInfo getCollectorInfo() {
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
public synchronized void setCollectorAddr(String collectorAddr) {
public synchronized void setCollectorInfo(CollectorInfo info) {
maybeInitBuilder();
if (collectorAddr == null) {
builder.clearCollectorAddr();
return;
if (info == null) {
builder.clearCollectorInfo();
}
builder.setCollectorAddr(collectorAddr);
this.collectorInfo = info;
}
@Override
@ -718,6 +731,16 @@ public class AllocateResponsePBImpl extends AllocateResponse {
return ((NodeReportPBImpl)t).getProto();
}
private synchronized CollectorInfoPBImpl convertFromProtoFormat(
CollectorInfoProto p) {
return new CollectorInfoPBImpl(p);
}
private synchronized CollectorInfoProto convertToProtoFormat(
CollectorInfo t) {
return ((CollectorInfoPBImpl)t).getProto();
}
private synchronized ContainerPBImpl convertFromProtoFormat(
ContainerProto p) {
return new ContainerPBImpl(p);

View File

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

View File

@ -23,6 +23,7 @@ import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.service.CompositeService;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.CollectorInfo;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
import org.apache.hadoop.yarn.client.api.impl.TimelineV2ClientImpl;
import org.apache.hadoop.yarn.exceptions.YarnException;
@ -83,10 +84,13 @@ public abstract class TimelineV2Client extends CompositeService {
/**
* <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>
*
* @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);
}

View File

@ -19,7 +19,11 @@
package org.apache.hadoop.yarn.client.api.impl;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.lang.reflect.UndeclaredThrowableException;
import java.net.InetSocketAddress;
import java.net.URI;
import java.security.PrivilegedExceptionAction;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
@ -36,15 +40,22 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.CollectorInfo;
import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
import org.apache.hadoop.yarn.client.api.TimelineV2Client;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
import com.google.common.annotations.VisibleForTesting;
import com.sun.jersey.api.client.ClientResponse;
import com.sun.jersey.core.util.MultivaluedMapImpl;
@ -59,6 +70,8 @@ public class TimelineV2ClientImpl extends TimelineV2Client {
private TimelineEntityDispatcher entityDispatcher;
private volatile String timelineServiceAddress;
@VisibleForTesting
volatile Token currentTimelineToken = null;
// Retry parameters for identifying new timeline service
// TODO consider to merge with connection retry
@ -69,6 +82,8 @@ public class TimelineV2ClientImpl extends TimelineV2Client {
private ApplicationId contextAppId;
private UserGroupInformation authUgi;
public TimelineV2ClientImpl(ApplicationId appId) {
super(TimelineV2ClientImpl.class.getName());
this.contextAppId = appId;
@ -88,7 +103,6 @@ public class TimelineV2ClientImpl extends TimelineV2Client {
UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
UserGroupInformation realUgi = ugi.getRealUser();
String doAsUser = null;
UserGroupInformation authUgi = null;
if (realUgi != null) {
authUgi = realUgi;
doAsUser = ugi.getShortUserName();
@ -96,7 +110,6 @@ public class TimelineV2ClientImpl extends TimelineV2Client {
authUgi = ugi;
doAsUser = null;
}
// TODO need to add/cleanup filter retry later for ATSV2. similar to V1
DelegationTokenAuthenticatedURL.Token token =
new DelegationTokenAuthenticatedURL.Token();
@ -140,8 +153,72 @@ public class TimelineV2ClientImpl extends TimelineV2Client {
}
@Override
public void setTimelineServiceAddress(String address) {
this.timelineServiceAddress = address;
public void setTimelineCollectorInfo(CollectorInfo collectorInfo) {
if (collectorInfo == null) {
LOG.warn("Not setting collector info as it is null.");
return;
}
// First update the token so that it is available when collector address is
// used.
if (collectorInfo.getCollectorToken() != null) {
// Use collector address to update token service if its not available.
setTimelineDelegationToken(
collectorInfo.getCollectorToken(), collectorInfo.getCollectorAddr());
}
// Update timeline service address.
if (collectorInfo.getCollectorAddr() != null &&
!collectorInfo.getCollectorAddr().isEmpty() &&
!collectorInfo.getCollectorAddr().equals(timelineServiceAddress)) {
this.timelineServiceAddress = collectorInfo.getCollectorAddr();
LOG.info("Updated timeline service address to " + timelineServiceAddress);
}
}
private void setTimelineDelegationToken(Token delegationToken,
String collectorAddr) {
// Checks below are to ensure that an invalid token is not updated in UGI.
// This is required because timeline token is set via a public API.
if (!delegationToken.getKind().equals(
TimelineDelegationTokenIdentifier.KIND_NAME.toString())) {
LOG.warn("Timeline token to be updated should be of kind " +
TimelineDelegationTokenIdentifier.KIND_NAME);
return;
}
if (collectorAddr == null || collectorAddr.isEmpty()) {
collectorAddr = timelineServiceAddress;
}
// Token need not be updated if 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
@ -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,
MultivaluedMap<String, String> params, Object obj)
throws IOException, YarnException {
ClientResponse resp;
ClientResponse resp = null;
try {
resp = connector.getClient().resource(base).path(path).queryParams(params)
.accept(MediaType.APPLICATION_JSON).type(MediaType.APPLICATION_JSON)
.put(ClientResponse.class, obj);
} catch (RuntimeException re) {
// 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);
throw new IOException(re);
resp = authUgi.doAs(new PrivilegedExceptionAction<ClientResponse>() {
@Override
public ClientResponse run() throws Exception {
return doPutObjects(base, path, params, obj);
}
});
} catch (UndeclaredThrowableException ue) {
Throwable cause = ue.getCause();
if (cause instanceof IOException) {
throw (IOException)cause;
} else {
throw new IOException(cause);
}
} catch (InterruptedException ie) {
throw (IOException) new InterruptedIOException().initCause(ie);
}
if (resp == null || resp.getStatusInfo()
.getStatusCode() != ClientResponse.Status.OK.getStatusCode()) {

View File

@ -2341,6 +2341,39 @@
<value>259200000</value>
</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 -->
<property>
@ -3145,6 +3178,17 @@
<value>64</value>
</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>
<description>
Flag to enable cross-origin (CORS) support for timeline service v1.x or

View File

@ -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.SubmitApplicationRequestPBImpl;
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationResponsePBImpl;
import org.apache.hadoop.yarn.api.records.CollectorInfo;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@ -406,6 +407,7 @@ public class TestPBImplRecords extends BasePBImplRecordsTest {
generateByNewInstance(CommitResponse.class);
generateByNewInstance(ApplicationTimeout.class);
generateByNewInstance(QueueConfigurations.class);
generateByNewInstance(CollectorInfo.class);
}
@Test

View File

@ -18,6 +18,11 @@
package org.apache.hadoop.yarn.client.api.impl;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import java.io.IOException;
import java.net.URI;
import java.util.ArrayList;
@ -27,11 +32,16 @@ import javax.ws.rs.core.MultivaluedMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.CollectorInfo;
import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@ -151,7 +161,7 @@ public class TestTimelineClientV2Impl {
maxRetries);
c.init(conf);
c.start();
c.setTimelineServiceAddress("localhost:12345");
c.setTimelineCollectorInfo(CollectorInfo.newInstance("localhost:12345"));
try {
c.putEntities(new TimelineEntity());
} 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
public void testAfterStop() throws Exception {
client.setSleepBeforeReturn(true);

View File

@ -20,14 +20,14 @@ package org.apache.hadoop.yarn.server.applicationhistoryservice;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.LinkedHashSet;
import java.util.Set;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.http.HttpServer2;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.metrics2.source.JvmMetrics;
import org.apache.hadoop.security.AuthenticationFilterInitializer;
import org.apache.hadoop.security.HttpCrossOriginFilterInitializer;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.service.CompositeService;
@ -47,10 +47,9 @@ import org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore;
import org.apache.hadoop.yarn.server.timeline.TimelineDataManager;
import org.apache.hadoop.yarn.server.timeline.TimelineStore;
import org.apache.hadoop.yarn.server.timeline.security.TimelineACLsManager;
import org.apache.hadoop.yarn.server.timeline.security.TimelineAuthenticationFilter;
import org.apache.hadoop.yarn.server.timeline.security.TimelineAuthenticationFilterInitializer;
import org.apache.hadoop.yarn.server.timeline.security.TimelineDelegationTokenSecretManagerService;
import org.apache.hadoop.yarn.server.timeline.security.TimelineV1DelegationTokenSecretManagerService;
import org.apache.hadoop.yarn.server.timeline.webapp.CrossOriginFilterInitializer;
import org.apache.hadoop.yarn.server.util.timeline.TimelineServerUtils;
import org.apache.hadoop.yarn.webapp.WebApp;
import org.apache.hadoop.yarn.webapp.WebApps;
import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
@ -75,7 +74,7 @@ public class ApplicationHistoryServer extends CompositeService {
private ApplicationACLsManager aclsManager;
private ApplicationHistoryManager historyManager;
private TimelineStore timelineStore;
private TimelineDelegationTokenSecretManagerService secretManagerService;
private TimelineV1DelegationTokenSecretManagerService secretManagerService;
private TimelineDataManager timelineDataManager;
private WebApp webApp;
private JvmPauseMonitor pauseMonitor;
@ -223,9 +222,9 @@ public class ApplicationHistoryServer extends CompositeService {
TimelineStore.class), conf);
}
private TimelineDelegationTokenSecretManagerService
private TimelineV1DelegationTokenSecretManagerService
createTimelineDelegationTokenSecretManagerService(Configuration conf) {
return new TimelineDelegationTokenSecretManagerService();
return new TimelineV1DelegationTokenSecretManagerService();
}
private TimelineDataManager createTimelineDataManager(Configuration conf) {
@ -237,63 +236,33 @@ public class ApplicationHistoryServer extends CompositeService {
@SuppressWarnings("unchecked")
private void startWebApp() {
Configuration conf = getConfig();
TimelineAuthenticationFilter.setTimelineDelegationTokenSecretManager(
secretManagerService.getTimelineDelegationTokenSecretManager());
// Always load pseudo authentication filter to parse "user.name" in an URL
// to identify a HTTP request's user in insecure mode.
// When Kerberos authentication type is set (i.e., secure mode is turned on),
// the customized filter will be loaded by the timeline server to do Kerberos
// + DT authentication.
String initializers = conf.get("hadoop.http.filter.initializers");
boolean modifiedInitializers = false;
initializers =
initializers == null || initializers.length() == 0 ? "" : initializers;
String initializers = conf.get("hadoop.http.filter.initializers", "");
Set<String> defaultInitializers = new LinkedHashSet<String>();
// Add CORS filter
if (!initializers.contains(CrossOriginFilterInitializer.class.getName())) {
if(conf.getBoolean(YarnConfiguration
.TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED, YarnConfiguration
.TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED_DEFAULT)) {
if (initializers.contains(HttpCrossOriginFilterInitializer.class.getName())) {
initializers =
initializers.replaceAll(HttpCrossOriginFilterInitializer.class.getName(),
if(conf.getBoolean(YarnConfiguration.
TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED,
YarnConfiguration.
TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED_DEFAULT)) {
if (initializers.contains(
HttpCrossOriginFilterInitializer.class.getName())) {
initializers = initializers.replaceAll(
HttpCrossOriginFilterInitializer.class.getName(),
CrossOriginFilterInitializer.class.getName());
} else {
defaultInitializers.add(CrossOriginFilterInitializer.class.getName());
}
else {
if (initializers.length() != 0) {
initializers += ",";
}
initializers += CrossOriginFilterInitializer.class.getName();
}
modifiedInitializers = true;
}
}
if (!initializers.contains(TimelineAuthenticationFilterInitializer.class
.getName())) {
if (initializers.length() != 0) {
initializers += ",";
}
initializers += TimelineAuthenticationFilterInitializer.class.getName();
modifiedInitializers = true;
}
String[] parts = initializers.split(",");
ArrayList<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);
}
TimelineServerUtils.addTimelineAuthFilter(
initializers, defaultInitializers, secretManagerService);
TimelineServerUtils.setTimelineFilters(
conf, initializers, defaultInitializers);
String bindAddress = WebAppUtils.getWebAppBindURL(conf,
YarnConfiguration.TIMELINE_SERVICE_BIND_HOST,
WebAppUtils.getAHSWebAppURLWithoutScheme(conf));

View File

@ -26,7 +26,6 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
import org.apache.hadoop.security.token.delegation.DelegationKey;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
@ -37,18 +36,16 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* The service wrapper of {@link TimelineDelegationTokenSecretManager}
* The service wrapper of {@link TimelineV1DelegationTokenSecretManager}.
*/
@Private
@Unstable
public class TimelineDelegationTokenSecretManagerService extends
AbstractService {
private TimelineDelegationTokenSecretManager secretManager = null;
public class TimelineV1DelegationTokenSecretManagerService extends
TimelineDelgationTokenSecretManagerService {
private TimelineStateStore stateStore = null;
public TimelineDelegationTokenSecretManagerService() {
super(TimelineDelegationTokenSecretManagerService.class.getName());
public TimelineV1DelegationTokenSecretManagerService() {
super(TimelineV1DelegationTokenSecretManagerService.class.getName());
}
@Override
@ -58,19 +55,7 @@ public class TimelineDelegationTokenSecretManagerService extends
stateStore = createStateStore(conf);
stateStore.init(conf);
}
long secretKeyInterval =
conf.getLong(YarnConfiguration.TIMELINE_DELEGATION_KEY_UPDATE_INTERVAL,
YarnConfiguration.DEFAULT_TIMELINE_DELEGATION_KEY_UPDATE_INTERVAL);
long tokenMaxLifetime =
conf.getLong(YarnConfiguration.TIMELINE_DELEGATION_TOKEN_MAX_LIFETIME,
YarnConfiguration.DEFAULT_TIMELINE_DELEGATION_TOKEN_MAX_LIFETIME);
long tokenRenewInterval =
conf.getLong(YarnConfiguration.TIMELINE_DELEGATION_TOKEN_RENEW_INTERVAL,
YarnConfiguration.DEFAULT_TIMELINE_DELEGATION_TOKEN_RENEW_INTERVAL);
secretManager = new TimelineDelegationTokenSecretManager(secretKeyInterval,
tokenMaxLifetime, tokenRenewInterval, 3600000, stateStore);
super.init(conf);
super.serviceInit(conf);
}
@Override
@ -78,10 +63,9 @@ public class TimelineDelegationTokenSecretManagerService extends
if (stateStore != null) {
stateStore.start();
TimelineServiceState state = stateStore.loadState();
secretManager.recover(state);
((TimelineV1DelegationTokenSecretManager)
getTimelineDelegationTokenSecretManager()).recover(state);
}
secretManager.startThreads();
super.serviceStart();
}
@ -90,9 +74,18 @@ public class TimelineDelegationTokenSecretManagerService extends
if (stateStore != null) {
stateStore.stop();
}
super.serviceStop();
}
secretManager.stopThreads();
super.stop();
@Override
protected AbstractDelegationTokenSecretManager
<TimelineDelegationTokenIdentifier>
createTimelineDelegationTokenSecretManager(long secretKeyInterval,
long tokenMaxLifetime, long tokenRenewInterval,
long tokenRemovalScanInterval) {
return new TimelineV1DelegationTokenSecretManager(secretKeyInterval,
tokenMaxLifetime, tokenRenewInterval, tokenRemovalScanInterval,
stateStore);
}
protected TimelineStateStore createStateStore(
@ -104,27 +97,20 @@ public class TimelineDelegationTokenSecretManagerService extends
}
/**
* Ge the instance of {link #TimelineDelegationTokenSecretManager}
*
* @return the instance of {link #TimelineDelegationTokenSecretManager}
* Delegation token secret manager for ATSv1 and ATSv1.5.
*/
public TimelineDelegationTokenSecretManager
getTimelineDelegationTokenSecretManager() {
return secretManager;
}
@Private
@Unstable
public static class TimelineDelegationTokenSecretManager extends
public static class TimelineV1DelegationTokenSecretManager extends
AbstractDelegationTokenSecretManager<TimelineDelegationTokenIdentifier> {
public static final Logger LOG =
LoggerFactory.getLogger(TimelineDelegationTokenSecretManager.class);
LoggerFactory.getLogger(TimelineV1DelegationTokenSecretManager.class);
private TimelineStateStore stateStore;
/**
* Create a timeline secret manager
* Create a timeline v1 secret manager.
* @param delegationKeyUpdateInterval the number of milliseconds for rolling
* new secret keys.
* @param delegationTokenMaxLifetime the maximum lifetime of the delegation
@ -135,7 +121,7 @@ public class TimelineDelegationTokenSecretManagerService extends
* scanned for expired tokens in milliseconds
* @param stateStore timeline service state store
*/
public TimelineDelegationTokenSecretManager(
public TimelineV1DelegationTokenSecretManager(
long delegationKeyUpdateInterval,
long delegationTokenMaxLifetime,
long delegationTokenRenewInterval,
@ -236,5 +222,4 @@ public class TimelineDelegationTokenSecretManagerService extends
}
}
}
}

View File

@ -55,27 +55,31 @@ import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
/**
* Test cases for authentication via TimelineAuthenticationFilter while
* publishing entities for ATSv1.
*/
@RunWith(Parameterized.class)
public class TestTimelineAuthenticationFilter {
public class TestTimelineAuthenticationFilterForV1 {
private static final String FOO_USER = "foo";
private static final String BAR_USER = "bar";
private static final String HTTP_USER = "HTTP";
private static final File testRootDir = new File(
private static final File TEST_ROOT_DIR = new File(
System.getProperty("test.build.dir", "target/test-dir"),
TestTimelineAuthenticationFilter.class.getName() + "-root");
TestTimelineAuthenticationFilterForV1.class.getName() + "-root");
private static File httpSpnegoKeytabFile = new File(
KerberosTestUtils.getKeytabFile());
private static String httpSpnegoPrincipal =
KerberosTestUtils.getServerPrincipal();
private static final String BASEDIR =
System.getProperty("test.build.dir", "target/test-dir") + "/"
+ TestTimelineAuthenticationFilter.class.getSimpleName();
+ TestTimelineAuthenticationFilterForV1.class.getSimpleName();
@Parameterized.Parameters
public static Collection<Object[]> withSsl() {
return Arrays.asList(new Object[][] { { false }, { true } });
return Arrays.asList(new Object[][] {{false}, {true}});
}
private static MiniKdc testMiniKDC;
@ -85,14 +89,14 @@ public class TestTimelineAuthenticationFilter {
private static Configuration conf;
private static boolean withSsl;
public TestTimelineAuthenticationFilter(boolean withSsl) {
TestTimelineAuthenticationFilter.withSsl = withSsl;
public TestTimelineAuthenticationFilterForV1(boolean withSsl) {
TestTimelineAuthenticationFilterForV1.withSsl = withSsl;
}
@BeforeClass
public static void setup() {
try {
testMiniKDC = new MiniKdc(MiniKdc.createConf(), testRootDir);
testMiniKDC = new MiniKdc(MiniKdc.createConf(), TEST_ROOT_DIR);
testMiniKDC.start();
testMiniKDC.createPrincipal(
httpSpnegoKeytabFile, HTTP_USER + "/localhost");
@ -111,11 +115,11 @@ public class TestTimelineAuthenticationFilter {
KerberosAuthenticationHandler.KEYTAB,
httpSpnegoKeytabFile.getAbsolutePath());
conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
"kerberos");
"kerberos");
conf.set(YarnConfiguration.TIMELINE_SERVICE_PRINCIPAL,
httpSpnegoPrincipal);
httpSpnegoPrincipal);
conf.set(YarnConfiguration.TIMELINE_SERVICE_KEYTAB,
httpSpnegoKeytabFile.getAbsolutePath());
httpSpnegoKeytabFile.getAbsolutePath());
conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
conf.setClass(YarnConfiguration.TIMELINE_SERVICE_STORE,
MemoryTimelineStore.class, TimelineStore.class);
@ -136,8 +140,8 @@ public class TestTimelineAuthenticationFilter {
FileUtil.fullyDelete(base);
base.mkdirs();
keystoresDir = new File(BASEDIR).getAbsolutePath();
sslConfDir =
KeyStoreTestUtil.getClasspathDir(TestTimelineAuthenticationFilter.class);
sslConfDir = KeyStoreTestUtil.getClasspathDir(
TestTimelineAuthenticationFilterForV1.class);
KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, conf, false);
}
@ -145,6 +149,7 @@ public class TestTimelineAuthenticationFilter {
testTimelineServer.init(conf);
testTimelineServer.start();
} catch (Exception e) {
e.printStackTrace();
assertTrue("Couldn't setup TimelineServer", false);
}
}
@ -181,14 +186,14 @@ public class TestTimelineAuthenticationFilter {
TimelineClient client = createTimelineClientForUGI();
TimelineEntity entityToStore = new TimelineEntity();
entityToStore.setEntityType(
TestTimelineAuthenticationFilter.class.getName());
TestTimelineAuthenticationFilterForV1.class.getName());
entityToStore.setEntityId("entity1");
entityToStore.setStartTime(0L);
TimelinePutResponse putResponse = client.putEntities(entityToStore);
Assert.assertEquals(0, putResponse.getErrors().size());
TimelineEntity entityToRead =
testTimelineServer.getTimelineStore().getEntity(
"entity1", TestTimelineAuthenticationFilter.class.getName(), null);
testTimelineServer.getTimelineStore().getEntity("entity1",
TestTimelineAuthenticationFilterForV1.class.getName(), null);
Assert.assertNotNull(entityToRead);
return null;
}
@ -202,13 +207,14 @@ public class TestTimelineAuthenticationFilter {
public Void call() throws Exception {
TimelineClient client = createTimelineClientForUGI();
TimelineDomain domainToStore = new TimelineDomain();
domainToStore.setId(TestTimelineAuthenticationFilter.class.getName());
domainToStore.setId(
TestTimelineAuthenticationFilterForV1.class.getName());
domainToStore.setReaders("*");
domainToStore.setWriters("*");
client.putDomain(domainToStore);
TimelineDomain domainToRead =
testTimelineServer.getTimelineStore().getDomain(
TestTimelineAuthenticationFilter.class.getName());
TestTimelineAuthenticationFilterForV1.class.getName());
Assert.assertNotNull(domainToRead);
return null;
}
@ -218,22 +224,24 @@ public class TestTimelineAuthenticationFilter {
@Test
public void testDelegationTokenOperations() throws Exception {
TimelineClient httpUserClient =
KerberosTestUtils.doAs(HTTP_USER + "/localhost", new Callable<TimelineClient>() {
@Override
public TimelineClient call() throws Exception {
return createTimelineClientForUGI();
}
});
KerberosTestUtils.doAs(HTTP_USER + "/localhost",
new Callable<TimelineClient>() {
@Override
public TimelineClient call() throws Exception {
return createTimelineClientForUGI();
}
});
UserGroupInformation httpUser =
KerberosTestUtils.doAs(HTTP_USER + "/localhost", new Callable<UserGroupInformation>() {
@Override
public UserGroupInformation call() throws Exception {
return UserGroupInformation.getCurrentUser();
}
});
KerberosTestUtils.doAs(HTTP_USER + "/localhost",
new Callable<UserGroupInformation>() {
@Override
public UserGroupInformation call() throws Exception {
return UserGroupInformation.getCurrentUser();
}
});
// Let HTTP user to get the delegation for itself
Token<TimelineDelegationTokenIdentifier> token =
httpUserClient.getDelegationToken(httpUser.getShortUserName());
httpUserClient.getDelegationToken(httpUser.getShortUserName());
Assert.assertNotNull(token);
TimelineDelegationTokenIdentifier tDT = token.decodeIdentifier();
Assert.assertNotNull(tDT);
@ -317,7 +325,8 @@ public class TestTimelineAuthenticationFilter {
barUserClient.getDelegationToken(httpUser.getShortUserName());
Assert.fail();
} catch (Exception e) {
Assert.assertTrue(e.getCause() instanceof AuthorizationException || e.getCause() instanceof AuthenticationException);
Assert.assertTrue(e.getCause() instanceof AuthorizationException ||
e.getCause() instanceof AuthenticationException);
}
}
}

View File

@ -24,6 +24,7 @@ import java.util.Set;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.NodeLabel;
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.server.api.records.NodeStatus;
import org.apache.hadoop.yarn.util.Records;
@ -47,7 +48,7 @@ public abstract class NodeHeartbeatRequest {
public static NodeHeartbeatRequest newInstance(NodeStatus nodeStatus,
MasterKey lastKnownContainerTokenMasterKey,
MasterKey lastKnownNMTokenMasterKey, Set<NodeLabel> nodeLabels,
Map<ApplicationId, String> registeredCollectors) {
Map<ApplicationId, AppCollectorData> registeringCollectors) {
NodeHeartbeatRequest nodeHeartbeatRequest =
Records.newRecord(NodeHeartbeatRequest.class);
nodeHeartbeatRequest.setNodeStatus(nodeStatus);
@ -56,7 +57,7 @@ public abstract class NodeHeartbeatRequest {
nodeHeartbeatRequest
.setLastKnownNMTokenMasterKey(lastKnownNMTokenMasterKey);
nodeHeartbeatRequest.setNodeLabels(nodeLabels);
nodeHeartbeatRequest.setRegisteredCollectors(registeredCollectors);
nodeHeartbeatRequest.setRegisteringCollectors(registeringCollectors);
return nodeHeartbeatRequest;
}
@ -79,7 +80,9 @@ public abstract class NodeHeartbeatRequest {
List<LogAggregationReport> logAggregationReportsForApps);
// This tells RM registered collectors' address info on this node
public abstract Map<ApplicationId, String> getRegisteredCollectors();
public abstract void setRegisteredCollectors(Map<ApplicationId,
String> appCollectorsMap);
public abstract Map<ApplicationId, AppCollectorData>
getRegisteringCollectors();
public abstract void setRegisteringCollectors(Map<ApplicationId,
AppCollectorData> appCollectorsMap);
}

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.server.api.records.NodeAction;
@ -47,10 +48,9 @@ public abstract class NodeHeartbeatResponse {
public abstract List<ApplicationId> getApplicationsToCleanup();
// This tells NM the collectors' address info of related apps
public abstract Map<ApplicationId, String> getAppCollectorsMap();
public abstract void setAppCollectorsMap(
Map<ApplicationId, String> appCollectorsMap);
public abstract Map<ApplicationId, AppCollectorData> getAppCollectors();
public abstract void setAppCollectors(
Map<ApplicationId, AppCollectorData> appCollectorsMap);
public abstract void setResponseId(int responseId);

View File

@ -22,14 +22,15 @@ import java.util.Arrays;
import org.apache.hadoop.classification.InterfaceAudience.Private;
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;
@Private
public abstract class ReportNewCollectorInfoRequest {
public static ReportNewCollectorInfoRequest newInstance(
List<AppCollectorsMap> appCollectorsList) {
List<AppCollectorData> appCollectorsList) {
ReportNewCollectorInfoRequest request =
Records.newRecord(ReportNewCollectorInfoRequest.class);
request.setAppCollectorsList(appCollectorsList);
@ -37,17 +38,17 @@ public abstract class ReportNewCollectorInfoRequest {
}
public static ReportNewCollectorInfoRequest newInstance(
ApplicationId id, String collectorAddr) {
ApplicationId id, String collectorAddr, Token token) {
ReportNewCollectorInfoRequest request =
Records.newRecord(ReportNewCollectorInfoRequest.class);
request.setAppCollectorsList(
Arrays.asList(AppCollectorsMap.newInstance(id, collectorAddr)));
Arrays.asList(AppCollectorData.newInstance(id, collectorAddr, token)));
return request;
}
public abstract List<AppCollectorsMap> getAppCollectorsList();
public abstract List<AppCollectorData> getAppCollectorsList();
public abstract void setAppCollectorsList(
List<AppCollectorsMap> appCollectorsList);
List<AppCollectorData> appCollectorsList);
}

View File

@ -26,16 +26,20 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.NodeLabel;
import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.NodeLabelPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
import org.apache.hadoop.yarn.proto.YarnProtos.NodeLabelProto;
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto;
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeStatusProto;
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorDataProto;
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.LogAggregationReportProto;
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorsMapProto;
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatRequestProto;
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatRequestProtoOrBuilder;
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeLabelsProto;
@ -58,7 +62,7 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
private Set<NodeLabel> labels = null;
private List<LogAggregationReport> logAggregationReportsForApps = null;
private Map<ApplicationId, String> registeredCollectors = null;
private Map<ApplicationId, AppCollectorData> registeringCollectors = null;
public NodeHeartbeatRequestPBImpl() {
builder = NodeHeartbeatRequestProto.newBuilder();
@ -114,8 +118,8 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
if (this.logAggregationReportsForApps != null) {
addLogAggregationStatusForAppsToProto();
}
if (this.registeredCollectors != null) {
addRegisteredCollectorsToProto();
if (this.registeringCollectors != null) {
addRegisteringCollectorsToProto();
}
}
@ -158,14 +162,23 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
return ((LogAggregationReportPBImpl) value).getProto();
}
private void addRegisteredCollectorsToProto() {
private void addRegisteringCollectorsToProto() {
maybeInitBuilder();
builder.clearRegisteredCollectors();
for (Map.Entry<ApplicationId, String> entry :
registeredCollectors.entrySet()) {
builder.addRegisteredCollectors(AppCollectorsMapProto.newBuilder()
.setAppId(convertToProtoFormat(entry.getKey()))
.setAppCollectorAddr(entry.getValue()));
builder.clearRegisteringCollectors();
for (Map.Entry<ApplicationId, AppCollectorData> entry :
registeringCollectors.entrySet()) {
AppCollectorData data = entry.getValue();
AppCollectorDataProto.Builder appCollectorDataBuilder =
AppCollectorDataProto.newBuilder()
.setAppId(convertToProtoFormat(entry.getKey()))
.setAppCollectorAddr(data.getCollectorAddr())
.setRmIdentifier(data.getRMIdentifier())
.setVersion(data.getVersion());
if (data.getCollectorToken() != null) {
appCollectorDataBuilder.setAppCollectorToken(
convertToProtoFormat(data.getCollectorToken()));
}
builder.addRegisteringCollectors(appCollectorDataBuilder);
}
}
@ -251,35 +264,42 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
}
@Override
public Map<ApplicationId, String> getRegisteredCollectors() {
if (this.registeredCollectors != null) {
return this.registeredCollectors;
public Map<ApplicationId, AppCollectorData> getRegisteringCollectors() {
if (this.registeringCollectors != null) {
return this.registeringCollectors;
}
initRegisteredCollectors();
return registeredCollectors;
return registeringCollectors;
}
private void initRegisteredCollectors() {
NodeHeartbeatRequestProtoOrBuilder p = viaProto ? proto : builder;
List<AppCollectorsMapProto> list = p.getRegisteredCollectorsList();
List<AppCollectorDataProto> list = p.getRegisteringCollectorsList();
if (!list.isEmpty()) {
this.registeredCollectors = new HashMap<>();
for (AppCollectorsMapProto c : list) {
this.registeringCollectors = new HashMap<>();
for (AppCollectorDataProto c : list) {
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
public void setRegisteredCollectors(
Map<ApplicationId, String> registeredCollectors) {
public void setRegisteringCollectors(
Map<ApplicationId, AppCollectorData> registeredCollectors) {
if (registeredCollectors == null || registeredCollectors.isEmpty()) {
return;
}
maybeInitBuilder();
this.registeredCollectors = new HashMap<ApplicationId, String>();
this.registeredCollectors.putAll(registeredCollectors);
this.registeringCollectors = new HashMap<>();
this.registeringCollectors.putAll(registeredCollectors);
}
private NodeStatusPBImpl convertFromProtoFormat(NodeStatusProto p) {
@ -306,6 +326,14 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
return ((MasterKeyPBImpl)t).getProto();
}
private TokenPBImpl convertFromProtoFormat(TokenProto p) {
return new TokenPBImpl(p);
}
private TokenProto convertToProtoFormat(Token t) {
return ((TokenPBImpl) t).getProto();
}
@Override
public Set<NodeLabel> getNodeLabels() {
initNodeLabels();

View File

@ -26,26 +26,30 @@ import java.util.Iterator;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SignalContainerRequestPBImpl;
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorDataProto;
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ContainerQueuingLimitProto;
import org.apache.hadoop.yarn.proto.YarnServiceProtos.SignalContainerRequestProto;
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto;
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.NodeHeartbeatResponseProtoOrBuilder;
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.SystemCredentialsForAppsProto;
@ -70,7 +74,7 @@ public class NodeHeartbeatResponsePBImpl extends NodeHeartbeatResponse {
private List<ApplicationId> applicationsToCleanup = null;
private Map<ApplicationId, ByteBuffer> systemCredentials = null;
private Resource resource = null;
private Map<ApplicationId, String> appCollectorsMap = null;
private Map<ApplicationId, AppCollectorData> appCollectorsMap = null;
private MasterKey containerTokenMasterKey = null;
private MasterKey nmTokenMasterKey = null;
@ -146,11 +150,21 @@ public class NodeHeartbeatResponsePBImpl extends NodeHeartbeatResponse {
private void addAppCollectorsMapToProto() {
maybeInitBuilder();
builder.clearAppCollectorsMap();
for (Map.Entry<ApplicationId, String> entry : appCollectorsMap.entrySet()) {
builder.addAppCollectorsMap(AppCollectorsMapProto.newBuilder()
.setAppId(convertToProtoFormat(entry.getKey()))
.setAppCollectorAddr(entry.getValue()));
builder.clearAppCollectors();
for (Map.Entry<ApplicationId, AppCollectorData> entry
: appCollectorsMap.entrySet()) {
AppCollectorData data = entry.getValue();
AppCollectorDataProto.Builder appCollectorDataBuilder =
AppCollectorDataProto.newBuilder()
.setAppId(convertToProtoFormat(entry.getKey()))
.setAppCollectorAddr(data.getCollectorAddr())
.setRmIdentifier(data.getRMIdentifier())
.setVersion(data.getVersion());
if (data.getCollectorToken() != null) {
appCollectorDataBuilder.setAppCollectorToken(
convertToProtoFormat(data.getCollectorToken()));
}
builder.addAppCollectors(appCollectorDataBuilder);
}
}
@ -568,7 +582,7 @@ public class NodeHeartbeatResponsePBImpl extends NodeHeartbeatResponse {
}
@Override
public Map<ApplicationId, String> getAppCollectorsMap() {
public Map<ApplicationId, AppCollectorData> getAppCollectors() {
if (this.appCollectorsMap != null) {
return this.appCollectorsMap;
}
@ -589,12 +603,19 @@ public class NodeHeartbeatResponsePBImpl extends NodeHeartbeatResponse {
private void initAppCollectorsMap() {
NodeHeartbeatResponseProtoOrBuilder p = viaProto ? proto : builder;
List<AppCollectorsMapProto> list = p.getAppCollectorsMapList();
List<AppCollectorDataProto> list = p.getAppCollectorsList();
if (!list.isEmpty()) {
this.appCollectorsMap = new HashMap<>();
for (AppCollectorsMapProto c : list) {
for (AppCollectorDataProto c : list) {
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
public void setAppCollectorsMap(
Map<ApplicationId, String> appCollectorsMap) {
if (appCollectorsMap == null || appCollectorsMap.isEmpty()) {
public void setAppCollectors(
Map<ApplicationId, AppCollectorData> appCollectors) {
if (appCollectors == null || appCollectors.isEmpty()) {
return;
}
maybeInitBuilder();
this.appCollectorsMap = new HashMap<ApplicationId, String>();
this.appCollectorsMap.putAll(appCollectorsMap);
this.appCollectorsMap = new HashMap<>();
this.appCollectorsMap.putAll(appCollectors);
}
@Override
@ -773,5 +794,13 @@ public class NodeHeartbeatResponsePBImpl extends NodeHeartbeatResponse {
SignalContainerRequest t) {
return ((SignalContainerRequestPBImpl)t).getProto();
}
private TokenProto convertToProtoFormat(Token t) {
return ((TokenPBImpl) t).getProto();
}
private TokenPBImpl convertFromProtoFormat(TokenProto p) {
return new TokenPBImpl(p);
}
}

View File

@ -20,12 +20,12 @@ package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;
import java.util.ArrayList;
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.ReportNewCollectorInfoRequestProtoOrBuilder;
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
ReportNewCollectorInfoRequest {
@ -36,7 +36,7 @@ public class ReportNewCollectorInfoRequestPBImpl extends
private ReportNewCollectorInfoRequestProto.Builder builder = null;
private boolean viaProto = false;
private List<AppCollectorsMap> collectorsList = null;
private List<AppCollectorData> collectorsList = null;
public ReportNewCollectorInfoRequestPBImpl() {
builder = ReportNewCollectorInfoRequestProto.newBuilder();
@ -96,9 +96,9 @@ public class ReportNewCollectorInfoRequestPBImpl extends
private void addLocalCollectorsToProto() {
maybeInitBuilder();
builder.clearAppCollectors();
List<AppCollectorsMapProto> protoList =
new ArrayList<AppCollectorsMapProto>();
for (AppCollectorsMap m : this.collectorsList) {
List<AppCollectorDataProto> protoList =
new ArrayList<AppCollectorDataProto>();
for (AppCollectorData m : this.collectorsList) {
protoList.add(convertToProtoFormat(m));
}
builder.addAllAppCollectors(protoList);
@ -106,16 +106,16 @@ public class ReportNewCollectorInfoRequestPBImpl extends
private void initLocalCollectorsList() {
ReportNewCollectorInfoRequestProtoOrBuilder p = viaProto ? proto : builder;
List<AppCollectorsMapProto> list =
List<AppCollectorDataProto> list =
p.getAppCollectorsList();
this.collectorsList = new ArrayList<AppCollectorsMap>();
for (AppCollectorsMapProto m : list) {
this.collectorsList = new ArrayList<AppCollectorData>();
for (AppCollectorDataProto m : list) {
this.collectorsList.add(convertFromProtoFormat(m));
}
}
@Override
public List<AppCollectorsMap> getAppCollectorsList() {
public List<AppCollectorData> getAppCollectorsList() {
if (this.collectorsList == null) {
initLocalCollectorsList();
}
@ -123,7 +123,7 @@ public class ReportNewCollectorInfoRequestPBImpl extends
}
@Override
public void setAppCollectorsList(List<AppCollectorsMap> appCollectorsList) {
public void setAppCollectorsList(List<AppCollectorData> appCollectorsList) {
maybeInitBuilder();
if (appCollectorsList == null) {
builder.clearAppCollectors();
@ -131,14 +131,14 @@ public class ReportNewCollectorInfoRequestPBImpl extends
this.collectorsList = appCollectorsList;
}
private AppCollectorsMapPBImpl convertFromProtoFormat(
AppCollectorsMapProto p) {
return new AppCollectorsMapPBImpl(p);
private AppCollectorDataPBImpl convertFromProtoFormat(
AppCollectorDataProto p) {
return new AppCollectorDataPBImpl(p);
}
private AppCollectorsMapProto convertToProtoFormat(
AppCollectorsMap m) {
return ((AppCollectorsMapPBImpl) m).getProto();
private AppCollectorDataProto convertToProtoFormat(
AppCollectorData m) {
return ((AppCollectorDataPBImpl) m).getProto();
}
}

View File

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

View File

@ -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.InterfaceStability.Unstable;
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
import org.apache.hadoop.yarn.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.YarnServerCommonServiceProtos.AppCollectorsMapProto;
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorsMapProtoOrBuilder;
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorDataProto;
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorDataProtoOrBuilder;
import com.google.protobuf.TextFormat;
@Private
@Unstable
public class AppCollectorsMapPBImpl extends AppCollectorsMap {
public class AppCollectorDataPBImpl extends AppCollectorData {
private AppCollectorsMapProto proto =
AppCollectorsMapProto.getDefaultInstance();
private AppCollectorDataProto proto =
AppCollectorDataProto.getDefaultInstance();
private AppCollectorsMapProto.Builder builder = null;
private AppCollectorDataProto.Builder builder = null;
private boolean viaProto = false;
private ApplicationId appId = null;
private String collectorAddr = null;
private Long rmIdentifier = null;
private Long version = null;
private Token collectorToken = null;
public AppCollectorsMapPBImpl() {
builder = AppCollectorsMapProto.newBuilder();
public AppCollectorDataPBImpl() {
builder = AppCollectorDataProto.newBuilder();
}
public AppCollectorsMapPBImpl(AppCollectorsMapProto proto) {
public AppCollectorDataPBImpl(AppCollectorDataProto proto) {
this.proto = proto;
viaProto = true;
}
public AppCollectorsMapProto getProto() {
public AppCollectorDataProto getProto() {
mergeLocalToProto();
proto = viaProto ? proto : builder.build();
viaProto = true;
@ -81,7 +85,7 @@ public class AppCollectorsMapPBImpl extends AppCollectorsMap {
@Override
public ApplicationId getApplicationId() {
AppCollectorsMapProtoOrBuilder p = viaProto ? proto : builder;
AppCollectorDataProtoOrBuilder p = viaProto ? proto : builder;
if (this.appId == null && p.hasAppId()) {
this.appId = convertFromProtoFormat(p.getAppId());
}
@ -90,7 +94,7 @@ public class AppCollectorsMapPBImpl extends AppCollectorsMap {
@Override
public String getCollectorAddr() {
AppCollectorsMapProtoOrBuilder p = viaProto ? proto : builder;
AppCollectorDataProtoOrBuilder p = viaProto ? proto : builder;
if (this.collectorAddr == null
&& p.hasAppCollectorAddr()) {
this.collectorAddr = p.getAppCollectorAddr();
@ -116,6 +120,68 @@ public class AppCollectorsMapPBImpl extends AppCollectorsMap {
this.collectorAddr = collectorAddr;
}
@Override
public long getRMIdentifier() {
AppCollectorDataProtoOrBuilder p = viaProto ? proto : builder;
if (this.rmIdentifier == null && p.hasRmIdentifier()) {
this.rmIdentifier = p.getRmIdentifier();
}
if (this.rmIdentifier != null) {
return this.rmIdentifier;
} else {
return AppCollectorData.DEFAULT_TIMESTAMP_VALUE;
}
}
@Override
public void setRMIdentifier(long rmId) {
maybeInitBuilder();
this.rmIdentifier = rmId;
builder.setRmIdentifier(rmId);
}
@Override
public long getVersion() {
AppCollectorDataProtoOrBuilder p = viaProto ? proto : builder;
if (this.version == null && p.hasRmIdentifier()) {
this.version = p.getRmIdentifier();
}
if (this.version != null) {
return this.version;
} else {
return AppCollectorData.DEFAULT_TIMESTAMP_VALUE;
}
}
@Override
public void setVersion(long version) {
maybeInitBuilder();
this.version = version;
builder.setVersion(version);
}
@Override
public Token getCollectorToken() {
AppCollectorDataProtoOrBuilder p = viaProto ? proto : builder;
if (this.collectorToken != null) {
return this.collectorToken;
}
if (!p.hasAppCollectorToken()) {
return null;
}
this.collectorToken = new TokenPBImpl(p.getAppCollectorToken());
return this.collectorToken;
}
@Override
public void setCollectorToken(Token token) {
maybeInitBuilder();
if (token == null) {
builder.clearAppCollectorToken();
}
this.collectorToken = token;
}
private ApplicationIdPBImpl convertFromProtoFormat(ApplicationIdProto p) {
return new ApplicationIdPBImpl(p);
}
@ -126,7 +192,7 @@ public class AppCollectorsMapPBImpl extends AppCollectorsMap {
private void maybeInitBuilder() {
if (viaProto || builder == null) {
builder = AppCollectorsMapProto.newBuilder(proto);
builder = AppCollectorDataProto.newBuilder(proto);
}
viaProto = false;
}
@ -147,6 +213,15 @@ public class AppCollectorsMapPBImpl extends AppCollectorsMap {
if (this.collectorAddr != null) {
builder.setAppCollectorAddr(this.collectorAddr);
}
if (this.rmIdentifier != null) {
builder.setRmIdentifier(this.rmIdentifier);
}
if (this.version != null) {
builder.setVersion(this.version);
}
if (this.collectorToken != null) {
builder.setAppCollectorToken(
((TokenPBImpl)this.collectorToken).getProto());
}
}
}

View File

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

View File

@ -23,27 +23,33 @@ import javax.servlet.ServletException;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationFilter;
import org.apache.hadoop.yarn.server.timeline.security.TimelineDelegationTokenSecretManagerService.TimelineDelegationTokenSecretManager;
import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
/**
* Timeline authentication filter provides delegation token support for ATSv1
* and ATSv2.
*/
@Private
@Unstable
public class TimelineAuthenticationFilter
extends DelegationTokenAuthenticationFilter {
private static TimelineDelegationTokenSecretManager secretManager;
private static AbstractDelegationTokenSecretManager
<TimelineDelegationTokenIdentifier> secretManager;
@Override
public void init(FilterConfig filterConfig) throws ServletException {
filterConfig.getServletContext().setAttribute(
DelegationTokenAuthenticationFilter.DELEGATION_TOKEN_SECRET_MANAGER_ATTR,
secretManager);
DelegationTokenAuthenticationFilter.
DELEGATION_TOKEN_SECRET_MANAGER_ATTR, secretManager);
super.init(filterConfig);
}
public static void setTimelineDelegationTokenSecretManager(
TimelineDelegationTokenSecretManager secretManager) {
TimelineAuthenticationFilter.secretManager = secretManager;
AbstractDelegationTokenSecretManager
<TimelineDelegationTokenIdentifier> secretMgr) {
TimelineAuthenticationFilter.secretManager = secretMgr;
}
}

View File

@ -51,30 +51,19 @@ import java.util.Map;
public class TimelineAuthenticationFilterInitializer extends FilterInitializer {
/**
* The configuration prefix of timeline HTTP authentication
* The configuration prefix of timeline HTTP authentication.
*/
public static final String PREFIX = "yarn.timeline-service.http-authentication.";
public static final String PREFIX =
"yarn.timeline-service.http-authentication.";
@VisibleForTesting
Map<String, String> 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) {
protected void setAuthFilterConfig(Configuration conf) {
filterConfig = new HashMap<String, String>();
// 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) {
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);
if (authType.equals(PseudoAuthenticationHandler.TYPE)) {
filterConfig.put(AuthenticationFilter.AUTH_TYPE,
@ -102,23 +126,7 @@ public class TimelineAuthenticationFilterInitializer extends FilterInitializer {
} else if (authType.equals(KerberosAuthenticationHandler.TYPE)) {
filterConfig.put(AuthenticationFilter.AUTH_TYPE,
KerberosDelegationTokenAuthenticationHandler.class.getName());
// Resolve _HOST into bind address
String bindAddress = conf.get(HttpServer2.BIND_ADDRESS);
String principal =
filterConfig.get(KerberosAuthenticationHandler.PRINCIPAL);
if (principal != null) {
try {
principal = SecurityUtil.getServerPrincipal(principal, bindAddress);
} catch (IOException ex) {
throw new RuntimeException(
"Could not resolve Kerberos principal name: " + ex.toString(), ex);
}
filterConfig.put(KerberosAuthenticationHandler.PRINCIPAL,
principal);
}
}
filterConfig.put(DelegationTokenAuthenticationHandler.TOKEN_KIND,
TimelineDelegationTokenIdentifier.KIND_NAME.toString());

View File

@ -0,0 +1,83 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.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;
}
}

View File

@ -0,0 +1,26 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/**
* Package org.apache.hadoop.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;

View File

@ -0,0 +1,92 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.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());
}
}
}

View File

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

View File

@ -22,6 +22,7 @@ option java_generic_services = true;
option java_generate_equals_and_hash = true;
package hadoop.yarn;
import "Security.proto";
import "yarn_protos.proto";
import "yarn_server_common_protos.proto";
import "yarn_service_protos.proto";
@ -90,7 +91,7 @@ message NodeHeartbeatRequestProto {
optional MasterKeyProto last_known_nm_token_master_key = 3;
optional NodeLabelsProto nodeLabels = 4;
repeated LogAggregationReportProto log_aggregation_reports_for_apps = 5;
repeated AppCollectorsMapProto registered_collectors = 6;
repeated AppCollectorDataProto registering_collectors = 6;
}
message LogAggregationReportProto {
@ -116,7 +117,7 @@ message NodeHeartbeatResponseProto {
repeated SignalContainerRequestProto containers_to_signal = 13;
optional ResourceProto resource = 14;
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
repeated ContainerProto containers_to_update = 17;
}
@ -134,16 +135,19 @@ message SystemCredentialsForAppsProto {
////////////////////////////////////////////////////////////////////////
////// From collector_nodemanager_protocol ////////////////////////////
////////////////////////////////////////////////////////////////////////
message AppCollectorsMapProto {
optional ApplicationIdProto appId = 1;
optional string appCollectorAddr = 2;
message AppCollectorDataProto {
optional ApplicationIdProto app_id = 1;
optional string app_collector_addr = 2;
optional int64 rm_identifier = 3 [default = -1];
optional int64 version = 4 [default = -1];
optional hadoop.common.TokenProto app_collector_token = 5;
}
//////////////////////////////////////////////////////
/////// collector_nodemanager_protocol //////////////
//////////////////////////////////////////////////////
message ReportNewCollectorInfoRequestProto {
repeated AppCollectorsMapProto app_collectors = 1;
repeated AppCollectorDataProto app_collectors = 1;
}
message ReportNewCollectorInfoResponseProto {

View File

@ -30,6 +30,7 @@ import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
@ -72,12 +73,13 @@ import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC;
import org.apache.hadoop.yarn.ipc.RPCUtil;
import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocol;
import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextRequest;
import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextResponse;
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoRequest;
import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoResponse;
import org.apache.hadoop.yarn.server.api.records.AppCollectorsMap;
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
import org.apache.hadoop.yarn.util.Records;
import org.junit.Assert;
import org.junit.Test;
@ -93,6 +95,21 @@ public class TestRPC {
"collectors' number in ReportNewCollectorInfoRequest is not ONE.";
public static final String DEFAULT_COLLECTOR_ADDR = "localhost:0";
private static final Token DEFAULT_COLLECTOR_TOKEN;
static {
TimelineDelegationTokenIdentifier identifier =
new TimelineDelegationTokenIdentifier();
identifier.setOwner(new Text("user"));
identifier.setRenewer(new Text("user"));
identifier.setRealUser(new Text("user"));
long now = Time.now();
identifier.setIssueDate(now);
identifier.setMaxDate(now + 1000L);
identifier.setMasterKeyId(500);
identifier.setSequenceNumber(5);
DEFAULT_COLLECTOR_TOKEN = Token.newInstance(identifier.getBytes(),
identifier.getKind().toString(), identifier.getBytes(), "localhost:0");
}
public static final ApplicationId DEFAULT_APP_ID =
ApplicationId.newInstance(0, 0);
@ -173,7 +190,16 @@ public class TestRPC {
try {
ReportNewCollectorInfoRequest request =
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);
} catch (YarnException e) {
Assert.fail("RPC call failured is not expected here.");
@ -429,14 +455,16 @@ public class TestRPC {
public ReportNewCollectorInfoResponse reportNewCollectorInfo(
ReportNewCollectorInfoRequest request)
throws YarnException, IOException {
List<AppCollectorsMap> appCollectors = request.getAppCollectorsList();
List<AppCollectorData> appCollectors = request.getAppCollectorsList();
if (appCollectors.size() == 1) {
// check default appID and collectorAddr
AppCollectorsMap appCollector = appCollectors.get(0);
AppCollectorData appCollector = appCollectors.get(0);
Assert.assertEquals(appCollector.getApplicationId(),
DEFAULT_APP_ID);
Assert.assertEquals(appCollector.getCollectorAddr(),
DEFAULT_COLLECTOR_ADDR);
Assert.assertTrue(appCollector.getCollectorToken() == null ||
appCollector.getCollectorToken().equals(DEFAULT_COLLECTOR_TOKEN));
} else {
throw new YarnException(ILLEGAL_NUMBER_MESSAGE);
}

View File

@ -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.NodeLabel;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
@ -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.RegisterNodeManagerResponsePBImpl;
import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.UnRegisterNodeManagerRequestPBImpl;
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.server.api.records.NodeAction;
import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
@ -109,14 +111,14 @@ public class TestYarnServerApiClasses {
original.setLastKnownNMTokenMasterKey(getMasterKey());
original.setNodeStatus(getNodeStatus());
original.setNodeLabels(getValidNodeLabels());
Map<ApplicationId, String> collectors = getCollectors();
original.setRegisteredCollectors(collectors);
Map<ApplicationId, AppCollectorData> collectors = getCollectors(false);
original.setRegisteringCollectors(collectors);
NodeHeartbeatRequestPBImpl copy = new NodeHeartbeatRequestPBImpl(
original.getProto());
assertEquals(1, copy.getLastKnownContainerTokenMasterKey().getKeyId());
assertEquals(1, copy.getLastKnownNMTokenMasterKey().getKeyId());
assertEquals("localhost", copy.getNodeStatus().getNodeId().getHost());
assertEquals(collectors, copy.getRegisteredCollectors());
assertEquals(collectors, copy.getRegisteringCollectors());
// check labels are coming with valid values
Assert.assertTrue(original.getNodeLabels()
.containsAll(copy.getNodeLabels()));
@ -128,6 +130,16 @@ public class TestYarnServerApiClasses {
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.
*/
@ -153,8 +165,8 @@ public class TestYarnServerApiClasses {
original.setNextHeartBeatInterval(1000);
original.setNodeAction(NodeAction.NORMAL);
original.setResponseId(100);
Map<ApplicationId, String> collectors = getCollectors();
original.setAppCollectorsMap(collectors);
Map<ApplicationId, AppCollectorData> collectors = getCollectors(false);
original.setAppCollectors(collectors);
NodeHeartbeatResponsePBImpl copy = new NodeHeartbeatResponsePBImpl(
original.getProto());
@ -164,7 +176,7 @@ public class TestYarnServerApiClasses {
assertEquals(1, copy.getContainerTokenMasterKey().getKeyId());
assertEquals(1, copy.getNMTokenMasterKey().getKeyId());
assertEquals("testDiagnosticMessage", copy.getDiagnosticsMessage());
assertEquals(collectors, copy.getAppCollectorsMap());
assertEquals(collectors, copy.getAppCollectors());
assertEquals(false, copy.getAreNodeLabelsAcceptedByRM());
}
@ -177,6 +189,16 @@ public class TestYarnServerApiClasses {
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
public void testNodeHeartbeatResponsePBImplWithDecreasedContainers() {
NodeHeartbeatResponsePBImpl original = new NodeHeartbeatResponsePBImpl();
@ -347,12 +369,18 @@ public class TestYarnServerApiClasses {
return nodeLabels;
}
private Map<ApplicationId, String> getCollectors() {
private Map<ApplicationId, AppCollectorData> getCollectors(
boolean hasNullCollectorToken) {
ApplicationId appID = ApplicationId.newInstance(1L, 1);
String collectorAddr = "localhost:0";
Map<ApplicationId, String> collectorMap =
new HashMap<ApplicationId, String>();
collectorMap.put(appID, collectorAddr);
AppCollectorData data = AppCollectorData.newInstance(appID, collectorAddr);
if (!hasNullCollectorToken) {
data.setCollectorToken(
Token.newInstance(new byte[0], "kind", new byte[0], "s"));
}
Map<ApplicationId, AppCollectorData> collectorMap =
new HashMap<>();
collectorMap.put(appID, data);
return collectorMap;
}

View File

@ -27,7 +27,9 @@ import static org.apache.hadoop.yarn.server.timeline.security.TimelineAuthentica
import org.junit.Test;
import org.mockito.Mockito;
/**
* Tests {@link TimelineAuthenticationFilterInitializer}.
*/
public class TestTimelineAuthenticationFilterInitializer {
@Test

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManager;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
@ -65,11 +66,18 @@ public interface Context {
Map<ApplicationId, Credentials> getSystemCredentialsForApps();
/**
* Get the registered collectors that located on this NM.
* @return registered collectors, or null if the timeline service v.2 is not
* Get the list of collectors that are registering with the RM from this node.
* @return registering collectors, or null if the timeline service v.2 is not
* enabled
*/
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();

View File

@ -57,11 +57,13 @@ import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManager;
import org.apache.hadoop.yarn.server.nodemanager.collectormanager.NMCollectorService;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
import org.apache.hadoop.yarn.server.nodemanager.nodelabels.ConfigurationNodeLabelsProvider;
@ -464,8 +466,14 @@ public class NodeManager extends CompositeService
if (!rmWorkPreservingRestartEnabled) {
LOG.info("Cleaning up running containers on resync");
containerManager.cleanupContainersOnNMResync();
// Clear all known collectors for resync.
if (context.getKnownCollectors() != null) {
context.getKnownCollectors().clear();
}
} else {
LOG.info("Preserving containers on resync");
// Re-register known timeline collectors.
reregisterCollectors();
}
((NodeStatusUpdaterImpl) nodeStatusUpdater)
.rebootNodeStatusUpdaterAndRegisterWithRM();
@ -477,6 +485,38 @@ public class NodeManager extends CompositeService
}.start();
}
/**
* Reregisters all collectors known by this node to the RM. This method is
* called when the RM needs to resync with the node.
*/
protected void reregisterCollectors() {
Map<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 {
private NodeId nodeId = null;
@ -492,7 +532,10 @@ public class NodeManager extends CompositeService
protected final ConcurrentMap<ContainerId, Container> containers =
new ConcurrentSkipListMap<ContainerId, Container>();
private Map<ApplicationId, String> registeredCollectors;
private ConcurrentMap<ApplicationId, AppCollectorData>
registeringCollectors;
private ConcurrentMap<ApplicationId, AppCollectorData> knownCollectors;
protected final ConcurrentMap<ContainerId,
org.apache.hadoop.yarn.api.records.Container> increasedContainers =
@ -526,7 +569,8 @@ public class NodeManager extends CompositeService
NMStateStoreService stateStore, boolean isDistSchedulingEnabled,
Configuration conf) {
if (YarnConfiguration.timelineServiceV2Enabled(conf)) {
this.registeredCollectors = new ConcurrentHashMap<>();
this.registeringCollectors = new ConcurrentHashMap<>();
this.knownCollectors = new ConcurrentHashMap<>();
}
this.containerTokenSecretManager = containerTokenSecretManager;
this.nmTokenSecretManager = nmTokenSecretManager;
@ -681,18 +725,14 @@ public class NodeManager extends CompositeService
}
@Override
public Map<ApplicationId, String> getRegisteredCollectors() {
return this.registeredCollectors;
public ConcurrentMap<ApplicationId, AppCollectorData>
getRegisteringCollectors() {
return this.registeringCollectors;
}
public void addRegisteredCollectors(
Map<ApplicationId, String> newRegisteredCollectors) {
if (registeredCollectors != null) {
this.registeredCollectors.putAll(newRegisteredCollectors);
} else {
LOG.warn("collectors are added when the registered collectors are " +
"initialized");
}
@Override
public ConcurrentMap<ApplicationId, AppCollectorData> getKnownCollectors() {
return this.knownCollectors;
}
@Override

View File

@ -71,7 +71,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
import org.apache.hadoop.yarn.server.api.protocolrecords.UnRegisterNodeManagerRequest;
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
import org.apache.hadoop.yarn.server.api.records.MasterKey;
@ -760,7 +760,6 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
}
protected void startStatusUpdater() {
statusUpdaterRunnable = new StatusUpdaterRunnable();
statusUpdater =
new Thread(statusUpdaterRunnable, "Node Status Updater");
@ -1043,7 +1042,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
.getNMTokenSecretManager().getCurrentKey(),
nodeLabelsForHeartbeat,
NodeStatusUpdaterImpl.this.context
.getRegisteredCollectors());
.getRegisteringCollectors());
if (logAggregationEnabled) {
// 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())) {
updateTimelineClientsAddress(response);
updateTimelineCollectorData(response);
}
} catch (ConnectException e) {
@ -1164,40 +1163,48 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
}
}
private void updateTimelineClientsAddress(
private void updateTimelineCollectorData(
NodeHeartbeatResponse response) {
Map<ApplicationId, String> knownCollectorsMap =
response.getAppCollectorsMap();
if (knownCollectorsMap == null) {
Map<ApplicationId, AppCollectorData> incomingCollectorsMap =
response.getAppCollectors();
if (incomingCollectorsMap == null) {
if (LOG.isDebugEnabled()) {
LOG.debug("No collectors to update RM");
}
} else {
Set<Map.Entry<ApplicationId, String>> rmKnownCollectors =
knownCollectorsMap.entrySet();
for (Map.Entry<ApplicationId, String> entry : rmKnownCollectors) {
ApplicationId appId = entry.getKey();
String collectorAddr = entry.getValue();
return;
}
Map<ApplicationId, AppCollectorData> knownCollectors =
context.getKnownCollectors();
for (Map.Entry<ApplicationId, AppCollectorData> entry
: incomingCollectorsMap.entrySet()) {
ApplicationId appId = entry.getKey();
AppCollectorData collectorData = entry.getValue();
// Only handle applications running on local node.
// Not include apps with timeline collectors running in local
Application application = context.getApplications().get(appId);
// TODO this logic could be problematic if the collector address
// gets updated due to NM restart or collector service failure
if (application != null &&
!context.getRegisteredCollectors().containsKey(appId)) {
// Only handle applications running on local node.
Application application = context.getApplications().get(appId);
if (application != null) {
// Update collector data if the newly received data happens after
// the known data (updates the known data).
AppCollectorData existingData = knownCollectors.get(appId);
if (AppCollectorData.happensBefore(existingData, collectorData)) {
if (LOG.isDebugEnabled()) {
LOG.debug("Sync a new collector address: " + collectorAddr +
" for application: " + appId + " from RM.");
LOG.debug("Sync a new collector address: "
+ collectorData.getCollectorAddr()
+ " for application: " + appId + " from RM.");
}
// Update information for clients.
NMTimelinePublisher nmTimelinePublisher =
context.getNMTimelinePublisher();
if (nmTimelinePublisher != null) {
nmTimelinePublisher.setTimelineServiceAddress(
application.getAppId(), 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());
}
}

View File

@ -26,6 +26,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.service.CompositeService;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@ -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.ReportNewCollectorInfoRequest;
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.NodeManager;
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;
/**
@ -74,16 +75,21 @@ public class NMCollectorService extends CompositeService implements
Configuration serverConf = new Configuration(conf);
// TODO Security settings.
YarnRPC rpc = YarnRPC.create(conf);
// Kerberos based authentication to be used for CollectorNodemanager
// protocol if security is enabled.
server =
rpc.getServer(CollectorNodemanagerProtocol.class, this,
collectorServerAddress, serverConf,
this.context.getNMTokenSecretManager(),
collectorServerAddress, serverConf, null,
conf.getInt(YarnConfiguration.NM_COLLECTOR_SERVICE_THREAD_COUNT,
YarnConfiguration.DEFAULT_NM_COLLECTOR_SERVICE_THREAD_COUNT));
if (conf.getBoolean(
CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, false)) {
server.refreshServiceAcl(conf, new NMPolicyProvider());
}
server.start();
collectorServerAddress = conf.updateConnectAddr(
YarnConfiguration.NM_BIND_HOST,
@ -95,7 +101,6 @@ public class NMCollectorService extends CompositeService implements
LOG.info("NMCollectorService started at " + collectorServerAddress);
}
@Override
public void serviceStop() throws Exception {
if (server != null) {
@ -108,23 +113,31 @@ public class NMCollectorService extends CompositeService implements
@Override
public ReportNewCollectorInfoResponse reportNewCollectorInfo(
ReportNewCollectorInfoRequest request) throws YarnException, IOException {
List<AppCollectorsMap> newCollectorsList = request.getAppCollectorsList();
List<AppCollectorData> newCollectorsList = request.getAppCollectorsList();
if (newCollectorsList != null && !newCollectorsList.isEmpty()) {
Map<ApplicationId, String> newCollectorsMap =
new HashMap<ApplicationId, String>();
for (AppCollectorsMap collector : newCollectorsList) {
Map<ApplicationId, AppCollectorData> newCollectorsMap =
new HashMap<>();
for (AppCollectorData collector : newCollectorsList) {
ApplicationId appId = collector.getApplicationId();
String collectorAddr = collector.getCollectorAddr();
newCollectorsMap.put(appId, collectorAddr);
newCollectorsMap.put(appId, collector);
// set registered collector address to TimelineClient.
// TODO: Do we need to do this after we received confirmation from
// the RM?
NMTimelinePublisher nmTimelinePublisher =
context.getNMTimelinePublisher();
if (nmTimelinePublisher != null) {
nmTimelinePublisher.setTimelineServiceAddress(appId, collectorAddr);
nmTimelinePublisher.setTimelineServiceAddress(appId,
collector.getCollectorAddr());
}
}
((NodeManager.NMContext)context).addRegisteredCollectors(
newCollectorsMap);
Map<ApplicationId, AppCollectorData> registeringCollectors
= context.getRegisteringCollectors();
if (registeringCollectors != null) {
registeringCollectors.putAll(newCollectorsMap);
} else {
LOG.warn("collectors are added when the registered collectors are " +
"initialized");
}
}
return ReportNewCollectorInfoResponse.newInstance();

View File

@ -244,7 +244,8 @@ public class AuxServices extends AbstractService
for (AuxiliaryService serv : serviceMap.values()) {
try {
serv.initializeContainer(new ContainerInitializationContext(
event.getUser(), event.getContainer().getContainerId(),
event.getContainer().getUser(),
event.getContainer().getContainerId(),
event.getContainer().getResource(), event.getContainer()
.getContainerTokenIdentifier().getContainerType()));
} catch (Throwable th) {

View File

@ -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.utils.BuilderUtils;
import org.apache.hadoop.yarn.server.utils.YarnServerSecurityUtils;
import org.apache.hadoop.yarn.util.SystemClock;
import org.apache.hadoop.yarn.util.resource.Resources;
import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
@ -401,6 +402,16 @@ public class ContainerManagerImpl extends CompositeService implements
LOG.debug(
"Recovering Flow context: " + fc + " for an application " + appId);
}
} else {
// in upgrade situations, where there is no prior existing flow context,
// default would be used.
fc = new FlowContext(TimelineUtils.generateDefaultFlowName(null, appId),
YarnConfiguration.DEFAULT_FLOW_VERSION, appId.getClusterTimestamp());
if (LOG.isDebugEnabled()) {
LOG.debug(
"No prior existing flow context found. Using default Flow context: "
+ fc + " for an application " + appId);
}
}
LOG.info("Recovering application " + appId);
@ -1052,10 +1063,11 @@ public class ContainerManagerImpl extends CompositeService implements
Credentials credentials =
YarnServerSecurityUtils.parseCredentials(launchContext);
long containerStartTime = SystemClock.getInstance().getTime();
Container container =
new ContainerImpl(getConfig(), this.dispatcher,
launchContext, credentials, metrics, containerTokenIdentifier,
context);
context, containerStartTime);
ApplicationId applicationID =
containerId.getApplicationAttemptId().getApplicationId();
if (context.getContainers().putIfAbsent(containerId, container) != null) {
@ -1112,7 +1124,7 @@ public class ContainerManagerImpl extends CompositeService implements
}
this.context.getNMStateStore().storeContainer(containerId,
containerTokenIdentifier.getVersion(), request);
containerTokenIdentifier.getVersion(), containerStartTime, request);
dispatcher.getEventHandler().handle(
new ApplicationContainerInitEvent(container));

View File

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

View File

@ -45,6 +45,7 @@ import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.proto.YarnProtos;
import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.ContainerManagerApplicationProto;
import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.FlowContextProto;
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
import org.apache.hadoop.yarn.server.nodemanager.Context;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEvent;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEventType;
@ -558,6 +559,29 @@ public class ApplicationImpl implements Application {
@SuppressWarnings("unchecked")
static class AppCompletelyDoneTransition implements
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
public void transition(ApplicationImpl app, ApplicationEvent event) {
@ -566,20 +590,7 @@ public class ApplicationImpl implements Application {
new LogHandlerAppFinishedEvent(app.appId));
app.context.getNMTokenSecretManager().appFinished(app.getAppId());
// Remove collectors info for finished apps.
// 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());
}
updateCollectorStatus(app);
}
}

View File

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

View File

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

View File

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

View File

@ -71,7 +71,7 @@ public class NMNullStateStoreService extends NMStateStoreService {
@Override
public void storeContainer(ContainerId containerId, int version,
StartContainerRequest startRequest) throws IOException {
long startTime, StartContainerRequest startRequest) throws IOException {
}
@Override

View File

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

View File

@ -23,6 +23,7 @@ import org.apache.hadoop.security.authorize.PolicyProvider;
import org.apache.hadoop.security.authorize.Service;
import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocolPB;
import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocolPB;
/**
@ -32,18 +33,21 @@ import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocolPB;
@InterfaceStability.Unstable
public class NMPolicyProvider extends PolicyProvider {
private static final Service[] nodeManagerServices =
private static final Service[] NODE_MANAGER_SERVICES =
new Service[] {
new Service(
YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_CONTAINER_MANAGEMENT_PROTOCOL,
ContainerManagementProtocolPB.class),
new Service(YarnConfiguration.YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCE_LOCALIZER,
LocalizationProtocolPB.class)
};
new Service(YarnConfiguration.
YARN_SECURITY_SERVICE_AUTHORIZATION_CONTAINER_MANAGEMENT_PROTOCOL,
ContainerManagementProtocolPB.class),
new Service(YarnConfiguration.
YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCE_LOCALIZER,
LocalizationProtocolPB.class),
new Service(YarnConfiguration.
YARN_SECURITY_SERVICE_AUTHORIZATION_COLLECTOR_NODEMANAGER_PROTOCOL,
CollectorNodemanagerProtocolPB.class)
};
@Override
public Service[] getServices() {
return nodeManagerServices;
return NODE_MANAGER_SERVICES;
}
}

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.yarn.server.nodemanager.timelineservice;
import java.io.IOException;
import java.security.PrivilegedExceptionAction;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
@ -26,8 +27,10 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.service.CompositeService;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.CollectorInfo;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
@ -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.monitor.ContainersMonitorImpl.ContainerMetric;
import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
import org.apache.hadoop.yarn.util.TimelineServiceHelper;
import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
import com.google.common.annotations.VisibleForTesting;
@ -77,6 +81,8 @@ public class NMTimelinePublisher extends CompositeService {
private String httpAddress;
private UserGroupInformation nmLoginUGI;
private final Map<ApplicationId, TimelineV2Client> appToClientMap;
public NMTimelinePublisher(Context context) {
@ -91,6 +97,9 @@ public class NMTimelinePublisher extends CompositeService {
dispatcher.register(NMTimelineEventType.class,
new ForwardingEventHandler());
addIfService(dispatcher);
this.nmLoginUGI = UserGroupInformation.isSecurityEnabled() ?
UserGroupInformation.getLoginUser() :
UserGroupInformation.getCurrentUser();
super.serviceInit(conf);
}
@ -149,6 +158,8 @@ public class NMTimelinePublisher extends CompositeService {
Math.round(cpuUsagePercentPerCore));
entity.addMetric(cpuMetric);
}
entity.setIdPrefix(TimelineServiceHelper.
invertLong(container.getContainerStartTime()));
ApplicationId appId = container.getContainerId().getApplicationAttemptId()
.getApplicationId();
try {
@ -195,15 +206,17 @@ public class NMTimelinePublisher extends CompositeService {
tEvent.setId(ContainerMetricsConstants.CREATED_EVENT_TYPE);
tEvent.setTimestamp(event.getTimestamp());
long containerStartTime = container.getContainerStartTime();
entity.addEvent(tEvent);
entity.setCreatedTime(event.getTimestamp());
entity.setCreatedTime(containerStartTime);
entity.setIdPrefix(TimelineServiceHelper.invertLong(containerStartTime));
dispatcher.getEventHandler().handle(new TimelinePublishEvent(entity,
containerId.getApplicationAttemptId().getApplicationId()));
}
@SuppressWarnings("unchecked")
private void publishContainerFinishedEvent(ContainerStatus containerStatus,
long timeStamp) {
long containerFinishTime, long containerStartTime) {
ContainerId containerId = containerStatus.getContainerId();
TimelineEntity entity = createContainerEntity(containerId);
@ -215,13 +228,14 @@ public class NMTimelinePublisher extends CompositeService {
entityInfo.put(ContainerMetricsConstants.STATE_INFO,
ContainerState.COMPLETE.toString());
entityInfo.put(ContainerMetricsConstants.CONTAINER_FINISHED_TIME,
timeStamp);
containerFinishTime);
entity.setInfo(entityInfo);
TimelineEvent tEvent = new TimelineEvent();
tEvent.setId(ContainerMetricsConstants.FINISHED_EVENT_TYPE);
tEvent.setTimestamp(timeStamp);
tEvent.setTimestamp(containerFinishTime);
entity.addEvent(tEvent);
entity.setIdPrefix(TimelineServiceHelper.invertLong(containerStartTime));
dispatcher.getEventHandler().handle(new TimelinePublishEvent(entity,
containerId.getApplicationAttemptId().getApplicationId()));
@ -237,6 +251,8 @@ public class NMTimelinePublisher extends CompositeService {
tEvent.setId(eventType);
tEvent.setTimestamp(event.getTimestamp());
entity.addEvent(tEvent);
entity.setIdPrefix(TimelineServiceHelper.
invertLong(container.getContainerStartTime()));
ApplicationId appId =
container.getContainerId().getApplicationAttemptId().getApplicationId();
@ -300,7 +316,7 @@ public class NMTimelinePublisher extends CompositeService {
ApplicationContainerFinishedEvent evnt =
(ApplicationContainerFinishedEvent) event;
publishContainerFinishedEvent(evnt.getContainerStatus(),
event.getTimestamp());
event.getTimestamp(), evnt.getContainerStartTime());
break;
default:
@ -391,11 +407,23 @@ public class NMTimelinePublisher extends CompositeService {
public void createTimelineClient(ApplicationId appId) {
if (!appToClientMap.containsKey(appId)) {
TimelineV2Client timelineClient =
TimelineV2Client.createTimelineClient(appId);
timelineClient.init(getConfig());
timelineClient.start();
appToClientMap.put(appId, timelineClient);
try {
TimelineV2Client timelineClient =
nmLoginUGI.doAs(new PrivilegedExceptionAction<TimelineV2Client>() {
@Override
public TimelineV2Client run() throws Exception {
TimelineV2Client timelineClient =
TimelineV2Client.createTimelineClient(appId);
timelineClient.init(getConfig());
timelineClient.start();
return timelineClient;
}
});
appToClientMap.put(appId, timelineClient);
} catch (IOException | InterruptedException | RuntimeException |
Error e) {
LOG.warn("Unable to create timeline client for app " + appId, e);
}
}
}
@ -410,11 +438,11 @@ public class NMTimelinePublisher extends CompositeService {
String collectorAddr) {
TimelineV2Client client = appToClientMap.get(appId);
if (client != null) {
client.setTimelineServiceAddress(collectorAddr);
client.setTimelineCollectorInfo(CollectorInfo.newInstance(collectorAddr));
}
}
private TimelineV2Client getTimelineClient(ApplicationId appId) {
return appToClientMap.get(appId);
}
}
}

View File

@ -59,6 +59,7 @@ import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
import org.apache.hadoop.yarn.server.nodemanager.Context;
@ -659,7 +660,13 @@ public abstract class BaseAMRMProxyTest {
}
@Override
public Map<ApplicationId, String> getRegisteredCollectors() {
public ConcurrentMap<ApplicationId, AppCollectorData>
getRegisteringCollectors() {
return null;
}
@Override
public ConcurrentMap<ApplicationId, AppCollectorData> getKnownCollectors() {
return null;
}

View File

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

View File

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

View File

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

View File

@ -235,4 +235,8 @@ public class MockContainer implements Container {
public boolean isRecovering() {
return false;
}
public long getContainerStartTime() {
return 0;
}
}

View File

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

View File

@ -45,8 +45,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.Container;
@ -61,6 +59,7 @@ import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
@ -254,7 +253,7 @@ public class ApplicationMasterService extends AbstractService implements
// Remove collector address when app get finished.
if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) {
rmApp.removeCollectorAddr();
((RMAppImpl) rmApp).removeCollectorData();
}
// checking whether the app exits in RMStateStore at first not to throw
// ApplicationDoesNotExistInCacheException before and after

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterReque
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.CollectorInfo;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
@ -293,9 +294,10 @@ final class DefaultAMSProcessor implements ApplicationMasterServiceProcessor {
// add collector address for this application
if (YarnConfiguration.timelineServiceV2Enabled(
getRmContext().getYarnConfiguration())) {
response.setCollectorAddr(
getRmContext().getRMApps().get(appAttemptId.getApplicationId())
.getCollectorAddr());
CollectorInfo collectorInfo = app.getCollectorInfo();
if (collectorInfo != null) {
response.setCollectorInfo(collectorInfo);
}
}
// add preemption to the allocateResponse message (if any)

View File

@ -27,6 +27,7 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
@ -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.UnRegisterNodeManagerRequest;
import org.apache.hadoop.yarn.server.api.protocolrecords.UnRegisterNodeManagerResponse;
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.server.api.records.NodeAction;
import org.apache.hadoop.yarn.server.api.records.NodeStatus;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NodeLabelsUtils;
import org.apache.hadoop.yarn.server.resourcemanager.resource.DynamicResourceConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
@ -118,6 +121,8 @@ public class ResourceTrackerService extends AbstractService implements
private boolean isDelegatedCentralizedNodeLabelsConf;
private DynamicResourceConfiguration drConf;
private final AtomicLong timelineCollectorVersion = new AtomicLong(0);
public ResourceTrackerService(RMContext rmContext,
NodesListManager nodesListManager,
NMLivelinessMonitor nmLivelinessMonitor,
@ -521,16 +526,6 @@ public class ResourceTrackerService extends AbstractService implements
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.
if (rmNode.getState() == NodeState.DECOMMISSIONING &&
decommissioningWatcher.checkReadyToBeDecommissioned(
@ -545,6 +540,13 @@ public class ResourceTrackerService extends AbstractService implements
NodeAction.SHUTDOWN, message);
}
boolean timelineV2Enabled =
YarnConfiguration.timelineServiceV2Enabled(getConfig());
if (timelineV2Enabled) {
// Check & update collectors info from request.
updateAppCollectorsMap(request);
}
// Heartbeat response
NodeHeartbeatResponse nodeHeartBeatResponse = YarnServerBuilderUtils
.newNodeHeartbeatResponse(lastNodeHeartbeatResponse.
@ -613,44 +615,66 @@ public class ResourceTrackerService extends AbstractService implements
private void setAppCollectorsMapToResponse(
List<ApplicationId> runningApps, NodeHeartbeatResponse response) {
Map<ApplicationId, String> liveAppCollectorsMap = new
HashMap<ApplicationId, String>();
Map<ApplicationId, AppCollectorData> liveAppCollectorsMap = new
HashMap<>();
Map<ApplicationId, RMApp> rmApps = rmContext.getRMApps();
// Set collectors for all running apps on this node.
for (ApplicationId appId : runningApps) {
String appCollectorAddr = rmApps.get(appId).getCollectorAddr();
if (appCollectorAddr != null) {
liveAppCollectorsMap.put(appId, appCollectorAddr);
} else {
if (LOG.isDebugEnabled()) {
LOG.debug("Collector for applicaton: " + appId +
" hasn't registered yet!");
RMApp app = rmApps.get(appId);
if (app != null) {
AppCollectorData appCollectorData = rmApps.get(appId)
.getCollectorData();
if (appCollectorData != null) {
liveAppCollectorsMap.put(appId, appCollectorData);
} else {
if (LOG.isDebugEnabled()) {
LOG.debug("Collector for applicaton: " + appId +
" hasn't registered yet!");
}
}
}
}
response.setAppCollectorsMap(liveAppCollectorsMap);
response.setAppCollectors(liveAppCollectorsMap);
}
private void updateAppCollectorsMap(NodeHeartbeatRequest request) {
Map<ApplicationId, String> registeredCollectorsMap =
request.getRegisteredCollectors();
if (registeredCollectorsMap != null
&& !registeredCollectorsMap.isEmpty()) {
Map<ApplicationId, AppCollectorData> registeringCollectorsMap =
request.getRegisteringCollectors();
if (registeringCollectorsMap != null
&& !registeringCollectorsMap.isEmpty()) {
Map<ApplicationId, RMApp> rmApps = rmContext.getRMApps();
for (Map.Entry<ApplicationId, String> entry:
registeredCollectorsMap.entrySet()) {
for (Map.Entry<ApplicationId, AppCollectorData> entry:
registeringCollectorsMap.entrySet()) {
ApplicationId appId = entry.getKey();
String collectorAddr = entry.getValue();
if (collectorAddr != null && !collectorAddr.isEmpty()) {
AppCollectorData collectorData = entry.getValue();
if (collectorData != null) {
if (!collectorData.isStamped()) {
// Stamp the collector if we have not done so
collectorData.setRMIdentifier(
ResourceManager.getClusterTimeStamp());
collectorData.setVersion(
timelineCollectorVersion.getAndIncrement());
}
RMApp rmApp = rmApps.get(appId);
if (rmApp == null) {
LOG.warn("Cannot update collector info because application ID: " +
appId + " is not found in RMContext!");
} else {
String previousCollectorAddr = rmApp.getCollectorAddr();
if (previousCollectorAddr == null
|| !previousCollectorAddr.equals(collectorAddr)) {
rmApp.setCollectorAddr(collectorAddr);
synchronized (rmApp) {
AppCollectorData previousCollectorData = rmApp.getCollectorData();
if (AppCollectorData.happensBefore(previousCollectorData,
collectorData)) {
// Sending collector update event.
// Note: RM has to store the newly received collector data
// synchronously. Otherwise, the RM may send out stale collector
// data before this update is done, and the RM then crashes, the
// newly updated collector data will get lost.
LOG.info("Update collector information for application " + appId
+ " with new address: " + collectorData.getCollectorAddr()
+ " timestamp: " + collectorData.getRMIdentifier()
+ ", " + collectorData.getVersion());
((RMAppImpl) rmApp).setCollectorData(collectorData);
}
}
}
}

View File

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

View File

@ -23,6 +23,8 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.ipc.CallerContext;
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@ -30,6 +32,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
import org.apache.hadoop.yarn.api.records.CollectorInfo;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
import org.apache.hadoop.yarn.api.records.NodeId;
@ -39,6 +42,7 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
@ -180,28 +184,27 @@ public interface RMApp extends EventHandler<RMAppEvent> {
String getTrackingUrl();
/**
* The collector address for the application. It should be used only if the
* timeline service v.2 is enabled.
* The timeline collector information for the application. It should be used
* only if the timeline service v.2 is enabled.
*
* @return the address for the application's collector, or null if the
* timeline service v.2 is not enabled.
* @return the data for the application's collector, including collector
* address, RM ID, version and collector token. Return null if the timeline
* service v.2 is not enabled.
*/
String getCollectorAddr();
@InterfaceAudience.Private
@InterfaceStability.Unstable
AppCollectorData getCollectorData();
/**
* Set collector address for the application. It should be used only if the
* timeline service v.2 is enabled.
* The timeline collector information to be sent to AM. It should be used
* 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);
/**
* Remove collector address when application is finished or killed. It should
* be used only if the timeline service v.2 is enabled.
*/
void removeCollectorAddr();
@InterfaceAudience.Private
@InterfaceStability.Unstable
CollectorInfo getCollectorInfo();
/**
* The original tracking url for the application master.
* @return the original tracking url for the application master.

View File

@ -30,9 +30,6 @@ public enum RMAppEventType {
// Source: Scheduler
APP_ACCEPTED,
// TODO add source later
COLLECTOR_UPDATE,
// Source: RMAppAttempt
ATTEMPT_REGISTERED,
ATTEMPT_UNREGISTERED,

View File

@ -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.ApplicationTimeout;
import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
import org.apache.hadoop.yarn.api.records.CollectorInfo;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
import org.apache.hadoop.yarn.api.records.NodeId;
@ -72,6 +73,7 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier;
import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService;
import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEventType;
@ -165,7 +167,8 @@ public class RMAppImpl implements RMApp, Recoverable {
private long storedFinishTime = 0;
private int firstAttemptIdInStateStore = 1;
private int nextAttemptId = 1;
private volatile String collectorAddr;
private AppCollectorData collectorData;
private CollectorInfo collectorInfo;
// This field isn't protected by readlock now.
private volatile RMAppAttempt currentAttempt;
private String queue;
@ -529,7 +532,7 @@ public class RMAppImpl implements RMApp, Recoverable {
*/
public void startTimelineCollector() {
AppLevelTimelineCollector collector =
new AppLevelTimelineCollector(applicationId);
new AppLevelTimelineCollector(applicationId, user);
rmContext.getRMTimelineCollectorManager().putIfAbsent(
applicationId, collector);
}
@ -611,18 +614,22 @@ public class RMAppImpl implements RMApp, Recoverable {
}
@Override
public String getCollectorAddr() {
return this.collectorAddr;
public AppCollectorData getCollectorData() {
return this.collectorData;
}
@Override
public void setCollectorAddr(String collectorAddress) {
this.collectorAddr = collectorAddress;
public void setCollectorData(AppCollectorData incomingData) {
this.collectorData = incomingData;
this.collectorInfo = CollectorInfo.newInstance(
incomingData.getCollectorAddr(), incomingData.getCollectorToken());
}
@Override
public void removeCollectorAddr() {
this.collectorAddr = null;
public CollectorInfo getCollectorInfo() {
return this.collectorInfo;
}
public void removeCollectorData() {
this.collectorData = null;
}
@Override

View File

@ -24,6 +24,7 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@ -39,6 +40,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
import org.apache.hadoop.yarn.server.api.records.NodeStatus;
@ -60,6 +62,8 @@ public class MockNM {
private String version;
private Map<ContainerId, ContainerStatus> containerStats =
new HashMap<ContainerId, ContainerStatus>();
private Map<ApplicationId, AppCollectorData> registeringCollectors
= new ConcurrentHashMap<>();
public MockNM(String nodeIdStr, int memory, ResourceTrackerService resourceTracker) {
// scale vcores based on the requested memory
@ -117,6 +121,15 @@ public class MockNM {
true, ++responseId);
}
public void addRegisteringCollector(ApplicationId appId,
AppCollectorData data) {
this.registeringCollectors.put(appId, data);
}
public Map<ApplicationId, AppCollectorData> getRegisteringCollectors() {
return this.registeringCollectors;
}
public RegisterNodeManagerResponse registerNode() throws Exception {
return registerNode(null, null);
}
@ -229,6 +242,9 @@ public class MockNM {
req.setNodeStatus(status);
req.setLastKnownContainerTokenMasterKey(this.currentContainerTokenMasterKey);
req.setLastKnownNMTokenMasterKey(this.currentNMTokenMasterKey);
req.setRegisteringCollectors(this.registeringCollectors);
NodeHeartbeatResponse heartbeatResponse =
resourceTracker.nodeHeartbeat(req);

View File

@ -0,0 +1,126 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.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());
}
}

View File

@ -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.RegisterNodeManagerResponse;
import org.apache.hadoop.yarn.server.api.protocolrecords.UnRegisterNodeManagerRequest;
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
import org.apache.hadoop.yarn.server.api.records.NodeAction;
import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
import org.apache.hadoop.yarn.server.api.records.NodeStatus;
@ -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.RMNodeLabelsManager;
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.attempt.RMAppAttemptImpl;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
@ -1011,13 +1013,23 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
RMNodeImpl node2 =
(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";
app1.setCollectorAddr(collectorAddr1);
app1.setCollectorData(AppCollectorData.newInstance(
app1.getApplicationId(), collectorAddr1));
String collectorAddr2 = "5.4.3.2:1";
RMApp app2 = rm.submitApp(1024);
app2.setCollectorAddr(collectorAddr2);
RMAppImpl app2 = (RMAppImpl) rm.submitApp(1024);
app2.setCollectorData(AppCollectorData.newInstance(
app2.getApplicationId(), collectorAddr2));
String collectorAddr3 = "5.4.3.2:2";
app2.setCollectorData(AppCollectorData.newInstance(
app2.getApplicationId(), collectorAddr3, 0, 1));
String collectorAddr4 = "5.4.3.2:3";
app2.setCollectorData(AppCollectorData.newInstance(
app2.getApplicationId(), collectorAddr4, 1, 0));
// Create a running container for app1 running on nm1
ContainerId runningContainerId1 = BuilderUtils.newContainerId(
@ -1055,14 +1067,18 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
Assert.assertEquals(app2.getApplicationId(), node2.getRunningApps().get(0));
nodeHeartbeat1 = nm1.nodeHeartbeat(true);
Map<ApplicationId, String> map1 = nodeHeartbeat1.getAppCollectorsMap();
Map<ApplicationId, AppCollectorData> map1
= nodeHeartbeat1.getAppCollectors();
Assert.assertEquals(1, map1.size());
Assert.assertEquals(collectorAddr1, map1.get(app1.getApplicationId()));
Assert.assertEquals(collectorAddr1,
map1.get(app1.getApplicationId()).getCollectorAddr());
nodeHeartbeat2 = nm2.nodeHeartbeat(true);
Map<ApplicationId, String> map2 = nodeHeartbeat2.getAppCollectorsMap();
Map<ApplicationId, AppCollectorData> map2
= nodeHeartbeat2.getAppCollectors();
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)

View File

@ -31,6 +31,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
import org.apache.hadoop.yarn.api.records.CollectorInfo;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
@ -43,6 +44,7 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics;
@ -97,15 +99,7 @@ public abstract class MockAsm extends MockApps {
throw new UnsupportedOperationException("Not supported yet.");
}
@Override
public String getCollectorAddr() {
throw new UnsupportedOperationException("Not supported yet.");
}
@Override
public void setCollectorAddr(String collectorAddr) {
throw new UnsupportedOperationException("Not supported yet.");
}
@Override
public void removeCollectorAddr() {
public AppCollectorData getCollectorData() {
throw new UnsupportedOperationException("Not supported yet.");
}
@Override
@ -246,6 +240,11 @@ public abstract class MockAsm extends MockApps {
public boolean isAppInCompletedStates() {
throw new UnsupportedOperationException("Not supported yet.");
}
@Override
public CollectorInfo getCollectorInfo() {
throw new UnsupportedOperationException("Not supported yet.");
}
}
public static RMApp newApplication(int i) {

View File

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

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
import org.apache.hadoop.yarn.api.records.CollectorInfo;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
import org.apache.hadoop.yarn.api.records.NodeId;
@ -43,6 +44,7 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
@ -305,17 +307,8 @@ public class MockRMApp implements RMApp {
throw new UnsupportedOperationException("Not supported yet.");
}
public String getCollectorAddr() {
throw new UnsupportedOperationException("Not supported yet.");
}
@Override
public void removeCollectorAddr() {
throw new UnsupportedOperationException("Not supported yet.");
}
@Override
public void setCollectorAddr(String collectorAddr) {
public AppCollectorData getCollectorData() {
throw new UnsupportedOperationException("Not supported yet.");
}
@ -333,4 +326,9 @@ public class MockRMApp implements RMApp {
public boolean isAppInCompletedStates() {
return false;
}
@Override
public CollectorInfo getCollectorInfo() {
throw new UnsupportedOperationException("Not supported yet.");
}
}

View File

@ -125,6 +125,17 @@
<groupId>commons-logging</groupId>
<artifactId>commons-logging</artifactId>
</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>
<build>

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