@Test(timeout = 20000) public void testKilledDuringSetup() throws Exception { Configuration conf = new Configuration(); conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir); 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, null); JobId jobId = job.getID(); job.handle(new JobEvent(jobId, JobEventType.JOB_INIT)); assertJobState(job, JobStateInternal.INITED); job.handle(new JobStartEvent(jobId)); assertJobState(job, JobStateInternal.SETUP); job.handle(new JobEvent(job.getID(), JobEventType.JOB_KILL)); assertJobState(job, JobStateInternal.KILLED); dispatcher.stop(); commitHandler.stop(); }
@Test(timeout = 20000) public void testCheckJobCompleteSuccess() throws Exception { Configuration conf = new Configuration(); conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir); 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, null); completeJobTasks(job); assertJobState(job, JobStateInternal.COMMITTING); // let the committer complete and verify the job succeeds syncBarrier.await(); assertJobState(job, JobStateInternal.SUCCEEDED); job.handle(new JobEvent(job.getID(), JobEventType.JOB_TASK_ATTEMPT_COMPLETED)); assertJobState(job, JobStateInternal.SUCCEEDED); job.handle(new JobEvent(job.getID(), JobEventType.JOB_MAP_TASK_RESCHEDULED)); assertJobState(job, JobStateInternal.SUCCEEDED); dispatcher.stop(); commitHandler.stop(); }
@Test public void testAbortJobCalledAfterKillingTasks() throws IOException { Configuration conf = new Configuration(); conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir); conf.set(MRJobConfig.MR_AM_COMMITTER_CANCEL_TIMEOUT_MS, "1000"); InlineDispatcher dispatcher = new InlineDispatcher(); dispatcher.init(conf); dispatcher.start(); OutputCommitter committer = Mockito.mock(OutputCommitter.class); CommitterEventHandler commitHandler = createCommitterEventHandler(dispatcher, committer); commitHandler.init(conf); commitHandler.start(); JobImpl job = createRunningStubbedJob(conf, dispatcher, 2, null); // Fail one task. This should land the JobImpl in the FAIL_WAIT state job.handle( new JobTaskEvent(MRBuilderUtils.newTaskId(job.getID(), 1, TaskType.MAP), TaskState.FAILED)); // Verify abort job hasn't been called Mockito.verify(committer, Mockito.never()) .abortJob((JobContext) Mockito.any(), (State) Mockito.any()); assertJobState(job, JobStateInternal.FAIL_WAIT); // Verify abortJob is called once and the job failed Mockito.verify(committer, Mockito.timeout(2000).times(1)) .abortJob((JobContext) Mockito.any(), (State) Mockito.any()); assertJobState(job, JobStateInternal.FAILED); dispatcher.stop(); }
@Test(timeout = 20000) public void testRebootedDuringCommit() throws Exception { Configuration conf = new Configuration(); conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir); conf.setInt(MRJobConfig.MR_AM_MAX_ATTEMPTS, 2); 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(); AppContext mockContext = mock(AppContext.class); when(mockContext.isLastAMRetry()).thenReturn(true); when(mockContext.hasSuccessfullyUnregistered()).thenReturn(false); JobImpl job = createRunningStubbedJob(conf, dispatcher, 2, mockContext); completeJobTasks(job); assertJobState(job, JobStateInternal.COMMITTING); syncBarrier.await(); job.handle(new JobEvent(job.getID(), JobEventType.JOB_AM_REBOOT)); assertJobState(job, JobStateInternal.REBOOT); // return the external state as ERROR since this is last retry. Assert.assertEquals(JobState.RUNNING, job.getState()); when(mockContext.hasSuccessfullyUnregistered()).thenReturn(true); Assert.assertEquals(JobState.ERROR, job.getState()); dispatcher.stop(); commitHandler.stop(); }
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()); } }
@Test(timeout = 20000) public void testRebootedDuringSetup() throws Exception { Configuration conf = new Configuration(); conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir); 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(); AppContext mockContext = mock(AppContext.class); when(mockContext.isLastAMRetry()).thenReturn(false); JobImpl job = createStubbedJob(conf, dispatcher, 2, mockContext); JobId jobId = job.getID(); job.handle(new JobEvent(jobId, JobEventType.JOB_INIT)); assertJobState(job, JobStateInternal.INITED); job.handle(new JobStartEvent(jobId)); assertJobState(job, JobStateInternal.SETUP); job.handle(new JobEvent(job.getID(), JobEventType.JOB_AM_REBOOT)); assertJobState(job, JobStateInternal.REBOOT); // return the external state as RUNNING since otherwise JobClient will // exit when it polls the AM for job state Assert.assertEquals(JobState.RUNNING, job.getState()); dispatcher.stop(); commitHandler.stop(); }
@Test public void testJobNoTasks() { Configuration conf = new Configuration(); conf.setInt(MRJobConfig.NUM_REDUCES, 0); conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir); conf.set(MRJobConfig.WORKFLOW_ID, "testId"); conf.set(MRJobConfig.WORKFLOW_NAME, "testName"); conf.set(MRJobConfig.WORKFLOW_NODE_NAME, "testNodeName"); conf.set(MRJobConfig.WORKFLOW_ADJACENCY_PREFIX_STRING + "key1", "value1"); conf.set(MRJobConfig.WORKFLOW_ADJACENCY_PREFIX_STRING + "key2", "value2"); conf.set(MRJobConfig.WORKFLOW_TAGS, "tag1,tag2"); AsyncDispatcher dispatcher = new AsyncDispatcher(); dispatcher.init(conf); dispatcher.start(); OutputCommitter committer = mock(OutputCommitter.class); CommitterEventHandler commitHandler = createCommitterEventHandler(dispatcher, committer); commitHandler.init(conf); commitHandler.start(); JobSubmittedEventHandler jseHandler = new JobSubmittedEventHandler( "testId", "testName", "testNodeName", "\"key2\"=\"value2\" \"key1\"=\"value1\" ", "tag1,tag2"); dispatcher.register(EventType.class, jseHandler); JobImpl job = createStubbedJob(conf, dispatcher, 0, null); job.handle(new JobEvent(job.getID(), JobEventType.JOB_INIT)); assertJobState(job, JobStateInternal.INITED); job.handle(new JobStartEvent(job.getID())); assertJobState(job, JobStateInternal.SUCCEEDED); dispatcher.stop(); commitHandler.stop(); try { Assert.assertTrue(jseHandler.getAssertValue()); } catch (InterruptedException e) { Assert.fail("Workflow related attributes are not tested properly"); } }
@Test(timeout = 20000) public void testKilledDuringCommit() throws Exception { Configuration conf = new Configuration(); conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir); 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, null); 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 testTransitionsAtFailed() throws IOException { Configuration conf = new Configuration(); AsyncDispatcher dispatcher = new AsyncDispatcher(); dispatcher.init(conf); dispatcher.start(); OutputCommitter committer = mock(OutputCommitter.class); doThrow(new IOException("forcefail")).when(committer).setupJob(any(JobContext.class)); CommitterEventHandler commitHandler = createCommitterEventHandler(dispatcher, committer); commitHandler.init(conf); commitHandler.start(); AppContext mockContext = mock(AppContext.class); when(mockContext.hasSuccessfullyUnregistered()).thenReturn(false); JobImpl job = createStubbedJob(conf, dispatcher, 2, mockContext); JobId jobId = job.getID(); job.handle(new JobEvent(jobId, JobEventType.JOB_INIT)); assertJobState(job, JobStateInternal.INITED); job.handle(new JobStartEvent(jobId)); assertJobState(job, JobStateInternal.FAILED); job.handle(new JobEvent(jobId, JobEventType.JOB_TASK_COMPLETED)); assertJobState(job, JobStateInternal.FAILED); job.handle(new JobEvent(jobId, JobEventType.JOB_TASK_ATTEMPT_COMPLETED)); assertJobState(job, JobStateInternal.FAILED); job.handle(new JobEvent(jobId, JobEventType.JOB_MAP_TASK_RESCHEDULED)); assertJobState(job, JobStateInternal.FAILED); job.handle(new JobEvent(jobId, JobEventType.JOB_TASK_ATTEMPT_FETCH_FAILURE)); assertJobState(job, JobStateInternal.FAILED); Assert.assertEquals(JobState.RUNNING, job.getState()); when(mockContext.hasSuccessfullyUnregistered()).thenReturn(true); Assert.assertEquals(JobState.FAILED, job.getState()); dispatcher.stop(); commitHandler.stop(); }