Sfoglia il codice sorgente

MAPREDUCE-7420. [JDK17] Upgrade Junit 4 to 5 in hadoop-mapreduce-client-core Part1. (#7363)

Co-authored-by: Chris Nauroth <cnauroth@apache.org>
Reviewed-by: Chris Nauroth <cnauroth@apache.org>
Signed-off-by: Shilun Fan <slfan1989@apache.org>
slfan1989 2 mesi fa
parent
commit
8424c155f1
65 ha cambiato i file con 1274 aggiunte e 1136 eliminazioni
  1. 5 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
  2. 11 10
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestClock.java
  3. 12 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestClusterStatus.java
  4. 45 45
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestCounters.java
  5. 64 52
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestFileInputFormat.java
  6. 23 20
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestFileOutputCommitter.java
  7. 6 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestIndexCache.java
  8. 11 11
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobAclsManager.java
  9. 48 43
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobConf.java
  10. 11 10
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobEndNotifier.java
  11. 9 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobInfo.java
  12. 4 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobQueueClient.java
  13. 65 66
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestLineRecordReader.java
  14. 10 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestLocatedFileStatusFetcher.java
  15. 5 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestMapFileOutputFormat.java
  16. 33 38
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestMapTask.java
  17. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestMaster.java
  18. 20 12
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestOldMethodsJobID.java
  19. 18 10
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestQueue.java
  20. 9 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestSkipBadRecords.java
  21. 12 10
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestTask.java
  22. 9 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestTaskLog.java
  23. 17 12
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestTaskProgressReporter.java
  24. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/lib/TestCombineFileRecordReader.java
  25. 17 12
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/lib/db/TestDBInputFormat.java
  26. 5 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestCluster.java
  27. 4 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestContextFactory.java
  28. 13 10
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJob.java
  29. 7 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobMonitorAndPrint.java
  30. 37 36
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java
  31. 8 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploaderWithSharedCache.java
  32. 20 19
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobSubmissionFiles.java
  33. 9 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestShufflePlugin.java
  34. 107 111
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestTaskID.java
  35. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/checkpoint/TestFSCheckpointID.java
  36. 12 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/checkpoint/TestFSCheckpointService.java
  37. 39 40
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/filecache/TestClientDistributedCacheManager.java
  38. 37 41
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/filecache/TestDistributedCache.java
  39. 17 15
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestHistoryViewerPrinter.java
  40. 4 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/db/DriverForTest.java
  41. 16 10
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDbClasses.java
  42. 19 13
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestSplitters.java
  43. 7 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/BaseTestLineRecordReaderBZip2.java
  44. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileRecordReader.java
  45. 100 79
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestFileInputFormat.java
  46. 65 68
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestLineRecordReader.java
  47. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestJobControl.java
  48. 38 37
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java
  49. 3 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputFormat.java
  50. 4 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMapFileOutputFormat.java
  51. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestPathOutputCommitter.java
  52. 13 17
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestPathOutputCommitterFactory.java
  53. 10 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestPreemptableFileOutputCommitter.java
  54. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestRehashPartitioner.java
  55. 13 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/security/TestTokenCache.java
  56. 8 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/split/TestJobSplitWriter.java
  57. 8 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/split/TestJobSplitWriterWithEC.java
  58. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestEventFetcher.java
  59. 54 32
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestFetcher.java
  60. 35 39
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestMergeManager.java
  61. 36 37
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestMerger.java
  62. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestShuffleClientMetrics.java
  63. 18 18
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestShuffleScheduler.java
  64. 19 21
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/tools/TestCLI.java
  65. 5 0
      hadoop-project/pom.xml

+ 5 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml

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

+ 11 - 10
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestClock.java

@@ -18,21 +18,22 @@
  */
 package org.apache.hadoop.mapred;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
-import static org.junit.Assert.*;
 /**
- *  test Clock class
- *
+ * test Clock class
  */
 public class TestClock {
 
-  @Test  (timeout=10000)
+  @Test
+  @Timeout(value = 10)
   public void testClock(){
-    Clock clock= new Clock();
-    long templateTime=System.currentTimeMillis();
-    long time=clock.getTime();
-    assertEquals(templateTime, time,30);
-
+    Clock clock = new Clock();
+    long templateTime = System.currentTimeMillis();
+    long time = clock.getTime();
+    assertEquals(templateTime, time, 30);
   }
 }

+ 12 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestClusterStatus.java

@@ -17,25 +17,29 @@
  */
 package org.apache.hadoop.mapred;
 
-import org.junit.Assert;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
-import org.junit.Test;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class TestClusterStatus {
 
   private ClusterStatus clusterStatus = new ClusterStatus();
 
   @SuppressWarnings("deprecation")
-  @Test (timeout = 10000)
-  public void testGraylistedTrackers() {
-    Assert.assertEquals(0, clusterStatus.getGraylistedTrackers());
-    Assert.assertTrue(clusterStatus.getGraylistedTrackerNames().isEmpty());
+  @Test
+  @Timeout(value = 10)
+  public void testGrayListedTrackers() {
+    assertEquals(0, clusterStatus.getGraylistedTrackers());
+    assertTrue(clusterStatus.getGraylistedTrackerNames().isEmpty());
   }
 
   @SuppressWarnings("deprecation")
-  @Test (timeout = 10000)
+  @Test
+  @Timeout(value = 10)
   public void testJobTrackerState() {
-    Assert.assertEquals(JobTracker.State.RUNNING,
+    assertEquals(JobTracker.State.RUNNING,
         clusterStatus.getJobTrackerState());
   }
 }

+ 45 - 45
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestCounters.java

@@ -17,9 +17,12 @@
  */
 package org.apache.hadoop.mapred;
 
-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.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 java.io.IOException;
 import java.text.ParseException;
@@ -27,8 +30,6 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Random;
 
-import org.junit.Assert;
-
 import org.apache.hadoop.mapred.Counters.Counter;
 import org.apache.hadoop.mapred.Counters.CountersExceededException;
 import org.apache.hadoop.mapred.Counters.Group;
@@ -38,7 +39,7 @@ import org.apache.hadoop.mapreduce.JobCounter;
 import org.apache.hadoop.mapreduce.TaskCounter;
 import org.apache.hadoop.mapreduce.counters.FrameworkCounterGroup;
 import org.apache.hadoop.mapreduce.counters.CounterGroupFactory.FrameworkGroupFactory;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -85,14 +86,14 @@ public class TestCounters {
    */
   private void testCounter(Counters counter) throws ParseException {
     String compactEscapedString = counter.makeEscapedCompactString();
-    assertFalse("compactEscapedString should not contain null",
-                compactEscapedString.contains("null"));
+    assertFalse(compactEscapedString.contains("null"),
+        "compactEscapedString should not contain null");
     
     Counters recoveredCounter = 
       Counters.fromEscapedCompactString(compactEscapedString);
     // Check for recovery from string
-    assertEquals("Recovered counter does not match on content", 
-                 counter, recoveredCounter);
+    assertEquals(counter, recoveredCounter,
+        "Recovered counter does not match on content");
   }
   
   @Test
@@ -134,19 +135,19 @@ public class TestCounters {
       long expectedValue = initValue;
       Counter counter = counters.findCounter("foo", "bar");
       counter.setValue(initValue);
-      assertEquals("Counter value is not initialized correctly",
-                   expectedValue, counter.getValue());
+      assertEquals(expectedValue, counter.getValue(),
+          "Counter value is not initialized correctly");
       for (int j = 0; j < NUMBER_INC; j++) {
         int incValue = rand.nextInt();
         counter.increment(incValue);
         expectedValue += incValue;
-        assertEquals("Counter value is not incremented correctly",
-                     expectedValue, counter.getValue());
+        assertEquals(expectedValue, counter.getValue(),
+            "Counter value is not incremented correctly");
       }
       expectedValue = rand.nextInt();
       counter.setValue(expectedValue);
-      assertEquals("Counter value is not set correctly",
-                   expectedValue, counter.getValue());
+      assertEquals(expectedValue, counter.getValue(),
+          "Counter value is not set correctly");
     }
   }
   
@@ -174,29 +175,28 @@ public class TestCounters {
 
   @SuppressWarnings("deprecation")
   private void checkLegacyNames(Counters counters) {
-    assertEquals("New name", 1, counters.findCounter(
-        TaskCounter.class.getName(), "MAP_INPUT_RECORDS").getValue());
-    assertEquals("Legacy name", 1, counters.findCounter(
+    assertEquals(1, counters.findCounter(
+        TaskCounter.class.getName(), "MAP_INPUT_RECORDS").getValue(), "New name");
+    assertEquals(1, counters.findCounter(
         "org.apache.hadoop.mapred.Task$Counter",
-        "MAP_INPUT_RECORDS").getValue());
-    assertEquals("Legacy enum", 1,
-        counters.findCounter(Task.Counter.MAP_INPUT_RECORDS).getValue());
+        "MAP_INPUT_RECORDS").getValue(), "Legacy name");
+    assertEquals(1, counters.findCounter(Task.Counter.MAP_INPUT_RECORDS).getValue(), "Legacy enum");
 
-    assertEquals("New name", 1, counters.findCounter(
-        JobCounter.class.getName(), "DATA_LOCAL_MAPS").getValue());
-    assertEquals("Legacy name", 1, counters.findCounter(
+    assertEquals(1, counters.findCounter(
+        JobCounter.class.getName(), "DATA_LOCAL_MAPS").getValue(), "New name");
+    assertEquals(1, counters.findCounter(
         "org.apache.hadoop.mapred.JobInProgress$Counter",
-        "DATA_LOCAL_MAPS").getValue());
-    assertEquals("Legacy enum", 1,
-        counters.findCounter(JobInProgress.Counter.DATA_LOCAL_MAPS).getValue());
+        "DATA_LOCAL_MAPS").getValue(), "Legacy name");
+    assertEquals(1,
+        counters.findCounter(JobInProgress.Counter.DATA_LOCAL_MAPS).getValue(), "Legacy enum");
 
-    assertEquals("New name", 1, counters.findCounter(
-        FileSystemCounter.class.getName(), "FILE_BYTES_READ").getValue());
-    assertEquals("New name and method", 1, counters.findCounter("file",
-        FileSystemCounter.BYTES_READ).getValue());
-    assertEquals("Legacy name", 1, counters.findCounter(
+    assertEquals(1, counters.findCounter(
+        FileSystemCounter.class.getName(), "FILE_BYTES_READ").getValue(), "New name");
+    assertEquals(1, counters.findCounter("file",
+        FileSystemCounter.BYTES_READ).getValue(), "New name and method");
+    assertEquals(1, counters.findCounter(
         "FileSystemCounters",
-        "FILE_BYTES_READ").getValue());
+        "FILE_BYTES_READ").getValue(), "Legacy name");
   }
   
   @SuppressWarnings("deprecation")
@@ -266,8 +266,8 @@ public class TestCounters {
     assertEquals("group1.counter1:1", counters.makeCompactString());
     counters.incrCounter("group2", "counter2", 3);
     String cs = counters.makeCompactString();
-    assertTrue("Bad compact string",
-        cs.equals(GC1 + ',' + GC2) || cs.equals(GC2 + ',' + GC1));
+    assertTrue(cs.equals(GC1 + ',' + GC2) || cs.equals(GC2 + ',' + GC1),
+        "Bad compact string");
   }
   
   @Test
@@ -321,7 +321,7 @@ public class TestCounters {
     } catch (CountersExceededException e) {
       return;
     }
-    Assert.fail("Should've thrown " + ecls.getSimpleName());
+    fail("Should've thrown " + ecls.getSimpleName());
   }
 
   public static void main(String[] args) throws IOException {
@@ -341,12 +341,12 @@ public class TestCounters {
   
     org.apache.hadoop.mapreduce.Counter count1 = 
         counterGroup.findCounter(JobCounter.NUM_FAILED_MAPS.toString());
-    Assert.assertNotNull(count1);
+    assertNotNull(count1);
     
     // Verify no exception get thrown when finding an unknown counter
     org.apache.hadoop.mapreduce.Counter count2 = 
         counterGroup.findCounter("Unknown");
-    Assert.assertNull(count2);
+    assertNull(count2);
   }
 
   @SuppressWarnings("rawtypes")
@@ -363,19 +363,19 @@ public class TestCounters {
     org.apache.hadoop.mapreduce.Counter count1 =
         counterGroup.findCounter(
             TaskCounter.PHYSICAL_MEMORY_BYTES.toString());
-    Assert.assertNotNull(count1);
+    assertNotNull(count1);
     count1.increment(10);
     count1.increment(10);
-    Assert.assertEquals(20, count1.getValue());
+    assertEquals(20, count1.getValue());
 
     // Verify no exception get thrown when finding an unknown counter
     org.apache.hadoop.mapreduce.Counter count2 =
         counterGroup.findCounter(
             TaskCounter.MAP_PHYSICAL_MEMORY_BYTES_MAX.toString());
-    Assert.assertNotNull(count2);
+    assertNotNull(count2);
     count2.increment(5);
     count2.increment(10);
-    Assert.assertEquals(10, count2.getValue());
+    assertEquals(10, count2.getValue());
   }
 
   @Test
@@ -385,12 +385,12 @@ public class TestCounters {
   
     org.apache.hadoop.mapreduce.Counter count1 = 
         fsGroup.findCounter("ANY_BYTES_READ");
-    Assert.assertNotNull(count1);
+    assertNotNull(count1);
     
     // Verify no exception get thrown when finding an unknown counter
     org.apache.hadoop.mapreduce.Counter count2 = 
         fsGroup.findCounter("Unknown");
-    Assert.assertNull(count2);
+    assertNull(count2);
   }
   
 }

+ 64 - 52
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestFileInputFormat.java

@@ -34,17 +34,19 @@ import org.apache.hadoop.fs.RawLocalFileSystem;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.util.Lists;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-@RunWith(value = Parameterized.class)
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertInstanceOf;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+
 public class TestFileInputFormat {
   
   private static final Logger LOG =
@@ -56,19 +58,18 @@ public class TestFileInputFormat {
   private static FileSystem localFs;
   
   private int numThreads;
-  
-  public TestFileInputFormat(int numThreads) {
-    this.numThreads = numThreads;
-    LOG.info("Running with numThreads: " + numThreads);
+
+  public void initTestFileInputFormat(int pNumThreads) {
+    this.numThreads = pNumThreads;
+    LOG.info("Running with numThreads: " + pNumThreads);
   }
-  
-  @Parameters
+
   public static Collection<Object[]> data() {
-    Object[][] data = new Object[][] { { 1 }, { 5 }};
+    Object[][] data = new Object[][]{{1}, {5}};
     return Arrays.asList(data);
   }
   
-  @Before
+  @BeforeEach
   public void setup() throws IOException {
     LOG.info("Using Test Dir: " + TEST_ROOT_DIR);
     localFs = FileSystem.getLocal(new Configuration());
@@ -76,13 +77,15 @@ public class TestFileInputFormat {
     localFs.mkdirs(TEST_ROOT_DIR);
   }
   
-  @After
+  @AfterEach
   public void cleanup() throws IOException {
     localFs.delete(TEST_ROOT_DIR, true);
   }
-  
-  @Test
-  public void testListLocatedStatus() throws Exception {
+
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testListLocatedStatus(int pNumThreads) throws Exception {
+    initTestFileInputFormat(pNumThreads);
     Configuration conf = getConfiguration();
     conf.setBoolean("fs.test.impl.disable.cache", false);
     conf.setInt(FileInputFormat.LIST_STATUS_NUM_THREADS, numThreads);
@@ -90,20 +93,21 @@ public class TestFileInputFormat {
         "test:///a1/a2");
     MockFileSystem mockFs =
         (MockFileSystem) new Path("test:///").getFileSystem(conf);
-    Assert.assertEquals("listLocatedStatus already called",
-        0, mockFs.numListLocatedStatusCalls);
+    assertEquals(0, mockFs.numListLocatedStatusCalls,
+        "listLocatedStatus already called");
     JobConf job = new JobConf(conf);
     TextInputFormat fileInputFormat = new TextInputFormat();
     fileInputFormat.configure(job);
     InputSplit[] splits = fileInputFormat.getSplits(job, 1);
-    Assert.assertEquals("Input splits are not correct", 2, splits.length);
-    Assert.assertEquals("listLocatedStatuss calls",
-        1, mockFs.numListLocatedStatusCalls);
+    assertEquals(2, splits.length, "Input splits are not correct");
+    assertEquals(1, mockFs.numListLocatedStatusCalls, "listLocatedStatus calls");
     FileSystem.closeAll();
   }
 
-  @Test
-  public void testIgnoreDirs() throws Exception {
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testIgnoreDirs(int pNumThreads) throws Exception {
+    initTestFileInputFormat(pNumThreads);
     Configuration conf = getConfiguration();
     conf.setBoolean(FileInputFormat.INPUT_DIR_NONRECURSIVE_IGNORE_SUBDIRS, true);
     conf.setInt(FileInputFormat.LIST_STATUS_NUM_THREADS, numThreads);
@@ -113,12 +117,14 @@ public class TestFileInputFormat {
     TextInputFormat fileInputFormat = new TextInputFormat();
     fileInputFormat.configure(job);
     InputSplit[] splits = fileInputFormat.getSplits(job, 1);
-    Assert.assertEquals("Input splits are not correct", 1, splits.length);
+    assertEquals(1, splits.length, "Input splits are not correct");
     FileSystem.closeAll();
   }
 
-  @Test
-  public void testSplitLocationInfo() throws Exception {
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testSplitLocationInfo(int pNumThreads) throws Exception {
+    initTestFileInputFormat(pNumThreads);
     Configuration conf = getConfiguration();
     conf.set(org.apache.hadoop.mapreduce.lib.input.FileInputFormat.INPUT_DIR,
         "test:///a1/a2");
@@ -127,21 +133,23 @@ public class TestFileInputFormat {
     fileInputFormat.configure(job);
     FileSplit[] splits = (FileSplit[]) fileInputFormat.getSplits(job, 1);
     String[] locations = splits[0].getLocations();
-    Assert.assertEquals(2, locations.length);
+    assertEquals(2, locations.length);
     SplitLocationInfo[] locationInfo = splits[0].getLocationInfo();
-    Assert.assertEquals(2, locationInfo.length);
+    assertEquals(2, locationInfo.length);
     SplitLocationInfo localhostInfo = locations[0].equals("localhost") ?
         locationInfo[0] : locationInfo[1];
     SplitLocationInfo otherhostInfo = locations[0].equals("otherhost") ?
         locationInfo[0] : locationInfo[1];
-    Assert.assertTrue(localhostInfo.isOnDisk());
-    Assert.assertTrue(localhostInfo.isInMemory());
-    Assert.assertTrue(otherhostInfo.isOnDisk());
-    Assert.assertFalse(otherhostInfo.isInMemory());
+    assertTrue(localhostInfo.isOnDisk());
+    assertTrue(localhostInfo.isInMemory());
+    assertTrue(otherhostInfo.isOnDisk());
+    assertFalse(otherhostInfo.isInMemory());
   }
-  
-  @Test
-  public void testListStatusSimple() throws IOException {
+
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testListStatusSimple(int pNumThreads) throws IOException {
+    initTestFileInputFormat(pNumThreads);
     Configuration conf = new Configuration();
     conf.setInt(FileInputFormat.LIST_STATUS_NUM_THREADS, numThreads);
 
@@ -154,12 +162,13 @@ public class TestFileInputFormat {
     FileStatus[] statuses = fif.listStatus(jobConf);
 
     org.apache.hadoop.mapreduce.lib.input.TestFileInputFormat
-        .verifyFileStatuses(expectedPaths, Lists.newArrayList(statuses),
-            localFs);
+        .verifyFileStatuses(expectedPaths, Lists.newArrayList(statuses), localFs);
   }
 
-  @Test
-  public void testListStatusNestedRecursive() throws IOException {
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testListStatusNestedRecursive(int pNumThreads) throws IOException {
+    initTestFileInputFormat(pNumThreads);
     Configuration conf = new Configuration();
     conf.setInt(FileInputFormat.LIST_STATUS_NUM_THREADS, numThreads);
 
@@ -175,8 +184,10 @@ public class TestFileInputFormat {
             localFs);
   }
 
-  @Test
-  public void testListStatusNestedNonRecursive() throws IOException {
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testListStatusNestedNonRecursive(int pNumThreads) throws IOException {
+    initTestFileInputFormat(pNumThreads);
     Configuration conf = new Configuration();
     conf.setInt(FileInputFormat.LIST_STATUS_NUM_THREADS, numThreads);
 
@@ -189,11 +200,13 @@ public class TestFileInputFormat {
 
     org.apache.hadoop.mapreduce.lib.input.TestFileInputFormat
         .verifyFileStatuses(expectedPaths, Lists.newArrayList(statuses),
-            localFs);
+        localFs);
   }
 
-  @Test
-  public void testListStatusErrorOnNonExistantDir() throws IOException {
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testListStatusErrorOnNonExistantDir(int pNumThreads) throws IOException {
+    initTestFileInputFormat(pNumThreads);
     Configuration conf = new Configuration();
     conf.setInt(FileInputFormat.LIST_STATUS_NUM_THREADS, numThreads);
 
@@ -204,13 +217,12 @@ public class TestFileInputFormat {
     fif.configure(jobConf);
     try {
       fif.listStatus(jobConf);
-      Assert.fail("Expecting an IOException for a missing Input path");
+      fail("Expecting an IOException for a missing Input path");
     } catch (IOException e) {
       Path expectedExceptionPath = new Path(TEST_ROOT_DIR, "input2");
       expectedExceptionPath = localFs.makeQualified(expectedExceptionPath);
-      Assert.assertTrue(e instanceof InvalidInputException);
-      Assert.assertEquals(
-          "Input path does not exist: " + expectedExceptionPath.toString(),
+      assertInstanceOf(InvalidInputException.class, e);
+      assertEquals("Input path does not exist: " + expectedExceptionPath.toString(),
           e.getMessage());
     }
   }

+ 23 - 20
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestFileOutputCommitter.java

@@ -24,10 +24,13 @@ import java.io.IOException;
 import java.net.URI;
 import java.nio.charset.StandardCharsets;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.*;
-import org.junit.Assert;
+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 org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -118,10 +121,10 @@ public class TestFileOutputCommitter {
     Path jobTempDir1 = committer.getCommittedTaskPath(tContext);
     File jtd1 = new File(jobTempDir1.toUri().getPath());
     if (commitVersion == 1) {
-      assertTrue("Version 1 commits to temporary dir " + jtd1, jtd1.exists());
+      assertTrue(jtd1.exists(), "Version 1 commits to temporary dir " + jtd1);
       validateContent(jobTempDir1);
     } else {
-      assertFalse("Version 2 commits to output dir " + jtd1, jtd1.exists());
+      assertFalse(jtd1.exists(), "Version 2 commits to output dir " + jtd1);
     }
 
     //now while running the second app attempt,
@@ -142,15 +145,15 @@ public class TestFileOutputCommitter {
     Path jobTempDir2 = committer2.getCommittedTaskPath(tContext2);
     File jtd2 = new File(jobTempDir2.toUri().getPath());
     if (recoveryVersion == 1) {
-      assertTrue("Version 1 recovers to " + jtd2, jtd2.exists());
+      assertTrue(jtd2.exists(), "Version 1 recovers to " + jtd2);
       validateContent(jobTempDir2);
     } else {
-        assertFalse("Version 2 commits to output dir " + jtd2, jtd2.exists());
-        if (commitVersion == 1) {
-          assertTrue("Version 2  recovery moves to output dir from "
-              + jtd1 , jtd1.list().length == 0);
-        }
+      assertFalse(jtd2.exists(), "Version 2 commits to output dir " + jtd2);
+      if (commitVersion == 1) {
+        assertEquals(0, jtd1.list().length,
+            "Version 2  recovery moves to output dir from " + jtd1);
       }
+    }
 
     committer2.commitJob(jContext2);
     validateContent(outDir);
@@ -253,12 +256,12 @@ public class TestFileOutputCommitter {
       committer.commitJob(jContext);
       // (1,1), (1,2), (2,1) shouldn't reach to here.
       if (version == 1 || maxAttempts <= 1) {
-        Assert.fail("Commit successful: wrong behavior for version 1.");
+        fail("Commit successful: wrong behavior for version 1.");
       }
     } catch (IOException e) {
       // (2,2) shouldn't reach to here.
       if (version == 2 && maxAttempts > 2) {
-        Assert.fail("Commit failed: wrong behavior for version 2.");
+        fail("Commit failed: wrong behavior for version 2.");
       }
     }
 
@@ -309,12 +312,12 @@ public class TestFileOutputCommitter {
     try {
       committer.commitJob(jContext);
       if (version == 1) {
-        Assert.fail("Duplicate commit successful: wrong behavior " +
+        fail("Duplicate commit successful: wrong behavior " +
             "for version 1.");
       }
     } catch (IOException e) {
       if (version == 2) {
-        Assert.fail("Duplicate commit failed: wrong behavior for version 2.");
+        fail("Duplicate commit failed: wrong behavior for version 2.");
       }
     }
     FileUtil.fullyDelete(new File(outDir.toString()));
@@ -464,12 +467,12 @@ public class TestFileOutputCommitter {
     Path workPath = committer.getWorkPath(tContext, outDir);
     File wp = new File(workPath.toUri().getPath());
     File expectedFile = new File(wp, partFile);
-    assertFalse("task temp dir still exists", expectedFile.exists());
+    assertFalse(expectedFile.exists(), "task temp dir still exists");
 
     committer.abortJob(jContext, JobStatus.State.FAILED);
     expectedFile = new File(out, FileOutputCommitter.TEMP_DIR_NAME);
-    assertFalse("job temp dir still exists", expectedFile.exists());
-    assertEquals("Output directory not empty", 0, out.listFiles().length);
+    assertFalse(expectedFile.exists(), "job temp dir still exists");
+    assertEquals(0, out.listFiles().length, "Output directory not empty");
     FileUtil.fullyDelete(out);
   }
 
@@ -541,7 +544,7 @@ public class TestFileOutputCommitter {
     assertNotNull(th);
     assertTrue(th instanceof IOException);
     assertTrue(th.getMessage().contains("fake delete failed"));
-    assertTrue(expectedFile + " does not exists", expectedFile.exists());
+    assertTrue(expectedFile.exists(), expectedFile + " does not exists");
 
     th = null;
     try {
@@ -552,7 +555,7 @@ public class TestFileOutputCommitter {
     assertNotNull(th);
     assertTrue(th instanceof IOException);
     assertTrue(th.getMessage().contains("fake delete failed"));
-    assertTrue("job temp dir does not exists", jobTmpDir.exists());
+    assertTrue(jobTmpDir.exists(), "job temp dir does not exists");
     FileUtil.fullyDelete(new File(outDir.toString()));
   }
 

+ 6 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestIndexCache.java

@@ -32,16 +32,18 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 
-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;
+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 TestIndexCache {
   private JobConf conf;
   private FileSystem fs;
   private Path p;
 
-  @Before
+  @BeforeEach
   public void setUp() throws IOException {
     conf = new JobConf();
     fs = FileSystem.getLocal(conf).getRaw();

+ 11 - 11
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobAclsManager.java

@@ -17,8 +17,8 @@
  */
 package org.apache.hadoop.mapred;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -28,7 +28,7 @@ import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 /**
  * Test the job acls manager
@@ -56,10 +56,10 @@ public class TestJobAclsManager {
     // cluster admin should have access
     boolean val = aclsManager.checkAccess(callerUGI, JobACL.VIEW_JOB, jobOwner,
         jobACLs.get(JobACL.VIEW_JOB));
-    assertTrue("cluster admin should have view access", val);
+    assertTrue(val, "cluster admin should have view access");
     val = aclsManager.checkAccess(callerUGI, JobACL.MODIFY_JOB, jobOwner,
         jobACLs.get(JobACL.MODIFY_JOB));
-    assertTrue("cluster admin should have modify access", val);
+    assertTrue(val, "cluster admin should have modify access");
   }
 
   @Test
@@ -80,20 +80,20 @@ public class TestJobAclsManager {
     // random user should not have access
     boolean val = aclsManager.checkAccess(callerUGI, JobACL.VIEW_JOB, jobOwner,
         jobACLs.get(JobACL.VIEW_JOB));
-    assertFalse("random user should not have view access", val);
+    assertFalse(val, "random user should not have view access");
     val = aclsManager.checkAccess(callerUGI, JobACL.MODIFY_JOB, jobOwner,
         jobACLs.get(JobACL.MODIFY_JOB));
-    assertFalse("random user should not have modify access", val);
+    assertFalse(val, "random user should not have modify access");
 
     callerUGI = UserGroupInformation.createUserForTesting(jobOwner,
         new String[] {});
     // Owner should have access
     val = aclsManager.checkAccess(callerUGI, JobACL.VIEW_JOB, jobOwner,
         jobACLs.get(JobACL.VIEW_JOB));
-    assertTrue("owner should have view access", val);
+    assertTrue(val, "owner should have view access");
     val = aclsManager.checkAccess(callerUGI, JobACL.MODIFY_JOB, jobOwner,
         jobACLs.get(JobACL.MODIFY_JOB));
-    assertTrue("owner should have modify access", val);
+    assertTrue(val, "owner should have modify access");
   }
 
   @Test
@@ -114,7 +114,7 @@ public class TestJobAclsManager {
     // acls off so anyone should have access
     boolean val = aclsManager.checkAccess(callerUGI, JobACL.VIEW_JOB, jobOwner,
         jobACLs.get(JobACL.VIEW_JOB));
-    assertTrue("acls off so anyone should have access", val);
+    assertTrue(val, "acls off so anyone should have access");
   }
 
   @Test
@@ -137,6 +137,6 @@ public class TestJobAclsManager {
     // acls off so anyone should have access
     boolean val = aclsManager.checkAccess(callerUGI, JobACL.VIEW_JOB, jobOwner,
         jobACLs.get(JobACL.VIEW_JOB));
-    assertTrue("user in admin group should have access", val);
+    assertTrue(val, "user in admin group should have access");
   }
 }

+ 48 - 43
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobConf.java

@@ -20,12 +20,16 @@ package org.apache.hadoop.mapred;
 
 import java.util.regex.Pattern;
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.*;
+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 org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.MRJobConfig;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 /**
  * test JobConf
@@ -37,7 +41,8 @@ public class TestJobConf {
    * test getters and setters of JobConf
    */
   @SuppressWarnings("deprecation")
-  @Test (timeout=5000)
+  @Test
+  @Timeout(value = 5)
   public void testJobConf() {
     JobConf conf = new JobConf();
     // test default value
@@ -153,10 +158,10 @@ public class TestJobConf {
     
     // make sure mapreduce.map|reduce.java.opts are not set by default
     // so that they won't override mapred.child.java.opts
-    assertNull("mapreduce.map.java.opts should not be set by default",
-        conf.get(JobConf.MAPRED_MAP_TASK_JAVA_OPTS));
-    assertNull("mapreduce.reduce.java.opts should not be set by default",
-        conf.get(JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS));
+    assertNull(conf.get(JobConf.MAPRED_MAP_TASK_JAVA_OPTS),
+        "mapreduce.map.java.opts should not be set by default");
+    assertNull(conf.get(JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS),
+        "mapreduce.reduce.java.opts should not be set by default");
   }
 
   /**
@@ -164,30 +169,31 @@ public class TestJobConf {
    * old property names
    */
   @SuppressWarnings("deprecation")
-  @Test (timeout = 10000)
+  @Test
+  @Timeout(value = 10)
   public void testDeprecatedPropertyNameForTaskVmem() {
     JobConf configuration = new JobConf();
 
     configuration.setLong(JobConf.MAPRED_JOB_MAP_MEMORY_MB_PROPERTY, 1024);
     configuration.setLong(JobConf.MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY, 1024);
-    Assert.assertEquals(1024, configuration.getMemoryForMapTask());
-    Assert.assertEquals(1024, configuration.getMemoryForReduceTask());
+    assertEquals(1024, configuration.getMemoryForMapTask());
+    assertEquals(1024, configuration.getMemoryForReduceTask());
     // Make sure new property names aren't broken by the old ones
     configuration.setLong(JobConf.MAPREDUCE_JOB_MAP_MEMORY_MB_PROPERTY, 1025);
     configuration.setLong(JobConf.MAPREDUCE_JOB_REDUCE_MEMORY_MB_PROPERTY, 1025);
-    Assert.assertEquals(1025, configuration.getMemoryForMapTask());
-    Assert.assertEquals(1025, configuration.getMemoryForReduceTask());
+    assertEquals(1025, configuration.getMemoryForMapTask());
+    assertEquals(1025, configuration.getMemoryForReduceTask());
 
     configuration.setMemoryForMapTask(2048);
     configuration.setMemoryForReduceTask(2048);
-    Assert.assertEquals(2048, configuration.getLong(
+    assertEquals(2048, configuration.getLong(
         JobConf.MAPRED_JOB_MAP_MEMORY_MB_PROPERTY, -1));
-    Assert.assertEquals(2048, configuration.getLong(
+    assertEquals(2048, configuration.getLong(
         JobConf.MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY, -1));
     // Make sure new property names aren't broken by the old ones
-    Assert.assertEquals(2048, configuration.getLong(
+    assertEquals(2048, configuration.getLong(
         JobConf.MAPREDUCE_JOB_MAP_MEMORY_MB_PROPERTY, -1));
-    Assert.assertEquals(2048, configuration.getLong(
+    assertEquals(2048, configuration.getLong(
         JobConf.MAPREDUCE_JOB_REDUCE_MEMORY_MB_PROPERTY, -1));
   }
 
@@ -196,9 +202,9 @@ public class TestJobConf {
   public void testProfileParamsDefaults() {
     JobConf configuration = new JobConf();
     String result = configuration.getProfileParams();
-    Assert.assertNotNull(result);
-    Assert.assertTrue(result.contains("file=%s"));
-    Assert.assertTrue(result.startsWith("-agentlib:hprof"));
+    assertNotNull(result);
+    assertTrue(result.contains("file=%s"));
+    assertTrue(result.startsWith("-agentlib:hprof"));
   }
 
   @Test
@@ -206,7 +212,7 @@ public class TestJobConf {
     JobConf configuration = new JobConf();
 
     configuration.setProfileParams("test");
-    Assert.assertEquals("test", configuration.get(MRJobConfig.TASK_PROFILE_PARAMS));
+    assertEquals("test", configuration.get(MRJobConfig.TASK_PROFILE_PARAMS));
   }
 
   @Test
@@ -214,7 +220,7 @@ public class TestJobConf {
     JobConf configuration = new JobConf();
 
     configuration.set(MRJobConfig.TASK_PROFILE_PARAMS, "test");
-    Assert.assertEquals("test", configuration.getProfileParams());
+    assertEquals("test", configuration.getProfileParams());
   }
 
   /**
@@ -275,15 +281,15 @@ public class TestJobConf {
     JobConf configuration = new JobConf();
 
     configuration.set(JobConf.MAPRED_TASK_MAXVMEM_PROPERTY, "-3");
-    Assert.assertEquals(MRJobConfig.DEFAULT_MAP_MEMORY_MB,
+    assertEquals(MRJobConfig.DEFAULT_MAP_MEMORY_MB,
         configuration.getMemoryForMapTask());
-    Assert.assertEquals(MRJobConfig.DEFAULT_REDUCE_MEMORY_MB,
+    assertEquals(MRJobConfig.DEFAULT_REDUCE_MEMORY_MB,
         configuration.getMemoryForReduceTask());
 
     configuration.set(MRJobConfig.MAP_MEMORY_MB, "4");
     configuration.set(MRJobConfig.REDUCE_MEMORY_MB, "5");
-    Assert.assertEquals(4, configuration.getMemoryForMapTask());
-    Assert.assertEquals(5, configuration.getMemoryForReduceTask());
+    assertEquals(4, configuration.getMemoryForMapTask());
+    assertEquals(5, configuration.getMemoryForReduceTask());
 
   }
 
@@ -296,9 +302,9 @@ public class TestJobConf {
 
     configuration.set(MRJobConfig.MAP_MEMORY_MB, "-5");
     configuration.set(MRJobConfig.REDUCE_MEMORY_MB, "-6");
-    Assert.assertEquals(MRJobConfig.DEFAULT_MAP_MEMORY_MB,
+    assertEquals(MRJobConfig.DEFAULT_MAP_MEMORY_MB,
         configuration.getMemoryForMapTask());
-    Assert.assertEquals(MRJobConfig.DEFAULT_REDUCE_MEMORY_MB,
+    assertEquals(MRJobConfig.DEFAULT_REDUCE_MEMORY_MB,
         configuration.getMemoryForReduceTask());
   }
 
@@ -325,14 +331,14 @@ public class TestJobConf {
     configuration = new JobConf();
     configuration.set(MRJobConfig.MAP_MEMORY_MB, String.valueOf(-1));
     configuration.set(MRJobConfig.REDUCE_MEMORY_MB, String.valueOf(-1));
-    configuration.set("mapred.task.maxvmem", String.valueOf(1 * 1024 * 1024));
+    configuration.set("mapred.task.maxvmem", String.valueOf(1024 * 1024));
     assertThat(configuration.getMaxVirtualMemoryForTask())
-        .isEqualTo(1 * 1024 * 1024);
+        .isEqualTo(1024 * 1024);
 
     configuration = new JobConf();
-    configuration.set("mapred.task.maxvmem", String.valueOf(1 * 1024 * 1024));
+    configuration.set("mapred.task.maxvmem", String.valueOf(1024 * 1024));
     assertThat(configuration.getMaxVirtualMemoryForTask())
-        .isEqualTo(1 * 1024 * 1024);
+        .isEqualTo(1024 * 1024);
 
     //set test case
 
@@ -357,11 +363,10 @@ public class TestJobConf {
   @Test
   public void testMaxTaskFailuresPerTracker() {
     JobConf jobConf = new JobConf(true);
-    Assert.assertTrue("By default JobContext.MAX_TASK_FAILURES_PER_TRACKER was "
-      + "not less than JobContext.MAP_MAX_ATTEMPTS and REDUCE_MAX_ATTEMPTS"
-      ,jobConf.getMaxTaskFailuresPerTracker() < jobConf.getMaxMapAttempts() &&
-      jobConf.getMaxTaskFailuresPerTracker() < jobConf.getMaxReduceAttempts()
-      );
+    assertTrue(jobConf.getMaxTaskFailuresPerTracker() < jobConf.getMaxMapAttempts() &&
+        jobConf.getMaxTaskFailuresPerTracker() < jobConf.getMaxReduceAttempts(),
+        "By default JobContext.MAX_TASK_FAILURES_PER_TRACKER was "
+        + "not less than JobContext.MAP_MAX_ATTEMPTS and REDUCE_MAX_ATTEMPTS");
   }
 
   /**
@@ -370,14 +375,14 @@ public class TestJobConf {
   @Test
   public void testParseMaximumHeapSizeMB() {
     // happy cases
-    Assert.assertEquals(4096, JobConf.parseMaximumHeapSizeMB("-Xmx4294967296"));
-    Assert.assertEquals(4096, JobConf.parseMaximumHeapSizeMB("-Xmx4194304k"));
-    Assert.assertEquals(4096, JobConf.parseMaximumHeapSizeMB("-Xmx4096m"));
-    Assert.assertEquals(4096, JobConf.parseMaximumHeapSizeMB("-Xmx4g"));
+    assertEquals(4096, JobConf.parseMaximumHeapSizeMB("-Xmx4294967296"));
+    assertEquals(4096, JobConf.parseMaximumHeapSizeMB("-Xmx4194304k"));
+    assertEquals(4096, JobConf.parseMaximumHeapSizeMB("-Xmx4096m"));
+    assertEquals(4096, JobConf.parseMaximumHeapSizeMB("-Xmx4g"));
 
     // sad cases
-    Assert.assertEquals(-1, JobConf.parseMaximumHeapSizeMB("-Xmx4?"));
-    Assert.assertEquals(-1, JobConf.parseMaximumHeapSizeMB(""));
+    assertEquals(-1, JobConf.parseMaximumHeapSizeMB("-Xmx4?"));
+    assertEquals(-1, JobConf.parseMaximumHeapSizeMB(""));
   }
 
   /**

+ 11 - 10
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobEndNotifier.java

@@ -31,13 +31,14 @@ import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
-import org.junit.After;
-import org.junit.Before;
-import static org.junit.Assert.*;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.http.HttpServer2;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 public class TestJobEndNotifier {
   HttpServer2 server;
@@ -49,7 +50,7 @@ public class TestJobEndNotifier {
     public static URI requestUri;
 
     @Override
-    public void doGet(HttpServletRequest request, 
+    public void doGet(HttpServletRequest request,
                       HttpServletResponse response
                       ) throws ServletException, IOException {
       InputStreamReader in = new InputStreamReader(request.getInputStream());
@@ -73,7 +74,7 @@ public class TestJobEndNotifier {
     public static volatile int calledTimes = 0;
 
     @Override
-    public void doGet(HttpServletRequest request, 
+    public void doGet(HttpServletRequest request,
                       HttpServletResponse response
                       ) throws ServletException, IOException {
       boolean timedOut = false;
@@ -84,7 +85,7 @@ public class TestJobEndNotifier {
       } catch (InterruptedException e) {
         timedOut = true;
       }
-      assertTrue("DelayServlet should be interrupted", timedOut);
+      assertTrue(timedOut, "DelayServlet should be interrupted");
     }
   }
 
@@ -94,7 +95,7 @@ public class TestJobEndNotifier {
     public static volatile int calledTimes = 0;
 
     @Override
-    public void doGet(HttpServletRequest request, 
+    public void doGet(HttpServletRequest request,
                       HttpServletResponse response
                       ) throws ServletException, IOException {
       calledTimes++;
@@ -102,7 +103,7 @@ public class TestJobEndNotifier {
     }
   }
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     new File(System.getProperty("build.webapps", "build/webapps") + "/test"
         ).mkdirs();
@@ -122,7 +123,7 @@ public class TestJobEndNotifier {
     FailServlet.calledTimes = 0;
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     server.stop();
   }

+ 9 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobInfo.java

@@ -30,18 +30,20 @@ import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.JobStatus.State;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 /**
  * test class JobInfo
- * 
- * 
+ *
+ *
  */
 public class TestJobInfo {
-  @Test (timeout=5000)
+  @Test
+  @Timeout(value = 5)
   public void testJobInfo() throws IOException {
     JobID jid = new JobID("001", 1);
     Text user = new Text("User");
@@ -60,7 +62,8 @@ public class TestJobInfo {
 
   }
   
-  @Test(timeout = 5000)
+  @Test
+  @Timeout(value = 5)
   public void testTaskID() throws IOException, InterruptedException {
     JobID jobid = new JobID("1014873536921", 6);
     TaskID tid = new TaskID(jobid, TaskType.MAP, 0);

+ 4 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobQueueClient.java

@@ -23,10 +23,9 @@ import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.PrintWriter;
 
-import org.junit.Assert;
-
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class TestJobQueueClient {
   /**
@@ -47,8 +46,8 @@ public class TestJobQueueClient {
     PrintWriter writer = new PrintWriter(bbos);
     queueClient.printJobQueueInfo(parent, writer);
 
-    Assert.assertTrue("printJobQueueInfo did not print grandchild's name",
-      bbos.toString().contains("GrandChildQueue"));
+    assertTrue(bbos.toString().contains("GrandChildQueue"),
+        "printJobQueueInfo did not print grandchild's name");
   }
 
 }

+ 65 - 66
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestLineRecordReader.java

@@ -19,10 +19,11 @@
 package org.apache.hadoop.mapred;
 
 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.assertTrue;
+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.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.File;
 import java.io.FileInputStream;
@@ -44,7 +45,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.compress.BZip2Codec;
 import org.apache.hadoop.io.compress.CodecPool;
 import org.apache.hadoop.io.compress.Decompressor;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 public class TestLineRecordReader {
   private static Path workDir = new Path(new Path(System.getProperty(
@@ -54,7 +55,7 @@ public class TestLineRecordReader {
   private void testSplitRecords(String testFileName, long firstSplitLength)
       throws IOException {
     URL testFileUrl = getClass().getClassLoader().getResource(testFileName);
-    assertNotNull("Cannot find " + testFileName, testFileUrl);
+    assertNotNull(testFileUrl, "Cannot find " + testFileName);
     File testFile = new File(testFileUrl.getFile());
     long testFileSize = testFile.length();
     Path testFilePath = new Path(testFile.getAbsolutePath());
@@ -67,8 +68,8 @@ public class TestLineRecordReader {
       throws IOException {
     conf.setInt(org.apache.hadoop.mapreduce.lib.input.
         LineRecordReader.MAX_LINE_LENGTH, Integer.MAX_VALUE);
-    assertTrue("unexpected test data at " + testFilePath,
-        testFileSize > firstSplitLength);
+    assertTrue(testFileSize > firstSplitLength,
+        "unexpected test data at " + testFilePath);
 
     String delimiter = conf.get("textinputformat.record.delimiter");
     byte[] recordDelimiterBytes = null;
@@ -107,8 +108,8 @@ public class TestLineRecordReader {
     }
     reader.close();
 
-    assertEquals("Unexpected number of records in split",
-        numRecordsNoSplits, numRecordsFirstSplit + numRecordsRemainingSplits);
+    assertEquals(numRecordsNoSplits, numRecordsFirstSplit + numRecordsRemainingSplits,
+        "Unexpected number of records in split");
   }
 
   private void testLargeSplitRecordForFile(Configuration conf,
@@ -116,8 +117,8 @@ public class TestLineRecordReader {
       throws IOException {
     conf.setInt(org.apache.hadoop.mapreduce.lib.input.
         LineRecordReader.MAX_LINE_LENGTH, Integer.MAX_VALUE);
-    assertTrue("unexpected firstSplitLength:" + firstSplitLength,
-        testFileSize < firstSplitLength);
+    assertTrue(testFileSize < firstSplitLength, "unexpected firstSplitLength:" +
+        firstSplitLength);
     String delimiter = conf.get("textinputformat.record.delimiter");
     byte[] recordDelimiterBytes = null;
     if (null != delimiter) {
@@ -144,8 +145,8 @@ public class TestLineRecordReader {
       ++numRecordsFirstSplit;
     }
     reader.close();
-    assertEquals("Unexpected number of records in split",
-        numRecordsNoSplits, numRecordsFirstSplit);
+    assertEquals(numRecordsNoSplits, numRecordsFirstSplit,
+        "Unexpected number of records in split");
   }
 
   @Test
@@ -185,11 +186,13 @@ public class TestLineRecordReader {
     testSplitRecords("blockEndingInCR.txt.bz2", 136494);
   }
 
-  @Test(expected=IOException.class)
-  public void testSafeguardSplittingUnsplittableFiles() throws IOException {
+  @Test
+  public void testSafeguardSplittingUnSplittableFiles() throws IOException {
     // The LineRecordReader must fail when trying to read a file that
     // was compressed using an unsplittable file format
-    testSplitRecords("TestSafeguardSplittingUnsplittableFiles.txt.gz", 2);
+    assertThrows(IOException.class, () -> {
+      testSplitRecords("TestSafeguardSplittingUnsplittableFiles.txt.gz", 2);
+    });
   }
 
   // Use the LineRecordReader to read records from the file
@@ -237,7 +240,7 @@ public class TestLineRecordReader {
       count = fis.read(data);
     }
     fis.close();
-    assertTrue("Test file data too big for buffer", count < data.length);
+    assertTrue(count < data.length, "Test file data too big for buffer");
     return new String(data, 0, count, StandardCharsets.UTF_8).split("\n");
   }
 
@@ -249,7 +252,7 @@ public class TestLineRecordReader {
     ArrayList<String> records = readRecords(testFileUrl, splitSize);
     String[] actuals = readRecordsDirectly(testFileUrl, bzip);
 
-    assertEquals("Wrong number of records", actuals.length, records.size());
+    assertEquals(actuals.length, records.size(), "Wrong number of records");
 
     boolean hasLargeRecord = false;
     for (int i = 0; i < actuals.length; ++i) {
@@ -259,8 +262,8 @@ public class TestLineRecordReader {
       }
     }
 
-    assertTrue("Invalid test data. Doesn't have a large enough record",
-               hasLargeRecord);
+    assertTrue(hasLargeRecord,
+        "Invalid test data. Doesn't have a large enough record");
   }
 
   @Test
@@ -286,7 +289,7 @@ public class TestLineRecordReader {
     // confirm the BOM is skipped by LineRecordReader
     String UTF8_BOM = "\uFEFF";
     URL testFileUrl = getClass().getClassLoader().getResource("testBOM.txt");
-    assertNotNull("Cannot find testBOM.txt", testFileUrl);
+    assertNotNull(testFileUrl, "Cannot find testBOM.txt");
     File testFile = new File(testFileUrl.getFile());
     Path testFilePath = new Path(testFile.getAbsolutePath());
     long testFileSize = testFile.length();
@@ -314,15 +317,15 @@ public class TestLineRecordReader {
     }
     reader.close();
 
-    assertTrue("BOM is not skipped", skipBOM);
+    assertTrue(skipBOM, "BOM is not skipped");
   }
 
   @Test
   public void testMultipleClose() throws IOException {
     URL testFileUrl = getClass().getClassLoader().
         getResource("recordSpanningMultipleSplits.txt.bz2");
-    assertNotNull("Cannot find recordSpanningMultipleSplits.txt.bz2",
-        testFileUrl);
+    assertNotNull(testFileUrl,
+        "Cannot find recordSpanningMultipleSplits.txt.bz2");
     File testFile = new File(testFileUrl.getFile());
     Path testFilePath = new Path(testFile.getAbsolutePath());
     long testFileSize = testFile.length();
@@ -493,22 +496,22 @@ public class TestLineRecordReader {
     LongWritable key = new LongWritable();
     Text value = new Text();
     // Get first record: "abcdefghij"
-    assertTrue("Expected record got nothing", reader.next(key, value));
-    assertEquals("Wrong length for record value", 10, value.getLength());
+    assertTrue(reader.next(key, value), "Expected record got nothing");
+    assertEquals(10, value.getLength(), "Wrong length for record value");
     // Position should be 12 right after "abcdefghij++"
-    assertEquals("Wrong position after record read", 12, reader.getPos());
+    assertEquals(12, reader.getPos(), "Wrong position after record read");
     // Get second record: "kl"
-    assertTrue("Expected record got nothing", reader.next(key, value));
-    assertEquals("Wrong length for record value", 2, value.getLength());
+    assertTrue(reader.next(key, value), "Expected record got nothing");
+    assertEquals(2, value.getLength(), "Wrong length for record value");
     // Position should be 16 right after "abcdefghij++kl++"
-    assertEquals("Wrong position after record read", 16, reader.getPos());
+    assertEquals(16, reader.getPos(), "Wrong position after record read");
     // Get third record: "mno"
-    assertTrue("Expected record got nothing", reader.next(key, value));
-    assertEquals("Wrong length for record value", 3, value.getLength());
+    assertTrue(reader.next(key, value), "Expected record got nothing");
+    assertEquals(3, value.getLength(), "Wrong length for record value");
     // Position should be 19 right after "abcdefghij++kl++mno"
-    assertEquals("Wrong position after record read", 19, reader.getPos());
+    assertEquals(19, reader.getPos(), "Wrong position after record read");
     assertFalse(reader.next(key, value));
-    assertEquals("Wrong position after record read", 19, reader.getPos());
+    assertEquals(19, reader.getPos(), "Wrong position after record read");
     reader.close();
     // No record is in the second split because the second split will drop
     // the first record, which was already reported by the first split.
@@ -517,9 +520,9 @@ public class TestLineRecordReader {
     reader = new LineRecordReader(conf, split, recordDelimiterBytes);
     // The position should be 19 right after "abcdefghij++kl++mno" and should
     // not change
-    assertEquals("Wrong position after record read", 19, reader.getPos());
-    assertFalse("Unexpected record returned", reader.next(key, value));
-    assertEquals("Wrong position after record read", 19, reader.getPos());
+    assertEquals(19, reader.getPos(), "Wrong position after record read");
+    assertFalse(reader.next(key, value), "Unexpected record returned");
+    assertEquals(19, reader.getPos(), "Wrong position after record read");
     reader.close();
 
     // multi char delimiter with starting part of the delimiter in the data
@@ -529,28 +532,28 @@ public class TestLineRecordReader {
     split = new FileSplit(inputFile, 0, splitLength, (String[]) null);
     reader = new LineRecordReader(conf, split, recordDelimiterBytes);
     // Get first record: "abcd+efgh"
-    assertTrue("Expected record got nothing", reader.next(key, value));
-    assertEquals("Wrong position after record read", 11, reader.getPos());
-    assertEquals("Wrong length for record value", 9, value.getLength());
+    assertTrue(reader.next(key, value), "Expected record got nothing");
+    assertEquals(11, reader.getPos(), "Wrong position after record read");
+    assertEquals(9, value.getLength(), "Wrong length for record value");
     // should have jumped over the delimiter, no record
-    assertFalse("Unexpected record returned", reader.next(key, value));
-    assertEquals("Wrong position after record read", 11, reader.getPos());
+    assertFalse(reader.next(key, value), "Unexpected record returned");
+    assertEquals(11, reader.getPos(), "Wrong position after record read");
     reader.close();
     // next split: check for duplicate or dropped records
     split = new FileSplit(inputFile, splitLength,
         inputData.length() - splitLength, (String[]) null);
     reader = new LineRecordReader(conf, split, recordDelimiterBytes);
     // Get second record: "ijk" first in this split
-    assertTrue("Expected record got nothing", reader.next(key, value));
-    assertEquals("Wrong position after record read", 16, reader.getPos());
-    assertEquals("Wrong length for record value", 3, value.getLength());
+    assertTrue(reader.next(key, value), "Expected record got nothing");
+    assertEquals(16, reader.getPos(), "Wrong position after record read");
+    assertEquals(3, value.getLength(), "Wrong length for record value");
     // Get third record: "mno" second in this split
-    assertTrue("Expected record got nothing", reader.next(key, value));
-    assertEquals("Wrong position after record read", 19, reader.getPos());
-    assertEquals("Wrong length for record value", 3, value.getLength());
+    assertTrue(reader.next(key, value), "Expected record got nothing");
+    assertEquals(19, reader.getPos(), "Wrong position after record read");
+    assertEquals(3, value.getLength(), "Wrong length for record value");
     // should be at the end of the input
     assertFalse(reader.next(key, value));
-    assertEquals("Wrong position after record read", 19, reader.getPos());
+    assertEquals(19, reader.getPos(), "Wrong position after record read");
     reader.close();
 
     inputData = "abcd|efgh|+|ij|kl|+|mno|pqr";
@@ -565,21 +568,19 @@ public class TestLineRecordReader {
         split = new FileSplit(inputFile, 0, bufferSize, (String[]) null);
         reader = new LineRecordReader(conf, split, recordDelimiterBytes);
         // Get first record: "abcd|efgh" always possible
-        assertTrue("Expected record got nothing", reader.next(key, value));
+        assertTrue(reader.next(key, value), "Expected record got nothing");
         assertThat(value.toString()).isEqualTo("abcd|efgh");
-        assertEquals("Wrong position after record read", 9, value.getLength());
+        assertEquals(9, value.getLength(), "Wrong position after record read");
         // Position should be 12 right after "|+|"
         int recordPos = 12;
-        assertEquals("Wrong position after record read", recordPos,
-            reader.getPos());
+        assertEquals(recordPos, reader.getPos(), "Wrong position after record read");
         // get the next record: "ij|kl" if the split/buffer allows it
         if (reader.next(key, value)) {
           // check the record info: "ij|kl"
           assertThat(value.toString()).isEqualTo("ij|kl");
           // Position should be 20 right after "|+|"
           recordPos = 20;
-          assertEquals("Wrong position after record read", recordPos,
-              reader.getPos());
+          assertEquals(recordPos, reader.getPos(), "Wrong position after record read");
         }
         // get the third record: "mno|pqr" if the split/buffer allows it
         if (reader.next(key, value)) {
@@ -587,13 +588,11 @@ public class TestLineRecordReader {
           assertThat(value.toString()).isEqualTo("mno|pqr");
           // Position should be 27 at the end of the string now
           recordPos = inputData.length();
-          assertEquals("Wrong position after record read", recordPos,
-              reader.getPos());
+          assertEquals(recordPos, reader.getPos(), "Wrong position after record read");
         }
         // no more records can be read we should still be at the last position
-        assertFalse("Unexpected record returned", reader.next(key, value));
-        assertEquals("Wrong position after record read", recordPos,
-            reader.getPos());
+        assertFalse(reader.next(key, value), "Unexpected record returned");
+        assertEquals(recordPos, reader.getPos(), "Wrong position after record read");
         reader.close();
       }
     }
@@ -666,14 +665,14 @@ public class TestLineRecordReader {
     // (833 bytes is the last block start in the used data file)
     int firstSplitLength = 100;
     URL testFileUrl = getClass().getClassLoader().getResource(testFileName);
-    assertNotNull("Cannot find " + testFileName, testFileUrl);
+    assertNotNull(testFileUrl, "Cannot find " + testFileName);
     File testFile = new File(testFileUrl.getFile());
     long testFileSize = testFile.length();
     Path testFilePath = new Path(testFile.getAbsolutePath());
-    assertTrue("Split size is smaller than header length",
-        firstSplitLength > 9);
-    assertTrue("Split size is larger than compressed file size " +
-        testFilePath, testFileSize > firstSplitLength);
+    assertTrue(firstSplitLength > 9, "Split size is smaller than header length");
+    assertTrue(testFileSize > firstSplitLength,
+        "Split size is larger than compressed file size " +
+        testFilePath);
 
     Configuration conf = new Configuration();
     conf.setInt(org.apache.hadoop.mapreduce.lib.input.

+ 10 - 9
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestLocatedFileStatusFetcher.java

@@ -22,10 +22,9 @@ import java.io.File;
 import java.io.IOException;
 import java.util.concurrent.CountDownLatch;
 
-import org.junit.After;
-import org.junit.Assert;
-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.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -37,6 +36,8 @@ import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.test.AbstractHadoopTestBase;
 import org.apache.hadoop.test.GenericTestUtils;
 
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
 /**
  *  Test that the executor service has been shut down
  *  when the LocatedFileStatusFetcher is interrupted.
@@ -49,14 +50,14 @@ public class TestLocatedFileStatusFetcher extends AbstractHadoopTestBase {
   private File dir = GenericTestUtils.getTestDir("test-lfs-fetcher");
   private static final CountDownLatch LATCH = new CountDownLatch(1);
 
-  @Before
+  @BeforeEach
   public void setup() throws Exception {
     conf = new Configuration(false);
     conf.set("fs.file.impl", MockFileSystem.class.getName());
     fileSys = FileSystem.getLocal(conf);
   }
 
-  @After
+  @AfterEach
   public void after() {
     if (mkdirs) {
       FileUtil.fullyDelete(dir);
@@ -83,7 +84,7 @@ public class TestLocatedFileStatusFetcher extends AbstractHadoopTestBase {
           fetcher.getFileStatuses();
         } catch (Exception e) {
           // This should interrupt condition.await()
-          Assert.assertTrue(e instanceof InterruptedException);
+          assertTrue(e instanceof InterruptedException);
         }
       }
     };
@@ -94,8 +95,8 @@ public class TestLocatedFileStatusFetcher extends AbstractHadoopTestBase {
     t.interrupt();
     t.join();
     // Check the status for executor service
-    Assert.assertTrue("The executor service should have been shut down",
-        fetcher.getListeningExecutorService().isShutdown());
+    assertTrue(fetcher.getListeningExecutorService().isShutdown(),
+        "The executor service should have been shut down");
   }
 
   static class MockFileSystem extends LocalFileSystem {

+ 5 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestMapFileOutputFormat.java

@@ -18,14 +18,14 @@
 
 package org.apache.hadoop.mapred;
 
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertFalse;
 
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.MapFile.Reader;
-import org.junit.After;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
 import org.mockito.Mockito;
 
 public class TestMapFileOutputFormat {
@@ -38,10 +38,10 @@ public class TestMapFileOutputFormat {
     Reader reader = Mockito.mock(Reader.class);
     Reader[] readers = new Reader[]{reader};
     outputFormat.getEntry(readers, new MyPartitioner(), new Text(), new Text());
-    assertTrue(!MyPartitioner.isGetPartitionCalled());
+    assertFalse(MyPartitioner.isGetPartitionCalled());
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     MyPartitioner.setGetPartitionCalled(false);
   }

+ 33 - 38
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestMapTask.java

@@ -31,18 +31,18 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.TaskCounter;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.util.Progress;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 import java.io.File;
 import java.io.IOException;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
@@ -53,21 +53,18 @@ public class TestMapTask {
           System.getProperty("java.io.tmpdir", "/tmp")),
       TestMapTask.class.getName());
 
-  @Before
+  @BeforeEach
   public void setup() throws Exception {
     if(!testRootDir.exists()) {
       testRootDir.mkdirs();
     }
   }
 
-  @After
+  @AfterEach
   public void cleanup() throws Exception {
     FileUtil.fullyDelete(testRootDir);
   }
 
-  @Rule
-  public ExpectedException exception = ExpectedException.none();
-
   // Verify output files for shuffle have group read permission even when
   // the configured umask normally would prevent it.
   @Test
@@ -94,38 +91,36 @@ public class TestMapTask {
     Path outputFile = mof.getOutputFile();
     FileSystem lfs = FileSystem.getLocal(conf);
     FsPermission perms = lfs.getFileStatus(outputFile).getPermission();
-    Assert.assertEquals("Incorrect output file perms",
-        (short)0640, perms.toShort());
+    assertEquals((short) 0640, perms.toShort(), "Incorrect output file perms");
     Path indexFile = mof.getOutputIndexFile();
     perms = lfs.getFileStatus(indexFile).getPermission();
-    Assert.assertEquals("Incorrect index file perms",
-        (short)0640, perms.toShort());
+    assertEquals((short) 0640, perms.toShort(), "Incorrect index file perms");
   }
 
   @Test
   public void testSpillFilesCountLimitInvalidValue() throws Exception {
-    JobConf conf = new JobConf();
-    conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "077");
-    conf.set(MRConfig.LOCAL_DIR, testRootDir.getAbsolutePath());
-    conf.setInt(MRJobConfig.SPILL_FILES_COUNT_LIMIT, -2);
-    MapOutputFile mof = new MROutputFiles();
-    mof.setConf(conf);
-    TaskAttemptID attemptId = new TaskAttemptID("12345", 1, TaskType.MAP, 1, 1);
-    MapTask mockTask = mock(MapTask.class);
-    doReturn(mof).when(mockTask).getMapOutputFile();
-    doReturn(attemptId).when(mockTask).getTaskID();
-    doReturn(new Progress()).when(mockTask).getSortPhase();
-    TaskReporter mockReporter = mock(TaskReporter.class);
-    doReturn(new Counter()).when(mockReporter).getCounter(any(TaskCounter.class));
-    MapOutputCollector.Context ctx = new MapOutputCollector.Context(mockTask, conf, mockReporter);
-    MapOutputBuffer<Object, Object> mob = new MapOutputBuffer<>();
-
-    exception.expect(IOException.class);
-    exception.expectMessage("Invalid value for \"mapreduce.task.spill.files.count.limit\", " +
-        "current value: -2");
-
-    mob.init(ctx);
-    mob.close();
+    String message = "Invalid value for \"mapreduce.task.spill.files.count.limit\", " +
+        "current value: -2";
+    assertThrows(IOException.class, () -> {
+      JobConf conf = new JobConf();
+      conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "077");
+      conf.set(MRConfig.LOCAL_DIR, testRootDir.getAbsolutePath());
+      conf.setInt(MRJobConfig.SPILL_FILES_COUNT_LIMIT, -2);
+      MapOutputFile mof = new MROutputFiles();
+      mof.setConf(conf);
+      TaskAttemptID attemptId = new TaskAttemptID("12345", 1, TaskType.MAP, 1, 1);
+      MapTask mockTask = mock(MapTask.class);
+      doReturn(mof).when(mockTask).getMapOutputFile();
+      doReturn(attemptId).when(mockTask).getTaskID();
+      doReturn(new Progress()).when(mockTask).getSortPhase();
+      TaskReporter mockReporter = mock(TaskReporter.class);
+      doReturn(new Counter()).when(mockReporter).getCounter(any(TaskCounter.class));
+      MapOutputCollector.Context ctx = new MapOutputCollector.Context(mockTask, conf, mockReporter);
+      MapOutputBuffer<Object, Object> mob = new MapOutputBuffer<>();
+
+      mob.init(ctx);
+      mob.close();
+    }, message);
   }
 
   @Test
@@ -168,6 +163,6 @@ public class TestMapTask {
 
     mob.close();
 
-    Assert.assertTrue(gotExceptionWithMessage);
+    assertTrue(gotExceptionWithMessage);
   }
 }

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestMaster.java

@@ -22,8 +22,8 @@ import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.fail;
-import org.junit.Test;
+import static org.junit.jupiter.api.Assertions.fail;
+import org.junit.jupiter.api.Test;
 
 public class TestMaster {
 

+ 20 - 12
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestOldMethodsJobID.java

@@ -26,10 +26,12 @@ import java.io.IOException;
 
 import org.apache.hadoop.mapred.TaskCompletionEvent.Status;
 import org.apache.hadoop.mapreduce.TaskType;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
 
 /**
  * Test deprecated methods
@@ -39,11 +41,12 @@ public class TestOldMethodsJobID {
 
   /**
    * test deprecated methods of TaskID
-   * @throws IOException
+   * @throws IOException In the process of unit testing, an IO exception occurred.
    */
   @SuppressWarnings("deprecation")
-  @Test (timeout=5000)
-  public void testDepricatedMethods() throws IOException {
+  @Test
+  @Timeout(value = 5)
+  public void testDeprecatedMethods() throws IOException {
     JobID jid = new JobID();
     TaskID test = new TaskID(jid, true, 1);
     assertThat(test.getTaskType()).isEqualTo(TaskType.MAP);
@@ -71,10 +74,11 @@ public class TestOldMethodsJobID {
   
   /**
    * test JobID
-   * @throws IOException 
+   * @throws IOException
    */
   @SuppressWarnings("deprecation")
-  @Test (timeout=5000)
+  @Test
+  @Timeout(value = 5)
   public void testJobID() throws IOException{
     JobID jid = new JobID("001",2);
     ByteArrayOutputStream out = new ByteArrayOutputStream();
@@ -86,7 +90,8 @@ public class TestOldMethodsJobID {
    * test deprecated methods of TaskCompletionEvent
    */
   @SuppressWarnings("deprecation")
-  @Test (timeout=5000)
+  @Test
+  @Timeout(value = 5)
   public void testTaskCompletionEvent() {
     TaskAttemptID taid = new TaskAttemptID("001", 1, TaskType.REDUCE, 2, 3);
     TaskCompletionEvent template = new TaskCompletionEvent(12, taid, 13, true,
@@ -113,7 +118,8 @@ public class TestOldMethodsJobID {
    * @throws IOException
    */
   @SuppressWarnings("deprecation")
-  @Test (timeout=5000)
+  @Test
+  @Timeout(value = 5)
   public void testJobProfile() throws IOException {
 
     JobProfile profile = new JobProfile("user", "job_001_03", "jobFile", "uri",
@@ -138,7 +144,8 @@ public class TestOldMethodsJobID {
    * test TaskAttemptID 
    */
   @SuppressWarnings( "deprecation" )
-  @Test (timeout=5000)
+  @Test
+  @Timeout(value = 5)
   public void testTaskAttemptID (){
     TaskAttemptID task  = new TaskAttemptID("001",2,true,3,4);
     assertEquals("attempt_001_0002_m_000003_4", TaskAttemptID.getTaskAttemptIDsPattern("001", 2, true, 3, 4));
@@ -150,10 +157,11 @@ public class TestOldMethodsJobID {
   
   /**
    * test Reporter.NULL
-   * 
+   *
    */
   
-  @Test (timeout=5000)
+  @Test
+  @Timeout(value = 5)
   public void testReporter(){
     Reporter nullReporter=Reporter.NULL;
     assertNull(nullReporter.getCounter(null));

+ 18 - 10
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestQueue.java

@@ -31,13 +31,18 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.security.UserGroupInformation;
-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.junit.jupiter.api.Timeout;
 
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
+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.mock;
+import static org.mockito.Mockito.when;
 
 /**
  * TestCounters checks the sanity and recoverability of Queue
@@ -46,12 +51,12 @@ public class TestQueue {
   private static File testDir = new File(System.getProperty("test.build.data",
       "/tmp"), TestJobConf.class.getSimpleName());
 
-  @Before
+  @BeforeEach
   public void setup() {
     testDir.mkdirs();
   }
 
-  @After
+  @AfterEach
   public void cleanup() {
     FileUtil.fullyDelete(testDir);
   }
@@ -62,7 +67,8 @@ public class TestQueue {
    * 
    * @throws IOException
    */
-  @Test (timeout=5000)
+  @Test
+  @Timeout(value = 5)
   public void testQueue() throws IOException {
     File f = null;
     try {
@@ -194,7 +200,8 @@ public class TestQueue {
     return conf;
   }
 
-  @Test (timeout=5000)
+  @Test
+  @Timeout(value = 5)
   public void testDefaultConfig() {
     QueueManager manager = new QueueManager(true);
     assertThat(manager.getRoot().getChildren().size()).isEqualTo(2);
@@ -206,7 +213,8 @@ public class TestQueue {
    * @throws IOException
    */
 
-  @Test (timeout=5000)
+  @Test
+  @Timeout(value = 5)
   public void test2Queue() throws IOException {
     Configuration conf = getConfiguration();
 

+ 9 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestSkipBadRecords.java

@@ -19,8 +19,13 @@ package org.apache.hadoop.mapred;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.junit.Test;
-import static org.junit.Assert.*;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+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;
 
 /**
  * test SkipBadRecords
@@ -28,7 +33,8 @@ import static org.junit.Assert.*;
  * 
  */
 public class TestSkipBadRecords {
-  @Test (timeout=5000)
+  @Test
+  @Timeout(value = 5)
   public void testSkipBadRecords() {
     // test default values
     Configuration conf = new Configuration();

+ 12 - 10
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestTask.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.mapred;
 
+import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.when;
 
@@ -26,13 +27,13 @@ import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.ExitUtil.ExitException;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
 import org.mockito.Mock;
-import org.mockito.junit.MockitoJUnitRunner;
+import org.mockito.junit.jupiter.MockitoExtension;
 
-@RunWith(MockitoJUnitRunner.class)
+@ExtendWith(MockitoExtension.class)
 public class TestTask {
   @Mock
   private TaskUmbilicalProtocol umbilical;
@@ -42,7 +43,7 @@ public class TestTask {
 
   private Task task;
 
-  @Before
+  @BeforeEach
   public void setup() {
     task = new StubTask();
     ExitUtil.disableSystemExit();
@@ -55,11 +56,12 @@ public class TestTask {
     task.statusUpdate(umbilical);
   }
 
-  @Test(expected = ExitException.class)
+  @Test
   public void testStatusUpdateExitsInNonUberMode() throws Exception {
-    setupTest(false);
-
-    task.statusUpdate(umbilical);
+    assertThrows(ExitException.class, () -> {
+      setupTest(false);
+      task.statusUpdate(umbilical);
+    });
   }
 
   private void setupTest(boolean uberized)

+ 9 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestTaskLog.java

@@ -18,7 +18,7 @@
 package org.apache.hadoop.mapred;
 
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -30,8 +30,9 @@ import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.mapred.TaskLog.LogName;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.junit.AfterClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 /**
  * TestCounters checks the sanity and recoverability of Queue
@@ -43,7 +44,7 @@ public class TestTaskLog {
       "target" + File.separatorChar + "test-dir")
       + File.separatorChar + testDirName;
 
-  @AfterClass
+  @AfterAll
   public static void cleanup() {
     FileUtil.fullyDelete(new File(testDir));
   }
@@ -53,7 +54,8 @@ public class TestTaskLog {
    * 
    * @throws IOException
    */
-  @Test (timeout=50000)
+  @Test
+  @Timeout(value = 50)
   public void testTaskLog() throws IOException {
     // test TaskLog
     System.setProperty(
@@ -130,7 +132,8 @@ public class TestTaskLog {
    * 
    * @throws IOException
    */
-  @Test (timeout=50000)
+  @Test
+  @Timeout(value = 50)
   public void testTaskLogWithoutTaskLogDir() throws IOException {
     // TaskLog tasklog= new TaskLog();
     System.clearProperty(YarnConfiguration.YARN_APP_CONTAINER_LOG_DIR);

+ 17 - 12
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestTaskProgressReporter.java

@@ -34,12 +34,13 @@ import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.checkpoint.TaskCheckpointID;
 import org.apache.hadoop.util.ExitUtil;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 public class TestTaskProgressReporter {
   private static int statusUpdateTimes = 0;
@@ -181,17 +182,18 @@ public class TestTaskProgressReporter {
     }
   }
 
-  @Before
+  @BeforeEach
   public void setup() {
     statusUpdateTimes = 0;
   }
 
-  @After
+  @AfterEach
   public void cleanup() {
     FileSystem.clearStatistics();
   }
 
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testScratchDirSize() throws Exception {
     String tmpPath = TEST_DIR + "/testBytesWrittenLimit-tmpFile-"
         + new Random(System.currentTimeMillis()).nextInt();
@@ -260,10 +262,11 @@ public class TestTaskProgressReporter {
     task.done(fakeUmbilical, reporter);
     reporter.resetDoneFlag();
     t.join(1000L);
-    Assert.assertEquals(fastFail, threadExited);
+    assertEquals(fastFail, threadExited);
   }
 
-  @Test (timeout=10000)
+  @Test
+  @Timeout(value = 10)
   public void testTaskProgress() throws Exception {
     JobConf job = new JobConf();
     job.setLong(MRJobConfig.TASK_PROGRESS_REPORT_INTERVAL, 1000);
@@ -279,13 +282,15 @@ public class TestTaskProgressReporter {
     assertThat(statusUpdateTimes).isEqualTo(2);
   }
 
-  @Test(timeout=10000)
+  @Test
+  @Timeout(value = 10)
   public void testBytesWrittenRespectingLimit() throws Exception {
     // add 1024 to the limit to account for writes not controlled by the test
     testBytesWrittenLimit(LOCAL_BYTES_WRITTEN + 1024, false);
   }
 
-  @Test(timeout=10000)
+  @Test
+  @Timeout(value = 10)
   public void testBytesWrittenExceedingLimit() throws Exception {
     testBytesWrittenLimit(LOCAL_BYTES_WRITTEN - 1, true);
   }
@@ -335,6 +340,6 @@ public class TestTaskProgressReporter {
     task.setTaskDone();
     reporter.resetDoneFlag();
     t.join();
-    Assert.assertEquals(failFast, threadExited);
+    assertEquals(failFast, threadExited);
   }
 }

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/lib/TestCombineFileRecordReader.java

@@ -30,12 +30,12 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.hadoop.fs.FileUtil;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.mockito.Mockito;
-import org.junit.Assert;
 
 import java.io.IOException;
 
+import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 
@@ -77,7 +77,7 @@ public class TestCombineFileRecordReader {
       CombineFileRecordReader cfrr = new CombineFileRecordReader(conf, combineFileSplit,
         reporter,  TextRecordReaderWrapper.class);
       verify(reporter).progress();
-      Assert.assertFalse(cfrr.next(key,value));
+      assertFalse(cfrr.next(key, value));
       verify(reporter, times(3)).progress();
     } finally {
       FileUtil.fullyDelete(new File(outDir.toString()));

+ 17 - 12
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/lib/db/TestDBInputFormat.java

@@ -31,18 +31,22 @@ import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapred.lib.db.DBConfiguration;
 import org.apache.hadoop.mapreduce.lib.db.DriverForTest;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
+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.mockito.Mockito.mock;
 
 public class TestDBInputFormat {
 
   /**
    * test DBInputFormat class. Class should split result for chunks
-   * @throws Exception
+   * @throws Exception Execution that occurs during unit test execution.
    */
-  @Test(timeout = 10000)
+  @Test
+  @Timeout(value = 10)
   public void testDBInputFormat() throws Exception {
     JobConf configuration = new JobConf();
     setupDriver(configuration);
@@ -70,7 +74,8 @@ public class TestDBInputFormat {
   /** 
    * test configuration for db. should works DBConfiguration.* parameters. 
    */
-  @Test (timeout = 5000)
+  @Test
+  @Timeout(value = 5)
   public void testSetInput() {
     JobConf configuration = new JobConf();
 
@@ -121,11 +126,12 @@ public class TestDBInputFormat {
   }
 
   /**
-   * 
-   * test DBRecordReader. This reader should creates keys, values, know about position.. 
+   *
+   * test DBRecordReader. This reader should creates keys, values, know about position..
    */
   @SuppressWarnings("unchecked")
-  @Test (timeout = 5000)
+  @Test
+  @Timeout(value = 5)
   public void testDBRecordReader() throws Exception {
 
     JobConf job = mock(JobConf.class);
@@ -139,9 +145,8 @@ public class TestDBInputFormat {
     LongWritable key = reader.createKey();
     assertEquals(0, key.get());
     DBWritable value = reader.createValue();
-    assertEquals(
-        "org.apache.hadoop.mapred.lib.db.DBInputFormat$NullDBWritable", value
-            .getClass().getName());
+    assertEquals("org.apache.hadoop.mapred.lib.db.DBInputFormat$NullDBWritable", value
+        .getClass().getName());
     assertEquals(0, reader.getPos());
     assertFalse(reader.next(key, value));
 

+ 5 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestCluster.java

@@ -19,15 +19,15 @@ package org.apache.hadoop.mapreduce;
 
 import org.apache.hadoop.conf.Configuration;
 
+import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.times;
-import static org.junit.Assert.assertNotNull;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
 import org.apache.hadoop.mapreduce.protocol.ClientProtocolProvider;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
@@ -55,7 +55,7 @@ public class TestCluster {
 
     // Check that we get the acceptable client, even after
     // failure in instantiation.
-    assertNotNull("ClientProtocol is expected", testCluster.getClient());
+    assertNotNull(testCluster.getClient(), "ClientProtocol is expected");
     // Check if we do not try to load the providers after a failure.
     verify(iterator, times(2)).next();
   }

+ 4 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestContextFactory.java

@@ -22,8 +22,8 @@ import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.mapreduce.lib.map.WrappedMapper;
 import org.apache.hadoop.mapreduce.task.JobContextImpl;
 import org.apache.hadoop.mapreduce.task.MapContextImpl;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 public class TestContextFactory {
 
@@ -31,7 +31,7 @@ public class TestContextFactory {
   Configuration conf;
   JobContext jobContext;
   
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     conf = new Configuration();
     jobId = new JobID("test", 1);
@@ -56,7 +56,7 @@ public class TestContextFactory {
     ContextFactory.cloneMapContext(mapperContext, conf, null, null);
   }
 
-  @Before
+  @BeforeEach
   public void tearDown() throws Exception {
     
   }

+ 13 - 10
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJob.java

@@ -18,8 +18,12 @@
 
 package org.apache.hadoop.mapreduce;
 
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertSame;
+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;
 
 import java.io.IOException;
 
@@ -31,8 +35,7 @@ import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 public class TestJob {
   @Test
@@ -52,7 +55,7 @@ public class TestJob {
     when(client.getTaskCompletionEvents(jobid, 0, 10)).thenReturn(
         TaskCompletionEvent.EMPTY_ARRAY);
     Job job = Job.getInstance(cluster, status, new JobConf());
-    Assert.assertNotNull(job.toString());
+    assertNotNull(job.toString());
   }
 
   @Test
@@ -68,8 +71,8 @@ public class TestJob {
     Job job = Job.getInstance(cluster, status, new JobConf());
 
     // ensurer job status is RUNNING
-    Assert.assertNotNull(job.getStatus());
-    Assert.assertTrue(job.getStatus().getState() == State.RUNNING);
+    assertNotNull(job.getStatus());
+    assertTrue(job.getStatus().getState() == State.RUNNING);
 
     // when updating job status, job client could not retrieve
     // job status, and status reset to null
@@ -78,15 +81,15 @@ public class TestJob {
     try {
       job.updateStatus();
     } catch (IOException e) {
-      Assert.assertTrue(e != null
+      assertTrue(e != null
           && e.getMessage().contains("Job status not available"));
     }
 
     try {
       ControlledJob cj = new ControlledJob(job, null);
-      Assert.assertNotNull(cj.toString());
+      assertNotNull(cj.toString());
     } catch (NullPointerException e) {
-      Assert.fail("job API fails with NPE");
+      fail("job API fails with NPE");
     }
   }
 

+ 7 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobMonitorAndPrint.java

@@ -18,12 +18,12 @@
 
 package org.apache.hadoop.mapreduce;
 
-
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.anyInt;
-import static org.mockito.ArgumentMatchers.isA;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.anyInt;
 import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.isA;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.when;
@@ -33,9 +33,8 @@ import java.io.IOException;
 import java.io.LineNumberReader;
 import java.io.StringReader;
 
-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;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.TaskReport;
@@ -57,7 +56,7 @@ public class TestJobMonitorAndPrint {
   private Configuration conf;
   private ClientProtocol clientProtocol;
 
-  @Before
+  @BeforeEach
   public void setUp() throws IOException {
     conf = new Configuration();
     clientProtocol = mock(ClientProtocol.class);

+ 37 - 36
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java

@@ -18,15 +18,18 @@
 
 package org.apache.hadoop.mapreduce;
 
+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.any;
 import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.eq;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.spy;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.net.URI;
@@ -47,8 +50,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.mapred.JobConf;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.mockito.ArgumentCaptor;
 import org.mockito.verification.VerificationMode;
 
@@ -64,26 +66,25 @@ public class TestJobResourceUploader {
     JobResourceUploader uploader =
         new JobResourceUploader(FileSystem.getLocal(conf), false);
 
-    Assert.assertEquals("Failed: absolute, no scheme, with fragment",
-        "/testWithFragment.txt",
-        uploader.stringToPath("/testWithFragment.txt#fragment.txt").toString());
+    assertEquals("/testWithFragment.txt",
+        uploader.stringToPath("/testWithFragment.txt#fragment.txt").toString(),
+        "Failed: absolute, no scheme, with fragment");
 
-    Assert.assertEquals("Failed: absolute, with scheme, with fragment",
-        "file:/testWithFragment.txt",
-        uploader.stringToPath("file:///testWithFragment.txt#fragment.txt")
-            .toString());
+    assertEquals("file:/testWithFragment.txt",
+        uploader.stringToPath("file:///testWithFragment.txt#fragment.txt").toString(),
+        "Failed: absolute, with scheme, with fragment");
 
-    Assert.assertEquals("Failed: relative, no scheme, with fragment",
-        "testWithFragment.txt",
-        uploader.stringToPath("testWithFragment.txt#fragment.txt").toString());
+    assertEquals("testWithFragment.txt",
+        uploader.stringToPath("testWithFragment.txt#fragment.txt").toString(),
+        "Failed: relative, no scheme, with fragment");
 
-    Assert.assertEquals("Failed: relative, no scheme, no fragment",
-        "testWithFragment.txt",
-        uploader.stringToPath("testWithFragment.txt").toString());
+    assertEquals("testWithFragment.txt",
+        uploader.stringToPath("testWithFragment.txt").toString(),
+        "Failed: relative, no scheme, no fragment");
 
-    Assert.assertEquals("Failed: absolute, with scheme, no fragment",
-        "file:/testWithFragment.txt",
-        uploader.stringToPath("file:///testWithFragment.txt").toString());
+    assertEquals("file:/testWithFragment.txt",
+        uploader.stringToPath("file:///testWithFragment.txt").toString(),
+        "Failed: absolute, with scheme, no fragment");
   }
 
   @Test
@@ -421,7 +422,7 @@ public class TestJobResourceUploader {
 
     ArgumentCaptor<Path> pathCaptor = ArgumentCaptor.forClass(Path.class);
     verify(fs).makeQualified(pathCaptor.capture());
-    Assert.assertEquals("Path", expectedRemotePath, pathCaptor.getValue());
+    assertEquals(expectedRemotePath, pathCaptor.getValue(), "Path");
   }
 
   private void testErasureCodingSetting(boolean defaultBehavior)
@@ -482,8 +483,8 @@ public class TestJobResourceUploader {
     validateResourcePathsSub(job.getCacheArchives(), expectedArchives);
     // We use a different job object here because the jobjar was set on a
     // different job object
-    Assert.assertEquals("Job jar path is different than expected!",
-        expectedJobJar, job.getJar());
+    assertEquals(expectedJobJar, job.getJar(),
+        "Job jar path is different than expected!");
   }
 
   private void validateResourcePathsSub(URI[] actualURIs,
@@ -491,14 +492,14 @@ public class TestJobResourceUploader {
     List<URI> actualList = Arrays.asList(actualURIs);
     Set<String> expectedSet = new HashSet<>(Arrays.asList(expectedURIs));
     if (actualList.size() != expectedSet.size()) {
-      Assert.fail("Expected list of resources (" + expectedSet.size()
+      fail("Expected list of resources (" + expectedSet.size()
           + ") and actual list of resources (" + actualList.size()
           + ") are different lengths!");
     }
 
     for (URI u : actualList) {
       if (!expectedSet.contains(u.toString())) {
-        Assert.fail("Resource list contained unexpected path: " + u.toString());
+        fail("Resource list contained unexpected path: " + u);
       }
     }
   }
@@ -511,7 +512,7 @@ public class TestJobResourceUploader {
       ResourceViolation violation) throws IOException {
 
     if (!checkShouldSucceed && violation == null) {
-      Assert.fail("Test is misconfigured. checkShouldSucceed is set to false"
+      fail("Test is misconfigured. checkShouldSucceed is set to false"
           + " and a ResourceViolation is not specified.");
     }
 
@@ -528,36 +529,36 @@ public class TestJobResourceUploader {
           conf.getStringCollection("tmpjars"),
           conf.getStringCollection("tmparchives"),
           conf.getJar(), statCache);
-      Assert.assertTrue("Limits check succeeded when it should have failed.",
-          checkShouldSucceed);
+      assertTrue(checkShouldSucceed,
+          "Limits check succeeded when it should have failed.");
     } catch (IOException e) {
       if (checkShouldSucceed) {
-        Assert.fail("Limits check failed when it should have succeeded: " + e);
+        fail("Limits check failed when it should have succeeded: " + e);
       }
       switch (violation) {
       case NUMBER_OF_RESOURCES:
         if (!e.getMessage().contains(
             JobResourceUploader.MAX_RESOURCE_ERR_MSG)) {
-          Assert.fail("Test failed unexpectedly: " + e);
+          fail("Test failed unexpectedly: " + e);
         }
         break;
 
       case TOTAL_RESOURCE_SIZE:
         if (!e.getMessage().contains(
             JobResourceUploader.MAX_TOTAL_RESOURCE_MB_ERR_MSG)) {
-          Assert.fail("Test failed unexpectedly: " + e);
+          fail("Test failed unexpectedly: " + e);
         }
         break;
 
       case SINGLE_RESOURCE_SIZE:
         if (!e.getMessage().contains(
             JobResourceUploader.MAX_SINGLE_RESOURCE_MB_ERR_MSG)) {
-          Assert.fail("Test failed unexpectedly: " + e);
+          fail("Test failed unexpectedly: " + e);
         }
         break;
 
       default:
-        Assert.fail("Test failed unexpectedly: " + e);
+        fail("Test failed unexpectedly: " + e);
         break;
       }
     }

+ 8 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploaderWithSharedCache.java

@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.mapreduce;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyString;
 import static org.mockito.ArgumentMatchers.eq;
@@ -51,10 +51,10 @@ import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.client.api.SharedCacheClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
-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.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 import org.slf4j.Logger;
@@ -75,12 +75,12 @@ public class TestJobResourceUploaderWithSharedCache {
       new Path(MRJobConfig.DEFAULT_MR_AM_STAGING_DIR);
   private String input = "roses.are.red\nviolets.are.blue\nbunnies.are.pink\n";
 
-  @Before
+  @BeforeEach
   public void cleanup() throws Exception {
     remoteFs.delete(remoteStagingDir, true);
   }
 
-  @BeforeClass
+  @BeforeAll
   public static void setup() throws IOException {
     // create configuration, dfs, file system
     localFs = FileSystem.getLocal(conf);
@@ -92,7 +92,7 @@ public class TestJobResourceUploaderWithSharedCache {
     remoteFs = dfs.getFileSystem();
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() {
     try {
       if (localFs != null) {

+ 20 - 19
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobSubmissionFiles.java

@@ -26,11 +26,12 @@ import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.io.IOException;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -80,25 +81,25 @@ public class TestJobSubmissionFiles {
         JobSubmissionFiles.getStagingDir(cluster, conf, user));
   }
 
-  @Test(expected = IOException.class)
+  @Test()
   public void testGetStagingWhenFileOwnerNameAndCurrentUserNameDoesNotMatch()
       throws IOException, InterruptedException {
-    Cluster cluster = mock(Cluster.class);
-    Configuration conf = new Configuration();
-    String stagingDirOwner = "someuser";
-    Path stagingPath = mock(Path.class);
-    UserGroupInformation user = UserGroupInformation
-        .createUserForTesting(USER_1, GROUP_NAMES);
-    assertEquals(USER_1, user.getUserName());
-    FileSystem fs = new FileSystemTestHelper.MockFileSystem();
-    FileStatus fileStatus = new FileStatus(1, true, 1, 1, 100L, 100L,
-        FsPermission.getDefault(), stagingDirOwner, stagingDirOwner,
-        stagingPath);
-    when(stagingPath.getFileSystem(conf)).thenReturn(fs);
-    when(fs.getFileStatus(stagingPath)).thenReturn(fileStatus);
-    when(cluster.getStagingAreaDir()).thenReturn(stagingPath);
-    assertEquals(stagingPath,
-        JobSubmissionFiles.getStagingDir(cluster, conf, user));
+    assertThrows(IOException.class, () -> {
+      Cluster cluster = mock(Cluster.class);
+      Configuration conf = new Configuration();
+      String stagingDirOwner = "someuser";
+      Path stagingPath = mock(Path.class);
+      UserGroupInformation user = UserGroupInformation.createUserForTesting(USER_1, GROUP_NAMES);
+      assertEquals(USER_1, user.getUserName());
+      FileSystem fs = new FileSystemTestHelper.MockFileSystem();
+      FileStatus fileStatus = new FileStatus(1, true, 1, 1, 100L, 100L,
+          FsPermission.getDefault(), stagingDirOwner, stagingDirOwner, stagingPath);
+      when(stagingPath.getFileSystem(conf)).thenReturn(fs);
+      when(fs.getFileStatus(stagingPath)).thenReturn(fileStatus);
+      when(cluster.getStagingAreaDir()).thenReturn(stagingPath);
+      assertEquals(stagingPath,
+          JobSubmissionFiles.getStagingDir(cluster, conf, user));
+    });
   }
 
   @Test

+ 9 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestShufflePlugin.java

@@ -18,9 +18,10 @@
 
 package org.apache.hadoop.mapreduce;
 
-import org.junit.Test;
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
+import org.junit.jupiter.api.Test;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.mock;
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.mapred.Task.CombineOutputCollector;
@@ -104,14 +105,14 @@ public class TestShufflePlugin<K, V> {
       ShuffleConsumerPlugin shuffleConsumerPlugin = null;
       Class<? extends ShuffleConsumerPlugin> clazz =
         jobConf.getClass(MRConfig.SHUFFLE_CONSUMER_PLUGIN, Shuffle.class, ShuffleConsumerPlugin.class);
-      assertNotNull("Unable to get " + MRConfig.SHUFFLE_CONSUMER_PLUGIN, clazz);
+      assertNotNull(clazz, "Unable to get " + MRConfig.SHUFFLE_CONSUMER_PLUGIN);
 
       // load 3rd party plugin through core's factory method
       shuffleConsumerPlugin = ReflectionUtils.newInstance(clazz, jobConf);
-      assertNotNull("Unable to load " + MRConfig.SHUFFLE_CONSUMER_PLUGIN, shuffleConsumerPlugin);
+      assertNotNull(shuffleConsumerPlugin, "Unable to load " + MRConfig.SHUFFLE_CONSUMER_PLUGIN);
     }
     catch (Exception e) {
-      assertTrue("Threw exception:" + e, false);
+      assertTrue(false, "Threw exception:" + e);
     }
   }
 
@@ -161,7 +162,7 @@ public class TestShufflePlugin<K, V> {
       shuffleConsumerPlugin.close();
     }
     catch (Exception e) {
-      assertTrue("Threw exception:" + e, false);
+      assertTrue(false, "Threw exception:" + e);
     }
 
     // verify that these APIs are available for 3rd party plugins
@@ -184,7 +185,7 @@ public class TestShufflePlugin<K, V> {
       mockLocalDirAllocator.getLocalPathToRead("", mockJobConf);
     }
     catch (Exception e) {
-      assertTrue("Threw exception:" + e, false);
+      assertTrue(false, "Threw exception:" + e);
     }
   }
 }

+ 107 - 111
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestTaskID.java

@@ -23,8 +23,13 @@ import java.nio.ByteBuffer;
 
 import org.apache.hadoop.io.DataInputByteBuffer;
 import org.apache.hadoop.io.WritableUtils;
-import org.junit.Test;
-import static org.junit.Assert.*;
+import org.junit.jupiter.api.Test;
+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.assertSame;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 /**
  * Test the {@link TaskID} class.
@@ -38,13 +43,13 @@ public class TestTaskID {
     JobID jobId = new JobID("1234", 0);
     TaskID taskId = new TaskID(jobId, TaskType.MAP, 0);
 
-    assertSame("TaskID did not store the JobID correctly",
-        jobId, taskId.getJobID());
+    assertSame(jobId, taskId.getJobID(),
+        "TaskID did not store the JobID correctly");
 
     taskId = new TaskID();
 
-    assertEquals("Job ID was set unexpectedly in default contsructor",
-        "", taskId.getJobID().getJtIdentifier());
+    assertEquals("", taskId.getJobID().getJtIdentifier(),
+        "Job ID was set unexpectedly in default constructor");
   }
 
   /**
@@ -58,18 +63,18 @@ public class TestTaskID {
       TaskID taskId = new TaskID(jobId, type, 0);
 
       if (type == TaskType.MAP) {
-        assertTrue("TaskID for map task did not correctly identify itself "
-            + "as a map task", taskId.isMap());
+        assertTrue(taskId.isMap(),
+            "TaskID for map task did not correctly identify itself as a map task");
       } else {
-        assertFalse("TaskID for " + type + " task incorrectly identified "
-            + "itself as a map task", taskId.isMap());
+        assertFalse(taskId.isMap(),
+            "TaskID for " + type + " task incorrectly identified itself as a map task");
       }
     }
 
     TaskID taskId = new TaskID();
 
-    assertFalse("TaskID of default type incorrectly identified itself as a "
-        + "map task", taskId.isMap());
+    assertFalse(taskId.isMap(),
+        "TaskID of default type incorrectly identified itself as a map task");
   }
 
   /**
@@ -82,14 +87,14 @@ public class TestTaskID {
     for (TaskType type : TaskType.values()) {
       TaskID taskId = new TaskID(jobId, type, 0);
 
-      assertEquals("TaskID incorrectly reported its type",
-          type, taskId.getTaskType());
+      assertEquals(type, taskId.getTaskType(),
+          "TaskID incorrectly reported its type");
     }
 
     TaskID taskId = new TaskID();
 
-    assertEquals("TaskID of default type incorrectly reported its type",
-        TaskType.REDUCE, taskId.getTaskType());
+    assertEquals(TaskType.REDUCE, taskId.getTaskType(),
+        "TaskID of default type incorrectly reported its type");
   }
 
   /**
@@ -102,18 +107,20 @@ public class TestTaskID {
     TaskID taskId1 = new TaskID(jobId1, TaskType.MAP, 0);
     TaskID taskId2 = new TaskID(jobId1, TaskType.MAP, 0);
 
-    assertTrue("The equals() method reported two equal task IDs were not equal",
-        taskId1.equals(taskId2));
+    assertTrue(taskId1.equals(taskId2),
+        "The equals() method reported two equal task IDs were not equal");
 
     taskId2 = new TaskID(jobId2, TaskType.MAP, 0);
 
-    assertFalse("The equals() method reported two task IDs with different "
-        + "job IDs were equal", taskId1.equals(taskId2));
+    assertFalse(taskId1.equals(taskId2),
+        "The equals() method reported two task IDs with different " +
+        "job IDs were equal");
 
     taskId2 = new TaskID(jobId1, TaskType.MAP, 1);
 
-    assertFalse("The equals() method reported two task IDs with different IDs "
-        + "were equal", taskId1.equals(taskId2));
+    assertFalse(taskId1.equals(taskId2),
+        "The equals() method reported two task IDs with different IDs " +
+        "were equal");
 
     TaskType[] types = TaskType.values();
 
@@ -123,20 +130,21 @@ public class TestTaskID {
         taskId2 = new TaskID(jobId1, types[j], 0);
 
         if (i == j) {
-          assertTrue("The equals() method reported two equal task IDs were not "
-              + "equal", taskId1.equals(taskId2));
+          assertTrue(taskId1.equals(taskId2),
+              "The equals() method reported two equal task IDs were not equal");
         } else {
-          assertFalse("The equals() method reported two task IDs with "
-              + "different types were equal", taskId1.equals(taskId2));
+          assertFalse(taskId1.equals(taskId2),
+              "The equals() method reported two task IDs with " +
+              "different types were equal");
         }
       }
     }
 
-    assertFalse("The equals() method matched against a JobID object",
-        taskId1.equals(jobId1));
+    assertFalse(taskId1.equals(jobId1),
+        "The equals() method matched against a JobID object");
 
-    assertFalse("The equals() method matched against a null object",
-        taskId1.equals(null));
+    assertFalse(taskId1.equals(null),
+        "The equals() method matched against a null object");
   }
 
   /**
@@ -148,13 +156,13 @@ public class TestTaskID {
     TaskID taskId1 = new TaskID(jobId, TaskType.REDUCE, 0);
     TaskID taskId2 = new TaskID(jobId, TaskType.REDUCE, 0);
 
-    assertEquals("The compareTo() method returned non-zero for two equal "
-        + "task IDs", 0, taskId1.compareTo(taskId2));
+    assertEquals(0, taskId1.compareTo(taskId2),
+        "The compareTo() method returned non-zero for two equal task IDs");
 
     taskId2 = new TaskID(jobId, TaskType.MAP, 1);
 
-    assertTrue("The compareTo() method did not weigh task type more than task "
-        + "ID", taskId1.compareTo(taskId2) > 0);
+    assertTrue(taskId1.compareTo(taskId2) > 0,
+        "The compareTo() method did not weigh task type more than task ID");
 
     TaskType[] types = TaskType.values();
 
@@ -164,14 +172,16 @@ public class TestTaskID {
         taskId2 = new TaskID(jobId, types[j], 0);
 
         if (i == j) {
-          assertEquals("The compareTo() method returned non-zero for two equal "
-              + "task IDs", 0, taskId1.compareTo(taskId2));
+          assertEquals(0, taskId1.compareTo(taskId2),
+              "The compareTo() method returned non-zero for two equal task IDs");
         } else if (i < j) {
-          assertTrue("The compareTo() method did not order " + types[i]
-              + " before " + types[j], taskId1.compareTo(taskId2) < 0);
+          assertTrue(taskId1.compareTo(taskId2) < 0,
+              "The compareTo() method did not order " + types[i]
+              + " before " + types[j]);
         } else {
-          assertTrue("The compareTo() method did not order " + types[i]
-              + " after " + types[j], taskId1.compareTo(taskId2) > 0);
+          assertTrue(taskId1.compareTo(taskId2) > 0,
+              "The compareTo() method did not order " + types[i]
+              + " after " + types[j]);
         }
       }
     }
@@ -203,8 +213,8 @@ public class TestTaskID {
       String str = String.format("task_1234_0001_%c_000000",
           TaskID.getRepresentingCharacter(type));
 
-      assertEquals("The toString() method returned the wrong value",
-          str, taskId.toString());
+      assertEquals(str, taskId.toString(),
+          "The toString() method returned the wrong value");
     }
   }
 
@@ -222,8 +232,8 @@ public class TestTaskID {
       String str = String.format("_1234_0001_%c_000000",
           TaskID.getRepresentingCharacter(type));
 
-      assertEquals("The appendTo() method appended the wrong value",
-          str, taskId.appendTo(builder).toString());
+      assertEquals(str, taskId.appendTo(builder).toString(),
+          "The appendTo() method appended the wrong value.");
     }
 
     try {
@@ -246,8 +256,8 @@ public class TestTaskID {
       TaskID taskId1 = new TaskID(jobId, types[i], i);
       TaskID taskId2 = new TaskID(jobId, types[i], i);
 
-      assertTrue("The hashcode() method gave unequal hash codes for two equal "
-          + "task IDs", taskId1.hashCode() == taskId2.hashCode());
+      assertEquals(taskId1.hashCode(), taskId2.hashCode(),
+          "The hashcode() method gave unequal hash codes for two equal task IDs");
     }
   }
 
@@ -273,8 +283,8 @@ public class TestTaskID {
 
     instance.readFields(in);
 
-    assertEquals("The readFields() method did not produce the expected task ID",
-        "task_1234_0001_r_000000", instance.toString());
+    assertEquals("task_1234_0001_r_000000", instance.toString(),
+        "The readFields() method did not produce the expected task ID");
   }
 
   /**
@@ -294,17 +304,17 @@ public class TestTaskID {
 
     in.reset(ByteBuffer.wrap(baos.toByteArray()));
 
-    assertEquals("The write() method did not write the expected task ID",
-        0, in.readInt());
-    assertEquals("The write() method did not write the expected job ID",
-        1, in.readInt());
-    assertEquals("The write() method did not write the expected job "
-        + "identifier length", 4, WritableUtils.readVInt(in));
+    assertEquals(0, in.readInt(),
+        "The write() method did not write the expected task ID");
+    assertEquals(1, in.readInt(),
+        "The write() method did not write the expected job ID");
+    assertEquals(4, WritableUtils.readVInt(in),
+        "The write() method did not write the expected job identifier length");
     in.readFully(buffer, 0, 4);
-    assertEquals("The write() method did not write the expected job "
-        + "identifier length", "1234", new String(buffer));
-    assertEquals("The write() method did not write the expected task type",
-        TaskType.JOB_SETUP, WritableUtils.readEnum(in, TaskType.class));
+    assertEquals("1234", new String(buffer),
+        "The write() method did not write the expected job identifier length");
+    assertEquals(TaskType.JOB_SETUP, WritableUtils.readEnum(in, TaskType.class),
+        "The write() method did not write the expected task type");
   }
 
   /**
@@ -312,34 +322,28 @@ public class TestTaskID {
    */
   @Test
   public void testForName() {
-    assertEquals("The forName() method did not parse the task ID string "
-        + "correctly", "task_1_0001_m_000000",
-        TaskID.forName("task_1_0001_m_000").toString());
-    assertEquals("The forName() method did not parse the task ID string "
-        + "correctly", "task_23_0002_r_000001",
-        TaskID.forName("task_23_0002_r_0001").toString());
-    assertEquals("The forName() method did not parse the task ID string "
-        + "correctly", "task_345_0003_s_000002",
-        TaskID.forName("task_345_0003_s_00002").toString());
-    assertEquals("The forName() method did not parse the task ID string "
-        + "correctly", "task_6789_0004_c_000003",
-        TaskID.forName("task_6789_0004_c_000003").toString());
-    assertEquals("The forName() method did not parse the task ID string "
-        + "correctly", "task_12345_0005_t_4000000",
-        TaskID.forName("task_12345_0005_t_4000000").toString());
+    assertEquals("task_1_0001_m_000000", TaskID.forName("task_1_0001_m_000").toString(),
+        "The forName() method did not parse the task ID string correctly");
+    assertEquals("task_23_0002_r_000001", TaskID.forName("task_23_0002_r_0001").toString(),
+        "The forName() method did not parse the task ID string correctly");
+    assertEquals("task_345_0003_s_000002", TaskID.forName("task_345_0003_s_00002").toString(),
+        "The forName() method did not parse the task ID string correctly");
+    assertEquals("task_6789_0004_c_000003", TaskID.forName("task_6789_0004_c_000003").toString(),
+        "The forName() method did not parse the task ID string correctly");
+    assertEquals("task_12345_0005_t_4000000",
+        TaskID.forName("task_12345_0005_t_4000000").toString(),
+        "The forName() method did not parse the task ID string correctly");
 
     try {
       TaskID.forName("tisk_12345_0005_t_4000000");
-      fail("The forName() method parsed an invalid job ID: "
-          + "tisk_12345_0005_t_4000000");
+      fail("The forName() method parsed an invalid job ID: tisk_12345_0005_t_4000000");
     } catch (IllegalArgumentException ex) {
       // Expected
     }
 
     try {
       TaskID.forName("tisk_12345_0005_t_4000000");
-      fail("The forName() method parsed an invalid job ID: "
-          + "tisk_12345_0005_t_4000000");
+      fail("The forName() method parsed an invalid job ID: tisk_12345_0005_t_4000000");
     } catch (IllegalArgumentException ex) {
       // Expected
     }
@@ -414,21 +418,18 @@ public class TestTaskID {
    */
   @Test
   public void testGetRepresentingCharacter() {
-    assertEquals("The getRepresentingCharacter() method did not return the "
-        + "expected character", 'm',
-        TaskID.getRepresentingCharacter(TaskType.MAP));
-    assertEquals("The getRepresentingCharacter() method did not return the "
-        + "expected character", 'r',
-        TaskID.getRepresentingCharacter(TaskType.REDUCE));
-    assertEquals("The getRepresentingCharacter() method did not return the "
-        + "expected character", 's',
-        TaskID.getRepresentingCharacter(TaskType.JOB_SETUP));
-    assertEquals("The getRepresentingCharacter() method did not return the "
-        + "expected character", 'c',
-        TaskID.getRepresentingCharacter(TaskType.JOB_CLEANUP));
-    assertEquals("The getRepresentingCharacter() method did not return the "
-        + "expected character", 't',
-        TaskID.getRepresentingCharacter(TaskType.TASK_CLEANUP));
+    assertEquals('m', TaskID.getRepresentingCharacter(TaskType.MAP),
+        "The getRepresentingCharacter() method did not return the expected character");
+    assertEquals('r', TaskID.getRepresentingCharacter(TaskType.REDUCE),
+        "The getRepresentingCharacter() method did not return the " +
+        "expected character");
+    assertEquals('s', TaskID.getRepresentingCharacter(TaskType.JOB_SETUP),
+        "The getRepresentingCharacter() method did not return the "
+        + "expected character");
+    assertEquals('c', TaskID.getRepresentingCharacter(TaskType.JOB_CLEANUP),
+        "The getRepresentingCharacter() method did not return the expected character");
+    assertEquals('t', TaskID.getRepresentingCharacter(TaskType.TASK_CLEANUP),
+        "The getRepresentingCharacter() method did not return the expected character");
   }
 
   /**
@@ -436,23 +437,18 @@ public class TestTaskID {
    */
   @Test
   public void testGetTaskTypeChar() {
-    assertEquals("The getTaskType() method did not return the expected type",
-        TaskType.MAP,
-        TaskID.getTaskType('m'));
-    assertEquals("The getTaskType() method did not return the expected type",
-        TaskType.REDUCE,
-        TaskID.getTaskType('r'));
-    assertEquals("The getTaskType() method did not return the expected type",
-        TaskType.JOB_SETUP,
-        TaskID.getTaskType('s'));
-    assertEquals("The getTaskType() method did not return the expected type",
-        TaskType.JOB_CLEANUP,
-        TaskID.getTaskType('c'));
-    assertEquals("The getTaskType() method did not return the expected type",
-        TaskType.TASK_CLEANUP,
-        TaskID.getTaskType('t'));
-    assertNull("The getTaskType() method did not return null for an unknown "
-        + "type", TaskID.getTaskType('x'));
+    assertEquals(TaskType.MAP, TaskID.getTaskType('m'),
+        "The getTaskType() method did not return the expected type");
+    assertEquals(TaskType.REDUCE, TaskID.getTaskType('r'),
+        "The getTaskType() method did not return the expected type");
+    assertEquals(TaskType.JOB_SETUP, TaskID.getTaskType('s'),
+        "The getTaskType() method did not return the expected type");
+    assertEquals(TaskType.JOB_CLEANUP,
+        TaskID.getTaskType('c'), "The getTaskType() method did not return the expected type");
+    assertEquals(TaskType.TASK_CLEANUP,
+        TaskID.getTaskType('t'), "The getTaskType() method did not return the expected type");
+    assertNull(TaskID.getTaskType('x'),
+        "The getTaskType() method did not return null for an unknown type");
   }
 
   /**
@@ -460,7 +456,7 @@ public class TestTaskID {
    */
   @Test
   public void testGetAllTaskTypes() {
-    assertEquals("The getAllTaskTypes method did not return the expected "
-        + "string", "(m|r|s|c|t)", TaskID.getAllTaskTypes());
+    assertEquals("(m|r|s|c|t)", TaskID.getAllTaskTypes(),
+        "The getAllTaskTypes method did not return the expected string");
   }
 }

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/checkpoint/TestFSCheckpointID.java

@@ -22,7 +22,7 @@ import java.io.IOException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 public class TestFSCheckpointID {
 

+ 12 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/checkpoint/TestFSCheckpointService.java

@@ -27,10 +27,18 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.mapreduce.checkpoint.CheckpointService.CheckpointWriteChannel;
-import org.junit.Test;
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
-import org.mockito.*;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.isA;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 public class TestFSCheckpointService {
 

+ 39 - 40
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/filecache/TestClientDistributedCacheManager.java

@@ -35,13 +35,14 @@ import org.apache.hadoop.io.SequenceFile.CompressionType;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assume.assumeTrue;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+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.Assumptions.assumeTrue;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -69,7 +70,7 @@ public class TestClientDistributedCacheManager {
   private Path secondCacheFile;
   private Configuration conf;
   
-  @Before
+  @BeforeEach
   public void setup() throws IOException {
     conf = new Configuration();
     fs = FileSystem.get(conf);
@@ -79,7 +80,7 @@ public class TestClientDistributedCacheManager {
     createTempFile(secondCacheFile, conf);
   }
   
-  @After
+  @AfterEach
   public void tearDown() throws IOException {
     if (fs.delete(TEST_ROOT_DIR, true)) {
       LOG.warn("Failed to delete test root dir and its content under "
@@ -100,12 +101,10 @@ public class TestClientDistributedCacheManager {
     FileStatus firstStatus = statCache.get(firstCacheFile.toUri());
     FileStatus secondStatus = statCache.get(secondCacheFile.toUri());
     
-    assertNotNull(firstCacheFile + " was not found in the stats cache",
-        firstStatus);
-    assertNotNull(secondCacheFile + " was not found in the stats cache",
-        secondStatus);
-    assertEquals("Missing/extra entries found in the stats cache",
-        2, statCache.size());
+    assertNotNull(firstStatus, firstCacheFile + " was not found in the stats cache");
+    assertNotNull(secondStatus, secondCacheFile + " was not found in the stats cache");
+    assertEquals(2, statCache.size(),
+        "Missing/extra entries found in the stats cache");
     String expected = firstStatus.getModificationTime() + ","
         + secondStatus.getModificationTime();
     assertEquals(expected, jobConf.get(MRJobConfig.CACHE_FILE_TIMESTAMPS));
@@ -118,13 +117,13 @@ public class TestClientDistributedCacheManager {
 
     FileStatus thirdStatus = statCache.get(TEST_VISIBILITY_CHILD_DIR.toUri());
 
-    assertEquals("Missing/extra entries found in the stats cache",
-        1, statCache.size());
-    assertNotNull(TEST_VISIBILITY_CHILD_DIR
-        + " was not found in the stats cache", thirdStatus);
+    assertEquals(1, statCache.size(),
+        "Missing/extra entries found in the stats cache");
+    assertNotNull(thirdStatus, TEST_VISIBILITY_CHILD_DIR
+        + " was not found in the stats cache");
     expected = Long.toString(thirdStatus.getModificationTime());
-    assertEquals("Incorrect timestamp for " + TEST_VISIBILITY_CHILD_DIR,
-        expected, jobConf.get(MRJobConfig.CACHE_FILE_TIMESTAMPS));
+    assertEquals(expected, jobConf.get(MRJobConfig.CACHE_FILE_TIMESTAMPS),
+        "Incorrect timestamp for " + TEST_VISIBILITY_CHILD_DIR);
   }
   
   @Test
@@ -145,17 +144,17 @@ public class TestClientDistributedCacheManager {
     jobConf = job.getConfiguration();
 
     // skip test if scratch dir is not PUBLIC
-    assumeTrue(TEST_VISIBILITY_PARENT_DIR + " is not public",
-        ClientDistributedCacheManager.isPublic(
-            jobConf, TEST_VISIBILITY_PARENT_DIR.toUri(), statCache));
+    assumeTrue(ClientDistributedCacheManager.isPublic(
+        jobConf, TEST_VISIBILITY_PARENT_DIR.toUri(), statCache),
+        TEST_VISIBILITY_PARENT_DIR + " is not public");
 
     ClientDistributedCacheManager.determineCacheVisibilities(jobConf,
         statCache);
     // We use get() instead of getBoolean() so we can tell the difference
     // between wrong and missing
-    assertEquals("The file paths were not found to be publicly visible "
-        + "even though the full path is publicly accessible",
-        "true,true", jobConf.get(MRJobConfig.CACHE_FILE_VISIBILITIES));
+    assertEquals("true,true", jobConf.get(MRJobConfig.CACHE_FILE_VISIBILITIES),
+        "The file paths were not found to be publicly visible " +
+        "even though the full path is publicly accessible");
     checkCacheEntries(statCache, null, firstCacheFile, relativePath);
 
     job = Job.getInstance(conf);
@@ -167,9 +166,9 @@ public class TestClientDistributedCacheManager {
         statCache);
     // We use get() instead of getBoolean() so we can tell the difference
     // between wrong and missing
-    assertEquals("The file path was not found to be publicly visible "
-        + "even though the full path is publicly accessible",
-        "true", jobConf.get(MRJobConfig.CACHE_FILE_VISIBILITIES));
+    assertEquals("true", jobConf.get(MRJobConfig.CACHE_FILE_VISIBILITIES),
+        "The file path was not found to be publicly visible " +
+        "even though the full path is publicly accessible");
     checkCacheEntries(statCache, null, wildcardPath.getParent());
 
     Path qualifiedParent = fs.makeQualified(TEST_VISIBILITY_PARENT_DIR);
@@ -185,9 +184,9 @@ public class TestClientDistributedCacheManager {
         statCache);
     // We use get() instead of getBoolean() so we can tell the difference
     // between wrong and missing
-    assertEquals("The file paths were found to be publicly visible "
-        + "even though the parent directory is not publicly accessible",
-        "false,false", jobConf.get(MRJobConfig.CACHE_FILE_VISIBILITIES));
+    assertEquals("false,false", jobConf.get(MRJobConfig.CACHE_FILE_VISIBILITIES),
+        "The file paths were found to be publicly visible " +
+        "even though the parent directory is not publicly accessible");
     checkCacheEntries(statCache, qualifiedParent,
         firstCacheFile, relativePath);
 
@@ -200,9 +199,9 @@ public class TestClientDistributedCacheManager {
         statCache);
     // We use get() instead of getBoolean() so we can tell the difference
     // between wrong and missing
-    assertEquals("The file path was found to be publicly visible "
-        + "even though the parent directory is not publicly accessible",
-        "false", jobConf.get(MRJobConfig.CACHE_FILE_VISIBILITIES));
+    assertEquals("false", jobConf.get(MRJobConfig.CACHE_FILE_VISIBILITIES),
+        "The file path was found to be publicly visible " +
+        "even though the parent directory is not publicly accessible");
     checkCacheEntries(statCache, qualifiedParent, wildcardPath.getParent());
   }
 
@@ -236,10 +235,10 @@ public class TestClientDistributedCacheManager {
     missing.removeAll(expected);
     extra.removeAll(uris);
 
-    assertTrue("File status cache does not contain an entries for " + missing,
-        missing.isEmpty());
-    assertTrue("File status cache contains extra extries: " + extra,
-        extra.isEmpty());
+    assertTrue(missing.isEmpty(),
+        "File status cache does not contain an entries for " + missing);
+    assertTrue(extra.isEmpty(),
+        "File status cache contains extra entries: " + extra);
   }
 
   @SuppressWarnings("deprecation")

+ 37 - 41
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/filecache/TestDistributedCache.java

@@ -21,8 +21,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.MRJobConfig;
-import org.junit.Test;
-import static org.junit.Assert.*;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
 
 /**
  * Test the {@link DistributedCache} class.
@@ -44,18 +46,16 @@ public class TestDistributedCache {
     }
 
     DistributedCache.addFileToClassPath(new Path("file:///a"), conf);
-    assertEquals("The mapreduce.job.classpath.files property was not "
-        + "set correctly", "file:/a", conf.get(MRJobConfig.CLASSPATH_FILES));
-    assertEquals("The mapreduce.job.cache.files property was not set "
-        + "correctly", "file:///a", conf.get(MRJobConfig.CACHE_FILES));
+    assertEquals("file:/a", conf.get(MRJobConfig.CLASSPATH_FILES),
+        "The mapreduce.job.classpath.files property was not set correctly");
+    assertEquals("file:///a", conf.get(MRJobConfig.CACHE_FILES),
+        "The mapreduce.job.cache.files property was not set correctly");
 
     DistributedCache.addFileToClassPath(new Path("file:///b"), conf);
-    assertEquals("The mapreduce.job.classpath.files property was not "
-        + "set correctly", "file:/a,file:/b",
-        conf.get(MRJobConfig.CLASSPATH_FILES));
-    assertEquals("The mapreduce.job.cache.files property was not set "
-        + "correctly", "file:///a,file:///b",
-        conf.get(MRJobConfig.CACHE_FILES));
+    assertEquals("file:/a,file:/b", conf.get(MRJobConfig.CLASSPATH_FILES),
+        "The mapreduce.job.classpath.files property was not set correctly");
+    assertEquals("file:///a,file:///b", conf.get(MRJobConfig.CACHE_FILES),
+        "The mapreduce.job.cache.files property was not set correctly");
 
     // Now test with 3 args
     FileSystem fs = FileSystem.newInstance(conf);
@@ -69,18 +69,17 @@ public class TestDistributedCache {
     }
 
     DistributedCache.addFileToClassPath(new Path("file:///a"), conf, fs);
-    assertEquals("The mapreduce.job.classpath.files property was not "
-        + "set correctly", "file:/a", conf.get(MRJobConfig.CLASSPATH_FILES));
-    assertEquals("The mapreduce.job.cache.files property was not set "
-        + "correctly", "file:///a", conf.get(MRJobConfig.CACHE_FILES));
+    assertEquals("file:/a", conf.get(MRJobConfig.CLASSPATH_FILES),
+        "The mapreduce.job.classpath.files property was not set correctly");
+    assertEquals("file:///a", conf.get(MRJobConfig.CACHE_FILES),
+        "The mapreduce.job.cache.files property was not set correctly");
 
     DistributedCache.addFileToClassPath(new Path("file:///b"), conf, fs);
-    assertEquals("The mapreduce.job.classpath.files property was not "
-        + "set correctly", "file:/a,file:/b",
-        conf.get(MRJobConfig.CLASSPATH_FILES));
-    assertEquals("The mapreduce.job.cache.files property was not set "
-        + "correctly", "file:///a,file:///b",
-        conf.get(MRJobConfig.CACHE_FILES));
+    assertEquals("file:/a,file:/b", conf.get(MRJobConfig.CLASSPATH_FILES),
+        "The mapreduce.job.classpath.files property was not set correctly");
+    assertEquals("file:///a,file:///b",
+        conf.get(MRJobConfig.CACHE_FILES),
+        "The mapreduce.job.cache.files property was not set correctly");
 
     // Now test with 4th arg true
     conf.clear();
@@ -93,18 +92,16 @@ public class TestDistributedCache {
     }
 
     DistributedCache.addFileToClassPath(new Path("file:///a"), conf, fs, true);
-    assertEquals("The mapreduce.job.classpath.files property was not "
-        + "set correctly", "file:/a", conf.get(MRJobConfig.CLASSPATH_FILES));
-    assertEquals("The mapreduce.job.cache.files property was not set "
-        + "correctly", "file:///a", conf.get(MRJobConfig.CACHE_FILES));
+    assertEquals("file:/a", conf.get(MRJobConfig.CLASSPATH_FILES),
+        "The mapreduce.job.classpath.files property was not set correctly");
+    assertEquals("file:///a", conf.get(MRJobConfig.CACHE_FILES),
+        "The mapreduce.job.cache.files property was not set correctly");
 
     DistributedCache.addFileToClassPath(new Path("file:///b"), conf, fs, true);
-    assertEquals("The mapreduce.job.classpath.files property was not "
-        + "set correctly", "file:/a,file:/b",
-        conf.get(MRJobConfig.CLASSPATH_FILES));
-    assertEquals("The mapreduce.job.cache.files property was not set "
-        + "correctly", "file:///a,file:///b",
-        conf.get(MRJobConfig.CACHE_FILES));
+    assertEquals("file:/a,file:/b", conf.get(MRJobConfig.CLASSPATH_FILES),
+        "The mapreduce.job.classpath.files property was not set correctly");
+    assertEquals("file:///a,file:///b", conf.get(MRJobConfig.CACHE_FILES),
+        "The mapreduce.job.cache.files property was not set correctly");
 
     // And finally with 4th arg false
     conf.clear();
@@ -117,16 +114,15 @@ public class TestDistributedCache {
     }
 
     DistributedCache.addFileToClassPath(new Path("file:///a"), conf, fs, false);
-    assertEquals("The mapreduce.job.classpath.files property was not "
-        + "set correctly", "file:/a", conf.get(MRJobConfig.CLASSPATH_FILES));
-    assertEquals("The mapreduce.job.cache.files property was not set "
-        + "correctly", "", conf.get(MRJobConfig.CACHE_FILES, ""));
+    assertEquals("file:/a", conf.get(MRJobConfig.CLASSPATH_FILES),
+        "The mapreduce.job.classpath.files property was not set correctly");
+    assertEquals("", conf.get(MRJobConfig.CACHE_FILES, ""),
+        "The mapreduce.job.cache.files property was not set correctly");
 
     DistributedCache.addFileToClassPath(new Path("file:///b"), conf, fs, false);
-    assertEquals("The mapreduce.job.classpath.files property was not "
-        + "set correctly", "file:/a,file:/b",
-        conf.get(MRJobConfig.CLASSPATH_FILES));
-    assertEquals("The mapreduce.job.cache.files property was not set "
-        + "correctly", "", conf.get(MRJobConfig.CACHE_FILES, ""));
+    assertEquals("file:/a,file:/b", conf.get(MRJobConfig.CLASSPATH_FILES),
+        "The mapreduce.job.classpath.files property was not set correctly");
+    assertEquals("", conf.get(MRJobConfig.CACHE_FILES, ""),
+        "The mapreduce.job.cache.files property was not set correctly");
   }
 }

+ 17 - 15
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestHistoryViewerPrinter.java

@@ -24,11 +24,9 @@ import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.JobStatus;
 import org.apache.hadoop.mapreduce.TaskType;
-import org.assertj.core.api.Assertions;
-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.skyscreamer.jsonassert.JSONAssert;
 import org.skyscreamer.jsonassert.JSONCompareMode;
 import org.slf4j.Logger;
@@ -42,6 +40,10 @@ import java.util.List;
 import java.util.TimeZone;
 import java.util.Locale;
 
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+
 public class TestHistoryViewerPrinter {
 
   private static final Logger LOG =
@@ -51,12 +53,12 @@ public class TestHistoryViewerPrinter {
 
   private static final Locale DEFAULT_LOCALE = Locale.getDefault();
 
-  @BeforeClass
+  @BeforeAll
   public static void setUp() throws Exception {
     Locale.setDefault(Locale.ENGLISH);
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() throws Exception {
     Locale.setDefault(DEFAULT_LOCALE);
   }
@@ -68,7 +70,7 @@ public class TestHistoryViewerPrinter {
         new HumanReadableHistoryViewerPrinter(job, false, "http://",
             TimeZone.getTimeZone("GMT"));
     String outStr = run(printer);
-    Assert.assertEquals("\n" +
+    assertEquals("\n" +
         "Hadoop job: job_1317928501754_0001\n" +
         "=====================================\n" +
         "User: rkanter\n" +
@@ -167,7 +169,7 @@ public class TestHistoryViewerPrinter {
     final List<String> linesFromStr1 = Arrays.asList(str1.trim().split("\n"));
     final List<String> linesFromStr2 = Arrays.asList(str2.trim().split("\n"));
 
-    Assertions.assertThat(linesFromStr1).containsExactlyInAnyOrderElementsOf(linesFromStr2);
+    assertThat(linesFromStr1).containsExactlyInAnyOrderElementsOf(linesFromStr2);
   }
 
   @Test
@@ -1028,12 +1030,12 @@ public class TestHistoryViewerPrinter {
     // We are not interested in anything but the duplicate counter
     int count1 = outStr.indexOf(
         "|Map-Reduce Framework          |Map input records             |");
-    Assert.assertNotEquals("First counter occurrence not found", -1, count1);
+    assertNotEquals(-1, count1, "First counter occurrence not found");
     int count2 = outStr.indexOf(
         "|Map-Reduce Framework          |Map input records             |",
         count1 + 1);
-    Assert.assertEquals("Duplicate counter found at: " + count1 +
-        " and " + count2, -1, count2);
+    assertEquals(-1, count2, "Duplicate counter found at: " + count1 +
+        " and " + count2);
   }
 
   @Test
@@ -1048,12 +1050,12 @@ public class TestHistoryViewerPrinter {
     // We are not interested in anything but the duplicate counter
     int count1 = outStr.indexOf(
         "\"counterName\":\"MAP_INPUT_RECORDS\"");
-    Assert.assertNotEquals("First counter occurrence not found", -1, count1);
+    assertNotEquals(-1, count1, "First counter occurrence not found");
     int count2 = outStr.indexOf(
         "\"counterName\":\"MAP_INPUT_RECORDS\"",
         count1 + 1);
-    Assert.assertEquals("Duplicate counter found at: " + count1 +
-        " and " + count2, -1, count2);
+    assertEquals(-1, count2, "Duplicate counter found at: " + count1 +
+        " and " + count2);
   }
 
   private String run(HistoryViewerPrinter printer) throws Exception {

+ 4 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/db/DriverForTest.java

@@ -30,7 +30,10 @@ import java.util.Properties;
 import java.util.logging.Logger;
 
 import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.Mockito.*;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 /**
  * class emulates a connection to database

+ 16 - 10
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDbClasses.java

@@ -29,27 +29,31 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.lib.db.DBInputFormat.DBInputSplit;
 import org.apache.hadoop.mapreduce.lib.db.DBInputFormat.NullDBWritable;
 import org.apache.hadoop.mapreduce.lib.db.DataDrivenDBInputFormat.DataDrivenDBInputSplit;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 public class TestDbClasses {
   /**
    * test splitters from DataDrivenDBInputFormat. For different data types may
    * be different splitter
    */
-  @Test(timeout = 10000)
+  @Test
+  @Timeout(value = 10)
   public void testDataDrivenDBInputFormatSplitter() {
-    DataDrivenDBInputFormat<NullDBWritable> format = new DataDrivenDBInputFormat<NullDBWritable>();
+    DataDrivenDBInputFormat<NullDBWritable> format = new DataDrivenDBInputFormat<>();
     testCommonSplitterTypes(format);
-    assertEquals(DateSplitter.class, format.getSplitter(Types.TIMESTAMP)
-        .getClass());
+    assertEquals(DateSplitter.class, format.getSplitter(Types.TIMESTAMP).getClass());
     assertEquals(DateSplitter.class, format.getSplitter(Types.DATE).getClass());
     assertEquals(DateSplitter.class, format.getSplitter(Types.TIME).getClass());
   }
 
-  @Test(timeout = 10000)
+  @Test
+  @Timeout(value = 10)
   public void testDataDrivenDBInputFormat() throws Exception {
     JobContext jobContext = mock(JobContext.class);
     Configuration configuration = new Configuration();
@@ -79,7 +83,8 @@ public class TestDbClasses {
         configuration.get(DBConfiguration.INPUT_BOUNDING_QUERY));
   }
 
-  @Test(timeout = 10000)
+  @Test
+  @Timeout(value = 10)
   public void testOracleDataDrivenDBInputFormat() throws Exception {
     OracleDataDrivenDBInputFormat<NullDBWritable> format = 
         new OracleDataDrivenDBInputFormatForTest();
@@ -96,7 +101,8 @@ public class TestDbClasses {
    * test generate sql script for OracleDBRecordReader.
    */
 
-  @Test(timeout = 20000)
+  @Test
+  @Timeout(value = 20)
   public void testOracleDBRecordReader() throws Exception {
     DBInputSplit splitter = new DBInputSplit(1, 10);
     Configuration configuration = new Configuration();

+ 19 - 13
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestSplitters.java

@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.mapreduce.lib.db;
 
-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 static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -33,8 +33,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.lib.db.DataDrivenDBInputFormat.DataDrivenDBInputSplit;
-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;
 
 /**
  * Test Splitters. Splitters should build parts of sql sentences for split result. 
@@ -43,13 +44,14 @@ public class TestSplitters {
 
   private Configuration configuration;
   
-  @Before
+  @BeforeEach
   public void setup() {
     configuration = new Configuration();
     configuration.setInt(MRJobConfig.NUM_MAPS, 2);
   }
   
-  @Test(timeout=2000)
+  @Test
+  @Timeout(value = 2)
   public void testBooleanSplitter() throws Exception{
     BooleanSplitter splitter = new BooleanSplitter();
     ResultSet result = mock(ResultSet.class);
@@ -77,7 +79,8 @@ public class TestSplitters {
         "column = FALSE column = FALSE", ".*column = TRUE"}, splits);
   }
   
-  @Test(timeout=2000)
+  @Test
+  @Timeout(value = 2)
   public void testFloatSplitter() throws Exception{
     FloatSplitter splitter = new FloatSplitter();
     
@@ -96,7 +99,8 @@ public class TestSplitters {
         "column1 >= 6.0 column1 <= 7.0"}, splits);
   }
 
-  @Test(timeout=2000)
+  @Test
+  @Timeout(value = 2)
   public void testBigDecimalSplitter() throws Exception{
     BigDecimalSplitter splitter = new BigDecimalSplitter();
     ResultSet result = mock(ResultSet.class);
@@ -114,7 +118,8 @@ public class TestSplitters {
         "column1 >= 11 column1 <= 12"}, splits);
   }
 
-  @Test(timeout=2000)
+  @Test
+  @Timeout(value = 2)
   public void testIntegerSplitter() throws Exception{
     IntegerSplitter splitter = new IntegerSplitter();
     ResultSet result = mock(ResultSet.class);
@@ -132,7 +137,8 @@ public class TestSplitters {
         "column1 >= 13 column1 < 18", "column1 >= 18 column1 <= 19"}, splits);
   }
 
-  @Test(timeout=2000)
+  @Test
+  @Timeout(value = 2)
   public void testTextSplitter() throws Exception{
     TextSplitter splitter = new TextSplitter();
     ResultSet result = mock(ResultSet.class);
@@ -154,10 +160,10 @@ public class TestSplitters {
     for (int i = 0; i < expectedSplitRE.length; i++) {
       DataDrivenDBInputSplit split = (DataDrivenDBInputSplit) splits.get(i);
       String actualExpr = split.getLowerClause() + " " + split.getUpperClause();
-      assertTrue("Split #" + (i+1) + " expression is wrong."
+      assertTrue(Pattern.matches(expectedSplitRE[i], actualExpr),
+          "Split #" + (i+1) + " expression is wrong."
           + " Expected " + expectedSplitRE[i]
-          + " Actual " + actualExpr,
-          Pattern.matches(expectedSplitRE[i], actualExpr));
+          + " Actual " + actualExpr);
     }
   }
   

+ 7 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/BaseTestLineRecordReaderBZip2.java

@@ -24,9 +24,9 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.StringJoiner;
 
-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.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -37,7 +37,7 @@ import org.apache.hadoop.io.compress.bzip2.BZip2Utils;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
 import static org.apache.hadoop.io.compress.bzip2.BZip2TextFileWriter.BLOCK_SIZE;
 import static org.apache.hadoop.util.Preconditions.checkArgument;
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 public abstract class BaseTestLineRecordReaderBZip2 {
 
@@ -63,7 +63,7 @@ public abstract class BaseTestLineRecordReaderBZip2 {
     return tempFile;
   }
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     conf = new Configuration();
 
@@ -77,7 +77,7 @@ public abstract class BaseTestLineRecordReaderBZip2 {
     tempFile = new Path(inputDir, "test.txt.bz2");
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     fs.delete(tempFile, /* recursive */ false);
   }
@@ -409,7 +409,7 @@ public abstract class BaseTestLineRecordReaderBZip2 {
     private void assertSplit(SplitRange splitRange) throws IOException {
       String message = splitRange.toString();
       long actual = reader.countRecords(splitRange.start, splitRange.length);
-      assertEquals(message, splitRange.expectedNumRecords, actual);
+      assertEquals(splitRange.expectedNumRecords, actual, message);
     }
   }
 

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileRecordReader.java

@@ -22,7 +22,6 @@ import java.io.FileWriter;
 import java.io.IOException;
 import java.io.File;
 
-import org.junit.Assert;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
@@ -35,8 +34,9 @@ import org.apache.hadoop.mapred.Task.TaskReporter;
 
 import org.mockito.Mockito;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
+import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 
@@ -86,7 +86,7 @@ public class TestCombineFileRecordReader {
       cfrr.initialize(combineFileSplit,taskAttemptContext);
 
       verify(reporter).progress();
-      Assert.assertFalse(cfrr.nextKeyValue());
+      assertFalse(cfrr.nextKeyValue());
       verify(reporter, times(3)).progress();
     } finally {
       FileUtil.fullyDelete(new File(outDir.toString()));

+ 100 - 79
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestFileInputFormat.java

@@ -45,17 +45,21 @@ import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.util.Lists;
 import org.apache.hadoop.util.Sets;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-@RunWith(value = Parameterized.class)
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertInstanceOf;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+
 public class TestFileInputFormat {
   
   private static final Logger LOG =
@@ -68,18 +72,17 @@ public class TestFileInputFormat {
   
   private int numThreads;
   
-  public TestFileInputFormat(int numThreads) {
-    this.numThreads = numThreads;
-    LOG.info("Running with numThreads: " + numThreads);
+  public void initTestFileInputFormat(int pNumThreads) {
+    this.numThreads = pNumThreads;
+    LOG.info("Running with numThreads: " + pNumThreads);
   }
-  
-  @Parameters
+
   public static Collection<Object[]> data() {
-    Object[][] data = new Object[][] { { 1 }, { 5 }};
+    Object[][] data = new Object[][]{{1}, {5}};
     return Arrays.asList(data);
   }
   
-  @Before
+  @BeforeEach
   public void setup() throws IOException {
     LOG.info("Using Test Dir: " + TEST_ROOT_DIR);
     localFs = FileSystem.getLocal(new Configuration());
@@ -87,20 +90,22 @@ public class TestFileInputFormat {
     localFs.mkdirs(TEST_ROOT_DIR);
   }
   
-  @After
+  @AfterEach
   public void cleanup() throws IOException {
     localFs.delete(TEST_ROOT_DIR, true);
   }
 
-  @Test
-  public void testNumInputFilesRecursively() throws Exception {
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testNumInputFilesRecursively(int pNumThreads) throws Exception {
+    initTestFileInputFormat(pNumThreads);
     Configuration conf = getConfiguration();
     conf.set(FileInputFormat.INPUT_DIR_RECURSIVE, "true");
     conf.setInt(FileInputFormat.LIST_STATUS_NUM_THREADS, numThreads);
     Job job = Job.getInstance(conf);
     FileInputFormat<?, ?> fileInputFormat = new TextInputFormat();
     List<InputSplit> splits = fileInputFormat.getSplits(job);
-    Assert.assertEquals("Input splits are not correct", 3, splits.size());
+    assertEquals(3, splits.size(), "Input splits are not correct");
     verifySplits(Lists.newArrayList("test:/a1/a2/file2", "test:/a1/a2/file3",
         "test:/a1/file1"), splits);
 
@@ -113,50 +118,58 @@ public class TestFileInputFormat {
         "test:/a1/file1"), splits);
   }
 
-  @Test
-  public void testNumInputFilesWithoutRecursively() throws Exception {
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testNumInputFilesWithoutRecursively(int pNumThreads) throws Exception {
+    initTestFileInputFormat(pNumThreads);
     Configuration conf = getConfiguration();
     conf.setInt(FileInputFormat.LIST_STATUS_NUM_THREADS, numThreads);
     Job job = Job.getInstance(conf);
     FileInputFormat<?, ?> fileInputFormat = new TextInputFormat();
     List<InputSplit> splits = fileInputFormat.getSplits(job);
-    Assert.assertEquals("Input splits are not correct", 2, splits.size());
+    assertEquals(2, splits.size(), "Input splits are not correct");
     verifySplits(Lists.newArrayList("test:/a1/a2", "test:/a1/file1"), splits);
   }
 
-  @Test
-  public void testNumInputFilesIgnoreDirs() throws Exception {
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testNumInputFilesIgnoreDirs(int pNumThreads) throws Exception {
+    initTestFileInputFormat(pNumThreads);
     Configuration conf = getConfiguration();
     conf.setInt(FileInputFormat.LIST_STATUS_NUM_THREADS, numThreads);
     conf.setBoolean(FileInputFormat.INPUT_DIR_NONRECURSIVE_IGNORE_SUBDIRS, true);
     Job job = Job.getInstance(conf);
     FileInputFormat<?, ?> fileInputFormat = new TextInputFormat();
     List<InputSplit> splits = fileInputFormat.getSplits(job);
-    Assert.assertEquals("Input splits are not correct", 1, splits.size());
+    assertEquals(1, splits.size(), "Input splits are not correct");
     verifySplits(Lists.newArrayList("test:/a1/file1"), splits);
   }
 
-  @Test
-  public void testListLocatedStatus() throws Exception {
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testListLocatedStatus(int pNumThreads) throws Exception {
+    initTestFileInputFormat(pNumThreads);
     Configuration conf = getConfiguration();
     conf.setInt(FileInputFormat.LIST_STATUS_NUM_THREADS, numThreads);
     conf.setBoolean("fs.test.impl.disable.cache", false);
     conf.set(FileInputFormat.INPUT_DIR, "test:///a1/a2");
     MockFileSystem mockFs =
         (MockFileSystem) new Path("test:///").getFileSystem(conf);
-    Assert.assertEquals("listLocatedStatus already called",
-        0, mockFs.numListLocatedStatusCalls);
+    assertEquals(0, mockFs.numListLocatedStatusCalls,
+        "listLocatedStatus already called");
     Job job = Job.getInstance(conf);
     FileInputFormat<?, ?> fileInputFormat = new TextInputFormat();
     List<InputSplit> splits = fileInputFormat.getSplits(job);
-    Assert.assertEquals("Input splits are not correct", 2, splits.size());
-    Assert.assertEquals("listLocatedStatuss calls",
-        1, mockFs.numListLocatedStatusCalls);
+    assertEquals(2, splits.size(), "Input splits are not correct");
+    assertEquals(1, mockFs.numListLocatedStatusCalls,
+        "listLocatedStatus calls");
     FileSystem.closeAll();
   }
-  
-  @Test
-  public void testSplitLocationInfo() throws Exception {
+
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testSplitLocationInfo(int pNumThreads) throws Exception {
+    initTestFileInputFormat(pNumThreads);
     Configuration conf = getConfiguration();
     conf.set(org.apache.hadoop.mapreduce.lib.input.FileInputFormat.INPUT_DIR,
         "test:///a1/a2");
@@ -164,21 +177,23 @@ public class TestFileInputFormat {
     TextInputFormat fileInputFormat = new TextInputFormat();
     List<InputSplit> splits = fileInputFormat.getSplits(job);
     String[] locations = splits.get(0).getLocations();
-    Assert.assertEquals(2, locations.length);
+    assertEquals(2, locations.length);
     SplitLocationInfo[] locationInfo = splits.get(0).getLocationInfo();
-    Assert.assertEquals(2, locationInfo.length);
+    assertEquals(2, locationInfo.length);
     SplitLocationInfo localhostInfo = locations[0].equals("localhost") ?
         locationInfo[0] : locationInfo[1];
     SplitLocationInfo otherhostInfo = locations[0].equals("otherhost") ?
         locationInfo[0] : locationInfo[1];
-    Assert.assertTrue(localhostInfo.isOnDisk());
-    Assert.assertTrue(localhostInfo.isInMemory());
-    Assert.assertTrue(otherhostInfo.isOnDisk());
-    Assert.assertFalse(otherhostInfo.isInMemory());
+    assertTrue(localhostInfo.isOnDisk());
+    assertTrue(localhostInfo.isInMemory());
+    assertTrue(otherhostInfo.isOnDisk());
+    assertFalse(otherhostInfo.isInMemory());
   }
 
-  @Test
-  public void testListStatusSimple() throws IOException {
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testListStatusSimple(int pNumThreads) throws IOException {
+    initTestFileInputFormat(pNumThreads);
     Configuration conf = new Configuration();
     conf.setInt(FileInputFormat.LIST_STATUS_NUM_THREADS, numThreads);
 
@@ -191,8 +206,10 @@ public class TestFileInputFormat {
     verifyFileStatuses(expectedPaths, statuses, localFs);
   }
 
-  @Test
-  public void testListStatusNestedRecursive() throws IOException {
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testListStatusNestedRecursive(int pNumThreads) throws IOException {
+    initTestFileInputFormat(pNumThreads);
     Configuration conf = new Configuration();
     conf.setInt(FileInputFormat.LIST_STATUS_NUM_THREADS, numThreads);
 
@@ -205,8 +222,10 @@ public class TestFileInputFormat {
   }
 
 
-  @Test
-  public void testListStatusNestedNonRecursive() throws IOException {
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testListStatusNestedNonRecursive(int pNumThreads) throws IOException {
+    initTestFileInputFormat(pNumThreads);
     Configuration conf = new Configuration();
     conf.setInt(FileInputFormat.LIST_STATUS_NUM_THREADS, numThreads);
 
@@ -218,8 +237,10 @@ public class TestFileInputFormat {
     verifyFileStatuses(expectedPaths, statuses, localFs);
   }
 
-  @Test
-  public void testListStatusErrorOnNonExistantDir() throws IOException {
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testListStatusErrorOnNonExistantDir(int pNumThreads) throws IOException {
+    initTestFileInputFormat(pNumThreads);
     Configuration conf = new Configuration();
     conf.setInt(FileInputFormat.LIST_STATUS_NUM_THREADS, numThreads);
 
@@ -228,22 +249,24 @@ public class TestFileInputFormat {
     FileInputFormat<?, ?> fif = new TextInputFormat();
     try {
       fif.listStatus(job);
-      Assert.fail("Expecting an IOException for a missing Input path");
+      fail("Expecting an IOException for a missing Input path");
     } catch (IOException e) {
       Path expectedExceptionPath = new Path(TEST_ROOT_DIR, "input2");
       expectedExceptionPath = localFs.makeQualified(expectedExceptionPath);
-      Assert.assertTrue(e instanceof InvalidInputException);
-      Assert.assertEquals(
+      assertInstanceOf(InvalidInputException.class, e);
+      assertEquals(
           "Input path does not exist: " + expectedExceptionPath.toString(),
           e.getMessage());
     }
   }
 
-  @Test
-  public void testShrinkStatus() throws IOException {
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testShrinkStatus(int pNumThreads) throws IOException {
+    initTestFileInputFormat(pNumThreads);
     Configuration conf = getConfiguration();
     MockFileSystem mockFs =
-            (MockFileSystem) new Path("test:///").getFileSystem(conf);
+        (MockFileSystem) new Path("test:///").getFileSystem(conf);
     Path dir1  = new Path("test:/a1");
     RemoteIterator<LocatedFileStatus> statuses = mockFs.listLocatedStatus(dir1);
     boolean verified = false;
@@ -251,36 +274,36 @@ public class TestFileInputFormat {
       LocatedFileStatus orig = statuses.next();
       LocatedFileStatus shrink =
           (LocatedFileStatus)FileInputFormat.shrinkStatus(orig);
-      Assert.assertTrue(orig.equals(shrink));
+      assertTrue(orig.equals(shrink));
       if (shrink.getBlockLocations() != null) {
-        Assert.assertEquals(orig.getBlockLocations().length,
+        assertEquals(orig.getBlockLocations().length,
             shrink.getBlockLocations().length);
         for (int i = 0; i < shrink.getBlockLocations().length; i++) {
           verified = true;
           BlockLocation location = shrink.getBlockLocations()[i];
           BlockLocation actual = orig.getBlockLocations()[i];
-          Assert.assertNotNull(((HdfsBlockLocation)actual).getLocatedBlock());
-          Assert.assertEquals(BlockLocation.class.getName(),
+          assertNotNull(((HdfsBlockLocation)actual).getLocatedBlock());
+          assertEquals(BlockLocation.class.getName(),
               location.getClass().getName());
-          Assert.assertArrayEquals(actual.getHosts(), location.getHosts());
-          Assert.assertArrayEquals(actual.getCachedHosts(),
+          assertArrayEquals(actual.getHosts(), location.getHosts());
+          assertArrayEquals(actual.getCachedHosts(),
               location.getCachedHosts());
-          Assert.assertArrayEquals(actual.getStorageIds(),
+          assertArrayEquals(actual.getStorageIds(),
               location.getStorageIds());
-          Assert.assertArrayEquals(actual.getStorageTypes(),
+          assertArrayEquals(actual.getStorageTypes(),
               location.getStorageTypes());
-          Assert.assertArrayEquals(actual.getTopologyPaths(),
+          assertArrayEquals(actual.getTopologyPaths(),
               location.getTopologyPaths());
-          Assert.assertArrayEquals(actual.getNames(), location.getNames());
-          Assert.assertEquals(actual.getLength(), location.getLength());
-          Assert.assertEquals(actual.getOffset(), location.getOffset());
-          Assert.assertEquals(actual.isCorrupt(), location.isCorrupt());
+          assertArrayEquals(actual.getNames(), location.getNames());
+          assertEquals(actual.getLength(), location.getLength());
+          assertEquals(actual.getOffset(), location.getOffset());
+          assertEquals(actual.isCorrupt(), location.isCorrupt());
         }
       } else {
-        Assert.assertTrue(orig.getBlockLocations() == null);
+        assertTrue(orig.getBlockLocations() == null);
       }
     }
-    Assert.assertTrue(verified);
+    assertTrue(verified);
   }
 
   public static List<Path> configureTestSimple(Configuration conf, FileSystem localFs)
@@ -396,7 +419,7 @@ public class TestFileInputFormat {
 
   public static void verifyFileStatuses(List<Path> expectedPaths,
       List<FileStatus> fetchedStatuses, final FileSystem localFs) {
-    Assert.assertEquals(expectedPaths.size(), fetchedStatuses.size());
+    assertEquals(expectedPaths.size(), fetchedStatuses.size());
 
     Iterable<Path> fqExpectedPaths =
         expectedPaths.stream().map(
@@ -406,12 +429,11 @@ public class TestFileInputFormat {
     Set<Path> expectedPathSet = Sets.newHashSet(fqExpectedPaths);
     for (FileStatus fileStatus : fetchedStatuses) {
       if (!expectedPathSet.remove(localFs.makeQualified(fileStatus.getPath()))) {
-        Assert.fail("Found extra fetched status: " + fileStatus.getPath());
+        fail("Found extra fetched status: " + fileStatus.getPath());
       }
     }
-    Assert.assertEquals(
-        "Not all expectedPaths matched: " + expectedPathSet.toString(), 0,
-        expectedPathSet.size());
+    assertEquals(0, expectedPathSet.size(),
+        "Not all expectedPaths matched: " + expectedPathSet);
   }
 
 
@@ -424,12 +446,11 @@ public class TestFileInputFormat {
     Set<String> expectedSet = Sets.newHashSet(expected);
     for (String splitPathString : pathsFromSplits) {
       if (!expectedSet.remove(splitPathString)) {
-        Assert.fail("Found extra split: " + splitPathString);
+        fail("Found extra split: " + splitPathString);
       }
     }
-    Assert.assertEquals(
-        "Not all expectedPaths matched: " + expectedSet.toString(), 0,
-        expectedSet.size());
+    assertEquals(0, expectedSet.size(),
+        "Not all expectedPaths matched: " + expectedSet);
   }
   
   private Configuration getConfiguration() {

+ 65 - 68
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestLineRecordReader.java

@@ -18,11 +18,12 @@
 
 package org.apache.hadoop.mapreduce.lib.input;
 
-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.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.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.File;
 import java.io.FileInputStream;
@@ -47,7 +48,7 @@ import org.apache.hadoop.io.compress.Decompressor;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 public class TestLineRecordReader {
   private static Path workDir = new Path(new Path(System.getProperty(
@@ -57,7 +58,7 @@ public class TestLineRecordReader {
   private void testSplitRecords(String testFileName, long firstSplitLength)
       throws IOException {
     URL testFileUrl = getClass().getClassLoader().getResource(testFileName);
-    assertNotNull("Cannot find " + testFileName, testFileUrl);
+    assertNotNull(testFileUrl, "Cannot find " + testFileName);
     File testFile = new File(testFileUrl.getFile());
     long testFileSize = testFile.length();
     Path testFilePath = new Path(testFile.getAbsolutePath());
@@ -70,8 +71,8 @@ public class TestLineRecordReader {
       throws IOException {
     conf.setInt(org.apache.hadoop.mapreduce.lib.input.
         LineRecordReader.MAX_LINE_LENGTH, Integer.MAX_VALUE);
-    assertTrue("unexpected test data at " + testFilePath,
-        testFileSize > firstSplitLength);
+    assertTrue(testFileSize > firstSplitLength,
+        "unexpected test data at " + testFilePath);
 
     String delimiter = conf.get("textinputformat.record.delimiter");
     byte[] recordDelimiterBytes = null;
@@ -111,8 +112,8 @@ public class TestLineRecordReader {
       ++numRecordsRemainingSplits;
     }
     reader.close();
-    assertEquals("Unexpected number of records in split ", numRecordsNoSplits,
-        numRecordsFirstSplit + numRecordsRemainingSplits);
+    assertEquals(numRecordsNoSplits, numRecordsFirstSplit + numRecordsRemainingSplits,
+        "Unexpected number of records in split ");
   }
 
   @Test
@@ -131,11 +132,12 @@ public class TestLineRecordReader {
     testSplitRecords("blockEndingInCRThenLF.txt.bz2", 136498);
   }
 
-  @Test(expected=IOException.class)
+  @Test()
   public void testSafeguardSplittingUnsplittableFiles() throws IOException {
     // The LineRecordReader must fail when trying to read a file that
     // was compressed using an unsplittable file format
-    testSplitRecords("TestSafeguardSplittingUnsplittableFiles.txt.gz", 2);
+    assertThrows(IOException.class, () ->
+        testSplitRecords("TestSafeguardSplittingUnsplittableFiles.txt.gz", 2));
   }
 
   // Use the LineRecordReader to read records from the file
@@ -182,7 +184,7 @@ public class TestLineRecordReader {
       count = fis.read(data);
     }
     fis.close();
-    assertTrue("Test file data too big for buffer", count < data.length);
+    assertTrue(count < data.length, "Test file data too big for buffer");
     return new String(data, 0, count, StandardCharsets.UTF_8).split("\n");
   }
 
@@ -194,7 +196,7 @@ public class TestLineRecordReader {
     ArrayList<String> records = readRecords(testFileUrl, splitSize);
     String[] actuals = readRecordsDirectly(testFileUrl, bzip);
 
-    assertEquals("Wrong number of records", actuals.length, records.size());
+    assertEquals(actuals.length, records.size(), "Wrong number of records");
 
     boolean hasLargeRecord = false;
     for (int i = 0; i < actuals.length; ++i) {
@@ -204,8 +206,8 @@ public class TestLineRecordReader {
       }
     }
 
-    assertTrue("Invalid test data. Doesn't have a large enough record",
-               hasLargeRecord);
+    assertTrue(hasLargeRecord,
+        "Invalid test data. Doesn't have a large enough record");
   }
 
   @Test
@@ -233,7 +235,7 @@ public class TestLineRecordReader {
     // confirm the BOM is skipped by LineRecordReader
     String UTF8_BOM = "\uFEFF";
     URL testFileUrl = getClass().getClassLoader().getResource("testBOM.txt");
-    assertNotNull("Cannot find testBOM.txt", testFileUrl);
+    assertNotNull(testFileUrl, "Cannot find testBOM.txt");
     File testFile = new File(testFileUrl.getFile());
     Path testFilePath = new Path(testFile.getAbsolutePath());
     long testFileSize = testFile.length();
@@ -262,15 +264,14 @@ public class TestLineRecordReader {
     }
     reader.close();
 
-    assertTrue("BOM is not skipped", skipBOM);
+    assertTrue(skipBOM, "BOM is not skipped");
   }
 
   @Test
   public void testMultipleClose() throws IOException {
     URL testFileUrl = getClass().getClassLoader().
         getResource("recordSpanningMultipleSplits.txt.bz2");
-    assertNotNull("Cannot find recordSpanningMultipleSplits.txt.bz2",
-        testFileUrl);
+    assertNotNull(testFileUrl, "Cannot find recordSpanningMultipleSplits.txt.bz2");
     File testFile = new File(testFileUrl.getFile());
     Path testFilePath = new Path(testFile.getAbsolutePath());
     long testFileSize = testFile.length();
@@ -424,27 +425,27 @@ public class TestLineRecordReader {
     LineRecordReader reader = new LineRecordReader(recordDelimiterBytes);
     reader.initialize(split, context);
     // Get first record: "abcdefghij"
-    assertTrue("Expected record got nothing", reader.nextKeyValue());
+    assertTrue(reader.nextKeyValue(), "Expected record got nothing");
     LongWritable key = reader.getCurrentKey();
     Text value = reader.getCurrentValue();
-    assertEquals("Wrong length for record value", 10, value.getLength());
-    assertEquals("Wrong position after record read", 0, key.get());
+    assertEquals(10, value.getLength(), "Wrong length for record value");
+    assertEquals(0, key.get(), "Wrong position after record read");
     // Get second record: "kl"
-    assertTrue("Expected record got nothing", reader.nextKeyValue());
-    assertEquals("Wrong length for record value", 2, value.getLength());
+    assertTrue(reader.nextKeyValue(), "Expected record got nothing");
+    assertEquals(2, value.getLength(), "Wrong length for record value");
     // Key should be 12 right after "abcdefghij++"
-    assertEquals("Wrong position after record read", 12, key.get());
+    assertEquals(12, key.get(), "Wrong position after record read");
     // Get third record: "mno"
-    assertTrue("Expected record got nothing", reader.nextKeyValue());
-    assertEquals("Wrong length for record value", 3, value.getLength());
+    assertTrue(reader.nextKeyValue(), "Expected record got nothing");
+    assertEquals(3, value.getLength(), "Wrong length for record value");
     // Key should be 16 right after "abcdefghij++kl++"
-    assertEquals("Wrong position after record read", 16, key.get());
+    assertEquals(16, key.get(), "Wrong position after record read");
     assertFalse(reader.nextKeyValue());
     // Key should be 19 right after "abcdefghij++kl++mno"
-    assertEquals("Wrong position after record read", 19, key.get());
+    assertEquals(19, key.get(), "Wrong position after record read");
     // after refresh should be empty
     key = reader.getCurrentKey();
-    assertNull("Unexpected key returned", key);
+    assertNull(key, "Unexpected key returned");
     reader.close();
     split = new FileSplit(inputFile, splitLength,
         inputData.length() - splitLength, (String[])null);
@@ -452,9 +453,9 @@ public class TestLineRecordReader {
     reader.initialize(split, context);
     // No record is in the second split because the second split dropped
     // the first record, which was already reported by the first split.
-    assertFalse("Unexpected record returned", reader.nextKeyValue());
+    assertFalse(reader.nextKeyValue(), "Unexpected record returned");
     key = reader.getCurrentKey();
-    assertNull("Unexpected key returned", key);
+    assertNull(key, "Unexpected key returned");
     reader.close();
 
     // multi char delimiter with starting part of the delimiter in the data
@@ -465,36 +466,35 @@ public class TestLineRecordReader {
     reader = new LineRecordReader(recordDelimiterBytes);
     reader.initialize(split, context);
     // Get first record: "abcd+efgh"
-    assertTrue("Expected record got nothing", reader.nextKeyValue());
+    assertTrue(reader.nextKeyValue(), "Expected record got nothing");
     key = reader.getCurrentKey();
     value = reader.getCurrentValue();
-    assertEquals("Wrong position after record read", 0, key.get());
-    assertEquals("Wrong length for record value", 9, value.getLength());
+    assertEquals(0, key.get(), "Wrong position after record read");
+    assertEquals(9, value.getLength(), "Wrong length for record value");
     // should have jumped over the delimiter, no record
     assertFalse(reader.nextKeyValue());
-    assertEquals("Wrong position after record read", 11, key.get());
+    assertEquals(11, key.get(), "Wrong position after record read");
     // after refresh should be empty
     key = reader.getCurrentKey();
-    assertNull("Unexpected key returned", key);
+    assertNull(key, "Unexpected key returned");
     reader.close();
     // next split: check for duplicate or dropped records
-    split = new FileSplit(inputFile, splitLength,
-        inputData.length () - splitLength, (String[])null);
+    split = new FileSplit(inputFile, splitLength, inputData.length() - splitLength, null);
     reader = new LineRecordReader(recordDelimiterBytes);
     reader.initialize(split, context);
-    assertTrue("Expected record got nothing", reader.nextKeyValue());
+    assertTrue(reader.nextKeyValue(), "Expected record got nothing");
     key = reader.getCurrentKey();
     value = reader.getCurrentValue();
     // Get second record: "ijk" first in this split
-    assertEquals("Wrong position after record read", 11, key.get());
-    assertEquals("Wrong length for record value", 3, value.getLength());
+    assertEquals(11, key.get(), "Wrong position after record read");
+    assertEquals(3, value.getLength(), "Wrong length for record value");
     // Get third record: "mno" second in this split
-    assertTrue("Expected record got nothing", reader.nextKeyValue());
-    assertEquals("Wrong position after record read", 16, key.get());
-    assertEquals("Wrong length for record value", 3, value.getLength());
+    assertTrue(reader.nextKeyValue(), "Expected record got nothing");
+    assertEquals(16, key.get(), "Wrong position after record read");
+    assertEquals(3, value.getLength(), "Wrong length for record value");
     // should be at the end of the input
     assertFalse(reader.nextKeyValue());
-    assertEquals("Wrong position after record read", 19, key.get());
+    assertEquals(19, key.get(), "Wrong position after record read");
     reader.close();
 
     inputData = "abcd|efgh|+|ij|kl|+|mno|pqr";
@@ -508,44 +508,41 @@ public class TestLineRecordReader {
         // track where we are in the inputdata
         int keyPosition = 0;
         conf.setInt("io.file.buffer.size", bufferSize);
-        split = new FileSplit(inputFile, 0, bufferSize, (String[]) null);
+        split = new FileSplit(inputFile, 0, bufferSize, null);
         reader = new LineRecordReader(recordDelimiterBytes);
         reader.initialize(split, context);
         // Get the first record: "abcd|efgh" always possible
-        assertTrue("Expected record got nothing", reader.nextKeyValue());
+        assertTrue(reader.nextKeyValue(), "Expected record got nothing");
         key = reader.getCurrentKey();
         value = reader.getCurrentValue();
-        assertTrue("abcd|efgh".equals(value.toString()));
+        assertEquals("abcd|efgh", value.toString());
         // Position should be 0 right at the start
-        assertEquals("Wrong position after record read", keyPosition,
-            key.get());
+        assertEquals(keyPosition, key.get(), "Wrong position after record read");
         // Position should be 12 right after the first "|+|"
         keyPosition = 12;
         // get the next record: "ij|kl" if the split/buffer allows it
         if (reader.nextKeyValue()) {
           // check the record info: "ij|kl"
-          assertTrue("ij|kl".equals(value.toString()));
-          assertEquals("Wrong position after record read", keyPosition,
-              key.get());
+          assertEquals("ij|kl", value.toString());
+          assertEquals(keyPosition, key.get(),
+              "Wrong position after record read");
           // Position should be 20 after the second "|+|"
           keyPosition = 20;
         }
         // get the third record: "mno|pqr" if the split/buffer allows it
         if (reader.nextKeyValue()) {
           // check the record info: "mno|pqr"
-          assertTrue("mno|pqr".equals(value.toString()));
-          assertEquals("Wrong position after record read", keyPosition,
-              key.get());
+          assertEquals("mno|pqr", value.toString());
+          assertEquals(keyPosition, key.get(), "Wrong position after record read");
           // Position should be the end of the input
           keyPosition = inputData.length();
         }
-        assertFalse("Unexpected record returned", reader.nextKeyValue());
+        assertFalse(reader.nextKeyValue(), "Unexpected record returned");
         // no more records can be read we should be at the last position
-        assertEquals("Wrong position after record read", keyPosition,
-            key.get());
+        assertEquals(keyPosition, key.get(), "Wrong position after record read");
         // after refresh should be empty
         key = reader.getCurrentKey();
-        assertNull("Unexpected key returned", key);
+        assertNull(key, "Unexpected key returned");
         reader.close();
       }
     }
@@ -628,14 +625,14 @@ public class TestLineRecordReader {
     // (833 bytes is the last block start in the used data file)
     int firstSplitLength = 100;
     URL testFileUrl = getClass().getClassLoader().getResource(testFileName);
-    assertNotNull("Cannot find " + testFileName, testFileUrl);
+    assertNotNull(testFileUrl, "Cannot find " + testFileName);
     File testFile = new File(testFileUrl.getFile());
     long testFileSize = testFile.length();
     Path testFilePath = new Path(testFile.getAbsolutePath());
-    assertTrue("Split size is smaller than header length",
-        firstSplitLength > 9);
-    assertTrue("Split size is larger than compressed file size " +
-        testFilePath, testFileSize > firstSplitLength);
+    assertTrue(firstSplitLength > 9, "Split size is smaller than header length");
+    assertTrue(testFileSize > firstSplitLength,
+        "Split size is larger than compressed file size " +
+        testFilePath);
 
     Configuration conf = new Configuration();
     conf.setInt(org.apache.hadoop.mapreduce.lib.input.

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestJobControl.java

@@ -17,12 +17,12 @@
  */
 package org.apache.hadoop.mapreduce.lib.jobcontrol;
 
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 public class TestJobControl {
 

+ 38 - 37
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java

@@ -28,13 +28,10 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import static org.junit.Assert.*;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.apache.hadoop.util.concurrent.HadoopExecutors;
-import org.junit.Assert;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -60,6 +57,11 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.assertj.core.api.Assertions.assertThat;
+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.atLeast;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.verify;
@@ -97,12 +99,12 @@ public class TestFileOutputCommitter {
     fs.delete(outDir, true);
   }
   
-  @Before
+  @BeforeEach
   public void setUp() throws IOException {
     cleanup();
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws IOException {
     cleanup();
   }
@@ -168,10 +170,10 @@ public class TestFileOutputCommitter {
     Path jobTempDir1 = committer.getCommittedTaskPath(tContext);
     File jtd = new File(jobTempDir1.toUri().getPath());
     if (commitVersion == 1) {
-      assertTrue("Version 1 commits to temporary dir " + jtd, jtd.exists());
+      assertTrue(jtd.exists(), "Version 1 commits to temporary dir " + jtd);
       validateContent(jtd);
     } else {
-      assertFalse("Version 2 commits to output dir " + jtd, jtd.exists());
+      assertFalse(jtd.exists(), "Version 2 commits to output dir " + jtd);
     }
 
     //now while running the second app attempt, 
@@ -190,13 +192,12 @@ public class TestFileOutputCommitter {
 
     committer2.recoverTask(tContext2);
     if (recoveryVersion == 1) {
-      assertTrue("Version 1 recovers to " + jtd2, jtd2.exists());
+      assertTrue(jtd2.exists(), "Version 1 recovers to " + jtd2);
       validateContent(jtd2);
     } else {
-      assertFalse("Version 2 commits to output dir " + jtd2, jtd2.exists());
+      assertFalse(jtd2.exists(), "Version 2 commits to output dir " + jtd2);
       if (commitVersion == 1) {
-        assertTrue("Version 2  recovery moves to output dir from "
-            + jtd , jtd.list().length == 0);
+        assertEquals(0, jtd.list().length, "Version 2  recovery moves to output dir from " + jtd);
       }
     }
 
@@ -226,7 +227,7 @@ public class TestFileOutputCommitter {
   
   private void validateContent(File dir) throws IOException {
     File expectedFile = new File(dir, partFile);
-    assertTrue("Could not find "+expectedFile, expectedFile.exists());
+    assertTrue(expectedFile.exists(), "Could not find "+expectedFile);
     StringBuilder expectedOutput = new StringBuilder();
     expectedOutput.append(key1).append('\t').append(val1).append("\n");
     expectedOutput.append(val1).append("\n");
@@ -292,25 +293,25 @@ public class TestFileOutputCommitter {
         new Path(outDir, FileOutputCommitter.PENDING_DIR_NAME).toString());
     File taskOutputDir = new File(Path.getPathWithoutSchemeAndAuthority(
         committer.getWorkPath()).toString());
-    assertTrue("job temp dir does not exist", jobOutputDir.exists());
-    assertTrue("task temp dir does not exist", taskOutputDir.exists());
+    assertTrue(jobOutputDir.exists(), "job temp dir does not exist");
+    assertTrue(taskOutputDir.exists(), "task temp dir does not exist");
 
     // do commit
     committer.commitTask(tContext);
-    assertTrue("job temp dir does not exist", jobOutputDir.exists());
+    assertTrue(jobOutputDir.exists(), "job temp dir does not exist");
     if (version == 1 || taskCleanup) {
       // Task temp dir gets renamed in v1 and deleted if taskCleanup is
       // enabled in v2
-      assertFalse("task temp dir still exists", taskOutputDir.exists());
+      assertFalse(taskOutputDir.exists(), "task temp dir still exists");
     } else {
       // By default, in v2 the task temp dir is only deleted during commitJob
-      assertTrue("task temp dir does not exist", taskOutputDir.exists());
+      assertTrue(taskOutputDir.exists(), "task temp dir does not exist");
     }
 
     // Entire job temp directory gets deleted, including task temp dir
     committer.commitJob(jContext);
-    assertFalse("job temp dir still exists", jobOutputDir.exists());
-    assertFalse("task temp dir still exists", taskOutputDir.exists());
+    assertFalse(jobOutputDir.exists(), "job temp dir still exists");
+    assertFalse(taskOutputDir.exists(), "task temp dir still exists");
 
     // validate output
     validateContent(outDir);
@@ -374,11 +375,11 @@ public class TestFileOutputCommitter {
     try {
       committer.commitJob(jContext);
       if (version == 1) {
-        Assert.fail("Duplicate commit success: wrong behavior for version 1.");
+        fail("Duplicate commit success: wrong behavior for version 1.");
       }
     } catch (IOException e) {
       if (version == 2) {
-        Assert.fail("Duplicate commit failed: wrong behavior for version 2.");
+        fail("Duplicate commit failed: wrong behavior for version 2.");
       }
     }
     FileUtil.fullyDelete(new File(outDir.toString()));
@@ -428,12 +429,12 @@ public class TestFileOutputCommitter {
       committer.commitJob(jContext);
       // (1,1), (1,2), (2,1) shouldn't reach to here.
       if (version == 1 || maxAttempts <= 1) {
-        Assert.fail("Commit successful: wrong behavior for version 1.");
+        fail("Commit successful: wrong behavior for version 1.");
       }
     } catch (IOException e) {
       // (2,2) shouldn't reach to here.
       if (version == 2 && maxAttempts > 2) {
-        Assert.fail("Commit failed: wrong behavior for version 2.");
+        fail("Commit failed: wrong behavior for version 2.");
       }
     }
 
@@ -511,7 +512,7 @@ public class TestFileOutputCommitter {
 
     try {
       committer.commitJob(jContext);
-      Assert.fail("Commit successful: wrong behavior for the first time " +
+      fail("Commit successful: wrong behavior for the first time " +
           "commit.");
     } catch (IOException e) {
       // commit again.
@@ -519,12 +520,12 @@ public class TestFileOutputCommitter {
         committer.commitJob(jContext);
         // version 1 shouldn't reach to here.
         if (version == 1) {
-          Assert.fail("Commit successful after retry: wrong behavior for " +
+          fail("Commit successful after retry: wrong behavior for " +
               "version 1.");
         }
       } catch (FileNotFoundException ex) {
         if (version == 2) {
-          Assert.fail("Commit failed after retry: wrong behavior for" +
+          fail("Commit failed after retry: wrong behavior for" +
               " version 2.");
         }
         assertTrue(ex.getMessage().contains(committer.getJobAttemptPath(
@@ -624,14 +625,14 @@ public class TestFileOutputCommitter {
     committer.abortTask(tContext);
     File expectedFile = new File(new Path(committer.getWorkPath(), partFile)
         .toString());
-    assertFalse("task temp dir still exists", expectedFile.exists());
+    assertFalse(expectedFile.exists(), "task temp dir still exists");
 
     committer.abortJob(jContext, JobStatus.State.FAILED);
     expectedFile = new File(new Path(outDir, FileOutputCommitter.PENDING_DIR_NAME)
         .toString());
-    assertFalse("job temp dir still exists", expectedFile.exists());
-    assertEquals("Output directory not empty", 0, new File(outDir.toString())
-        .listFiles().length);
+    assertFalse(expectedFile.exists(), "job temp dir still exists");
+    assertEquals(0, new File(outDir.toString())
+        .listFiles().length, "Output directory not empty");
     FileUtil.fullyDelete(new File(outDir.toString()));
   }
 
@@ -701,7 +702,7 @@ public class TestFileOutputCommitter {
     Path ttd = committer.getTaskAttemptPath(tContext);
     File taskTmpDir = new File(ttd.toUri().getPath());
     File expectedFile = new File(taskTmpDir, partFile);
-    assertTrue(expectedFile + " does not exists", expectedFile.exists());
+    assertTrue(expectedFile.exists(), expectedFile + " does not exists");
 
     th = null;
     try {
@@ -712,7 +713,7 @@ public class TestFileOutputCommitter {
     assertNotNull(th);
     assertTrue(th instanceof IOException);
     assertTrue(th.getMessage().contains("fake delete failed"));
-    assertTrue("job temp dir does not exists", jobTmpDir.exists());
+    assertTrue(jobTmpDir.exists(), "job temp dir does not exists");
     FileUtil.fullyDelete(new File(outDir.toString()));
   }
 
@@ -814,8 +815,8 @@ public class TestFileOutputCommitter {
       amCommitter.commitJob(jContext);
       final RawLocalFileSystem lfs = new RawLocalFileSystem();
       lfs.setConf(conf);
-      assertFalse("Must not end up with sub_dir/sub_dir",
-          lfs.exists(new Path(OUT_SUB_DIR, SUB_DIR)));
+      assertFalse(lfs.exists(new Path(OUT_SUB_DIR, SUB_DIR)),
+          "Must not end up with sub_dir/sub_dir");
 
       // validate output
       validateContent(OUT_SUB_DIR);

+ 3 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputFormat.java

@@ -19,8 +19,9 @@
 package org.apache.hadoop.mapreduce.lib.output;
 
 import java.io.IOException;
-import org.junit.Test;
-import static org.junit.Assert.*;
+import org.junit.jupiter.api.Test;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;

+ 4 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMapFileOutputFormat.java

@@ -18,15 +18,15 @@
 
 package org.apache.hadoop.mapreduce.lib.output;
 
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.MapFile.Reader;
 import org.apache.hadoop.mapreduce.Partitioner;
-import org.junit.After;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
 import org.mockito.Mockito;
 
 public class TestMapFileOutputFormat {
@@ -42,7 +42,7 @@ public class TestMapFileOutputFormat {
     assertTrue(!MyPartitioner.isGetPartitionCalled());
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     MyPartitioner.setGetPartitionCalled(false);
   }

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestPathOutputCommitter.java

@@ -21,8 +21,8 @@ package org.apache.hadoop.mapreduce.lib.output;
 import java.io.IOException;
 import java.net.URI;
 
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -44,7 +44,7 @@ import org.apache.hadoop.security.Credentials;
 /**
  * Test the path output committer binding to FileOutputFormat.
  */
-public class TestPathOutputCommitter extends Assert {
+public class TestPathOutputCommitter extends Assertions {
 
   @Test
   public void testFileOutputCommitterOverrride() throws Throwable {

+ 13 - 17
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestPathOutputCommitterFactory.java

@@ -20,8 +20,8 @@ package org.apache.hadoop.mapreduce.lib.output;
 
 import java.io.IOException;
 
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -39,7 +39,7 @@ import static org.apache.hadoop.test.LambdaTestUtils.intercept;
  * and fallback behavior.
  */
 @SuppressWarnings("unchecked")
-public class TestPathOutputCommitterFactory extends Assert {
+public class TestPathOutputCommitterFactory extends Assertions {
 
   private static final String HTTP_COMMITTER_FACTORY = String.format(
       COMMITTER_FACTORY_SCHEME_PATTERN, "http");
@@ -156,9 +156,7 @@ public class TestPathOutputCommitterFactory extends Assert {
     SimpleCommitter sc = createCommitter(
         NamedCommitterFactory.class,
         SimpleCommitter.class, HDFS_PATH, conf);
-    assertEquals("Wrong output path from " + sc,
-        HDFS_PATH,
-        sc.getOutputPath());
+    assertEquals(HDFS_PATH, sc.getOutputPath(), "Wrong output path from " + sc);
   }
 
   /**
@@ -214,8 +212,8 @@ public class TestPathOutputCommitterFactory extends Assert {
     T f = createCommitterFactory(factoryClass, path, conf);
     PathOutputCommitter committer = f.createOutputCommitter(path,
         taskAttempt(conf));
-    assertEquals(" Wrong committer for path " + path + " from factory " + f,
-        committerClass, committer.getClass());
+    assertEquals(committerClass, committer.getClass(),
+        " Wrong committer for path " + path + " from factory " + f);
     return (U) committer;
   }
 
@@ -235,8 +233,8 @@ public class TestPathOutputCommitterFactory extends Assert {
       TaskAttemptContext context) throws IOException {
     PathOutputCommitter committer = PathOutputCommitterFactory
         .createCommitter(path, context);
-    assertEquals(" Wrong committer for path " + path,
-        committerClass, committer.getClass());
+    assertEquals(committerClass, committer.getClass(),
+        " Wrong committer for path " + path);
     return (U) committer;
   }
 
@@ -253,8 +251,8 @@ public class TestPathOutputCommitterFactory extends Assert {
       Path path,
       Configuration conf) {
     PathOutputCommitterFactory factory = getCommitterFactory(path, conf);
-    assertEquals(" Wrong factory for path " + path,
-        factoryClass, factory.getClass());
+    assertEquals(factoryClass, factory.getClass(),
+        " Wrong factory for path " + path);
     return (T)factory;
   }
 
@@ -295,9 +293,8 @@ public class TestPathOutputCommitterFactory extends Assert {
     TextOutputFormat<String, String> off = new TextOutputFormat<>();
     SimpleCommitter committer = (SimpleCommitter)
         off.getOutputCommitter(taskAttempt(conf));
-    assertEquals("Wrong output path from "+ committer,
-        HTTP_PATH,
-        committer.getOutputPath());
+    assertEquals(HTTP_PATH,
+        committer.getOutputPath(), "Wrong output path from "+ committer);
   }
 
   /**
@@ -316,8 +313,7 @@ public class TestPathOutputCommitterFactory extends Assert {
     TextOutputFormat<String, String> off = new TextOutputFormat<>();
     SimpleCommitter committer = (SimpleCommitter)
         off.getOutputCommitter(taskAttempt(conf));
-    assertNull("Output path from "+ committer,
-        committer.getOutputPath());
+    assertNull(committer.getOutputPath(), "Output path from "+ committer);
   }
 
   /**

+ 10 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestPreemptableFileOutputCommitter.java

@@ -22,8 +22,16 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 
 import org.apache.hadoop.mapreduce.task.annotation.Checkpointable;
-import org.junit.Test;
-import static org.mockito.Mockito.*;
+import org.junit.jupiter.api.Test;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestRehashPartitioner.java

@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.mapreduce.lib.partition;
 
-import static org.junit.Assert.*;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.util.Arrays;
 import java.util.Collections;
@@ -26,7 +26,7 @@ import org.apache.commons.lang3.ArrayUtils;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.NullWritable;
 
-import org.junit.*;
+import org.junit.jupiter.api.Test;
 
 public class TestRehashPartitioner {
 
@@ -66,6 +66,6 @@ public class TestRehashPartitioner {
         badbuckets++;
     }
     System.out.println(badbuckets + " of "+PARTITIONS+" are too small or large buckets");
-    assertTrue("too many overflow buckets", badbuckets < PARTITIONS * MAX_BADBUCKETS);
+    assertTrue(badbuckets < PARTITIONS * MAX_BADBUCKETS, "too many overflow buckets");
   }
 }

+ 13 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/security/TestTokenCache.java

@@ -19,9 +19,16 @@
 package org.apache.hadoop.mapreduce.security;
 
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
+import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.net.URI;
@@ -38,8 +45,8 @@ import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
@@ -47,7 +54,7 @@ public class TestTokenCache {
   private static Configuration conf;
   private static String renewer;
   
-  @BeforeClass
+  @BeforeAll
   public static void setup() throws Exception {
     conf = new Configuration();
     conf.set(YarnConfiguration.RM_PRINCIPAL, "mapred/host@REALM");
@@ -206,6 +213,6 @@ public class TestTokenCache {
     TokenCache.obtainTokensForNamenodesInternal(fs1, creds, conf, renewer);
     String fs_addr = fs1.getCanonicalServiceName();
     Token<?> nnt = TokenCache.getDelegationToken(creds, fs_addr);
-    assertNotNull("Token for nn is null", nnt);
+    assertNotNull(nnt, "Token for nn is null");
   }
 }

+ 8 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/split/TestJobSplitWriter.java

@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.mapreduce.split;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.io.File;
 
@@ -29,7 +29,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 public class TestJobSplitWriter {
 
@@ -52,9 +52,9 @@ public class TestJobSplitWriter {
       JobSplit.TaskSplitMetaInfo[] infos =
           SplitMetaInfoReader.readSplitMetaInfo(new JobID(), fs, conf,
               submitDir);
-      assertEquals("unexpected number of splits", 1, infos.length);
-      assertEquals("unexpected number of split locations",
-          4, infos[0].getLocations().length);
+      assertEquals(1, infos.length, "unexpected number of splits");
+      assertEquals(4, infos[0].getLocations().length,
+          "unexpected number of split locations");
     } finally {
       FileUtil.fullyDelete(TEST_DIR);
     }
@@ -76,9 +76,9 @@ public class TestJobSplitWriter {
       JobSplit.TaskSplitMetaInfo[] infos =
           SplitMetaInfoReader.readSplitMetaInfo(new JobID(), fs, conf,
               submitDir);
-      assertEquals("unexpected number of splits", 1, infos.length);
-      assertEquals("unexpected number of split locations",
-          4, infos[0].getLocations().length);
+      assertEquals(1, infos.length, "unexpected number of splits");
+      assertEquals(4, infos[0].getLocations().length,
+          "unexpected number of split locations");
     } finally {
       FileUtil.fullyDelete(TEST_DIR);
     }

+ 8 - 9
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/split/TestJobSplitWriterWithEC.java

@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.mapreduce.split;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.io.File;
 import java.io.IOException;
@@ -40,9 +40,9 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
-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;
 
 
 /**
@@ -60,7 +60,7 @@ public class TestJobSplitWriterWithEC {
   private Path submitDir;
   private Path testFile;
 
-  @Before
+  @BeforeEach
   public void setup() throws Exception {
     Configuration hdfsConf = new HdfsConfiguration();
     hdfsConf.setLong(DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_KEY, 0);
@@ -84,7 +84,7 @@ public class TestJobSplitWriterWithEC {
         fs.getUri().toString() + testFile.toString());
   }
 
-  @After
+  @AfterEach
   public void after() {
     cluster.close();
   }
@@ -121,8 +121,7 @@ public class TestJobSplitWriterWithEC {
         SplitMetaInfoReader.readSplitMetaInfo(new JobID(), fs, conf,
             submitDir);
 
-    assertEquals("Number of splits", 1, splitInfo.length);
-    assertEquals("Number of block locations", 14,
-        splitInfo[0].getLocations().length);
+    assertEquals(1, splitInfo.length, "Number of splits");
+    assertEquals(14, splitInfo[0].getLocations().length, "Number of block locations");
   }
 }

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestEventFetcher.java

@@ -36,7 +36,7 @@ import org.apache.hadoop.mapred.TaskAttemptID;
 import org.apache.hadoop.mapred.TaskCompletionEvent;
 import org.apache.hadoop.mapred.TaskUmbilicalProtocol;
 import org.apache.hadoop.mapreduce.TaskType;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.mockito.InOrder;
 
 public class TestEventFetcher {

+ 54 - 32
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestFetcher.java

@@ -26,14 +26,31 @@ import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.MRJobConfig;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.rules.TestName;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+import org.junit.jupiter.api.TestInfo;
 
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
+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.anyBoolean;
+import static org.mockito.Mockito.anyInt;
+import static org.mockito.Mockito.anyLong;
+import static org.mockito.Mockito.anyString;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -58,7 +75,6 @@ import org.apache.hadoop.mapreduce.security.SecureShuffleUtils;
 import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.Time;
-import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 import org.slf4j.Logger;
@@ -87,12 +103,10 @@ public class TestFetcher {
   final TaskAttemptID map2ID = TaskAttemptID.forName("attempt_0_1_m_2_1");
   FileSystem fs = null;
 
-  @Rule public TestName name = new TestName();
-
-  @Before
+  @BeforeEach
   @SuppressWarnings("unchecked") // mocked generics
-  public void setup() {
-    LOG.info(">>>> " + name.getMethodName());
+  public void setup(TestInfo testInfo) {
+    LOG.info(">>>> " + testInfo.getDisplayName());
     // to avoid threading issues with JUnit 4.13+
     ReadaheadPool.resetInstance();
     job = new JobConf();
@@ -117,11 +131,11 @@ public class TestFetcher {
     when(ss.getMapsForHost(host)).thenReturn(maps);
   }
 
-  @After
-  public void teardown() throws IllegalArgumentException, IOException {
-    LOG.info("<<<< " + name.getMethodName());
+  @AfterEach
+  public void teardown(TestInfo testInfo) throws IllegalArgumentException, IOException {
+    LOG.info("<<<< " + testInfo.getDisplayName());
     if (fs != null) {
-      fs.delete(new Path(name.getMethodName()),true);
+      fs.delete(new Path(testInfo.getDisplayName()), true);
     }
   }
   
@@ -155,7 +169,8 @@ public class TestFetcher {
     verify(ss).reportLocalError(any(IOException.class));
   }
   
-  @Test(timeout=30000)
+  @Test
+  @Timeout(value = 30)
   public void testCopyFromHostConnectionTimeout() throws Exception {
     when(connection.getInputStream()).thenThrow(
         new SocketTimeoutException("This is a fake timeout :)"));
@@ -275,7 +290,7 @@ public class TestFetcher {
     when(connection.getInputStream()).thenReturn(in);
 
     for (int i = 0; i < 3; ++i) {
-      Fetcher<Text,Text> underTest = new FakeFetcher<Text,Text>(jobWithRetry, 
+      Fetcher<Text, Text> underTest = new FakeFetcher<Text, Text>(jobWithRetry,
           id, ss, mm, r, metrics, except, key, connection);
       underTest.copyFromHost(host);
     }
@@ -328,7 +343,8 @@ public class TestFetcher {
   }
   
   @SuppressWarnings("unchecked")
-  @Test(timeout=10000) 
+  @Test
+  @Timeout(value = 10)
   public void testCopyFromHostCompressFailure() throws Exception {
     InMemoryMapOutput<Text, Text> immo = mock(InMemoryMapOutput.class);
 
@@ -354,7 +370,7 @@ public class TestFetcher {
     
     doThrow(new java.lang.InternalError()).when(immo)
         .shuffle(any(MapHost.class), any(InputStream.class), anyLong(), 
-            anyLong(), any(ShuffleClientMetrics.class), any(Reporter.class));
+        anyLong(), any(ShuffleClientMetrics.class), any(Reporter.class));
 
     underTest.copyFromHost(host);
        
@@ -365,7 +381,8 @@ public class TestFetcher {
   }
   
   @SuppressWarnings("unchecked")
-  @Test(timeout=10000) 
+  @Test
+  @Timeout(value = 10)
   public void testCopyFromHostOnAnyException() throws Exception {
     InMemoryMapOutput<Text, Text> immo = mock(InMemoryMapOutput.class);
 
@@ -391,7 +408,7 @@ public class TestFetcher {
 
     doThrow(new ArrayIndexOutOfBoundsException()).when(immo)
         .shuffle(any(MapHost.class), any(InputStream.class), anyLong(),
-            anyLong(), any(ShuffleClientMetrics.class), any(Reporter.class));
+        anyLong(), any(ShuffleClientMetrics.class), any(Reporter.class));
 
     underTest.copyFromHost(host);
 
@@ -402,11 +419,12 @@ public class TestFetcher {
   }
 
   @SuppressWarnings("unchecked")
-  @Test(timeout=10000)
+  @Test
+  @Timeout(value = 10)
   public void testCopyFromHostWithRetry() throws Exception {
     InMemoryMapOutput<Text, Text> immo = mock(InMemoryMapOutput.class);
     ss = mock(ShuffleSchedulerImpl.class);
-    Fetcher<Text,Text> underTest = new FakeFetcher<Text,Text>(jobWithRetry, 
+    Fetcher<Text, Text> underTest = new FakeFetcher<Text, Text>(jobWithRetry,
         id, ss, mm, r, metrics, except, key, connection, true);
 
     String replyHash = SecureShuffleUtils.generateHash(encHash.getBytes(), key);
@@ -435,16 +453,17 @@ public class TestFetcher {
         }
         return null;
       }
-    }).when(immo).shuffle(any(MapHost.class), any(InputStream.class), anyLong(), 
+    }).when(immo).shuffle(any(MapHost.class), any(InputStream.class), anyLong(),
         anyLong(), any(ShuffleClientMetrics.class), any(Reporter.class));
 
     underTest.copyFromHost(host);
     verify(ss, never()).copyFailed(any(TaskAttemptID.class),any(MapHost.class),
-                                   anyBoolean(), anyBoolean());
+        anyBoolean(), anyBoolean());
   }
 
   @SuppressWarnings("unchecked")
-  @Test(timeout=10000)
+  @Test
+  @Timeout(value = 10)
   public void testCopyFromHostWithRetryThenTimeout() throws Exception {
     InMemoryMapOutput<Text, Text> immo = mock(InMemoryMapOutput.class);
     Fetcher<Text,Text> underTest = new FakeFetcher<Text,Text>(jobWithRetry,
@@ -530,9 +549,9 @@ public class TestFetcher {
   }
 
   @Test
-  public void testCorruptedIFile() throws Exception {
+  public void testCorruptedIFile(TestInfo testInfo) throws Exception {
     final int fetcher = 7;
-    Path onDiskMapOutputPath = new Path(name.getMethodName() + "/foo");
+    Path onDiskMapOutputPath = new Path(testInfo.getDisplayName() + "/foo");
     Path shuffledToDisk =
         OnDiskMapOutput.getTempPath(onDiskMapOutputPath, fetcher);
     fs = FileSystem.getLocal(job).getRaw();
@@ -593,7 +612,8 @@ public class TestFetcher {
     }
   }
 
-  @Test(timeout=10000)
+  @Test
+  @Timeout(value = 10)
   public void testInterruptInMemory() throws Exception {
     final int FETCHER = 2;
     IFileWrappedMapOutput<Text,Text> immo = spy(new InMemoryMapOutput<Text,Text>(
@@ -636,7 +656,8 @@ public class TestFetcher {
     verify(immo).abort();
   }
 
-  @Test(timeout=10000)
+  @Test
+  @Timeout(value = 10)
   public void testInterruptOnDisk() throws Exception {
     final int FETCHER = 7;
     Path p = new Path("file:///tmp/foo");
@@ -686,7 +707,8 @@ public class TestFetcher {
   }
 
   @SuppressWarnings("unchecked")
-  @Test(timeout=10000)
+  @Test
+  @Timeout(value = 10)
   public void testCopyFromHostWithRetryUnreserve() throws Exception {
     InMemoryMapOutput<Text, Text> immo = mock(InMemoryMapOutput.class);
     Fetcher<Text,Text> underTest = new FakeFetcher<Text,Text>(jobWithRetry,

+ 35 - 39
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestMergeManager.java

@@ -18,8 +18,8 @@
 package org.apache.hadoop.mapreduce.task.reduce;
 
 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 static org.mockito.Mockito.mock;
 
 import java.io.IOException;
@@ -43,12 +43,13 @@ import org.apache.hadoop.mapred.MapOutputFile;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.task.reduce.MergeManagerImpl.CompressAwarePath;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 public class TestMergeManager {
 
-  @Test(timeout=10000)
+  @Test
+  @Timeout(value = 10)
   public void testMemoryMerge() throws Exception {
     final int TOTAL_MEM_BYTES = 10000;
     final int OUTPUT_SIZE = 7950;
@@ -65,13 +66,11 @@ public class TestMergeManager {
 
     // reserve enough map output to cause a merge when it is committed
     MapOutput<Text, Text> out1 = mgr.reserve(null, OUTPUT_SIZE, 0);
-    Assert.assertTrue("Should be a memory merge",
-                      (out1 instanceof InMemoryMapOutput));
+    assertTrue((out1 instanceof InMemoryMapOutput), "Should be a memory merge");
     InMemoryMapOutput<Text, Text> mout1 = (InMemoryMapOutput<Text, Text>)out1;
     fillOutput(mout1);
     MapOutput<Text, Text> out2 = mgr.reserve(null, OUTPUT_SIZE, 0);
-    Assert.assertTrue("Should be a memory merge",
-                      (out2 instanceof InMemoryMapOutput));
+    assertTrue((out2 instanceof InMemoryMapOutput), "Should be a memory merge");
     InMemoryMapOutput<Text, Text> mout2 = (InMemoryMapOutput<Text, Text>)out2;
     fillOutput(mout2);
 
@@ -85,17 +84,15 @@ public class TestMergeManager {
     mout2.commit();
     mergeStart.await();
 
-    Assert.assertEquals(1, mgr.getNumMerges());
+    assertEquals(1, mgr.getNumMerges());
 
     // reserve enough map output to cause another merge when committed
     out1 = mgr.reserve(null, OUTPUT_SIZE, 0);
-    Assert.assertTrue("Should be a memory merge",
-                       (out1 instanceof InMemoryMapOutput));
+    assertTrue((out1 instanceof InMemoryMapOutput), "Should be a memory merge");
     mout1 = (InMemoryMapOutput<Text, Text>)out1;
     fillOutput(mout1);
     out2 = mgr.reserve(null, OUTPUT_SIZE, 0);
-    Assert.assertTrue("Should be a memory merge",
-                       (out2 instanceof InMemoryMapOutput));
+    assertTrue((out2 instanceof InMemoryMapOutput), "Should be a memory merge");
     mout2 = (InMemoryMapOutput<Text, Text>)out2;
     fillOutput(mout2);
 
@@ -112,14 +109,13 @@ public class TestMergeManager {
 
     // start the second merge and verify
     mergeStart.await();
-    Assert.assertEquals(2, mgr.getNumMerges());
+    assertEquals(2, mgr.getNumMerges());
 
     // trigger the end of the second merge
     mergeComplete.await();
 
-    Assert.assertEquals(2, mgr.getNumMerges());
-    Assert.assertEquals("exception reporter invoked",
-        0, reporter.getNumExceptions());
+    assertEquals(2, mgr.getNumMerges());
+    assertEquals(0, reporter.getNumExceptions(), "exception reporter invoked");
   }
 
   private void fillOutput(InMemoryMapOutput<Text, Text> output) throws IOException {
@@ -214,7 +210,8 @@ public class TestMergeManager {
   }
 
   @SuppressWarnings({ "unchecked", "deprecation" })
-  @Test(timeout=10000)
+  @Test
+  @Timeout(value = 10)
   public void testOnDiskMerger() throws IOException {
     JobConf jobConf = new JobConf();
     final int SORT_FACTOR = 5;
@@ -222,9 +219,9 @@ public class TestMergeManager {
 
     MapOutputFile mapOutputFile = new MROutputFiles();
     FileSystem fs = FileSystem.getLocal(jobConf);
-    MergeManagerImpl<IntWritable, IntWritable> manager =
-      new MergeManagerImpl<IntWritable, IntWritable>(null, jobConf, fs, null
-        , null, null, null, null, null, null, null, null, null, mapOutputFile);
+    MergeManagerImpl<IntWritable, IntWritable> manager = new MergeManagerImpl<>(
+        null, jobConf, fs, null,
+        null, null, null, null, null, null, null, null, null, mapOutputFile);
 
     MergeThread onDiskMerger = manager.getOnDiskMerger();
     int mergeFactor = onDiskMerger.getMergeFactor();
@@ -240,22 +237,21 @@ public class TestMergeManager {
     Random rand = new Random();
     for(int i = 0; i < 2*SORT_FACTOR; ++i) {
       Path path = new Path("somePath");
-      CompressAwarePath cap = new CompressAwarePath(path, 1l, rand.nextInt());
+      CompressAwarePath cap = new CompressAwarePath(path, 1L, rand.nextInt());
       manager.closeOnDiskFile(cap);
     }
 
     //Check that the files pending to be merged are in sorted order.
     LinkedList<List<CompressAwarePath>> pendingToBeMerged = onDiskMerger.getPendingToBeMerged();
-    assertTrue("No inputs were added to list pending to merge",
-      pendingToBeMerged.size() > 0);
+    assertTrue(pendingToBeMerged.size() > 0,
+        "No inputs were added to list pending to merge");
     for(int i = 0; i < pendingToBeMerged.size(); ++i) {
       List<CompressAwarePath> inputs = pendingToBeMerged.get(i);
       for(int j = 1; j < inputs.size(); ++j) {
-        assertTrue("Not enough / too many inputs were going to be merged",
-          inputs.size() > 0 && inputs.size() <= SORT_FACTOR);
-        assertTrue("Inputs to be merged were not sorted according to size: ",
-          inputs.get(j).getCompressedSize()
-          >= inputs.get(j-1).getCompressedSize());
+        assertTrue(inputs.size() > 0 && inputs.size() <= SORT_FACTOR,
+            "Not enough / too many inputs were going to be merged");
+        assertTrue(inputs.get(j).getCompressedSize() >= inputs.get(j - 1).getCompressedSize(),
+            "Inputs to be merged were not sorted according to size: ");
       }
     }
 
@@ -280,16 +276,16 @@ public class TestMergeManager {
     // M4 = M1 fraction of map outputs remaining in memory for a reduce
     conf.setFloat(MRJobConfig.REDUCE_INPUT_BUFFER_PERCENT, 1.0f);
 
-    final MergeManagerImpl<Text, Text> mgr = new MergeManagerImpl<Text, Text>(
+    final MergeManagerImpl<Text, Text> mgr = new MergeManagerImpl<>(
         null, conf, mock(LocalFileSystem.class), null, null, null, null, null,
         null, null, null, null, null, new MROutputFiles());
-    assertTrue("Large shuffle area unusable: " + mgr.memoryLimit,
-        mgr.memoryLimit > Integer.MAX_VALUE);
+    assertTrue(mgr.memoryLimit > Integer.MAX_VALUE,
+        "Large shuffle area unusable: " + mgr.memoryLimit);
     final long maxInMemReduce = mgr.getMaxInMemReduceLimit();
-    assertTrue("Large in-memory reduce area unusable: " + maxInMemReduce,
-        maxInMemReduce > Integer.MAX_VALUE);
-    assertEquals("maxSingleShuffleLimit to be capped at Integer.MAX_VALUE",
-        Integer.MAX_VALUE, mgr.maxSingleShuffleLimit);
+    assertTrue(maxInMemReduce > Integer.MAX_VALUE,
+        "Large in-memory reduce area unusable: " + maxInMemReduce);
+    assertEquals(Integer.MAX_VALUE, mgr.maxSingleShuffleLimit,
+        "maxSingleShuffleLimit to be capped at Integer.MAX_VALUE");
     verifyReservedMapOutputType(mgr, 10L, "MEMORY");
     verifyReservedMapOutputType(mgr, 1L + Integer.MAX_VALUE, "DISK");
   }
@@ -298,8 +294,8 @@ public class TestMergeManager {
       long size, String expectedShuffleMode) throws IOException {
     final TaskAttemptID mapId = TaskAttemptID.forName("attempt_0_1_m_1_1");
     final MapOutput<Text, Text> mapOutput = mgr.reserve(mapId, size, 1);
-    assertEquals("Shuffled bytes: " + size, expectedShuffleMode,
-        mapOutput.getDescription());
+    assertEquals(expectedShuffleMode,
+        mapOutput.getDescription(), "Shuffled bytes: " + size);
     mgr.unreserve(size);
   }
 

+ 36 - 37
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestMerger.java

@@ -18,6 +18,9 @@
 package org.apache.hadoop.mapreduce.task.reduce;
 
 import static org.assertj.core.api.Assertions.assertThat;
+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.doAnswer;
 import static org.mockito.Mockito.mock;
@@ -65,24 +68,20 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Progress;
 import org.apache.hadoop.util.Progressable;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInfo;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
 public class TestMerger {
   private static File testRootDir;
-  @Rule
-  public TestName unitTestName = new TestName();
   private File unitTestDir;
   private JobConf jobConf;
   private FileSystem fs;
 
-  @BeforeClass
+  @BeforeAll
   public static void setupClass() throws Exception {
     // setup the test root directory
     testRootDir =
@@ -90,9 +89,9 @@ public class TestMerger {
             TestMerger.class);
   }
 
-  @Before
-  public void setup() throws IOException {
-    unitTestDir = new File(testRootDir, unitTestName.getMethodName());
+  @BeforeEach
+  public void setup(TestInfo testInfo) throws IOException {
+    unitTestDir = new File(testRootDir, testInfo.getDisplayName());
     unitTestDir.mkdirs();
     jobConf = new JobConf();
     // Set the temp directories a subdir of the test directory.
@@ -154,7 +153,7 @@ public class TestMerger {
 
     inMemoryMerger.merge(mapOutputs1);
 
-    Assert.assertEquals(1, mergeManager.onDiskMapOutputs.size());
+    assertEquals(1, mergeManager.onDiskMapOutputs.size());
 
     TaskAttemptID reduceId2 = new TaskAttemptID(
         new TaskID(jobId, TaskType.REDUCE, 3), 0);
@@ -189,7 +188,7 @@ public class TestMerger {
 
     inMemoryMerger2.merge(mapOutputs2);
 
-    Assert.assertEquals(2, mergeManager.onDiskMapOutputs.size());
+    assertEquals(2, mergeManager.onDiskMapOutputs.size());
 
     List<CompressAwarePath> paths = new ArrayList<CompressAwarePath>();
     Iterator<CompressAwarePath> iterator =
@@ -214,21 +213,21 @@ public class TestMerger {
     MergeThread<CompressAwarePath,Text,Text> onDiskMerger = mergeManager.createOnDiskMerger();
     onDiskMerger.merge(paths);
 
-    Assert.assertEquals(1, mergeManager.onDiskMapOutputs.size());
+    assertEquals(1, mergeManager.onDiskMapOutputs.size());
 
     keys = new ArrayList<String>();
     values = new ArrayList<String>();
     readOnDiskMapOutput(jobConf, fs,
         mergeManager.onDiskMapOutputs.iterator().next(), keys, values);
     assertThat(keys).isEqualTo(Arrays.asList("apple", "apple", "banana",
-            "banana", "carrot", "carrot"));
+        "banana", "carrot", "carrot"));
     assertThat(values).isEqualTo(Arrays.asList("awesome", "disgusting",
-            "pretty good", "bla", "amazing", "delicious"));
+        "pretty good", "bla", "amazing", "delicious"));
 
     mergeManager.close();
-    Assert.assertEquals(0, mergeManager.inMemoryMapOutputs.size());
-    Assert.assertEquals(0, mergeManager.inMemoryMergedMapOutputs.size());
-    Assert.assertEquals(0, mergeManager.onDiskMapOutputs.size());
+    assertEquals(0, mergeManager.inMemoryMapOutputs.size());
+    assertEquals(0, mergeManager.inMemoryMergedMapOutputs.size());
+    assertEquals(0, mergeManager.onDiskMapOutputs.size());
   }
 
   private byte[] writeMapOutput(Configuration conf, Map<String, String> keysToValues)
@@ -295,35 +294,35 @@ public class TestMerger {
     // Reading 6 keys total, 3 each in 2 segments, so each key read moves the
     // progress forward 1/6th of the way. Initially the first keys from each
     // segment have been read as part of the merge setup, so progress = 2/6.
-    Assert.assertEquals(2/6.0f, mergeQueue.getProgress().get(), epsilon);
+    assertEquals(2/6.0f, mergeQueue.getProgress().get(), epsilon);
 
     // The first next() returns one of the keys already read during merge setup
-    Assert.assertTrue(mergeQueue.next());
-    Assert.assertEquals(2/6.0f, mergeQueue.getProgress().get(), epsilon);
+    assertTrue(mergeQueue.next());
+    assertEquals(2/6.0f, mergeQueue.getProgress().get(), epsilon);
 
     // Subsequent next() calls should read one key and move progress
-    Assert.assertTrue(mergeQueue.next());
-    Assert.assertEquals(3/6.0f, mergeQueue.getProgress().get(), epsilon);
-    Assert.assertTrue(mergeQueue.next());
-    Assert.assertEquals(4/6.0f, mergeQueue.getProgress().get(), epsilon);
+    assertTrue(mergeQueue.next());
+    assertEquals(3/6.0f, mergeQueue.getProgress().get(), epsilon);
+    assertTrue(mergeQueue.next());
+    assertEquals(4/6.0f, mergeQueue.getProgress().get(), epsilon);
 
     // At this point we've exhausted all of the keys in one segment
     // so getting the next key will return the already cached key from the
     // other segment
-    Assert.assertTrue(mergeQueue.next());
-    Assert.assertEquals(4/6.0f, mergeQueue.getProgress().get(), epsilon);
+    assertTrue(mergeQueue.next());
+    assertEquals(4/6.0f, mergeQueue.getProgress().get(), epsilon);
 
     // Subsequent next() calls should read one key and move progress
-    Assert.assertTrue(mergeQueue.next());
-    Assert.assertEquals(5/6.0f, mergeQueue.getProgress().get(), epsilon);
-    Assert.assertTrue(mergeQueue.next());
-    Assert.assertEquals(1.0f, mergeQueue.getProgress().get(), epsilon);
+    assertTrue(mergeQueue.next());
+    assertEquals(5/6.0f, mergeQueue.getProgress().get(), epsilon);
+    assertTrue(mergeQueue.next());
+    assertEquals(1.0f, mergeQueue.getProgress().get(), epsilon);
 
     // Now there should be no more input
-    Assert.assertFalse(mergeQueue.next());
-    Assert.assertEquals(1.0f, mergeQueue.getProgress().get(), epsilon);
-    Assert.assertTrue(mergeQueue.getKey() == null);
-    Assert.assertEquals(0, mergeQueue.getValue().getData().length);
+    assertFalse(mergeQueue.next());
+    assertEquals(1.0f, mergeQueue.getProgress().get(), epsilon);
+    assertTrue(mergeQueue.getKey() == null);
+    assertEquals(0, mergeQueue.getValue().getData().length);
   }
 
   private Progressable getReporter() {

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestShuffleClientMetrics.java

@@ -21,9 +21,9 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobID;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.metrics2.MetricsTag;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
-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;
 

+ 18 - 18
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestShuffleScheduler.java

@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.mapreduce.task.reduce;
 
+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.mock;
 import static org.mockito.Mockito.verify;
 
@@ -37,8 +40,7 @@ import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.util.Progress;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 public class TestShuffleScheduler {
 
@@ -69,15 +71,13 @@ public class TestShuffleScheduler {
     TaskID taskId1 = new TaskID(jobId, TaskType.REDUCE, 1);
     scheduler.tipFailed(taskId1);
 
-    Assert.assertEquals("Progress should be 0.5", 0.5f, progress.getProgress(),
-        0.0f);
-    Assert.assertFalse(scheduler.waitUntilDone(1));
+    assertEquals(0.5f, progress.getProgress(), 0.0f, "Progress should be 0.5");
+    assertFalse(scheduler.waitUntilDone(1));
 
     TaskID taskId0 = new TaskID(jobId, TaskType.REDUCE, 0);
     scheduler.tipFailed(taskId0);
-    Assert.assertEquals("Progress should be 1.0", 1.0f, progress.getProgress(),
-        0.0f);
-    Assert.assertTrue(scheduler.waitUntilDone(1));
+    assertEquals(1.0f, progress.getProgress(), 0.0f, "Progress should be 1.0");
+    assertTrue(scheduler.waitUntilDone(1));
   }
 
   @SuppressWarnings("rawtypes")
@@ -134,7 +134,7 @@ public class TestShuffleScheduler {
     //adding the 1st interval, 40MB from 60s to 100s
     long bytes = (long)40 * 1024 * 1024;
     scheduler.copySucceeded(attemptID0, new MapHost(null, null), bytes, 60000, 100000, output);
-    Assert.assertEquals(copyMessage(1, 1, 1), progress.toString());
+    assertEquals(copyMessage(1, 1, 1), progress.toString());
 
     TaskAttemptID attemptID1 = new TaskAttemptID(
         new org.apache.hadoop.mapred.TaskID(
@@ -143,7 +143,7 @@ public class TestShuffleScheduler {
     //adding the 2nd interval before the 1st interval, 50MB from 0s to 50s
     bytes = (long)50 * 1024 * 1024;
     scheduler.copySucceeded(attemptID1, new MapHost(null, null), bytes, 0, 50000, output);
-    Assert.assertEquals(copyMessage(2, 1, 1), progress.toString());
+    assertEquals(copyMessage(2, 1, 1), progress.toString());
 
     TaskAttemptID attemptID2 = new TaskAttemptID(
         new org.apache.hadoop.mapred.TaskID(
@@ -153,7 +153,7 @@ public class TestShuffleScheduler {
     //110MB from 25s to 80s
     bytes = (long)110 * 1024 * 1024;
     scheduler.copySucceeded(attemptID2, new MapHost(null, null), bytes, 25000, 80000, output);
-    Assert.assertEquals(copyMessage(3, 2, 2), progress.toString());
+    assertEquals(copyMessage(3, 2, 2), progress.toString());
 
     TaskAttemptID attemptID3 = new TaskAttemptID(
         new org.apache.hadoop.mapred.TaskID(
@@ -162,7 +162,7 @@ public class TestShuffleScheduler {
     //adding the 4th interval just after the 2nd interval, 100MB from 100s to 300s
     bytes = (long)100 * 1024 * 1024;
     scheduler.copySucceeded(attemptID3, new MapHost(null, null), bytes, 100000, 300000, output);
-    Assert.assertEquals(copyMessage(4, 0.5, 1), progress.toString());
+    assertEquals(copyMessage(4, 0.5, 1), progress.toString());
 
     TaskAttemptID attemptID4 = new TaskAttemptID(
         new org.apache.hadoop.mapred.TaskID(
@@ -171,7 +171,7 @@ public class TestShuffleScheduler {
     //adding the 5th interval between after 4th, 50MB from 350s to 400s
     bytes = (long)50 * 1024 * 1024;
     scheduler.copySucceeded(attemptID4, new MapHost(null, null), bytes, 350000, 400000, output);
-    Assert.assertEquals(copyMessage(5, 1, 1), progress.toString());
+    assertEquals(copyMessage(5, 1, 1), progress.toString());
 
 
     TaskAttemptID attemptID5 = new TaskAttemptID(
@@ -180,7 +180,7 @@ public class TestShuffleScheduler {
     //adding the 6th interval between after 5th, 50MB from 450s to 500s
     bytes = (long)50 * 1024 * 1024;
     scheduler.copySucceeded(attemptID5, new MapHost(null, null), bytes, 450000, 500000, output);
-    Assert.assertEquals(copyMessage(6, 1, 1), progress.toString());
+    assertEquals(copyMessage(6, 1, 1), progress.toString());
 
     TaskAttemptID attemptID6 = new TaskAttemptID(
         new org.apache.hadoop.mapred.TaskID(
@@ -188,7 +188,7 @@ public class TestShuffleScheduler {
     //adding the 7th interval between after 5th and 6th interval, 20MB from 320s to 340s
     bytes = (long)20 * 1024 * 1024;
     scheduler.copySucceeded(attemptID6, new MapHost(null, null), bytes, 320000, 340000, output);
-    Assert.assertEquals(copyMessage(7, 1, 1), progress.toString());
+    assertEquals(copyMessage(7, 1, 1), progress.toString());
 
     TaskAttemptID attemptID7 = new TaskAttemptID(
         new org.apache.hadoop.mapred.TaskID(
@@ -196,7 +196,7 @@ public class TestShuffleScheduler {
     //adding the 8th interval overlapping with 4th, 5th, and 7th 30MB from 290s to 350s
     bytes = (long)30 * 1024 * 1024;
     scheduler.copySucceeded(attemptID7, new MapHost(null, null), bytes, 290000, 350000, output);
-    Assert.assertEquals(copyMessage(8, 0.5, 1), progress.toString());
+    assertEquals(copyMessage(8, 0.5, 1), progress.toString());
 
     TaskAttemptID attemptID8 = new TaskAttemptID(
         new org.apache.hadoop.mapred.TaskID(
@@ -204,7 +204,7 @@ public class TestShuffleScheduler {
     //adding the 9th interval overlapping with 5th and 6th, 50MB from 400s to 450s
     bytes = (long)50 * 1024 * 1024;
     scheduler.copySucceeded(attemptID8, new MapHost(null, null), bytes, 400000, 450000, output);
-    Assert.assertEquals(copyMessage(9, 1, 1), progress.toString());
+    assertEquals(copyMessage(9, 1, 1), progress.toString());
 
     TaskAttemptID attemptID9 = new TaskAttemptID(
         new org.apache.hadoop.mapred.TaskID(
@@ -212,7 +212,7 @@ public class TestShuffleScheduler {
     //adding the 10th interval overlapping with all intervals, 500MB from 0s to 500s
     bytes = (long)500 * 1024 * 1024;
     scheduler.copySucceeded(attemptID9, new MapHost(null, null), bytes, 0, 500000, output);
-    Assert.assertEquals(copyMessage(10, 1, 2), progress.toString());
+    assertEquals(copyMessage(10, 1, 2), progress.toString());
   }
 
   @SuppressWarnings("rawtypes")

+ 19 - 21
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/tools/TestCLI.java

@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.mapreduce.tools;
 
-import static org.junit.Assert.*;
-
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
@@ -31,9 +29,11 @@ import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.JobPriority;
 import org.apache.hadoop.mapreduce.JobStatus;
 import org.apache.hadoop.mapreduce.JobStatus.State;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
+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.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -71,13 +71,11 @@ public class TestCLI {
     int retCode_completed = cli.run(new String[] { "-list-attempt-ids",
         jobIdStr, "REDUCE", "completed" });
 
-    assertEquals("MAP is a valid input,exit code should be 0", 0, retCode_MAP);
-    assertEquals("map is a valid input,exit code should be 0", 0, retCode_map);
-    assertEquals("REDUCE is a valid input,exit code should be 0", 0,
-        retCode_REDUCE);
-    assertEquals(
-        "REDUCE and completed are a valid inputs to -list-attempt-ids,exit code should be 0",
-        0, retCode_completed);
+    assertEquals(0, retCode_MAP, "MAP is a valid input,exit code should be 0");
+    assertEquals(0, retCode_map, "map is a valid input,exit code should be 0");
+    assertEquals(0, retCode_REDUCE, "REDUCE is a valid input,exit code should be 0");
+    assertEquals(0, retCode_completed,
+        "REDUCE and completed are a valid inputs to -list-attempt-ids,exit code should be 0");
 
     verify(job, times(2)).getTaskReports(TaskType.MAP);
     verify(job, times(2)).getTaskReports(TaskType.REDUCE);
@@ -106,14 +104,14 @@ public class TestCLI {
     int retCode_invalidJobId = cli.run(new String[] { "-list-attempt-ids",
         jobIdStr2, "MAP", "running" });
 
-    assertEquals("JOB_SETUP is an invalid input,exit code should be -1", -1,
-        retCode_JOB_SETUP);
-    assertEquals("JOB_CLEANUP is an invalid input,exit code should be -1", -1,
-        retCode_JOB_CLEANUP);
-    assertEquals("complete is an invalid input,exit code should be -1", -1,
-        retCode_invalidTaskState);
-    assertEquals("Non existing job id should be skippted with -1", -1,
-        retCode_invalidJobId);
+    assertEquals(-1, retCode_JOB_SETUP,
+        "JOB_SETUP is an invalid input,exit code should be -1");
+    assertEquals(-1, retCode_JOB_CLEANUP,
+        "JOB_CLEANUP is an invalid input,exit code should be -1");
+    assertEquals(-1, retCode_invalidTaskState,
+        "complete is an invalid input,exit code should be -1");
+    assertEquals(-1, retCode_invalidJobId,
+        "Non existing job id should be skipped with -1");
 
   }
 
@@ -173,7 +171,7 @@ public class TestCLI {
     cli.cluster = mockCluster;
 
     Job job = cli.getJob(JobID.forName("job_1234654654_001"));
-    Assert.assertTrue("job is not null", job == null);
+    assertTrue(job == null, "job is not null");
   }
 
   @Test
@@ -189,7 +187,7 @@ public class TestCLI {
     cli.cluster = mockCluster;
 
     Job job = cli.getJob(JobID.forName("job_1234654654_001"));
-    Assert.assertTrue("job is null", job != null);
+    assertNotNull(job, "job is null");
   }
 
   @Test

+ 5 - 0
hadoop-project/pom.xml

@@ -1297,6 +1297,11 @@
         <artifactId>mockito-all</artifactId>
         <version>1.10.19</version>
       </dependency>
+      <dependency>
+        <groupId>org.mockito</groupId>
+        <artifactId>mockito-junit-jupiter</artifactId>
+        <version>3.12.4</version>
+      </dependency>
       <dependency>
         <groupId>org.objenesis</groupId>
         <artifactId>objenesis</artifactId>