|
@@ -17,23 +17,30 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.mapred;
|
|
|
|
|
|
-import java.util.function.Supplier;
|
|
|
-import org.apache.hadoop.mapred.Counters.Counter;
|
|
|
-import org.apache.hadoop.mapreduce.checkpoint.EnumCounter;
|
|
|
-
|
|
|
import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.List;
|
|
|
import java.util.concurrent.ConcurrentMap;
|
|
|
import java.util.concurrent.atomic.AtomicReference;
|
|
|
+import java.util.function.Supplier;
|
|
|
+
|
|
|
+import org.junit.After;
|
|
|
+import org.junit.Test;
|
|
|
+import org.junit.runner.RunWith;
|
|
|
+import org.mockito.ArgumentCaptor;
|
|
|
+import org.mockito.Captor;
|
|
|
+import org.mockito.Mock;
|
|
|
+import org.mockito.junit.MockitoJUnitRunner;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
+import org.apache.hadoop.mapred.Counters.Counter;
|
|
|
import org.apache.hadoop.mapreduce.MRJobConfig;
|
|
|
import org.apache.hadoop.mapreduce.TaskType;
|
|
|
import org.apache.hadoop.mapreduce.TypeConverter;
|
|
|
import org.apache.hadoop.mapreduce.checkpoint.CheckpointID;
|
|
|
+import org.apache.hadoop.mapreduce.checkpoint.EnumCounter;
|
|
|
import org.apache.hadoop.mapreduce.checkpoint.FSCheckpointID;
|
|
|
import org.apache.hadoop.mapreduce.checkpoint.TaskCheckpointID;
|
|
|
import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
|
|
@@ -48,9 +55,9 @@ import org.apache.hadoop.mapreduce.v2.app.TaskHeartbeatHandler;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.Job;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.rm.preemption.AMPreemptionPolicy;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.rm.preemption.CheckpointAMPreemptionPolicy;
|
|
|
-import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
|
|
|
import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.yarn.event.Dispatcher;
|
|
@@ -60,17 +67,22 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
|
|
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
|
|
import org.apache.hadoop.yarn.util.ControlledClock;
|
|
|
import org.apache.hadoop.yarn.util.SystemClock;
|
|
|
-import org.junit.After;
|
|
|
-import org.junit.Test;
|
|
|
-import org.junit.runner.RunWith;
|
|
|
-import org.mockito.ArgumentCaptor;
|
|
|
-import org.mockito.Captor;
|
|
|
-import org.mockito.Mock;
|
|
|
-import org.mockito.junit.MockitoJUnitRunner;
|
|
|
|
|
|
import static org.assertj.core.api.Assertions.assertThat;
|
|
|
-import static org.junit.Assert.*;
|
|
|
-import static org.mockito.Mockito.*;
|
|
|
+import static org.junit.Assert.assertEquals;
|
|
|
+import static org.junit.Assert.assertFalse;
|
|
|
+import static org.junit.Assert.assertNotNull;
|
|
|
+import static org.junit.Assert.assertNull;
|
|
|
+import static org.junit.Assert.assertTrue;
|
|
|
+import static org.junit.Assert.fail;
|
|
|
+import static org.mockito.Mockito.any;
|
|
|
+import static org.mockito.Mockito.doReturn;
|
|
|
+import static org.mockito.Mockito.eq;
|
|
|
+import static org.mockito.Mockito.mock;
|
|
|
+import static org.mockito.Mockito.never;
|
|
|
+import static org.mockito.Mockito.times;
|
|
|
+import static org.mockito.Mockito.verify;
|
|
|
+import static org.mockito.Mockito.when;
|
|
|
|
|
|
/**
|
|
|
* Tests the behavior of TaskAttemptListenerImpl.
|
|
@@ -417,6 +429,23 @@ public class TestTaskAttemptListenerImpl {
|
|
|
verify(hbHandler).progressing(eq(attemptId));
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testPingUpdateProgress() throws IOException, InterruptedException {
|
|
|
+ configureMocks();
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ conf.setBoolean(MRJobConfig.MR_TASK_ENABLE_PING_FOR_LIVELINESS_CHECK, true);
|
|
|
+ listener.init(conf);
|
|
|
+ listener.start();
|
|
|
+ listener.registerPendingTask(task, wid);
|
|
|
+ listener.registerLaunchedTask(attemptId, wid);
|
|
|
+ verify(hbHandler).register(attemptId);
|
|
|
+
|
|
|
+ // make sure a ping does report progress
|
|
|
+ AMFeedback feedback = listener.statusUpdate(attemptID, null);
|
|
|
+ assertTrue(feedback.getTaskFound());
|
|
|
+ verify(hbHandler, times(1)).progressing(eq(attemptId));
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testSingleStatusUpdate()
|
|
|
throws IOException, InterruptedException {
|