|
@@ -21,6 +21,7 @@ package org.apache.hadoop.mapreduce.v2.app;
|
|
|
import static org.junit.Assert.assertFalse;
|
|
|
import static org.mockito.ArgumentMatchers.any;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
+import static org.mockito.Mockito.never;
|
|
|
import static org.mockito.Mockito.times;
|
|
|
import static org.mockito.Mockito.verify;
|
|
|
|
|
@@ -48,7 +49,7 @@ import java.util.concurrent.ConcurrentMap;
|
|
|
|
|
|
public class TestTaskHeartbeatHandler {
|
|
|
|
|
|
- @SuppressWarnings({ "rawtypes", "unchecked" })
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
@Test
|
|
|
public void testTaskTimeout() throws InterruptedException {
|
|
|
EventHandler mockHandler = mock(EventHandler.class);
|
|
@@ -81,6 +82,46 @@ public class TestTaskHeartbeatHandler {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ public void testTaskTimeoutDisable() throws InterruptedException {
|
|
|
+ EventHandler mockHandler = mock(EventHandler.class);
|
|
|
+ Clock clock = SystemClock.getInstance();
|
|
|
+ TaskHeartbeatHandler hb = new TaskHeartbeatHandler(mockHandler, clock, 1);
|
|
|
+
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ conf.setLong(MRJobConfig.TASK_STUCK_TIMEOUT_MS, 0); // no timeout
|
|
|
+ conf.setInt(MRJobConfig.TASK_TIMEOUT, 0); // no timeout
|
|
|
+ // set TASK_PROGRESS_REPORT_INTERVAL to a value smaller than TASK_TIMEOUT
|
|
|
+ // so that TASK_TIMEOUT is not overridden
|
|
|
+ conf.setLong(MRJobConfig.TASK_PROGRESS_REPORT_INTERVAL, 0);
|
|
|
+ conf.setInt(MRJobConfig.TASK_TIMEOUT_CHECK_INTERVAL_MS, 10); //10 ms
|
|
|
+
|
|
|
+ hb.init(conf);
|
|
|
+ hb.start();
|
|
|
+ try {
|
|
|
+ ApplicationId appId = ApplicationId.newInstance(0L, 5);
|
|
|
+ JobId jobId = MRBuilderUtils.newJobId(appId, 4);
|
|
|
+ TaskId tid = MRBuilderUtils.newTaskId(jobId, 3, TaskType.MAP);
|
|
|
+ TaskAttemptId taid = MRBuilderUtils.newTaskAttemptId(tid, 2);
|
|
|
+ hb.register(taid);
|
|
|
+
|
|
|
+ ConcurrentMap<TaskAttemptId, TaskHeartbeatHandler.ReportTime>
|
|
|
+ runningAttempts = hb.getRunningAttempts();
|
|
|
+ for (Map.Entry<TaskAttemptId, TaskHeartbeatHandler.ReportTime> entry
|
|
|
+ : runningAttempts.entrySet()) {
|
|
|
+ assertFalse(entry.getValue().isReported());
|
|
|
+ }
|
|
|
+
|
|
|
+ Thread.sleep(100);
|
|
|
+
|
|
|
+ // Timeout is disabled, so the task should not be canceled
|
|
|
+ verify(mockHandler, never()).handle(any(Event.class));
|
|
|
+ } finally {
|
|
|
+ hb.stop();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@SuppressWarnings("unchecked")
|
|
|
@Test
|
|
|
public void testTaskStuck() throws InterruptedException {
|