Explorar el Código

MAPREDUCE-7418. [JDK17] Upgrade Junit 4 to 5 in hadoop-mapreduce-client-app. (#7350)

Co-authored-by: Chris Nauroth <cnauroth@apache.org>
Reviewed-by: Chris Nauroth <cnauroth@apache.org>
Signed-off-by: Shilun Fan <slfan1989@apache.org>
slfan1989 hace 2 meses
padre
commit
d552670b86
Se han modificado 54 ficheros con 1659 adiciones y 1696 borrados
  1. 4 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/pom.xml
  2. 12 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestLocalContainerLauncher.java
  3. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptFinishingMonitor.java
  4. 21 23
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java
  5. 9 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestYarnChild.java
  6. 14 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java
  7. 126 142
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
  8. 8 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobSummary.java
  9. 4 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/api/records/TestTaskAttemptReport.java
  10. 4 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/api/records/TestTaskReport.java
  11. 32 41
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
  12. 8 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java
  13. 7 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestAMInfos.java
  14. 6 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestCheckpointPreemptionPolicy.java
  15. 23 26
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java
  16. 75 85
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFetchFailure.java
  17. 47 52
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestJobEndNotifier.java
  18. 39 46
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKill.java
  19. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKillAMPreemptionPolicy.java
  20. 53 70
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRApp.java
  21. 9 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppComponentDependencies.java
  22. 46 46
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java
  23. 44 52
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRClientService.java
  24. 89 114
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java
  25. 19 22
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java
  26. 24 20
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java
  27. 10 15
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestTaskHeartbeatHandler.java
  28. 21 20
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/commit/TestCommitterEventHandler.java
  29. 94 76
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java
  30. 47 45
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestMapReduceChildJVM.java
  31. 9 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestShuffleProvider.java
  32. 153 162
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java
  33. 14 16
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttemptContainerRequest.java
  34. 14 14
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java
  35. 34 31
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
  36. 12 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java
  37. 16 15
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java
  38. 5 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/metrics/TestMRAppMetrics.java
  39. 6 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMCommunicator.java
  40. 185 176
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
  41. 10 11
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestResourceCalculatorUtils.java
  42. 25 24
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/speculate/TestDataStatistics.java
  43. 6 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/speculate/forecast/TestSimpleExponentialForecast.java
  44. 28 28
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebApp.java
  45. 18 18
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServices.java
  46. 4 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempt.java
  47. 42 46
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempts.java
  48. 21 20
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobConf.java
  49. 78 83
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobs.java
  50. 45 47
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesTasks.java
  51. 11 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAppController.java
  52. 7 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestBlocks.java
  53. 5 0
      hadoop-project/pom.xml
  54. 12 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java

+ 4 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/pom.xml

@@ -173,6 +173,10 @@
       <artifactId>junit-vintage-engine</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-junit-jupiter</artifactId>
+    </dependency>
   </dependencies>
 
   <build>

+ 12 - 9
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestLocalContainerLauncher.java

@@ -53,15 +53,17 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+
 public class TestLocalContainerLauncher {
   private static final Logger LOG =
       LoggerFactory.getLogger(TestLocalContainerLauncher.class);
@@ -75,7 +77,7 @@ public class TestLocalContainerLauncher {
     fs.delete(p, true);
   }
 
-  @BeforeClass
+  @BeforeAll
   public static void setupTestDirs() throws IOException {
     testWorkDir = new File("target",
         TestLocalContainerLauncher.class.getCanonicalName());
@@ -89,7 +91,7 @@ public class TestLocalContainerLauncher {
     }
   }
 
-  @AfterClass
+  @AfterAll
   public static void cleanupTestDirs() throws IOException {
     if (testWorkDir != null) {
       delete(testWorkDir);
@@ -97,7 +99,8 @@ public class TestLocalContainerLauncher {
   }
 
   @SuppressWarnings("rawtypes")
-  @Test(timeout=10000)
+  @Test
+  @Timeout(value = 10)
   public void testKillJob() throws Exception {
     JobConf conf = new JobConf();
     AppContext context = mock(AppContext.class);
@@ -198,8 +201,8 @@ public class TestLocalContainerLauncher {
     final Path mapOut = mrOutputFiles.getOutputFileForWrite(1);
     conf.set(MRConfig.LOCAL_DIR, localDirs[1].toString());
     final Path mapOutIdx = mrOutputFiles.getOutputIndexFileForWrite(1);
-    Assert.assertNotEquals("Paths must be different!",
-        mapOut.getParent(), mapOutIdx.getParent());
+    assertNotEquals(mapOut.getParent(), mapOutIdx.getParent(),
+        "Paths must be different!");
 
     // make both dirs part of LOCAL_DIR
     conf.setStrings(MRConfig.LOCAL_DIR, localDirs);

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptFinishingMonitor.java

@@ -37,8 +37,8 @@ import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.util.SystemClock;
 
-import org.junit.Test;
-import static org.junit.Assert.assertTrue;
+import org.junit.jupiter.api.Test;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -87,7 +87,7 @@ public class TestTaskAttemptFinishingMonitor {
     }
     taskAttemptFinishingMonitor.stop();
 
-    assertTrue("Finishing attempt didn't time out.", eventHandler.timedOut);
+    assertTrue(eventHandler.timedOut, "Finishing attempt didn't time out.");
 
   }
 

+ 21 - 23
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java

@@ -19,19 +19,19 @@ package org.apache.hadoop.mapred;
 
 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.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+import org.junit.jupiter.api.extension.ExtendWith;
 import org.mockito.ArgumentCaptor;
 import org.mockito.Captor;
 import org.mockito.Mock;
-import org.mockito.junit.MockitoJUnitRunner;
+import org.mockito.junit.jupiter.MockitoExtension;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -69,12 +69,12 @@ import org.apache.hadoop.yarn.util.ControlledClock;
 import org.apache.hadoop.yarn.util.SystemClock;
 
 import static org.assertj.core.api.Assertions.assertThat;
-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.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.Mockito.any;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.eq;
@@ -87,7 +87,7 @@ import static org.mockito.Mockito.when;
 /**
  * Tests the behavior of TaskAttemptListenerImpl.
  */
-@RunWith(MockitoJUnitRunner.class)
+@ExtendWith(MockitoExtension.class)
 public class TestTaskAttemptListenerImpl {
   private static final String ATTEMPT1_ID =
       "attempt_123456789012_0001_m_000001_0";
@@ -172,7 +172,7 @@ public class TestTaskAttemptListenerImpl {
     }
   }
 
-  @After
+  @AfterEach
   public void after() throws IOException {
     if (listener != null) {
       listener.close();
@@ -180,7 +180,8 @@ public class TestTaskAttemptListenerImpl {
     }
   }
 
-  @Test  (timeout=5000)
+  @Test
+  @Timeout(value = 5)
   public void testGetTask() throws IOException {
     configureMocks();
     startListener(false);
@@ -238,7 +239,8 @@ public class TestTaskAttemptListenerImpl {
 
   }
 
-  @Test (timeout=5000)
+  @Test
+  @Timeout(value = 5)
   public void testJVMId() {
 
     JVMId jvmid = new JVMId("test", 1, true, 2);
@@ -247,7 +249,8 @@ public class TestTaskAttemptListenerImpl {
     assertEquals(0, jvmid.compareTo(jvmid1));
   }
 
-  @Test (timeout=10000)
+  @Test
+  @Timeout(value = 10)
   public void testGetMapCompletionEvents() throws IOException {
     TaskAttemptCompletionEvent[] empty = {};
     TaskAttemptCompletionEvent[] taskEvents = {
@@ -257,12 +260,6 @@ public class TestTaskAttemptListenerImpl {
         createTce(3, false, TaskAttemptCompletionEventStatus.FAILED) };
     TaskAttemptCompletionEvent[] mapEvents = { taskEvents[0], taskEvents[2] };
     Job mockJob = mock(Job.class);
-    when(mockJob.getTaskAttemptCompletionEvents(0, 100))
-      .thenReturn(taskEvents);
-    when(mockJob.getTaskAttemptCompletionEvents(0, 2))
-      .thenReturn(Arrays.copyOfRange(taskEvents, 0, 2));
-    when(mockJob.getTaskAttemptCompletionEvents(2, 100))
-      .thenReturn(Arrays.copyOfRange(taskEvents, 2, 4));
     when(mockJob.getMapAttemptCompletionEvents(0, 100)).thenReturn(
         TypeConverter.fromYarn(mapEvents));
     when(mockJob.getMapAttemptCompletionEvents(0, 2)).thenReturn(
@@ -312,7 +309,8 @@ public class TestTaskAttemptListenerImpl {
     return tce;
   }
 
-  @Test (timeout=10000)
+  @Test
+  @Timeout(value = 10)
   public void testCommitWindow() throws IOException {
     SystemClock clock = SystemClock.getInstance();
 

+ 9 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestYarnChild.java

@@ -21,10 +21,15 @@ import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ClusterStorageCapacityExceededException;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.anyString;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 /**
  * Tests the behavior of YarnChild.
@@ -36,7 +41,7 @@ public class TestYarnChild {
   final static private String KILL_LIMIT_EXCEED_CONF_NAME =
       "mapreduce.job.dfs.storage.capacity.kill-limit-exceed";
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     task = mock(Task.class);
     umbilical = mock(TaskUmbilicalProtocol.class);

+ 14 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java

@@ -19,8 +19,8 @@
 package org.apache.hadoop.mapreduce.jobhistory;
 
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -40,7 +40,8 @@ import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 public class TestEvents {
 
@@ -50,7 +51,8 @@ public class TestEvents {
    * 
    * @throws Exception
    */
-  @Test(timeout = 10000)
+  @Test
+  @Timeout(value = 10)
   public void testTaskAttemptFinishedEvent() throws Exception {
 
     JobID jid = new JobID("001", 1);
@@ -79,7 +81,8 @@ public class TestEvents {
    * @throws Exception
    */
 
-  @Test(timeout = 10000)
+  @Test
+  @Timeout(value = 10)
   public void testJobPriorityChange() throws Exception {
     org.apache.hadoop.mapreduce.JobID jid = new JobID("001", 1);
     JobPriorityChangeEvent test = new JobPriorityChangeEvent(jid,
@@ -89,7 +92,8 @@ public class TestEvents {
 
   }
   
-  @Test(timeout = 10000)
+  @Test
+  @Timeout(value = 10)
   public void testJobQueueChange() throws Exception {
     org.apache.hadoop.mapreduce.JobID jid = new JobID("001", 1);
     JobQueueChangeEvent test = new JobQueueChangeEvent(jid,
@@ -103,7 +107,8 @@ public class TestEvents {
    * 
    * @throws Exception
    */
-  @Test(timeout = 10000)
+  @Test
+  @Timeout(value = 10)
   public void testTaskUpdated() throws Exception {
     JobID jid = new JobID("001", 1);
     TaskID tid = new TaskID(jid, TaskType.REDUCE, 2);
@@ -118,7 +123,8 @@ public class TestEvents {
    * instance of HistoryEvent Different HistoryEvent should have a different
    * datum.
    */
-  @Test(timeout = 10000)
+  @Test
+  @Timeout(value = 10)
   public void testEvents() throws Exception {
 
     EventReader reader = new EventReader(new DataInputStream(

+ 126 - 142
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java

@@ -19,9 +19,9 @@
 package org.apache.hadoop.mapreduce.jobhistory;
 
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.doReturn;
@@ -81,11 +81,11 @@ import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
 import org.apache.hadoop.yarn.server.timeline.TimelineStore;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.mockito.Mockito;
 
 import com.fasterxml.jackson.databind.JsonNode;
@@ -101,7 +101,7 @@ public class TestJobHistoryEventHandler {
   private static MiniDFSCluster dfsCluster = null;
   private static String coreSitePath;
 
-  @BeforeClass
+  @BeforeAll
   public static void setUpClass() throws Exception {
     coreSitePath = "." + File.separator + "target" + File.separator +
             "test-classes" + File.separator + "core-site.xml";
@@ -109,17 +109,18 @@ public class TestJobHistoryEventHandler {
     dfsCluster = new MiniDFSCluster.Builder(conf).build();
   }
 
-  @AfterClass
+  @AfterAll
   public static void cleanUpClass() throws Exception {
     dfsCluster.shutdown();
   }
 
-  @After
+  @AfterEach
   public void cleanTest() throws Exception {
     new File(coreSitePath).delete();
   }
 
-  @Test (timeout=50000)
+  @Test
+  @Timeout(value = 50)
   public void testFirstFlushOnCompletionEvent() throws Exception {
     TestParams t = new TestParams();
     Configuration conf = new Configuration();
@@ -162,7 +163,8 @@ public class TestJobHistoryEventHandler {
     }
   }
 
-  @Test (timeout=50000)
+  @Test
+  @Timeout(value = 50)
   public void testMaxUnflushedCompletionEvents() throws Exception {
     TestParams t = new TestParams();
     Configuration conf = new Configuration();
@@ -207,7 +209,8 @@ public class TestJobHistoryEventHandler {
     }
   }
 
-  @Test (timeout=50000)
+  @Test
+  @Timeout(value = 50)
   public void testUnflushedTimer() throws Exception {
     TestParams t = new TestParams();
     Configuration conf = new Configuration();
@@ -238,19 +241,20 @@ public class TestJobHistoryEventHandler {
       }
 
       handleNextNEvents(jheh, 9);
-      Assert.assertTrue(jheh.getFlushTimerStatus());
+      assertTrue(jheh.getFlushTimerStatus());
       verify(mockWriter, times(0)).flush();
 
       Thread.sleep(2 * 4 * 1000l); // 4 seconds should be enough. Just be safe.
       verify(mockWriter).flush();
-      Assert.assertFalse(jheh.getFlushTimerStatus());
+      assertFalse(jheh.getFlushTimerStatus());
     } finally {
       jheh.stop();
       verify(mockWriter).close();
     }
   }
 
-  @Test (timeout=50000)
+  @Test
+  @Timeout(value = 50)
   public void testBatchedFlushJobEndMultiplier() throws Exception {
     TestParams t = new TestParams();
     Configuration conf = new Configuration();
@@ -295,7 +299,8 @@ public class TestJobHistoryEventHandler {
   }
 
   // In case of all types of events, process Done files if it's last AM retry
-  @Test (timeout=50000)
+  @Test
+  @Timeout(value = 50)
   public void testProcessDoneFilesOnLastAMRetry() throws Exception {
     TestParams t = new TestParams(true);
     Configuration conf = new Configuration();
@@ -341,7 +346,8 @@ public class TestJobHistoryEventHandler {
   }
 
   // Skip processing Done files in case of ERROR, if it's not last AM retry
-  @Test (timeout=50000)
+  @Test
+  @Timeout(value = 50)
   public void testProcessDoneFilesNotLastAMRetry() throws Exception {
     TestParams t = new TestParams(false);
     Configuration conf = new Configuration();
@@ -421,16 +427,15 @@ public class TestJobHistoryEventHandler {
 
       // load the job_conf.xml in JHS directory and verify property redaction.
       Path jhsJobConfFile = getJobConfInIntermediateDoneDir(conf, params.jobId);
-      Assert.assertTrue("The job_conf.xml file is not in the JHS directory",
-          FileContext.getFileContext(conf).util().exists(jhsJobConfFile));
+      assertTrue(FileContext.getFileContext(conf).util().exists(jhsJobConfFile),
+          "The job_conf.xml file is not in the JHS directory");
       Configuration jhsJobConf = new Configuration();
 
       try (InputStream input = FileSystem.get(conf).open(jhsJobConfFile)) {
         jhsJobConf.addResource(input);
-        Assert.assertEquals(
-            sensitivePropertyName + " is not redacted in HDFS.",
-            MRJobConfUtil.REDACTION_REPLACEMENT_VAL,
-            jhsJobConf.get(sensitivePropertyName));
+        assertEquals(MRJobConfUtil.REDACTION_REPLACEMENT_VAL,
+            jhsJobConf.get(sensitivePropertyName),
+            sensitivePropertyName + " is not redacted in HDFS.");
       }
     } finally {
       jheh.stop();
@@ -456,7 +461,8 @@ public class TestJobHistoryEventHandler {
     fs.delete(new Path(intermDoneDirPrefix), true);
   }
 
-  @Test (timeout=50000)
+  @Test
+  @Timeout(value = 50)
   public void testDefaultFsIsUsedForHistory() throws Exception {
     // Create default configuration pointing to the minicluster
     Configuration conf = new Configuration();
@@ -490,11 +496,11 @@ public class TestJobHistoryEventHandler {
       // If we got here then event handler worked but we don't know with which
       // file system. Now we check that history stuff was written to minicluster
       FileSystem dfsFileSystem = dfsCluster.getFileSystem();
-      assertTrue("Minicluster contains some history files",
-          dfsFileSystem.globStatus(new Path(t.dfsWorkDir + "/*")).length != 0);
+      assertTrue(dfsFileSystem.globStatus(new Path(t.dfsWorkDir + "/*")).length != 0,
+          "Minicluster contains some history files");
       FileSystem localFileSystem = LocalFileSystem.get(conf);
-      assertFalse("No history directory on non-default file system",
-          localFileSystem.exists(new Path(t.dfsWorkDir)));
+      assertFalse(localFileSystem.exists(new Path(t.dfsWorkDir)),
+          "No history directory on non-default file system");
     } finally {
       jheh.stop();
       purgeHdfsHistoryIntermediateDoneDirectory(conf);
@@ -509,7 +515,7 @@ public class TestJobHistoryEventHandler {
         "/mapred/history/done_intermediate");
     conf.set(MRJobConfig.USER_NAME, System.getProperty("user.name"));
     String pathStr = JobHistoryUtils.getHistoryIntermediateDoneDirForUser(conf);
-    Assert.assertEquals("/mapred/history/done_intermediate/" +
+    assertEquals("/mapred/history/done_intermediate/" +
         System.getProperty("user.name"), pathStr);
 
     // Test fully qualified path
@@ -523,13 +529,14 @@ public class TestJobHistoryEventHandler {
     conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY,
             "file:///");
     pathStr = JobHistoryUtils.getHistoryIntermediateDoneDirForUser(conf);
-    Assert.assertEquals(dfsCluster.getURI().toString() +
+    assertEquals(dfsCluster.getURI().toString() +
         "/mapred/history/done_intermediate/" + System.getProperty("user.name"),
         pathStr);
   }
 
   // test AMStartedEvent for submitTime and startTime
-  @Test (timeout=50000)
+  @Test
+  @Timeout(value = 50)
   public void testAMStartedEvent() throws Exception {
     TestParams t = new TestParams();
     Configuration conf = new Configuration();
@@ -571,7 +578,8 @@ public class TestJobHistoryEventHandler {
 
   // Have JobHistoryEventHandler handle some events and make sure they get
   // stored to the Timeline store
-  @Test (timeout=50000)
+  @Test
+  @Timeout(value = 50)
   public void testTimelineEventHandling() throws Exception {
     TestParams t = new TestParams(RunningAppContext.class, false);
     Configuration conf = new YarnConfiguration();
@@ -598,14 +606,12 @@ public class TestJobHistoryEventHandler {
       jheh.getDispatcher().await();
       TimelineEntities entities = ts.getEntities("MAPREDUCE_JOB", null, null,
               null, null, null, null, null, null, null);
-      Assert.assertEquals(1, entities.getEntities().size());
+      assertEquals(1, entities.getEntities().size());
       TimelineEntity tEntity = entities.getEntities().get(0);
-      Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId());
-      Assert.assertEquals(1, tEntity.getEvents().size());
-      Assert.assertEquals(EventType.AM_STARTED.toString(),
-              tEntity.getEvents().get(0).getEventType());
-      Assert.assertEquals(currentTime - 10,
-              tEntity.getEvents().get(0).getTimestamp());
+      assertEquals(t.jobId.toString(), tEntity.getEntityId());
+      assertEquals(1, tEntity.getEvents().size());
+      assertEquals(EventType.AM_STARTED.toString(), tEntity.getEvents().get(0).getEventType());
+      assertEquals(currentTime - 10, tEntity.getEvents().get(0).getTimestamp());
 
       handleEvent(jheh, new JobHistoryEvent(t.jobId,
               new JobSubmittedEvent(TypeConverter.fromYarn(t.jobId), "name",
@@ -615,18 +621,14 @@ public class TestJobHistoryEventHandler {
       jheh.getDispatcher().await();
       entities = ts.getEntities("MAPREDUCE_JOB", null, null, null,
               null, null, null, null, null, null);
-      Assert.assertEquals(1, entities.getEntities().size());
+      assertEquals(1, entities.getEntities().size());
       tEntity = entities.getEntities().get(0);
-      Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId());
-      Assert.assertEquals(2, tEntity.getEvents().size());
-      Assert.assertEquals(EventType.JOB_SUBMITTED.toString(),
-              tEntity.getEvents().get(0).getEventType());
-      Assert.assertEquals(EventType.AM_STARTED.toString(),
-              tEntity.getEvents().get(1).getEventType());
-      Assert.assertEquals(currentTime + 10,
-              tEntity.getEvents().get(0).getTimestamp());
-      Assert.assertEquals(currentTime - 10,
-              tEntity.getEvents().get(1).getTimestamp());
+      assertEquals(t.jobId.toString(), tEntity.getEntityId());
+      assertEquals(2, tEntity.getEvents().size());
+      assertEquals(EventType.JOB_SUBMITTED.toString(), tEntity.getEvents().get(0).getEventType());
+      assertEquals(EventType.AM_STARTED.toString(), tEntity.getEvents().get(1).getEventType());
+      assertEquals(currentTime + 10, tEntity.getEvents().get(0).getTimestamp());
+      assertEquals(currentTime - 10, tEntity.getEvents().get(1).getTimestamp());
 
       handleEvent(jheh, new JobHistoryEvent(t.jobId,
               new JobQueueChangeEvent(TypeConverter.fromYarn(t.jobId), "q2"),
@@ -634,22 +636,17 @@ public class TestJobHistoryEventHandler {
       jheh.getDispatcher().await();
       entities = ts.getEntities("MAPREDUCE_JOB", null, null, null,
               null, null, null, null, null, null);
-      Assert.assertEquals(1, entities.getEntities().size());
+      assertEquals(1, entities.getEntities().size());
       tEntity = entities.getEntities().get(0);
-      Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId());
-      Assert.assertEquals(3, tEntity.getEvents().size());
-      Assert.assertEquals(EventType.JOB_SUBMITTED.toString(),
-              tEntity.getEvents().get(0).getEventType());
-      Assert.assertEquals(EventType.AM_STARTED.toString(),
-              tEntity.getEvents().get(1).getEventType());
-      Assert.assertEquals(EventType.JOB_QUEUE_CHANGED.toString(),
-              tEntity.getEvents().get(2).getEventType());
-      Assert.assertEquals(currentTime + 10,
-              tEntity.getEvents().get(0).getTimestamp());
-      Assert.assertEquals(currentTime - 10,
-              tEntity.getEvents().get(1).getTimestamp());
-      Assert.assertEquals(currentTime - 20,
-              tEntity.getEvents().get(2).getTimestamp());
+      assertEquals(t.jobId.toString(), tEntity.getEntityId());
+      assertEquals(3, tEntity.getEvents().size());
+      assertEquals(EventType.JOB_SUBMITTED.toString(), tEntity.getEvents().get(0).getEventType());
+      assertEquals(EventType.AM_STARTED.toString(), tEntity.getEvents().get(1).getEventType());
+      assertEquals(EventType.JOB_QUEUE_CHANGED.toString(),
+          tEntity.getEvents().get(2).getEventType());
+      assertEquals(currentTime + 10, tEntity.getEvents().get(0).getTimestamp());
+      assertEquals(currentTime - 10, tEntity.getEvents().get(1).getTimestamp());
+      assertEquals(currentTime - 20, tEntity.getEvents().get(2).getTimestamp());
 
       handleEvent(jheh, new JobHistoryEvent(t.jobId,
               new JobFinishedEvent(TypeConverter.fromYarn(t.jobId), 0, 0, 0, 0,
@@ -657,26 +654,19 @@ public class TestJobHistoryEventHandler {
       jheh.getDispatcher().await();
       entities = ts.getEntities("MAPREDUCE_JOB", null, null, null,
               null, null, null, null, null, null);
-      Assert.assertEquals(1, entities.getEntities().size());
+      assertEquals(1, entities.getEntities().size());
       tEntity = entities.getEntities().get(0);
-      Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId());
-      Assert.assertEquals(4, tEntity.getEvents().size());
-      Assert.assertEquals(EventType.JOB_SUBMITTED.toString(),
-              tEntity.getEvents().get(0).getEventType());
-      Assert.assertEquals(EventType.JOB_FINISHED.toString(),
-              tEntity.getEvents().get(1).getEventType());
-      Assert.assertEquals(EventType.AM_STARTED.toString(),
-              tEntity.getEvents().get(2).getEventType());
-      Assert.assertEquals(EventType.JOB_QUEUE_CHANGED.toString(),
-              tEntity.getEvents().get(3).getEventType());
-      Assert.assertEquals(currentTime + 10,
-              tEntity.getEvents().get(0).getTimestamp());
-      Assert.assertEquals(currentTime,
-              tEntity.getEvents().get(1).getTimestamp());
-      Assert.assertEquals(currentTime - 10,
-              tEntity.getEvents().get(2).getTimestamp());
-      Assert.assertEquals(currentTime - 20,
-              tEntity.getEvents().get(3).getTimestamp());
+      assertEquals(t.jobId.toString(), tEntity.getEntityId());
+      assertEquals(4, tEntity.getEvents().size());
+      assertEquals(EventType.JOB_SUBMITTED.toString(), tEntity.getEvents().get(0).getEventType());
+      assertEquals(EventType.JOB_FINISHED.toString(), tEntity.getEvents().get(1).getEventType());
+      assertEquals(EventType.AM_STARTED.toString(), tEntity.getEvents().get(2).getEventType());
+      assertEquals(EventType.JOB_QUEUE_CHANGED.toString(),
+          tEntity.getEvents().get(3).getEventType());
+      assertEquals(currentTime + 10, tEntity.getEvents().get(0).getTimestamp());
+      assertEquals(currentTime, tEntity.getEvents().get(1).getTimestamp());
+      assertEquals(currentTime - 10, tEntity.getEvents().get(2).getTimestamp());
+      assertEquals(currentTime - 20, tEntity.getEvents().get(3).getTimestamp());
 
       handleEvent(jheh, new JobHistoryEvent(t.jobId,
             new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(t.jobId),
@@ -685,64 +675,54 @@ public class TestJobHistoryEventHandler {
       jheh.getDispatcher().await();
       entities = ts.getEntities("MAPREDUCE_JOB", null, null, null,
               null, null, null, null, null, null);
-      Assert.assertEquals(1, entities.getEntities().size());
+      assertEquals(1, entities.getEntities().size());
       tEntity = entities.getEntities().get(0);
-      Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId());
-      Assert.assertEquals(5, tEntity.getEvents().size());
-      Assert.assertEquals(EventType.JOB_KILLED.toString(),
-              tEntity.getEvents().get(0).getEventType());
-      Assert.assertEquals(EventType.JOB_SUBMITTED.toString(),
-              tEntity.getEvents().get(1).getEventType());
-      Assert.assertEquals(EventType.JOB_FINISHED.toString(),
-              tEntity.getEvents().get(2).getEventType());
-      Assert.assertEquals(EventType.AM_STARTED.toString(),
-              tEntity.getEvents().get(3).getEventType());
-      Assert.assertEquals(EventType.JOB_QUEUE_CHANGED.toString(),
-              tEntity.getEvents().get(4).getEventType());
-      Assert.assertEquals(currentTime + 20,
-              tEntity.getEvents().get(0).getTimestamp());
-      Assert.assertEquals(currentTime + 10,
-              tEntity.getEvents().get(1).getTimestamp());
-      Assert.assertEquals(currentTime,
-              tEntity.getEvents().get(2).getTimestamp());
-      Assert.assertEquals(currentTime - 10,
-              tEntity.getEvents().get(3).getTimestamp());
-      Assert.assertEquals(currentTime - 20,
-              tEntity.getEvents().get(4).getTimestamp());
+      assertEquals(t.jobId.toString(), tEntity.getEntityId());
+      assertEquals(5, tEntity.getEvents().size());
+      assertEquals(EventType.JOB_KILLED.toString(), tEntity.getEvents().get(0).getEventType());
+      assertEquals(EventType.JOB_SUBMITTED.toString(), tEntity.getEvents().get(1).getEventType());
+      assertEquals(EventType.JOB_FINISHED.toString(), tEntity.getEvents().get(2).getEventType());
+      assertEquals(EventType.AM_STARTED.toString(), tEntity.getEvents().get(3).getEventType());
+      assertEquals(EventType.JOB_QUEUE_CHANGED.toString(),
+          tEntity.getEvents().get(4).getEventType());
+      assertEquals(currentTime + 20, tEntity.getEvents().get(0).getTimestamp());
+      assertEquals(currentTime + 10, tEntity.getEvents().get(1).getTimestamp());
+      assertEquals(currentTime, tEntity.getEvents().get(2).getTimestamp());
+      assertEquals(currentTime - 10, tEntity.getEvents().get(3).getTimestamp());
+      assertEquals(currentTime - 20, tEntity.getEvents().get(4).getTimestamp());
 
       handleEvent(jheh, new JobHistoryEvent(t.jobId,
             new TaskStartedEvent(t.taskID, 0, TaskType.MAP, "")));
       jheh.getDispatcher().await();
       entities = ts.getEntities("MAPREDUCE_TASK", null, null, null,
               null, null, null, null, null, null);
-      Assert.assertEquals(1, entities.getEntities().size());
+      assertEquals(1, entities.getEntities().size());
       tEntity = entities.getEntities().get(0);
-      Assert.assertEquals(t.taskID.toString(), tEntity.getEntityId());
-      Assert.assertEquals(1, tEntity.getEvents().size());
-      Assert.assertEquals(EventType.TASK_STARTED.toString(),
-              tEntity.getEvents().get(0).getEventType());
-      Assert.assertEquals(TaskType.MAP.toString(),
-              tEntity.getEvents().get(0).getEventInfo().get("TASK_TYPE"));
+      assertEquals(t.taskID.toString(), tEntity.getEntityId());
+      assertEquals(1, tEntity.getEvents().size());
+      assertEquals(EventType.TASK_STARTED.toString(), tEntity.getEvents().get(0).getEventType());
+      assertEquals(TaskType.MAP.toString(),
+          tEntity.getEvents().get(0).getEventInfo().get("TASK_TYPE"));
 
       handleEvent(jheh, new JobHistoryEvent(t.jobId,
             new TaskStartedEvent(t.taskID, 0, TaskType.REDUCE, "")));
       jheh.getDispatcher().await();
       entities = ts.getEntities("MAPREDUCE_TASK", null, null, null,
               null, null, null, null, null, null);
-      Assert.assertEquals(1, entities.getEntities().size());
+      assertEquals(1, entities.getEntities().size());
       tEntity = entities.getEntities().get(0);
-      Assert.assertEquals(t.taskID.toString(), tEntity.getEntityId());
-      Assert.assertEquals(2, tEntity.getEvents().size());
-      Assert.assertEquals(EventType.TASK_STARTED.toString(),
-              tEntity.getEvents().get(1).getEventType());
-      Assert.assertEquals(TaskType.REDUCE.toString(),
-              tEntity.getEvents().get(0).getEventInfo().get("TASK_TYPE"));
-      Assert.assertEquals(TaskType.MAP.toString(),
-              tEntity.getEvents().get(1).getEventInfo().get("TASK_TYPE"));
+      assertEquals(t.taskID.toString(), tEntity.getEntityId());
+      assertEquals(2, tEntity.getEvents().size());
+      assertEquals(EventType.TASK_STARTED.toString(), tEntity.getEvents().get(1).getEventType());
+      assertEquals(TaskType.REDUCE.toString(),
+          tEntity.getEvents().get(0).getEventInfo().get("TASK_TYPE"));
+      assertEquals(TaskType.MAP.toString(),
+          tEntity.getEvents().get(1).getEventInfo().get("TASK_TYPE"));
     }
   }
 
-  @Test (timeout=50000)
+  @Test
+  @Timeout(value = 50)
   public void testCountersToJSON() throws Exception {
     JobHistoryEventHandler jheh = new JobHistoryEventHandler(null, 0);
     Counters counters = new Counters();
@@ -775,30 +755,31 @@ public class TestJobHistoryEventHandler {
         + "{\"NAME\":\"MATT_SMITH\",\"DISPLAY_NAME\":\"Matt Smith\",\"VALUE\":"
         + "11},{\"NAME\":\"PETER_CAPALDI\",\"DISPLAY_NAME\":\"Peter Capaldi\","
         + "\"VALUE\":12}]}]";
-    Assert.assertEquals(expected, jsonStr);
+    assertEquals(expected, jsonStr);
   }
 
-  @Test (timeout=50000)
+  @Test
+  @Timeout(value = 50)
   public void testCountersToJSONEmpty() throws Exception {
     JobHistoryEventHandler jheh = new JobHistoryEventHandler(null, 0);
     Counters counters = null;
     JsonNode jsonNode = JobHistoryEventUtils.countersToJSON(counters);
     String jsonStr = new ObjectMapper().writeValueAsString(jsonNode);
     String expected = "[]";
-    Assert.assertEquals(expected, jsonStr);
+    assertEquals(expected, jsonStr);
 
     counters = new Counters();
     jsonNode = JobHistoryEventUtils.countersToJSON(counters);
     jsonStr = new ObjectMapper().writeValueAsString(jsonNode);
     expected = "[]";
-    Assert.assertEquals(expected, jsonStr);
+    assertEquals(expected, jsonStr);
 
     counters.addGroup("DOCTORS", "Incarnations of the Doctor");
     jsonNode = JobHistoryEventUtils.countersToJSON(counters);
     jsonStr = new ObjectMapper().writeValueAsString(jsonNode);
     expected = "[{\"NAME\":\"DOCTORS\",\"DISPLAY_NAME\":\"Incarnations of the "
         + "Doctor\",\"COUNTERS\":[]}]";
-    Assert.assertEquals(expected, jsonStr);
+    assertEquals(expected, jsonStr);
   }
 
   private void queueEvent(JHEvenHandlerForTest jheh, JobHistoryEvent event) {
@@ -912,8 +893,8 @@ public class TestJobHistoryEventHandler {
     }
     jheh.stop();
     //Make sure events were handled
-    assertTrue("handleEvent should've been called only 4 times but was "
-      + jheh.eventsHandled, jheh.eventsHandled == 4);
+    assertTrue(jheh.eventsHandled == 4, "handleEvent should've been called only 4 times but was "
+        + jheh.eventsHandled);
 
     //Create a new jheh because the last stop closed the eventWriter etc.
     jheh = new JHEventHandlerForSigtermTest(mockedContext, 0);
@@ -934,14 +915,14 @@ public class TestJobHistoryEventHandler {
     }
     jheh.stop();
     //Make sure events were handled, 4 + 1 finish event
-    assertTrue("handleEvent should've been called only 5 times but was "
-        + jheh.eventsHandled, jheh.eventsHandled == 5);
-    assertTrue("Last event handled wasn't JobUnsuccessfulCompletionEvent",
-        jheh.lastEventHandled.getHistoryEvent()
-        instanceof JobUnsuccessfulCompletionEvent);
+    assertTrue(jheh.eventsHandled == 5, "handleEvent should've been called only 5 times but was "
+        + jheh.eventsHandled);
+    assertTrue(jheh.lastEventHandled.getHistoryEvent() instanceof JobUnsuccessfulCompletionEvent,
+        "Last event handled wasn't JobUnsuccessfulCompletionEvent");
   }
 
-  @Test (timeout=50000)
+  @Test
+  @Timeout(value = 50)
   public void testSetTrackingURLAfterHistoryIsWritten() throws Exception {
     TestParams t = new TestParams(true);
     Configuration conf = new Configuration();
@@ -972,7 +953,8 @@ public class TestJobHistoryEventHandler {
     }
   }
 
-  @Test (timeout=50000)
+  @Test
+  @Timeout(value = 50)
   public void testDontSetTrackingURLIfHistoryWriteFailed() throws Exception {
     TestParams t = new TestParams(true);
     Configuration conf = new Configuration();
@@ -1003,7 +985,8 @@ public class TestJobHistoryEventHandler {
       jheh.stop();
     }
   }
-  @Test (timeout=50000)
+  @Test
+  @Timeout(value = 50)
   public void testDontSetTrackingURLIfHistoryWriteThrows() throws Exception {
     TestParams t = new TestParams(true);
     Configuration conf = new Configuration();
@@ -1039,7 +1022,8 @@ public class TestJobHistoryEventHandler {
     }
   }
 
-  @Test(timeout = 50000)
+  @Test
+  @Timeout(value = 50)
   public void testJobHistoryFilePermissions() throws Exception {
     TestParams t = new TestParams(true);
     Configuration conf = new Configuration();

+ 8 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobSummary.java

@@ -19,12 +19,13 @@
 package org.apache.hadoop.mapreduce.jobhistory;
 
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -34,7 +35,7 @@ public class TestJobSummary {
       LoggerFactory.getLogger(TestJobSummary.class);
   private JobSummary summary = new JobSummary();
 
-  @Before
+  @BeforeEach
   public void before() {
     JobId mockJobId = mock(JobId.class);
     when(mockJobId.toString()).thenReturn("testJobId");
@@ -64,8 +65,8 @@ public class TestJobSummary {
     summary.setJobName("aa\rbb\ncc\r\ndd");
     String out = summary.getJobSummaryString();
     LOG.info("summary: " + out);
-    Assert.assertFalse(out.contains("\r"));
-    Assert.assertFalse(out.contains("\n"));
-    Assert.assertTrue(out.contains("aa\\rbb\\ncc\\r\\ndd"));
+    assertFalse(out.contains("\r"));
+    assertFalse(out.contains("\n"));
+    assertTrue(out.contains("aa\\rbb\\ncc\\r\\ndd"));
   }
 }

+ 4 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/api/records/TestTaskAttemptReport.java

@@ -24,12 +24,12 @@ import org.apache.hadoop.mapreduce.v2.app.MockJobs;
 import org.apache.hadoop.mapreduce.v2.proto.MRProtos;
 import org.apache.hadoop.yarn.util.Records;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class TestTaskAttemptReport {
 

+ 4 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/api/records/TestTaskReport.java

@@ -24,12 +24,12 @@ import org.apache.hadoop.mapreduce.v2.app.MockJobs;
 import org.apache.hadoop.mapreduce.v2.proto.MRProtos;
 import org.apache.hadoop.yarn.util.Records;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class TestTaskReport {
 

+ 32 - 41
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java

@@ -98,10 +98,11 @@ import org.apache.hadoop.yarn.state.StateMachine;
 import org.apache.hadoop.yarn.state.StateMachineFactory;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.SystemClock;
-import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 /**
  * Mock MRAppMaster. Doesn't start RPC servers.
@@ -326,8 +327,8 @@ public class MRApp extends MRAppMaster {
       iState = job.getInternalState();
     }
     LOG.info("Job {} Internal State is : {}", job.getID(), iState);
-    Assert.assertEquals("Task Internal state is not correct (timedout)",
-        finalState, iState);
+    assertEquals(
+        finalState, iState, "Task Internal state is not correct (timeout)");
   }
 
   public void waitForInternalState(TaskImpl task,
@@ -339,8 +340,8 @@ public class MRApp extends MRAppMaster {
       iState = task.getInternalState();
     }
     LOG.info("Task {} Internal State is : {}", task.getID(), iState);
-    Assert.assertEquals("Task Internal state is not correct (timedout)",
-        finalState, iState);
+    assertEquals(
+        finalState, iState, "Task Internal state is not correct (timeout)");
   }
 
   public void waitForInternalState(TaskAttemptImpl attempt,
@@ -352,8 +353,8 @@ public class MRApp extends MRAppMaster {
       iState = attempt.getInternalState();
     }
     LOG.info("TaskAttempt {} Internal State is : {}", attempt.getID(), iState);
-    Assert.assertEquals("TaskAttempt Internal state is not correct (timedout)",
-        finalState, iState);
+    assertEquals(finalState, iState,
+        "TaskAttempt Internal state is not correct (timeout)");
   }
 
   public void waitForState(TaskAttempt attempt, 
@@ -367,9 +368,8 @@ public class MRApp extends MRAppMaster {
     }
     LOG.info("TaskAttempt {} State is : {}", attempt.getID(),
         report.getTaskAttemptState());
-    Assert.assertEquals("TaskAttempt state is not correct (timedout)",
-        finalState,
-        report.getTaskAttemptState());
+    assertEquals(finalState,
+        report.getTaskAttemptState(), "TaskAttempt state is not correct (timeout)");
   }
 
   public void waitForState(Task task, TaskState finalState) throws Exception {
@@ -381,8 +381,8 @@ public class MRApp extends MRAppMaster {
       report = task.getReport();
     }
     LOG.info("Task {} State is : {}", task.getID(), report.getTaskState());
-    Assert.assertEquals("Task state is not correct (timedout)", finalState,
-        report.getTaskState());
+    assertEquals(finalState,
+        report.getTaskState(), "Task state is not correct (timeout)");
   }
 
   public void waitForState(Job job, JobState finalState) throws Exception {
@@ -394,14 +394,14 @@ public class MRApp extends MRAppMaster {
       Thread.sleep(WAIT_FOR_STATE_INTERVAL);
     }
     LOG.info("Job {} State is : {}", job.getID(), report.getJobState());
-    Assert.assertEquals("Job state is not correct (timedout)", finalState, 
-        job.getState());
+    assertEquals(finalState,
+        job.getState(), "Job state is not correct (timeout)");
   }
 
   public void waitForState(Service.STATE finalState) throws Exception {
     if (finalState == Service.STATE.STOPPED) {
-       Assert.assertTrue("Timeout while waiting for MRApp to stop",
-           waitForServiceToStop(20 * 1000));
+      assertTrue(waitForServiceToStop(20 * 1000),
+          "Timeout while waiting for MRApp to stop");
     } else {
       int timeoutSecs = 0;
       while (!finalState.equals(getServiceState())
@@ -409,8 +409,8 @@ public class MRApp extends MRAppMaster {
         Thread.sleep(WAIT_FOR_STATE_INTERVAL);
       }
       LOG.info("MRApp State is : {}", getServiceState());
-      Assert.assertEquals("MRApp state is not correct (timedout)", finalState,
-          getServiceState());
+      assertEquals(finalState, getServiceState(),
+          "MRApp state is not correct (timeout)");
     }
   }
 
@@ -418,23 +418,23 @@ public class MRApp extends MRAppMaster {
     for (Job job : getContext().getAllJobs().values()) {
       JobReport jobReport = job.getReport();
       LOG.info("Job start time :{}", jobReport.getStartTime());
-      LOG.info("Job finish time :", jobReport.getFinishTime());
-      Assert.assertTrue("Job start time is not less than finish time",
-          jobReport.getStartTime() <= jobReport.getFinishTime());
-      Assert.assertTrue("Job finish time is in future",
-          jobReport.getFinishTime() <= System.currentTimeMillis());
+      LOG.info("Job finish time :{}", jobReport.getFinishTime());
+      assertTrue(jobReport.getStartTime() <= jobReport.getFinishTime(),
+          "Job start time is not less than finish time");
+      assertTrue(jobReport.getFinishTime() <= System.currentTimeMillis(),
+          "Job finish time is in future");
       for (Task task : job.getTasks().values()) {
         TaskReport taskReport = task.getReport();
         LOG.info("Task {} start time : {}", task.getID(),
             taskReport.getStartTime());
         LOG.info("Task {} finish time : {}", task.getID(),
             taskReport.getFinishTime());
-        Assert.assertTrue("Task start time is not less than finish time",
-            taskReport.getStartTime() <= taskReport.getFinishTime());
+        assertTrue(taskReport.getStartTime() <= taskReport.getFinishTime(),
+            "Task start time is not less than finish time");
         for (TaskAttempt attempt : task.getAttempts().values()) {
           TaskAttemptReport attemptReport = attempt.getReport();
-          Assert.assertTrue("Attempt start time is not less than finish time",
-              attemptReport.getStartTime() <= attemptReport.getFinishTime());
+          assertTrue(attemptReport.getStartTime() <= attemptReport.getFinishTime(),
+              "Attempt start time is not less than finish time");
         }
       }
     }
@@ -443,7 +443,7 @@ public class MRApp extends MRAppMaster {
   @Override
   protected Job createJob(Configuration conf, JobStateInternal forcedState, 
       String diagnostic) {
-    UserGroupInformation currentUser = null;
+    UserGroupInformation currentUser;
     try {
       currentUser = UserGroupInformation.getCurrentUser();
     } catch (IOException e) {
@@ -455,15 +455,10 @@ public class MRApp extends MRAppMaster {
             getCommitter(), isNewApiCommitter(),
             currentUser.getUserName(), getContext(),
             forcedState, diagnostic);
-    ((AppContext) getContext()).getAllJobs().put(newJob.getID(), newJob);
+    getContext().getAllJobs().put(newJob.getID(), newJob);
 
     getDispatcher().register(JobFinishEvent.Type.class,
-        new EventHandler<JobFinishEvent>() {
-          @Override
-          public void handle(JobFinishEvent event) {
-            stop();
-          }
-        });
+        (EventHandler<JobFinishEvent>) event -> stop());
 
     return newJob;
   }
@@ -507,11 +502,7 @@ public class MRApp extends MRAppMaster {
   @Override
   protected EventHandler<JobHistoryEvent> createJobHistoryHandler(
       AppContext context) {//disable history
-    return new EventHandler<JobHistoryEvent>() {
-      @Override
-      public void handle(JobHistoryEvent event) {
-      }
-    };
+    return event -> {};
   }
   
   @Override
@@ -790,7 +781,7 @@ public class MRApp extends MRAppMaster {
   public static ContainerTokenIdentifier newContainerTokenIdentifier(
       Token containerToken) throws IOException {
     org.apache.hadoop.security.token.Token<ContainerTokenIdentifier> token =
-        new org.apache.hadoop.security.token.Token<ContainerTokenIdentifier>(
+        new org.apache.hadoop.security.token.Token<>(
             containerToken.getIdentifier()
                 .array(), containerToken.getPassword().array(), new Text(
                 containerToken.getKind()),

+ 8 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java

@@ -56,7 +56,8 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.util.Records;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.slf4j.event.Level;
 
 public class MRAppBenchmark {
@@ -150,8 +151,8 @@ public class MRAppBenchmark {
                         .getApplicationAttemptId(), containerCount++);
 
                   //System.out.println("Allocating " + containerCount);
-                  
-                  Container container = 
+
+                  Container container =
                       recordFactory.newRecordInstance(Container.class);
                   container.setId(cId);
                   NodeId nodeId = NodeId.newInstance("dummy", 1234);
@@ -196,7 +197,8 @@ public class MRAppBenchmark {
     }
   }
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void benchmark1() throws Exception {
     int maps = 100; // Adjust for benchmarking. Start with thousands.
     int reduces = 0;
@@ -275,7 +277,8 @@ public class MRAppBenchmark {
     });
   }
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void benchmark2() throws Exception {
     int maps = 100; // Adjust for benchmarking, start with a couple of thousands
     int reduces = 50;

+ 7 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestAMInfos.java

@@ -21,8 +21,6 @@ package org.apache.hadoop.mapreduce.v2.app;
 import java.util.Iterator;
 import java.util.List;
 
-import org.junit.Assert;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
@@ -33,7 +31,9 @@ import org.apache.hadoop.mapreduce.v2.app.TestRecovery.MRAppWithHistory;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.job.Task;
 import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 public class TestAMInfos {
 
@@ -50,7 +50,7 @@ public class TestAMInfos {
 
     long am1StartTime = app.getAllAMInfos().get(0).getStartTime();
 
-    Assert.assertEquals("No of tasks not correct", 1, job.getTasks().size());
+    assertEquals(1, job.getTasks().size(), "No of tasks not correct");
     Iterator<Task> it = job.getTasks().values().iterator();
     Task mapTask = it.next();
     app.waitForState(mapTask, TaskState.RUNNING);
@@ -71,14 +71,14 @@ public class TestAMInfos {
     conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
     job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
-    Assert.assertEquals("No of tasks not correct", 1, job.getTasks().size());
+    assertEquals(1, job.getTasks().size(), "No of tasks not correct");
     it = job.getTasks().values().iterator();
     mapTask = it.next();
     // There should be two AMInfos
     List<AMInfo> amInfos = app.getAllAMInfos();
-    Assert.assertEquals(2, amInfos.size());
+    assertEquals(2, amInfos.size());
     AMInfo amInfoOne = amInfos.get(0);
-    Assert.assertEquals(am1StartTime, amInfoOne.getStartTime());
+    assertEquals(am1StartTime, amInfoOne.getStartTime());
     app.stop();
   }
 }

+ 6 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestCheckpointPreemptionPolicy.java

@@ -22,8 +22,9 @@ import org.apache.hadoop.yarn.api.records.PreemptionMessage;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -58,8 +59,8 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 public class TestCheckpointPreemptionPolicy {
 
@@ -77,7 +78,7 @@ public class TestCheckpointPreemptionPolicy {
 
   private int minAlloc = 1024;
 
-  @Before
+  @BeforeEach
   @SuppressWarnings("rawtypes") // mocked generics
   public void setup() {
     ApplicationId appId = ApplicationId.newInstance(200, 1);

+ 23 - 26
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java

@@ -24,7 +24,6 @@ import java.util.Iterator;
 import java.util.Map;
 
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptFailEvent;
-import org.junit.Assert;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.TaskAttemptListenerImpl;
@@ -48,7 +47,9 @@ import org.apache.hadoop.mapreduce.v2.app.rm.preemption.AMPreemptionPolicy;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy.ContainerManagementProtocolProxyData;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 /**
  * Tests the state machine with respect to Job/Task/TaskAttempt failure 
@@ -68,20 +69,19 @@ public class TestFail {
     Job job = app.submit(conf);
     app.waitForState(job, JobState.SUCCEEDED);
     Map<TaskId,Task> tasks = job.getTasks();
-    Assert.assertEquals("Num tasks is not correct", 1, tasks.size());
+    assertEquals(1, tasks.size(), "Num tasks is not correct");
     Task task = tasks.values().iterator().next();
-    Assert.assertEquals("Task state not correct", TaskState.SUCCEEDED,
-        task.getReport().getTaskState());
+    assertEquals(TaskState.SUCCEEDED, task.getReport().getTaskState(), "Task state not correct");
     Map<TaskAttemptId, TaskAttempt> attempts =
         tasks.values().iterator().next().getAttempts();
-    Assert.assertEquals("Num attempts is not correct", 2, attempts.size());
+    assertEquals(2, attempts.size(), "Num attempts is not correct");
     //one attempt must be failed 
     //and another must have succeeded
     Iterator<TaskAttempt> it = attempts.values().iterator();
-    Assert.assertEquals("Attempt state not correct", TaskAttemptState.FAILED,
-        it.next().getReport().getTaskAttemptState());
-    Assert.assertEquals("Attempt state not correct", TaskAttemptState.SUCCEEDED,
-        it.next().getReport().getTaskAttemptState());
+    assertEquals(TaskAttemptState.FAILED, it.next().getReport().getTaskAttemptState(),
+        "Attempt state not correct");
+    assertEquals(TaskAttemptState.SUCCEEDED, it.next().getReport().getTaskAttemptState(),
+        "Attempt state not correct");
   }
 
   @Test
@@ -159,17 +159,15 @@ public class TestFail {
     Job job = app.submit(conf);
     app.waitForState(job, JobState.FAILED);
     Map<TaskId,Task> tasks = job.getTasks();
-    Assert.assertEquals("Num tasks is not correct", 1, tasks.size());
+    assertEquals(1, tasks.size(), "Num tasks is not correct");
     Task task = tasks.values().iterator().next();
-    Assert.assertEquals("Task state not correct", TaskState.FAILED,
-        task.getReport().getTaskState());
+    assertEquals(TaskState.FAILED, task.getReport().getTaskState(), "Task state not correct");
     Map<TaskAttemptId, TaskAttempt> attempts =
         tasks.values().iterator().next().getAttempts();
-    Assert.assertEquals("Num attempts is not correct", maxAttempts,
-        attempts.size());
+    assertEquals(maxAttempts, attempts.size(), "Num attempts is not correct");
     for (TaskAttempt attempt : attempts.values()) {
-      Assert.assertEquals("Attempt state not correct", TaskAttemptState.FAILED,
-          attempt.getReport().getTaskAttemptState());
+      assertEquals(TaskAttemptState.FAILED, attempt.getReport().getTaskAttemptState(),
+          "Attempt state not correct");
     }
   }
 
@@ -185,13 +183,12 @@ public class TestFail {
     Job job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
     Map<TaskId, Task> tasks = job.getTasks();
-    Assert.assertEquals("Num tasks is not correct", 1, tasks.size());
+    assertEquals(1, tasks.size(), "Num tasks is not correct");
     Task task = tasks.values().iterator().next();
     app.waitForState(task, TaskState.SCHEDULED);
     Map<TaskAttemptId, TaskAttempt> attempts = tasks.values().iterator()
         .next().getAttempts();
-    Assert.assertEquals("Num attempts is not correct", maxAttempts, attempts
-        .size());
+    assertEquals(maxAttempts, attempts.size(), "Num attempts is not correct");
     TaskAttempt attempt = attempts.values().iterator().next();
     app.waitForInternalState((TaskAttemptImpl) attempt,
         TaskAttemptStateInternal.ASSIGNED);
@@ -204,7 +201,7 @@ public class TestFail {
   static class MRAppWithFailingTaskAndUnusedContainer extends MRApp {
 
     public MRAppWithFailingTaskAndUnusedContainer() {
-      super(1, 0, false, "TaskFailWithUnsedContainer", true);
+      super(1, 0, false, "TaskFailWithUnusedContainer", true);
     }
 
     @Override
@@ -241,7 +238,7 @@ public class TestFail {
           return null;
         }
       };
-    };
+    }
   }
 
   static class TimeOutTaskMRApp extends MRApp {
@@ -258,17 +255,17 @@ public class TestFail {
       //leading to Attempt failure
       return new TaskAttemptListenerImpl(getContext(), null, null, policy) {
         @Override
-        public void startRpcServer(){};
+        public void startRpcServer(){}
         @Override
-        public void stopRpcServer(){};
+        public void stopRpcServer(){}
         @Override
         public InetSocketAddress getAddress() {
           return NetUtils.createSocketAddr("localhost", 1234);
         }
 
         protected void serviceInit(Configuration conf) throws Exception {
-          conf.setInt(MRJobConfig.TASK_TIMEOUT, 1*1000);//reduce timeout
-          conf.setInt(MRJobConfig.TASK_TIMEOUT_CHECK_INTERVAL_MS, 1*1000);
+          conf.setInt(MRJobConfig.TASK_TIMEOUT, 1000); //reduce timeout
+          conf.setInt(MRJobConfig.TASK_TIMEOUT_CHECK_INTERVAL_MS, 1000);
           conf.setDouble(MRJobConfig.TASK_LOG_PROGRESS_DELTA_THRESHOLD, 0.01);
           super.serviceInit(conf);
         }

+ 75 - 85
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFetchFailure.java

@@ -19,7 +19,8 @@
 package org.apache.hadoop.mapreduce.v2.app;
 
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -50,8 +51,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.yarn.event.EventHandler;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 public class TestFetchFailure {
 
@@ -65,8 +65,7 @@ public class TestFetchFailure {
     Job job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
     //all maps would be running
-    Assert.assertEquals("Num tasks not correct",
-       2, job.getTasks().size());
+    assertEquals(2, job.getTasks().size(), "Num tasks not correct");
     Iterator<Task> it = job.getTasks().values().iterator();
     Task mapTask = it.next();
     Task reduceTask = it.next();
@@ -97,10 +96,9 @@ public class TestFetchFailure {
 
     TaskAttemptCompletionEvent[] events =
       job.getTaskAttemptCompletionEvents(0, 100);
-    Assert.assertEquals("Num completion events not correct",
-        1, events.length);
-    Assert.assertEquals("Event status not correct",
-        TaskAttemptCompletionEventStatus.SUCCEEDED, events[0].getStatus());
+    assertEquals(1, events.length, "Num completion events not correct");
+    assertEquals(TaskAttemptCompletionEventStatus.SUCCEEDED, events[0].getStatus(),
+        "Event status not correct");
     
     // wait for reduce to start running
     app.waitForState(reduceTask, TaskState.RUNNING);
@@ -117,11 +115,11 @@ public class TestFetchFailure {
     app.waitForState(mapTask, TaskState.RUNNING);
     
     //map attempt must have become FAILED
-    Assert.assertEquals("Map TaskAttempt state not correct",
-        TaskAttemptState.FAILED, mapAttempt1.getState());
+    assertEquals(TaskAttemptState.FAILED, mapAttempt1.getState(),
+        "Map TaskAttempt state not correct");
 
-    Assert.assertEquals("Num attempts in Map Task not correct",
-        2, mapTask.getAttempts().size());
+    assertEquals(2, mapTask.getAttempts().size(),
+        "Num attempts in Map Task not correct");
     
     Iterator<TaskAttempt> atIt = mapTask.getAttempts().values().iterator();
     atIt.next();
@@ -144,39 +142,37 @@ public class TestFetchFailure {
     app.waitForState(job, JobState.SUCCEEDED);
     
     //previous completion event now becomes obsolete
-    Assert.assertEquals("Event status not correct",
-        TaskAttemptCompletionEventStatus.OBSOLETE, events[0].getStatus());
+    assertEquals(TaskAttemptCompletionEventStatus.OBSOLETE, events[0].getStatus(),
+        "Event status not correct");
     
     events = job.getTaskAttemptCompletionEvents(0, 100);
-    Assert.assertEquals("Num completion events not correct",
-        4, events.length);
-    Assert.assertEquals("Event map attempt id not correct",
-        mapAttempt1.getID(), events[0].getAttemptId());
-    Assert.assertEquals("Event map attempt id not correct",
-        mapAttempt1.getID(), events[1].getAttemptId());
-    Assert.assertEquals("Event map attempt id not correct",
-        mapAttempt2.getID(), events[2].getAttemptId());
-    Assert.assertEquals("Event redude attempt id not correct",
-        reduceAttempt.getID(), events[3].getAttemptId());
-    Assert.assertEquals("Event status not correct for map attempt1",
-        TaskAttemptCompletionEventStatus.OBSOLETE, events[0].getStatus());
-    Assert.assertEquals("Event status not correct for map attempt1",
-        TaskAttemptCompletionEventStatus.FAILED, events[1].getStatus());
-    Assert.assertEquals("Event status not correct for map attempt2",
-        TaskAttemptCompletionEventStatus.SUCCEEDED, events[2].getStatus());
-    Assert.assertEquals("Event status not correct for reduce attempt1",
-        TaskAttemptCompletionEventStatus.SUCCEEDED, events[3].getStatus());
+    assertEquals(4, events.length, "Num completion events not correct");
+    assertEquals(mapAttempt1.getID(), events[0].getAttemptId(),
+        "Event map attempt id not correct");
+    assertEquals(mapAttempt1.getID(), events[1].getAttemptId(),
+        "Event map attempt id not correct");
+    assertEquals(mapAttempt2.getID(), events[2].getAttemptId(),
+        "Event map attempt id not correct");
+    assertEquals(reduceAttempt.getID(), events[3].getAttemptId(),
+        "Event reduce attempt id not correct");
+    assertEquals(TaskAttemptCompletionEventStatus.OBSOLETE, events[0].getStatus(),
+        "Event status not correct for map attempt1");
+    assertEquals(TaskAttemptCompletionEventStatus.FAILED, events[1].getStatus(),
+        "Event status not correct for map attempt1");
+    assertEquals(TaskAttemptCompletionEventStatus.SUCCEEDED, events[2].getStatus(),
+        "Event status not correct for map attempt2");
+    assertEquals(TaskAttemptCompletionEventStatus.SUCCEEDED, events[3].getStatus(),
+        "Event status not correct for reduce attempt1");
 
     TaskCompletionEvent mapEvents[] =
         job.getMapAttemptCompletionEvents(0, 2);
     TaskCompletionEvent convertedEvents[] = TypeConverter.fromYarn(events);
-    Assert.assertEquals("Incorrect number of map events", 2, mapEvents.length);
-    Assert.assertArrayEquals("Unexpected map events",
-        Arrays.copyOfRange(convertedEvents, 0, 2), mapEvents);
+    assertEquals(2, mapEvents.length, "Incorrect number of map events");
+    assertArrayEquals(Arrays.copyOfRange(convertedEvents, 0, 2), mapEvents,
+        "Unexpected map events");
     mapEvents = job.getMapAttemptCompletionEvents(2, 200);
-    Assert.assertEquals("Incorrect number of map events", 1, mapEvents.length);
-    Assert.assertEquals("Unexpected map event", convertedEvents[2],
-        mapEvents[0]);
+    assertEquals(1, mapEvents.length, "Incorrect number of map events");
+    assertEquals(convertedEvents[2], mapEvents[0], "Unexpected map event");
   }
   
   /**
@@ -197,8 +193,7 @@ public class TestFetchFailure {
     Job job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
     //all maps would be running
-    Assert.assertEquals("Num tasks not correct",
-        2, job.getTasks().size());
+    assertEquals(2, job.getTasks().size(), "Num tasks not correct");
     Iterator<Task> it = job.getTasks().values().iterator();
     Task mapTask = it.next();
     Task reduceTask = it.next();
@@ -218,10 +213,9 @@ public class TestFetchFailure {
 
     TaskAttemptCompletionEvent[] events = 
       job.getTaskAttemptCompletionEvents(0, 100);
-    Assert.assertEquals("Num completion events not correct",
-        1, events.length);
-    Assert.assertEquals("Event status not correct",
-        TaskAttemptCompletionEventStatus.SUCCEEDED, events[0].getStatus());
+    assertEquals(1, events.length, "Num completion events not correct");
+    assertEquals(TaskAttemptCompletionEventStatus.SUCCEEDED, events[0].getStatus(),
+        "Event status not correct");
 
     // wait for reduce to start running
     app.waitForState(reduceTask, TaskState.RUNNING);
@@ -250,8 +244,7 @@ public class TestFetchFailure {
     job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
     //all maps would be running
-    Assert.assertEquals("Num tasks not correct",
-        2, job.getTasks().size());
+    assertEquals(2, job.getTasks().size(), "Num tasks not correct");
     it = job.getTasks().values().iterator();
     mapTask = it.next();
     reduceTask = it.next();
@@ -277,7 +270,7 @@ public class TestFetchFailure {
 
     app.waitForState(job, JobState.SUCCEEDED);
     events = job.getTaskAttemptCompletionEvents(0, 100);
-    Assert.assertEquals("Num completion events not correct", 2, events.length);
+    assertEquals(2, events.length, "Num completion events not correct");
   }
   
   @Test
@@ -290,8 +283,7 @@ public class TestFetchFailure {
     Job job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
     //all maps would be running
-    Assert.assertEquals("Num tasks not correct",
-       4, job.getTasks().size());
+    assertEquals(4, job.getTasks().size(), "Num tasks not correct");
     Iterator<Task> it = job.getTasks().values().iterator();
     Task mapTask = it.next();
     Task reduceTask = it.next();
@@ -313,10 +305,10 @@ public class TestFetchFailure {
     
     TaskAttemptCompletionEvent[] events = 
       job.getTaskAttemptCompletionEvents(0, 100);
-    Assert.assertEquals("Num completion events not correct",
-        1, events.length);
-    Assert.assertEquals("Event status not correct",
-        TaskAttemptCompletionEventStatus.SUCCEEDED, events[0].getStatus());
+    assertEquals(1, events.length,
+        "Num completion events not correct");
+    assertEquals(TaskAttemptCompletionEventStatus.SUCCEEDED, events[0].getStatus(),
+        "Event status not correct");
     
     // wait for reduce to start running
     app.waitForState(reduceTask, TaskState.RUNNING);
@@ -354,16 +346,16 @@ public class TestFetchFailure {
     app.waitForState(mapTask, TaskState.RUNNING);
     
     //map attempt must have become FAILED
-    Assert.assertEquals("Map TaskAttempt state not correct",
-        TaskAttemptState.FAILED, mapAttempt1.getState());
+    assertEquals(TaskAttemptState.FAILED, mapAttempt1.getState(),
+        "Map TaskAttempt state not correct");
 
     assertThat(mapAttempt1.getDiagnostics().get(0))
         .isEqualTo("Too many fetch failures. Failing the attempt. "
             + "Last failure reported by "
             + reduceAttempt3.getID().toString() + " from host host3");
 
-    Assert.assertEquals("Num attempts in Map Task not correct",
-        2, mapTask.getAttempts().size());
+    assertEquals(2, mapTask.getAttempts().size(),
+        "Num attempts in Map Task not correct");
     
     Iterator<TaskAttempt> atIt = mapTask.getAttempts().values().iterator();
     atIt.next();
@@ -396,39 +388,37 @@ public class TestFetchFailure {
     app.waitForState(job, JobState.SUCCEEDED);
     
     //previous completion event now becomes obsolete
-    Assert.assertEquals("Event status not correct",
-        TaskAttemptCompletionEventStatus.OBSOLETE, events[0].getStatus());
+    assertEquals(TaskAttemptCompletionEventStatus.OBSOLETE, events[0].getStatus(),
+        "Event status not correct");
     
     events = job.getTaskAttemptCompletionEvents(0, 100);
-    Assert.assertEquals("Num completion events not correct",
-        6, events.length);
-    Assert.assertEquals("Event map attempt id not correct",
-        mapAttempt1.getID(), events[0].getAttemptId());
-    Assert.assertEquals("Event map attempt id not correct",
-        mapAttempt1.getID(), events[1].getAttemptId());
-    Assert.assertEquals("Event map attempt id not correct",
-        mapAttempt2.getID(), events[2].getAttemptId());
-    Assert.assertEquals("Event reduce attempt id not correct",
-        reduceAttempt.getID(), events[3].getAttemptId());
-    Assert.assertEquals("Event status not correct for map attempt1",
-        TaskAttemptCompletionEventStatus.OBSOLETE, events[0].getStatus());
-    Assert.assertEquals("Event status not correct for map attempt1",
-        TaskAttemptCompletionEventStatus.FAILED, events[1].getStatus());
-    Assert.assertEquals("Event status not correct for map attempt2",
-        TaskAttemptCompletionEventStatus.SUCCEEDED, events[2].getStatus());
-    Assert.assertEquals("Event status not correct for reduce attempt1",
-        TaskAttemptCompletionEventStatus.SUCCEEDED, events[3].getStatus());
+    assertEquals(6, events.length, "Num completion events not correct");
+    assertEquals(mapAttempt1.getID(), events[0].getAttemptId(),
+        "Event map attempt id not correct");
+    assertEquals(mapAttempt1.getID(), events[1].getAttemptId(),
+        "Event map attempt id not correct");
+    assertEquals(mapAttempt2.getID(), events[2].getAttemptId(),
+        "Event map attempt id not correct");
+    assertEquals(reduceAttempt.getID(), events[3].getAttemptId(),
+        "Event reduce attempt id not correct");
+    assertEquals(TaskAttemptCompletionEventStatus.OBSOLETE, events[0].getStatus(),
+        "Event status not correct for map attempt1");
+    assertEquals(TaskAttemptCompletionEventStatus.FAILED, events[1].getStatus(),
+        "Event status not correct for map attempt1");
+    assertEquals(TaskAttemptCompletionEventStatus.SUCCEEDED, events[2].getStatus(),
+        "Event status not correct for map attempt2");
+    assertEquals(TaskAttemptCompletionEventStatus.SUCCEEDED, events[3].getStatus(),
+        "Event status not correct for reduce attempt1");
 
     TaskCompletionEvent mapEvents[] =
         job.getMapAttemptCompletionEvents(0, 2);
     TaskCompletionEvent convertedEvents[] = TypeConverter.fromYarn(events);
-    Assert.assertEquals("Incorrect number of map events", 2, mapEvents.length);
-    Assert.assertArrayEquals("Unexpected map events",
-        Arrays.copyOfRange(convertedEvents, 0, 2), mapEvents);
+    assertEquals(2, mapEvents.length, "Incorrect number of map events");
+    assertArrayEquals(Arrays.copyOfRange(convertedEvents, 0, 2), mapEvents,
+        "Unexpected map events");
     mapEvents = job.getMapAttemptCompletionEvents(2, 200);
-    Assert.assertEquals("Incorrect number of map events", 1, mapEvents.length);
-    Assert.assertEquals("Unexpected map event", convertedEvents[2],
-        mapEvents[0]);
+    assertEquals(1, mapEvents.length, "Incorrect number of map events");
+    assertEquals(convertedEvents[2], mapEvents[0], "Unexpected map event");
   }
 
   private void updateStatus(MRApp app, TaskAttempt attempt, Phase phase) {
@@ -452,7 +442,7 @@ public class TestFetchFailure {
       TaskAttempt mapAttempt, String hostname) {
     app.getContext().getEventHandler().handle(
         new JobTaskAttemptFetchFailureEvent(
-            reduceAttempt.getID(), 
+            reduceAttempt.getID(),
             Arrays.asList(new TaskAttemptId[] {mapAttempt.getID()}),
                 hostname));
   }

+ 47 - 52
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestJobEndNotifier.java

@@ -18,6 +18,10 @@
 
 package org.apache.hadoop.mapreduce.v2.app;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
@@ -59,8 +63,7 @@ import org.apache.hadoop.mapreduce.v2.app.rm.RMCommunicator;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 /**
  * Tests job end notification
@@ -74,18 +77,16 @@ public class TestJobEndNotifier extends JobEndNotifier {
     conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_MAX_ATTEMPTS, "0");
     conf.set(MRJobConfig.MR_JOB_END_RETRY_ATTEMPTS, "10");
     setConf(conf);
-    Assert.assertTrue("Expected numTries to be 0, but was " + numTries,
-      numTries == 0 );
+    assertEquals(0, numTries, "Expected numTries to be 0, but was " + numTries);
 
     conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_MAX_ATTEMPTS, "1");
     setConf(conf);
-    Assert.assertTrue("Expected numTries to be 1, but was " + numTries,
-      numTries == 1 );
+    assertEquals(1, numTries, "Expected numTries to be 1, but was " + numTries);
 
     conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_MAX_ATTEMPTS, "20");
     setConf(conf);
-    Assert.assertTrue("Expected numTries to be 11, but was " + numTries,
-      numTries == 11 ); //11 because number of _retries_ is 10
+    assertEquals(11, numTries,
+        "Expected numTries to be 11, but was " + numTries); //11 because number of _retries_ is 10
   }
 
   //Test maximum retry interval is capped by
@@ -94,54 +95,49 @@ public class TestJobEndNotifier extends JobEndNotifier {
     conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_MAX_RETRY_INTERVAL, "5000");
     conf.set(MRJobConfig.MR_JOB_END_RETRY_INTERVAL, "1000");
     setConf(conf);
-    Assert.assertTrue("Expected waitInterval to be 1000, but was "
-      + waitInterval, waitInterval == 1000);
+    assertEquals(1000, waitInterval, "Expected waitInterval to be 1000, but was " + waitInterval);
 
     conf.set(MRJobConfig.MR_JOB_END_RETRY_INTERVAL, "10000");
     setConf(conf);
-    Assert.assertTrue("Expected waitInterval to be 5000, but was "
-      + waitInterval, waitInterval == 5000);
+    assertEquals(5000, waitInterval, "Expected waitInterval to be 5000, but was " + waitInterval);
 
     //Test negative numbers are set to default
     conf.set(MRJobConfig.MR_JOB_END_RETRY_INTERVAL, "-10");
     setConf(conf);
-    Assert.assertTrue("Expected waitInterval to be 5000, but was "
-      + waitInterval, waitInterval == 5000);
+    assertEquals(5000, waitInterval, "Expected waitInterval to be 5000, but was " + waitInterval);
   }
 
   private void testTimeout(Configuration conf) {
     conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_TIMEOUT, "1000");
     setConf(conf);
-    Assert.assertTrue("Expected timeout to be 1000, but was "
-      + timeout, timeout == 1000);
+    assertEquals(1000, timeout, "Expected timeout to be 1000, but was " + timeout);
   }
 
   private void testProxyConfiguration(Configuration conf) {
     conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_PROXY, "somehost");
     setConf(conf);
-    Assert.assertTrue("Proxy shouldn't be set because port wasn't specified",
-      proxyToUse.type() == Proxy.Type.DIRECT);
+    assertTrue(proxyToUse.type() == Proxy.Type.DIRECT,
+        "Proxy shouldn't be set because port wasn't specified");
     conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_PROXY, "somehost:someport");
     setConf(conf);
-    Assert.assertTrue("Proxy shouldn't be set because port wasn't numeric",
-      proxyToUse.type() == Proxy.Type.DIRECT);
+    assertTrue(proxyToUse.type() == Proxy.Type.DIRECT,
+        "Proxy shouldn't be set because port wasn't numeric");
     conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_PROXY, "somehost:1000");
     setConf(conf);
-    Assert.assertEquals("Proxy should have been set but wasn't ",
-      "HTTP @ somehost:1000", proxyToUse.toString());
+    assertEquals("HTTP @ somehost:1000", proxyToUse.toString(),
+        "Proxy should have been set but wasn't ");
     conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_PROXY, "socks@somehost:1000");
     setConf(conf);
-    Assert.assertEquals("Proxy should have been socks but wasn't ",
-      "SOCKS @ somehost:1000", proxyToUse.toString());
+    assertEquals("SOCKS @ somehost:1000", proxyToUse.toString(),
+        "Proxy should have been socks but wasn't ");
     conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_PROXY, "SOCKS@somehost:1000");
     setConf(conf);
-    Assert.assertEquals("Proxy should have been socks but wasn't ",
-      "SOCKS @ somehost:1000", proxyToUse.toString());
+    assertEquals("SOCKS @ somehost:1000", proxyToUse.toString(),
+        "Proxy should have been socks but wasn't ");
     conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_PROXY, "sfafn@somehost:1000");
     setConf(conf);
-    Assert.assertEquals("Proxy should have been http but wasn't ",
-      "HTTP @ somehost:1000", proxyToUse.toString());
-    
+    assertEquals("HTTP @ somehost:1000", proxyToUse.toString(),
+        "Proxy should have been http but wasn't ");
   }
 
   /**
@@ -181,10 +177,10 @@ public class TestJobEndNotifier extends JobEndNotifier {
     this.setConf(conf);
     this.notify(jobReport);
     long endTime = System.currentTimeMillis();
-    Assert.assertEquals("Only 1 try was expected but was : "
-      + this.notificationCount, 1, this.notificationCount);
-    Assert.assertTrue("Should have taken more than 5 seconds it took "
-      + (endTime - startTime), endTime - startTime > 5000);
+    assertEquals(1, this.notificationCount, "Only 1 try was expected but was : "
+        + this.notificationCount);
+    assertTrue(endTime - startTime > 5000, "Should have taken more than 5 seconds it took "
+        + (endTime - startTime));
 
     conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_MAX_ATTEMPTS, "3");
     conf.set(MRJobConfig.MR_JOB_END_RETRY_ATTEMPTS, "3");
@@ -196,10 +192,10 @@ public class TestJobEndNotifier extends JobEndNotifier {
     this.setConf(conf);
     this.notify(jobReport);
     endTime = System.currentTimeMillis();
-    Assert.assertEquals("Only 3 retries were expected but was : "
-      + this.notificationCount, 3, this.notificationCount);
-    Assert.assertTrue("Should have taken more than 9 seconds it took "
-      + (endTime - startTime), endTime - startTime > 9000);
+    assertEquals(3, this.notificationCount, "Only 3 retries were expected but was : " +
+        this.notificationCount);
+    assertTrue(endTime - startTime > 9000, "Should have taken more than 9 seconds it took " +
+        (endTime - startTime));
 
   }
 
@@ -222,12 +218,11 @@ public class TestJobEndNotifier extends JobEndNotifier {
       doThrow(runtimeException).when(app).stop();
     }
     app.shutDownJob();
-    Assert.assertTrue(app.isLastAMRetry());
-    Assert.assertEquals(1, JobEndServlet.calledTimes);
-    Assert.assertEquals("jobid=" + job.getID() + "&status=SUCCEEDED",
+    assertTrue(app.isLastAMRetry());
+    assertEquals(1, JobEndServlet.calledTimes);
+    assertEquals("jobid=" + job.getID() + "&status=SUCCEEDED",
         JobEndServlet.requestUri.getQuery());
-    Assert.assertEquals(JobState.SUCCEEDED.toString(),
-        JobEndServlet.foundJobState);
+    assertEquals(JobState.SUCCEEDED.toString(), JobEndServlet.foundJobState);
     server.stop();
   }
 
@@ -262,10 +257,10 @@ public class TestJobEndNotifier extends JobEndNotifier {
     app.shutDownJob();
     // Not the last AM attempt. So user should that the job is still running.
     app.waitForState(job, JobState.RUNNING);
-    Assert.assertFalse(app.isLastAMRetry());
-    Assert.assertEquals(0, JobEndServlet.calledTimes);
-    Assert.assertNull(JobEndServlet.requestUri);
-    Assert.assertNull(JobEndServlet.foundJobState);
+    assertFalse(app.isLastAMRetry());
+    assertEquals(0, JobEndServlet.calledTimes);
+    assertNull(JobEndServlet.requestUri);
+    assertNull(JobEndServlet.foundJobState);
     server.stop();
   }
 
@@ -294,11 +289,11 @@ public class TestJobEndNotifier extends JobEndNotifier {
     // Unregistration fails: isLastAMRetry is recalculated, this is
     ///reboot will stop service internally, we don't need to shutdown twice
     app.waitForServiceToStop(10000);
-    Assert.assertFalse(app.isLastAMRetry());
+    assertFalse(app.isLastAMRetry());
     // Since it's not last retry, JobEndServlet didn't called
-    Assert.assertEquals(0, JobEndServlet.calledTimes);
-    Assert.assertNull(JobEndServlet.requestUri);
-    Assert.assertNull(JobEndServlet.foundJobState);
+    assertEquals(0, JobEndServlet.calledTimes);
+    assertNull(JobEndServlet.requestUri);
+    assertNull(JobEndServlet.foundJobState);
     server.stop();
   }
 
@@ -321,8 +316,8 @@ public class TestJobEndNotifier extends JobEndNotifier {
     this.notify(jobReport);
     final URL urlToNotify = CustomNotifier.urlToNotify;
 
-    Assert.assertEquals("http://example.com?jobId=mock-Id&jobStatus=SUCCEEDED",
-                        urlToNotify.toString());
+    assertEquals("http://example.com?jobId=mock-Id&jobStatus=SUCCEEDED",
+        urlToNotify.toString());
   }
 
   public static final class CustomNotifier implements CustomJobEndNotifier {

+ 39 - 46
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKill.java

@@ -23,7 +23,6 @@ import java.util.Map;
 import java.util.concurrent.CountDownLatch;
 
 import org.apache.hadoop.service.Service;
-import org.junit.Assert;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
@@ -48,7 +47,9 @@ import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.Event;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 /**
  * Tests the state machine with respect to Job/Task/TaskAttempt kill scenarios.
@@ -66,7 +67,7 @@ public class TestKill {
     //blocked
     Job job = app.submit(new Configuration());
     
-    //wait and vailidate for Job to become RUNNING
+    //wait and validate for Job to become RUNNING
     app.waitForInternalState((JobImpl) job, JobStateInternal.RUNNING);
 
     //send the kill signal to Job
@@ -83,18 +84,15 @@ public class TestKill {
     app.waitForState(Service.STATE.STOPPED);
 
     Map<TaskId,Task> tasks = job.getTasks();
-    Assert.assertEquals("No of tasks is not correct", 1, 
-        tasks.size());
+    assertEquals(1, tasks.size(), "No of tasks is not correct");
     Task task = tasks.values().iterator().next();
-    Assert.assertEquals("Task state not correct", TaskState.KILLED, 
-        task.getReport().getTaskState());
+    assertEquals(TaskState.KILLED, task.getReport().getTaskState(), "Task state not correct");
     Map<TaskAttemptId, TaskAttempt> attempts = 
       tasks.values().iterator().next().getAttempts();
-    Assert.assertEquals("No of attempts is not correct", 1, 
-        attempts.size());
+    assertEquals(1, attempts.size(), "No of attempts is not correct");
     Iterator<TaskAttempt> it = attempts.values().iterator();
-    Assert.assertEquals("Attempt state not correct", TaskAttemptState.KILLED, 
-          it.next().getReport().getTaskAttemptState());
+    assertEquals(TaskAttemptState.KILLED, it.next().getReport().getTaskAttemptState(),
+        "Attempt state not correct");
   }
 
   @Test
@@ -104,11 +102,10 @@ public class TestKill {
     //this will start the job but job won't complete as Task is blocked
     Job job = app.submit(new Configuration());
     
-    //wait and vailidate for Job to become RUNNING
+    //wait and validate for Job to become RUNNING
     app.waitForInternalState((JobImpl) job, JobStateInternal.RUNNING);
     Map<TaskId,Task> tasks = job.getTasks();
-    Assert.assertEquals("No of tasks is not correct", 2, 
-        tasks.size());
+    assertEquals(2, tasks.size(), "No of tasks is not correct");
     Iterator<Task> it = tasks.values().iterator();
     Task task1 = it.next();
     Task task2 = it.next();
@@ -126,23 +123,22 @@ public class TestKill {
     //first Task is killed and second is Succeeded
     //Job is succeeded
     
-    Assert.assertEquals("Task state not correct", TaskState.KILLED, 
-        task1.getReport().getTaskState());
-    Assert.assertEquals("Task state not correct", TaskState.SUCCEEDED, 
-        task2.getReport().getTaskState());
+    assertEquals(TaskState.KILLED, task1.getReport().getTaskState(),
+        "Task state not correct");
+    assertEquals(TaskState.SUCCEEDED, task2.getReport().getTaskState(),
+        "Task state not correct");
     Map<TaskAttemptId, TaskAttempt> attempts = task1.getAttempts();
-    Assert.assertEquals("No of attempts is not correct", 1, 
-        attempts.size());
+    assertEquals(1, attempts.size(),
+        "No of attempts is not correct");
     Iterator<TaskAttempt> iter = attempts.values().iterator();
-    Assert.assertEquals("Attempt state not correct", TaskAttemptState.KILLED, 
-          iter.next().getReport().getTaskAttemptState());
+    assertEquals(TaskAttemptState.KILLED, iter.next().getReport().getTaskAttemptState(),
+        "Attempt state not correct");
 
     attempts = task2.getAttempts();
-    Assert.assertEquals("No of attempts is not correct", 1, 
-        attempts.size());
+    assertEquals(1, attempts.size(), "No of attempts is not correct");
     iter = attempts.values().iterator();
-    Assert.assertEquals("Attempt state not correct", TaskAttemptState.SUCCEEDED, 
-          iter.next().getReport().getTaskAttemptState());
+    assertEquals(TaskAttemptState.SUCCEEDED, iter.next().getReport().getTaskAttemptState(),
+        "Attempt state not correct");
   }
 
   @Test
@@ -194,7 +190,7 @@ public class TestKill {
     Job job = app.submit(new Configuration());
     JobId jobId = app.getJobId();
     app.waitForState(job, JobState.RUNNING);
-    Assert.assertEquals("Num tasks not correct", 2, job.getTasks().size());
+    assertEquals(2, job.getTasks().size(), "Num tasks not correct");
     Iterator<Task> it = job.getTasks().values().iterator();
     Task mapTask = it.next();
     Task reduceTask = it.next();
@@ -232,7 +228,7 @@ public class TestKill {
     Job job = app.submit(new Configuration());
     JobId jobId = app.getJobId();
     app.waitForState(job, JobState.RUNNING);
-    Assert.assertEquals("Num tasks not correct", 2, job.getTasks().size());
+    assertEquals(2, job.getTasks().size(), "Num tasks not correct");
     Iterator<Task> it = job.getTasks().values().iterator();
     Task mapTask = it.next();
     Task reduceTask = it.next();
@@ -280,7 +276,7 @@ public class TestKill {
     Job job = app.submit(new Configuration());
     JobId jobId = app.getJobId();
     app.waitForState(job, JobState.RUNNING);
-    Assert.assertEquals("Num tasks not correct", 2, job.getTasks().size());
+    assertEquals(2, job.getTasks().size(), "Num tasks not correct");
     Iterator<Task> it = job.getTasks().values().iterator();
     Task mapTask = it.next();
     Task reduceTask = it.next();
@@ -367,11 +363,10 @@ public class TestKill {
     //this will start the job but job won't complete as Task is blocked
     Job job = app.submit(new Configuration());
     
-    //wait and vailidate for Job to become RUNNING
+    //wait and validate for Job to become RUNNING
     app.waitForState(job, JobState.RUNNING);
     Map<TaskId,Task> tasks = job.getTasks();
-    Assert.assertEquals("No of tasks is not correct", 2, 
-        tasks.size());
+    assertEquals(2, tasks.size(), "No of tasks is not correct");
     Iterator<Task> it = tasks.values().iterator();
     Task task1 = it.next();
     Task task2 = it.next();
@@ -394,26 +389,24 @@ public class TestKill {
     
     //first Task will have two attempts 1st is killed, 2nd Succeeds
     //both Tasks and Job succeeds
-    Assert.assertEquals("Task state not correct", TaskState.SUCCEEDED, 
-        task1.getReport().getTaskState());
-    Assert.assertEquals("Task state not correct", TaskState.SUCCEEDED, 
-        task2.getReport().getTaskState());
+    assertEquals(TaskState.SUCCEEDED, task1.getReport().getTaskState(),
+        "Task state not correct");
+    assertEquals(TaskState.SUCCEEDED, task2.getReport().getTaskState(),
+        "Task state not correct");
  
     Map<TaskAttemptId, TaskAttempt> attempts = task1.getAttempts();
-    Assert.assertEquals("No of attempts is not correct", 2, 
-        attempts.size());
+    assertEquals(2, attempts.size(), "No of attempts is not correct");
     Iterator<TaskAttempt> iter = attempts.values().iterator();
-    Assert.assertEquals("Attempt state not correct", TaskAttemptState.KILLED, 
-          iter.next().getReport().getTaskAttemptState());
-    Assert.assertEquals("Attempt state not correct", TaskAttemptState.SUCCEEDED, 
-        iter.next().getReport().getTaskAttemptState());
+    assertEquals(TaskAttemptState.KILLED, iter.next().getReport().getTaskAttemptState(),
+        "Attempt state not correct");
+    assertEquals(TaskAttemptState.SUCCEEDED, iter.next().getReport().getTaskAttemptState(),
+        "Attempt state not correct");
     
     attempts = task2.getAttempts();
-    Assert.assertEquals("No of attempts is not correct", 1, 
-        attempts.size());
+    assertEquals(1, attempts.size(), "No of attempts is not correct");
     iter = attempts.values().iterator();
-    Assert.assertEquals("Attempt state not correct", TaskAttemptState.SUCCEEDED, 
-          iter.next().getReport().getTaskAttemptState());
+    assertEquals(TaskAttemptState.SUCCEEDED, iter.next().getReport().getTaskAttemptState(),
+        "Attempt state not correct");
   }
 
   static class BlockingMRApp extends MRApp {

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKillAMPreemptionPolicy.java

@@ -47,7 +47,7 @@ import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 public class TestKillAMPreemptionPolicy {
   private final RecordFactory recordFactory = RecordFactoryProvider

+ 53 - 70
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRApp.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.mapreduce.v2.app;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertSame;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
@@ -30,7 +32,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import java.util.function.Supplier;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Assert;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.MRJobConfig;
@@ -68,8 +69,7 @@ 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.factory.providers.RecordFactoryProvider;
-import org.junit.Test;
-import org.mockito.Mockito;
+import org.junit.jupiter.api.Test;
 
 /**
  * Tests the state machine of MR App.
@@ -83,7 +83,7 @@ public class TestMRApp {
     Job job = app.submit(new Configuration());
     app.waitForState(job, JobState.SUCCEEDED);
     app.verifyCompleted();
-    Assert.assertEquals(System.getProperty("user.name"),job.getUserName());
+    assertEquals(System.getProperty("user.name"), job.getUserName());
   }
 
   @Test
@@ -106,7 +106,7 @@ public class TestMRApp {
     MRApp app = new MRApp(1, 0, false, this.getClass().getName(), true);
     Job job = app.submit(new Configuration());
     app.waitForState(job, JobState.RUNNING);
-    Assert.assertEquals("Num tasks not correct", 1, job.getTasks().size());
+    assertEquals(1, job.getTasks().size(), "Num tasks not correct");
     Iterator<Task> it = job.getTasks().values().iterator();
     Task task = it.next();
     app.waitForState(task, TaskState.RUNNING);
@@ -151,7 +151,7 @@ public class TestMRApp {
     Job job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
     //all maps would be running
-    Assert.assertEquals("Num tasks not correct", 3, job.getTasks().size());
+    assertEquals(3, job.getTasks().size(), "Num tasks not correct");
     Iterator<Task> it = job.getTasks().values().iterator();
     Task mapTask1 = it.next();
     Task mapTask2 = it.next();
@@ -170,8 +170,8 @@ public class TestMRApp {
     app.waitForState(task2Attempt, TaskAttemptState.RUNNING);
     
     // reduces must be in NEW state
-    Assert.assertEquals("Reduce Task state not correct",
-        TaskState.NEW, reduceTask.getReport().getTaskState());
+    assertEquals(TaskState.NEW, reduceTask.getReport().getTaskState(),
+        "Reduce Task state not correct");
     
     //send the done signal to the 1st map task
     app.getContext().getEventHandler().handle(
@@ -210,7 +210,7 @@ public class TestMRApp {
     int runCount = 0;
     AsyncDispatcher dispatcher = new AsyncDispatcher();
     dispatcher.init(new Configuration());
-    Dispatcher disp = Mockito.spy(dispatcher);
+    Dispatcher disp = spy(dispatcher);
     MRApp app = new MRAppWithHistory(2, 2, false, this.getClass().getName(),
         true, ++runCount, disp);
     Configuration conf = new Configuration();
@@ -224,7 +224,7 @@ public class TestMRApp {
 
     final Job job1 = app.submit(conf);
     app.waitForState(job1, JobState.RUNNING);
-    Assert.assertEquals("Num tasks not correct", 4, job1.getTasks().size());
+    assertEquals(4, job1.getTasks().size(), "Num tasks not correct");
     Iterator<Task> it = job1.getTasks().values().iterator();
     Task mapTask1 = it.next();
     Task mapTask2 = it.next();
@@ -239,7 +239,7 @@ public class TestMRApp {
         .next();
     NodeId node1 = task1Attempt.getNodeId();
     NodeId node2 = task2Attempt.getNodeId();
-    Assert.assertEquals(node1, node2);
+    assertEquals(node1, node2);
 
     // send the done signal to the task
     app.getContext()
@@ -260,22 +260,19 @@ public class TestMRApp {
     final int checkIntervalMillis = 100;
     final int waitForMillis = 800;
 
-    waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        TaskAttemptCompletionEvent[] events = job1
-            .getTaskAttemptCompletionEvents(0, 100);
-        return events.length == 2;
-      }
+    waitFor(() -> {
+      TaskAttemptCompletionEvent[] events = job1
+          .getTaskAttemptCompletionEvents(0, 100);
+      return events.length == 2;
     }, checkIntervalMillis, waitForMillis);
 
     TaskAttemptCompletionEvent[] events = job1.getTaskAttemptCompletionEvents
         (0, 100);
-    Assert.assertEquals("Expecting 2 completion events for success", 2,
-        events.length);
+    assertEquals(2,
+        events.length, "Expecting 2 completion events for success");
 
     // send updated nodes info
-    ArrayList<NodeReport> updatedNodes = new ArrayList<NodeReport>();
+    ArrayList<NodeReport> updatedNodes = new ArrayList<>();
     NodeReport nr = RecordFactoryProvider.getRecordFactory(null)
         .newRecordInstance(NodeReport.class);
     nr.setNodeId(node1);
@@ -287,18 +284,15 @@ public class TestMRApp {
     app.waitForState(task1Attempt, TaskAttemptState.KILLED);
     app.waitForState(task2Attempt, TaskAttemptState.KILLED);
 
-    waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        TaskAttemptCompletionEvent[] events = job1
-            .getTaskAttemptCompletionEvents(0, 100);
-        return events.length == 4;
-      }
+    waitFor(() -> {
+      TaskAttemptCompletionEvent[] events1 = job1
+          .getTaskAttemptCompletionEvents(0, 100);
+      return events1.length == 4;
     }, checkIntervalMillis, waitForMillis);
 
     events = job1.getTaskAttemptCompletionEvents(0, 100);
-    Assert.assertEquals("Expecting 2 more completion events for killed", 4,
-        events.length);
+    assertEquals(4,
+        events.length, "Expecting 2 more completion events for killed");
     // 2 map task attempts which were killed above should be requested from
     // container allocator with the previous map task marked as failed. If
     // this happens allocator will request the container for this mapper from
@@ -335,8 +329,8 @@ public class TestMRApp {
     }, checkIntervalMillis, waitForMillis);
 
     events = job1.getTaskAttemptCompletionEvents(0, 100);
-    Assert.assertEquals("Expecting 1 more completion events for success", 5,
-        events.length);
+    assertEquals(5,
+        events.length, "Expecting 1 more completion events for success");
 
     // Crash the app again.
     app.stop();
@@ -351,7 +345,7 @@ public class TestMRApp {
 
     final Job job2 = app.submit(conf);
     app.waitForState(job2, JobState.RUNNING);
-    Assert.assertEquals("No of tasks not correct", 4, job2.getTasks().size());
+    assertEquals(4, job2.getTasks().size(), "No of tasks not correct");
     it = job2.getTasks().values().iterator();
     mapTask1 = it.next();
     mapTask2 = it.next();
@@ -362,19 +356,15 @@ public class TestMRApp {
     app.waitForState(mapTask1, TaskState.SUCCEEDED);
     app.waitForState(mapTask2, TaskState.RUNNING);
 
-    waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        TaskAttemptCompletionEvent[] events = job2
-            .getTaskAttemptCompletionEvents(0, 100);
-        return events.length == 2;
-      }
+    waitFor(() -> {
+      TaskAttemptCompletionEvent[] events12 = job2
+          .getTaskAttemptCompletionEvents(0, 100);
+      return events12.length == 2;
     }, checkIntervalMillis, waitForMillis);
 
     events = job2.getTaskAttemptCompletionEvents(0, 100);
-    Assert.assertEquals(
-        "Expecting 2 completion events for killed & success of map1", 2,
-        events.length);
+    assertEquals(2, events.length,
+        "Expecting 2 completion events for killed & success of map1");
 
     task2Attempt = mapTask2.getAttempts().values().iterator().next();
     app.getContext()
@@ -384,18 +374,15 @@ public class TestMRApp {
                 TaskAttemptEventType.TA_DONE));
     app.waitForState(mapTask2, TaskState.SUCCEEDED);
 
-    waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        TaskAttemptCompletionEvent[] events = job2
-            .getTaskAttemptCompletionEvents(0, 100);
-        return events.length == 3;
-      }
+    waitFor(() -> {
+      TaskAttemptCompletionEvent[] events13 = job2
+          .getTaskAttemptCompletionEvents(0, 100);
+      return events13.length == 3;
     }, checkIntervalMillis, waitForMillis);
 
     events = job2.getTaskAttemptCompletionEvents(0, 100);
-    Assert.assertEquals("Expecting 1 more completion events for success", 3,
-        events.length);
+    assertEquals(3,
+        events.length, "Expecting 1 more completion events for success");
 
     app.waitForState(reduceTask1, TaskState.RUNNING);
     app.waitForState(reduceTask2, TaskState.RUNNING);
@@ -424,17 +411,14 @@ public class TestMRApp {
                 TaskAttemptEventType.TA_DONE));
     app.waitForState(reduceTask2, TaskState.SUCCEEDED);
 
-    waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        TaskAttemptCompletionEvent[] events = job2
-            .getTaskAttemptCompletionEvents(0, 100);
-        return events.length == 5;
-      }
+    waitFor(() -> {
+      TaskAttemptCompletionEvent[] events14 = job2
+          .getTaskAttemptCompletionEvents(0, 100);
+      return events14.length == 5;
     }, checkIntervalMillis, waitForMillis);
     events = job2.getTaskAttemptCompletionEvents(0, 100);
-    Assert.assertEquals("Expecting 2 more completion events for reduce success",
-        5, events.length);
+    assertEquals(5, events.length,
+        "Expecting 2 more completion events for reduce success");
 
     // job succeeds
     app.waitForState(job2, JobState.SUCCEEDED);
@@ -472,7 +456,7 @@ public class TestMRApp {
     MRApp app = new MRApp(1, 0, false, this.getClass().getName(), true);
     Job job = app.submit(new Configuration());
     app.waitForState(job, JobState.RUNNING);
-    Assert.assertEquals("Num tasks not correct", 1, job.getTasks().size());
+    assertEquals(1, job.getTasks().size(), "Num tasks not correct");
     Iterator<Task> it = job.getTasks().values().iterator();
     Task task = it.next();
     app.waitForState(task, TaskState.RUNNING);
@@ -493,7 +477,7 @@ public class TestMRApp {
     JobImpl job = (JobImpl) app.submit(new Configuration());
     app.waitForInternalState(job, JobStateInternal.SUCCEEDED);
     // AM is not unregistered
-    Assert.assertEquals(JobState.RUNNING, job.getState());
+    assertEquals(JobState.RUNNING, job.getState());
     // imitate that AM is unregistered
     app.successfullyUnregistered.set(true);
     app.waitForState(job, JobState.SUCCEEDED);
@@ -505,7 +489,7 @@ public class TestMRApp {
     MRApp app = new MRApp(1, 0, false, this.getClass().getName(), true);
     Job job = app.submit(new Configuration());
     app.waitForState(job, JobState.RUNNING);
-    Assert.assertEquals("Num tasks not correct", 1, job.getTasks().size());
+    assertEquals(1, job.getTasks().size(), "Num tasks not correct");
     Iterator<Task> it = job.getTasks().values().iterator();
     Task task = it.next();
     app.waitForState(task, TaskState.RUNNING);
@@ -514,7 +498,7 @@ public class TestMRApp {
     app.getContext().getEventHandler().handle(new JobEvent(job.getID(),
       JobEventType.JOB_AM_REBOOT));
 
-    // return exteranl state as RUNNING since otherwise the JobClient will
+    // return external state as RUNNING since otherwise the JobClient will
     // prematurely exit.
     app.waitForState(job, JobState.RUNNING);
   }
@@ -530,7 +514,7 @@ public class TestMRApp {
     Configuration conf = new Configuration();
     Job job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
-    Assert.assertEquals("Num tasks not correct", 1, job.getTasks().size());
+    assertEquals(1, job.getTasks().size(), "Num tasks not correct");
     Iterator<Task> it = job.getTasks().values().iterator();
     Task task = it.next();
     app.waitForState(task, TaskState.RUNNING);
@@ -540,7 +524,7 @@ public class TestMRApp {
       JobEventType.JOB_AM_REBOOT));
 
     app.waitForInternalState((JobImpl) job, JobStateInternal.REBOOT);
-    // return exteranl state as RUNNING if this is the last retry while
+    // return external state as RUNNING if this is the last retry while
     // unregistration fails
     app.waitForState(job, JobState.RUNNING);
   }
@@ -611,7 +595,7 @@ public class TestMRApp {
             super.handle(event);
           }
         };
-      };
+      }
     };
     Job job = app.submit(new Configuration());
     app.waitForState(job, JobState.SUCCEEDED);
@@ -624,8 +608,7 @@ public class TestMRApp {
         (TaskAttemptImpl) taskAttempts.iterator().next();
     // Container from RM should pass through to the launcher. Container object
     // should be the same.
-   Assert.assertTrue(taskAttempt.container 
-     == containerObtainedByContainerLauncher);
+    assertSame(taskAttempt.container, containerObtainedByContainerLauncher);
   }
 
   private final class MRAppWithHistory extends MRApp {

+ 9 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppComponentDependencies.java

@@ -20,8 +20,6 @@ package org.apache.hadoop.mapreduce.v2.app;
 
 import java.io.IOException;
 
-import org.junit.Assert;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler;
@@ -35,11 +33,15 @@ import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 public class TestMRAppComponentDependencies {
 
-  @Test(timeout = 20000)
+  @Test
+  @Timeout(value = 20)
   public void testComponentStopOrder() throws Exception {
     @SuppressWarnings("resource")
     TestMRApp app = new TestMRApp(1, 1, true, this.getClass().getName(), true);
@@ -54,8 +56,8 @@ public class TestMRAppComponentDependencies {
     }
 
     // assert JobHistoryEventHandlerStopped and then clientServiceStopped
-    Assert.assertEquals(1, app.JobHistoryEventHandlerStopped);
-    Assert.assertEquals(2, app.clientServiceStopped);
+    assertEquals(1, app.JobHistoryEventHandlerStopped);
+    assertEquals(2, app.clientServiceStopped);
   }
 
   private final class TestMRApp extends MRApp {
@@ -74,7 +76,7 @@ public class TestMRAppComponentDependencies {
     @Override
     protected Job createJob(Configuration conf, JobStateInternal forcedState,
         String diagnostic) {
-      UserGroupInformation currentUser = null;
+      UserGroupInformation currentUser;
       try {
         currentUser = UserGroupInformation.getCurrentUser();
       } catch (IOException e) {

+ 46 - 46
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java

@@ -18,9 +18,10 @@
 package org.apache.hadoop.mapreduce.v2.app;
 
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.ArgumentMatchers.anyBoolean;
 import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.doReturn;
@@ -40,7 +41,6 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.junit.Assert;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContext;
@@ -84,10 +84,11 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.mockito.ArgumentCaptor;
 import org.mockito.Mockito;
 import org.slf4j.event.Level;
@@ -104,7 +105,7 @@ public class TestMRAppMaster {
   static String stagingDir = new Path(testDir, "staging").toString();
   private static FileContext localFS = null;
 
-  @BeforeClass
+  @BeforeAll
   public static void setup() throws AccessControlException,
       FileNotFoundException, IllegalArgumentException, IOException {
     //Do not error out if metrics are inited multiple times
@@ -116,7 +117,7 @@ public class TestMRAppMaster {
     new File(testDir.toString()).mkdir();
   }
 
-  @Before
+  @BeforeEach
   public void prepare() throws IOException {
     File dir = new File(stagingDir);
     if(dir.exists()) {
@@ -125,7 +126,7 @@ public class TestMRAppMaster {
     dir.mkdirs();
   }
 
-  @AfterClass
+  @AfterAll
   public static void cleanup() throws IOException {
     localFS.delete(testDir, true);
   }
@@ -226,8 +227,8 @@ public class TestMRAppMaster {
             "host", -1, -1, System.currentTimeMillis());
     MRAppMaster.initAndStartAppMaster(appMaster, conf, userName);
     appMaster.stop();
-    assertTrue("Job launch time should not be negative.",
-            appMaster.jobLaunchTime.get() >= 0);
+    assertTrue(
+           appMaster.jobLaunchTime.get() >= 0, "Job launch time should not be negative.");
   }
 
   @Test
@@ -343,7 +344,8 @@ public class TestMRAppMaster {
     appMaster.stop();
   }
 
-  @Test (timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testMRAppMasterMaxAppAttempts() throws IOException,
       InterruptedException {
     // No matter what's the maxAppAttempt or attempt id, the isLastRetry always
@@ -368,8 +370,8 @@ public class TestMRAppMaster {
           new MRAppMasterTest(applicationAttemptId, containerId, "host", -1, -1,
               System.currentTimeMillis(), false, true);
       MRAppMaster.initAndStartAppMaster(appMaster, conf, userName);
-      assertEquals("isLastAMRetry is correctly computed.", expectedBools[i],
-          appMaster.isLastAMRetry());
+      assertEquals(expectedBools[i],
+          appMaster.isLastAMRetry(), "isLastAMRetry is correctly computed.");
     }
   }
 
@@ -418,14 +420,14 @@ public class TestMRAppMaster {
     Text kind = new Text("MyTokenKind");
     Text service = new Text("host:port");
     Token<? extends TokenIdentifier> myToken =
-        new Token<TokenIdentifier>(identifier, password, kind, service);
+        new Token<>(identifier, password, kind, service);
     Text tokenAlias = new Text("myToken");
     credentials.addToken(tokenAlias, myToken);
 
     Text appTokenService = new Text("localhost:0");
     Token<AMRMTokenIdentifier> appToken =
-        new Token<AMRMTokenIdentifier>(identifier, password,
-            AMRMTokenIdentifier.KIND_NAME, appTokenService);
+        new Token<>(identifier, password,
+        AMRMTokenIdentifier.KIND_NAME, appTokenService);
     credentials.addToken(appTokenService, appToken);
 
     Text keyAlias = new Text("mySecretKeyAlias");
@@ -436,7 +438,7 @@ public class TestMRAppMaster {
     JobConf conf = new JobConf();
 
     Path tokenFilePath = new Path(testDir, "tokens-file");
-    Map<String, String> newEnv = new HashMap<String, String>();
+    Map<String, String> newEnv = new HashMap<>();
     newEnv.put(UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION, tokenFilePath
       .toUri().getPath());
     setNewEnvironmentHack(newEnv);
@@ -465,38 +467,36 @@ public class TestMRAppMaster {
 
     // Now validate the task credentials
     Credentials appMasterCreds = appMaster.getCredentials();
-    Assert.assertNotNull(appMasterCreds);
-    Assert.assertEquals(1, appMasterCreds.numberOfSecretKeys());
-    Assert.assertEquals(1, appMasterCreds.numberOfTokens());
+    assertNotNull(appMasterCreds);
+    assertEquals(1, appMasterCreds.numberOfSecretKeys());
+    assertEquals(1, appMasterCreds.numberOfTokens());
 
     // Validate the tokens - app token should not be present
     Token<? extends TokenIdentifier> usedToken =
         appMasterCreds.getToken(tokenAlias);
-    Assert.assertNotNull(usedToken);
-    Assert.assertEquals(storedToken, usedToken);
+    assertNotNull(usedToken);
+    assertEquals(storedToken, usedToken);
 
     // Validate the keys
     byte[] usedKey = appMasterCreds.getSecretKey(keyAlias);
-    Assert.assertNotNull(usedKey);
-    Assert.assertEquals("mySecretKey", new String(usedKey));
+    assertNotNull(usedKey);
+    assertEquals("mySecretKey", new String(usedKey));
 
     // The credentials should also be added to conf so that OuputCommitter can
     // access it - app token should not be present
     Credentials confCredentials = conf.getCredentials();
-    Assert.assertEquals(1, confCredentials.numberOfSecretKeys());
-    Assert.assertEquals(1, confCredentials.numberOfTokens());
-    Assert.assertEquals(storedToken, confCredentials.getToken(tokenAlias));
-    Assert.assertEquals("mySecretKey",
-      new String(confCredentials.getSecretKey(keyAlias)));
+    assertEquals(1, confCredentials.numberOfSecretKeys());
+    assertEquals(1, confCredentials.numberOfTokens());
+    assertEquals(storedToken, confCredentials.getToken(tokenAlias));
+    assertEquals("mySecretKey", new String(confCredentials.getSecretKey(keyAlias)));
 
     // Verify the AM's ugi - app token should be present
     Credentials ugiCredentials = appMaster.getUgi().getCredentials();
-    Assert.assertEquals(1, ugiCredentials.numberOfSecretKeys());
-    Assert.assertEquals(2, ugiCredentials.numberOfTokens());
-    Assert.assertEquals(storedToken, ugiCredentials.getToken(tokenAlias));
-    Assert.assertEquals(appToken, ugiCredentials.getToken(appTokenService));
-    Assert.assertEquals("mySecretKey",
-      new String(ugiCredentials.getSecretKey(keyAlias)));
+    assertEquals(1, ugiCredentials.numberOfSecretKeys());
+    assertEquals(2, ugiCredentials.numberOfTokens());
+    assertEquals(storedToken, ugiCredentials.getToken(tokenAlias));
+    assertEquals(appToken, ugiCredentials.getToken(appTokenService));
+    assertEquals("mySecretKey", new String(ugiCredentials.getSecretKey(keyAlias)));
 
 
   }
@@ -525,10 +525,10 @@ public class TestMRAppMaster {
     doNothing().when(appMaster).serviceStop();
     // Test normal shutdown.
     appMaster.shutDownJob();
-    Assert.assertTrue("Expected shutDownJob to terminate.",
-                      ExitUtil.terminateCalled());
-    Assert.assertEquals("Expected shutDownJob to exit with status code of 0.",
-        0, ExitUtil.getFirstExitException().status);
+    assertTrue(ExitUtil.terminateCalled(),
+        "Expected shutDownJob to terminate.");
+    assertEquals(0, ExitUtil.getFirstExitException().status,
+        "Expected shutDownJob to exit with status code of 0.");
 
     // Test shutdown with exception.
     ExitUtil.resetFirstExitException();
@@ -536,10 +536,10 @@ public class TestMRAppMaster {
     doThrow(new RuntimeException(msg))
             .when(appMaster).notifyIsLastAMRetry(anyBoolean());
     appMaster.shutDownJob();
-    assertTrue("Expected message from ExitUtil.ExitException to be " + msg,
-        ExitUtil.getFirstExitException().getMessage().contains(msg));
-    Assert.assertEquals("Expected shutDownJob to exit with status code of 1.",
-        1, ExitUtil.getFirstExitException().status);
+    assertTrue(ExitUtil.getFirstExitException().getMessage().contains(msg),
+        "Expected message from ExitUtil.ExitException to be " + msg);
+    assertEquals(1, ExitUtil.getFirstExitException().status,
+        "Expected shutDownJob to exit with status code of 1.");
   }
 
   private void verifyFailedStatus(MRAppMasterTest appMaster,

+ 44 - 52
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRClientService.java

@@ -18,7 +18,11 @@
 
 package org.apache.hadoop.mapreduce.v2.app;
 
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
@@ -26,8 +30,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicReference;
 
-import org.junit.Assert;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.JobID;
@@ -70,7 +72,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 public class TestMRClientService {
 
@@ -82,7 +84,7 @@ public class TestMRClientService {
     Configuration conf = new Configuration();
     Job job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
-    Assert.assertEquals("Num tasks not correct", 1, job.getTasks().size());
+    assertEquals(1, job.getTasks().size(), "Num tasks not correct");
     Iterator<Task> it = job.getTasks().values().iterator();
     Task task = it.next();
     app.waitForState(task, TaskState.RUNNING);
@@ -116,8 +118,7 @@ public class TestMRClientService {
     GetCountersRequest gcRequest =
         recordFactory.newRecordInstance(GetCountersRequest.class);    
     gcRequest.setJobId(job.getID());
-    Assert.assertNotNull("Counters is null",
-        proxy.getCounters(gcRequest).getCounters());
+    assertNotNull(proxy.getCounters(gcRequest).getCounters(), "Counters is null");
 
     GetJobReportRequest gjrRequest =
         recordFactory.newRecordInstance(GetJobReportRequest.class);
@@ -131,14 +132,14 @@ public class TestMRClientService {
     gtaceRequest.setJobId(job.getID());
     gtaceRequest.setFromEventId(0);
     gtaceRequest.setMaxEvents(10);
-    Assert.assertNotNull("TaskCompletionEvents is null", 
-        proxy.getTaskAttemptCompletionEvents(gtaceRequest).getCompletionEventList());
+    assertNotNull(proxy.getTaskAttemptCompletionEvents(gtaceRequest).getCompletionEventList(),
+        "TaskCompletionEvents is null");
 
     GetDiagnosticsRequest gdRequest =
         recordFactory.newRecordInstance(GetDiagnosticsRequest.class);
     gdRequest.setTaskAttemptId(attempt.getID());
-    Assert.assertNotNull("Diagnostics is null", 
-        proxy.getDiagnostics(gdRequest).getDiagnosticsList());
+    assertNotNull(proxy.getDiagnostics(gdRequest).getDiagnosticsList(),
+        "Diagnostics is null");
 
     GetTaskAttemptReportRequest gtarRequest =
         recordFactory.newRecordInstance(GetTaskAttemptReportRequest.class);
@@ -151,31 +152,29 @@ public class TestMRClientService {
     GetTaskReportRequest gtrRequest =
         recordFactory.newRecordInstance(GetTaskReportRequest.class);
     gtrRequest.setTaskId(task.getID());
-    Assert.assertNotNull("TaskReport is null", 
-        proxy.getTaskReport(gtrRequest).getTaskReport());
+    assertNotNull(proxy.getTaskReport(gtrRequest).getTaskReport(),
+        "TaskReport is null");
 
     GetTaskReportsRequest gtreportsRequest =
         recordFactory.newRecordInstance(GetTaskReportsRequest.class);
     gtreportsRequest.setJobId(job.getID());
     gtreportsRequest.setTaskType(TaskType.MAP);
-    Assert.assertNotNull("TaskReports for map is null", 
-        proxy.getTaskReports(gtreportsRequest).getTaskReportList());
+    assertNotNull(proxy.getTaskReports(gtreportsRequest).getTaskReportList(),
+        "TaskReports for map is null");
 
     gtreportsRequest =
         recordFactory.newRecordInstance(GetTaskReportsRequest.class);
     gtreportsRequest.setJobId(job.getID());
     gtreportsRequest.setTaskType(TaskType.REDUCE);
-    Assert.assertNotNull("TaskReports for reduce is null", 
-        proxy.getTaskReports(gtreportsRequest).getTaskReportList());
+    assertNotNull(proxy.getTaskReports(gtreportsRequest).getTaskReportList(),
+        "TaskReports for reduce is null");
 
     List<String> diag = proxy.getDiagnostics(gdRequest).getDiagnosticsList();
-    Assert.assertEquals("Num diagnostics not correct", 1 , diag.size());
-    Assert.assertEquals("Diag 1 not correct",
-        diagnostic1, diag.get(0).toString());
+    assertEquals(1, diag.size(), "Num diagnostics not correct");
+    assertEquals(diagnostic1, diag.get(0), "Diag 1 not correct");
 
     TaskReport taskReport = proxy.getTaskReport(gtrRequest).getTaskReport();
-    Assert.assertEquals("Num diagnostics not correct", 1,
-        taskReport.getDiagnosticsCount());
+    assertEquals(1, taskReport.getDiagnosticsCount(), "Num diagnostics not correct");
 
     //send the done signal to the task
     app.getContext().getEventHandler().handle(
@@ -207,7 +206,7 @@ public class TestMRClientService {
     conf.set(MRJobConfig.JOB_ACL_VIEW_JOB, "viewonlyuser");
     Job job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
-    Assert.assertEquals("Num tasks not correct", 1, job.getTasks().size());
+    assertEquals(1, job.getTasks().size(), "Num tasks not correct");
     Iterator<Task> it = job.getTasks().values().iterator();
     Task task = it.next();
     app.waitForState(task, TaskState.RUNNING);
@@ -217,18 +216,13 @@ public class TestMRClientService {
     UserGroupInformation viewOnlyUser =
         UserGroupInformation.createUserForTesting(
             "viewonlyuser", new String[] {});
-    Assert.assertTrue("viewonlyuser cannot view job",
-        job.checkAccess(viewOnlyUser, JobACL.VIEW_JOB));
-    Assert.assertFalse("viewonlyuser can modify job",
-        job.checkAccess(viewOnlyUser, JobACL.MODIFY_JOB));
+    assertTrue(job.checkAccess(viewOnlyUser, JobACL.VIEW_JOB), "viewonlyuser cannot view job");
+    assertFalse(job.checkAccess(viewOnlyUser, JobACL.MODIFY_JOB), "viewonlyuser can modify job");
     MRClientProtocol client = viewOnlyUser.doAs(
-        new PrivilegedExceptionAction<MRClientProtocol>() {
-          @Override
-          public MRClientProtocol run() throws Exception {
-            YarnRPC rpc = YarnRPC.create(conf);
-            return (MRClientProtocol) rpc.getProxy(MRClientProtocol.class,
-                app.clientService.getBindAddress(), conf);
-          }
+        (PrivilegedExceptionAction<MRClientProtocol>) () -> {
+          YarnRPC rpc = YarnRPC.create(conf);
+          return (MRClientProtocol) rpc.getProxy(MRClientProtocol.class,
+              app.clientService.getBindAddress(), conf);
         });
 
     KillJobRequest killJobRequest = recordFactory.newRecordInstance(
@@ -273,29 +267,27 @@ public class TestMRClientService {
   }
 
   private void verifyJobReport(JobReport jr) {
-    Assert.assertNotNull("JobReport is null", jr);
+    assertNotNull(jr, "JobReport is null");
     List<AMInfo> amInfos = jr.getAMInfos();
-    Assert.assertEquals(1, amInfos.size());
-    Assert.assertEquals(JobState.RUNNING, jr.getJobState());
+    assertEquals(1, amInfos.size());
+    assertEquals(JobState.RUNNING, jr.getJobState());
     AMInfo amInfo = amInfos.get(0);
-    Assert.assertEquals(MRApp.NM_HOST, amInfo.getNodeManagerHost());
-    Assert.assertEquals(MRApp.NM_PORT, amInfo.getNodeManagerPort());
-    Assert.assertEquals(MRApp.NM_HTTP_PORT, amInfo.getNodeManagerHttpPort());
-    Assert.assertEquals(1, amInfo.getAppAttemptId().getAttemptId());
-    Assert.assertEquals(1, amInfo.getContainerId().getApplicationAttemptId()
-        .getAttemptId());
-    Assert.assertTrue(amInfo.getStartTime() > 0);
-    Assert.assertFalse(jr.isUber());
+    assertEquals(MRApp.NM_HOST, amInfo.getNodeManagerHost());
+    assertEquals(MRApp.NM_PORT, amInfo.getNodeManagerPort());
+    assertEquals(MRApp.NM_HTTP_PORT, amInfo.getNodeManagerHttpPort());
+    assertEquals(1, amInfo.getAppAttemptId().getAttemptId());
+    assertEquals(1, amInfo.getContainerId().getApplicationAttemptId().getAttemptId());
+    assertTrue(amInfo.getStartTime() > 0);
+    assertFalse(jr.isUber());
   }
   
   private void verifyTaskAttemptReport(TaskAttemptReport tar) {
-    Assert.assertEquals(TaskAttemptState.RUNNING, tar.getTaskAttemptState());
-    Assert.assertNotNull("TaskAttemptReport is null", tar);
-    Assert.assertEquals(MRApp.NM_HOST, tar.getNodeManagerHost());
-    Assert.assertEquals(MRApp.NM_PORT, tar.getNodeManagerPort());
-    Assert.assertEquals(MRApp.NM_HTTP_PORT, tar.getNodeManagerHttpPort());
-    Assert.assertEquals(1, tar.getContainerId().getApplicationAttemptId()
-        .getAttemptId());
+    assertEquals(TaskAttemptState.RUNNING, tar.getTaskAttemptState());
+    assertNotNull(tar, "TaskAttemptReport is null");
+    assertEquals(MRApp.NM_HOST, tar.getNodeManagerHost());
+    assertEquals(MRApp.NM_PORT, tar.getNodeManagerPort());
+    assertEquals(MRApp.NM_HTTP_PORT, tar.getNodeManagerHttpPort());
+    assertEquals(1, tar.getContainerId().getApplicationAttemptId().getAttemptId());
   }
   
   class MRAppWithClientService extends MRApp {

+ 89 - 114
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java

@@ -19,9 +19,9 @@
 package org.apache.hadoop.mapreduce.v2.app;
 
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.Mockito.atLeast;
 import static org.mockito.Mockito.mock;
 
@@ -43,7 +43,6 @@ import java.util.concurrent.TimeoutException;
 
 import org.apache.hadoop.mapreduce.util.MRJobConfUtil;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptFailEvent;
-import org.junit.Assert;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -108,8 +107,9 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.SystemClock;
 
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.mockito.ArgumentCaptor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -127,7 +127,7 @@ public class TestRecovery {
   private Text val1 = new Text("val1");
   private Text val2 = new Text("val2");
 
-  @BeforeClass
+  @BeforeAll
   public static void setupClass() throws Exception {
     // setup the test root directory
     testRootDir =
@@ -159,8 +159,7 @@ public class TestRecovery {
     app.waitForState(job, JobState.RUNNING);
     long jobStartTime = job.getReport().getStartTime();
     //all maps would be running
-    Assert.assertEquals("No of tasks not correct",
-       3, job.getTasks().size());
+    assertEquals(3, job.getTasks().size(), "No of tasks not correct");
     Iterator<Task> it = job.getTasks().values().iterator();
     Task mapTask1 = it.next();
     Task mapTask2 = it.next();
@@ -193,7 +192,7 @@ public class TestRecovery {
       Thread.sleep(2000);
       LOG.info("Waiting for next attempt to start");
     }
-    Assert.assertEquals(2, mapTask1.getAttempts().size());
+    assertEquals(2, mapTask1.getAttempts().size());
     Iterator<TaskAttempt> itr = mapTask1.getAttempts().values().iterator();
     itr.next();
     TaskAttempt task1Attempt2 = itr.next();
@@ -214,7 +213,7 @@ public class TestRecovery {
       Thread.sleep(2000);
       LOG.info("Waiting for next attempt to start");
     }
-    Assert.assertEquals(3, mapTask1.getAttempts().size());
+    assertEquals(3, mapTask1.getAttempts().size());
     itr = mapTask1.getAttempts().values().iterator();
     itr.next();
     itr.next();
@@ -235,7 +234,7 @@ public class TestRecovery {
       Thread.sleep(2000);
       LOG.info("Waiting for next attempt to start");
     }
-    Assert.assertEquals(4, mapTask1.getAttempts().size());
+    assertEquals(4, mapTask1.getAttempts().size());
     itr = mapTask1.getAttempts().values().iterator();
     itr.next();
     itr.next();
@@ -273,8 +272,7 @@ public class TestRecovery {
     job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
     //all maps would be running
-    Assert.assertEquals("No of tasks not correct",
-       3, job.getTasks().size());
+    assertEquals(3, job.getTasks().size(), "No of tasks not correct");
     it = job.getTasks().values().iterator();
     mapTask1 = it.next();
     mapTask2 = it.next();
@@ -309,30 +307,26 @@ public class TestRecovery {
     
     app.waitForState(job, JobState.SUCCEEDED);
     app.verifyCompleted();
-    Assert.assertEquals("Job Start time not correct",
-        jobStartTime, job.getReport().getStartTime());
-    Assert.assertEquals("Task Start time not correct",
-        task1StartTime, mapTask1.getReport().getStartTime());
-    Assert.assertEquals("Task Finish time not correct",
-        task1FinishTime, mapTask1.getReport().getFinishTime());
-    Assert.assertEquals(2, job.getAMInfos().size());
+    assertEquals(jobStartTime, job.getReport().getStartTime(), "Job Start time not correct");
+    assertEquals(task1StartTime, mapTask1.getReport().getStartTime(),
+        "Task Start time not correct");
+    assertEquals(task1FinishTime, mapTask1.getReport().getFinishTime(),
+        "Task Finish time not correct");
+    assertEquals(2, job.getAMInfos().size());
     int attemptNum = 1;
     // Verify AMInfo
     for (AMInfo amInfo : job.getAMInfos()) {
-      Assert.assertEquals(attemptNum++, amInfo.getAppAttemptId()
-          .getAttemptId());
-      Assert.assertEquals(amInfo.getAppAttemptId(), amInfo.getContainerId()
-          .getApplicationAttemptId());
-      Assert.assertEquals(MRApp.NM_HOST, amInfo.getNodeManagerHost());
-      Assert.assertEquals(MRApp.NM_PORT, amInfo.getNodeManagerPort());
-      Assert.assertEquals(MRApp.NM_HTTP_PORT, amInfo.getNodeManagerHttpPort());
+      assertEquals(attemptNum++, amInfo.getAppAttemptId().getAttemptId());
+      assertEquals(amInfo.getAppAttemptId(), amInfo.getContainerId().getApplicationAttemptId());
+      assertEquals(MRApp.NM_HOST, amInfo.getNodeManagerHost());
+      assertEquals(MRApp.NM_PORT, amInfo.getNodeManagerPort());
+      assertEquals(MRApp.NM_HTTP_PORT, amInfo.getNodeManagerHttpPort());
     }
     long am1StartTimeReal = job.getAMInfos().get(0).getStartTime();
     long am2StartTimeReal = job.getAMInfos().get(1).getStartTime();
-    Assert.assertTrue(am1StartTimeReal >= am1StartTimeEst
-        && am1StartTimeReal <= am2StartTimeEst);
-    Assert.assertTrue(am2StartTimeReal >= am2StartTimeEst
-        && am2StartTimeReal <= System.currentTimeMillis());
+    assertTrue(am1StartTimeReal >= am1StartTimeEst && am1StartTimeReal <= am2StartTimeEst);
+    assertTrue(am2StartTimeReal >= am2StartTimeEst &&
+        am2StartTimeReal <= System.currentTimeMillis());
     // TODO Add verification of additional data from jobHistory - whatever was
     // available in the failed attempt should be available here
   }
@@ -372,7 +366,7 @@ public class TestRecovery {
     app.waitForState(job, JobState.RUNNING);
 
     // all maps would be running
-    Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size());
+    assertEquals(3, job.getTasks().size(), "No of tasks not correct");
     Iterator<Task> it = job.getTasks().values().iterator();
     Task mapTask1 = it.next();
     Task mapTask2 = it.next();
@@ -430,7 +424,7 @@ public class TestRecovery {
     job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
 
-    Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size());
+    assertEquals(3, job.getTasks().size(), "No of tasks not correct");
     it = job.getTasks().values().iterator();
     mapTask1 = it.next();
     mapTask2 = it.next();
@@ -517,7 +511,7 @@ public class TestRecovery {
     app.waitForState(job, JobState.RUNNING);
 
     // all maps would be running
-    Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size());
+    assertEquals(3, job.getTasks().size(), "No of tasks not correct");
     Iterator<Task> it = job.getTasks().values().iterator();
     Task mapTask1 = it.next();
     Task mapTask2 = it.next();
@@ -576,7 +570,7 @@ public class TestRecovery {
     job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
 
-    Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size());
+    assertEquals(3, job.getTasks().size(), "No of tasks not correct");
     it = job.getTasks().values().iterator();
     mapTask1 = it.next();
     mapTask2 = it.next();
@@ -642,8 +636,8 @@ public class TestRecovery {
     app = new MRAppWithHistory(1, 1, false, this.getClass().getName(), false,
         ++runCount);
     Job jobAttempt2 = app.submit(conf);
-    Assert.assertTrue("Recovery from previous job attempt is processed even " +
-        "though intermediate data encryption is enabled.", !app.recovered());
+    assertFalse(app.recovered(), "Recovery from previous job attempt is processed even " +
+        "though intermediate data encryption is enabled.");
 
     // The map task succeeded from previous job attempt will not be recovered
     // because the data spill encryption is enabled.
@@ -695,7 +689,7 @@ public class TestRecovery {
     app.waitForState(job, JobState.RUNNING);
 
     // all maps would be running
-    Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size());
+    assertEquals(3, job.getTasks().size(), "No of tasks not correct");
     Iterator<Task> it = job.getTasks().values().iterator();
     Task mapTask1 = it.next();
     Task mapTask2 = it.next();
@@ -754,7 +748,7 @@ public class TestRecovery {
     job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
 
-    Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size());
+    assertEquals(3, job.getTasks().size(), "No of tasks not correct");
     it = job.getTasks().values().iterator();
     mapTask1 = it.next();
     mapTask2 = it.next();
@@ -814,8 +808,7 @@ public class TestRecovery {
     Job job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
     //all maps would be running
-    Assert.assertEquals("No of tasks not correct",
-       3, job.getTasks().size());
+    assertEquals(3, job.getTasks().size(), "No of tasks not correct");
     Iterator<Task> it = job.getTasks().values().iterator();
     Task mapTask1 = it.next();
     Task mapTask2 = it.next();
@@ -834,8 +827,8 @@ public class TestRecovery {
     app.waitForState(task2Attempt, TaskAttemptState.RUNNING);
     
     // reduces must be in NEW state
-    Assert.assertEquals("Reduce Task state not correct",
-        TaskState.RUNNING, reduceTask.getReport().getTaskState());
+    assertEquals(TaskState.RUNNING, reduceTask.getReport().getTaskState(),
+        "Reduce Task state not correct");
 
     //send the done signal to the 1st map
     app.getContext().getEventHandler().handle(
@@ -863,8 +856,7 @@ public class TestRecovery {
     job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
     //all maps would be running
-    Assert.assertEquals("No of tasks not correct",
-       3, job.getTasks().size());
+    assertEquals(3, job.getTasks().size(), "No of tasks not correct");
     it = job.getTasks().values().iterator();
     mapTask1 = it.next();
     mapTask2 = it.next();
@@ -906,8 +898,7 @@ public class TestRecovery {
     job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
     //all maps would be running
-    Assert.assertEquals("No of tasks not correct",
-       3, job.getTasks().size());
+    assertEquals(3, job.getTasks().size(), "No of tasks not correct");
     it = job.getTasks().values().iterator();
     mapTask1 = it.next();
     mapTask2 = it.next();
@@ -941,8 +932,7 @@ public class TestRecovery {
     conf.set(FileOutputFormat.OUTDIR, outputDir.toString());
     Job job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
-    Assert.assertEquals("No of tasks not correct",
-       3, job.getTasks().size());
+    assertEquals(3, job.getTasks().size(), "No of tasks not correct");
     Iterator<Task> it = job.getTasks().values().iterator();
     Task mapTask1 = it.next();
     Task reduceTask1 = it.next();
@@ -967,7 +957,7 @@ public class TestRecovery {
     app.waitForState(mapTask1, TaskState.SUCCEEDED);
 
     // Verify the shuffle-port
-    Assert.assertEquals(5467, task1Attempt1.getShufflePort());
+    assertEquals(5467, task1Attempt1.getShufflePort());
     
     app.waitForState(reduceTask1, TaskState.RUNNING);
     TaskAttempt reduce1Attempt1 = reduceTask1.getAttempts().values().iterator().next();
@@ -999,8 +989,7 @@ public class TestRecovery {
     conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
     job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
-    Assert.assertEquals("No of tasks not correct",
-       3, job.getTasks().size());
+    assertEquals(3, job.getTasks().size(), "No of tasks not correct");
     it = job.getTasks().values().iterator();
     mapTask1 = it.next();
     reduceTask1 = it.next();
@@ -1011,7 +1000,7 @@ public class TestRecovery {
 
     // Verify the shuffle-port after recovery
     task1Attempt1 = mapTask1.getAttempts().values().iterator().next();
-    Assert.assertEquals(5467, task1Attempt1.getShufflePort());
+    assertEquals(5467, task1Attempt1.getShufflePort());
     
     // first reduce will be recovered, no need to send done
     app.waitForState(reduceTask1, TaskState.SUCCEEDED); 
@@ -1052,7 +1041,7 @@ public class TestRecovery {
     conf.set(FileOutputFormat.OUTDIR, outputDir.toString());
     Job job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
-    Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size());
+    assertEquals(3, job.getTasks().size(), "No of tasks not correct");
     //stop the app before the job completes.
     app.stop();
     app.close();
@@ -1062,11 +1051,10 @@ public class TestRecovery {
         ++runCount);
     job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
-    Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size());
+    assertEquals(3, job.getTasks().size(), "No of tasks not correct");
     TestFileOutputCommitter committer = (
         TestFileOutputCommitter) app.getCommitter();
-    assertTrue("commiter.abortJob() has not been called",
-        committer.isAbortJobCalled());
+    assertTrue(committer.isAbortJobCalled(), "commiter.abortJob() has not been called");
     app.close();
   }
 
@@ -1087,7 +1075,7 @@ public class TestRecovery {
     conf.set(FileOutputFormat.OUTDIR, outputDir.toString());
     Job job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
-    Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size());
+    assertEquals(3, job.getTasks().size(), "No of tasks not correct");
     //stop the app before the job completes.
     app.stop();
     app.close();
@@ -1097,11 +1085,11 @@ public class TestRecovery {
         ++runCount);
     job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
-    Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size());
+    assertEquals(3, job.getTasks().size(), "No of tasks not correct");
     TestFileOutputCommitter committer = (
         TestFileOutputCommitter) app.getCommitter();
-    assertFalse("commiter.abortJob() has been called",
-        committer.isAbortJobCalled());
+    assertFalse(committer.isAbortJobCalled(),
+        "commiter.abortJob() has been called");
     app.close();
   }
 
@@ -1117,8 +1105,7 @@ public class TestRecovery {
     conf.set(FileOutputFormat.OUTDIR, outputDir.toString());
     Job job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
-    Assert.assertEquals("No of tasks not correct",
-       3, job.getTasks().size());
+    assertEquals(3, job.getTasks().size(), "No of tasks not correct");
     Iterator<Task> it = job.getTasks().values().iterator();
     Task mapTask1 = it.next();
     Task mapTask2 = it.next();
@@ -1148,7 +1135,7 @@ public class TestRecovery {
     app.waitForState(mapTask1, TaskState.SUCCEEDED);
 
     // Verify the shuffle-port
-    Assert.assertEquals(5467, task1Attempt1.getShufflePort());
+    assertEquals(5467, task1Attempt1.getShufflePort());
 
     //stop the app before the job completes.
     app.stop();
@@ -1165,8 +1152,7 @@ public class TestRecovery {
     conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
     job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
-    Assert.assertEquals("No of tasks not correct",
-       3, job.getTasks().size());
+    assertEquals(3, job.getTasks().size(), "No of tasks not correct");
     it = job.getTasks().values().iterator();
     mapTask1 = it.next();
     mapTask2 = it.next();
@@ -1177,7 +1163,7 @@ public class TestRecovery {
 
     // Verify the shuffle-port after recovery
     task1Attempt1 = mapTask1.getAttempts().values().iterator().next();
-    Assert.assertEquals(5467, task1Attempt1.getShufflePort());
+    assertEquals(5467, task1Attempt1.getShufflePort());
     
     app.waitForState(mapTask2, TaskState.RUNNING);
     
@@ -1198,7 +1184,7 @@ public class TestRecovery {
     app.waitForState(mapTask2, TaskState.SUCCEEDED);
 
     // Verify the shuffle-port
-    Assert.assertEquals(5467, task2Attempt1.getShufflePort());
+    assertEquals(5467, task2Attempt1.getShufflePort());
     
     app.waitForState(reduceTask1, TaskState.RUNNING);
     TaskAttempt reduce1Attempt1 = reduceTask1.getAttempts().values().iterator().next();
@@ -1232,8 +1218,7 @@ public class TestRecovery {
     conf.set(FileOutputFormat.OUTDIR, outputDir.toString());
     Job job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
-    Assert.assertEquals("No of tasks not correct",
-       3, job.getTasks().size());
+    assertEquals(3, job.getTasks().size(), "No of tasks not correct");
     Iterator<Task> it = job.getTasks().values().iterator();
     Task mapTask1 = it.next();
     Task reduceTask1 = it.next();
@@ -1258,7 +1243,7 @@ public class TestRecovery {
     app.waitForState(mapTask1, TaskState.SUCCEEDED);
 
     // Verify the shuffle-port
-    Assert.assertEquals(5467, task1Attempt1.getShufflePort());
+    assertEquals(5467, task1Attempt1.getShufflePort());
     
     app.waitForState(reduceTask1, TaskState.RUNNING);
     TaskAttempt reduce1Attempt1 = reduceTask1.getAttempts().values().iterator().next();
@@ -1290,8 +1275,7 @@ public class TestRecovery {
     conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
     job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
-    Assert.assertEquals("No of tasks not correct",
-       3, job.getTasks().size());
+    assertEquals(3, job.getTasks().size(), "No of tasks not correct");
     it = job.getTasks().values().iterator();
     mapTask1 = it.next();
     reduceTask1 = it.next();
@@ -1302,7 +1286,7 @@ public class TestRecovery {
 
     // Verify the shuffle-port after recovery
     task1Attempt1 = mapTask1.getAttempts().values().iterator().next();
-    Assert.assertEquals(5467, task1Attempt1.getShufflePort());
+    assertEquals(5467, task1Attempt1.getShufflePort());
     
     // first reduce will be recovered, no need to send done
     app.waitForState(reduceTask1, TaskState.SUCCEEDED); 
@@ -1352,8 +1336,7 @@ public class TestRecovery {
     app.waitForState(job, JobState.RUNNING);
     long jobStartTime = job.getReport().getStartTime();
     //all maps would be running
-    Assert.assertEquals("No of tasks not correct",
-       3, job.getTasks().size());
+    assertEquals(3, job.getTasks().size(), "No of tasks not correct");
 
     Iterator<Task> it = job.getTasks().values().iterator();
     Task mapTask1 = it.next();
@@ -1426,8 +1409,7 @@ public class TestRecovery {
     job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
     //all maps would be running
-    Assert.assertEquals("No of tasks not correct",
-       3, job.getTasks().size());
+    assertEquals(3, job.getTasks().size(), "No of tasks not correct");
     it = job.getTasks().values().iterator();
     mapTask1 = it.next();
     mapTask2 = it.next();
@@ -1463,34 +1445,31 @@ public class TestRecovery {
 
     app.waitForState(job, JobState.SUCCEEDED);
     app.verifyCompleted();
-    Assert.assertEquals("Job Start time not correct",
-        jobStartTime, job.getReport().getStartTime());
-    Assert.assertEquals("Task Start time not correct",
-        task1StartTime, mapTask1.getReport().getStartTime());
-    Assert.assertEquals("Task Finish time not correct",
-        task1FinishTime, mapTask1.getReport().getFinishTime());
-    Assert.assertEquals(2, job.getAMInfos().size());
+    assertEquals(jobStartTime, job.getReport().getStartTime(), "Job Start time not correct");
+    assertEquals(task1StartTime, mapTask1.getReport().getStartTime(),
+        "Task Start time not correct");
+    assertEquals(task1FinishTime, mapTask1.getReport().getFinishTime(),
+        "Task Finish time not correct");
+    assertEquals(2, job.getAMInfos().size());
     int attemptNum = 1;
     // Verify AMInfo
     for (AMInfo amInfo : job.getAMInfos()) {
-      Assert.assertEquals(attemptNum++, amInfo.getAppAttemptId()
-          .getAttemptId());
-      Assert.assertEquals(amInfo.getAppAttemptId(), amInfo.getContainerId()
-          .getApplicationAttemptId());
-      Assert.assertEquals(MRApp.NM_HOST, amInfo.getNodeManagerHost());
-      Assert.assertEquals(MRApp.NM_PORT, amInfo.getNodeManagerPort());
-      Assert.assertEquals(MRApp.NM_HTTP_PORT, amInfo.getNodeManagerHttpPort());
+      assertEquals(attemptNum++, amInfo.getAppAttemptId().getAttemptId());
+      assertEquals(amInfo.getAppAttemptId(), amInfo.getContainerId().getApplicationAttemptId());
+      assertEquals(MRApp.NM_HOST, amInfo.getNodeManagerHost());
+      assertEquals(MRApp.NM_PORT, amInfo.getNodeManagerPort());
+      assertEquals(MRApp.NM_HTTP_PORT, amInfo.getNodeManagerHttpPort());
     }
     long am1StartTimeReal = job.getAMInfos().get(0).getStartTime();
     long am2StartTimeReal = job.getAMInfos().get(1).getStartTime();
-    Assert.assertTrue(am1StartTimeReal >= am1StartTimeEst
-        && am1StartTimeReal <= am2StartTimeEst);
-    Assert.assertTrue(am2StartTimeReal >= am2StartTimeEst
-        && am2StartTimeReal <= System.currentTimeMillis());
+    assertTrue(am1StartTimeReal >= am1StartTimeEst && am1StartTimeReal <= am2StartTimeEst);
+    assertTrue(am2StartTimeReal >= am2StartTimeEst &&
+        am2StartTimeReal <= System.currentTimeMillis());
 
   }
 
-  @Test(timeout=30000)
+  @Test
+  @Timeout(value = 30)
   public void testRecoveryWithoutShuffleSecret() throws Exception {
 
     int runCount = 0;
@@ -1504,8 +1483,7 @@ public class TestRecovery {
     Job job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
     //all maps would be running
-    Assert.assertEquals("No of tasks not correct",
-       3, job.getTasks().size());
+    assertEquals(3, job.getTasks().size(), "No of tasks not correct");
     Iterator<Task> it = job.getTasks().values().iterator();
     Task mapTask1 = it.next();
     Task mapTask2 = it.next();
@@ -1551,8 +1529,7 @@ public class TestRecovery {
     job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
     //all maps would be running
-    Assert.assertEquals("No of tasks not correct",
-       3, job.getTasks().size());
+    assertEquals(3, job.getTasks().size(), "No of tasks not correct");
     it = job.getTasks().values().iterator();
     mapTask1 = it.next();
     mapTask2 = it.next();
@@ -1891,16 +1868,16 @@ public class TestRecovery {
       ArgumentCaptor<Event> arg, List<EventType> expectedJobHistoryEvents,
       long expectedMapLaunches, long expectedFailedMaps) {
 
-    assertEquals("Final State of Task", finalState, checkTask.getState());
+    assertEquals(finalState, checkTask.getState(), "Final State of Task");
 
     Map<TaskAttemptId, TaskAttempt> recoveredAttempts =
         checkTask.getAttempts();
-    assertEquals("Expected Number of Task Attempts",
-        finalAttemptStates.size(), recoveredAttempts.size());
+    assertEquals(finalAttemptStates.size(), recoveredAttempts.size(),
+        "Expected Number of Task Attempts");
     for (TaskAttemptID taID : finalAttemptStates.keySet()) {
-      assertEquals("Expected Task Attempt State",
-          finalAttemptStates.get(taID),
-          recoveredAttempts.get(TypeConverter.toYarn(taID)).getState());
+      assertEquals(finalAttemptStates.get(taID),
+          recoveredAttempts.get(TypeConverter.toYarn(taID)).getState(),
+          "Expected Task Attempt State");
     }
 
     Iterator<Event> ie = arg.getAllValues().iterator();
@@ -1948,12 +1925,10 @@ public class TestRecovery {
       }
     }
     assertTrue(jobTaskEventReceived || (finalState == TaskState.RUNNING));
-    assertEquals("Did not process all expected JobHistoryEvents",
-        0, expectedJobHistoryEvents.size());
-    assertEquals("Expected Map Launches",
-        expectedMapLaunches, totalLaunchedMaps);
-    assertEquals("Expected Failed Maps",
-        expectedFailedMaps, totalFailedMaps);
+    assertEquals(0, expectedJobHistoryEvents.size(),
+        "Did not process all expected JobHistoryEvents");
+    assertEquals(expectedMapLaunches, totalLaunchedMaps, "Expected Map Launches");
+    assertEquals(expectedFailedMaps, totalFailedMaps, "Expected Failed Maps");
   }
 
   private MapTaskImpl getMockMapTask(long clusterTimestamp, EventHandler eh) {

+ 19 - 22
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java

@@ -78,13 +78,13 @@ import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.ControlledClock;
 import org.apache.hadoop.yarn.util.SystemClock;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.offset;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 @SuppressWarnings({"unchecked", "rawtypes"})
 public class TestRuntimeEstimators {
@@ -152,16 +152,16 @@ public class TestRuntimeEstimators {
     conf.setDouble(MRJobConfig.SPECULATIVECAP_TOTAL_TASKS, 0.001);
     conf.setInt(MRJobConfig.SPECULATIVE_MINIMUM_ALLOWED_TASKS, 5);
     speculator = new DefaultSpeculator(conf, myAppContext, estimator, clock);
-    Assert.assertEquals("wrong SPECULATIVE_RETRY_AFTER_NO_SPECULATE value",
-        500L, speculator.getSoonestRetryAfterNoSpeculate());
-    Assert.assertEquals("wrong SPECULATIVE_RETRY_AFTER_SPECULATE value",
-        5000L, speculator.getSoonestRetryAfterSpeculate());
+    assertEquals(500L, speculator.getSoonestRetryAfterNoSpeculate(),
+        "wrong SPECULATIVE_RETRY_AFTER_NO_SPECULATE value");
+    assertEquals(5000L, speculator.getSoonestRetryAfterSpeculate(),
+        "wrong SPECULATIVE_RETRY_AFTER_SPECULATE value");
     assertThat(speculator.getProportionRunningTasksSpeculatable())
         .isCloseTo(0.1, offset(0.00001));
     assertThat(speculator.getProportionTotalTasksSpeculatable())
         .isCloseTo(0.001, offset(0.00001));
-    Assert.assertEquals("wrong SPECULATIVE_MINIMUM_ALLOWED_TASKS value",
-        5, speculator.getMinimumAllowedSpeculativeTasks());
+    assertEquals(5, speculator.getMinimumAllowedSpeculativeTasks(),
+        "wrong SPECULATIVE_MINIMUM_ALLOWED_TASKS value");
 
     dispatcher.register(Speculator.EventType.class, speculator);
 
@@ -188,7 +188,7 @@ public class TestRuntimeEstimators {
     int undoneReduces = REDUCE_TASKS;
 
     // build a task sequence where all the maps precede any of the reduces
-    List<Task> allTasksSequence = new LinkedList<Task>();
+    List<Task> allTasksSequence = new LinkedList<>();
 
     allTasksSequence.addAll(myJob.getTasks(TaskType.MAP).values());
     allTasksSequence.addAll(myJob.getTasks(TaskType.REDUCE).values());
@@ -244,8 +244,8 @@ public class TestRuntimeEstimators {
       }
     }
 
-    Assert.assertEquals("We got the wrong number of successful speculations.",
-        expectedSpeculations, successfulSpeculations.get());
+    assertEquals(expectedSpeculations, successfulSpeculations.get(),
+        "We got the wrong number of successful speculations.");
   }
 
   @Test
@@ -279,8 +279,8 @@ public class TestRuntimeEstimators {
       TaskId taskID = event.getTaskID();
       Task task = myJob.getTask(taskID);
 
-      Assert.assertEquals
-          ("Wrong type event", TaskEventType.T_ADD_SPEC_ATTEMPT, event.getType());
+      assertEquals(TaskEventType.T_ADD_SPEC_ATTEMPT, event.getType(),
+          "Wrong type event");
 
       System.out.println("SpeculationRequestEventHandler.handle adds a speculation task for " + taskID);
 
@@ -296,8 +296,7 @@ public class TestRuntimeEstimators {
 
   class MyTaskImpl implements Task {
     private final TaskId taskID;
-    private final Map<TaskAttemptId, TaskAttempt> attempts
-        = new ConcurrentHashMap<TaskAttemptId, TaskAttempt>(4);
+    private final Map<TaskAttemptId, TaskAttempt> attempts = new ConcurrentHashMap<>(4);
 
     MyTaskImpl(JobId jobID, int index, TaskType type) {
       taskID = recordFactory.newRecordInstance(TaskId.class);
@@ -353,8 +352,7 @@ public class TestRuntimeEstimators {
 
     @Override
     public Map<TaskAttemptId, TaskAttempt> getAttempts() {
-      Map<TaskAttemptId, TaskAttempt> result
-          = new HashMap<TaskAttemptId, TaskAttempt>(attempts.size());
+      Map<TaskAttemptId, TaskAttempt> result = new HashMap<>(attempts.size());
       result.putAll(attempts);
       return result;
     }
@@ -389,9 +387,9 @@ public class TestRuntimeEstimators {
 
   class MyJobImpl implements Job {
     private final JobId jobID;
-    private final Map<TaskId, Task> allTasks = new HashMap<TaskId, Task>();
-    private final Map<TaskId, Task> mapTasks = new HashMap<TaskId, Task>();
-    private final Map<TaskId, Task> reduceTasks = new HashMap<TaskId, Task>();
+    private final Map<TaskId, Task> allTasks = new HashMap<>();
+    private final Map<TaskId, Task> mapTasks = new HashMap<>();
+    private final Map<TaskId, Task> reduceTasks = new HashMap<>();
 
     MyJobImpl(JobId jobID, int numMaps, int numReduces) {
       this.jobID = jobID;
@@ -570,7 +568,7 @@ public class TestRuntimeEstimators {
   /*
    * We follow the pattern of the real XxxImpl .  We create a job and initialize
    * it with a full suite of tasks which in turn have one attempt each in the
-   * NEW state.  Attempts transition only from NEW to RUNNING to SUCCEEDED .
+   * NEW state.  Attempts transition only from NEW to RUNNING to SUCCEED .
    */
   class MyTaskAttemptImpl implements TaskAttempt {
     private final TaskAttemptId myAttemptID;
@@ -936,7 +934,6 @@ public class TestRuntimeEstimators {
 
     @Override
     public void setHistoryUrl(String historyUrl) {
-      return;
     }
   }
 }

+ 24 - 20
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java

@@ -18,8 +18,9 @@
 
 package org.apache.hadoop.mapreduce.v2.app;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyBoolean;
 import static org.mockito.Mockito.mock;
@@ -61,9 +62,9 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 
 /**
@@ -78,7 +79,7 @@ import org.junit.Test;
    private final static RecordFactory recordFactory = RecordFactoryProvider.
        getRecordFactory(null);
 
-   @After
+   @AfterEach
    public void tearDown() {
      conf.setBoolean(MRJobConfig.PRESERVE_FAILED_TASK_FILES, false);
    }
@@ -135,7 +136,7 @@ import org.junit.Test;
      JobId jobid = recordFactory.newRecordInstance(JobId.class);
      jobid.setAppId(appId);
      ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
-     Assert.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
+     assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
      MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc,
          JobStateInternal.RUNNING, MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS);
      appMaster.init(conf);
@@ -146,7 +147,8 @@ import org.junit.Test;
      verify(fs).delete(stagingJobPath, true);
    }
 
-   @Test (timeout = 30000)
+   @Test
+   @Timeout(value = 30)
    public void testNoDeletionofStagingOnReboot() throws IOException {
      conf.set(MRJobConfig.MAPREDUCE_JOB_DIR, stagingJobDir);
      fs = mock(FileSystem.class);
@@ -158,7 +160,7 @@ import org.junit.Test;
          0);
      ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1);
      ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
-     Assert.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
+     assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
      MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc,
          JobStateInternal.REBOOT, MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS);
      appMaster.init(conf);
@@ -197,7 +199,8 @@ import org.junit.Test;
      verify(fs).delete(stagingJobPath, true);
    }
    
-   @Test (timeout = 30000)
+   @Test
+   @Timeout(value = 30)
    public void testDeletionofStagingOnKill() throws IOException {
      conf.set(MRJobConfig.MAPREDUCE_JOB_DIR, stagingJobDir);
      fs = mock(FileSystem.class);
@@ -242,13 +245,13 @@ import org.junit.Test;
      ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
      MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc); //no retry
      appMaster.init(conf);
-     assertTrue("appMaster.isLastAMRetry() is false", appMaster.isLastAMRetry());
+     assertTrue(appMaster.isLastAMRetry(), "appMaster.isLastAMRetry() is false");
      //simulate the process being killed
      MRAppMaster.MRAppMasterShutdownHook hook = 
        new MRAppMaster.MRAppMasterShutdownHook(appMaster);
      hook.run();
-     assertTrue("MRAppMaster isn't stopped",
-                appMaster.isInState(Service.STATE.STOPPED));
+     assertTrue(appMaster.isInState(Service.STATE.STOPPED),
+         "MRAppMaster isn't stopped");
      verify(fs).delete(stagingJobPath, true);
    }
 
@@ -270,7 +273,7 @@ import org.junit.Test;
      JobId jobid = recordFactory.newRecordInstance(JobId.class);
      jobid.setAppId(appId);
      ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
-     Assert.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
+     assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
      MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc,
              JobStateInternal.FAILED, MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS);
      appMaster.init(conf);
@@ -298,7 +301,7 @@ import org.junit.Test;
      JobId jobid = recordFactory.newRecordInstance(JobId.class);
      jobid.setAppId(appId);
      ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
-     Assert.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
+     assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
      MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc,
              JobStateInternal.RUNNING, MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS);
      appMaster.init(conf);
@@ -324,7 +327,7 @@ import org.junit.Test;
     JobId jobid = recordFactory.newRecordInstance(JobId.class);
     jobid.setAppId(appId);
     ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
-    Assert.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
+    assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
     MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc,
             JobStateInternal.RUNNING, MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS);
     appMaster.init(conf);
@@ -355,7 +358,7 @@ import org.junit.Test;
     JobId jobid = recordFactory.newRecordInstance(JobId.class);
     jobid.setAppId(appId);
     ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
-    Assert.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
+    assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
     MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc,
             JobStateInternal.RUNNING, MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS);
     appMaster.init(conf);
@@ -583,7 +586,8 @@ import org.junit.Test;
     };
   }
 
-  @Test(timeout=20000)
+  @Test
+  @Timeout(value = 20)
   public void testStagingCleanupOrder() throws Exception {
     MRAppTestCleanup app = new MRAppTestCleanup(1, 1, true,
         this.getClass().getName(), true);
@@ -598,7 +602,7 @@ import org.junit.Test;
     }
 
     // assert ContainerAllocatorStopped and then tagingDirCleanedup
-    Assert.assertEquals(1, app.ContainerAllocatorStopped);
-    Assert.assertEquals(2, app.stagingDirCleanedup);
+    assertEquals(1, app.ContainerAllocatorStopped);
+    assertEquals(2, app.stagingDirCleanedup);
   }
  }

+ 10 - 15
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestTaskHeartbeatHandler.java

@@ -18,14 +18,15 @@
 
 package org.apache.hadoop.mapreduce.v2.app;
 
-import static org.junit.Assert.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 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;
 
-import java.util.function.Supplier;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
@@ -40,8 +41,7 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.ControlledClock;
 import org.apache.hadoop.yarn.util.SystemClock;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.util.Map;
 import java.util.concurrent.ConcurrentMap;
@@ -214,20 +214,15 @@ public class TestTaskHeartbeatHandler {
       JobId jobId = MRBuilderUtils.newJobId(appId, 4);
       TaskId tid = MRBuilderUtils.newTaskId(jobId, 3, TaskType.MAP);
       final TaskAttemptId taid = MRBuilderUtils.newTaskAttemptId(tid, 2);
-      Assert.assertFalse(hb.hasRecentlyUnregistered(taid));
+      assertFalse(hb.hasRecentlyUnregistered(taid));
       hb.register(taid);
-      Assert.assertFalse(hb.hasRecentlyUnregistered(taid));
+      assertFalse(hb.hasRecentlyUnregistered(taid));
       hb.unregister(taid);
-      Assert.assertTrue(hb.hasRecentlyUnregistered(taid));
+      assertTrue(hb.hasRecentlyUnregistered(taid));
       long unregisterTimeout = conf.getLong(MRJobConfig.TASK_EXIT_TIMEOUT,
           MRJobConfig.TASK_EXIT_TIMEOUT_DEFAULT);
       clock.setTime(unregisterTimeout + 1);
-      GenericTestUtils.waitFor(new Supplier<Boolean>() {
-        @Override
-        public Boolean get() {
-          return !hb.hasRecentlyUnregistered(taid);
-        }
-      }, 10, 10000);
+      GenericTestUtils.waitFor(() -> !hb.hasRecentlyUnregistered(taid), 10, 10000);
     } finally {
       hb.stop();
     }
@@ -260,7 +255,7 @@ public class TestTaskHeartbeatHandler {
         new TaskHeartbeatHandler(null, SystemClock.getInstance(), 1);
     hb.init(conf);
 
-    Assert.assertTrue("The value of the task timeout is incorrect.",
-        hb.getTaskTimeOut() == expectedTimeout);
+    assertEquals(hb.getTaskTimeOut(), expectedTimeout,
+        "The value of the task timeout is incorrect.");
   }
 }

+ 21 - 20
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/commit/TestCommitterEventHandler.java

@@ -18,7 +18,11 @@
 
 package org.apache.hadoop.mapreduce.v2.app.commit;
 
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.times;
@@ -27,7 +31,7 @@ import static org.mockito.Mockito.when;
 
 import java.util.concurrent.ConcurrentLinkedQueue;
 
-import org.junit.Assert;
+import org.junit.jupiter.api.Assertions;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.JobContext;
@@ -39,9 +43,6 @@ import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
 
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
-
 import java.io.File;
 import java.io.IOException;
 
@@ -62,9 +63,9 @@ import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.SystemClock;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 
 public class TestCommitterEventHandler {
   public static class WaitForItHandler implements EventHandler<Event> {
@@ -95,13 +96,13 @@ public class TestCommitterEventHandler {
   
   static String stagingDir = "target/test-staging/";
 
-  @BeforeClass
+  @BeforeAll
   public static void setup() {    
     File dir = new File(stagingDir);
     stagingDir = dir.getAbsolutePath();
   }
 
-  @Before
+  @BeforeEach
   public void cleanup() throws IOException {
     File dir = new File(stagingDir);
     if(dir.exists()) {
@@ -146,11 +147,11 @@ public class TestCommitterEventHandler {
       Thread.sleep(10);
       timeToWaitMs -= 10;
     }
-    Assert.assertEquals("committer did not register a heartbeat callback",
-        1, rmhh.getNumCallbacks());
+    Assertions.assertEquals(1, rmhh.getNumCallbacks(),
+        "committer did not register a heartbeat callback");
     verify(committer, never()).commitJob(any(JobContext.class));
-    Assert.assertEquals("committer should not have committed",
-        0, jeh.numCommitCompletedEvents);
+    Assertions.assertEquals(0, jeh.numCommitCompletedEvents,
+        "committer should not have committed");
 
     // set a fresh heartbeat and verify commit completes
     rmhh.setLastHeartbeatTime(clock.getTime());
@@ -159,8 +160,8 @@ public class TestCommitterEventHandler {
       Thread.sleep(10);
       timeToWaitMs -= 10;
     }
-    Assert.assertEquals("committer did not complete commit after RM hearbeat",
-        1, jeh.numCommitCompletedEvents);
+    Assertions.assertEquals(1, jeh.numCommitCompletedEvents,
+        "committer did not complete commit after RM heartbeat");
     verify(committer, times(1)).commitJob(any());
 
     //Clean up so we can try to commit again (Don't do this at home)
@@ -174,8 +175,8 @@ public class TestCommitterEventHandler {
       Thread.sleep(10);
       timeToWaitMs -= 10;
     }
-    Assert.assertEquals("committer did not commit",
-        2, jeh.numCommitCompletedEvents);
+    Assertions.assertEquals(2, jeh.numCommitCompletedEvents,
+        "committer did not commit");
     verify(committer, times(2)).commitJob(any());
 
     ceh.stop();
@@ -262,9 +263,9 @@ public class TestCommitterEventHandler {
       assertNotNull(e);
       assertTrue(e instanceof JobCommitCompletedEvent);
       FileSystem fs = FileSystem.get(conf);
-      assertTrue(startCommitFile.toString(), fs.exists(startCommitFile));
-      assertTrue(endCommitSuccessFile.toString(), fs.exists(endCommitSuccessFile));
-      assertFalse(endCommitFailureFile.toString(), fs.exists(endCommitFailureFile));
+      assertTrue(fs.exists(startCommitFile), startCommitFile.toString());
+      assertTrue(fs.exists(endCommitSuccessFile), endCommitSuccessFile.toString());
+      assertFalse(fs.exists(endCommitFailureFile), endCommitFailureFile.toString());
       verify(mockCommitter).commitJob(any(JobContext.class));
     } finally {
       handler.stop();

+ 94 - 76
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java

@@ -18,11 +18,20 @@
 
 package org.apache.hadoop.mapreduce.v2.app.job.impl;
 
-import static org.mockito.ArgumentMatchers.any;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+
+import static org.mockito.Mockito.any;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.timeout;
+import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
 import java.io.File;
@@ -105,11 +114,10 @@ import org.apache.hadoop.yarn.state.StateMachine;
 import org.apache.hadoop.yarn.state.StateMachineFactory;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.SystemClock;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.mockito.Mockito;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 
 /**
@@ -120,13 +128,13 @@ public class TestJobImpl {
   
   static String stagingDir = "target/test-staging/";
 
-  @BeforeClass
+  @BeforeAll
   public static void setup() {    
     File dir = new File(stagingDir);
     stagingDir = dir.getAbsolutePath();
   }
 
-  @Before
+  @BeforeEach
   public void cleanup() throws IOException {
     File dir = new File(stagingDir);
     if(dir.exists()) {
@@ -169,13 +177,14 @@ public class TestJobImpl {
     dispatcher.stop();
     commitHandler.stop();
     try {
-      Assert.assertTrue(jseHandler.getAssertValue());
+      assertTrue(jseHandler.getAssertValue());
     } catch (InterruptedException e) {
-      Assert.fail("Workflow related attributes are not tested properly");
+      fail("Workflow related attributes are not tested properly");
     }
   }
 
-  @Test(timeout=20000)
+  @Test
+  @Timeout(value = 20)
   public void testCommitJobFailsJob() throws Exception {
     Configuration conf = new Configuration();
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
@@ -200,7 +209,8 @@ public class TestJobImpl {
     commitHandler.stop();
   }
 
-  @Test(timeout=20000)
+  @Test
+  @Timeout(value = 20)
   public void testCheckJobCompleteSuccess() throws Exception {
     Configuration conf = new Configuration();
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
@@ -252,7 +262,8 @@ public class TestJobImpl {
     commitHandler.stop();
   }
 
-  @Test(timeout=20000)
+  @Test
+  @Timeout(value = 20)
   public void testRebootedDuringSetup() throws Exception{
     Configuration conf = new Configuration();
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
@@ -289,13 +300,14 @@ public class TestJobImpl {
     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());
+    assertEquals(JobState.RUNNING, job.getState());
 
     dispatcher.stop();
     commitHandler.stop();
   }
 
-  @Test(timeout=20000)
+  @Test
+  @Timeout(value = 20)
   public void testRebootedDuringCommit() throws Exception {
     Configuration conf = new Configuration();
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
@@ -321,15 +333,16 @@ public class TestJobImpl {
     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());
+    assertEquals(JobState.RUNNING, job.getState());
     when(mockContext.hasSuccessfullyUnregistered()).thenReturn(true);
-    Assert.assertEquals(JobState.ERROR, job.getState());
+    assertEquals(JobState.ERROR, job.getState());
 
     dispatcher.stop();
     commitHandler.stop();
   }
 
-  @Test(timeout=20000)
+  @Test
+  @Timeout(value = 20)
   public void testKilledDuringSetup() throws Exception {
     Configuration conf = new Configuration();
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
@@ -366,7 +379,8 @@ public class TestJobImpl {
     commitHandler.stop();
   }
 
-  @Test(timeout=20000)
+  @Test
+  @Timeout(value = 20)
   public void testKilledDuringCommit() throws Exception {
     Configuration conf = new Configuration();
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
@@ -399,7 +413,7 @@ public class TestJobImpl {
     InlineDispatcher dispatcher = new InlineDispatcher();
     dispatcher.init(conf);
     dispatcher.start();
-    OutputCommitter committer = Mockito.mock(OutputCommitter.class);
+    OutputCommitter committer = mock(OutputCommitter.class);
     CommitterEventHandler commitHandler =
         createCommitterEventHandler(dispatcher, committer);
     commitHandler.init(conf);
@@ -411,19 +425,20 @@ public class TestJobImpl {
       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());
+    verify(committer, never())
+        .abortJob((JobContext) any(), (State) 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());
+    verify(committer, timeout(2000).times(1))
+        .abortJob((JobContext) any(), (State) any());
     assertJobState(job, JobStateInternal.FAILED);
 
     dispatcher.stop();
   }
 
-  @Test (timeout=10000)
+  @Test
+  @Timeout(value = 10)
   public void testFailAbortDoesntHang() throws IOException {
     Configuration conf = new Configuration();
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
@@ -432,7 +447,7 @@ public class TestJobImpl {
     DrainDispatcher dispatcher = new DrainDispatcher();
     dispatcher.init(conf);
     dispatcher.start();
-    OutputCommitter committer = Mockito.mock(OutputCommitter.class);
+    OutputCommitter committer = mock(OutputCommitter.class);
     CommitterEventHandler commitHandler =
         createCommitterEventHandler(dispatcher, committer);
     commitHandler.init(conf);
@@ -454,14 +469,15 @@ public class TestJobImpl {
 
     dispatcher.await();
     //Verify abortJob is called once and the job failed
-    Mockito.verify(committer, Mockito.timeout(2000).times(1))
-      .abortJob((JobContext) Mockito.any(), (State) Mockito.any());
+    verify(committer, timeout(2000).times(1))
+        .abortJob((JobContext) any(), (State) any());
     assertJobState(job, JobStateInternal.FAILED);
 
     dispatcher.stop();
   }
 
-  @Test(timeout=20000)
+  @Test
+  @Timeout(value = 20)
   public void testKilledDuringFailAbort() throws Exception {
     Configuration conf = new Configuration();
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
@@ -503,7 +519,8 @@ public class TestJobImpl {
     commitHandler.stop();
   }
 
-  @Test(timeout=20000)
+  @Test
+  @Timeout(value = 20)
   public void testKilledDuringKillAbort() throws Exception {
     Configuration conf = new Configuration();
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
@@ -546,7 +563,8 @@ public class TestJobImpl {
     commitHandler.stop();
   }
 
-  @Test(timeout=20000)
+  @Test
+  @Timeout(value = 20)
   public void testUnusableNodeTransition() throws Exception {
     Configuration conf = new Configuration();
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
@@ -599,7 +617,7 @@ public class TestJobImpl {
         job.handle(new JobTaskAttemptCompletedEvent(tce));
         // complete the task itself
         job.handle(new JobTaskEvent(taskId, TaskState.SUCCEEDED));
-        Assert.assertEquals(JobState.RUNNING, job.getState());
+        assertEquals(JobState.RUNNING, job.getState());
       }
     }
 
@@ -699,13 +717,13 @@ public class TestJobImpl {
      * much value. Instead, we validate the T_KILL events.
      */
     if (killMappers) {
-      Assert.assertEquals("Number of killed events", 2, killedEvents.size());
-      Assert.assertEquals("AttemptID", "task_1234567890000_0001_m_000000",
-          killedEvents.get(0).getTaskID().toString());
-      Assert.assertEquals("AttemptID", "task_1234567890000_0001_m_000001",
-          killedEvents.get(1).getTaskID().toString());
+      assertEquals(2, killedEvents.size(), "Number of killed events");
+      assertEquals("task_1234567890000_0001_m_000000",
+          killedEvents.get(0).getTaskID().toString(), "AttemptID");
+      assertEquals("task_1234567890000_0001_m_000001",
+          killedEvents.get(1).getTaskID().toString(), "AttemptID");
     } else {
-      Assert.assertEquals("Number of killed events", 0, killedEvents.size());
+      assertEquals(0, killedEvents.size(), "Number of killed events");
     }
   }
 
@@ -738,8 +756,8 @@ public class TestJobImpl {
     // Verify access
     JobImpl job1 = new JobImpl(jobId, null, conf1, null, null, null, null, null,
         null, null, null, true, user1, 0, null, null, null, null);
-    Assert.assertTrue(job1.checkAccess(ugi1, JobACL.VIEW_JOB));
-    Assert.assertFalse(job1.checkAccess(ugi2, JobACL.VIEW_JOB));
+    assertTrue(job1.checkAccess(ugi1, JobACL.VIEW_JOB));
+    assertFalse(job1.checkAccess(ugi2, JobACL.VIEW_JOB));
 
     // Setup configuration access to the user1 (owner) and user2
     Configuration conf2 = new Configuration();
@@ -749,8 +767,8 @@ public class TestJobImpl {
     // Verify access
     JobImpl job2 = new JobImpl(jobId, null, conf2, null, null, null, null, null,
         null, null, null, true, user1, 0, null, null, null, null);
-    Assert.assertTrue(job2.checkAccess(ugi1, JobACL.VIEW_JOB));
-    Assert.assertTrue(job2.checkAccess(ugi2, JobACL.VIEW_JOB));
+    assertTrue(job2.checkAccess(ugi1, JobACL.VIEW_JOB));
+    assertTrue(job2.checkAccess(ugi2, JobACL.VIEW_JOB));
 
     // Setup configuration access with security enabled and access to all
     Configuration conf3 = new Configuration();
@@ -760,8 +778,8 @@ public class TestJobImpl {
     // Verify access
     JobImpl job3 = new JobImpl(jobId, null, conf3, null, null, null, null, null,
         null, null, null, true, user1, 0, null, null, null, null);
-    Assert.assertTrue(job3.checkAccess(ugi1, JobACL.VIEW_JOB));
-    Assert.assertTrue(job3.checkAccess(ugi2, JobACL.VIEW_JOB));
+    assertTrue(job3.checkAccess(ugi1, JobACL.VIEW_JOB));
+    assertTrue(job3.checkAccess(ugi2, JobACL.VIEW_JOB));
 
     // Setup configuration access without security enabled
     Configuration conf4 = new Configuration();
@@ -771,8 +789,8 @@ public class TestJobImpl {
     // Verify access
     JobImpl job4 = new JobImpl(jobId, null, conf4, null, null, null, null, null,
         null, null, null, true, user1, 0, null, null, null, null);
-    Assert.assertTrue(job4.checkAccess(ugi1, JobACL.VIEW_JOB));
-    Assert.assertTrue(job4.checkAccess(ugi2, JobACL.VIEW_JOB));
+    assertTrue(job4.checkAccess(ugi1, JobACL.VIEW_JOB));
+    assertTrue(job4.checkAccess(ugi2, JobACL.VIEW_JOB));
 
     // Setup configuration access without security enabled
     Configuration conf5 = new Configuration();
@@ -782,8 +800,8 @@ public class TestJobImpl {
     // Verify access
     JobImpl job5 = new JobImpl(jobId, null, conf5, null, null, null, null, null,
         null, null, null, true, user1, 0, null, null, null, null);
-    Assert.assertTrue(job5.checkAccess(ugi1, null));
-    Assert.assertTrue(job5.checkAccess(ugi2, null));
+    assertTrue(job5.checkAccess(ugi1, null));
+    assertTrue(job5.checkAccess(ugi2, null));
   }
 
   @Test
@@ -804,8 +822,8 @@ public class TestJobImpl {
         mrAppMetrics, null, true, null, 0, null, mockContext, null, null);
     job.handle(diagUpdateEvent);
     String diagnostics = job.getReport().getDiagnostics();
-    Assert.assertNotNull(diagnostics);
-    Assert.assertTrue(diagnostics.contains(diagMsg));
+    assertNotNull(diagnostics);
+    assertTrue(diagnostics.contains(diagMsg));
 
     job = new JobImpl(jobId, Records
         .newRecord(ApplicationAttemptId.class), new Configuration(),
@@ -816,8 +834,8 @@ public class TestJobImpl {
     job.handle(new JobEvent(jobId, JobEventType.JOB_KILL));
     job.handle(diagUpdateEvent);
     diagnostics = job.getReport().getDiagnostics();
-    Assert.assertNotNull(diagnostics);
-    Assert.assertTrue(diagnostics.contains(diagMsg));
+    assertNotNull(diagnostics);
+    assertTrue(diagnostics.contains(diagMsg));
   }
 
   @Test
@@ -826,13 +844,13 @@ public class TestJobImpl {
     // with default values, no of maps is 2
     Configuration conf = new Configuration();
     boolean isUber = testUberDecision(conf);
-    Assert.assertFalse(isUber);
+    assertFalse(isUber);
 
     // enable uber mode, no of maps is 2
     conf = new Configuration();
     conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, true);
     isUber = testUberDecision(conf);
-    Assert.assertTrue(isUber);
+    assertTrue(isUber);
 
     // enable uber mode, no of maps is 2, no of reduces is 1 and uber task max
     // reduces is 0
@@ -841,7 +859,7 @@ public class TestJobImpl {
     conf.setInt(MRJobConfig.JOB_UBERTASK_MAXREDUCES, 0);
     conf.setInt(MRJobConfig.NUM_REDUCES, 1);
     isUber = testUberDecision(conf);
-    Assert.assertFalse(isUber);
+    assertFalse(isUber);
 
     // enable uber mode, no of maps is 2, no of reduces is 1 and uber task max
     // reduces is 1
@@ -850,14 +868,14 @@ public class TestJobImpl {
     conf.setInt(MRJobConfig.JOB_UBERTASK_MAXREDUCES, 1);
     conf.setInt(MRJobConfig.NUM_REDUCES, 1);
     isUber = testUberDecision(conf);
-    Assert.assertTrue(isUber);
+    assertTrue(isUber);
 
     // enable uber mode, no of maps is 2 and uber task max maps is 0
     conf = new Configuration();
     conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, true);
     conf.setInt(MRJobConfig.JOB_UBERTASK_MAXMAPS, 1);
     isUber = testUberDecision(conf);
-    Assert.assertFalse(isUber);
+    assertFalse(isUber);
     
  // enable uber mode of 0 reducer no matter how much memory assigned to reducer
     conf = new Configuration();
@@ -866,7 +884,7 @@ public class TestJobImpl {
     conf.setInt(MRJobConfig.REDUCE_MEMORY_MB, 2048);
     conf.setInt(MRJobConfig.REDUCE_CPU_VCORES, 10);
     isUber = testUberDecision(conf);
-    Assert.assertTrue(isUber);
+    assertTrue(isUber);
   }
 
   private boolean testUberDecision(Configuration conf) {
@@ -931,9 +949,9 @@ public class TestJobImpl {
     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());
+    assertEquals(JobState.RUNNING, job.getState());
     when(mockContext.hasSuccessfullyUnregistered()).thenReturn(true);
-    Assert.assertEquals(JobState.FAILED, job.getState());
+    assertEquals(JobState.FAILED, job.getState());
 
     dispatcher.stop();
     commitHandler.stop();
@@ -960,12 +978,12 @@ public class TestJobImpl {
     JobEvent mockJobEvent = mock(JobEvent.class);
 
     JobStateInternal jobSI = initTransition.transition(job, mockJobEvent);
-    Assert.assertTrue("When init fails, return value from InitTransition.transition should equal NEW.",
-                      jobSI.equals(JobStateInternal.NEW));
-    Assert.assertTrue("Job diagnostics should contain YarnRuntimeException",
-                      job.getDiagnostics().toString().contains("YarnRuntimeException"));
-    Assert.assertTrue("Job diagnostics should contain " + EXCEPTIONMSG,
-                      job.getDiagnostics().toString().contains(EXCEPTIONMSG));
+    assertEquals(jobSI, JobStateInternal.NEW,
+        "When init fails, return value from InitTransition.transition should equal NEW.");
+    assertTrue(job.getDiagnostics().toString().contains("YarnRuntimeException"),
+        "Job diagnostics should contain YarnRuntimeException");
+    assertTrue(job.getDiagnostics().toString().contains(EXCEPTIONMSG),
+        "Job diagnostics should contain " + EXCEPTIONMSG);
   }
 
   @Test
@@ -986,7 +1004,7 @@ public class TestJobImpl {
     assertJobState(job, JobStateInternal.SETUP);
     // Update priority of job to 5, and it will be updated
     job.setJobPriority(submittedPriority);
-    Assert.assertEquals(submittedPriority, job.getReport().getJobPriority());
+    assertEquals(submittedPriority, job.getReport().getJobPriority());
 
     job.handle(new JobSetupCompletedEvent(jobId));
     assertJobState(job, JobStateInternal.RUNNING);
@@ -996,10 +1014,10 @@ public class TestJobImpl {
     job.setJobPriority(updatedPriority);
     assertJobState(job, JobStateInternal.RUNNING);
     Priority jobPriority = job.getReport().getJobPriority();
-    Assert.assertNotNull(jobPriority);
+    assertNotNull(jobPriority);
 
     // Verify whether changed priority is same as what is set in Job.
-    Assert.assertEquals(updatedPriority, jobPriority);
+    assertEquals(updatedPriority, jobPriority);
   }
 
   @Test
@@ -1013,14 +1031,14 @@ public class TestJobImpl {
     filePolicies.put("file1", true);
     filePolicies.put("jar1", true);
     Job.setFileSharedCacheUploadPolicies(config, filePolicies);
-    Assert.assertEquals(
+    assertEquals(
         2, Job.getArchiveSharedCacheUploadPolicies(config).size());
-    Assert.assertEquals(
+    assertEquals(
         2, Job.getFileSharedCacheUploadPolicies(config).size());
     JobImpl.cleanupSharedCacheUploadPolicies(config);
-    Assert.assertEquals(
+    assertEquals(
         0, Job.getArchiveSharedCacheUploadPolicies(config).size());
-    Assert.assertEquals(
+    assertEquals(
         0, Job.getFileSharedCacheUploadPolicies(config).size());
   }
 
@@ -1088,14 +1106,14 @@ public class TestJobImpl {
       job.handle(new JobTaskEvent(
           MRBuilderUtils.newTaskId(job.getID(), 1, TaskType.MAP),
           TaskState.SUCCEEDED));
-      Assert.assertEquals(JobState.RUNNING, job.getState());
+      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());
+      assertEquals(JobState.RUNNING, job.getState());
     }
   }
 
@@ -1109,7 +1127,7 @@ public class TestJobImpl {
         break;
       }
     }
-    Assert.assertEquals(state, job.getInternalState());
+    assertEquals(state, job.getInternalState());
   }
 
   private void createSpiedMapTasks(Map<NodeReport, TaskId>

+ 47 - 45
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestMapReduceChildJVM.java

@@ -22,7 +22,6 @@ import java.util.ArrayList;
 import java.util.Map;
 
 import org.apache.hadoop.mapreduce.TaskType;
-import org.junit.Assert;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.JobConf;
@@ -37,16 +36,22 @@ import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherEvent;
 import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerRemoteLaunchEvent;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
 public class TestMapReduceChildJVM {
 
   private static final Logger LOG =
       LoggerFactory.getLogger(TestMapReduceChildJVM.class);
 
-  @Test (timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testCommandLine() throws Exception {
 
     MyMRApp app = new MyMRApp(1, 0, true, this.getClass().getName(), true);
@@ -56,7 +61,7 @@ public class TestMapReduceChildJVM {
     app.waitForState(job, JobState.SUCCEEDED);
     app.verifyCompleted();
 
-    Assert.assertEquals(
+    assertEquals(
       "[" + MRApps.crossPlatformify("JAVA_HOME") + "/bin/java" +
       " -Djava.net.preferIPv4Stack=true" +
       " -Dhadoop.metrics.log.level=WARN " +
@@ -72,23 +77,24 @@ public class TestMapReduceChildJVM {
       " 1><LOG_DIR>/stdout" +
       " 2><LOG_DIR>/stderr ]", app.launchCmdList.get(0));
     
-    Assert.assertTrue("HADOOP_ROOT_LOGGER not set for job",
-      app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"));
-    Assert.assertEquals("INFO,console",
-      app.cmdEnvironment.get("HADOOP_ROOT_LOGGER"));
-    Assert.assertTrue("HADOOP_CLIENT_OPTS not set for job",
-      app.cmdEnvironment.containsKey("HADOOP_CLIENT_OPTS"));
-    Assert.assertEquals("", app.cmdEnvironment.get("HADOOP_CLIENT_OPTS"));
+    assertTrue(app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"),
+        "HADOOP_ROOT_LOGGER not set for job");
+    assertEquals("INFO,console", app.cmdEnvironment.get("HADOOP_ROOT_LOGGER"));
+    assertTrue(app.cmdEnvironment.containsKey("HADOOP_CLIENT_OPTS"),
+        "HADOOP_CLIENT_OPTS not set for job");
+    assertEquals("", app.cmdEnvironment.get("HADOOP_CLIENT_OPTS"));
   }
 
-  @Test (timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testReduceCommandLineWithSeparateShuffle() throws Exception {
     final Configuration conf = new Configuration();
     conf.setBoolean(MRJobConfig.REDUCE_SEPARATE_SHUFFLE_LOG, true);
     testReduceCommandLine(conf);
   }
 
-  @Test (timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testReduceCommandLineWithSeparateCRLAShuffle() throws Exception {
     final Configuration conf = new Configuration();
     conf.setBoolean(MRJobConfig.REDUCE_SEPARATE_SHUFFLE_LOG, true);
@@ -97,7 +103,8 @@ public class TestMapReduceChildJVM {
     testReduceCommandLine(conf);
   }
 
-  @Test (timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testReduceCommandLine() throws Exception {
     final Configuration conf = new Configuration();
     testReduceCommandLine(conf);
@@ -120,7 +127,7 @@ public class TestMapReduceChildJVM {
         ? "shuffleCRLA"
         : "shuffleCLA";
 
-    Assert.assertEquals(
+    assertEquals(
         "[" + MRApps.crossPlatformify("JAVA_HOME") + "/bin/java" +
             " -Djava.net.preferIPv4Stack=true" +
             " -Dhadoop.metrics.log.level=WARN " +
@@ -140,16 +147,16 @@ public class TestMapReduceChildJVM {
             " 1><LOG_DIR>/stdout" +
             " 2><LOG_DIR>/stderr ]", app.launchCmdList.get(0));
 
-    Assert.assertTrue("HADOOP_ROOT_LOGGER not set for job",
-        app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"));
-    Assert.assertEquals("INFO,console",
-        app.cmdEnvironment.get("HADOOP_ROOT_LOGGER"));
-    Assert.assertTrue("HADOOP_CLIENT_OPTS not set for job",
-        app.cmdEnvironment.containsKey("HADOOP_CLIENT_OPTS"));
-    Assert.assertEquals("", app.cmdEnvironment.get("HADOOP_CLIENT_OPTS"));
+    assertTrue(app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"),
+        "HADOOP_ROOT_LOGGER not set for job");
+    assertEquals("INFO,console", app.cmdEnvironment.get("HADOOP_ROOT_LOGGER"));
+    assertTrue(app.cmdEnvironment.containsKey("HADOOP_CLIENT_OPTS"),
+        "HADOOP_CLIENT_OPTS not set for job");
+    assertEquals("", app.cmdEnvironment.get("HADOOP_CLIENT_OPTS"));
   }
   
-  @Test (timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testCommandLineWithLog4JConfig() throws Exception {
 
     MyMRApp app = new MyMRApp(1, 0, true, this.getClass().getName(), true);
@@ -162,7 +169,7 @@ public class TestMapReduceChildJVM {
     app.waitForState(job, JobState.SUCCEEDED);
     app.verifyCompleted();
 
-    Assert.assertEquals(
+    assertEquals(
       "[" + MRApps.crossPlatformify("JAVA_HOME") + "/bin/java" +
       " -Djava.net.preferIPv4Stack=true" +
       " -Dhadoop.metrics.log.level=WARN " +
@@ -204,10 +211,8 @@ public class TestMapReduceChildJVM {
         MRJobConfig.DEFAULT_HEAP_MEMORY_MB_RATIO);
 
     // Verify map and reduce java opts are not set by default
-    Assert.assertNull("Default map java opts!",
-        conf.get(MRJobConfig.MAP_JAVA_OPTS));
-    Assert.assertNull("Default reduce java opts!",
-        conf.get(MRJobConfig.REDUCE_JAVA_OPTS));
+    assertNull(conf.get(MRJobConfig.MAP_JAVA_OPTS), "Default map java opts!");
+    assertNull(conf.get(MRJobConfig.REDUCE_JAVA_OPTS), "Default reduce java opts!");
     // Set the memory-mbs and java-opts
     if (mapMb > 0) {
       conf.setInt(MRJobConfig.MAP_MEMORY_MB, mapMb);
@@ -243,8 +248,8 @@ public class TestMapReduceChildJVM {
             : MRJobConfig.REDUCE_JAVA_OPTS);
         heapMb = JobConf.parseMaximumHeapSizeMB(javaOpts);
       }
-      Assert.assertEquals("Incorrect heapsize in the command opts",
-          heapMb, JobConf.parseMaximumHeapSizeMB(cmd));
+      assertEquals(heapMb, JobConf.parseMaximumHeapSizeMB(cmd),
+          "Incorrect heapsize in the command opts");
     }
   }
 
@@ -289,13 +294,12 @@ public class TestMapReduceChildJVM {
     app.waitForState(job, JobState.SUCCEEDED);
     app.verifyCompleted();
     
-    Assert.assertTrue("HADOOP_ROOT_LOGGER not set for job",
-        app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"));
-    Assert.assertEquals("WARN,console",
-        app.cmdEnvironment.get("HADOOP_ROOT_LOGGER"));
-    Assert.assertTrue("HADOOP_CLIENT_OPTS not set for job",
-        app.cmdEnvironment.containsKey("HADOOP_CLIENT_OPTS"));
-    Assert.assertEquals("test", app.cmdEnvironment.get("HADOOP_CLIENT_OPTS"));
+    assertTrue(app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"),
+        "HADOOP_ROOT_LOGGER not set for job");
+    assertEquals("WARN,console", app.cmdEnvironment.get("HADOOP_ROOT_LOGGER"));
+    assertTrue(app.cmdEnvironment.containsKey("HADOOP_CLIENT_OPTS"),
+        "HADOOP_CLIENT_OPTS not set for job");
+    assertEquals("test", app.cmdEnvironment.get("HADOOP_CLIENT_OPTS"));
 
     // Try one more.
     app = new MyMRApp(1, 0, true, this.getClass().getName(), true);
@@ -305,10 +309,9 @@ public class TestMapReduceChildJVM {
     app.waitForState(job, JobState.SUCCEEDED);
     app.verifyCompleted();
     
-    Assert.assertTrue("HADOOP_ROOT_LOGGER not set for job",
-        app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"));
-    Assert.assertEquals("trace",
-        app.cmdEnvironment.get("HADOOP_ROOT_LOGGER"));
+    assertTrue(app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"),
+        "HADOOP_ROOT_LOGGER not set for job");
+    assertEquals("trace", app.cmdEnvironment.get("HADOOP_ROOT_LOGGER"));
 
     // Try one using the mapreduce.task.env.var=value syntax
     app = new MyMRApp(1, 0, true, this.getClass().getName(), true);
@@ -319,9 +322,8 @@ public class TestMapReduceChildJVM {
     app.waitForState(job, JobState.SUCCEEDED);
     app.verifyCompleted();
 
-    Assert.assertTrue("HADOOP_ROOT_LOGGER not set for job",
-        app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"));
-    Assert.assertEquals("DEBUG,console",
-        app.cmdEnvironment.get("HADOOP_ROOT_LOGGER"));
+    assertTrue(app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"),
+        "HADOOP_ROOT_LOGGER not set for job");
+    assertEquals("DEBUG,console", app.cmdEnvironment.get("HADOOP_ROOT_LOGGER"));
   }
 }

+ 9 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestShuffleProvider.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.mapreduce.v2.app.job.impl;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -53,8 +55,7 @@ import org.apache.hadoop.yarn.server.api.AuxiliaryService;
 import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext;
 import org.apache.hadoop.yarn.server.api.ApplicationTerminationContext;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.junit.Test;
-import org.junit.Assert;
+import org.junit.jupiter.api.Test;
 
 public class TestShuffleProvider {
 
@@ -110,9 +111,12 @@ public class TestShuffleProvider {
             credentials);
 
     Map<String, ByteBuffer> serviceDataMap = launchCtx.getServiceData();
-    Assert.assertNotNull("TestShuffleHandler1 is missing", serviceDataMap.get(TestShuffleHandler1.MAPREDUCE_TEST_SHUFFLE_SERVICEID));
-    Assert.assertNotNull("TestShuffleHandler2 is missing", serviceDataMap.get(TestShuffleHandler2.MAPREDUCE_TEST_SHUFFLE_SERVICEID));
-    Assert.assertTrue("mismatch number of services in map", serviceDataMap.size() == 3); // 2 that we entered + 1 for the built-in shuffle-provider
+    assertNotNull(serviceDataMap.get(TestShuffleHandler1.MAPREDUCE_TEST_SHUFFLE_SERVICEID),
+        "TestShuffleHandler1 is missing");
+    assertNotNull(serviceDataMap.get(TestShuffleHandler2.MAPREDUCE_TEST_SHUFFLE_SERVICEID),
+        "TestShuffleHandler2 is missing");
+    // 2 that we entered + 1 for the built-in shuffle-provider
+    assertEquals(3, serviceDataMap.size(), "mismatch number of services in map");
   }
 
   static public class StubbedFS extends RawLocalFileSystem {

+ 153 - 162
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java

@@ -20,9 +20,11 @@ package org.apache.hadoop.mapreduce.v2.app.job.impl;
 
 import static org.apache.hadoop.test.GenericTestUtils.waitFor;
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
@@ -41,10 +43,9 @@ import java.util.concurrent.CopyOnWriteArrayList;
 import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptFailEvent;
 import org.apache.hadoop.yarn.util.resource.CustomResourceTypesConfigurationProvider;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -111,7 +112,7 @@ import org.apache.log4j.AppenderSkeleton;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.apache.log4j.spi.LoggingEvent;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.mockito.ArgumentCaptor;
 
 import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList;
@@ -151,17 +152,17 @@ public class TestTaskAttempt{
     }
   }
 
-  @BeforeClass
+  @BeforeAll
   public static void setupBeforeClass() {
     ResourceUtils.resetResourceTypes(new Configuration());
   }
 
-  @Before
+  @BeforeEach
   public void before() {
     TaskAttemptImpl.RESOURCE_REQUEST_CACHE.clear();
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
     ResourceUtils.resetResourceTypes(new Configuration());
   }
@@ -289,7 +290,7 @@ public class TestTaskAttempt{
     ArgumentCaptor<Event> arg = ArgumentCaptor.forClass(Event.class);
     verify(eventHandler, times(2)).handle(arg.capture());
     if (!(arg.getAllValues().get(1) instanceof ContainerRequestEvent)) {
-      Assert.fail("Second Event not of type ContainerRequestEvent");
+      fail("Second Event not of type ContainerRequestEvent");
     }
     ContainerRequestEvent cre =
         (ContainerRequestEvent) arg.getAllValues().get(1);
@@ -323,7 +324,7 @@ public class TestTaskAttempt{
     ArgumentCaptor<Event> arg = ArgumentCaptor.forClass(Event.class);
     verify(eventHandler, times(2)).handle(arg.capture());
     if (!(arg.getAllValues().get(1) instanceof ContainerRequestEvent)) {
-      Assert.fail("Second Event not of type ContainerRequestEvent");
+      fail("Second Event not of type ContainerRequestEvent");
     }
     Map<String, Boolean> expected = new HashMap<String, Boolean>();
     expected.put("host1", true);
@@ -361,16 +362,16 @@ public class TestTaskAttempt{
     Job job = app.submit(conf);
     app.waitForState(job, JobState.RUNNING);
     Map<TaskId, Task> tasks = job.getTasks();
-    Assert.assertEquals("Num tasks is not correct", 2, tasks.size());
+    assertEquals(2, tasks.size(), "Num tasks is not correct");
     Iterator<Task> taskIter = tasks.values().iterator();
     Task mTask = taskIter.next();
     app.waitForState(mTask, TaskState.RUNNING);
     Task rTask = taskIter.next();
     app.waitForState(rTask, TaskState.RUNNING);
     Map<TaskAttemptId, TaskAttempt> mAttempts = mTask.getAttempts();
-    Assert.assertEquals("Num attempts is not correct", 1, mAttempts.size());
+    assertEquals(1, mAttempts.size(), "Num attempts is not correct");
     Map<TaskAttemptId, TaskAttempt> rAttempts = rTask.getAttempts();
-    Assert.assertEquals("Num attempts is not correct", 1, rAttempts.size());
+    assertEquals(1, rAttempts.size(), "Num attempts is not correct");
     TaskAttempt mta = mAttempts.values().iterator().next();
     TaskAttempt rta = rAttempts.values().iterator().next();
     app.waitForState(mta, TaskAttemptState.RUNNING);
@@ -392,21 +393,21 @@ public class TestTaskAttempt{
 
     int memoryMb = (int) containerResource.getMemorySize();
     int vcores = containerResource.getVirtualCores();
-    Assert.assertEquals((int) Math.ceil((float) memoryMb / minContainerSize),
+    assertEquals((int) Math.ceil((float) memoryMb / minContainerSize),
         counters.findCounter(JobCounter.SLOTS_MILLIS_MAPS).getValue());
-    Assert.assertEquals((int) Math.ceil((float) memoryMb / minContainerSize),
+    assertEquals((int) Math.ceil((float) memoryMb / minContainerSize),
         counters.findCounter(JobCounter.SLOTS_MILLIS_REDUCES).getValue());
-    Assert.assertEquals(1,
+    assertEquals(1,
         counters.findCounter(JobCounter.MILLIS_MAPS).getValue());
-    Assert.assertEquals(1,
+    assertEquals(1,
         counters.findCounter(JobCounter.MILLIS_REDUCES).getValue());
-    Assert.assertEquals(memoryMb,
+    assertEquals(memoryMb,
         counters.findCounter(JobCounter.MB_MILLIS_MAPS).getValue());
-    Assert.assertEquals(memoryMb,
+    assertEquals(memoryMb,
         counters.findCounter(JobCounter.MB_MILLIS_REDUCES).getValue());
-    Assert.assertEquals(vcores,
+    assertEquals(vcores,
         counters.findCounter(JobCounter.VCORES_MILLIS_MAPS).getValue());
-    Assert.assertEquals(vcores,
+    assertEquals(vcores,
         counters.findCounter(JobCounter.VCORES_MILLIS_REDUCES).getValue());
   }
 
@@ -452,23 +453,23 @@ public class TestTaskAttempt{
     app.waitForState(job, JobState.FAILED);
     Map<TaskId, Task> tasks = job.getTasks();
 
-    Assert.assertEquals("Num tasks is not correct", 1, tasks.size());
+    assertEquals(1, tasks.size(), "Num tasks is not correct");
     Task task = tasks.values().iterator().next();
-    Assert.assertEquals("Task state not correct", TaskState.FAILED, task
-        .getReport().getTaskState());
+    assertEquals(TaskState.FAILED, task
+        .getReport().getTaskState(), "Task state not correct");
     Map<TaskAttemptId, TaskAttempt> attempts = tasks.values().iterator().next()
         .getAttempts();
-    Assert.assertEquals("Num attempts is not correct", 4, attempts.size());
+    assertEquals(4, attempts.size(), "Num attempts is not correct");
 
     Iterator<TaskAttempt> it = attempts.values().iterator();
     TaskAttemptReport report = it.next().getReport();
-    Assert.assertEquals("Attempt state not correct", TaskAttemptState.FAILED,
-        report.getTaskAttemptState());
-    Assert.assertEquals("Diagnostic Information is not Correct",
-        "Test Diagnostic Event", report.getDiagnosticInfo());
+    assertEquals(TaskAttemptState.FAILED,
+        report.getTaskAttemptState(), "Attempt state not correct");
+    assertEquals("Test Diagnostic Event", report.getDiagnosticInfo(),
+        "Diagnostic Information is not Correct");
     report = it.next().getReport();
-    Assert.assertEquals("Attempt state not correct", TaskAttemptState.FAILED,
-        report.getTaskAttemptState());
+    assertEquals(TaskAttemptState.FAILED,
+        report.getTaskAttemptState(), "Attempt state not correct");
   }
 
   private void testTaskAttemptAssignedFailHistory
@@ -477,8 +478,8 @@ public class TestTaskAttempt{
     Job job = app.submit(conf);
     app.waitForState(job, JobState.FAILED);
     Map<TaskId, Task> tasks = job.getTasks();
-    Assert.assertTrue("No Ta Started JH Event", app.getTaStartJHEvent());
-    Assert.assertTrue("No Ta Failed JH Event", app.getTaFailedJHEvent());
+    assertTrue(app.getTaStartJHEvent(), "No Ta Started JH Event");
+    assertTrue(app.getTaFailedJHEvent(), "No Ta Failed JH Event");
   }
 
   private void testTaskAttemptAssignedKilledHistory
@@ -518,8 +519,8 @@ public class TestTaskAttempt{
           if (event.getType() == org.apache.hadoop.mapreduce.jobhistory.EventType.MAP_ATTEMPT_FAILED) {
             TaskAttemptUnsuccessfulCompletion datum = (TaskAttemptUnsuccessfulCompletion) event
                 .getHistoryEvent().getDatum();
-            Assert.assertEquals("Diagnostic Information is not Correct",
-                "Test Diagnostic Event", datum.get(8).toString());
+            assertEquals("Test Diagnostic Event", datum.get(8).toString(),
+                "Diagnostic Information is not Correct");
           }
         }
       };
@@ -638,8 +639,8 @@ public class TestTaskAttempt{
     taImpl.handle(new TaskAttemptEvent(attemptId,
         TaskAttemptEventType.TA_CONTAINER_LAUNCH_FAILED));
     assertFalse(eventHandler.internalError);
-    assertEquals("Task attempt is not assigned on the local node", 
-        Locality.NODE_LOCAL, taImpl.getLocality());
+    assertEquals(Locality.NODE_LOCAL, taImpl.getLocality(),
+        "Task attempt is not assigned on the local node");
   }
 
   @Test
@@ -695,10 +696,10 @@ public class TestTaskAttempt{
         .isEqualTo(TaskAttemptState.RUNNING);
     taImpl.handle(new TaskAttemptEvent(attemptId,
         TaskAttemptEventType.TA_CONTAINER_CLEANED));
-    assertFalse("InternalError occurred trying to handle TA_CONTAINER_CLEANED",
-        eventHandler.internalError);
-    assertEquals("Task attempt is not assigned on the local rack",
-        Locality.RACK_LOCAL, taImpl.getLocality());
+    assertFalse(eventHandler.internalError,
+        "InternalError occurred trying to handle TA_CONTAINER_CLEANED");
+    assertEquals(Locality.RACK_LOCAL, taImpl.getLocality(),
+        "Task attempt is not assigned on the local rack");
   }
 
   @Test
@@ -757,10 +758,10 @@ public class TestTaskAttempt{
         .isEqualTo(TaskAttemptState.COMMIT_PENDING);
     taImpl.handle(new TaskAttemptEvent(attemptId,
         TaskAttemptEventType.TA_CONTAINER_CLEANED));
-    assertFalse("InternalError occurred trying to handle TA_CONTAINER_CLEANED",
-        eventHandler.internalError);
-    assertEquals("Task attempt is assigned locally", Locality.OFF_SWITCH,
-        taImpl.getLocality());
+    assertFalse(eventHandler.internalError,
+        "InternalError occurred trying to handle TA_CONTAINER_CLEANED");
+    assertEquals(Locality.OFF_SWITCH,
+        taImpl.getLocality(), "Task attempt is assigned locally");
   }
 
   @Test
@@ -832,8 +833,8 @@ public class TestTaskAttempt{
     assertThat(taImpl.getState())
         .withFailMessage("Task attempt is not in FAILED state, still")
         .isEqualTo(TaskAttemptState.FAILED);
-    assertFalse("InternalError occurred trying to handle TA_CONTAINER_CLEANED",
-        eventHandler.internalError);
+    assertFalse(eventHandler.internalError,
+        "InternalError occurred trying to handle TA_CONTAINER_CLEANED");
   }
 
 
@@ -883,16 +884,14 @@ public class TestTaskAttempt{
         TaskAttemptEventType.TA_SCHEDULE));
     taImpl.handle(new TaskAttemptDiagnosticsUpdateEvent(attemptId,
         "Task got killed"));
-    assertFalse(
-        "InternalError occurred trying to handle TA_DIAGNOSTICS_UPDATE on assigned task",
-        eventHandler.internalError);
+    assertFalse(eventHandler.internalError,
+        "InternalError occurred trying to handle TA_DIAGNOSTICS_UPDATE on assigned task");
     try {
       taImpl.handle(new TaskAttemptEvent(attemptId,
           TaskAttemptEventType.TA_KILL));
-      Assert.assertTrue("No exception on UNASSIGNED STATE KILL event", true);
+      assertTrue(true, "No exception on UNASSIGNED STATE KILL event");
     } catch (Exception e) {
-      Assert.assertFalse(
-          "Exception not expected for UNASSIGNED STATE KILL event", true);
+      fail("Exception not expected for UNASSIGNED STATE KILL event");
     }
   }
 
@@ -962,8 +961,8 @@ public class TestTaskAttempt{
     assertThat(taImpl.getState())
         .withFailMessage("Task attempt is not in KILLED state, still")
         .isEqualTo(TaskAttemptState.KILLED);
-    assertFalse("InternalError occurred trying to handle TA_CONTAINER_CLEANED",
-      eventHandler.internalError);
+    assertFalse(eventHandler.internalError,
+        "InternalError occurred trying to handle TA_CONTAINER_CLEANED");
   }
 
   @Test
@@ -1009,9 +1008,8 @@ public class TestTaskAttempt{
     when(container.getNodeHttpAddress()).thenReturn("localhost:0");
     taImpl.handle(new TaskAttemptDiagnosticsUpdateEvent(attemptId,
         "Task got killed"));
-    assertFalse(
-        "InternalError occurred trying to handle TA_DIAGNOSTICS_UPDATE on assigned task",
-        eventHandler.internalError);
+    assertFalse(eventHandler.internalError,
+        "InternalError occurred trying to handle TA_DIAGNOSTICS_UPDATE on assigned task");
   }
     
   @Test
@@ -1072,8 +1070,8 @@ public class TestTaskAttempt{
         .withFailMessage("Task attempt is not in SUCCEEDED state")
         .isEqualTo(TaskAttemptState.SUCCEEDED);
 
-    assertTrue("Task Attempt finish time is not greater than 0",
-        taImpl.getFinishTime() > 0);
+    assertTrue(taImpl.getFinishTime() > 0,
+        "Task Attempt finish time is not greater than 0");
 
     Long finishTime = taImpl.getFinishTime();
     Thread.sleep(5);
@@ -1084,9 +1082,9 @@ public class TestTaskAttempt{
         .withFailMessage("Task attempt is not in FAILED state")
         .isEqualTo(TaskAttemptState.FAILED);
 
-    assertEquals("After TA_TOO_MANY_FETCH_FAILURE,"
-        + " Task attempt finish time is not the same ",
-        finishTime, Long.valueOf(taImpl.getFinishTime()));
+    assertEquals(finishTime, Long.valueOf(taImpl.getFinishTime()),
+        "After TA_TOO_MANY_FETCH_FAILURE,"
+        + " Task attempt finish time is not the same ");
   }
 
   private void containerKillBeforeAssignment(boolean scheduleAttempt)
@@ -1114,7 +1112,7 @@ public class TestTaskAttempt{
     assertThat(taImpl.getInternalState())
         .withFailMessage("Task attempt's internal state is not KILLED")
         .isEqualTo(TaskAttemptStateInternal.KILLED);
-    assertFalse("InternalError occurred", eventHandler.internalError);
+    assertFalse(eventHandler.internalError, "InternalError occurred");
     TaskEvent event = eventHandler.lastTaskEvent;
     assertEquals(TaskEventType.T_ATTEMPT_KILLED, event.getType());
     // In NEW state, new map attempt should not be rescheduled.
@@ -1238,8 +1236,8 @@ public class TestTaskAttempt{
         .isEqualTo(TaskAttemptState.RUNNING);
     taImpl.handle(new TaskAttemptEvent(attemptId,
         TaskAttemptEventType.TA_KILL));
-    assertFalse("InternalError occurred trying to handle TA_KILL",
-        eventHandler.internalError);
+    assertFalse(eventHandler.internalError,
+        "InternalError occurred trying to handle TA_KILL");
     assertThat(taImpl.getInternalState())
         .withFailMessage("Task should be in KILL_CONTAINER_CLEANUP state")
         .isEqualTo(TaskAttemptStateInternal.KILL_CONTAINER_CLEANUP);
@@ -1301,8 +1299,8 @@ public class TestTaskAttempt{
         .isEqualTo(TaskAttemptStateInternal.COMMIT_PENDING);
     taImpl.handle(new TaskAttemptEvent(attemptId,
         TaskAttemptEventType.TA_KILL));
-    assertFalse("InternalError occurred trying to handle TA_KILL",
-        eventHandler.internalError);
+    assertFalse(eventHandler.internalError,
+        "InternalError occurred trying to handle TA_KILL");
     assertThat(taImpl.getInternalState())
         .withFailMessage("Task should be in KILL_CONTAINER_CLEANUP state")
         .isEqualTo(TaskAttemptStateInternal.KILL_CONTAINER_CLEANUP);
@@ -1348,7 +1346,7 @@ public class TestTaskAttempt{
         .withFailMessage("Task attempt is not in KILLED state")
         .isEqualTo(TaskAttemptState.KILLED);
 
-    assertFalse("InternalError occurred", eventHandler.internalError);
+    assertFalse(eventHandler.internalError, "InternalError occurred");
   }
 
   @Test
@@ -1359,32 +1357,30 @@ public class TestTaskAttempt{
     taImpl.handle(new TaskAttemptEvent(taImpl.getID(),
         TaskAttemptEventType.TA_DONE));
 
-    assertEquals("Task attempt is not in SUCCEEDED state",
-        TaskAttemptState.SUCCEEDED, taImpl.getState());
-    assertEquals("Task attempt's internal state is not " +
-            "SUCCESS_FINISHING_CONTAINER",
-        TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
-        taImpl.getInternalState());
+    assertEquals(TaskAttemptState.SUCCEEDED, taImpl.getState(),
+        "Task attempt is not in SUCCEEDED state");
+    assertEquals(TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
+        taImpl.getInternalState(), "Task attempt's internal state is not " +
+        "SUCCESS_FINISHING_CONTAINER");
 
     // If the map only task is killed when it is in SUCCESS_FINISHING_CONTAINER
     // state, the state will move to SUCCESS_CONTAINER_CLEANUP
     taImpl.handle(new TaskAttemptEvent(taImpl.getID(),
         TaskAttemptEventType.TA_KILL));
-    assertEquals("Task attempt is not in SUCCEEDED state",
-        TaskAttemptState.SUCCEEDED, taImpl.getState());
-    assertEquals("Task attempt's internal state is not " +
-            "SUCCESS_CONTAINER_CLEANUP",
-        TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP,
-        taImpl.getInternalState());
+    assertEquals(TaskAttemptState.SUCCEEDED,
+        taImpl.getState(), "Task attempt is not in SUCCEEDED state");
+    assertEquals(TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP,
+        taImpl.getInternalState(), "Task attempt's internal state is not " +
+        "SUCCESS_CONTAINER_CLEANUP");
 
     taImpl.handle(new TaskAttemptEvent(taImpl.getID(),
         TaskAttemptEventType.TA_CONTAINER_CLEANED));
-    assertEquals("Task attempt is not in SUCCEEDED state",
-        TaskAttemptState.SUCCEEDED, taImpl.getState());
-    assertEquals("Task attempt's internal state is not SUCCEEDED state",
-        TaskAttemptStateInternal.SUCCEEDED, taImpl.getInternalState());
+    assertEquals(TaskAttemptState.SUCCEEDED, taImpl.getState(),
+        "Task attempt is not in SUCCEEDED state");
+    assertEquals(TaskAttemptStateInternal.SUCCEEDED, taImpl.getInternalState(),
+        "Task attempt's internal state is not SUCCEEDED state");
 
-    assertFalse("InternalError occurred", eventHandler.internalError);
+    assertFalse(eventHandler.internalError, "InternalError occurred");
   }
 
   @Test
@@ -1414,7 +1410,7 @@ public class TestTaskAttempt{
     assertThat(taImpl.getInternalState())
         .withFailMessage("Task attempt's internal state is not KILLED")
         .isEqualTo(TaskAttemptStateInternal.KILLED);
-    assertFalse("InternalError occurred", eventHandler.internalError);
+    assertFalse(eventHandler.internalError, "InternalError occurred");
     TaskEvent event = eventHandler.lastTaskEvent;
     assertEquals(TaskEventType.T_ATTEMPT_KILLED, event.getType());
     // Send an attempt killed event to TaskImpl forwarding the same reschedule
@@ -1430,22 +1426,21 @@ public class TestTaskAttempt{
     taImpl.handle(new TaskAttemptEvent(taImpl.getID(),
         TaskAttemptEventType.TA_DONE));
 
-    assertEquals("Task attempt is not in SUCCEEDED state",
-        TaskAttemptState.SUCCEEDED, taImpl.getState());
-    assertEquals("Task attempt's internal state is not " +
-            "SUCCESS_FINISHING_CONTAINER",
-        TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
-        taImpl.getInternalState());
+    assertEquals(TaskAttemptState.SUCCEEDED, taImpl.getState(),
+        "Task attempt is not in SUCCEEDED state");
+    assertEquals(TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
+        taImpl.getInternalState(), "Task attempt's internal state is not " +
+        "SUCCESS_FINISHING_CONTAINER");
 
     taImpl.handle(new TaskAttemptEvent(taImpl.getID(),
         TaskAttemptEventType.TA_CONTAINER_CLEANED));
     // Succeeded
     taImpl.handle(new TaskAttemptKillEvent(taImpl.getID(),"", true));
-    assertEquals("Task attempt is not in SUCCEEDED state",
-        TaskAttemptState.SUCCEEDED, taImpl.getState());
-    assertEquals("Task attempt's internal state is not SUCCEEDED",
-        TaskAttemptStateInternal.SUCCEEDED, taImpl.getInternalState());
-    assertFalse("InternalError occurred", eventHandler.internalError);
+    assertEquals(TaskAttemptState.SUCCEEDED, taImpl.getState(),
+        "Task attempt is not in SUCCEEDED state");
+    assertEquals(TaskAttemptStateInternal.SUCCEEDED,
+        taImpl.getInternalState(), "Task attempt's internal state is not SUCCEEDED");
+    assertFalse(eventHandler.internalError, "InternalError occurred");
     TaskEvent event = eventHandler.lastTaskEvent;
     assertEquals(TaskEventType.T_ATTEMPT_SUCCEEDED, event.getType());
   }
@@ -1498,7 +1493,7 @@ public class TestTaskAttempt{
         .withFailMessage("Task attempt is not in FAILED state")
         .isEqualTo(TaskAttemptState.FAILED);
 
-    assertFalse("InternalError occurred", eventHandler.internalError);
+    assertFalse(eventHandler.internalError, "InternalError occurred");
   }
 
   @Test
@@ -1531,7 +1526,7 @@ public class TestTaskAttempt{
         .withFailMessage("Task attempt is not in FAILED state")
         .isEqualTo(TaskAttemptState.FAILED);
 
-    assertFalse("InternalError occurred", eventHandler.internalError);
+    assertFalse(eventHandler.internalError, "InternalError occurred");
   }
 
   @Test
@@ -1561,7 +1556,7 @@ public class TestTaskAttempt{
             "SUCCESS_FINISHING_CONTAINER")
         .isEqualTo(TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER);
 
-    assertFalse("InternalError occurred", eventHandler.internalError);
+    assertFalse(eventHandler.internalError, "InternalError occurred");
   }
 
   @Test
@@ -1592,7 +1587,7 @@ public class TestTaskAttempt{
                 "SUCCESS_CONTAINER_CLEANUP")
             .isEqualTo(TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP);
 
-    assertFalse("InternalError occurred", eventHandler.internalError);
+    assertFalse(eventHandler.internalError, "InternalError occurred");
   }
 
   @Test
@@ -1619,7 +1614,7 @@ public class TestTaskAttempt{
             "FAIL_CONTAINER_CLEANUP")
         .isEqualTo(TaskAttemptStateInternal.FAIL_CONTAINER_CLEANUP);
 
-    assertFalse("InternalError occurred", eventHandler.internalError);
+    assertFalse(eventHandler.internalError, "InternalError occurred");
   }
 
   @Test
@@ -1636,8 +1631,8 @@ public class TestTaskAttempt{
     ResourceInformation resourceInfo =
         getResourceInfoFromContainerRequest(taImpl, eventHandler).
         getResourceInformation(CUSTOM_RESOURCE_NAME);
-    assertEquals("Expecting the default unit (G)",
-        "G", resourceInfo.getUnits());
+    assertEquals("G", resourceInfo.getUnits(),
+        "Expecting the default unit (G)");
     assertEquals(7L, resourceInfo.getValue());
   }
 
@@ -1654,8 +1649,8 @@ public class TestTaskAttempt{
     ResourceInformation resourceInfo =
         getResourceInfoFromContainerRequest(taImpl, eventHandler).
         getResourceInformation(CUSTOM_RESOURCE_NAME);
-    assertEquals("Expecting the specified unit (m)",
-        "m", resourceInfo.getUnits());
+    assertEquals("m", resourceInfo.getUnits(),
+        "Expecting the specified unit (m)");
     assertEquals(3L, resourceInfo.getValue());
   }
 
@@ -1752,18 +1747,20 @@ public class TestTaskAttempt{
     }
   }
 
-  @Test(expected=IllegalArgumentException.class)
+  @Test
   public void testReducerMemoryRequestMultipleName() {
-    EventHandler eventHandler = mock(EventHandler.class);
-    Clock clock = SystemClock.getInstance();
-    JobConf jobConf = new JobConf();
-    for (String memoryName : ImmutableList.of(
-        MRJobConfig.RESOURCE_TYPE_NAME_MEMORY,
-        MRJobConfig.RESOURCE_TYPE_ALTERNATIVE_NAME_MEMORY)) {
-      jobConf.set(MRJobConfig.REDUCE_RESOURCE_TYPE_PREFIX + memoryName,
-          "3Gi");
-    }
-    createReduceTaskAttemptImplForTest(eventHandler, clock, jobConf);
+    assertThrows(IllegalArgumentException.class, ()->{
+      EventHandler eventHandler = mock(EventHandler.class);
+      Clock clock = SystemClock.getInstance();
+      JobConf jobConf = new JobConf();
+      for (String memoryName : ImmutableList.of(
+          MRJobConfig.RESOURCE_TYPE_NAME_MEMORY,
+          MRJobConfig.RESOURCE_TYPE_ALTERNATIVE_NAME_MEMORY)) {
+        jobConf.set(MRJobConfig.REDUCE_RESOURCE_TYPE_PREFIX + memoryName,
+            "3Gi");
+      }
+      createReduceTaskAttemptImplForTest(eventHandler, clock, jobConf);
+    });
   }
 
   @Test
@@ -1853,21 +1850,23 @@ public class TestTaskAttempt{
         containerRequestEvents.add((ContainerRequestEvent) e);
       }
     }
-    assertEquals("Expected one ContainerRequestEvent after scheduling "
-        + "task attempt", 1, containerRequestEvents.size());
+    assertEquals(1, containerRequestEvents.size(),
+        "Expected one ContainerRequestEvent after scheduling task attempt");
 
     return containerRequestEvents.get(0).getCapability();
   }
 
-  @Test(expected=IllegalArgumentException.class)
+  @Test
   public void testReducerCustomResourceTypeWithInvalidUnit() {
-    initResourceTypes();
-    EventHandler eventHandler = mock(EventHandler.class);
-    Clock clock = SystemClock.getInstance();
-    JobConf jobConf = new JobConf();
-    jobConf.set(MRJobConfig.REDUCE_RESOURCE_TYPE_PREFIX
-        + CUSTOM_RESOURCE_NAME, "3z");
-    createReduceTaskAttemptImplForTest(eventHandler, clock, jobConf);
+    assertThrows(IllegalArgumentException.class, () -> {
+      initResourceTypes();
+      EventHandler eventHandler = mock(EventHandler.class);
+      Clock clock = SystemClock.getInstance();
+      JobConf jobConf = new JobConf();
+      jobConf.set(MRJobConfig.REDUCE_RESOURCE_TYPE_PREFIX
+          + CUSTOM_RESOURCE_NAME, "3z");
+      createReduceTaskAttemptImplForTest(eventHandler, clock, jobConf);
+    });
   }
 
   @Test
@@ -1882,22 +1881,19 @@ public class TestTaskAttempt{
     // move in two steps to the desired state (cannot get there directly)
     taImpl.handle(new TaskAttemptEvent(taImpl.getID(),
         TaskAttemptEventType.TA_DONE));
-    assertEquals("Task attempt's internal state is not " +
-        "SUCCESS_FINISHING_CONTAINER",
-        TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
-        taImpl.getInternalState());
+    assertEquals(TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
+        taImpl.getInternalState(), "Task attempt's internal state is not " +
+        "SUCCESS_FINISHING_CONTAINER");
 
     taImpl.handle(new TaskAttemptEvent(taImpl.getID(),
         TaskAttemptEventType.TA_TIMED_OUT));
-    assertEquals("Task attempt's internal state is not " +
-        "SUCCESS_CONTAINER_CLEANUP",
-        TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP,
-        taImpl.getInternalState());
+    assertEquals(TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP,
+        taImpl.getInternalState(), "Task attempt's internal state is not " +
+        "SUCCESS_CONTAINER_CLEANUP");
 
     taImpl.handle(new TaskAttemptKillEvent(mapTAId, "", true));
-    assertEquals("Task attempt is not in KILLED state",
-        TaskAttemptState.KILLED,
-        taImpl.getState());
+    assertEquals(TaskAttemptState.KILLED,
+        taImpl.getState(), "Task attempt is not in KILLED state");
   }
 
   @Test
@@ -1912,24 +1908,21 @@ public class TestTaskAttempt{
     // move in two steps to the desired state (cannot get there directly)
     taImpl.handle(new TaskAttemptEvent(taImpl.getID(),
         TaskAttemptEventType.TA_DONE));
-    assertEquals("Task attempt's internal state is not " +
-        "SUCCESS_FINISHING_CONTAINER",
-        TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
-        taImpl.getInternalState());
+    assertEquals(TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
+        taImpl.getInternalState(), "Task attempt's internal state is not " +
+        "SUCCESS_FINISHING_CONTAINER");
 
     taImpl.handle(new TaskAttemptEvent(taImpl.getID(),
         TaskAttemptEventType.TA_TIMED_OUT));
-    assertEquals("Task attempt's internal state is not " +
-        "SUCCESS_CONTAINER_CLEANUP",
-        TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP,
-        taImpl.getInternalState());
+    assertEquals(TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP,
+        taImpl.getInternalState(), "Task attempt's internal state is not " +
+        "SUCCESS_CONTAINER_CLEANUP");
 
     taImpl.handle(new TaskAttemptTooManyFetchFailureEvent(taImpl.getID(),
         reduceTAId, "Host"));
-    assertEquals("Task attempt is not in FAILED state",
-        TaskAttemptState.FAILED,
-        taImpl.getState());
-    assertFalse("InternalError occurred", eventHandler.internalError);
+    assertEquals(TaskAttemptState.FAILED,
+        taImpl.getState(), "Task attempt is not in FAILED state");
+    assertFalse(eventHandler.internalError, "InternalError occurred");
   }
 
   private void initResourceTypes() {
@@ -1951,17 +1944,15 @@ public class TestTaskAttempt{
     taImpl.handle(new TaskAttemptEvent(taImpl.getID(),
         TaskAttemptEventType.TA_DONE));
 
-    assertEquals("Task attempt's internal state is not " +
-        "SUCCESS_FINISHING_CONTAINER",
-        TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
-        taImpl.getInternalState());
+    assertEquals(TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
+        taImpl.getInternalState(), "Task attempt's internal state is not " +
+        "SUCCESS_FINISHING_CONTAINER");
 
     taImpl.handle(new TaskAttemptTooManyFetchFailureEvent(taImpl.getID(),
         reduceTAId, "Host"));
-    assertEquals("Task attempt is not in FAILED state",
-        TaskAttemptState.FAILED,
-        taImpl.getState());
-    assertFalse("InternalError occurred", eventHandler.internalError);
+    assertEquals(TaskAttemptState.FAILED,
+        taImpl.getState(), "Task attempt is not in FAILED state");
+    assertFalse(eventHandler.internalError, "InternalError occurred");
   }
 
   private void setupTaskAttemptFinishingMonitor(

+ 14 - 16
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttemptContainerRequest.java

@@ -18,6 +18,9 @@
 
 package org.apache.hadoop.mapreduce.v2.app.job.impl;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -27,8 +30,7 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.junit.After;
-import org.junit.Assert;
+import org.junit.jupiter.api.AfterEach;
 
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileStatus;
@@ -58,12 +60,12 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.util.SystemClock;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 @SuppressWarnings({"rawtypes"})
 public class TestTaskAttemptContainerRequest {
 
-  @After
+  @AfterEach
   public void cleanup() {
     UserGroupInformation.reset();
   }
@@ -72,8 +74,7 @@ public class TestTaskAttemptContainerRequest {
   public void testAttemptContainerRequest() throws Exception {
     final Text SECRET_KEY_ALIAS = new Text("secretkeyalias");
     final byte[] SECRET_KEY = ("secretkey").getBytes();
-    Map<ApplicationAccessType, String> acls =
-        new HashMap<ApplicationAccessType, String>(1);
+    Map<ApplicationAccessType, String> acls = new HashMap<>(1);
     acls.put(ApplicationAccessType.VIEW_APP, "otheruser");
     ApplicationId appId = ApplicationId.newInstance(1, 1);
     JobId jobId = MRBuilderUtils.newJobId(appId, 1);
@@ -95,7 +96,7 @@ public class TestTaskAttemptContainerRequest {
 
     Credentials credentials = new Credentials();
     credentials.addSecretKey(SECRET_KEY_ALIAS, SECRET_KEY);
-    Token<JobTokenIdentifier> jobToken = new Token<JobTokenIdentifier>(
+    Token<JobTokenIdentifier> jobToken = new Token<>(
         ("tokenid").getBytes(), ("tokenpw").getBytes(),
         new Text("tokenkind"), new Text("tokenservice"));
 
@@ -114,7 +115,7 @@ public class TestTaskAttemptContainerRequest {
             mock(WrappedJvmID.class), taListener,
             credentials);
 
-    Assert.assertEquals("ACLs mismatch", acls, launchCtx.getApplicationACLs());
+    assertEquals(acls, launchCtx.getApplicationACLs(), "ACLs mismatch");
     Credentials launchCredentials = new Credentials();
 
     DataInputByteBuffer dibb = new DataInputByteBuffer();
@@ -125,17 +126,14 @@ public class TestTaskAttemptContainerRequest {
     for (Token<? extends TokenIdentifier> token : credentials.getAllTokens()) {
       Token<? extends TokenIdentifier> launchToken =
           launchCredentials.getToken(token.getService());
-      Assert.assertNotNull("Token " + token.getService() + " is missing",
-          launchToken);
-      Assert.assertEquals("Token " + token.getService() + " mismatch",
-          token, launchToken);
+      assertNotNull(launchToken, "Token " + token.getService() + " is missing");
+      assertEquals(token, launchToken, "Token " + token.getService() + " mismatch");
     }
 
     // verify the secret key is in the launch context
-    Assert.assertNotNull("Secret key missing",
-        launchCredentials.getSecretKey(SECRET_KEY_ALIAS));
-    Assert.assertTrue("Secret key mismatch", Arrays.equals(SECRET_KEY,
-        launchCredentials.getSecretKey(SECRET_KEY_ALIAS)));
+    assertNotNull(launchCredentials.getSecretKey(SECRET_KEY_ALIAS), "Secret key missing");
+    assertTrue(Arrays.equals(SECRET_KEY,
+        launchCredentials.getSecretKey(SECRET_KEY_ALIAS)), "Secret key mismatch");
   }
 
   static public class StubbedFS extends RawLocalFileSystem {

+ 14 - 14
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java

@@ -17,10 +17,10 @@
  */
 package org.apache.hadoop.mapreduce.v2.app.job.impl;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -65,9 +65,9 @@ import org.apache.hadoop.yarn.event.InlineDispatcher;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.SystemClock;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -234,7 +234,7 @@ public class TestTaskImpl {
     
   }
   
-  @Before 
+  @BeforeEach
   @SuppressWarnings("unchecked")
   public void setup() {
      dispatcher = new InlineDispatcher();
@@ -273,7 +273,7 @@ public class TestTaskImpl {
         startCount, metrics, appContext, taskType);
   }
 
-  @After 
+  @AfterEach
   public void teardown() {
     taskAttempts.clear();
   }
@@ -587,10 +587,10 @@ public class TestTaskImpl {
     mockTask.handle(new TaskTAttemptEvent(getLastAttempt().getAttemptId(), 
         TaskEventType.T_ATTEMPT_SUCCEEDED));
     
-    assertFalse("First attempt should not commit",
-        mockTask.canCommit(taskAttempts.get(0).getAttemptId()));
-    assertTrue("Second attempt should commit",
-        mockTask.canCommit(getLastAttempt().getAttemptId()));
+    assertFalse(mockTask.canCommit(taskAttempts.get(0).getAttemptId()),
+        "First attempt should not commit");
+    assertTrue(mockTask.canCommit(getLastAttempt().getAttemptId()),
+        "Second attempt should commit");
 
     assertTaskSucceededState();
   }
@@ -879,7 +879,7 @@ public class TestTaskImpl {
     baseAttempt.setProgress(1.0f);
 
     Counters taskCounters = mockTask.getCounters();
-    assertEquals("wrong counters for task", specAttemptCounters, taskCounters);
+    assertEquals(specAttemptCounters, taskCounters, "wrong counters for task");
   }
 
   public static class MockTaskAttemptEventHandler implements EventHandler {

+ 34 - 31
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java

@@ -19,6 +19,9 @@
 package org.apache.hadoop.mapreduce.v2.app.launcher;
 
 import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.Mockito.mock;
 
 import java.io.IOException;
@@ -44,7 +47,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.ResourceLocalizationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.ResourceLocalizationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RestartContainerResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RollbackResponse;
-import org.junit.Assert;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.ipc.Server;
@@ -93,7 +95,8 @@ import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
 import org.apache.hadoop.yarn.util.Records;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -107,7 +110,8 @@ public class TestContainerLauncher {
   static final Logger LOG =
       LoggerFactory.getLogger(TestContainerLauncher.class);
 
-  @Test (timeout = 10000)
+  @Test
+  @Timeout(value = 10)
   public void testPoolSize() throws InterruptedException {
 
     ApplicationId appId = ApplicationId.newInstance(12345, 67);
@@ -127,10 +131,10 @@ public class TestContainerLauncher {
     // No events yet
     assertThat(containerLauncher.initialPoolSize).isEqualTo(
         MRJobConfig.DEFAULT_MR_AM_CONTAINERLAUNCHER_THREADPOOL_INITIAL_SIZE);
-    Assert.assertEquals(0, threadPool.getPoolSize());
-    Assert.assertEquals(containerLauncher.initialPoolSize,
+    assertEquals(0, threadPool.getPoolSize());
+    assertEquals(containerLauncher.initialPoolSize,
       threadPool.getCorePoolSize());
-    Assert.assertNull(containerLauncher.foundErrors);
+    assertNull(containerLauncher.foundErrors);
 
     containerLauncher.expectedCorePoolSize = containerLauncher.initialPoolSize;
     for (int i = 0; i < 10; i++) {
@@ -141,8 +145,8 @@ public class TestContainerLauncher {
         ContainerLauncher.EventType.CONTAINER_REMOTE_LAUNCH));
     }
     waitForEvents(containerLauncher, 10);
-    Assert.assertEquals(10, threadPool.getPoolSize());
-    Assert.assertNull(containerLauncher.foundErrors);
+    assertEquals(10, threadPool.getPoolSize());
+    assertNull(containerLauncher.foundErrors);
 
     // Same set of hosts, so no change
     containerLauncher.finishEventHandling = true;
@@ -153,7 +157,7 @@ public class TestContainerLauncher {
           + ". Timeout is " + timeOut);
       Thread.sleep(1000);
     }
-    Assert.assertEquals(10, containerLauncher.numEventsProcessed.get());
+    assertEquals(10, containerLauncher.numEventsProcessed.get());
     containerLauncher.finishEventHandling = false;
     for (int i = 0; i < 10; i++) {
       ContainerId containerId = ContainerId.newContainerId(appAttemptId,
@@ -165,8 +169,8 @@ public class TestContainerLauncher {
         ContainerLauncher.EventType.CONTAINER_REMOTE_LAUNCH));
     }
     waitForEvents(containerLauncher, 20);
-    Assert.assertEquals(10, threadPool.getPoolSize());
-    Assert.assertNull(containerLauncher.foundErrors);
+    assertEquals(10, threadPool.getPoolSize());
+    assertNull(containerLauncher.foundErrors);
 
     // Different hosts, there should be an increase in core-thread-pool size to
     // 21(11hosts+10buffer)
@@ -179,8 +183,8 @@ public class TestContainerLauncher {
       containerId, "host11:1234", null,
       ContainerLauncher.EventType.CONTAINER_REMOTE_LAUNCH));
     waitForEvents(containerLauncher, 21);
-    Assert.assertEquals(11, threadPool.getPoolSize());
-    Assert.assertNull(containerLauncher.foundErrors);
+    assertEquals(11, threadPool.getPoolSize());
+    assertNull(containerLauncher.foundErrors);
 
     containerLauncher.stop();
 
@@ -194,7 +198,8 @@ public class TestContainerLauncher {
     assertThat(containerLauncher.initialPoolSize).isEqualTo(20);
   }
 
-  @Test(timeout = 5000)
+  @Test
+  @Timeout(value = 5)
   public void testPoolLimits() throws InterruptedException {
     ApplicationId appId = ApplicationId.newInstance(12345, 67);
     ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(
@@ -222,8 +227,8 @@ public class TestContainerLauncher {
         ContainerLauncher.EventType.CONTAINER_REMOTE_LAUNCH));
     }
     waitForEvents(containerLauncher, 10);
-    Assert.assertEquals(10, threadPool.getPoolSize());
-    Assert.assertNull(containerLauncher.foundErrors);
+    assertEquals(10, threadPool.getPoolSize());
+    assertNull(containerLauncher.foundErrors);
 
     // 4 more different hosts, but thread pool size should be capped at 12
     containerLauncher.expectedCorePoolSize = 12 ;
@@ -233,14 +238,14 @@ public class TestContainerLauncher {
         ContainerLauncher.EventType.CONTAINER_REMOTE_LAUNCH));
     }
     waitForEvents(containerLauncher, 12);
-    Assert.assertEquals(12, threadPool.getPoolSize());
-    Assert.assertNull(containerLauncher.foundErrors);
+    assertEquals(12, threadPool.getPoolSize());
+    assertNull(containerLauncher.foundErrors);
 
     // Make some threads ideal so that remaining events are also done.
     containerLauncher.finishEventHandling = true;
     waitForEvents(containerLauncher, 14);
-    Assert.assertEquals(12, threadPool.getPoolSize());
-    Assert.assertNull(containerLauncher.foundErrors);
+    assertEquals(12, threadPool.getPoolSize());
+    assertNull(containerLauncher.foundErrors);
 
     containerLauncher.stop();
   }
@@ -254,11 +259,11 @@ public class TestContainerLauncher {
           + ". It is now " + containerLauncher.numEventsProcessing.get());
       Thread.sleep(1000);
     }
-    Assert.assertEquals(expectedNumEvents,
-      containerLauncher.numEventsProcessing.get());
+    assertEquals(expectedNumEvents, containerLauncher.numEventsProcessing.get());
   }
 
-  @Test(timeout = 15000)
+  @Test
+  @Timeout(value = 15)
   public void testSlowNM() throws Exception {
 
     conf = new Configuration();
@@ -290,15 +295,14 @@ public class TestContainerLauncher {
     app.waitForState(job, JobState.RUNNING);
 
     Map<TaskId, Task> tasks = job.getTasks();
-    Assert.assertEquals("Num tasks is not correct", 1, tasks.size());
+      assertEquals(1, tasks.size(), "Num tasks is not correct");
 
     Task task = tasks.values().iterator().next();
     app.waitForState(task, TaskState.SCHEDULED);
 
     Map<TaskAttemptId, TaskAttempt> attempts = tasks.values().iterator()
         .next().getAttempts();
-      Assert.assertEquals("Num attempts is not correct", maxAttempts,
-          attempts.size());
+      assertEquals(maxAttempts, attempts.size(), "Num attempts is not correct");
 
     TaskAttempt attempt = attempts.values().iterator().next();
       app.waitForInternalState((TaskAttemptImpl) attempt,
@@ -309,11 +313,10 @@ public class TestContainerLauncher {
     String diagnostics = attempt.getDiagnostics().toString();
     LOG.info("attempt.getDiagnostics: " + diagnostics);
 
-      Assert.assertTrue(diagnostics.contains("Container launch failed for "
+      assertTrue(diagnostics.contains("Container launch failed for "
           + "container_0_0000_01_000000 : "));
-      Assert
-          .assertTrue(diagnostics
-              .contains("java.net.SocketTimeoutException: 3000 millis timeout while waiting for channel"));
+      assertTrue(diagnostics.contains(
+          "java.net.SocketTimeoutException: 3000 millis timeout while waiting for channel"));
 
     } finally {
       server.stop();
@@ -440,7 +443,7 @@ public class TestContainerLauncher {
           MRApp.newContainerTokenIdentifier(request.getContainerToken());
 
       // Validate that the container is what RM is giving.
-      Assert.assertEquals(MRApp.NM_HOST + ":" + MRApp.NM_PORT,
+      assertEquals(MRApp.NM_HOST + ":" + MRApp.NM_PORT,
         containerTokenIdentifier.getNmHostAddress());
 
       StartContainersResponse response = recordFactory

+ 12 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java

@@ -79,8 +79,9 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.mockito.ArgumentCaptor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -94,7 +95,7 @@ public class TestContainerLauncherImpl {
   private Map<String, ByteBuffer> serviceResponse =
       new HashMap<String, ByteBuffer>();
 
-  @Before
+  @BeforeEach
   public void setup() throws IOException {
     serviceResponse.clear();
     serviceResponse.put(ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID,
@@ -168,7 +169,8 @@ public class TestContainerLauncherImpl {
     return MRBuilderUtils.newTaskAttemptId(tID, id);
   }
   
-  @Test(timeout = 5000)
+  @Test
+  @Timeout(value = 5)
   public void testHandle() throws Exception {
     LOG.info("STARTING testHandle");
     AppContext mockContext = mock(AppContext.class);
@@ -226,7 +228,8 @@ public class TestContainerLauncherImpl {
     }
   }
   
-  @Test(timeout = 5000)
+  @Test
+  @Timeout(value = 5)
   public void testOutOfOrder() throws Exception {
     LOG.info("STARTING testOutOfOrder");
     AppContext mockContext = mock(AppContext.class);
@@ -300,7 +303,8 @@ public class TestContainerLauncherImpl {
     }
   }
 
-  @Test(timeout = 5000)
+  @Test
+  @Timeout(value = 5)
   public void testMyShutdown() throws Exception {
     LOG.info("in test Shutdown");
 
@@ -352,7 +356,8 @@ public class TestContainerLauncherImpl {
   }
   
   @SuppressWarnings({ "rawtypes", "unchecked" })
-  @Test(timeout = 5000)
+  @Test
+  @Timeout(value = 5)
   public void testContainerCleaned() throws Exception {
     LOG.info("STARTING testContainerCleaned");
     

+ 16 - 15
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java

@@ -18,6 +18,10 @@
 package org.apache.hadoop.mapreduce.v2.app.local;
 
 import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.ArgumentMatchers.isA;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.times;
@@ -69,8 +73,7 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.util.resource.Resources;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.mockito.ArgumentCaptor;
 
 public class TestLocalContainerAllocator {
@@ -90,7 +93,7 @@ public class TestLocalContainerAllocator {
     lca.start();
     try {
       lca.heartbeat();
-      Assert.fail("heartbeat was supposed to throw");
+      fail("heartbeat was supposed to throw");
     } catch (YarnException e) {
       // YarnException is expected
     } finally {
@@ -104,7 +107,7 @@ public class TestLocalContainerAllocator {
     lca.start();
     try {
       lca.heartbeat();
-      Assert.fail("heartbeat was supposed to throw");
+      fail("heartbeat was supposed to throw");
     } catch (YarnRuntimeException e) {
       // YarnRuntimeException is expected
     } finally {
@@ -172,14 +175,13 @@ public class TestLocalContainerAllocator {
       }
     }
 
-    Assert.assertEquals("too many AMRM tokens", 1, tokenCount);
-    Assert.assertArrayEquals("token identifier not updated",
-        newToken.getIdentifier(), ugiToken.getIdentifier());
-    Assert.assertArrayEquals("token password not updated",
-        newToken.getPassword(), ugiToken.getPassword());
-    Assert.assertEquals("AMRM token service not updated",
-        new Text(ClientRMProxy.getAMRMTokenService(conf)),
-        ugiToken.getService());
+    assertEquals(1, tokenCount, "too many AMRM tokens");
+    assertArrayEquals(newToken.getIdentifier(), ugiToken.getIdentifier(),
+        "token identifier not updated");
+    assertArrayEquals(newToken.getPassword(), ugiToken.getPassword(),
+        "token password not updated");
+    assertEquals(new Text(ClientRMProxy.getAMRMTokenService(conf)),
+        ugiToken.getService(), "AMRM token service not updated");
   }
 
   @Test
@@ -202,7 +204,7 @@ public class TestLocalContainerAllocator {
     verify(eventHandler, times(1)).handle(containerAssignedCaptor.capture());
     Container container = containerAssignedCaptor.getValue().getContainer();
     Resource containerResource = container.getResource();
-    Assert.assertNotNull(containerResource);
+    assertNotNull(containerResource);
     assertThat(containerResource.getMemorySize()).isEqualTo(0);
     assertThat(containerResource.getVirtualCores()).isEqualTo(0);
   }
@@ -282,8 +284,7 @@ public class TestLocalContainerAllocator {
     @Override
     public AllocateResponse allocate(AllocateRequest request)
         throws YarnException, IOException {
-      Assert.assertEquals("response ID mismatch",
-          responseId, request.getResponseId());
+      assertEquals(responseId, request.getResponseId(), "response ID mismatch");
       ++responseId;
       org.apache.hadoop.yarn.api.records.Token yarnToken = null;
       if (amToken != null) {

+ 5 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/metrics/TestMRAppMetrics.java

@@ -25,14 +25,15 @@ import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import static org.apache.hadoop.test.MetricsAsserts.*;
 
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
-import org.junit.After;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
 
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 public class TestMRAppMetrics {
 
-  @After
+  @AfterEach
   public void tearDown() {
     DefaultMetricsSystem.shutdown();
   }

+ 6 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMCommunicator.java

@@ -23,7 +23,8 @@ import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMCommunicator.AllocatorRunnable;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.util.Clock;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.mockito.stubbing.Answer;
 
 import static org.mockito.Mockito.doThrow;
@@ -45,7 +46,8 @@ public class TestRMCommunicator {
     }
   }
 
-  @Test(timeout = 6000)
+  @Test
+  @Timeout(value = 6)
   public void testRMContainerAllocatorExceptionIsHandled() throws Exception {
     ClientService mockClientService = mock(ClientService.class);
     AppContext mockContext = mock(AppContext.class);
@@ -66,7 +68,8 @@ public class TestRMCommunicator {
     testRunnable.run();
   }
 
-  @Test(timeout = 2000)
+  @Test
+  @Timeout(value = 2)
   public void testRMContainerAllocatorYarnRuntimeExceptionIsHandled()
       throws Exception {
     ClientService mockClientService = mock(ClientService.class);

La diferencia del archivo ha sido suprimido porque es demasiado grande
+ 185 - 176
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java


+ 10 - 11
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestResourceCalculatorUtils.java

@@ -19,12 +19,12 @@
 package org.apache.hadoop.mapreduce.v2.app.rm;
 
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.util.EnumSet;
 
 import static org.apache.hadoop.yarn.proto.YarnServiceProtos.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 public class TestResourceCalculatorUtils {
   @Test
@@ -59,17 +59,16 @@ public class TestResourceCalculatorUtils {
       Resource nonZeroResource, int expectedNumberOfContainersForMemoryOnly,
       int expectedNumberOfContainersOverall) {
 
-    Assert.assertEquals("Incorrect number of available containers for Memory",
-        expectedNumberOfContainersForMemoryOnly,
+    assertEquals(expectedNumberOfContainersForMemoryOnly,
         ResourceCalculatorUtils.computeAvailableContainers(
-            clusterAvailableResources, nonZeroResource,
-            EnumSet.of(SchedulerResourceTypes.MEMORY)));
+        clusterAvailableResources, nonZeroResource,
+        EnumSet.of(SchedulerResourceTypes.MEMORY)),
+        "Incorrect number of available containers for Memory");
 
-    Assert.assertEquals("Incorrect number of available containers overall",
-        expectedNumberOfContainersOverall,
+    assertEquals(expectedNumberOfContainersOverall,
         ResourceCalculatorUtils.computeAvailableContainers(
-            clusterAvailableResources, nonZeroResource,
-            EnumSet.of(SchedulerResourceTypes.CPU,
-                SchedulerResourceTypes.MEMORY)));
+        clusterAvailableResources, nonZeroResource,
+        EnumSet.of(SchedulerResourceTypes.CPU, SchedulerResourceTypes.MEMORY)),
+        "Incorrect number of available containers overall");
   }
 }

+ 25 - 24
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/speculate/TestDataStatistics.java

@@ -18,8 +18,9 @@
 
 package org.apache.hadoop.mapreduce.v2.app.speculate;
 
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 public class TestDataStatistics {
 
@@ -28,46 +29,46 @@ public class TestDataStatistics {
   @Test
   public void testEmptyDataStatistics() throws Exception {
     DataStatistics statistics = new DataStatistics();
-    Assert.assertEquals(0, statistics.count(), TOL);
-    Assert.assertEquals(0, statistics.mean(), TOL);
-    Assert.assertEquals(0, statistics.var(), TOL);
-    Assert.assertEquals(0, statistics.std(), TOL);
-    Assert.assertEquals(0, statistics.outlier(1.0f), TOL);
+    assertEquals(0, statistics.count(), TOL);
+    assertEquals(0, statistics.mean(), TOL);
+    assertEquals(0, statistics.var(), TOL);
+    assertEquals(0, statistics.std(), TOL);
+    assertEquals(0, statistics.outlier(1.0f), TOL);
   }
   
   @Test
   public void testSingleEntryDataStatistics() throws Exception {
     DataStatistics statistics = new DataStatistics(17.29);
-    Assert.assertEquals(1, statistics.count(), TOL);
-    Assert.assertEquals(17.29, statistics.mean(), TOL);
-    Assert.assertEquals(0, statistics.var(), TOL);
-    Assert.assertEquals(0, statistics.std(), TOL);
-    Assert.assertEquals(17.29, statistics.outlier(1.0f), TOL);
+    assertEquals(1, statistics.count(), TOL);
+    assertEquals(17.29, statistics.mean(), TOL);
+    assertEquals(0, statistics.var(), TOL);
+    assertEquals(0, statistics.std(), TOL);
+    assertEquals(17.29, statistics.outlier(1.0f), TOL);
   }
   
   @Test
-  public void testMutiEntryDataStatistics() throws Exception {
+  public void testMultiEntryDataStatistics() throws Exception {
     DataStatistics statistics = new DataStatistics();
     statistics.add(17);
     statistics.add(29);
-    Assert.assertEquals(2, statistics.count(), TOL);
-    Assert.assertEquals(23.0, statistics.mean(), TOL);
-    Assert.assertEquals(36.0, statistics.var(), TOL);
-    Assert.assertEquals(6.0, statistics.std(), TOL);
-    Assert.assertEquals(29.0, statistics.outlier(1.0f), TOL);
+    assertEquals(2, statistics.count(), TOL);
+    assertEquals(23.0, statistics.mean(), TOL);
+    assertEquals(36.0, statistics.var(), TOL);
+    assertEquals(6.0, statistics.std(), TOL);
+    assertEquals(29.0, statistics.outlier(1.0f), TOL);
  }
   
   @Test
   public void testUpdateStatistics() throws Exception {
     DataStatistics statistics = new DataStatistics(17);
     statistics.add(29);
-    Assert.assertEquals(2, statistics.count(), TOL);
-    Assert.assertEquals(23.0, statistics.mean(), TOL);
-    Assert.assertEquals(36.0, statistics.var(), TOL);
+    assertEquals(2, statistics.count(), TOL);
+    assertEquals(23.0, statistics.mean(), TOL);
+    assertEquals(36.0, statistics.var(), TOL);
 
     statistics.updateStatistics(17, 29);
-    Assert.assertEquals(2, statistics.count(), TOL);
-    Assert.assertEquals(29.0, statistics.mean(), TOL);
-    Assert.assertEquals(0.0, statistics.var(), TOL);
+    assertEquals(2, statistics.count(), TOL);
+    assertEquals(29.0, statistics.mean(), TOL);
+    assertEquals(0.0, statistics.var(), TOL);
   }
 }

+ 6 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/speculate/forecast/TestSimpleExponentialForecast.java

@@ -23,8 +23,9 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.yarn.util.ControlledClock;
 
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 /**
  * Testing the statistical model of simple exponential estimator.
@@ -101,21 +102,18 @@ public class TestSimpleExponentialForecast {
   @Test
   public void testSimpleExponentialForecastLinearInc() throws Exception {
     int res = incTestSimpleExponentialForecast();
-    Assert.assertEquals("We got the wrong estimate from simple exponential.",
-        res, 0);
+    assertEquals(res, 0, "We got the wrong estimate from simple exponential.");
   }
 
   @Test
   public void testSimpleExponentialForecastLinearDec() throws Exception {
     int res = decTestSimpleExponentialForecast();
-    Assert.assertEquals("We got the wrong estimate from simple exponential.",
-        res, 0);
+    assertEquals(res, 0, "We got the wrong estimate from simple exponential.");
   }
 
   @Test
   public void testSimpleExponentialForecastZeros() throws Exception {
     int res = zeroTestSimpleExponentialForecast();
-    Assert.assertEquals("We got the wrong estimate from simple exponential.",
-        res, 0);
+    assertEquals(res, 0, "We got the wrong estimate from simple exponential.");
   }
 }

+ 28 - 28
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebApp.java

@@ -19,7 +19,6 @@
 package org.apache.hadoop.mapreduce.v2.app.webapp;
 
 import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.APP_ID;
-import static org.junit.Assert.assertEquals;
 
 import java.io.ByteArrayOutputStream;
 import java.io.File;
@@ -45,7 +44,6 @@ import org.apache.hadoop.yarn.webapp.WebApps;
 import org.glassfish.jersey.internal.inject.AbstractBinder;
 import org.glassfish.jersey.jettison.JettisonFeature;
 import org.glassfish.jersey.server.ResourceConfig;
-import org.junit.Assert;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.http.HttpConfig.Policy;
@@ -70,12 +68,14 @@ import org.apache.hadoop.yarn.server.webproxy.amfilter.AmFilterInitializer;
 import org.apache.hadoop.yarn.webapp.test.WebAppTests;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.apache.http.HttpStatus;
-import org.junit.After;
-import org.junit.Rule;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
 
 import com.google.inject.Injector;
-import org.junit.contrib.java.lang.system.EnvironmentVariables;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 public class TestAMWebApp {
 
@@ -84,7 +84,7 @@ public class TestAMWebApp {
           System.getProperty("java.io.tmpdir")),
       TestAMWebApp.class.getName());
 
-  @After
+  @AfterEach
   public void tearDown() {
     TEST_DIR.delete();
   }
@@ -217,7 +217,7 @@ public class TestAMWebApp {
     InputStream in = conn.getInputStream();
     ByteArrayOutputStream out = new ByteArrayOutputStream();
     IOUtils.copyBytes(in, out, 1024);
-    Assert.assertTrue(out.toString().contains("MapReduce Application"));
+    assertTrue(out.toString().contains("MapReduce Application"));
 
     // https:// is not accessible.
     URL httpsUrl = new URL("https://" + hostPort + "/mapreduce/");
@@ -225,7 +225,7 @@ public class TestAMWebApp {
       HttpURLConnection httpsConn =
           (HttpURLConnection) httpsUrl.openConnection();
       httpsConn.getInputStream();
-      Assert.fail("https:// is not accessible, expected to fail");
+      fail("https:// is not accessible, expected to fail");
     } catch (SSLException e) {
       // expected
     }
@@ -234,10 +234,6 @@ public class TestAMWebApp {
     app.verifyCompleted();
   }
 
-  @Rule
-  public final EnvironmentVariables environmentVariables
-      = new EnvironmentVariables();
-
   @Test
   public void testMRWebAppSSLEnabled() throws Exception {
     MRApp app = new MRApp(2, 2, true, this.getClass().getName(), true) {
@@ -256,9 +252,8 @@ public class TestAMWebApp {
     keystoreFile.getParentFile().mkdirs();
     KeyStoreTestUtil.createKeyStore(keystoreFile.getAbsolutePath(), "password",
         "server", keyPair.getPrivate(), cert);
-    environmentVariables.set("KEYSTORE_FILE_LOCATION",
-        keystoreFile.getAbsolutePath());
-    environmentVariables.set("KEYSTORE_PASSWORD", "password");
+    System.setProperty("KEYSTORE_FILE_LOCATION", keystoreFile.getAbsolutePath());
+    System.setProperty("KEYSTORE_PASSWORD", "password");
 
     Job job = app.submit(conf);
 
@@ -274,7 +269,7 @@ public class TestAMWebApp {
     InputStream in = httpsConn.getInputStream();
     ByteArrayOutputStream out = new ByteArrayOutputStream();
     IOUtils.copyBytes(in, out, 1024);
-    Assert.assertTrue(out.toString().contains("MapReduce Application"));
+    assertTrue(out.toString().contains("MapReduce Application"));
 
     // http:// is not accessible.
     URL httpUrl = new URL("http://" + hostPort + "/mapreduce/");
@@ -282,7 +277,7 @@ public class TestAMWebApp {
       HttpURLConnection httpConn =
           (HttpURLConnection) httpUrl.openConnection();
       httpConn.getResponseCode();
-      Assert.fail("http:// is not accessible, expected to fail");
+      fail("http:// is not accessible, expected to fail");
     } catch (SocketException e) {
       // expected
     }
@@ -291,6 +286,8 @@ public class TestAMWebApp {
     app.verifyCompleted();
 
     keystoreFile.delete();
+    System.clearProperty("KEYSTORE_FILE_LOCATION");
+    System.clearProperty("KEYSTORE_PASSWORD");
   }
 
   @Test
@@ -312,9 +309,8 @@ public class TestAMWebApp {
     keystoreFile.getParentFile().mkdirs();
     KeyStoreTestUtil.createKeyStore(keystoreFile.getAbsolutePath(), "password",
         "server", keyPair.getPrivate(), cert);
-    environmentVariables.set("KEYSTORE_FILE_LOCATION",
-        keystoreFile.getAbsolutePath());
-    environmentVariables.set("KEYSTORE_PASSWORD", "password");
+    System.setProperty("KEYSTORE_FILE_LOCATION", keystoreFile.getAbsolutePath());
+    System.setProperty("KEYSTORE_PASSWORD", "password");
 
     KeyPair clientKeyPair = KeyStoreTestUtil.generateKeyPair("RSA");
     X509Certificate clientCert = KeyStoreTestUtil.generateCertificate(
@@ -323,9 +319,8 @@ public class TestAMWebApp {
     truststoreFile.getParentFile().mkdirs();
     KeyStoreTestUtil.createTrustStore(truststoreFile.getAbsolutePath(),
         "password", "client", clientCert);
-    environmentVariables.set("TRUSTSTORE_FILE_LOCATION",
-        truststoreFile.getAbsolutePath());
-    environmentVariables.set("TRUSTSTORE_PASSWORD", "password");
+    System.setProperty("TRUSTSTORE_FILE_LOCATION", truststoreFile.getAbsolutePath());
+    System.setProperty("TRUSTSTORE_PASSWORD", "password");
 
     Job job = app.submit(conf);
 
@@ -341,7 +336,7 @@ public class TestAMWebApp {
     InputStream in = httpsConn.getInputStream();
     ByteArrayOutputStream out = new ByteArrayOutputStream();
     IOUtils.copyBytes(in, out, 1024);
-    Assert.assertTrue(out.toString().contains("MapReduce Application"));
+    assertTrue(out.toString().contains("MapReduce Application"));
 
     // Try with wrong client cert
     KeyPair otherClientKeyPair = KeyStoreTestUtil.generateKeyPair("RSA");
@@ -353,7 +348,7 @@ public class TestAMWebApp {
       HttpURLConnection httpConn =
           (HttpURLConnection) httpsUrl.openConnection();
       httpConn.getResponseCode();
-      Assert.fail("Wrong client certificate, expected to fail");
+      fail("Wrong client certificate, expected to fail");
     } catch (SSLException e) {
       // expected
     }
@@ -363,6 +358,11 @@ public class TestAMWebApp {
 
     keystoreFile.delete();
     truststoreFile.delete();
+
+    System.clearProperty("KEYSTORE_FILE_LOCATION");
+    System.clearProperty("KEYSTORE_PASSWORD");
+    System.clearProperty("TRUSTSTORE_FILE_LOCATION");
+    System.clearProperty("TRUSTSTORE_PASSWORD");
   }
 
   static String webProxyBase = null;
@@ -408,9 +408,9 @@ public class TestAMWebApp {
       String expectedURL = scheme + conf.get(YarnConfiguration.PROXY_ADDRESS)
           + ProxyUriUtils.getPath(app.getAppID(), "/mapreduce", true);
 
-      Assert.assertEquals(expectedURL,
+      assertEquals(expectedURL,
         conn.getHeaderField(HttpHeaders.LOCATION));
-      Assert.assertEquals(HttpStatus.SC_MOVED_TEMPORARILY,
+      assertEquals(HttpStatus.SC_MOVED_TEMPORARILY,
         conn.getResponseCode());
       app.waitForState(job, JobState.SUCCEEDED);
       app.verifyCompleted();

+ 18 - 18
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServices.java

@@ -19,9 +19,9 @@
 package org.apache.hadoop.mapreduce.v2.app.webapp;
 
 import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.Mockito.mock;
 
 import java.io.StringReader;
@@ -52,7 +52,7 @@ import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.w3c.dom.Document;
 import org.w3c.dom.Element;
 import org.w3c.dom.NodeList;
@@ -109,7 +109,7 @@ public class TestAMWebServices extends JerseyTestBase {
     assertEquals(MediaType.APPLICATION_JSON + ";" + JettyUtils.UTF_8,
         response.getMediaType().toString());
     JSONObject json = response.readEntity(JSONObject.class);
-    assertEquals("incorrect number of elements", 1, json.length());
+    assertEquals(1, json.length(), "incorrect number of elements");
     verifyAMInfo(json.getJSONObject("info"), appContext);
   }
 
@@ -121,7 +121,7 @@ public class TestAMWebServices extends JerseyTestBase {
     assertEquals(MediaType.APPLICATION_JSON + ";" + JettyUtils.UTF_8,
         response.getMediaType().toString());
     JSONObject json = response.readEntity(JSONObject.class);
-    assertEquals("incorrect number of elements", 1, json.length());
+    assertEquals(1, json.length(), "incorrect number of elements");
     verifyAMInfo(json.getJSONObject("info"), appContext);
   }
 
@@ -133,7 +133,7 @@ public class TestAMWebServices extends JerseyTestBase {
     assertEquals(MediaType.APPLICATION_JSON + ";" + JettyUtils.UTF_8,
         response.getMediaType().toString());
     JSONObject json = response.readEntity(JSONObject.class);
-    assertEquals("incorrect number of elements", 1, json.length());
+    assertEquals(1, json.length(), "incorrect number of elements");
     verifyAMInfo(json.getJSONObject("info"), appContext);
   }
 
@@ -157,7 +157,7 @@ public class TestAMWebServices extends JerseyTestBase {
     assertEquals(MediaType.APPLICATION_JSON + ";" + JettyUtils.UTF_8,
         response.getMediaType().toString());
     JSONObject json = response.readEntity(JSONObject.class);
-    assertEquals("incorrect number of elements", 1, json.length());
+    assertEquals(1, json.length(), "incorrect number of elements");
     verifyAMInfo(json.getJSONObject("info"), appContext);
   }
 
@@ -170,7 +170,7 @@ public class TestAMWebServices extends JerseyTestBase {
     assertEquals(MediaType.APPLICATION_JSON + ";" + JettyUtils.UTF_8,
         response.getMediaType().toString());
     JSONObject json = response.readEntity(JSONObject.class);
-    assertEquals("incorrect number of elements", 1, json.length());
+    assertEquals(1, json.length(), "incorrect number of elements");
     verifyAMInfo(json.getJSONObject("info"), appContext);
   }
 
@@ -182,7 +182,7 @@ public class TestAMWebServices extends JerseyTestBase {
     assertEquals(MediaType.APPLICATION_JSON + ";" + JettyUtils.UTF_8,
         response.getMediaType().toString());
     JSONObject json = response.readEntity(JSONObject.class);
-    assertEquals("incorrect number of elements", 1, json.length());
+    assertEquals(1, json.length(), "incorrect number of elements");
     verifyAMInfo(json.getJSONObject("info"), appContext);
   }
 
@@ -255,7 +255,7 @@ public class TestAMWebServices extends JerseyTestBase {
     assertEquals(MediaType.APPLICATION_JSON + ";" + JettyUtils.UTF_8,
         response.getMediaType().toString());
     JSONObject json = response.readEntity(JSONObject.class);
-    assertEquals("incorrect number of elements", 1, json.length());
+    assertEquals(1, json.length(), "incorrect number of elements");
     verifyBlacklistedNodesInfo(json, appContext);
   }
   
@@ -273,7 +273,7 @@ public class TestAMWebServices extends JerseyTestBase {
 
   public void verifyAMInfo(JSONObject info, AppContext ctx)
       throws JSONException {
-    assertEquals("incorrect number of elements", 5, info.length());
+    assertEquals(5, info.length(), "incorrect number of elements");
 
     verifyAMInfoGeneric(ctx, info.getString("appId"), info.getString("user"),
         info.getString("name"), info.getLong("startedOn"),
@@ -288,7 +288,7 @@ public class TestAMWebServices extends JerseyTestBase {
     is.setCharacterStream(new StringReader(xml));
     Document dom = db.parse(is);
     NodeList nodes = dom.getElementsByTagName("info");
-    assertEquals("incorrect number of elements", 1, nodes.getLength());
+    assertEquals(1, nodes.getLength(), "incorrect number of elements");
 
     for (int i = 0; i < nodes.getLength(); i++) {
       Element element = (Element) nodes.item(i);
@@ -311,8 +311,8 @@ public class TestAMWebServices extends JerseyTestBase {
     WebServicesTestUtils.checkStringMatch("name", ctx.getApplicationName(),
         name);
 
-    assertEquals("startedOn incorrect", ctx.getStartTime(), startedOn);
-    assertTrue("elapsedTime not greater then 0", (elapsedTime > 0));
+    assertEquals(ctx.getStartTime(), startedOn, "startedOn incorrect");
+    assertTrue((elapsedTime > 0), "elapsedTime not greater then 0");
 
   }
   
@@ -334,11 +334,11 @@ public class TestAMWebServices extends JerseyTestBase {
     is.setCharacterStream(new StringReader(xml));
     Document dom = db.parse(is);
     NodeList infonodes = dom.getElementsByTagName("blacklistednodesinfo");
-    assertEquals("incorrect number of elements", 1, infonodes.getLength());
+    assertEquals(1, infonodes.getLength(), "incorrect number of elements");
     NodeList nodes = dom.getElementsByTagName("blacklistedNodes");
     Set<String> blacklistedNodes = ctx.getBlacklistedNodes();
-    assertEquals("incorrect number of elements", blacklistedNodes.size(),
-        nodes.getLength());
+    assertEquals(blacklistedNodes.size(),
+        nodes.getLength(), "incorrect number of elements");
     for (int i = 0; i < nodes.getLength(); i++) {
       Element element = (Element) nodes.item(i);
       assertTrue(

+ 4 - 9
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempt.java

@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.mapreduce.v2.app.webapp;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -57,7 +57,7 @@ import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONObject;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.w3c.dom.Document;
 import org.w3c.dom.Element;
 import org.w3c.dom.NodeList;
@@ -128,11 +128,6 @@ public class TestAMWebServicesAttempt extends JerseyTestBase {
     }
   }
 
-  @Override
-  public void setUp() throws Exception {
-    super.setUp();
-  }
-
   public TestAMWebServicesAttempt() {
   }
 
@@ -160,7 +155,7 @@ public class TestAMWebServicesAttempt extends JerseyTestBase {
               + JettyUtils.UTF_8, response.getMediaType().toString());
           JSONObject json = response.readEntity(JSONObject.class);
           JSONObject jobState = json.getJSONObject("jobTaskAttemptState");
-          assertEquals("incorrect number of elements", 1, json.length());
+          assertEquals(1, json.length(), "incorrect number of elements");
           assertEquals(att.getState().toString(), jobState.get("state"));
         }
       }
@@ -229,7 +224,7 @@ public class TestAMWebServicesAttempt extends JerseyTestBase {
               + JettyUtils.UTF_8, response.getMediaType().toString());
           JSONObject json = response.readEntity(JSONObject.class);
           JSONObject jobState = json.getJSONObject("jobTaskAttemptState");
-          assertEquals("incorrect number of elements", 1, json.length());
+          assertEquals(1, json.length(), "incorrect number of elements");
           assertEquals(TaskAttemptState.KILLED.toString(), jobState.get("state"));
         }
       }

+ 42 - 46
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempts.java

@@ -19,11 +19,11 @@
 package org.apache.hadoop.mapreduce.v2.app.webapp;
 
 import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.Mockito.mock;
 
 import java.io.StringReader;
@@ -59,7 +59,7 @@ import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.w3c.dom.Document;
 import org.w3c.dom.Element;
 import org.w3c.dom.NodeList;
@@ -190,7 +190,7 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
         is.setCharacterStream(new StringReader(xml));
         Document dom = db.parse(is);
         NodeList attempts = dom.getElementsByTagName("taskAttempts");
-        assertEquals("incorrect number of elements", 1, attempts.getLength());
+        assertEquals(1, attempts.getLength(), "incorrect number of elements");
 
         NodeList nodes = dom.getElementsByTagName("taskAttempt");
         verifyAMTaskAttemptsXML(nodes, task);
@@ -220,7 +220,7 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
           assertEquals(MediaType.APPLICATION_JSON_TYPE + ";"
               + JettyUtils.UTF_8, response.getMediaType().toString());
           JSONObject json = response.readEntity(JSONObject.class);
-          assertEquals("incorrect number of elements", 1, json.length());
+          assertEquals(1, json.length(), "incorrect number of elements");
           JSONObject info = json.getJSONObject("taskAttempt");
           verifyAMTaskAttempt(info, att, task.getType());
         }
@@ -250,7 +250,7 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
           assertEquals(MediaType.APPLICATION_JSON_TYPE + ";"
               + JettyUtils.UTF_8, response.getMediaType().toString());
           JSONObject json = response.readEntity(JSONObject.class);
-          assertEquals("incorrect number of elements", 1, json.length());
+          assertEquals(1, json.length(), "incorrect number of elements");
           JSONObject info = json.getJSONObject("taskAttempt");
           verifyAMTaskAttempt(info, att, task.getType());
         }
@@ -280,7 +280,7 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
           assertEquals(MediaType.APPLICATION_JSON_TYPE + ";"
               + JettyUtils.UTF_8, response.getMediaType().toString());
           JSONObject json = response.readEntity(JSONObject.class);
-          assertEquals("incorrect number of elements", 1, json.length());
+          assertEquals(1, json.length(), "incorrect number of elements");
           JSONObject info = json.getJSONObject("taskAttempt");
           verifyAMTaskAttempt(info, att, task.getType());
         }
@@ -385,7 +385,7 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
           String entity = response.readEntity(String.class);
           JSONObject msg = new JSONObject(entity);
           JSONObject exception = msg.getJSONObject("RemoteException");
-          assertEquals("incorrect number of elements", 3, exception.length());
+          assertEquals(3, exception.length(), "incorrect number of elements");
           String message = exception.getString("message");
           String type = exception.getString("exception");
           String classname = exception.getString("javaClassName");
@@ -426,9 +426,9 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
   public void verifyAMTaskAttempt(JSONObject info, TaskAttempt att,
       TaskType ttype) throws JSONException {
     if (ttype == TaskType.REDUCE) {
-      assertEquals("incorrect number of elements", 17, info.length());
+      assertEquals(17, info.length(), "incorrect number of elements");
     } else {
-      assertEquals("incorrect number of elements", 12, info.length());
+      assertEquals(12, info.length(), "incorrect number of elements");
     }
 
     verifyTaskAttemptGeneric(att, ttype, info.getString("id"),
@@ -447,9 +447,9 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
 
   public void verifyAMTaskAttempts(JSONObject json, Task task)
       throws JSONException {
-    assertEquals("incorrect number of elements", 1, json.length());
+    assertEquals(1, json.length(), "incorrect number of elements");
     JSONObject attempts = json.getJSONObject("taskAttempts");
-    assertEquals("incorrect number of elements", 1, json.length());
+    assertEquals(1, json.length(), "incorrect number of elements");
     JSONObject taskAttempt = attempts.getJSONObject("taskAttempt");
     JSONArray arr = new JSONArray();
     arr.put(taskAttempt);
@@ -465,13 +465,13 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
           verifyAMTaskAttempt(info, att, task.getType());
         }
       }
-      assertTrue("task attempt with id: " + attid
-          + " not in web service output", found);
+      assertTrue(found, "task attempt with id: " + attid
+          + " not in web service output");
     }
   }
 
   public void verifyAMTaskAttemptsXML(NodeList nodes, Task task) {
-    assertEquals("incorrect number of elements", 1, nodes.getLength());
+    assertEquals(1, nodes.getLength(), "incorrect number of elements");
 
     for (TaskAttempt att : task.getAttempts().values()) {
       TaskAttemptId id = att.getID();
@@ -479,15 +479,15 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
       boolean found = false;
       for (int i = 0; i < nodes.getLength(); i++) {
         Element element = (Element) nodes.item(i);
-        assertFalse("task attempt should not contain any attributes, it can lead to incorrect JSON marshaling",
-            element.hasAttributes());
+        assertFalse(element.hasAttributes(), "task attempt should not contain any attributes, " +
+            "it can lead to incorrect JSON marshaling");
 
         if (attid.matches(WebServicesTestUtils.getXmlString(element, "id"))) {
           found = true;
           verifyAMTaskAttemptXML(element, att, task.getType());
         }
       }
-      assertTrue("task with id: " + attid + " not in web service output", found);
+      assertTrue(found, "task with id: " + attid + " not in web service output");
     }
   }
 
@@ -522,26 +522,24 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
         ta.getAssignedContainerID().toString(),
         assignedContainerId);
 
-    assertEquals("startTime wrong", ta.getLaunchTime(), startTime);
-    assertEquals("finishTime wrong", ta.getFinishTime(), finishTime);
-    assertEquals("elapsedTime wrong", finishTime - startTime, elapsedTime);
-    assertEquals("progress wrong", ta.getProgress() * 100, progress, 1e-3f);
+    assertEquals(ta.getLaunchTime(), startTime, "startTime wrong");
+    assertEquals(ta.getFinishTime(), finishTime, "finishTime wrong");
+    assertEquals(finishTime - startTime, elapsedTime, "elapsedTime wrong");
+    assertEquals(ta.getProgress() * 100, progress, 1e-3f, "progress wrong");
   }
 
   public void verifyReduceTaskAttemptGeneric(TaskAttempt ta,
       long shuffleFinishTime, long mergeFinishTime, long elapsedShuffleTime,
       long elapsedMergeTime, long elapsedReduceTime) {
 
-    assertEquals("shuffleFinishTime wrong", ta.getShuffleFinishTime(),
-        shuffleFinishTime);
-    assertEquals("mergeFinishTime wrong", ta.getSortFinishTime(),
-        mergeFinishTime);
-    assertEquals("elapsedShuffleTime wrong",
-        ta.getShuffleFinishTime() - ta.getLaunchTime(), elapsedShuffleTime);
-    assertEquals("elapsedMergeTime wrong",
-        ta.getSortFinishTime() - ta.getShuffleFinishTime(), elapsedMergeTime);
-    assertEquals("elapsedReduceTime wrong",
-        ta.getFinishTime() - ta.getSortFinishTime(), elapsedReduceTime);
+    assertEquals(ta.getShuffleFinishTime(), shuffleFinishTime, "shuffleFinishTime wrong");
+    assertEquals(ta.getSortFinishTime(), mergeFinishTime, "mergeFinishTime wrong");
+    assertEquals(ta.getShuffleFinishTime() - ta.getLaunchTime(), elapsedShuffleTime,
+        "elapsedShuffleTime wrong");
+    assertEquals(ta.getSortFinishTime() - ta.getShuffleFinishTime(), elapsedMergeTime,
+        "elapsedMergeTime wrong");
+    assertEquals(ta.getFinishTime() - ta.getSortFinishTime(), elapsedReduceTime,
+        "elapsedReduceTime wrong");
   }
 
   @Test
@@ -566,7 +564,7 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
           assertEquals(MediaType.APPLICATION_JSON_TYPE + ";"
               + JettyUtils.UTF_8, response.getMediaType().toString());
           JSONObject json = response.readEntity(JSONObject.class);
-          assertEquals("incorrect number of elements", 1, json.length());
+          assertEquals(1, json.length(), "incorrect number of elements");
           JSONObject info = json.getJSONObject("jobTaskAttemptCounters");
           verifyAMJobTaskAttemptCounters(info, att);
         }
@@ -611,7 +609,7 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
   public void verifyAMJobTaskAttemptCounters(JSONObject info, TaskAttempt att)
       throws JSONException {
 
-    assertEquals("incorrect number of elements", 2, info.length());
+    assertEquals(2, info.length(), "incorrect number of elements");
 
     WebServicesTestUtils.checkStringMatch("id", MRApps.toString(att.getID()),
         info.getString("id"));
@@ -622,15 +620,14 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
     for (int i = 0; i < counterGroups.length(); i++) {
       JSONObject counterGroup = counterGroups.getJSONObject(i);
       String name = counterGroup.getString("counterGroupName");
-      assertTrue("name not set", (name != null && !name.isEmpty()));
+      assertTrue((name != null && !name.isEmpty()), "name not set");
       JSONArray counters = counterGroup.getJSONArray("counter");
       for (int j = 0; j < counters.length(); j++) {
         JSONObject counter = counters.getJSONObject(j);
         String counterName = counter.getString("name");
-        assertTrue("name not set",
-            (counterName != null && !counterName.isEmpty()));
+        assertTrue((counterName != null && !counterName.isEmpty()), "name not set");
         long value = counter.getLong("value");
-        assertTrue("value  >= 0", value >= 0);
+        assertTrue(value >= 0, "value  >= 0");
       }
     }
   }
@@ -648,20 +645,19 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
 
       for (int j = 0; j < groups.getLength(); j++) {
         Element counters = (Element) groups.item(j);
-        assertNotNull("should have counters in the web service info", counters);
+        assertNotNull(counters, "should have counters in the web service info");
         String name = WebServicesTestUtils.getXmlString(counters,
             "counterGroupName");
-        assertTrue("name not set", (name != null && !name.isEmpty()));
+        assertTrue((name != null && !name.isEmpty()), "name not set");
         NodeList counterArr = counters.getElementsByTagName("counter");
         for (int z = 0; z < counterArr.getLength(); z++) {
           Element counter = (Element) counterArr.item(z);
           String counterName = WebServicesTestUtils.getXmlString(counter,
               "name");
-          assertTrue("counter name not set",
-              (counterName != null && !counterName.isEmpty()));
+          assertTrue((counterName != null && !counterName.isEmpty()), "counter name not set");
 
           long value = WebServicesTestUtils.getXmlLong(counter, "value");
-          assertTrue("value not >= 0", value >= 0);
+          assertTrue(value >= 0, "value not >= 0");
 
         }
       }

+ 21 - 20
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobConf.java

@@ -18,10 +18,10 @@
 
 package org.apache.hadoop.mapreduce.v2.app.webapp;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.Mockito.mock;
 
 import java.io.File;
@@ -57,8 +57,9 @@ import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
-import org.junit.AfterClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 import org.w3c.dom.Document;
 import org.w3c.dom.Element;
 import org.w3c.dom.NodeList;
@@ -130,13 +131,13 @@ public class TestAMWebServicesJobConf extends JerseyTestBase {
     }
   }
 
-  @Override
-  public void setUp() throws Exception {
-    super.setUp();
+
+  @BeforeAll
+  public static void setTestConfDir() throws Exception {
     testConfDir.mkdir();
   }
 
-  @AfterClass
+  @AfterAll
   static public void stop() {
     FileUtil.fullyDelete(testConfDir);
   }
@@ -154,7 +155,7 @@ public class TestAMWebServicesJobConf extends JerseyTestBase {
       assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8,
           response.getMediaType().toString());
       JSONObject json = response.readEntity(JSONObject.class);
-      assertEquals("incorrect number of elements", 1, json.length());
+      assertEquals(1, json.length(), "incorrect number of elements");
       JSONObject info = json.getJSONObject("conf");
       verifyAMJobConf(info, jobsMap.get(id));
     }
@@ -172,7 +173,7 @@ public class TestAMWebServicesJobConf extends JerseyTestBase {
       assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8,
           response.getMediaType().toString());
       JSONObject json = response.readEntity(JSONObject.class);
-      assertEquals("incorrect number of elements", 1, json.length());
+      assertEquals(1, json.length(), "incorrect number of elements");
       JSONObject info = json.getJSONObject("conf");
       verifyAMJobConf(info, jobsMap.get(id));
     }
@@ -190,7 +191,7 @@ public class TestAMWebServicesJobConf extends JerseyTestBase {
       assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8,
           response.getMediaType().toString());
       JSONObject json = response.readEntity(JSONObject.class);
-      assertEquals("incorrect number of elements", 1, json.length());
+      assertEquals(1, json.length(), "incorrect number of elements");
       JSONObject info = json.getJSONObject("conf");
       verifyAMJobConf(info, jobsMap.get(id));
     }
@@ -221,7 +222,7 @@ public class TestAMWebServicesJobConf extends JerseyTestBase {
 
   public void verifyAMJobConf(JSONObject info, Job job) throws JSONException {
 
-    assertEquals("incorrect number of elements", 2, info.length());
+    assertEquals(2, info.length(), "incorrect number of elements");
 
     WebServicesTestUtils.checkStringMatch("path", job.getConfFile().toString(),
         info.getString("path"));
@@ -232,14 +233,14 @@ public class TestAMWebServicesJobConf extends JerseyTestBase {
       JSONObject prop = properties.getJSONObject(i);
       String name = prop.getString("name");
       String value = prop.getString("value");
-      assertTrue("name not set", (name != null && !name.isEmpty()));
-      assertTrue("value not set", (value != null && !value.isEmpty()));
+      assertTrue((name != null && !name.isEmpty()), "name not set");
+      assertTrue((value != null && !value.isEmpty()), "value not set");
     }
   }
 
   public void verifyAMJobConfXML(NodeList nodes, Job job) {
 
-    assertEquals("incorrect number of elements", 1, nodes.getLength());
+    assertEquals(1, nodes.getLength(), "incorrect number of elements");
 
     for (int i = 0; i < nodes.getLength(); i++) {
       Element element = (Element) nodes.item(i);
@@ -252,11 +253,11 @@ public class TestAMWebServicesJobConf extends JerseyTestBase {
 
       for (int j = 0; j < properties.getLength(); j++) {
         Element property = (Element) properties.item(j);
-        assertNotNull("should have counters in the web service info", property);
+        assertNotNull(property, "should have counters in the web service info");
         String name = WebServicesTestUtils.getXmlString(property, "name");
         String value = WebServicesTestUtils.getXmlString(property, "value");
-        assertTrue("name not set", (name != null && !name.isEmpty()));
-        assertTrue("name not set", (value != null && !value.isEmpty()));
+        assertTrue((name != null && !name.isEmpty()), "name not set");
+        assertTrue((value != null && !value.isEmpty()), "name not set");
       }
     }
   }

+ 78 - 83
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobs.java

@@ -20,10 +20,10 @@ package org.apache.hadoop.mapreduce.v2.app.webapp;
 
 import static org.apache.hadoop.yarn.util.StringHelper.ujoin;
 import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.Mockito.mock;
 
 import java.io.StringReader;
@@ -60,7 +60,7 @@ import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.w3c.dom.Document;
 import org.w3c.dom.Element;
 import org.w3c.dom.NodeList;
@@ -119,7 +119,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
     assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8,
         response.getMediaType().toString());
     JSONObject json = response.readEntity(JSONObject.class);
-    assertEquals("incorrect number of elements", 1, json.length());
+    assertEquals(1, json.length(), "incorrect number of elements");
     JSONObject jobs = json.getJSONObject("jobs");
     JSONObject jobObject = jobs.getJSONObject("job");
     JSONArray arr = new JSONArray();
@@ -139,7 +139,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
     assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8,
         response.getMediaType().toString());
     JSONObject json = response.readEntity(JSONObject.class);
-    assertEquals("incorrect number of elements", 1, json.length());
+    assertEquals(1, json.length(), "incorrect number of elements");
     JSONObject jobs = json.getJSONObject("jobs");
     JSONObject jobObject = jobs.getJSONObject("job");
     JSONArray arr = new JSONArray();
@@ -158,7 +158,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
     assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8,
         response.getMediaType().toString());
     JSONObject json = response.readEntity(JSONObject.class);
-    assertEquals("incorrect number of elements", 1, json.length());
+    assertEquals(1, json.length(), "incorrect number of elements");
     JSONObject jobs = json.getJSONObject("jobs");
     JSONObject jobObject = jobs.getJSONObject("job");
     JSONArray arr = new JSONArray();
@@ -184,9 +184,9 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
     is.setCharacterStream(new StringReader(xml));
     Document dom = db.parse(is);
     NodeList jobs = dom.getElementsByTagName("jobs");
-    assertEquals("incorrect number of elements", 1, jobs.getLength());
+    assertEquals(1, jobs.getLength(), "incorrect number of elements");
     NodeList job = dom.getElementsByTagName("job");
-    assertEquals("incorrect number of elements", 1, job.getLength());
+    assertEquals(1, job.getLength(), "incorrect number of elements");
     verifyAMJobXML(job, appContext);
 
   }
@@ -204,7 +204,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
       assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8,
           response.getMediaType().toString());
       JSONObject json = response.readEntity(JSONObject.class);
-      assertEquals("incorrect number of elements", 1, json.length());
+      assertEquals(1, json.length(), "incorrect number of elements");
       JSONObject info = json.getJSONObject("job");
       verifyAMJob(info, jobsMap.get(id));
     }
@@ -224,7 +224,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
       assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8,
           response.getMediaType().toString());
       JSONObject json = response.readEntity(JSONObject.class);
-      assertEquals("incorrect number of elements", 1, json.length());
+      assertEquals(1, json.length(), "incorrect number of elements");
       JSONObject info = json.getJSONObject("job");
       verifyAMJob(info, jobsMap.get(id));
     }
@@ -242,7 +242,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
       assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8,
           response.getMediaType().toString());
       JSONObject json = response.readEntity(JSONObject.class);
-      assertEquals("incorrect number of elements", 1, json.length());
+      assertEquals(1, json.length(), "incorrect number of elements");
       JSONObject info = json.getJSONObject("job");
       verifyAMJob(info, jobsMap.get(id));
     }
@@ -264,7 +264,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
           response.getMediaType().toString());
       JSONObject msg = response.readEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
-      assertEquals("incorrect number of elements", 3, exception.length());
+      assertEquals(3, exception.length(), "incorrect number of elements");
       String message = exception.getString("message");
       String type = exception.getString("exception");
       String classname = exception.getString("javaClassName");
@@ -292,7 +292,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
           response.getMediaType().toString());
       JSONObject msg = response.readEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
-      assertEquals("incorrect number of elements", 3, exception.length());
+      assertEquals(3, exception.length(), "incorrect number of elements");
       String message = exception.getString("message");
       String type = exception.getString("exception");
       String classname = exception.getString("javaClassName");
@@ -316,7 +316,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
           response.getMediaType().toString());
       JSONObject msg = response.readEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
-      assertEquals("incorrect number of elements", 3, exception.length());
+      assertEquals(3, exception.length(), "incorrect number of elements");
       String message = exception.getString("message");
       String type = exception.getString("exception");
       String classname = exception.getString("javaClassName");
@@ -379,7 +379,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
           response.getMediaType().toString());
       JSONObject msg = response.readEntity(JSONObject.class);
       JSONObject exception = msg.getJSONObject("RemoteException");
-      assertEquals("incorrect number of elements", 3, exception.length());
+      assertEquals(3, exception.length(), "incorrect number of elements");
       String message = exception.getString("message");
       String type = exception.getString("exception");
       String classname = exception.getString("javaClassName");
@@ -419,7 +419,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
 
   public void verifyAMJob(JSONObject info, Job job) throws JSONException {
 
-    assertEquals("incorrect number of elements", 31, info.length());
+    assertEquals(31, info.length(), "incorrect number of elements");
 
     // everyone access fields
     verifyAMJobGeneric(job, info.getString("id"), info.getString("user"),
@@ -470,8 +470,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
         } else {
           fail("should have acls in the web service info");
         }
-        assertTrue("acl: " + expectName + " not found in webservice output",
-            found);
+        assertTrue(found, "acl: " + expectName + " not found in webservice output");
       }
     }
 
@@ -479,14 +478,14 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
 
   public void verifyAMJobXML(NodeList nodes, AppContext appContext) {
 
-    assertEquals("incorrect number of elements", 1, nodes.getLength());
+    assertEquals(1, nodes.getLength(), "incorrect number of elements");
 
     for (int i = 0; i < nodes.getLength(); i++) {
       Element element = (Element) nodes.item(i);
 
       Job job = appContext.getJob(MRApps.toJobID(WebServicesTestUtils
           .getXmlString(element, "id")));
-      assertNotNull("Job not found - output incorrect", job);
+      assertNotNull(job, "Job not found - output incorrect");
 
       verifyAMJobGeneric(job, WebServicesTestUtils.getXmlString(element, "id"),
           WebServicesTestUtils.getXmlString(element, "user"),
@@ -545,8 +544,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
           } else {
             fail("should have acls in the web service info");
           }
-          assertTrue("acl: " + expectName + " not found in webservice output",
-              found);
+          assertTrue(found, "acl: " + expectName + " not found in webservice output");
         }
       }
     }
@@ -560,27 +558,25 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
 
     WebServicesTestUtils.checkStringMatch("id", MRApps.toString(job.getID()),
         id);
-    WebServicesTestUtils.checkStringMatch("user", job.getUserName().toString(),
+    WebServicesTestUtils.checkStringMatch("user", job.getUserName(),
         user);
     WebServicesTestUtils.checkStringMatch("name", job.getName(), name);
     WebServicesTestUtils.checkStringMatch("state", job.getState().toString(),
         state);
 
-    assertEquals("startTime incorrect", report.getStartTime(), startTime);
-    assertEquals("finishTime incorrect", report.getFinishTime(), finishTime);
-    assertEquals("elapsedTime incorrect",
-        Times.elapsed(report.getStartTime(), report.getFinishTime()),
-        elapsedTime);
-    assertEquals("mapsTotal incorrect", job.getTotalMaps(), mapsTotal);
-    assertEquals("mapsCompleted incorrect", job.getCompletedMaps(),
-        mapsCompleted);
-    assertEquals("reducesTotal incorrect", job.getTotalReduces(), reducesTotal);
-    assertEquals("reducesCompleted incorrect", job.getCompletedReduces(),
-        reducesCompleted);
-    assertEquals("mapProgress incorrect", report.getMapProgress() * 100,
-        mapProgress, 0);
-    assertEquals("reduceProgress incorrect", report.getReduceProgress() * 100,
-        reduceProgress, 0);
+    assertEquals(report.getStartTime(), startTime, "startTime incorrect");
+    assertEquals(report.getFinishTime(), finishTime, "finishTime incorrect");
+    assertEquals(Times.elapsed(report.getStartTime(), report.getFinishTime()),
+        elapsedTime, "elapsedTime incorrect");
+    assertEquals(job.getTotalMaps(), mapsTotal, "mapsTotal incorrect");
+    assertEquals(job.getCompletedMaps(),
+        mapsCompleted, "mapsCompleted incorrect");
+    assertEquals(job.getTotalReduces(), reducesTotal, "reducesTotal incorrect");
+    assertEquals(job.getCompletedReduces(),
+        reducesCompleted, "reducesCompleted incorrect");
+    assertEquals(report.getMapProgress() * 100, mapProgress, 0, "mapProgress incorrect");
+    assertEquals(report.getReduceProgress() * 100,
+        reduceProgress, 0, "reduceProgress incorrect");
   }
 
   public void verifyAMJobGenericSecure(Job job, int mapsPending,
@@ -603,28 +599,27 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
     WebServicesTestUtils.checkStringMatch("diagnostics", diagString,
         diagnostics);
 
-    assertEquals("isUber incorrect", job.isUber(), uberized);
+    assertEquals(job.isUber(), uberized, "isUber incorrect");
 
     // unfortunately the following fields are all calculated in JobInfo
     // so not easily accessible without doing all the calculations again.
     // For now just make sure they are present.
-    assertTrue("mapsPending not >= 0", mapsPending >= 0);
-    assertTrue("mapsRunning not >= 0", mapsRunning >= 0);
-    assertTrue("reducesPending not >= 0", reducesPending >= 0);
-    assertTrue("reducesRunning not >= 0", reducesRunning >= 0);
+    assertTrue(mapsPending >= 0, "mapsPending not >= 0");
+    assertTrue(mapsRunning >= 0, "mapsRunning not >= 0");
+    assertTrue(reducesPending >= 0, "reducesPending not >= 0");
+    assertTrue(reducesRunning >= 0, "reducesRunning not >= 0");
 
-    assertTrue("newReduceAttempts not >= 0", newReduceAttempts >= 0);
-    assertTrue("runningReduceAttempts not >= 0", runningReduceAttempts >= 0);
-    assertTrue("failedReduceAttempts not >= 0", failedReduceAttempts >= 0);
-    assertTrue("killedReduceAttempts not >= 0", killedReduceAttempts >= 0);
-    assertTrue("successfulReduceAttempts not >= 0",
-        successfulReduceAttempts >= 0);
+    assertTrue(newReduceAttempts >= 0, "newReduceAttempts not >= 0");
+    assertTrue(runningReduceAttempts >= 0, "runningReduceAttempts not >= 0");
+    assertTrue(failedReduceAttempts >= 0, "failedReduceAttempts not >= 0");
+    assertTrue(killedReduceAttempts >= 0, "killedReduceAttempts not >= 0");
+    assertTrue(successfulReduceAttempts >= 0, "successfulReduceAttempts not >= 0");
 
-    assertTrue("newMapAttempts not >= 0", newMapAttempts >= 0);
-    assertTrue("runningMapAttempts not >= 0", runningMapAttempts >= 0);
-    assertTrue("failedMapAttempts not >= 0", failedMapAttempts >= 0);
-    assertTrue("killedMapAttempts not >= 0", killedMapAttempts >= 0);
-    assertTrue("successfulMapAttempts not >= 0", successfulMapAttempts >= 0);
+    assertTrue(newMapAttempts >= 0, "newMapAttempts not >= 0");
+    assertTrue(runningMapAttempts >= 0, "runningMapAttempts not >= 0");
+    assertTrue(failedMapAttempts >= 0, "failedMapAttempts not >= 0");
+    assertTrue(killedMapAttempts >= 0, "killedMapAttempts not >= 0");
+    assertTrue(successfulMapAttempts >= 0, "successfulMapAttempts not >= 0");
 
   }
 
@@ -640,7 +635,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
       assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8,
           response.getMediaType().toString());
       JSONObject json = response.readEntity(JSONObject.class);
-      assertEquals("incorrect number of elements", 1, json.length());
+      assertEquals(1, json.length(), "incorrect number of elements");
       JSONObject info = json.getJSONObject("jobCounters");
       verifyAMJobCounters(info, jobsMap.get(id));
     }
@@ -659,7 +654,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
       assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8,
           response.getMediaType().toString());
       JSONObject json = response.readEntity(JSONObject.class);
-      assertEquals("incorrect number of elements", 1, json.length());
+      assertEquals(1, json.length(), "incorrect number of elements");
       JSONObject info = json.getJSONObject("jobCounters");
       verifyAMJobCounters(info, jobsMap.get(id));
     }
@@ -677,7 +672,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
       assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8,
           response.getMediaType().toString());
       JSONObject json = response.readEntity(JSONObject.class);
-      assertEquals("incorrect number of elements", 1, json.length());
+      assertEquals(1, json.length(), "incorrect number of elements");
       JSONObject info = json.getJSONObject("jobCounters");
       verifyAMJobCounters(info, jobsMap.get(id));
     }
@@ -709,7 +704,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
   public void verifyAMJobCounters(JSONObject info, Job job)
       throws JSONException {
 
-    assertEquals("incorrect number of elements", 2, info.length());
+    assertEquals(2, info.length(), "incorrect number of elements");
 
     WebServicesTestUtils.checkStringMatch("id", MRApps.toString(job.getID()),
         info.getString("id"));
@@ -719,22 +714,22 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
     for (int i = 0; i < counterGroups.length(); i++) {
       JSONObject counterGroup = counterGroups.getJSONObject(i);
       String name = counterGroup.getString("counterGroupName");
-      assertTrue("name not set", (name != null && !name.isEmpty()));
+      assertTrue((name != null && !name.isEmpty()), "name not set");
       JSONArray counters = counterGroup.getJSONArray("counter");
       for (int j = 0; j < counters.length(); j++) {
         JSONObject counter = counters.getJSONObject(j);
         String counterName = counter.getString("name");
-        assertTrue("counter name not set",
-            (counterName != null && !counterName.isEmpty()));
+        assertTrue((counterName != null && !counterName.isEmpty()),
+            "counter name not set");
 
         long mapValue = counter.getLong("mapCounterValue");
-        assertTrue("mapCounterValue  >= 0", mapValue >= 0);
+        assertTrue(mapValue >= 0, "mapCounterValue  >= 0");
 
         long reduceValue = counter.getLong("reduceCounterValue");
-        assertTrue("reduceCounterValue  >= 0", reduceValue >= 0);
+        assertTrue(reduceValue >= 0, "reduceCounterValue  >= 0");
 
         long totalValue = counter.getLong("totalCounterValue");
-        assertTrue("totalCounterValue  >= 0", totalValue >= 0);
+        assertTrue(totalValue >= 0, "totalCounterValue  >= 0");
 
       }
     }
@@ -745,7 +740,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
     for (int i = 0; i < nodes.getLength(); i++) {
       Element element = (Element) nodes.item(i);
 
-      assertNotNull("Job not found - output incorrect", job);
+      assertNotNull(job, "Job not found - output incorrect");
 
       WebServicesTestUtils.checkStringMatch("id", MRApps.toString(job.getID()),
           WebServicesTestUtils.getXmlString(element, "id"));
@@ -755,29 +750,29 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
 
       for (int j = 0; j < groups.getLength(); j++) {
         Element counters = (Element) groups.item(j);
-        assertNotNull("should have counters in the web service info", counters);
+        assertNotNull(counters, "should have counters in the web service info");
         String name = WebServicesTestUtils.getXmlString(counters,
             "counterGroupName");
-        assertTrue("name not set", (name != null && !name.isEmpty()));
+        assertTrue((name != null && !name.isEmpty()), "name not set");
         NodeList counterArr = counters.getElementsByTagName("counter");
         for (int z = 0; z < counterArr.getLength(); z++) {
           Element counter = (Element) counterArr.item(z);
           String counterName = WebServicesTestUtils.getXmlString(counter,
               "name");
-          assertTrue("counter name not set",
-              (counterName != null && !counterName.isEmpty()));
+          assertTrue((counterName != null && !counterName.isEmpty()),
+              "counter name not set");
 
           long mapValue = WebServicesTestUtils.getXmlLong(counter,
               "mapCounterValue");
-          assertTrue("mapCounterValue not >= 0", mapValue >= 0);
+          assertTrue(mapValue >= 0, "mapCounterValue not >= 0");
 
           long reduceValue = WebServicesTestUtils.getXmlLong(counter,
               "reduceCounterValue");
-          assertTrue("reduceCounterValue  >= 0", reduceValue >= 0);
+          assertTrue(reduceValue >= 0, "reduceCounterValue  >= 0");
 
           long totalValue = WebServicesTestUtils.getXmlLong(counter,
               "totalCounterValue");
-          assertTrue("totalCounterValue  >= 0", totalValue >= 0);
+          assertTrue(totalValue >= 0, "totalCounterValue  >= 0");
         }
       }
     }
@@ -796,7 +791,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
       assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8,
           response.getMediaType().toString());
       JSONObject json = response.readEntity(JSONObject.class);
-      assertEquals("incorrect number of elements", 1, json.length());
+      assertEquals(1, json.length(), "incorrect number of elements");
       JSONObject info = json.getJSONObject("jobAttempts");
       verifyJobAttempts(info, jobsMap.get(id));
     }
@@ -814,7 +809,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
       assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8,
           response.getMediaType().toString());
       JSONObject json = response.readEntity(JSONObject.class);
-      assertEquals("incorrect number of elements", 1, json.length());
+      assertEquals(1, json.length(), "incorrect number of elements");
       JSONObject info = json.getJSONObject("jobAttempts");
       verifyJobAttempts(info, jobsMap.get(id));
     }
@@ -833,7 +828,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
       assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8,
           response.getMediaType().toString());
       JSONObject json = response.readEntity(JSONObject.class);
-      assertEquals("incorrect number of elements", 1, json.length());
+      assertEquals(1, json.length(), "incorrect number of elements");
       JSONObject info = json.getJSONObject("jobAttempts");
       verifyJobAttempts(info, jobsMap.get(id));
     }
@@ -858,7 +853,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
       is.setCharacterStream(new StringReader(xml));
       Document dom = db.parse(is);
       NodeList attempts = dom.getElementsByTagName("jobAttempts");
-      assertEquals("incorrect number of elements", 1, attempts.getLength());
+      assertEquals(1, attempts.getLength(), "incorrect number of elements");
       NodeList info = dom.getElementsByTagName("jobAttempt");
       verifyJobAttemptsXML(info, jobsMap.get(id));
     }
@@ -868,7 +863,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
       throws JSONException {
 
     JSONArray attempts = info.getJSONArray("jobAttempt");
-    assertEquals("incorrect number of elements", 2, attempts.length());
+    assertEquals(2, attempts.length(), "incorrect number of elements");
     for (int i = 0; i < attempts.length(); i++) {
       JSONObject attempt = attempts.getJSONObject(i);
       verifyJobAttemptsGeneric(job, attempt.getString("nodeHttpAddress"),
@@ -880,7 +875,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
 
   public void verifyJobAttemptsXML(NodeList nodes, Job job) {
 
-    assertEquals("incorrect number of elements", 2, nodes.getLength());
+    assertEquals(2, nodes.getLength(), "incorrect number of elements");
     for (int i = 0; i < nodes.getLength(); i++) {
       Element element = (Element) nodes.item(i);
       verifyJobAttemptsGeneric(job,
@@ -906,17 +901,17 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
             + nmHttpPort, nodeHttpAddress);
         WebServicesTestUtils.checkStringMatch("nodeId",
             NodeId.newInstance(nmHost, nmPort).toString(), nodeId);
-        assertTrue("startime not greater than 0", startTime > 0);
+        assertTrue(startTime > 0, "start time not greater than 0");
         WebServicesTestUtils.checkStringMatch("containerId", amInfo
             .getContainerId().toString(), containerId);
 
         String localLogsLink =ujoin("node", "containerlogs", containerId,
             job.getUserName());
 
-        assertTrue("logsLink", logsLink.contains(localLogsLink));
+        assertTrue(logsLink.contains(localLogsLink), "logsLink");
       }
     }
-    assertTrue("attempt: " + id + " was not found", attemptFound);
+    assertTrue(attemptFound, "attempt: " + id + " was not found");
   }
 
 }

+ 45 - 47
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesTasks.java

@@ -19,9 +19,9 @@
 package org.apache.hadoop.mapreduce.v2.app.webapp;
 
 import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.Mockito.mock;
 
 import java.io.StringReader;
@@ -56,7 +56,7 @@ import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.w3c.dom.Document;
 import org.w3c.dom.Element;
 import org.w3c.dom.NodeList;
@@ -119,10 +119,10 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
       assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8,
           response.getMediaType().toString());
       JSONObject json = response.readEntity(JSONObject.class);
-      assertEquals("incorrect number of elements", 1, json.length());
+      assertEquals(1, json.length(), "incorrect number of elements");
       JSONObject tasks = json.getJSONObject("tasks");
       JSONArray arr = tasks.getJSONArray("task");
-      assertEquals("incorrect number of elements", 2, arr.length());
+      assertEquals(2, arr.length(), "incorrect number of elements");
 
       verifyAMTask(arr, jobsMap.get(id), null);
     }
@@ -139,10 +139,10 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
       assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8,
           response.getMediaType().toString());
       JSONObject json = response.readEntity(JSONObject.class);
-      assertEquals("incorrect number of elements", 1, json.length());
+      assertEquals(1, json.length(), "incorrect number of elements");
       JSONObject tasks = json.getJSONObject("tasks");
       JSONArray arr = tasks.getJSONArray("task");
-      assertEquals("incorrect number of elements", 2, arr.length());
+      assertEquals(2, arr.length(), "incorrect number of elements");
 
       verifyAMTask(arr, jobsMap.get(id), null);
     }
@@ -160,10 +160,10 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
       assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8,
           response.getMediaType().toString());
       JSONObject json = response.readEntity(JSONObject.class);
-      assertEquals("incorrect number of elements", 1, json.length());
+      assertEquals(1, json.length(), "incorrect number of elements");
       JSONObject tasks = json.getJSONObject("tasks");
       JSONArray arr = tasks.getJSONArray("task");
-      assertEquals("incorrect number of elements", 2, arr.length());
+      assertEquals(2, arr.length(), "incorrect number of elements");
 
       verifyAMTask(arr, jobsMap.get(id), null);
     }
@@ -188,7 +188,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
       is.setCharacterStream(new StringReader(xml));
       Document dom = db.parse(is);
       NodeList tasks = dom.getElementsByTagName("tasks");
-      assertEquals("incorrect number of elements", 1, tasks.getLength());
+      assertEquals(1, tasks.getLength(), "incorrect number of elements");
       NodeList task = dom.getElementsByTagName("task");
       verifyAMTaskXML(task, jobsMap.get(id));
     }
@@ -207,12 +207,12 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
       assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8,
           response.getMediaType().toString());
       JSONObject json = response.readEntity(JSONObject.class);
-      assertEquals("incorrect number of elements", 1, json.length());
+      assertEquals(1, json.length(), "incorrect number of elements");
       JSONObject tasks = json.getJSONObject("tasks");
       JSONObject task = tasks.getJSONObject("task");
       JSONArray arr = new JSONArray();
       arr.put(task);
-      assertEquals("incorrect number of elements", 1, arr.length());
+      assertEquals(1, arr.length(), "incorrect number of elements");
       verifyAMTask(arr, jobsMap.get(id), type);
     }
   }
@@ -230,12 +230,12 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
       assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8,
           response.getMediaType().toString());
       JSONObject json = response.readEntity(JSONObject.class);
-      assertEquals("incorrect number of elements", 1, json.length());
+      assertEquals(1, json.length(), "incorrect number of elements");
       JSONObject tasks = json.getJSONObject("tasks");
       JSONObject task = tasks.getJSONObject("task");
       JSONArray arr = new JSONArray();
       arr.put(task);
-      assertEquals("incorrect number of elements", 1, arr.length());
+      assertEquals(1, arr.length(), "incorrect number of elements");
       verifyAMTask(arr, jobsMap.get(id), type);
     }
   }
@@ -261,7 +261,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
             response.getMediaType().toString());
         JSONObject msg = response.readEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
-        assertEquals("incorrect number of elements", 3, exception.length());
+        assertEquals(3, exception.length(), "incorrect number of elements");
         String message = exception.getString("message");
         String type = exception.getString("exception");
         String classname = exception.getString("javaClassName");
@@ -290,7 +290,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
         assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8,
             response.getMediaType().toString());
         JSONObject json = response.readEntity(JSONObject.class);
-        assertEquals("incorrect number of elements", 1, json.length());
+        assertEquals(1, json.length(), "incorrect number of elements");
         JSONObject info = json.getJSONObject("task");
         verifyAMSingleTask(info, task);
       }
@@ -312,7 +312,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
         assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8,
             response.getMediaType().toString());
         JSONObject json = response.readEntity(JSONObject.class);
-        assertEquals("incorrect number of elements", 1, json.length());
+        assertEquals(1, json.length(), "incorrect number of elements");
         JSONObject info = json.getJSONObject("task");
         verifyAMSingleTask(info, task);
       }
@@ -334,7 +334,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
         assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8,
             response.getMediaType().toString());
         JSONObject json = response.readEntity(JSONObject.class);
-        assertEquals("incorrect number of elements", 1, json.length());
+        assertEquals(1, json.length(), "incorrect number of elements");
         JSONObject info = json.getJSONObject("task");
         verifyAMSingleTask(info, task);
       }
@@ -360,7 +360,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
             response.getMediaType().toString());
         JSONObject msg = response.readEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
-        assertEquals("incorrect number of elements", 3, exception.length());
+        assertEquals(3, exception.length(), "incorrect number of elements");
         String message = exception.getString("message");
         String type = exception.getString("exception");
         String classname = exception.getString("javaClassName");
@@ -396,7 +396,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
         String entity = response.readEntity(String.class);
         JSONObject msg = new JSONObject(entity);
         JSONObject exception = msg.getJSONObject("RemoteException");
-        assertEquals("incorrect number of elements", 3, exception.length());
+        assertEquals(3, exception.length(), "incorrect number of elements");
         String message = exception.getString("message");
         String type = exception.getString("exception");
         String classname = exception.getString("javaClassName");
@@ -427,7 +427,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
             response.getMediaType().toString());
         JSONObject msg = response.readEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
-        assertEquals("incorrect number of elements", 3, exception.length());
+        assertEquals(3, exception.length(), "incorrect number of elements");
         String message = exception.getString("message");
         String type = exception.getString("exception");
         String classname = exception.getString("javaClassName");
@@ -462,7 +462,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
             response.getMediaType().toString());
         JSONObject msg = response.readEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
-        assertEquals("incorrect number of elements", 3, exception.length());
+        assertEquals(3, exception.length(), "incorrect number of elements");
         String message = exception.getString("message");
         String type = exception.getString("exception");
         String classname = exception.getString("javaClassName");
@@ -496,7 +496,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
             response.getMediaType().toString());
         JSONObject msg = response.readEntity(JSONObject.class);
         JSONObject exception = msg.getJSONObject("RemoteException");
-        assertEquals("incorrect number of elements", 3, exception.length());
+        assertEquals(3, exception.length(), "incorrect number of elements");
         String message = exception.getString("message");
         String type = exception.getString("exception");
         String classname = exception.getString("javaClassName");
@@ -545,7 +545,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
 
   public void verifyAMSingleTask(JSONObject info, Task task)
       throws JSONException {
-    assertEquals("incorrect number of elements", 9, info.length());
+    assertEquals(9, info.length(), "incorrect number of elements");
 
     verifyTaskGeneric(task, info.getString("id"), info.getString("state"),
         info.getString("type"), info.getString("successfulAttempt"),
@@ -569,7 +569,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
             verifyAMSingleTask(info, task);
           }
         }
-        assertTrue("task with id: " + tid + " not in web service output", found);
+        assertTrue(found, "task with id: " + tid + " not in web service output");
       }
     }
   }
@@ -588,12 +588,12 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
     WebServicesTestUtils.checkStringMatch("state", report.getTaskState()
         .toString(), state);
     // not easily checked without duplicating logic, just make sure its here
-    assertNotNull("successfulAttempt null", successfulAttempt);
-    assertEquals("startTime wrong", report.getStartTime(), startTime);
-    assertEquals("finishTime wrong", report.getFinishTime(), finishTime);
-    assertEquals("elapsedTime wrong", finishTime - startTime, elapsedTime);
-    assertEquals("progress wrong", report.getProgress() * 100, progress, 1e-3f);
-    assertEquals("status wrong", report.getStatus(), status);
+    assertNotNull(successfulAttempt, "successfulAttempt null");
+    assertEquals(report.getStartTime(), startTime, "startTime wrong");
+    assertEquals(report.getFinishTime(), finishTime, "finishTime wrong");
+    assertEquals(finishTime - startTime, elapsedTime, "elapsedTime wrong");
+    assertEquals(report.getProgress() * 100, progress, 1e-3f, "progress wrong");
+    assertEquals(report.getStatus(), status, "status wrong");
   }
 
   public void verifyAMSingleTaskXML(Element element, Task task) {
@@ -610,7 +610,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
 
   public void verifyAMTaskXML(NodeList nodes, Job job) {
 
-    assertEquals("incorrect number of elements", 2, nodes.getLength());
+    assertEquals(2, nodes.getLength(), "incorrect number of elements");
 
     for (Task task : job.getTasks().values()) {
       TaskId id = task.getID();
@@ -624,7 +624,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
           verifyAMSingleTaskXML(element, task);
         }
       }
-      assertTrue("task with id: " + tid + " not in web service output", found);
+      assertTrue(found, "task with id: " + tid + " not in web service output");
     }
   }
 
@@ -643,7 +643,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
         assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8,
             response.getMediaType().toString());
         JSONObject json = response.readEntity(JSONObject.class);
-        assertEquals("incorrect number of elements", 1, json.length());
+        assertEquals(1, json.length(), "incorrect number of elements");
         JSONObject info = json.getJSONObject("jobTaskCounters");
         verifyAMJobTaskCounters(info, task);
       }
@@ -665,7 +665,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
         assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8,
             response.getMediaType().toString());
         JSONObject json = response.readEntity(JSONObject.class);
-        assertEquals("incorrect number of elements", 1, json.length());
+        assertEquals(1, json.length(), "incorrect number of elements");
         JSONObject info = json.getJSONObject("jobTaskCounters");
         verifyAMJobTaskCounters(info, task);
       }
@@ -687,7 +687,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
         assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8,
             response.getMediaType().toString());
         JSONObject json = response.readEntity(JSONObject.class);
-        assertEquals("incorrect number of elements", 1, json.length());
+        assertEquals(1, json.length(), "incorrect number of elements");
         JSONObject info = json.getJSONObject("jobTaskCounters");
         verifyAMJobTaskCounters(info, task);
       }
@@ -723,7 +723,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
   public void verifyAMJobTaskCounters(JSONObject info, Task task)
       throws JSONException {
 
-    assertEquals("incorrect number of elements", 2, info.length());
+    assertEquals(2, info.length(), "incorrect number of elements");
 
     WebServicesTestUtils.checkStringMatch("id", MRApps.toString(task.getID()),
         info.getString("id"));
@@ -733,15 +733,14 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
     for (int i = 0; i < counterGroups.length(); i++) {
       JSONObject counterGroup = counterGroups.getJSONObject(i);
       String name = counterGroup.getString("counterGroupName");
-      assertTrue("name not set", (name != null && !name.isEmpty()));
+      assertTrue((name != null && !name.isEmpty()), "name not set");
       JSONArray counters = counterGroup.getJSONArray("counter");
       for (int j = 0; j < counters.length(); j++) {
         JSONObject counter = counters.getJSONObject(j);
         String counterName = counter.getString("name");
-        assertTrue("name not set",
-            (counterName != null && !counterName.isEmpty()));
+        assertTrue((counterName != null && !counterName.isEmpty()), "name not set");
         long value = counter.getLong("value");
-        assertTrue("value  >= 0", value >= 0);
+        assertTrue(value >= 0, "value  >= 0");
       }
     }
   }
@@ -760,20 +759,19 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
 
       for (int j = 0; j < groups.getLength(); j++) {
         Element counters = (Element) groups.item(j);
-        assertNotNull("should have counters in the web service info", counters);
+        assertNotNull(counters, "should have counters in the web service info");
         String name = WebServicesTestUtils.getXmlString(counters,
             "counterGroupName");
-        assertTrue("name not set", (name != null && !name.isEmpty()));
+        assertTrue((name != null && !name.isEmpty()), "name not set");
         NodeList counterArr = counters.getElementsByTagName("counter");
         for (int z = 0; z < counterArr.getLength(); z++) {
           Element counter = (Element) counterArr.item(z);
           String counterName = WebServicesTestUtils.getXmlString(counter,
               "name");
-          assertTrue("counter name not set",
-              (counterName != null && !counterName.isEmpty()));
+          assertTrue((counterName != null && !counterName.isEmpty()), "counter name not set");
 
           long value = WebServicesTestUtils.getXmlLong(counter, "value");
-          assertTrue("value not >= 0", value >= 0);
+          assertTrue(value >= 0, "value not >= 0");
 
         }
       }

+ 11 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAppController.java

@@ -17,7 +17,13 @@
  */
 package org.apache.hadoop.mapreduce.v2.app.webapp;
 
-import static org.mockito.Mockito.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.any;
+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.Iterator;
@@ -37,9 +43,8 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.webapp.Controller.RequestContext;
 import org.apache.hadoop.yarn.webapp.MimeType;
 import org.apache.hadoop.yarn.webapp.ResponseInfo;
-import org.junit.Before;
-import org.junit.Test;
-import static org.junit.Assert.*;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 public class TestAppController {
 
@@ -48,7 +53,7 @@ public class TestAppController {
   private Job job;
   private static final String taskId = "task_01_01_m_01";
 
-  @Before
+  @BeforeEach
   public void setUp() throws IOException {
     AppContext context = mock(AppContext.class);
     when(context.getApplicationID()).thenReturn(
@@ -275,8 +280,7 @@ public class TestAppController {
   public void testDownloadConfiguration() {
     appController.downloadConf();
     String jobConfXml = appController.getData();
-    assertTrue("Error downloading the job configuration file.",
-        !jobConfXml.contains("Error"));
+    assertTrue(!jobConfXml.contains("Error"), "Error downloading the job configuration file.");
   }
 
   /**

+ 7 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestBlocks.java

@@ -26,7 +26,7 @@ import java.util.Map;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.util.MRJobConfUtil;
 import org.apache.hadoop.yarn.webapp.View;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
@@ -51,8 +51,12 @@ import org.apache.hadoop.yarn.webapp.view.BlockForTest;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block;
 
-import static org.mockito.Mockito.*;
-import static org.junit.Assert.*;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
 
 public class TestBlocks {
   private ByteArrayOutputStream data = new ByteArrayOutputStream();

+ 5 - 0
hadoop-project/pom.xml

@@ -1321,6 +1321,11 @@
           </exclusion>
         </exclusions>
       </dependency>
+      <dependency>
+        <groupId>org.mockito</groupId>
+        <artifactId>mockito-junit-jupiter</artifactId>
+        <version>4.11.0</version>
+      </dependency>
       <dependency>
         <groupId>org.apache.avro</groupId>
         <artifactId>avro</artifactId>

+ 12 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java

@@ -369,9 +369,15 @@ public class WebApps {
 
         if (httpScheme.equals(WebAppUtils.HTTPS_PREFIX)) {
           String amKeystoreLoc = System.getenv("KEYSTORE_FILE_LOCATION");
+          if (StringUtils.isBlank(amKeystoreLoc)) {
+            amKeystoreLoc = System.getProperty("KEYSTORE_FILE_LOCATION");
+          }
           if (amKeystoreLoc != null) {
             LOG.info("Setting keystore location to " + amKeystoreLoc);
             String password = System.getenv("KEYSTORE_PASSWORD");
+            if (StringUtils.isBlank(password)) {
+              password = System.getProperty("KEYSTORE_PASSWORD");
+            }
             builder.keyStore(amKeystoreLoc, password, "jks");
           } else {
             LOG.info("Loading standard ssl config");
@@ -380,9 +386,15 @@ public class WebApps {
           builder.needsClientAuth(needsClientAuth);
           if (needsClientAuth) {
             String amTruststoreLoc = System.getenv("TRUSTSTORE_FILE_LOCATION");
+            if (StringUtils.isBlank(amTruststoreLoc)) {
+              amTruststoreLoc = System.getProperty("TRUSTSTORE_FILE_LOCATION");
+            }
             if (amTruststoreLoc != null) {
               LOG.info("Setting truststore location to " + amTruststoreLoc);
               String password = System.getenv("TRUSTSTORE_PASSWORD");
+              if (StringUtils.isBlank(password)) {
+                password = System.getProperty("TRUSTSTORE_PASSWORD");
+              }
               builder.trustStore(amTruststoreLoc, password, "jks");
             }
           }

Algunos archivos no se mostraron porque demasiados archivos cambiaron en este cambio