|
72 | 72 | import org.apache.hadoop.mapreduce.v2.app.job.event.JobEvent; |
73 | 73 | import org.apache.hadoop.mapreduce.v2.app.job.event.JobEventType; |
74 | 74 | import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerAssignedEvent; |
| 75 | +import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerLaunchedEvent; |
75 | 76 | import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptDiagnosticsUpdateEvent; |
76 | 77 | import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent; |
77 | 78 | import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType; |
@@ -450,6 +451,121 @@ public void testLaunchFailedWhileKilling() throws Exception { |
450 | 451 | assertFalse(eventHandler.internalError); |
451 | 452 | } |
452 | 453 |
|
| 454 | + @Test |
| 455 | + public void testContainerCleanedWhileRunning() throws Exception { |
| 456 | + ApplicationId appId = BuilderUtils.newApplicationId(1, 2); |
| 457 | + ApplicationAttemptId appAttemptId = |
| 458 | + BuilderUtils.newApplicationAttemptId(appId, 0); |
| 459 | + JobId jobId = MRBuilderUtils.newJobId(appId, 1); |
| 460 | + TaskId taskId = MRBuilderUtils.newTaskId(jobId, 1, TaskType.MAP); |
| 461 | + TaskAttemptId attemptId = MRBuilderUtils.newTaskAttemptId(taskId, 0); |
| 462 | + Path jobFile = mock(Path.class); |
| 463 | + |
| 464 | + MockEventHandler eventHandler = new MockEventHandler(); |
| 465 | + TaskAttemptListener taListener = mock(TaskAttemptListener.class); |
| 466 | + when(taListener.getAddress()).thenReturn(new InetSocketAddress("localhost", 0)); |
| 467 | + |
| 468 | + JobConf jobConf = new JobConf(); |
| 469 | + jobConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class); |
| 470 | + jobConf.setBoolean("fs.file.impl.disable.cache", true); |
| 471 | + jobConf.set(JobConf.MAPRED_MAP_TASK_ENV, ""); |
| 472 | + jobConf.set(MRJobConfig.APPLICATION_ATTEMPT_ID, "10"); |
| 473 | + |
| 474 | + TaskSplitMetaInfo splits = mock(TaskSplitMetaInfo.class); |
| 475 | + when(splits.getLocations()).thenReturn(new String[] {"127.0.0.1"}); |
| 476 | + |
| 477 | + AppContext appCtx = mock(AppContext.class); |
| 478 | + ClusterInfo clusterInfo = mock(ClusterInfo.class); |
| 479 | + Resource resource = mock(Resource.class); |
| 480 | + when(appCtx.getClusterInfo()).thenReturn(clusterInfo); |
| 481 | + when(clusterInfo.getMinContainerCapability()).thenReturn(resource); |
| 482 | + when(resource.getMemory()).thenReturn(1024); |
| 483 | + |
| 484 | + TaskAttemptImpl taImpl = |
| 485 | + new MapTaskAttemptImpl(taskId, 1, eventHandler, jobFile, 1, |
| 486 | + splits, jobConf, taListener, |
| 487 | + mock(OutputCommitter.class), mock(Token.class), new Credentials(), |
| 488 | + new SystemClock(), appCtx); |
| 489 | + |
| 490 | + NodeId nid = BuilderUtils.newNodeId("127.0.0.1", 0); |
| 491 | + ContainerId contId = BuilderUtils.newContainerId(appAttemptId, 3); |
| 492 | + Container container = mock(Container.class); |
| 493 | + when(container.getId()).thenReturn(contId); |
| 494 | + when(container.getNodeId()).thenReturn(nid); |
| 495 | + when(container.getNodeHttpAddress()).thenReturn("localhost:0"); |
| 496 | + |
| 497 | + taImpl.handle(new TaskAttemptEvent(attemptId, |
| 498 | + TaskAttemptEventType.TA_SCHEDULE)); |
| 499 | + taImpl.handle(new TaskAttemptContainerAssignedEvent(attemptId, |
| 500 | + container, mock(Map.class))); |
| 501 | + taImpl.handle(new TaskAttemptContainerLaunchedEvent(attemptId, 0)); |
| 502 | + assertEquals("Task attempt is not in running state", taImpl.getState(), |
| 503 | + TaskAttemptState.RUNNING); |
| 504 | + taImpl.handle(new TaskAttemptEvent(attemptId, |
| 505 | + TaskAttemptEventType.TA_CONTAINER_CLEANED)); |
| 506 | + assertFalse("InternalError occurred trying to handle TA_CONTAINER_CLEANED", |
| 507 | + eventHandler.internalError); |
| 508 | + } |
| 509 | + |
| 510 | + @Test |
| 511 | + public void testContainerCleanedWhileCommitting() throws Exception { |
| 512 | + ApplicationId appId = BuilderUtils.newApplicationId(1, 2); |
| 513 | + ApplicationAttemptId appAttemptId = |
| 514 | + BuilderUtils.newApplicationAttemptId(appId, 0); |
| 515 | + JobId jobId = MRBuilderUtils.newJobId(appId, 1); |
| 516 | + TaskId taskId = MRBuilderUtils.newTaskId(jobId, 1, TaskType.MAP); |
| 517 | + TaskAttemptId attemptId = MRBuilderUtils.newTaskAttemptId(taskId, 0); |
| 518 | + Path jobFile = mock(Path.class); |
| 519 | + |
| 520 | + MockEventHandler eventHandler = new MockEventHandler(); |
| 521 | + TaskAttemptListener taListener = mock(TaskAttemptListener.class); |
| 522 | + when(taListener.getAddress()).thenReturn(new InetSocketAddress("localhost", 0)); |
| 523 | + |
| 524 | + JobConf jobConf = new JobConf(); |
| 525 | + jobConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class); |
| 526 | + jobConf.setBoolean("fs.file.impl.disable.cache", true); |
| 527 | + jobConf.set(JobConf.MAPRED_MAP_TASK_ENV, ""); |
| 528 | + jobConf.set(MRJobConfig.APPLICATION_ATTEMPT_ID, "10"); |
| 529 | + |
| 530 | + TaskSplitMetaInfo splits = mock(TaskSplitMetaInfo.class); |
| 531 | + when(splits.getLocations()).thenReturn(new String[] {"127.0.0.1"}); |
| 532 | + |
| 533 | + AppContext appCtx = mock(AppContext.class); |
| 534 | + ClusterInfo clusterInfo = mock(ClusterInfo.class); |
| 535 | + Resource resource = mock(Resource.class); |
| 536 | + when(appCtx.getClusterInfo()).thenReturn(clusterInfo); |
| 537 | + when(clusterInfo.getMinContainerCapability()).thenReturn(resource); |
| 538 | + when(resource.getMemory()).thenReturn(1024); |
| 539 | + |
| 540 | + TaskAttemptImpl taImpl = |
| 541 | + new MapTaskAttemptImpl(taskId, 1, eventHandler, jobFile, 1, |
| 542 | + splits, jobConf, taListener, |
| 543 | + mock(OutputCommitter.class), mock(Token.class), new Credentials(), |
| 544 | + new SystemClock(), appCtx); |
| 545 | + |
| 546 | + NodeId nid = BuilderUtils.newNodeId("127.0.0.1", 0); |
| 547 | + ContainerId contId = BuilderUtils.newContainerId(appAttemptId, 3); |
| 548 | + Container container = mock(Container.class); |
| 549 | + when(container.getId()).thenReturn(contId); |
| 550 | + when(container.getNodeId()).thenReturn(nid); |
| 551 | + when(container.getNodeHttpAddress()).thenReturn("localhost:0"); |
| 552 | + |
| 553 | + taImpl.handle(new TaskAttemptEvent(attemptId, |
| 554 | + TaskAttemptEventType.TA_SCHEDULE)); |
| 555 | + taImpl.handle(new TaskAttemptContainerAssignedEvent(attemptId, |
| 556 | + container, mock(Map.class))); |
| 557 | + taImpl.handle(new TaskAttemptContainerLaunchedEvent(attemptId, 0)); |
| 558 | + taImpl.handle(new TaskAttemptEvent(attemptId, |
| 559 | + TaskAttemptEventType.TA_COMMIT_PENDING)); |
| 560 | + |
| 561 | + assertEquals("Task attempt is not in commit pending state", taImpl.getState(), |
| 562 | + TaskAttemptState.COMMIT_PENDING); |
| 563 | + taImpl.handle(new TaskAttemptEvent(attemptId, |
| 564 | + TaskAttemptEventType.TA_CONTAINER_CLEANED)); |
| 565 | + assertFalse("InternalError occurred trying to handle TA_CONTAINER_CLEANED", |
| 566 | + eventHandler.internalError); |
| 567 | + } |
| 568 | + |
453 | 569 | public static class MockEventHandler implements EventHandler { |
454 | 570 | public boolean internalError; |
455 | 571 |
|
|
0 commit comments