|
@@ -19,46 +19,51 @@
|
|
|
package org.apache.hadoop.mapreduce.v2.app.job.impl;
|
|
|
|
|
|
import static org.mockito.Matchers.any;
|
|
|
-import static org.mockito.Matchers.eq;
|
|
|
-import static org.mockito.Mockito.doNothing;
|
|
|
import static org.mockito.Mockito.doThrow;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
-import static org.mockito.Mockito.verify;
|
|
|
import static org.mockito.Mockito.when;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.util.EnumSet;
|
|
|
-import java.util.HashMap;
|
|
|
-import java.util.Map;
|
|
|
+import java.util.concurrent.BrokenBarrierException;
|
|
|
+import java.util.concurrent.CyclicBarrier;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.mapreduce.JobACL;
|
|
|
import org.apache.hadoop.mapreduce.JobContext;
|
|
|
import org.apache.hadoop.mapreduce.JobID;
|
|
|
+import org.apache.hadoop.mapreduce.JobStatus.State;
|
|
|
import org.apache.hadoop.mapreduce.MRConfig;
|
|
|
import org.apache.hadoop.mapreduce.MRJobConfig;
|
|
|
import org.apache.hadoop.mapreduce.OutputCommitter;
|
|
|
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
|
|
import org.apache.hadoop.mapreduce.TypeConverter;
|
|
|
-import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
|
|
|
import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
|
|
|
import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.JobState;
|
|
|
-import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
|
|
|
+import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
|
|
|
+import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.AppContext;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.commit.CommitterEventHandler;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.commit.CommitterEventType;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal;
|
|
|
-import org.apache.hadoop.mapreduce.v2.app.job.Task;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.JobDiagnosticsUpdateEvent;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.JobEvent;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.JobEventType;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.job.event.JobFinishEvent;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.job.event.JobTaskEvent;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl.InitTransition;
|
|
|
-import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl.JobNoTasksCompletedTransition;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.metrics.MRAppMetrics;
|
|
|
+import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
|
|
|
import org.apache.hadoop.security.Credentials;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.yarn.SystemClock;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
|
|
+import org.apache.hadoop.yarn.event.Dispatcher;
|
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.resourcetracker.InlineDispatcher;
|
|
|
import org.apache.hadoop.yarn.state.StateMachine;
|
|
|
import org.apache.hadoop.yarn.state.StateMachineFactory;
|
|
|
import org.apache.hadoop.yarn.util.Records;
|
|
@@ -69,121 +74,223 @@ import org.junit.Test;
|
|
|
/**
|
|
|
* Tests various functions of the JobImpl class
|
|
|
*/
|
|
|
-@SuppressWarnings({"unchecked", "rawtypes"})
|
|
|
+@SuppressWarnings({"rawtypes"})
|
|
|
public class TestJobImpl {
|
|
|
|
|
|
@Test
|
|
|
- public void testJobNoTasksTransition() {
|
|
|
- JobNoTasksCompletedTransition trans = new JobNoTasksCompletedTransition();
|
|
|
- JobImpl mockJob = mock(JobImpl.class);
|
|
|
+ public void testJobNoTasks() {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ conf.setInt(MRJobConfig.NUM_REDUCES, 0);
|
|
|
+ AsyncDispatcher dispatcher = new AsyncDispatcher();
|
|
|
+ dispatcher.init(conf);
|
|
|
+ dispatcher.start();
|
|
|
+ OutputCommitter committer = mock(OutputCommitter.class);
|
|
|
+ CommitterEventHandler commitHandler =
|
|
|
+ createCommitterEventHandler(dispatcher, committer);
|
|
|
+ commitHandler.init(conf);
|
|
|
+ commitHandler.start();
|
|
|
+
|
|
|
+ JobImpl job = createStubbedJob(conf, dispatcher, 0);
|
|
|
+ job.handle(new JobEvent(job.getID(), JobEventType.JOB_INIT));
|
|
|
+ assertJobState(job, JobStateInternal.INITED);
|
|
|
+ job.handle(new JobEvent(job.getID(), JobEventType.JOB_START));
|
|
|
+ assertJobState(job, JobStateInternal.SUCCEEDED);
|
|
|
+ dispatcher.stop();
|
|
|
+ commitHandler.stop();
|
|
|
+ }
|
|
|
|
|
|
- // Force checkJobCompleteSuccess to return null
|
|
|
- Task mockTask = mock(Task.class);
|
|
|
- Map<TaskId, Task> tasks = new HashMap<TaskId, Task>();
|
|
|
- tasks.put(mockTask.getID(), mockTask);
|
|
|
- mockJob.tasks = tasks;
|
|
|
+ @Test(timeout=20000)
|
|
|
+ public void testCommitJobFailsJob() throws Exception {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ AsyncDispatcher dispatcher = new AsyncDispatcher();
|
|
|
+ dispatcher.init(conf);
|
|
|
+ dispatcher.start();
|
|
|
+ CyclicBarrier syncBarrier = new CyclicBarrier(2);
|
|
|
+ OutputCommitter committer = new TestingOutputCommitter(syncBarrier, false);
|
|
|
+ CommitterEventHandler commitHandler =
|
|
|
+ createCommitterEventHandler(dispatcher, committer);
|
|
|
+ commitHandler.init(conf);
|
|
|
+ commitHandler.start();
|
|
|
+
|
|
|
+ JobImpl job = createRunningStubbedJob(conf, dispatcher, 2);
|
|
|
+ completeJobTasks(job);
|
|
|
+ assertJobState(job, JobStateInternal.COMMITTING);
|
|
|
+
|
|
|
+ // let the committer fail and verify the job fails
|
|
|
+ syncBarrier.await();
|
|
|
+ assertJobState(job, JobStateInternal.FAILED);
|
|
|
+ dispatcher.stop();
|
|
|
+ commitHandler.stop();
|
|
|
+ }
|
|
|
|
|
|
- when(mockJob.getInternalState()).thenReturn(JobStateInternal.ERROR);
|
|
|
- JobEvent mockJobEvent = mock(JobEvent.class);
|
|
|
- JobStateInternal state = trans.transition(mockJob, mockJobEvent);
|
|
|
- Assert.assertEquals("Incorrect state returned from JobNoTasksCompletedTransition",
|
|
|
- JobStateInternal.ERROR, state);
|
|
|
+ @Test(timeout=20000)
|
|
|
+ public void testCheckJobCompleteSuccess() throws Exception {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ AsyncDispatcher dispatcher = new AsyncDispatcher();
|
|
|
+ dispatcher.init(conf);
|
|
|
+ dispatcher.start();
|
|
|
+ CyclicBarrier syncBarrier = new CyclicBarrier(2);
|
|
|
+ OutputCommitter committer = new TestingOutputCommitter(syncBarrier, true);
|
|
|
+ CommitterEventHandler commitHandler =
|
|
|
+ createCommitterEventHandler(dispatcher, committer);
|
|
|
+ commitHandler.init(conf);
|
|
|
+ commitHandler.start();
|
|
|
+
|
|
|
+ JobImpl job = createRunningStubbedJob(conf, dispatcher, 2);
|
|
|
+ completeJobTasks(job);
|
|
|
+ assertJobState(job, JobStateInternal.COMMITTING);
|
|
|
+
|
|
|
+ // let the committer complete and verify the job succeeds
|
|
|
+ syncBarrier.await();
|
|
|
+ assertJobState(job, JobStateInternal.SUCCEEDED);
|
|
|
+ dispatcher.stop();
|
|
|
+ commitHandler.stop();
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
- public void testCommitJobFailsJob() {
|
|
|
-
|
|
|
- JobImpl mockJob = mock(JobImpl.class);
|
|
|
- mockJob.tasks = new HashMap<TaskId, Task>();
|
|
|
- OutputCommitter mockCommitter = mock(OutputCommitter.class);
|
|
|
- EventHandler mockEventHandler = mock(EventHandler.class);
|
|
|
- JobContext mockJobContext = mock(JobContext.class);
|
|
|
-
|
|
|
- when(mockJob.getCommitter()).thenReturn(mockCommitter);
|
|
|
- when(mockJob.getEventHandler()).thenReturn(mockEventHandler);
|
|
|
- when(mockJob.getJobContext()).thenReturn(mockJobContext);
|
|
|
- when(mockJob.finished(JobStateInternal.KILLED)).thenReturn(
|
|
|
- JobStateInternal.KILLED);
|
|
|
- when(mockJob.finished(JobStateInternal.FAILED)).thenReturn(
|
|
|
- JobStateInternal.FAILED);
|
|
|
- when(mockJob.finished(JobStateInternal.SUCCEEDED)).thenReturn(
|
|
|
- JobStateInternal.SUCCEEDED);
|
|
|
-
|
|
|
- try {
|
|
|
- doThrow(new IOException()).when(mockCommitter).commitJob(any(JobContext.class));
|
|
|
- } catch (IOException e) {
|
|
|
- // commitJob stubbed out, so this can't happen
|
|
|
- }
|
|
|
- doNothing().when(mockEventHandler).handle(any(JobHistoryEvent.class));
|
|
|
- JobStateInternal jobState = JobImpl.checkJobCompleteSuccess(mockJob);
|
|
|
- Assert.assertNotNull("checkJobCompleteSuccess incorrectly returns null " +
|
|
|
- "for successful job", jobState);
|
|
|
- Assert.assertEquals("checkJobCompleteSuccess returns incorrect state",
|
|
|
- JobStateInternal.FAILED, jobState);
|
|
|
- verify(mockJob).abortJob(
|
|
|
- eq(org.apache.hadoop.mapreduce.JobStatus.State.FAILED));
|
|
|
+ @Test(timeout=20000)
|
|
|
+ public void testKilledDuringSetup() throws Exception {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ AsyncDispatcher dispatcher = new AsyncDispatcher();
|
|
|
+ dispatcher.init(conf);
|
|
|
+ dispatcher.start();
|
|
|
+ OutputCommitter committer = new StubbedOutputCommitter() {
|
|
|
+ @Override
|
|
|
+ public synchronized void setupJob(JobContext jobContext)
|
|
|
+ throws IOException {
|
|
|
+ while (!Thread.interrupted()) {
|
|
|
+ try {
|
|
|
+ wait();
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ };
|
|
|
+ CommitterEventHandler commitHandler =
|
|
|
+ createCommitterEventHandler(dispatcher, committer);
|
|
|
+ commitHandler.init(conf);
|
|
|
+ commitHandler.start();
|
|
|
+
|
|
|
+ JobImpl job = createStubbedJob(conf, dispatcher, 2);
|
|
|
+ JobId jobId = job.getID();
|
|
|
+ job.handle(new JobEvent(jobId, JobEventType.JOB_INIT));
|
|
|
+ assertJobState(job, JobStateInternal.INITED);
|
|
|
+ job.handle(new JobEvent(jobId, JobEventType.JOB_START));
|
|
|
+ assertJobState(job, JobStateInternal.SETUP);
|
|
|
+
|
|
|
+ job.handle(new JobEvent(job.getID(), JobEventType.JOB_KILL));
|
|
|
+ assertJobState(job, JobStateInternal.KILLED);
|
|
|
+ dispatcher.stop();
|
|
|
+ commitHandler.stop();
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
- public void testCheckJobCompleteSuccess() {
|
|
|
-
|
|
|
- JobImpl mockJob = mock(JobImpl.class);
|
|
|
- mockJob.tasks = new HashMap<TaskId, Task>();
|
|
|
- OutputCommitter mockCommitter = mock(OutputCommitter.class);
|
|
|
- EventHandler mockEventHandler = mock(EventHandler.class);
|
|
|
- JobContext mockJobContext = mock(JobContext.class);
|
|
|
-
|
|
|
- when(mockJob.getCommitter()).thenReturn(mockCommitter);
|
|
|
- when(mockJob.getEventHandler()).thenReturn(mockEventHandler);
|
|
|
- when(mockJob.getJobContext()).thenReturn(mockJobContext);
|
|
|
- doNothing().when(mockJob).setFinishTime();
|
|
|
- doNothing().when(mockJob).logJobHistoryFinishedEvent();
|
|
|
- when(mockJob.finished(any(JobStateInternal.class))).thenReturn(
|
|
|
- JobStateInternal.SUCCEEDED);
|
|
|
-
|
|
|
- try {
|
|
|
- doNothing().when(mockCommitter).commitJob(any(JobContext.class));
|
|
|
- } catch (IOException e) {
|
|
|
- // commitJob stubbed out, so this can't happen
|
|
|
- }
|
|
|
- doNothing().when(mockEventHandler).handle(any(JobHistoryEvent.class));
|
|
|
- Assert.assertNotNull("checkJobCompleteSuccess incorrectly returns null " +
|
|
|
- "for successful job",
|
|
|
- JobImpl.checkJobCompleteSuccess(mockJob));
|
|
|
- Assert.assertEquals("checkJobCompleteSuccess returns incorrect state",
|
|
|
- JobStateInternal.SUCCEEDED, JobImpl.checkJobCompleteSuccess(mockJob));
|
|
|
+ @Test(timeout=20000)
|
|
|
+ public void testKilledDuringCommit() throws Exception {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ AsyncDispatcher dispatcher = new AsyncDispatcher();
|
|
|
+ dispatcher.init(conf);
|
|
|
+ dispatcher.start();
|
|
|
+ CyclicBarrier syncBarrier = new CyclicBarrier(2);
|
|
|
+ OutputCommitter committer = new WaitingOutputCommitter(syncBarrier, true);
|
|
|
+ CommitterEventHandler commitHandler =
|
|
|
+ createCommitterEventHandler(dispatcher, committer);
|
|
|
+ commitHandler.init(conf);
|
|
|
+ commitHandler.start();
|
|
|
+
|
|
|
+ JobImpl job = createRunningStubbedJob(conf, dispatcher, 2);
|
|
|
+ completeJobTasks(job);
|
|
|
+ assertJobState(job, JobStateInternal.COMMITTING);
|
|
|
+
|
|
|
+ syncBarrier.await();
|
|
|
+ job.handle(new JobEvent(job.getID(), JobEventType.JOB_KILL));
|
|
|
+ assertJobState(job, JobStateInternal.KILLED);
|
|
|
+ dispatcher.stop();
|
|
|
+ commitHandler.stop();
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
- public void testCheckJobCompleteSuccessFailed() {
|
|
|
- JobImpl mockJob = mock(JobImpl.class);
|
|
|
-
|
|
|
- // Make the completedTasks not equal the getTasks()
|
|
|
- Task mockTask = mock(Task.class);
|
|
|
- Map<TaskId, Task> tasks = new HashMap<TaskId, Task>();
|
|
|
- tasks.put(mockTask.getID(), mockTask);
|
|
|
- mockJob.tasks = tasks;
|
|
|
-
|
|
|
- try {
|
|
|
- // Just in case the code breaks and reaches these calls
|
|
|
- OutputCommitter mockCommitter = mock(OutputCommitter.class);
|
|
|
- EventHandler mockEventHandler = mock(EventHandler.class);
|
|
|
- doNothing().when(mockCommitter).commitJob(any(JobContext.class));
|
|
|
- doNothing().when(mockEventHandler).handle(any(JobHistoryEvent.class));
|
|
|
- } catch (IOException e) {
|
|
|
- e.printStackTrace();
|
|
|
- }
|
|
|
- Assert.assertNull("checkJobCompleteSuccess incorrectly returns not-null " +
|
|
|
- "for unsuccessful job",
|
|
|
- JobImpl.checkJobCompleteSuccess(mockJob));
|
|
|
+ @Test(timeout=20000)
|
|
|
+ public void testKilledDuringFailAbort() throws Exception {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ AsyncDispatcher dispatcher = new AsyncDispatcher();
|
|
|
+ dispatcher.init(conf);
|
|
|
+ dispatcher.start();
|
|
|
+ OutputCommitter committer = new StubbedOutputCommitter() {
|
|
|
+ @Override
|
|
|
+ public void setupJob(JobContext jobContext) throws IOException {
|
|
|
+ throw new IOException("forced failure");
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public synchronized void abortJob(JobContext jobContext, State state)
|
|
|
+ throws IOException {
|
|
|
+ while (!Thread.interrupted()) {
|
|
|
+ try {
|
|
|
+ wait();
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ };
|
|
|
+ CommitterEventHandler commitHandler =
|
|
|
+ createCommitterEventHandler(dispatcher, committer);
|
|
|
+ commitHandler.init(conf);
|
|
|
+ commitHandler.start();
|
|
|
+
|
|
|
+ JobImpl job = createStubbedJob(conf, dispatcher, 2);
|
|
|
+ JobId jobId = job.getID();
|
|
|
+ job.handle(new JobEvent(jobId, JobEventType.JOB_INIT));
|
|
|
+ assertJobState(job, JobStateInternal.INITED);
|
|
|
+ job.handle(new JobEvent(jobId, JobEventType.JOB_START));
|
|
|
+ assertJobState(job, JobStateInternal.FAIL_ABORT);
|
|
|
+
|
|
|
+ job.handle(new JobEvent(jobId, JobEventType.JOB_KILL));
|
|
|
+ assertJobState(job, JobStateInternal.KILLED);
|
|
|
+ dispatcher.stop();
|
|
|
+ commitHandler.stop();
|
|
|
}
|
|
|
|
|
|
+ @Test(timeout=20000)
|
|
|
+ public void testKilledDuringKillAbort() throws Exception {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ AsyncDispatcher dispatcher = new AsyncDispatcher();
|
|
|
+ dispatcher.init(conf);
|
|
|
+ dispatcher.start();
|
|
|
+ OutputCommitter committer = new StubbedOutputCommitter() {
|
|
|
+ @Override
|
|
|
+ public synchronized void abortJob(JobContext jobContext, State state)
|
|
|
+ throws IOException {
|
|
|
+ while (!Thread.interrupted()) {
|
|
|
+ try {
|
|
|
+ wait();
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ };
|
|
|
+ CommitterEventHandler commitHandler =
|
|
|
+ createCommitterEventHandler(dispatcher, committer);
|
|
|
+ commitHandler.init(conf);
|
|
|
+ commitHandler.start();
|
|
|
+
|
|
|
+ JobImpl job = createStubbedJob(conf, dispatcher, 2);
|
|
|
+ JobId jobId = job.getID();
|
|
|
+ job.handle(new JobEvent(jobId, JobEventType.JOB_INIT));
|
|
|
+ assertJobState(job, JobStateInternal.INITED);
|
|
|
+ job.handle(new JobEvent(jobId, JobEventType.JOB_START));
|
|
|
+ assertJobState(job, JobStateInternal.SETUP);
|
|
|
+
|
|
|
+ job.handle(new JobEvent(jobId, JobEventType.JOB_KILL));
|
|
|
+ assertJobState(job, JobStateInternal.KILL_ABORT);
|
|
|
+
|
|
|
+ job.handle(new JobEvent(jobId, JobEventType.JOB_KILL));
|
|
|
+ assertJobState(job, JobStateInternal.KILLED);
|
|
|
+ dispatcher.stop();
|
|
|
+ commitHandler.stop();
|
|
|
+ }
|
|
|
|
|
|
public static void main(String[] args) throws Exception {
|
|
|
TestJobImpl t = new TestJobImpl();
|
|
|
- t.testJobNoTasksTransition();
|
|
|
+ t.testJobNoTasks();
|
|
|
t.testCheckJobCompleteSuccess();
|
|
|
- t.testCheckJobCompleteSuccessFailed();
|
|
|
t.testCheckAccess();
|
|
|
t.testReportDiagnostics();
|
|
|
t.testUberDecision();
|
|
@@ -208,7 +315,7 @@ public class TestJobImpl {
|
|
|
|
|
|
// Verify access
|
|
|
JobImpl job1 = new JobImpl(jobId, null, conf1, null, null, null, null, null,
|
|
|
- null, null, null, true, null, 0, null, null);
|
|
|
+ null, null, true, null, 0, null, null);
|
|
|
Assert.assertTrue(job1.checkAccess(ugi1, JobACL.VIEW_JOB));
|
|
|
Assert.assertFalse(job1.checkAccess(ugi2, JobACL.VIEW_JOB));
|
|
|
|
|
@@ -219,7 +326,7 @@ public class TestJobImpl {
|
|
|
|
|
|
// Verify access
|
|
|
JobImpl job2 = new JobImpl(jobId, null, conf2, null, null, null, null, null,
|
|
|
- null, null, null, true, null, 0, null, null);
|
|
|
+ null, null, true, null, 0, null, null);
|
|
|
Assert.assertTrue(job2.checkAccess(ugi1, JobACL.VIEW_JOB));
|
|
|
Assert.assertTrue(job2.checkAccess(ugi2, JobACL.VIEW_JOB));
|
|
|
|
|
@@ -230,7 +337,7 @@ public class TestJobImpl {
|
|
|
|
|
|
// Verify access
|
|
|
JobImpl job3 = new JobImpl(jobId, null, conf3, null, null, null, null, null,
|
|
|
- null, null, null, true, null, 0, null, null);
|
|
|
+ null, null, true, null, 0, null, null);
|
|
|
Assert.assertTrue(job3.checkAccess(ugi1, JobACL.VIEW_JOB));
|
|
|
Assert.assertTrue(job3.checkAccess(ugi2, JobACL.VIEW_JOB));
|
|
|
|
|
@@ -241,7 +348,7 @@ public class TestJobImpl {
|
|
|
|
|
|
// Verify access
|
|
|
JobImpl job4 = new JobImpl(jobId, null, conf4, null, null, null, null, null,
|
|
|
- null, null, null, true, null, 0, null, null);
|
|
|
+ null, null, true, null, 0, null, null);
|
|
|
Assert.assertTrue(job4.checkAccess(ugi1, JobACL.VIEW_JOB));
|
|
|
Assert.assertTrue(job4.checkAccess(ugi2, JobACL.VIEW_JOB));
|
|
|
|
|
@@ -252,7 +359,7 @@ public class TestJobImpl {
|
|
|
|
|
|
// Verify access
|
|
|
JobImpl job5 = new JobImpl(jobId, null, conf5, null, null, null, null, null,
|
|
|
- null, null, null, true, null, 0, null, null);
|
|
|
+ null, null, true, null, 0, null, null);
|
|
|
Assert.assertTrue(job5.checkAccess(ugi1, null));
|
|
|
Assert.assertTrue(job5.checkAccess(ugi2, null));
|
|
|
}
|
|
@@ -270,8 +377,7 @@ public class TestJobImpl {
|
|
|
mock(EventHandler.class),
|
|
|
null, mock(JobTokenSecretManager.class), null,
|
|
|
new SystemClock(), null,
|
|
|
- mrAppMetrics, mock(OutputCommitter.class),
|
|
|
- true, null, 0, null, null);
|
|
|
+ mrAppMetrics, true, null, 0, null, null);
|
|
|
job.handle(diagUpdateEvent);
|
|
|
String diagnostics = job.getReport().getDiagnostics();
|
|
|
Assert.assertNotNull(diagnostics);
|
|
@@ -282,8 +388,7 @@ public class TestJobImpl {
|
|
|
mock(EventHandler.class),
|
|
|
null, mock(JobTokenSecretManager.class), null,
|
|
|
new SystemClock(), null,
|
|
|
- mrAppMetrics, mock(OutputCommitter.class),
|
|
|
- true, null, 0, null, null);
|
|
|
+ mrAppMetrics, true, null, 0, null, null);
|
|
|
job.handle(new JobEvent(jobId, JobEventType.JOB_KILL));
|
|
|
job.handle(diagUpdateEvent);
|
|
|
diagnostics = job.getReport().getDiagnostics();
|
|
@@ -338,20 +443,23 @@ public class TestJobImpl {
|
|
|
JobImpl job = new JobImpl(jobId, Records
|
|
|
.newRecord(ApplicationAttemptId.class), conf, mock(EventHandler.class),
|
|
|
null, mock(JobTokenSecretManager.class), null, null, null,
|
|
|
- mrAppMetrics, mock(OutputCommitter.class), true, null, 0, null, null);
|
|
|
- InitTransition initTransition = getInitTransition();
|
|
|
+ mrAppMetrics, true, null, 0, null, null);
|
|
|
+ InitTransition initTransition = getInitTransition(2);
|
|
|
JobEvent mockJobEvent = mock(JobEvent.class);
|
|
|
initTransition.transition(job, mockJobEvent);
|
|
|
boolean isUber = job.isUber();
|
|
|
return isUber;
|
|
|
}
|
|
|
|
|
|
- private static InitTransition getInitTransition() {
|
|
|
+ private static InitTransition getInitTransition(final int numSplits) {
|
|
|
InitTransition initTransition = new InitTransition() {
|
|
|
@Override
|
|
|
protected TaskSplitMetaInfo[] createSplits(JobImpl job, JobId jobId) {
|
|
|
- return new TaskSplitMetaInfo[] { new TaskSplitMetaInfo(),
|
|
|
- new TaskSplitMetaInfo() };
|
|
|
+ TaskSplitMetaInfo[] splits = new TaskSplitMetaInfo[numSplits];
|
|
|
+ for (int i = 0; i < numSplits; ++i) {
|
|
|
+ splits[i] = new TaskSplitMetaInfo();
|
|
|
+ }
|
|
|
+ return splits;
|
|
|
}
|
|
|
};
|
|
|
return initTransition;
|
|
@@ -360,19 +468,24 @@ public class TestJobImpl {
|
|
|
@Test
|
|
|
public void testTransitionsAtFailed() throws IOException {
|
|
|
Configuration conf = new Configuration();
|
|
|
- JobID jobID = JobID.forName("job_1234567890000_0001");
|
|
|
- JobId jobId = TypeConverter.toYarn(jobID);
|
|
|
+ AsyncDispatcher dispatcher = new AsyncDispatcher();
|
|
|
+ dispatcher.init(conf);
|
|
|
+ dispatcher.start();
|
|
|
+
|
|
|
OutputCommitter committer = mock(OutputCommitter.class);
|
|
|
doThrow(new IOException("forcefail"))
|
|
|
.when(committer).setupJob(any(JobContext.class));
|
|
|
- InlineDispatcher dispatcher = new InlineDispatcher();
|
|
|
- JobImpl job = new StubbedJob(jobId, Records
|
|
|
- .newRecord(ApplicationAttemptId.class), conf,
|
|
|
- dispatcher.getEventHandler(), committer, true, null);
|
|
|
+ CommitterEventHandler commitHandler =
|
|
|
+ createCommitterEventHandler(dispatcher, committer);
|
|
|
+ commitHandler.init(conf);
|
|
|
+ commitHandler.start();
|
|
|
|
|
|
- dispatcher.register(JobEventType.class, job);
|
|
|
+ JobImpl job = createStubbedJob(conf, dispatcher, 2);
|
|
|
+ JobId jobId = job.getID();
|
|
|
job.handle(new JobEvent(jobId, JobEventType.JOB_INIT));
|
|
|
- Assert.assertEquals(JobState.FAILED, job.getState());
|
|
|
+ assertJobState(job, JobStateInternal.INITED);
|
|
|
+ job.handle(new JobEvent(jobId, JobEventType.JOB_START));
|
|
|
+ assertJobState(job, JobStateInternal.FAILED);
|
|
|
|
|
|
job.handle(new JobEvent(jobId, JobEventType.JOB_TASK_COMPLETED));
|
|
|
Assert.assertEquals(JobState.FAILED, job.getState());
|
|
@@ -382,17 +495,86 @@ public class TestJobImpl {
|
|
|
Assert.assertEquals(JobState.FAILED, job.getState());
|
|
|
job.handle(new JobEvent(jobId, JobEventType.JOB_TASK_ATTEMPT_FETCH_FAILURE));
|
|
|
Assert.assertEquals(JobState.FAILED, job.getState());
|
|
|
+
|
|
|
+ dispatcher.stop();
|
|
|
+ commitHandler.stop();
|
|
|
+ }
|
|
|
+
|
|
|
+ private static CommitterEventHandler createCommitterEventHandler(
|
|
|
+ Dispatcher dispatcher, OutputCommitter committer) {
|
|
|
+ SystemClock clock = new SystemClock();
|
|
|
+ AppContext appContext = mock(AppContext.class);
|
|
|
+ when(appContext.getEventHandler()).thenReturn(
|
|
|
+ dispatcher.getEventHandler());
|
|
|
+ when(appContext.getClock()).thenReturn(clock);
|
|
|
+ CommitterEventHandler handler =
|
|
|
+ new CommitterEventHandler(appContext, committer);
|
|
|
+ dispatcher.register(CommitterEventType.class, handler);
|
|
|
+ return handler;
|
|
|
+ }
|
|
|
+
|
|
|
+ private static StubbedJob createStubbedJob(Configuration conf,
|
|
|
+ Dispatcher dispatcher, int numSplits) {
|
|
|
+ JobID jobID = JobID.forName("job_1234567890000_0001");
|
|
|
+ JobId jobId = TypeConverter.toYarn(jobID);
|
|
|
+ StubbedJob job = new StubbedJob(jobId,
|
|
|
+ Records.newRecord(ApplicationAttemptId.class), conf,
|
|
|
+ dispatcher.getEventHandler(), true, "somebody", numSplits);
|
|
|
+ dispatcher.register(JobEventType.class, job);
|
|
|
+ EventHandler mockHandler = mock(EventHandler.class);
|
|
|
+ dispatcher.register(TaskEventType.class, mockHandler);
|
|
|
+ dispatcher.register(org.apache.hadoop.mapreduce.jobhistory.EventType.class,
|
|
|
+ mockHandler);
|
|
|
+ dispatcher.register(JobFinishEvent.Type.class, mockHandler);
|
|
|
+ return job;
|
|
|
+ }
|
|
|
+
|
|
|
+ private static StubbedJob createRunningStubbedJob(Configuration conf,
|
|
|
+ Dispatcher dispatcher, int numSplits) {
|
|
|
+ StubbedJob job = createStubbedJob(conf, dispatcher, numSplits);
|
|
|
+ job.handle(new JobEvent(job.getID(), JobEventType.JOB_INIT));
|
|
|
+ assertJobState(job, JobStateInternal.INITED);
|
|
|
+ job.handle(new JobEvent(job.getID(), JobEventType.JOB_START));
|
|
|
+ assertJobState(job, JobStateInternal.RUNNING);
|
|
|
+ return job;
|
|
|
+ }
|
|
|
+
|
|
|
+ private static void completeJobTasks(JobImpl job) {
|
|
|
+ // complete the map tasks and the reduce tasks so we start committing
|
|
|
+ int numMaps = job.getTotalMaps();
|
|
|
+ for (int i = 0; i < numMaps; ++i) {
|
|
|
+ job.handle(new JobTaskEvent(
|
|
|
+ MRBuilderUtils.newTaskId(job.getID(), 1, TaskType.MAP),
|
|
|
+ TaskState.SUCCEEDED));
|
|
|
+ Assert.assertEquals(JobState.RUNNING, job.getState());
|
|
|
+ }
|
|
|
+ int numReduces = job.getTotalReduces();
|
|
|
+ for (int i = 0; i < numReduces; ++i) {
|
|
|
+ job.handle(new JobTaskEvent(
|
|
|
+ MRBuilderUtils.newTaskId(job.getID(), 1, TaskType.MAP),
|
|
|
+ TaskState.SUCCEEDED));
|
|
|
+ Assert.assertEquals(JobState.RUNNING, job.getState());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private static void assertJobState(JobImpl job, JobStateInternal state) {
|
|
|
+ int timeToWaitMsec = 5 * 1000;
|
|
|
+ while (timeToWaitMsec > 0 && job.getInternalState() != state) {
|
|
|
+ try {
|
|
|
+ Thread.sleep(10);
|
|
|
+ timeToWaitMsec -= 10;
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ Assert.assertEquals(state, job.getInternalState());
|
|
|
}
|
|
|
|
|
|
private static class StubbedJob extends JobImpl {
|
|
|
//override the init transition
|
|
|
- private final InitTransition initTransition = getInitTransition();
|
|
|
- StateMachineFactory<JobImpl, JobStateInternal, JobEventType, JobEvent> localFactory
|
|
|
- = stateMachineFactory.addTransition(JobStateInternal.NEW,
|
|
|
- EnumSet.of(JobStateInternal.INITED, JobStateInternal.FAILED),
|
|
|
- JobEventType.JOB_INIT,
|
|
|
- // This is abusive.
|
|
|
- initTransition);
|
|
|
+ private final InitTransition initTransition;
|
|
|
+ StateMachineFactory<JobImpl, JobStateInternal, JobEventType, JobEvent>
|
|
|
+ localFactory;
|
|
|
|
|
|
private final StateMachine<JobStateInternal, JobEventType, JobEvent>
|
|
|
localStateMachine;
|
|
@@ -404,15 +586,102 @@ public class TestJobImpl {
|
|
|
|
|
|
public StubbedJob(JobId jobId, ApplicationAttemptId applicationAttemptId,
|
|
|
Configuration conf, EventHandler eventHandler,
|
|
|
- OutputCommitter committer, boolean newApiCommitter, String user) {
|
|
|
+ boolean newApiCommitter, String user, int numSplits) {
|
|
|
super(jobId, applicationAttemptId, conf, eventHandler,
|
|
|
null, new JobTokenSecretManager(), new Credentials(),
|
|
|
- new SystemClock(), null, MRAppMetrics.create(), committer,
|
|
|
+ new SystemClock(), null, MRAppMetrics.create(),
|
|
|
newApiCommitter, user, System.currentTimeMillis(), null, null);
|
|
|
|
|
|
+ initTransition = getInitTransition(numSplits);
|
|
|
+ localFactory = stateMachineFactory.addTransition(JobStateInternal.NEW,
|
|
|
+ EnumSet.of(JobStateInternal.INITED, JobStateInternal.FAILED),
|
|
|
+ JobEventType.JOB_INIT,
|
|
|
+ // This is abusive.
|
|
|
+ initTransition);
|
|
|
+
|
|
|
// This "this leak" is okay because the retained pointer is in an
|
|
|
// instance variable.
|
|
|
localStateMachine = localFactory.make(this);
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ private static class StubbedOutputCommitter extends OutputCommitter {
|
|
|
+
|
|
|
+ public StubbedOutputCommitter() {
|
|
|
+ super();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void setupJob(JobContext jobContext) throws IOException {
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void setupTask(TaskAttemptContext taskContext) throws IOException {
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public boolean needsTaskCommit(TaskAttemptContext taskContext)
|
|
|
+ throws IOException {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void commitTask(TaskAttemptContext taskContext) throws IOException {
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void abortTask(TaskAttemptContext taskContext) throws IOException {
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private static class TestingOutputCommitter extends StubbedOutputCommitter {
|
|
|
+ CyclicBarrier syncBarrier;
|
|
|
+ boolean shouldSucceed;
|
|
|
+
|
|
|
+ public TestingOutputCommitter(CyclicBarrier syncBarrier,
|
|
|
+ boolean shouldSucceed) {
|
|
|
+ super();
|
|
|
+ this.syncBarrier = syncBarrier;
|
|
|
+ this.shouldSucceed = shouldSucceed;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void commitJob(JobContext jobContext) throws IOException {
|
|
|
+ try {
|
|
|
+ syncBarrier.await();
|
|
|
+ } catch (BrokenBarrierException e) {
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ }
|
|
|
+
|
|
|
+ if (!shouldSucceed) {
|
|
|
+ throw new IOException("forced failure");
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private static class WaitingOutputCommitter extends TestingOutputCommitter {
|
|
|
+ public WaitingOutputCommitter(CyclicBarrier syncBarrier,
|
|
|
+ boolean shouldSucceed) {
|
|
|
+ super(syncBarrier, shouldSucceed);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void commitJob(JobContext jobContext) throws IOException {
|
|
|
+ try {
|
|
|
+ syncBarrier.await();
|
|
|
+ } catch (BrokenBarrierException e) {
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ }
|
|
|
+
|
|
|
+ while (!Thread.interrupted()) {
|
|
|
+ try {
|
|
|
+ synchronized (this) {
|
|
|
+ wait();
|
|
|
+ }
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|