|
@@ -54,6 +54,7 @@ import org.apache.hadoop.mapreduce.TypeConverter;
|
|
|
import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.AppContext;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.MRAppMaster.RunningAppContext;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.Job;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal;
|
|
|
import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
|
|
@@ -65,6 +66,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
|
|
|
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
|
|
|
+import org.apache.hadoop.yarn.client.api.TimelineClient;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
|
import org.apache.hadoop.yarn.server.MiniYARNCluster;
|
|
@@ -486,7 +488,7 @@ public class TestJobHistoryEventHandler {
|
|
|
// stored to the Timeline store
|
|
|
@Test (timeout=50000)
|
|
|
public void testTimelineEventHandling() throws Exception {
|
|
|
- TestParams t = new TestParams(false);
|
|
|
+ TestParams t = new TestParams(RunningAppContext.class, false);
|
|
|
Configuration conf = new YarnConfiguration();
|
|
|
conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
|
|
|
MiniYARNCluster yarnCluster = null;
|
|
@@ -741,21 +743,30 @@ public class TestJobHistoryEventHandler {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private AppContext mockAppContext(ApplicationId appId, boolean isLastAMRetry) {
|
|
|
- JobId jobId = TypeConverter.toYarn(TypeConverter.fromYarn(appId));
|
|
|
- AppContext mockContext = mock(AppContext.class);
|
|
|
+ private Job mockJob() {
|
|
|
Job mockJob = mock(Job.class);
|
|
|
when(mockJob.getAllCounters()).thenReturn(new Counters());
|
|
|
when(mockJob.getTotalMaps()).thenReturn(10);
|
|
|
when(mockJob.getTotalReduces()).thenReturn(10);
|
|
|
when(mockJob.getName()).thenReturn("mockjob");
|
|
|
+ return mockJob;
|
|
|
+ }
|
|
|
+
|
|
|
+ private AppContext mockAppContext(Class<? extends AppContext> contextClass,
|
|
|
+ ApplicationId appId, boolean isLastAMRetry) {
|
|
|
+ JobId jobId = TypeConverter.toYarn(TypeConverter.fromYarn(appId));
|
|
|
+ AppContext mockContext = mock(contextClass);
|
|
|
+ Job mockJob = mockJob();
|
|
|
when(mockContext.getJob(jobId)).thenReturn(mockJob);
|
|
|
when(mockContext.getApplicationID()).thenReturn(appId);
|
|
|
when(mockContext.isLastAMRetry()).thenReturn(isLastAMRetry);
|
|
|
+ if (mockContext instanceof RunningAppContext) {
|
|
|
+ when(((RunningAppContext)mockContext).getTimelineClient()).
|
|
|
+ thenReturn(TimelineClient.createTimelineClient());
|
|
|
+ }
|
|
|
return mockContext;
|
|
|
}
|
|
|
|
|
|
-
|
|
|
private class TestParams {
|
|
|
boolean isLastAMRetry;
|
|
|
String workDir = setupTestWorkDir();
|
|
@@ -770,11 +781,14 @@ public class TestJobHistoryEventHandler {
|
|
|
AppContext mockAppContext;
|
|
|
|
|
|
public TestParams() {
|
|
|
- this(false);
|
|
|
+ this(AppContext.class, false);
|
|
|
}
|
|
|
public TestParams(boolean isLastAMRetry) {
|
|
|
+ this(AppContext.class, isLastAMRetry);
|
|
|
+ }
|
|
|
+ public TestParams(Class<? extends AppContext> contextClass, boolean isLastAMRetry) {
|
|
|
this.isLastAMRetry = isLastAMRetry;
|
|
|
- mockAppContext = mockAppContext(appId, this.isLastAMRetry);
|
|
|
+ mockAppContext = mockAppContext(contextClass, appId, this.isLastAMRetry);
|
|
|
}
|
|
|
}
|
|
|
|