Quellcode durchsuchen

MAPREDUCE-7421. [JDK17] Upgrade Junit 4 to 5 in hadoop-mapreduce-client-jobclient Part2. (#7372)

Co-authored-by: Chris Nauroth <cnauroth@apache.org>
Reviewed-by: Chris Nauroth <cnauroth@apache.org>
Signed-off-by: Shilun Fan <slfan1989@apache.org>
slfan1989 vor 2 Monaten
Ursprung
Commit
c42469e41e
78 geänderte Dateien mit 1242 neuen und 1153 gelöschten Zeilen
  1. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
  2. 6 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestClientProtocolProviderImpls.java
  3. 12 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestCounters.java
  4. 8 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestLargeSort.java
  5. 16 14
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestLocalRunner.java
  6. 49 47
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMRIntermediateDataEncryption.java
  7. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMROutputFormat.java
  8. 10 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapCollection.java
  9. 5 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapReduce.java
  10. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapReduceLazyOutput.java
  11. 18 15
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapperReducerCleanup.java
  12. 16 11
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNewCombinerGrouping.java
  13. 8 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java
  14. 26 28
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java
  15. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestValueIterReset.java
  16. 7 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java
  17. 3 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/filecache/TestURIFragments.java
  18. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/aggregate/TestMapReduceAggregates.java
  19. 11 11
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
  20. 29 29
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
  21. 6 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
  22. 4 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDBOutputFormat.java
  23. 10 10
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
  24. 6 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestIntegerSplitter.java
  25. 6 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestTextSplitter.java
  26. 5 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/fieldsel/TestMRFieldSelection.java
  27. 15 15
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java
  28. 15 13
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineSequenceFileInputFormat.java
  29. 22 19
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineTextInputFormat.java
  30. 7 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestDelegatingInputFormat.java
  31. 44 34
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestFixedLengthInputFormat.java
  32. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestLineRecordReaderJobs.java
  33. 8 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRCJCFileInputFormat.java
  34. 41 40
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRKeyValueTextInputFormat.java
  35. 10 12
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileAsBinaryInputFormat.java
  36. 7 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileAsTextInputFormat.java
  37. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileInputFilter.java
  38. 4 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
  39. 10 10
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestNLineInputFormat.java
  40. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestControlledJob.java
  41. 11 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
  42. 8 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControlWithMocks.java
  43. 28 28
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinDatamerge.java
  44. 33 33
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinProperties.java
  45. 16 19
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinTupleWritable.java
  46. 5 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestWrappedRRClassloader.java
  47. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
  48. 22 21
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
  49. 12 12
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRCJCFileOutputCommitter.java
  50. 21 18
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
  51. 19 22
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRSequenceFileAsBinaryOutputFormat.java
  52. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestBinaryPartitioner.java
  53. 7 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestInputSampler.java
  54. 36 36
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestKeyFieldHelper.java
  55. 19 18
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedPartitioner.java
  56. 11 10
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestTotalOrderPartitioner.java
  57. 10 11
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestBinaryTokenFile.java
  58. 7 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestJHSSecurity.java
  59. 6 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestMRCredentials.java
  60. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestUmbilicalProtocolWithJobToken.java
  61. 11 10
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/ssl/TestEncryptedShuffle.java
  62. 14 12
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/token/delegation/TestDelegationToken.java
  63. 14 14
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/util/TestMRAsyncDiskService.java
  64. 11 10
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRAMWithNonNormalizedCapabilities.java
  65. 8 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRAppWithCombiner.java
  66. 147 131
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
  67. 19 16
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithHistoryService.java
  68. 27 22
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithProfiler.java
  69. 22 20
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMROldApiJobs.java
  70. 7 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMiniMRProxyUser.java
  71. 6 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestNonExistentJob.java
  72. 38 38
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestRMNMInfo.java
  73. 92 83
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestSpeculativeExecOnCluster.java
  74. 37 35
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestSpeculativeExecution.java
  75. 26 21
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestSpeculativeExecutionWithMRApp.java
  76. 16 15
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestUberAM.java
  77. 5 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/util/TestMRCJCReflectionUtils.java
  78. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/util/TestMRCJCRunJar.java

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java

@@ -140,8 +140,8 @@ public class TestChild extends HadoopTestCase {
     assertTrue(fs.exists(outDir), "Job output directory doesn't exit!");
     FileStatus[] list = fs.listStatus(outDir, new OutputFilter());
     int numPartFiles = numReds == 0 ? numMaps : numReds;
-    assertTrue(list.length == numPartFiles, "Number of part-files is " + list.length + " and not "
-        + numPartFiles);
+    assertTrue(list.length == numPartFiles,
+        "Number of part-files is " + list.length + " and not " + numPartFiles);
     return job;
   }
 

+ 6 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestClientProtocolProviderImpls.java

@@ -27,10 +27,10 @@ import org.apache.hadoop.mapred.LocalJobRunner;
 import org.apache.hadoop.mapred.YARNRunner;
 import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
 import org.apache.hadoop.util.StringUtils;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 public class TestClientProtocolProviderImpls {
 
@@ -91,9 +91,9 @@ public class TestClientProtocolProviderImpls {
       fail("Cluster init should fail because of non-existing FileSystem");
     } catch (IOException ioEx) {
       final String stackTrace = StringUtils.stringifyException(ioEx);
-      assertTrue("No root cause detected",
-          stackTrace.contains(UnsupportedFileSystemException.class.getName())
-              && stackTrace.contains("nosuchfs"));
+      assertTrue(stackTrace.contains(
+          UnsupportedFileSystemException.class.getName()) && stackTrace.contains("nosuchfs"),
+          "No root cause detected");
     }
   }
 }

+ 12 - 9
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestCounters.java

@@ -23,9 +23,12 @@ import org.apache.hadoop.mapreduce.counters.LimitExceededException;
 import org.apache.hadoop.mapreduce.counters.Limits;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-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.assertSame;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
-import static org.junit.Assert.*;
 /**
  * TestCounters checks the sanity and recoverability of {@code Counters}
  */
@@ -46,19 +49,19 @@ public class TestCounters {
       long expectedValue = initValue;
       Counter counter = new Counters().findCounter("test", "foo");
       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");
     }
   }
 
@@ -148,6 +151,6 @@ public class TestCounters {
       LOG.info("got expected: "+ e);
       return;
     }
-    assertTrue("Should've thrown "+ ecls.getSimpleName(), false);
+    assertTrue(false, "Should've thrown "+ ecls.getSimpleName());
   }
 }

+ 8 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestLargeSort.java

@@ -23,25 +23,25 @@ import org.apache.hadoop.mapred.MiniMRClientCluster;
 import org.apache.hadoop.mapred.MiniMRClientClusterFactory;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-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 java.io.IOException;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 public class TestLargeSort {
   MiniMRClientCluster cluster;
 
-  @Before
+  @BeforeEach
   public void setup() throws IOException {
     Configuration conf = new YarnConfiguration();
     cluster = MiniMRClientClusterFactory.create(this.getClass(), 2, conf);
     cluster.start();
   }
 
-  @After
+  @AfterEach
   public void cleanup() throws IOException {
     if (cluster != null) {
       cluster.stop();
@@ -59,8 +59,8 @@ public class TestLargeSort {
       conf.setInt(MRJobConfig.IO_SORT_MB, ioSortMb);
       conf.setInt(LargeSorter.NUM_MAP_TASKS, 1);
       conf.setInt(LargeSorter.MBS_PER_MAP, ioSortMb);
-      assertEquals("Large sort failed for " + ioSortMb, 0,
-          ToolRunner.run(conf, new LargeSorter(), args));
+      assertEquals(0, ToolRunner.run(conf, new LargeSorter(), args),
+          "Large sort failed for " + ioSortMb);
     }
   }
 }

+ 16 - 14
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestLocalRunner.java

@@ -30,7 +30,8 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.input.FileSplit;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -44,10 +45,10 @@ import java.io.OutputStreamWriter;
 import java.util.ArrayList;
 import java.util.List;
 
-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.assertTrue;
 
 /**
  * Stress tests for the LocalJobRunner
@@ -235,9 +236,9 @@ public class TestLocalRunner {
 
     // Should get a single line of the form "0\t(count)"
     String line = r.readLine().trim();
-    assertTrue("Line does not have correct key", line.startsWith("0\t"));
+    assertTrue(line.startsWith("0\t"), "Line does not have correct key");
     int count = Integer.valueOf(line.substring(2));
-    assertEquals("Incorrect count generated!", TOTAL_RECORDS, count);
+    assertEquals(TOTAL_RECORDS, count, "Incorrect count generated!");
 
     r.close();
 
@@ -276,7 +277,7 @@ public class TestLocalRunner {
     FileOutputFormat.setOutputPath(job, outputPath);
 
     boolean ret = job.waitForCompletion(true);
-    assertTrue("job failed", ret);
+    assertTrue(ret, "job failed");
 
     // This job should have done *some* gc work.
     // It had to clean up 400,000 objects.
@@ -284,7 +285,7 @@ public class TestLocalRunner {
     Counter gcCounter = job.getCounters().findCounter(
         TaskCounter.GC_TIME_MILLIS);
     assertNotNull(gcCounter);
-    assertTrue("No time spent in gc", gcCounter.getValue() > 0);
+    assertTrue(gcCounter.getValue() > 0, "No time spent in gc");
   }
 
 
@@ -292,7 +293,8 @@ public class TestLocalRunner {
    * Run a test with several mappers in parallel, operating at different
    * speeds. Verify that the correct amount of output is created.
    */
-  @Test(timeout=120*1000)
+  @Test
+  @Timeout(value=120)
   public void testMultiMaps() throws Exception {
     Job job = Job.getInstance();
 
@@ -377,7 +379,7 @@ public class TestLocalRunner {
     FileOutputFormat.setOutputPath(job, outputPath);
 
     boolean success = job.waitForCompletion(true);
-    assertFalse("Job succeeded somehow", success);
+    assertFalse(success, "Job succeeded somehow");
   }
 
   /** An IF that creates no splits */
@@ -434,7 +436,7 @@ public class TestLocalRunner {
     FileOutputFormat.setOutputPath(job, outputPath);
 
     boolean success = job.waitForCompletion(true);
-    assertTrue("Empty job should work", success);
+    assertTrue(success, "Empty job should work");
   }
 
   /** @return the directory where numberfiles are written (mapper inputs)  */
@@ -510,7 +512,7 @@ public class TestLocalRunner {
     int expectedPerMapper = maxVal * (maxVal + 1) / 2;
     int expectedSum = expectedPerMapper * numMaps;
     LOG.info("expected sum: " + expectedSum + ", got " + valueSum);
-    assertEquals("Didn't get all our results back", expectedSum, valueSum);
+    assertEquals(expectedSum, valueSum, "Didn't get all our results back");
   }
 
   /**
@@ -551,7 +553,7 @@ public class TestLocalRunner {
     LocalJobRunner.setLocalMaxRunningReduces(job, parallelReduces);
 
     boolean result = job.waitForCompletion(true);
-    assertTrue("Job failed!!", result);
+    assertTrue(result, "Job failed!!");
 
     verifyNumberJob(numMaps);
   }

+ 49 - 47
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMRIntermediateDataEncryption.java

@@ -33,13 +33,10 @@ import java.util.concurrent.Future;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicLong;
 
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -66,6 +63,10 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.ToolRunner;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
 /**
  * This class tests the support of Intermediate data encryption
  * (Spill data encryption).
@@ -86,7 +87,6 @@ import org.apache.hadoop.util.ToolRunner;
  *    combine the output files into a single one.
  * 6- Verify that the maps spilled files.
  */
-@RunWith(Parameterized.class)
 public class TestMRIntermediateDataEncryption {
   public static final Logger LOG =
       LoggerFactory.getLogger(TestMRIntermediateDataEncryption.class);
@@ -113,35 +113,34 @@ public class TestMRIntermediateDataEncryption {
   /**
    * Test parameters.
    */
-  private final String testTitleName;
-  private final int numMappers;
-  private final int numReducers;
-  private final boolean isUber;
+  private String testTitleName;
+  private int numMappers;
+  private int numReducers;
+  private boolean isUber;
   private Configuration config;
   private Path jobOutputPath;
 
   /**
    * Initialized the parametrized JUnit test.
-   * @param testName the name of the unit test to be executed.
-   * @param mappers number of mappers in the tests.
-   * @param reducers number of the reducers.
-   * @param uberEnabled boolean flag for isUber
+   * @param pTestName the name of the unit test to be executed.
+   * @param pMappers number of mappers in the tests.
+   * @param pReducers number of the reducers.
+   * @param pUberEnabled boolean flag for isUber
+   * @throws Exception unit test error.
    */
-  public TestMRIntermediateDataEncryption(String testName, int mappers,
-      int reducers, boolean uberEnabled) {
-    this.testTitleName = testName;
-    this.numMappers = mappers;
-    this.numReducers = reducers;
-    this.isUber = uberEnabled;
+  public void initTestMRIntermediateDataEncryption(String pTestName,
+      int pMappers, int pReducers, boolean pUberEnabled) throws Exception {
+    this.testTitleName = pTestName;
+    this.numMappers = pMappers;
+    this.numReducers = pReducers;
+    this.isUber = pUberEnabled;
+    setup();
   }
 
   /**
    * List of arguments to run the JunitTest.
    * @return
    */
-  @Parameterized.Parameters(
-      name = "{index}: TestMRIntermediateDataEncryption.{0} .. "
-          + "mappers:{1}, reducers:{2}, isUber:{3})")
   public static Collection<Object[]> getTestParameters() {
     return Arrays.asList(new Object[][]{
         {"testSingleReducer", 3, 1, false},
@@ -151,7 +150,7 @@ public class TestMRIntermediateDataEncryption {
     });
   }
 
-  @BeforeClass
+  @BeforeAll
   public static void setupClass() throws Exception {
     // setup the test root directory
     testRootDir =
@@ -176,13 +175,13 @@ public class TestMRIntermediateDataEncryption {
     fs.mkdirs(jobsDirPath);
     jobInputDirPath = new Path(jobsDirPath, "in-dir");
     // run the input generator job.
-    Assert.assertEquals("Generating input should succeed", 0,
-        generateInputTextFile());
+    assertEquals(0,
+        generateInputTextFile(), "Generating input should succeed");
     // run the reference job
     runReferenceJob();
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() throws IOException {
     // shutdown clusters
     if (mrCluster != null) {
@@ -194,7 +193,7 @@ public class TestMRIntermediateDataEncryption {
     // make sure that generated input file is deleted
     final File textInputFile = new File(testRootDir, "input.txt");
     if (textInputFile.exists()) {
-      Assert.assertTrue(textInputFile.delete());
+      assertTrue(textInputFile.delete());
     }
   }
 
@@ -290,19 +289,19 @@ public class TestMRIntermediateDataEncryption {
     if (fs.exists(jobRefDirPath) && !fs.delete(jobRefDirPath, true)) {
       throw new IOException("Could not delete " + jobRefDirPath);
     }
-    Assert.assertTrue(fs.mkdirs(jobRefDirPath));
+    assertTrue(fs.mkdirs(jobRefDirPath));
     Path jobRefOutputPath = new Path(jobRefDirPath, "out-dir");
     Configuration referenceConf = new Configuration(commonConfig);
     referenceConf.setBoolean(MRJobConfig.MR_ENCRYPTED_INTERMEDIATE_DATA, false);
     Job jobReference = runWordCountJob(jobRefLabel, jobRefOutputPath,
         referenceConf, 4, 1);
-    Assert.assertTrue(jobReference.isSuccessful());
+    assertTrue(jobReference.isSuccessful());
     FileStatus[] fileStatusArr =
         fs.listStatus(jobRefOutputPath,
             new Utils.OutputFileUtils.OutputFilesFilter());
-    Assert.assertEquals(1, fileStatusArr.length);
+    assertEquals(1, fileStatusArr.length);
     checkSumReference = fs.getFileChecksum(fileStatusArr[0].getPath());
-    Assert.assertTrue(fs.delete(jobRefDirPath, true));
+    assertTrue(fs.delete(jobRefDirPath, true));
   }
 
   private static Job runWordCountJob(String postfixName, Path jOutputPath,
@@ -346,8 +345,8 @@ public class TestMRIntermediateDataEncryption {
    * @throws Exception if the output is missing or the combiner job fails.
    */
   private boolean validateJobOutput() throws Exception {
-    Assert.assertTrue("Job Output path [" + jobOutputPath + "] should exist",
-        fs.exists(jobOutputPath));
+    assertTrue(fs.exists(jobOutputPath),
+        "Job Output path [" + jobOutputPath + "] should exist");
     Path outputPath = jobOutputPath;
     if (numReducers != 1) {
       // combine the result into one file by running a combiner job
@@ -391,7 +390,6 @@ public class TestMRIntermediateDataEncryption {
     return checkSumReference.equals(jobFileChecksum);
   }
 
-  @Before
   public void setup() throws Exception {
     LOG.info("Starting TestMRIntermediateDataEncryption#{}.......",
         testTitleName);
@@ -419,8 +417,12 @@ public class TestMRIntermediateDataEncryption {
     config.setInt("mapreduce.reduce.maxattempts", 1);
   }
 
-  @Test
-  public void testWordCount() throws Exception {
+  @ParameterizedTest(name = "{index}: TestMRIntermediateDataEncryption.{0} .. "
+       + "mappers:{1}, reducers:{2}, isUber:{3})")
+  @MethodSource("getTestParameters")
+  public void testWordCount(String pTestName,
+      int pMappers, int pReducers, boolean pUberEnabled) throws Exception {
+    initTestMRIntermediateDataEncryption(pTestName, pMappers, pReducers, pUberEnabled);
     LOG.info("........Starting main Job Driver #{} starting at {}.......",
         testTitleName, Time.formatTime(System.currentTimeMillis()));
     SpillCallBackPathsFinder spillInjector =
@@ -435,7 +437,7 @@ public class TestMRIntermediateDataEncryption {
           testTitleName, Time.formatTime(System.currentTimeMillis())));
       Job job = runWordCountJob(testTitleName, jobOutputPath, config,
           numMappers, numReducers);
-      Assert.assertTrue(job.isSuccessful());
+      assertTrue(job.isSuccessful());
       long endTime = Time.monotonicNow();
       testSummary.append(String.format("%nJob %s ended at %s",
               job.getJobName(), Time.formatTime(System.currentTimeMillis())));
@@ -451,16 +453,16 @@ public class TestMRIntermediateDataEncryption {
                 fStatus.getPath(), fileSize));
       }
       // Validate the checksum of the output.
-      Assert.assertTrue(validateJobOutput());
+      assertTrue(validateJobOutput());
       // Check intermediate files and spilling.
       long spilledRecords =
           job.getCounters().findCounter(TaskCounter.SPILLED_RECORDS).getValue();
-      Assert.assertTrue("Spill records must be greater than 0",
-          spilledRecords > 0);
-      Assert.assertFalse("The encrypted spilled files should not be empty.",
-          spillInjector.getEncryptedSpilledFiles().isEmpty());
-      Assert.assertTrue("Invalid access to spill file positions",
-          spillInjector.getInvalidSpillEntries().isEmpty());
+      assertTrue(
+          spilledRecords > 0, "Spill records must be greater than 0");
+      assertFalse(spillInjector.getEncryptedSpilledFiles().isEmpty(),
+          "The encrypted spilled files should not be empty.");
+      assertTrue(spillInjector.getInvalidSpillEntries().isEmpty(),
+          "Invalid access to spill file positions");
     } finally {
       testSummary.append(spillInjector.getSpilledFileReport());
       LOG.info(testSummary.toString());

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMROutputFormat.java

@@ -29,9 +29,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapred.JobConf;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class TestMROutputFormat {
 

+ 10 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapCollection.java

@@ -32,9 +32,11 @@ import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
-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.assertTrue;
 
 public class TestMapCollection {
 
@@ -81,7 +83,7 @@ public class TestMapCollection {
       }
       len = WritableUtils.readVInt(in);
       for (int i = 0; i < len; ++i) {
-        assertEquals("Invalid byte at " + i, fillChar, in.readByte());
+        assertEquals(fillChar, in.readByte(), "Invalid byte at " + i);
       }
     }
     @Override
@@ -162,10 +164,10 @@ public class TestMapCollection {
         n2 = 0;
       }
       for (int i = s1 + n1; i < l1 - n1; ++i) {
-        assertEquals("Invalid key at " + s1, (int)KeyWritable.keyFill, b1[i]);
+        assertEquals((int)KeyWritable.keyFill, b1[i], "Invalid key at " + s1);
       }
       for (int i = s2 + n2; i < l2 - n2; ++i) {
-        assertEquals("Invalid key at " + s2, (int)KeyWritable.keyFill, b2[i]);
+        assertEquals((int)KeyWritable.keyFill, b2[i], "Invalid key at " + s2);
       }
       return l1 - l2;
     }
@@ -193,7 +195,7 @@ public class TestMapCollection {
     @Override
     protected void cleanup(Context context)
         throws IOException, InterruptedException {
-      assertEquals("Unexpected record count", expected, numrecs);
+      assertEquals(expected, numrecs, "Unexpected record count");
     }
   }
 
@@ -281,7 +283,7 @@ public class TestMapCollection {
         public float getProgress() { return (float) current / records; }
         @Override
         public void close() {
-          assertEquals("Unexpected count", records, current - 1);
+          assertEquals(records, current - 1, "Unexpected count");
         }
       };
     }
@@ -318,7 +320,7 @@ public class TestMapCollection {
     job.setSortComparatorClass(VariableComparator.class);
 
     LOG.info("Running " + name);
-    assertTrue("Job failed!", job.waitForCompletion(false));
+    assertTrue(job.waitForCompletion(false), "Job failed!");
   }
 
   @Test

+ 5 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapReduce.java

@@ -41,10 +41,10 @@ import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.MapFileOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
-import org.junit.After;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
 
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 /**********************************************************
  * MapredLoadTest generates a bunch of work that exercises
@@ -221,7 +221,7 @@ public class TestMapReduce {
   private static int counts = 100;
   private static Random r = new Random();
 
-  @After
+  @AfterEach
   public void cleanup() {
     FileUtil.fullyDelete(TEST_DIR);
   }
@@ -437,7 +437,7 @@ public class TestMapReduce {
     } finally {
       bw.close();
     }
-    assertTrue("testMapRed failed", success);
+    assertTrue(success, "testMapRed failed");
     fs.delete(testdir, true);
   }
 

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapReduceLazyOutput.java

@@ -39,9 +39,9 @@ import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.LazyOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 /**
  * A JUnit test to test the Map-Reduce framework's feature to create part

+ 18 - 15
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapperReducerCleanup.java

@@ -38,8 +38,11 @@ import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
 import org.apache.hadoop.mapreduce.lib.reduce.IntSumReducer;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class TestMapperReducerCleanup {
 
@@ -258,9 +261,9 @@ public class TestMapperReducerCleanup {
 
     job.waitForCompletion(true);
 
-    Assert.assertTrue(mapCleanup);
-    Assert.assertTrue(recordReaderCleanup);
-    Assert.assertTrue(recordWriterCleanup);
+    assertTrue(mapCleanup);
+    assertTrue(recordReaderCleanup);
+    assertTrue(recordWriterCleanup);
   }
 
   @Test
@@ -291,10 +294,10 @@ public class TestMapperReducerCleanup {
 
     job.waitForCompletion(true);
 
-    Assert.assertTrue(mapCleanup);
-    Assert.assertTrue(reduceCleanup);
-    Assert.assertTrue(recordReaderCleanup);
-    Assert.assertTrue(recordWriterCleanup);
+    assertTrue(mapCleanup);
+    assertTrue(reduceCleanup);
+    assertTrue(recordReaderCleanup);
+    assertTrue(recordWriterCleanup);
   }
   
   @Test
@@ -325,14 +328,14 @@ public class TestMapperReducerCleanup {
 
     job.waitForCompletion(true);
 
-    Assert.assertTrue(mapCleanup);
-    Assert.assertTrue(reduceCleanup);
-    Assert.assertTrue(recordReaderCleanup);
-    Assert.assertTrue(recordWriterCleanup);
+    assertTrue(mapCleanup);
+    assertTrue(reduceCleanup);
+    assertTrue(recordReaderCleanup);
+    assertTrue(recordWriterCleanup);
 
-    Assert.assertNotNull(job.getCluster());
+    assertNotNull(job.getCluster());
     job.close();
-    Assert.assertNull(job.getCluster());
+    assertNull(job.getCluster());
   }
 
 }

+ 16 - 11
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNewCombinerGrouping.java

@@ -18,8 +18,7 @@
 
 package org.apache.hadoop.mapreduce;
 
-import org.junit.After;
-import org.junit.Assert;
+import org.junit.jupiter.api.AfterEach;
 
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
@@ -31,7 +30,7 @@ import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
 
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.io.BufferedReader;
 import java.io.File;
@@ -42,6 +41,12 @@ import java.io.PrintWriter;
 import java.util.HashSet;
 import java.util.Set;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+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;
+
 public class TestNewCombinerGrouping {
   private static File testRootDir = GenericTestUtils.getRandomizedTestDir();
 
@@ -105,7 +110,7 @@ public class TestNewCombinerGrouping {
 
   }
 
-  @After
+  @AfterEach
   public void cleanup() {
     FileUtil.fullyDelete(testRootDir);
   }
@@ -156,30 +161,30 @@ public class TestNewCombinerGrouping {
       long combinerOutputRecords = counters.findCounter(
           "org.apache.hadoop.mapreduce.TaskCounter",
           "COMBINE_OUTPUT_RECORDS").getValue();
-      Assert.assertTrue(combinerInputRecords > 0);
-      Assert.assertTrue(combinerInputRecords > combinerOutputRecords);
+      assertTrue(combinerInputRecords > 0);
+      assertTrue(combinerInputRecords > combinerOutputRecords);
 
       BufferedReader br = new BufferedReader(new FileReader(
           new File(out, "part-r-00000")));
       Set<String> output = new HashSet<String>();
       String line = br.readLine();
-      Assert.assertNotNull(line);
+      assertNotNull(line);
       output.add(line.substring(0, 1) + line.substring(4, 5));
       line = br.readLine();
-      Assert.assertNotNull(line);
+      assertNotNull(line);
       output.add(line.substring(0, 1) + line.substring(4, 5));
       line = br.readLine();
-      Assert.assertNull(line);
+      assertNull(line);
       br.close();
 
       Set<String> expected = new HashSet<String>();
       expected.add("A2");
       expected.add("B5");
 
-      Assert.assertEquals(expected, output);
+      assertEquals(expected, output);
 
     } else {
-      Assert.fail("Job failed");
+      fail("Job failed");
     }
   }
 

+ 8 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java

@@ -29,11 +29,11 @@ import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
-import org.junit.Ignore;
-import org.junit.Test;
-import static org.junit.Assert.assertTrue;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
-@Ignore
+@Disabled
 public class TestNoJobSetupCleanup extends HadoopTestCase {
   private static String TEST_ROOT_DIR =
     new File(System.getProperty("test.build.data","/tmp"))
@@ -58,7 +58,7 @@ public class TestNoJobSetupCleanup extends HadoopTestCase {
     assertTrue(job.getTaskReports(TaskType.MAP).length == numMaps);
     assertTrue(job.getTaskReports(TaskType.REDUCE).length == numReds);
     FileSystem fs = FileSystem.get(conf);
-    assertTrue("Job output directory doesn't exit!", fs.exists(outDir));
+    assertTrue(fs.exists(outDir), "Job output directory doesn't exit!");
 
     // job commit done only in cleanup 
     // therefore output should still be in temp location
@@ -67,8 +67,9 @@ public class TestNoJobSetupCleanup extends HadoopTestCase {
     Path tempWorkingPath = new Path(tempWorkingPathStr);
     FileStatus[] list = fs.listStatus(tempWorkingPath, new OutputFilter());
     int numPartFiles = numReds == 0 ? numMaps : numReds;
-    assertTrue("Number of part-files is " + list.length + " and not "
-        + numPartFiles, list.length == numPartFiles);
+    assertTrue(list.length == numPartFiles,
+        "Number of part-files is " + list.length + " and not "
+        + numPartFiles);
     return job;
   }
 

+ 26 - 28
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java

@@ -29,18 +29,18 @@ import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil.DataCopyMapper;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil.DataCopyReducer;
 
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
-import org.junit.Test;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 /**
  * Tests context api and {@link StatusReporter#getProgress()} via 
  * {@link TaskAttemptContext#getProgress()} API . 
  */
-@Ignore
+@Disabled
 public class TestTaskContext extends HadoopTestCase {
   private static final Path rootTempDir =
     new Path(System.getProperty("test.build.data", "/tmp"));
@@ -49,14 +49,14 @@ public class TestTaskContext extends HadoopTestCase {
   
   private static FileSystem fs = null;
 
-  @BeforeClass
+  @BeforeAll
   public static void setup() throws Exception {
     fs = FileSystem.getLocal(new Configuration());
     fs.delete(testRootTempDir, true);
     fs.mkdirs(testRootTempDir);
   }
 
-  @AfterClass
+  @AfterAll
   public static void cleanup() throws Exception {
     fs.delete(testRootTempDir, true);
   }
@@ -82,7 +82,7 @@ public class TestTaskContext extends HadoopTestCase {
    * @throws ClassNotFoundException
    */
   @Test
-  @Ignore
+  @Disabled
   public void testContextStatus()
       throws IOException, InterruptedException, ClassNotFoundException {
     Path test = new Path(testRootTempDir, "testContextStatus");
@@ -94,7 +94,7 @@ public class TestTaskContext extends HadoopTestCase {
                 new Path(test, "in"), new Path(test, "out"), numMaps, 0);
     job.setMapperClass(MyMapper.class);
     job.waitForCompletion(true);
-    assertTrue("Job failed", job.isSuccessful());
+    assertTrue(job.isSuccessful(), "Job failed");
     TaskReport[] reports = job.getTaskReports(TaskType.MAP);
     assertEquals(numMaps, reports.length);
     assertEquals(myStatus, reports[0].getState());
@@ -117,7 +117,7 @@ public class TestTaskContext extends HadoopTestCase {
     
     // run the job and wait for completion
     job.waitForCompletion(true);
-    assertTrue("Job failed", job.isSuccessful());
+    assertTrue(job.isSuccessful(), "Job failed");
     
     // check map task reports
     // TODO fix testcase 
@@ -147,8 +147,8 @@ public class TestTaskContext extends HadoopTestCase {
     @Override
     protected void setup(Context context) throws IOException {
       // check if the map task attempt progress is 0
-      assertEquals("Invalid progress in map setup", 
-                   0.0f, context.getProgress(), 0f);
+      assertEquals(0.0f, context.getProgress(), 0f,
+          "Invalid progress in map setup");
       
       // define the progress boundaries
       if (context.getNumReduceTasks() == 0) {
@@ -167,8 +167,8 @@ public class TestTaskContext extends HadoopTestCase {
       // get the weighted map phase progress
       float weightedMapProgress = progressRange * mapPhaseProgress;
       // check the map progress
-      assertEquals("Invalid progress in map", 
-                   weightedMapProgress, context.getProgress(), 0f);
+      assertEquals(weightedMapProgress, context.getProgress(), 0f,
+          "Invalid progress in map");
       
       context.write(new Text(value.toString() + recordCount), value);
     };
@@ -176,8 +176,8 @@ public class TestTaskContext extends HadoopTestCase {
     protected void cleanup(Mapper.Context context) 
     throws IOException, InterruptedException {
       // check if the attempt progress is at the progress boundary 
-      assertEquals("Invalid progress in map cleanup", 
-                   progressRange, context.getProgress(), 0f);
+      assertEquals(progressRange, context.getProgress(), 0f,
+          "Invalid progress in map cleanup");
     };
   }
   
@@ -203,7 +203,7 @@ public class TestTaskContext extends HadoopTestCase {
     job.setMaxMapAttempts(1);
     
     job.waitForCompletion(true);
-    assertTrue("Job failed", job.isSuccessful());
+    assertTrue(job.isSuccessful(), "Job failed");
   }
   
   @SuppressWarnings("unchecked")
@@ -220,9 +220,8 @@ public class TestTaskContext extends HadoopTestCase {
       float weightedReducePhaseProgress = 
         REDUCE_PROGRESS_RANGE * reducePhaseProgress;
       // check that the shuffle phase progress is accounted for
-      assertEquals("Invalid progress in reduce setup",
-                   SHUFFLE_PROGRESS_RANGE + weightedReducePhaseProgress, 
-                   context.getProgress(), 0.01f);
+      assertEquals(SHUFFLE_PROGRESS_RANGE + weightedReducePhaseProgress,
+          context.getProgress(), 0.01f, "Invalid progress in reduce setup");
     };
     
     public void reduce(Text key, Iterator<Text> values, Context context)
@@ -230,16 +229,15 @@ public class TestTaskContext extends HadoopTestCase {
       float reducePhaseProgress =  ((float)++recordCount)/INPUT_LINES;
       float weightedReducePhaseProgress = 
         REDUCE_PROGRESS_RANGE * reducePhaseProgress;
-      assertEquals("Invalid progress in reduce", 
-                   SHUFFLE_PROGRESS_RANGE + weightedReducePhaseProgress, 
-                   context.getProgress(), 0.01f);
+      assertEquals(SHUFFLE_PROGRESS_RANGE + weightedReducePhaseProgress,
+          context.getProgress(), 0.01f, "Invalid progress in reduce");
     }
     
     protected void cleanup(Reducer.Context context) 
     throws IOException, InterruptedException {
       // check if the reduce task has progress of 1 in the end
-      assertEquals("Invalid progress in reduce cleanup", 
-                   1.0f, context.getProgress(), 0f);
+      assertEquals(1.0f, context.getProgress(), 0f,
+          "Invalid progress in reduce cleanup");
     };
   }
   
@@ -268,6 +266,6 @@ public class TestTaskContext extends HadoopTestCase {
     job.setMaxReduceAttempts(1);
     
     job.waitForCompletion(true);
-    assertTrue("Job failed", job.isSuccessful());
+    assertTrue(job.isSuccessful(), "Job failed");
   }
 }

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestValueIterReset.java

@@ -39,11 +39,11 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 /**
  * A JUnit test to test the Map-Reduce framework's support for the

+ 7 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java

@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.mapreduce;
 
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.spy;
@@ -42,7 +42,7 @@ import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 public class TestYarnClientProtocolProvider {
   private static final RecordFactory recordFactory = RecordFactoryProvider.
@@ -62,8 +62,8 @@ public class TestYarnClientProtocolProvider {
     }
     
     try {
-      assertTrue("client is not a LocalJobRunner",
-          cluster.getClient() instanceof LocalJobRunner);
+      assertTrue(cluster.getClient() instanceof LocalJobRunner,
+          "client is not a LocalJobRunner");
     } finally {
       if (cluster != null) {
         cluster.close();
@@ -75,7 +75,7 @@ public class TestYarnClientProtocolProvider {
       conf.set(MRConfig.FRAMEWORK_NAME, MRConfig.YARN_FRAMEWORK_NAME);
       cluster = new Cluster(conf);
       ClientProtocol client = cluster.getClient();
-      assertTrue("client is a YARNRunner", client instanceof YARNRunner);
+      assertTrue(client instanceof YARNRunner, "client is a YARNRunner");
     } catch (IOException e) {
 
     } finally {
@@ -121,8 +121,8 @@ public class TestYarnClientProtocolProvider {
       };
       yrunner.setResourceMgrDelegate(rmgrDelegate);
       Token t = cluster.getDelegationToken(new Text(" "));
-      assertTrue("Token kind is instead " + t.getKind().toString(),
-        "Testclusterkind".equals(t.getKind().toString()));
+      assertTrue("Testclusterkind".equals(t.getKind().toString()),
+          "Token kind is instead " + t.getKind().toString());
     } finally {
       if (cluster != null) {
         cluster.close();

+ 3 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/filecache/TestURIFragments.java

@@ -20,8 +20,9 @@ package org.apache.hadoop.mapreduce.filecache;
 import java.net.URI;
 import java.net.URISyntaxException;
 
-import static org.junit.Assert.*;
-import org.junit.Test;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import org.junit.jupiter.api.Test;
 
 public class TestURIFragments {
 

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/aggregate/TestMapReduceAggregates.java

@@ -28,13 +28,13 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.nio.charset.StandardCharsets;
 import java.text.NumberFormat;
 
-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;
 
 public class TestMapReduceAggregates {
 

+ 11 - 11
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java

@@ -29,10 +29,10 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 /**
  * Tests error conditions in ChainMapper/ChainReducer.
@@ -112,7 +112,7 @@ public class TestChainErrors extends HadoopTestCase {
         LongWritable.class, Text.class, null);
 
     job.waitForCompletion(true);
-    assertTrue("Job Not failed", !job.isSuccessful());
+    assertTrue(!job.isSuccessful(), "Job Not failed");
   }
 
   /**
@@ -138,7 +138,7 @@ public class TestChainErrors extends HadoopTestCase {
         LongWritable.class, Text.class, null);
 
     job.waitForCompletion(true);
-    assertTrue("Job Not failed", !job.isSuccessful());
+    assertTrue(!job.isSuccessful(), "Job Not failed");
   }
 
   /**
@@ -161,9 +161,9 @@ public class TestChainErrors extends HadoopTestCase {
         LongWritable.class, Text.class, null);
 
     job.waitForCompletion(true);
-    assertTrue("Job failed", job.isSuccessful());
-    assertEquals("Outputs doesn't match", expectedOutput, MapReduceTestUtil
-        .readOutput(outDir, conf));
+    assertTrue(job.isSuccessful(), "Job failed");
+    assertEquals(expectedOutput, MapReduceTestUtil
+        .readOutput(outDir, conf), "Outputs doesn't match");
   }
 
   /**
@@ -189,9 +189,9 @@ public class TestChainErrors extends HadoopTestCase {
         LongWritable.class, Text.class, null);
 
     job.waitForCompletion(true);
-    assertTrue("Job failed", job.isSuccessful());
-    assertEquals("Outputs doesn't match", expectedOutput, MapReduceTestUtil
-        .readOutput(outDir, conf));
+    assertTrue(job.isSuccessful(), "Job failed");
+    assertEquals(expectedOutput, MapReduceTestUtil
+        .readOutput(outDir, conf), "Outputs doesn't match");
   }
 
   // this map consumes all the input and output nothing

+ 29 - 29
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java

@@ -30,11 +30,11 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
-import static org.junit.Assert.fail;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class TestMapReduceChain extends HadoopTestCase {
 
@@ -110,31 +110,31 @@ public class TestMapReduceChain extends HadoopTestCase {
         LongWritable.class, Text.class, null);
 
     job.waitForCompletion(true);
-    assertTrue("Job failed", job.isSuccessful());
+    assertTrue(job.isSuccessful(), "Job failed");
 
     String str = "flag not set";
-    assertTrue(str, getFlag(conf, "map.setup.A"));
-    assertTrue(str, getFlag(conf, "map.setup.B"));
-    assertTrue(str, getFlag(conf, "map.setup.C"));
-    assertTrue(str, getFlag(conf, "reduce.setup.R"));
-    assertTrue(str, getFlag(conf, "map.setup.D"));
-    assertTrue(str, getFlag(conf, "map.setup.E"));
-    assertTrue(str, getFlag(conf, "map.setup.F"));
-
-    assertTrue(str, getFlag(conf, "map.A.value.1"));
-    assertTrue(str, getFlag(conf, "map.A.value.2"));
-    assertTrue(str, getFlag(conf, "map.B.value.1A"));
-    assertTrue(str, getFlag(conf, "map.B.value.2A"));
-    assertTrue(str, getFlag(conf, "map.C.value.1AB"));
-    assertTrue(str, getFlag(conf, "map.C.value.2AB"));
-    assertTrue(str, getFlag(conf, "reduce.R.value.1ABC"));
-    assertTrue(str, getFlag(conf, "reduce.R.value.2ABC"));
-    assertTrue(str, getFlag(conf, "map.D.value.1ABCR"));
-    assertTrue(str, getFlag(conf, "map.D.value.2ABCR"));
-    assertTrue(str, getFlag(conf, "map.E.value.1ABCRD"));
-    assertTrue(str, getFlag(conf, "map.E.value.2ABCRD"));
-    assertTrue(str, getFlag(conf, "map.F.value.1ABCRDE"));
-    assertTrue(str, getFlag(conf, "map.F.value.2ABCRDE"));
+    assertTrue(getFlag(conf, "map.setup.A"), str);
+    assertTrue(getFlag(conf, "map.setup.B"), str);
+    assertTrue(getFlag(conf, "map.setup.C"), str);
+    assertTrue(getFlag(conf, "reduce.setup.R"), str);
+    assertTrue(getFlag(conf, "map.setup.D"), str);
+    assertTrue(getFlag(conf, "map.setup.E"), str);
+    assertTrue(getFlag(conf, "map.setup.F"), str);
+
+    assertTrue(getFlag(conf, "map.A.value.1"), str);
+    assertTrue(getFlag(conf, "map.A.value.2"), str);
+    assertTrue(getFlag(conf, "map.B.value.1A"), str);
+    assertTrue(getFlag(conf, "map.B.value.2A"), str);
+    assertTrue(getFlag(conf, "map.C.value.1AB"), str);
+    assertTrue(getFlag(conf, "map.C.value.2AB"), str);
+    assertTrue(getFlag(conf, "reduce.R.value.1ABC"), str);
+    assertTrue(getFlag(conf, "reduce.R.value.2ABC"), str);
+    assertTrue(getFlag(conf, "map.D.value.1ABCR"), str);
+    assertTrue(getFlag(conf, "map.D.value.2ABCR"), str);
+    assertTrue(getFlag(conf, "map.E.value.1ABCRD"), str);
+    assertTrue(getFlag(conf, "map.E.value.2ABCRD"), str);
+    assertTrue(getFlag(conf, "map.F.value.1ABCRDE"), str);
+    assertTrue(getFlag(conf, "map.F.value.2ABCRDE"), str);
 
     assertTrue(getFlag(conf, "map.cleanup.A"));
     assertTrue(getFlag(conf, "map.cleanup.B"));
@@ -144,8 +144,8 @@ public class TestMapReduceChain extends HadoopTestCase {
     assertTrue(getFlag(conf, "map.cleanup.E"));
     assertTrue(getFlag(conf, "map.cleanup.F"));
 
-    assertEquals("Outputs doesn't match", expectedOutput, MapReduceTestUtil
-        .readOutput(outDir, conf));
+    assertEquals(expectedOutput, MapReduceTestUtil
+        .readOutput(outDir, conf), "Outputs doesn't match");
   }
 
   public static class AMap extends IDMap {

+ 6 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java

@@ -26,9 +26,9 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.lib.map.TokenCounterMapper;
 import org.apache.hadoop.mapreduce.lib.reduce.IntSumReducer;
-import org.junit.Test;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import org.junit.jupiter.api.Test;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.IOException;
 
@@ -64,9 +64,9 @@ public class TestSingleElementChain extends HadoopTestCase {
         IntWritable.class, Text.class, IntWritable.class, null);
 
     job.waitForCompletion(true);
-    assertTrue("Job failed", job.isSuccessful());
-    assertEquals("Outputs doesn't match", expectedOutput, MapReduceTestUtil
-        .readOutput(outDir, conf));
+    assertTrue(job.isSuccessful(), "Job failed");
+    assertEquals(expectedOutput, MapReduceTestUtil
+        .readOutput(outDir, conf), "Outputs doesn't match");
   }
 
 }

+ 4 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDBOutputFormat.java

@@ -24,11 +24,11 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.mapreduce.Job;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.fail;
 
 public class TestDBOutputFormat {
   private String[] fieldNames = new String[] { "id", "name", "value" };

+ 10 - 10
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java

@@ -31,9 +31,9 @@ import org.apache.hadoop.mapreduce.TaskCounter;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.util.StringUtils;
 import org.hsqldb.server.Server;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -48,8 +48,8 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 
-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 org.apache.hadoop.examples.DBCountPageView;
 
@@ -124,13 +124,13 @@ public class TestDataDrivenDBInputFormat extends HadoopTestCase {
     createConnection(driverClassName, url);
   }
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     initialize(DRIVER_CLASS, DB_URL);
     super.setUp();
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     super.tearDown();
     shutdown();
@@ -230,11 +230,11 @@ public class TestDataDrivenDBInputFormat extends HadoopTestCase {
         COL, COL);
 
     boolean ret = job.waitForCompletion(true);
-    assertTrue("job failed", ret);
+    assertTrue(ret, "job failed");
 
     // Check to see that we imported as much as we thought we did.
-    assertEquals("Did not get all the records", 4,
+    assertEquals(4,
         job.getCounters().findCounter(TaskCounter.REDUCE_OUTPUT_RECORDS)
-        .getValue());
+        .getValue(), "Did not get all the records");
   }
 }

+ 6 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestIntegerSplitter.java

@@ -17,13 +17,13 @@
  */
 package org.apache.hadoop.mapreduce.lib.db;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.sql.SQLException;
 import java.util.List;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
 
 public class TestIntegerSplitter {
   private long [] toLongArray(List<Long> in) {
@@ -55,9 +55,9 @@ public class TestIntegerSplitter {
   public void assertLongArrayEquals(long [] expected, long [] actual) {
     for (int i = 0; i < expected.length; i++) {
       try {
-        assertEquals("Failure at position " + i + "; got " + actual[i]
-            + " instead of " + expected[i] + "; actual array is " + formatLongArray(actual),
-            expected[i], actual[i]);
+        assertEquals(expected[i], actual[i],
+            "Failure at position " + i + "; got " + actual[i]
+            + " instead of " + expected[i] + "; actual array is " + formatLongArray(actual));
       } catch (ArrayIndexOutOfBoundsException oob) {
         fail("Expected array with " + expected.length + " elements; got " + actual.length
             + ". Actual array is " + formatLongArray(actual));

+ 6 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestTextSplitter.java

@@ -17,14 +17,14 @@
  */
 package org.apache.hadoop.mapreduce.lib.db;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.math.BigDecimal;
 import java.sql.SQLException;
 import java.util.List;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
 
 public class TestTextSplitter {
 
@@ -48,9 +48,9 @@ public class TestTextSplitter {
   public void assertArrayEquals(Object [] expected, Object [] actual) {
     for (int i = 0; i < expected.length; i++) {
       try {
-        assertEquals("Failure at position " + i + "; got " + actual[i]
-            + " instead of " + expected[i] + "; actual array is " + formatArray(actual),
-            expected[i], actual[i]);
+        assertEquals(expected[i], actual[i],
+            "Failure at position " + i + "; got " + actual[i]
+            + " instead of " + expected[i] + "; actual array is " + formatArray(actual));
       } catch (ArrayIndexOutOfBoundsException oob) {
         fail("Expected array with " + expected.length + " elements; got " + actual.length
             + ". Actual array is " + formatArray(actual));

+ 5 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/fieldsel/TestMRFieldSelection.java

@@ -23,12 +23,12 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.text.NumberFormat;
 
-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;
 
 public class TestMRFieldSelection {
 
@@ -70,7 +70,7 @@ private static NumberFormat idFormat = NumberFormat.getInstance();
     job.setNumReduceTasks(1);
 
     job.waitForCompletion(true);
-    assertTrue("Job Failed!", job.isSuccessful());
+    assertTrue(job.isSuccessful(), "Job Failed!");
 
     //
     // Finally, we compare the reconstructed answer key with the
@@ -78,7 +78,7 @@ private static NumberFormat idFormat = NumberFormat.getInstance();
     // in the original key.
     //
     String outdata = MapReduceTestUtil.readOutput(outDir, conf);
-    assertEquals("Outputs doesnt match.",expectedOutput.toString(), outdata);
+    assertEquals(expectedOutput.toString(), outdata, "Outputs doesnt match.");
     fs.delete(outDir, true);
   }
 

+ 15 - 15
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java

@@ -55,18 +55,18 @@ import org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat.OneBlockInfo
 import org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat.OneFileInfo;
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
 
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.mockito.Mock;
 import org.mockito.MockitoAnnotations;
 
 import org.apache.hadoop.thirdparty.com.google.common.collect.HashMultiset;
 
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.Mockito.atLeastOnce;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.reset;
@@ -106,7 +106,7 @@ public class TestCombineFileInputFormat {
   @Mock
   private List<String> mockList;
 
-  @Before
+  @BeforeEach
   public void initMocks() {
     MockitoAnnotations.initMocks(this);
   }
@@ -261,12 +261,12 @@ public class TestCombineFileInputFormat {
     CombineFileSplit split = new CombineFileSplit(files, lengths);
 
     RecordReader rr = inputFormat.createRecordReader(split, context1);
-    assertTrue("Unexpected RR type!", rr instanceof CombineFileRecordReader);
+    assertTrue(rr instanceof CombineFileRecordReader, "Unexpected RR type!");
 
     // Verify that the initial configuration is the one being used.
     // Right after construction the dummy key should have value "STATE1"
-    assertEquals("Invalid initial dummy key value", "STATE1",
-      rr.getCurrentKey().toString());
+    assertEquals("STATE1", rr.getCurrentKey().toString(),
+        "Invalid initial dummy key value");
 
     // Switch the active context for the RecordReader...
     Configuration conf2 = new Configuration();
@@ -275,8 +275,8 @@ public class TestCombineFileInputFormat {
     rr.initialize(split, context2);
 
     // And verify that the new context is updated into the child record reader.
-    assertEquals("Invalid secondary dummy key value", "STATE2",
-      rr.getCurrentKey().toString());
+    assertEquals("STATE2", rr.getCurrentKey().toString(),
+        "Invalid secondary dummy key value");
   }
 
   @Test
@@ -297,7 +297,7 @@ public class TestCombineFileInputFormat {
 
     CombineFileSplit split = new CombineFileSplit(files, lengths);
     RecordReader rr = inputFormat.createRecordReader(split, context);
-    assertTrue("Unexpected RR type!", rr instanceof CombineFileRecordReader);
+    assertTrue(rr instanceof CombineFileRecordReader, "Unexpected RR type!");
 
     // first initialize() call comes from MapTask. We'll do it here.
     rr.initialize(split, context);
@@ -1485,8 +1485,8 @@ public class TestCombineFileInputFormat {
        * {@link CombineFileInputFormat#createSplits},
        * create only one split on rack1. Otherwise create two splits.
        */
-      assertTrue("Split size should be 1 or 2.",
-          splits.size() == 1 || splits.size() == 2);
+      assertTrue(splits.size() == 1 || splits.size() == 2,
+          "Split size should be 1 or 2.");
       actual.clear();
       reset(mockList);
       for (InputSplit split : splits) {

+ 15 - 13
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineSequenceFileInputFormat.java

@@ -18,9 +18,9 @@
 
 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.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
 
 import java.io.IOException;
 import java.util.BitSet;
@@ -41,7 +41,8 @@ import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.task.MapContextImpl;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -64,7 +65,8 @@ public class TestCombineSequenceFileInputFormat {
     new Path(new Path(System.getProperty("test.build.data", "."), "data"),
              "TestCombineSequenceFileInputFormat");
 
-  @Test(timeout=10000)
+  @Test
+  @Timeout(value = 10)
   public void testFormat() throws IOException, InterruptedException {
     Job job = Job.getInstance(conf);
 
@@ -95,10 +97,10 @@ public class TestCombineSequenceFileInputFormat {
 
       // we should have a single split as the length is comfortably smaller than
       // the block size
-      assertEquals("We got more than one splits!", 1, splits.size());
+      assertEquals(1, splits.size(), "We got more than one splits!");
       InputSplit split = splits.get(0);
-      assertEquals("It should be CombineFileSplit",
-        CombineFileSplit.class, split.getClass());
+      assertEquals(CombineFileSplit.class, split.getClass(),
+          "It should be CombineFileSplit");
 
       // check the split
       BitSet bits = new BitSet(length);
@@ -109,23 +111,23 @@ public class TestCombineSequenceFileInputFormat {
         context.getTaskAttemptID(), reader, null, null,
         MapReduceTestUtil.createDummyReporter(), split);
       reader.initialize(split, mcontext);
-      assertEquals("reader class is CombineFileRecordReader.",
-        CombineFileRecordReader.class, reader.getClass());
+      assertEquals(CombineFileRecordReader.class, reader.getClass(),
+          "reader class is CombineFileRecordReader.");
 
       try {
         while (reader.nextKeyValue()) {
           IntWritable key = reader.getCurrentKey();
           BytesWritable value = reader.getCurrentValue();
-          assertNotNull("Value should not be null.", value);
+          assertNotNull(value, "Value should not be null.");
           final int k = key.get();
           LOG.debug("read " + k);
-          assertFalse("Key in multiple partitions.", bits.get(k));
+          assertFalse(bits.get(k), "Key in multiple partitions.");
           bits.set(k);
         }
       } finally {
         reader.close();
       }
-      assertEquals("Some keys in no partition.", length, bits.cardinality());
+      assertEquals(length, bits.cardinality(), "Some keys in no partition.");
     }
   }
 

+ 22 - 19
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineTextInputFormat.java

@@ -18,10 +18,10 @@
 
 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.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.IOException;
 import java.io.OutputStream;
@@ -48,7 +48,8 @@ import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.task.MapContextImpl;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -72,7 +73,8 @@ public class TestCombineTextInputFormat {
     new Path(new Path(System.getProperty("test.build.data", "."), "data"),
              "TestCombineTextInputFormat");
 
-  @Test(timeout=10000)
+  @Test
+  @Timeout(value = 10)
   public void testFormat() throws Exception {
     Job job = Job.getInstance(new Configuration(defaultConf));
 
@@ -100,10 +102,10 @@ public class TestCombineTextInputFormat {
 
       // we should have a single split as the length is comfortably smaller than
       // the block size
-      assertEquals("We got more than one splits!", 1, splits.size());
+      assertEquals(1, splits.size(), "We got more than one splits!");
       InputSplit split = splits.get(0);
-      assertEquals("It should be CombineFileSplit",
-        CombineFileSplit.class, split.getClass());
+      assertEquals(CombineFileSplit.class, split.getClass(),
+          "It should be CombineFileSplit");
 
       // check the split
       BitSet bits = new BitSet(length);
@@ -112,8 +114,8 @@ public class TestCombineTextInputFormat {
         createDummyMapTaskAttemptContext(job.getConfiguration());
       RecordReader<LongWritable, Text> reader =
         format.createRecordReader(split, context);
-      assertEquals("reader class is CombineFileRecordReader.",
-        CombineFileRecordReader.class, reader.getClass());
+      assertEquals(CombineFileRecordReader.class, reader.getClass(),
+          "reader class is CombineFileRecordReader.");
       MapContext<LongWritable,Text,LongWritable,Text> mcontext =
         new MapContextImpl<LongWritable,Text,LongWritable,Text>(job.getConfiguration(),
         context.getTaskAttemptID(), reader, null, null,
@@ -124,11 +126,11 @@ public class TestCombineTextInputFormat {
         int count = 0;
         while (reader.nextKeyValue()) {
           LongWritable key = reader.getCurrentKey();
-          assertNotNull("Key should not be null.", key);
+          assertNotNull(key, "Key should not be null.");
           Text value = reader.getCurrentValue();
           final int v = Integer.parseInt(value.toString());
           LOG.debug("read " + v);
-          assertFalse("Key in multiple partitions.", bits.get(v));
+          assertFalse(bits.get(v), "Key in multiple partitions.");
           bits.set(v);
           count++;
         }
@@ -136,7 +138,7 @@ public class TestCombineTextInputFormat {
       } finally {
         reader.close();
       }
-      assertEquals("Some keys in no partition.", length, bits.cardinality());
+      assertEquals(length, bits.cardinality(), "Some keys in no partition.");
     }
   }
 
@@ -223,7 +225,8 @@ public class TestCombineTextInputFormat {
   /**
    * Test using the gzip codec for reading
    */
-  @Test(timeout=10000)
+  @Test
+  @Timeout(value = 10)
   public void testGzip() throws IOException, InterruptedException {
     Configuration conf = new Configuration(defaultConf);
     CompressionCodec gzip = new GzipCodec();
@@ -237,9 +240,9 @@ public class TestCombineTextInputFormat {
     FileInputFormat.setInputPaths(job, workDir);
     CombineTextInputFormat format = new CombineTextInputFormat();
     List<InputSplit> splits = format.getSplits(job);
-    assertEquals("compressed splits == 1", 1, splits.size());
+    assertEquals(1, splits.size(), "compressed splits == 1");
     List<Text> results = readSplit(format, splits.get(0), job);
-    assertEquals("splits[0] length", 8, results.size());
+    assertEquals(8, results.size(), "splits[0] length");
 
     final String[] firstList =
       {"the quick", "brown", "fox jumped", "over", " the lazy", " dog"};
@@ -257,11 +260,11 @@ public class TestCombineTextInputFormat {
   private static void testResults(List<Text> results, String[] first,
     String[] second) {
     for (int i = 0; i < first.length; i++) {
-      assertEquals("splits[0]["+i+"]", first[i], results.get(i).toString());
+      assertEquals(first[i], results.get(i).toString(), "splits[0][" + i + "]");
     }
     for (int i = 0; i < second.length; i++) {
       int j = i + first.length;
-      assertEquals("splits[0]["+j+"]", second[i], results.get(j).toString());
+      assertEquals(second[i], results.get(j).toString(), "splits[0][" + j + "]");
     }
   }
 }

+ 7 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestDelegatingInputFormat.java

@@ -20,8 +20,9 @@ package org.apache.hadoop.mapreduce.lib.input;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.List;
-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.assertTrue;
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -83,10 +84,10 @@ public class TestDelegatingInputFormat {
        bins[index]++;
       }
 
-      assertEquals("count is not equal to num splits", numSplits, bins[0]);
-      assertEquals("count is not equal to num splits", numSplits, bins[1]);
-      assertEquals("count is not equal to 2 * num splits",
-        numSplits * 2, bins[2]);
+      assertEquals(numSplits, bins[0], "count is not equal to num splits");
+      assertEquals(numSplits, bins[1], "count is not equal to num splits");
+      assertEquals(numSplits * 2, bins[2],
+          "count is not equal to 2 * num splits");
     } finally {
       if (dfs != null) {
        dfs.shutdown();

+ 44 - 34
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestFixedLengthInputFormat.java

@@ -40,12 +40,14 @@ import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.task.MapContextImpl;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.junit.Assert.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class TestFixedLengthInputFormat {
 
@@ -60,7 +62,7 @@ public class TestFixedLengthInputFormat {
   private static char[] chars;
   private static Random charRand;
 
-  @BeforeClass
+  @BeforeAll
   public static void onlyOnce() {
     try {
       defaultConf = new Configuration();
@@ -82,7 +84,8 @@ public class TestFixedLengthInputFormat {
    * 20 random tests of various record, file, and split sizes.  All tests have
    * uncompressed file as input.
    */
-  @Test (timeout=500000)
+  @Test
+  @Timeout(value = 500)
   public void testFormat() throws Exception {
     runRandomTests(null);
   }
@@ -91,7 +94,8 @@ public class TestFixedLengthInputFormat {
    * 20 random tests of various record, file, and split sizes.  All tests have
    * compressed file as input.
    */
-  @Test (timeout=500000)
+  @Test
+  @Timeout(value = 500)
   public void testFormatCompressedIn() throws Exception {
     runRandomTests(new GzipCodec());
   }
@@ -99,7 +103,8 @@ public class TestFixedLengthInputFormat {
   /**
    * Test with no record length set.
    */
-  @Test (timeout=5000)
+  @Test
+  @Timeout(value = 5)
   public void testNoRecordLength() throws Exception {
     localFs.delete(workDir, true);
     Path file = new Path(workDir, "testFormat.txt");
@@ -127,13 +132,14 @@ public class TestFixedLengthInputFormat {
         LOG.info("Exception message:" + ioe.getMessage());
       }
     }
-    assertTrue("Exception for not setting record length:", exceptionThrown);
+    assertTrue(exceptionThrown, "Exception for not setting record length:");
   }
 
   /**
    * Test with record length set to 0
    */
-  @Test (timeout=5000)
+  @Test
+  @Timeout(value = 5)
   public void testZeroRecordLength() throws Exception {
     localFs.delete(workDir, true);
     Path file = new Path(workDir, "testFormat.txt");
@@ -163,13 +169,14 @@ public class TestFixedLengthInputFormat {
         LOG.info("Exception message:" + ioe.getMessage());
       }
     }
-    assertTrue("Exception for zero record length:", exceptionThrown);
+    assertTrue(exceptionThrown, "Exception for zero record length:");
   }
 
   /**
    * Test with record length set to a negative value
    */
-  @Test (timeout=5000)
+  @Test
+  @Timeout(value = 5)
   public void testNegativeRecordLength() throws Exception {
     localFs.delete(workDir, true);
     Path file = new Path(workDir, "testFormat.txt");
@@ -198,13 +205,14 @@ public class TestFixedLengthInputFormat {
         LOG.info("Exception message:" + ioe.getMessage());
       }
     }
-    assertTrue("Exception for negative record length:", exceptionThrown);
+    assertTrue(exceptionThrown, "Exception for negative record length:");
   }
 
   /**
    * Test with partial record at the end of a compressed input file.
    */
-  @Test (timeout=5000)
+  @Test
+  @Timeout(value = 5)
   public void testPartialRecordCompressedIn() throws Exception {
     CompressionCodec gzip = new GzipCodec();
     runPartialRecordTest(gzip);
@@ -213,7 +221,8 @@ public class TestFixedLengthInputFormat {
   /**
    * Test with partial record at the end of an uncompressed input file.
    */
-  @Test (timeout=5000)
+  @Test
+  @Timeout(value = 5)
   public void testPartialRecordUncompressedIn() throws Exception {
     runPartialRecordTest(null);
   }
@@ -221,7 +230,8 @@ public class TestFixedLengthInputFormat {
   /**
    * Test using the gzip codec with two input files.
    */
-  @Test (timeout=5000)
+  @Test
+  @Timeout(value = 5)
   public void testGzipWithTwoInputs() throws Exception {
     CompressionCodec gzip = new GzipCodec();
     localFs.delete(workDir, true);
@@ -236,19 +246,19 @@ public class TestFixedLengthInputFormat {
     writeFile(localFs, new Path(workDir, "part2.txt.gz"), gzip,
         "ten  nine eightsevensix  five four threetwo  one  ");
     List<InputSplit> splits = format.getSplits(job);
-    assertEquals("compressed splits == 2", 2, splits.size());
+    assertEquals(2, splits.size(), "compressed splits == 2");
     FileSplit tmp = (FileSplit) splits.get(0);
     if (tmp.getPath().getName().equals("part2.txt.gz")) {
       splits.set(0, splits.get(1));
       splits.set(1, tmp);
     }
     List<String> results = readSplit(format, splits.get(0), job);
-    assertEquals("splits[0] length", 10, results.size());
-    assertEquals("splits[0][5]", "six  ", results.get(5));
+    assertEquals(10, results.size(), "splits[0] length");
+    assertEquals("six  ", results.get(5), "splits[0][5]");
     results = readSplit(format, splits.get(1), job);
-    assertEquals("splits[1] length", 10, results.size());
-    assertEquals("splits[1][0]", "ten  ", results.get(0));
-    assertEquals("splits[1][1]", "nine ", results.get(1));
+    assertEquals(10, results.size(), "splits[1] length");
+    assertEquals("ten  ", results.get(0), "splits[1][0]");
+    assertEquals("nine ", results.get(1), "splits[1][1]");
   }
 
   // Create a file containing fixed length records with random data
@@ -366,28 +376,28 @@ public class TestFixedLengthInputFormat {
             reader, null, null, MapReduceTestUtil.createDummyReporter(), split);
         reader.initialize(split, mcontext);
         Class<?> clazz = reader.getClass();
-        assertEquals("RecordReader class should be FixedLengthRecordReader:", 
-            FixedLengthRecordReader.class, clazz);
+        assertEquals(FixedLengthRecordReader.class, clazz,
+            "RecordReader class should be FixedLengthRecordReader:");
         // Plow through the records in this split
         while (reader.nextKeyValue()) {
           key = reader.getCurrentKey();
           value = reader.getCurrentValue();
-          assertEquals("Checking key", (long)(recordNumber*recordLength),
-              key.get());
+          assertEquals((long) (recordNumber * recordLength),
+              key.get(), "Checking key");
           String valueString = new String(value.getBytes(), 0,
               value.getLength());
-          assertEquals("Checking record length:", recordLength,
-              value.getLength());
-          assertTrue("Checking for more records than expected:",
-              recordNumber < totalRecords);
+          assertEquals(recordLength,
+              value.getLength(), "Checking record length:");
+          assertTrue(recordNumber < totalRecords,
+              "Checking for more records than expected:");
           String origRecord = recordList.get(recordNumber);
-          assertEquals("Checking record content:", origRecord, valueString);
+          assertEquals(origRecord, valueString, "Checking record content:");
           recordNumber++;
         }
         reader.close();
       }
-      assertEquals("Total original records should be total read records:",
-          recordList.size(), recordNumber);
+      assertEquals(recordList.size(), recordNumber,
+          "Total original records should be total read records:");
     }
   }
 
@@ -449,7 +459,7 @@ public class TestFixedLengthInputFormat {
     FileInputFormat.setInputPaths(job, workDir);
     List<InputSplit> splits = format.getSplits(job);
     if (codec != null) {
-      assertEquals("compressed splits == 1", 1, splits.size());
+      assertEquals(1, splits.size(), "compressed splits == 1");
     }
     boolean exceptionThrown = false;
     for (InputSplit split : splits) {
@@ -460,7 +470,7 @@ public class TestFixedLengthInputFormat {
         LOG.info("Exception message:" + ioe.getMessage());
       }
     }
-    assertTrue("Exception for partial record:", exceptionThrown);
+    assertTrue(exceptionThrown, "Exception for partial record:");
   }
 
 }

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestLineRecordReaderJobs.java

@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.mapreduce.lib.input;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.io.IOException;
 import java.io.OutputStreamWriter;
@@ -32,7 +32,7 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 public class TestLineRecordReaderJobs {
 

+ 8 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRCJCFileInputFormat.java

@@ -23,10 +23,14 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
-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.assertEquals;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;

+ 41 - 40
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRKeyValueTextInputFormat.java

@@ -41,12 +41,14 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.task.MapContextImpl;
 import org.apache.hadoop.util.LineReader;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
-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.assertTrue;
 
 public class TestMRKeyValueTextInputFormat {
   private static final Logger LOG =
@@ -115,8 +117,8 @@ public class TestMRKeyValueTextInputFormat {
           RecordReader<Text, Text> reader = format.createRecordReader(
             splits.get(j), context);
           Class<?> clazz = reader.getClass();
-          assertEquals("reader class is KeyValueLineRecordReader.", 
-            KeyValueLineRecordReader.class, clazz);
+          assertEquals(KeyValueLineRecordReader.class, clazz,
+              "reader class is KeyValueLineRecordReader.");
           MapContext<Text, Text, Text, Text> mcontext = 
             new MapContextImpl<Text, Text, Text, Text>(job.getConfiguration(), 
             context.getTaskAttemptID(), reader, null, null, 
@@ -130,16 +132,16 @@ public class TestMRKeyValueTextInputFormat {
             while (reader.nextKeyValue()) {
               key = reader.getCurrentKey();
               clazz = key.getClass();
-              assertEquals("Key class is Text.", Text.class, clazz);
+              assertEquals(Text.class, clazz, "Key class is Text.");
               value = reader.getCurrentValue();
               clazz = value.getClass();
-              assertEquals("Value class is Text.", Text.class, clazz);
+              assertEquals(Text.class, clazz, "Value class is Text.");
               final int k = Integer.parseInt(key.toString());
               final int v = Integer.parseInt(value.toString());
-              assertEquals("Bad key", 0, k % 2);
-              assertEquals("Mismatched key/value", k / 2, v);
+              assertEquals(0, k % 2, "Bad key");
+              assertEquals(k / 2, v, "Mismatched key/value");
               LOG.debug("read " + v);
-              assertFalse("Key in multiple partitions.", bits.get(v));
+              assertFalse(bits.get(v), "Key in multiple partitions.");
               bits.set(v);
               count++;
             }
@@ -148,7 +150,7 @@ public class TestMRKeyValueTextInputFormat {
             reader.close();
           }
         }
-        assertEquals("Some keys in no partition.", length, bits.cardinality());
+        assertEquals(length, bits.cardinality(), "Some keys in no partition.");
       }
 
     }
@@ -200,7 +202,7 @@ public class TestMRKeyValueTextInputFormat {
 
       // try splitting the file in a variety of sizes
       KeyValueTextInputFormat format = new KeyValueTextInputFormat();
-      assertTrue("KVTIF claims not splittable", format.isSplitable(job, file));
+      assertTrue(format.isSplitable(job, file), "KVTIF claims not splittable");
       for (int i = 0; i < 3; i++) {
         int numSplits = random.nextInt(MAX_LENGTH / 2000) + 1;
         LOG.info("splitting: requesting = " + numSplits);
@@ -231,10 +233,10 @@ public class TestMRKeyValueTextInputFormat {
               value = reader.getCurrentValue();
               final int k = Integer.parseInt(key.toString());
               final int v = Integer.parseInt(value.toString());
-              assertEquals("Bad key", 0, k % 2);
-              assertEquals("Mismatched key/value", k / 2, v);
+              assertEquals(0, k % 2, "Bad key");
+              assertEquals(k / 2, v, "Mismatched key/value");
               LOG.debug("read " + k + "," + v);
-              assertFalse(k + "," + v + " in multiple partitions.",bits.get(v));
+              assertFalse(bits.get(v), k + "," + v + " in multiple partitions.");
               bits.set(v);
               count++;
             }
@@ -247,7 +249,7 @@ public class TestMRKeyValueTextInputFormat {
             reader.close();
           }
         }
-        assertEquals("Some keys in no partition.", length, bits.cardinality());
+        assertEquals(length, bits.cardinality(), "Some keys in no partition.");
       }
 
     }
@@ -262,11 +264,12 @@ public class TestMRKeyValueTextInputFormat {
     LineReader in = makeStream("abcd\u20acbdcd\u20ac");
     Text line = new Text();
     in.readLine(line);
-    assertEquals("readLine changed utf8 characters", 
-                 "abcd\u20acbdcd\u20ac", line.toString());
+    assertEquals("abcd\u20acbdcd\u20ac", line.toString(),
+        "readLine changed utf8 characters");
     in = makeStream("abc\u200axyz");
     in.readLine(line);
-    assertEquals("split on fake newline", "abc\u200axyz", line.toString());
+    assertEquals("abc\u200axyz", line.toString(),
+        "split on fake newline");
   }
 
   @Test
@@ -274,18 +277,18 @@ public class TestMRKeyValueTextInputFormat {
     LineReader in = makeStream("a\nbb\n\nccc\rdddd\r\neeeee");
     Text out = new Text();
     in.readLine(out);
-    assertEquals("line1 length", 1, out.getLength());
+    assertEquals(1, out.getLength(), "line1 length");
     in.readLine(out);
-    assertEquals("line2 length", 2, out.getLength());
+    assertEquals(2, out.getLength(), "line2 length");
     in.readLine(out);
-    assertEquals("line3 length", 0, out.getLength());
+    assertEquals(0, out.getLength(), "line3 length");
     in.readLine(out);
-    assertEquals("line4 length", 3, out.getLength());
+    assertEquals(3, out.getLength(), "line4 length");
     in.readLine(out);
-    assertEquals("line5 length", 4, out.getLength());
+    assertEquals(4, out.getLength(), "line5 length");
     in.readLine(out);
-    assertEquals("line5 length", 5, out.getLength());
-    assertEquals("end of file", 0, in.readLine(out));
+    assertEquals(5, out.getLength(), "line5 length");
+    assertEquals(0, in.readLine(out), "end of file");
   }
   
   private static void writeFile(FileSystem fs, Path name, 
@@ -340,29 +343,27 @@ public class TestMRKeyValueTextInputFormat {
     FileInputFormat.setInputPaths(job, workDir);
     KeyValueTextInputFormat format = new KeyValueTextInputFormat();
     List<InputSplit> splits = format.getSplits(job);
-    assertEquals("compressed splits == 2", 2, splits.size());
+    assertEquals(2, splits.size(), "compressed splits == 2");
     FileSplit tmp = (FileSplit) splits.get(0);
     if (tmp.getPath().getName().equals("part2.txt.gz")) {
       splits.set(0, splits.get(1));
       splits.set(1, tmp);
     }
     List<Text> results = readSplit(format, splits.get(0), job);
-    assertEquals("splits[0] length", 6, results.size());
-    assertEquals("splits[0][0]", "the quick", results.get(0).toString());
-    assertEquals("splits[0][1]", "brown", results.get(1).toString());
-    assertEquals("splits[0][2]", "fox jumped", results.get(2).toString());
-    assertEquals("splits[0][3]", "over", results.get(3).toString());
-    assertEquals("splits[0][4]", " the lazy", results.get(4).toString());
-    assertEquals("splits[0][5]", " dog", results.get(5).toString());
+    assertEquals(6, results.size(), "splits[0] length");
+    assertEquals("the quick", results.get(0).toString(), "splits[0][0]");
+    assertEquals("brown", results.get(1).toString(), "splits[0][1]");
+    assertEquals("fox jumped", results.get(2).toString(), "splits[0][2]");
+    assertEquals("over", results.get(3).toString(), "splits[0][3]");
+    assertEquals(" the lazy", results.get(4).toString(), "splits[0][4]");
+    assertEquals(" dog", results.get(5).toString(), "splits[0][5]");
     results = readSplit(format, splits.get(1), job);
-    assertEquals("splits[1] length", 2, results.size());
-    assertEquals("splits[1][0]", "this is a test", 
-                 results.get(0).toString());    
-    assertEquals("splits[1][1]", "of gzip", 
-                 results.get(1).toString());    
+    assertEquals(2, results.size(), "splits[1] length");
+    assertEquals("this is a test", results.get(0).toString(), "splits[1][0]");
+    assertEquals("of gzip", results.get(1).toString(), "splits[1][1]");
   }
   
-  public static void main(String[] args) throws Exception {
+/*  public static void main(String[] args) throws Exception {
     new TestMRKeyValueTextInputFormat().testFormat();
-  }
+  }*/
 }

+ 10 - 12
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileAsBinaryInputFormat.java

@@ -32,13 +32,13 @@ import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.task.MapContextImpl;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.io.IOException;
 import java.util.Random;
 
-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;
 
 public class TestMRSequenceFileAsBinaryInputFormat {
   private static final int RECORDS = 10000;
@@ -103,21 +103,19 @@ public class TestMRSequenceFileAsBinaryInputFormat {
           cmpkey.readFields(buf);
           buf.reset(bval.getBytes(), bval.getLength());
           cmpval.readFields(buf);
-          assertTrue(
-            "Keys don't match: " + "*" + cmpkey.toString() + ":" +
-            tkey.toString() + "*",
-            cmpkey.toString().equals(tkey.toString()));
-          assertTrue(
-            "Vals don't match: " + "*" + cmpval.toString() + ":" +
-            tval.toString() + "*",
-            cmpval.toString().equals(tval.toString()));
+          assertTrue(cmpkey.toString().equals(tkey.toString()),
+              "Keys don't match: " + "*" + cmpkey.toString() + ":" +
+              tkey.toString() + "*");
+          assertTrue(cmpval.toString().equals(tval.toString()),
+              "Vals don't match: " + "*" + cmpval.toString() + ":" +
+              tval.toString() + "*");
           ++count;
         }
       } finally {
         reader.close();
       }
     }
-    assertEquals("Some records not found", RECORDS, count);
+    assertEquals(RECORDS, count, "Some records not found");
   }
 
 }

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

@@ -33,13 +33,13 @@ import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.task.MapContextImpl;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.util.BitSet;
 import java.util.Random;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
 
 public class TestMRSequenceFileAsTextInputFormat {
   private static int MAX_LENGTH = 10000;
@@ -100,15 +100,15 @@ public class TestMRSequenceFileAsTextInputFormat {
             split);
           reader.initialize(split, mcontext);
           Class<?> readerClass = reader.getClass();
-          assertEquals("reader class is SequenceFileAsTextRecordReader.",
-            SequenceFileAsTextRecordReader.class, readerClass);        
+          assertEquals(SequenceFileAsTextRecordReader.class, readerClass,
+              "reader class is SequenceFileAsTextRecordReader.");
           Text key;
           try {
             int count = 0;
             while (reader.nextKeyValue()) {
               key = reader.getCurrentKey();
               int keyInt = Integer.parseInt(key.toString());
-              assertFalse("Key in multiple partitions.", bits.get(keyInt));
+              assertFalse(bits.get(keyInt), "Key in multiple partitions.");
               bits.set(keyInt);
               count++;
             }
@@ -116,7 +116,7 @@ public class TestMRSequenceFileAsTextInputFormat {
             reader.close();
           }
         }
-        assertEquals("Some keys in no partition.", length, bits.cardinality());
+        assertEquals(length, bits.cardinality(), "Some keys in no partition.");
       }
 
     }

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRSequenceFileInputFilter.java

@@ -32,7 +32,7 @@ import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.task.MapContextImpl;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -40,7 +40,7 @@ import java.io.IOException;
 import java.util.Random;
 
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 public class TestMRSequenceFileInputFilter {
   private static final Logger LOG =

+ 4 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java

@@ -36,9 +36,9 @@ import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
-import org.junit.Before;
-import org.junit.Test;
-import static org.junit.Assert.assertEquals;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 /**
  * @see TestDelegatingInputFormat
@@ -64,7 +64,7 @@ public class TestMultipleInputs extends HadoopTestCase {
     return dir;
   }
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     super.setUp();
     Path rootDir = getDir(ROOT_DIR);

+ 10 - 10
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestNLineInputFormat.java

@@ -30,14 +30,14 @@ import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.task.MapContextImpl;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.io.IOException;
 import java.io.OutputStreamWriter;
 import java.io.Writer;
 import java.util.List;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 public class TestNLineInputFormat {
   private static int MAX_LENGTH = 200;
@@ -96,15 +96,15 @@ public class TestNLineInputFormat {
     List<InputSplit> splits = format.getSplits(job);
     int count = 0;
     for (int i = 0; i < splits.size(); i++) {
-      assertEquals("There are no split locations", 0,
-                   splits.get(i).getLocations().length);
+      assertEquals(0,
+          splits.get(i).getLocations().length, "There are no split locations");
       TaskAttemptContext context = MapReduceTestUtil.
         createDummyMapTaskAttemptContext(job.getConfiguration());
       RecordReader<LongWritable, Text> reader = format.createRecordReader(
         splits.get(i), context);
       Class<?> clazz = reader.getClass();
-      assertEquals("reader class is LineRecordReader.", 
-        LineRecordReader.class, clazz);
+      assertEquals(LineRecordReader.class, clazz,
+          "reader class is LineRecordReader.");
       MapContext<LongWritable, Text, LongWritable, Text> mcontext = 
         new MapContextImpl<LongWritable, Text, LongWritable, Text>(
           job.getConfiguration(), context.getTaskAttemptID(), reader, null,
@@ -120,11 +120,11 @@ public class TestNLineInputFormat {
         reader.close();
       }
       if ( i == splits.size() - 1) {
-        assertEquals("number of lines in split(" + i + ") is wrong" ,
-                     lastN, count);
+        assertEquals(lastN, count,
+            "number of lines in split(" + i + ") is wrong");
       } else {
-        assertEquals("number of lines in split(" + i + ") is wrong" ,
-                     expectedN, count);
+        assertEquals(expectedN, count,
+            "number of lines in split(" + i + ") is wrong");
       }
     }
   }

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

@@ -18,10 +18,10 @@
 
 package org.apache.hadoop.mapreduce.lib.jobcontrol;
 
-import static org.junit.Assert.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertFalse;
 
 import org.apache.hadoop.conf.Configuration;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 /**
  */

+ 11 - 9
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java

@@ -28,14 +28,15 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 /**
  * This class performs unit test for Job/JobControl classes.
@@ -202,12 +203,13 @@ public class TestMapReduceJobControl extends HadoopTestCase {
     // wait till all the jobs complete
     waitTillAllFinished(theControl);
     
-    assertEquals("Some jobs failed", 0, theControl.getFailedJobList().size());
+    assertEquals(0, theControl.getFailedJobList().size(), "Some jobs failed");
     
     theControl.stop();
   }
   
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testControlledJob() throws Exception {
     LOG.info("Starting testControlledJob");
 
@@ -222,11 +224,11 @@ public class TestMapReduceJobControl extends HadoopTestCase {
         break;
       }
     }
-    Assert.assertNotNull(cjob1.getMapredJobId());
+    assertNotNull(cjob1.getMapredJobId());
 
     // wait till all the jobs complete
     waitTillAllFinished(theControl);
-    assertEquals("Some jobs failed", 0, theControl.getFailedJobList().size());
+    assertEquals(0, theControl.getFailedJobList().size(), "Some jobs failed");
     theControl.stop();
   }
 }

+ 8 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControlWithMocks.java

@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.mapreduce.lib.jobcontrol;
 
-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.verify;
 import static org.mockito.Mockito.when;
@@ -30,7 +30,7 @@ import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.Job;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 /**
  * Tests the JobControl API using mock and stub Job instances.
@@ -48,8 +48,8 @@ public class TestMapReduceJobControlWithMocks {
     
     runJobControl(jobControl);
     
-    assertEquals("Success list", 4, jobControl.getSuccessfulJobList().size());
-    assertEquals("Failed list", 0, jobControl.getFailedJobList().size());
+    assertEquals(4, jobControl.getSuccessfulJobList().size(), "Success list");
+    assertEquals(0, jobControl.getFailedJobList().size(), "Failed list");
     
     assertEquals(ControlledJob.State.SUCCESS, job1.getJobState());
     assertEquals(ControlledJob.State.SUCCESS, job2.getJobState());
@@ -70,8 +70,8 @@ public class TestMapReduceJobControlWithMocks {
     
     runJobControl(jobControl);
     
-    assertEquals("Success list", 1, jobControl.getSuccessfulJobList().size());
-    assertEquals("Failed list", 3, jobControl.getFailedJobList().size());
+    assertEquals(1, jobControl.getSuccessfulJobList().size(), "Success list");
+    assertEquals(3, jobControl.getFailedJobList().size(), "Failed list");
 
     assertEquals(ControlledJob.State.FAILED, job1.getJobState());
     assertEquals(ControlledJob.State.SUCCESS, job2.getJobState());
@@ -95,8 +95,8 @@ public class TestMapReduceJobControlWithMocks {
     
     runJobControl(jobControl);
     try {
-      assertEquals("Success list", 0, jobControl.getSuccessfulJobList().size());
-      assertEquals("Failed list", 1, jobControl.getFailedJobList().size());
+      assertEquals(0, jobControl.getSuccessfulJobList().size(), "Success list");
+      assertEquals(1, jobControl.getFailedJobList().size(), "Failed list");
 
       assertEquals(ControlledJob.State.FAILED, job1.getJobState());
     } finally {

+ 28 - 28
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinDatamerge.java

@@ -33,25 +33,25 @@ import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
 
-import org.junit.AfterClass;
-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 static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class TestJoinDatamerge {
 
   private static MiniDFSCluster cluster = null;
 
-  @BeforeClass
+  @BeforeAll
   public static void setUp() throws Exception {
     Configuration conf = new Configuration();
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() throws Exception {
     if (cluster != null) {
       cluster.shutdown();
@@ -116,7 +116,7 @@ public class TestJoinDatamerge {
 
     public void setup(Context context) {
       srcs = context.getConfiguration().getInt("testdatamerge.sources", 0);
-      assertTrue("Invalid src count: " + srcs, srcs > 0);
+      assertTrue(srcs > 0, "Invalid src count: " + srcs);
     }
   }
 
@@ -128,7 +128,7 @@ public class TestJoinDatamerge {
 
     public void setup(Context context) {
       srcs = context.getConfiguration().getInt("testdatamerge.sources", 0);
-      assertTrue("Invalid src count: " + srcs, srcs > 0);
+      assertTrue(srcs > 0, "Invalid src count: " + srcs);
     }
 
     public void reduce(IntWritable key, Iterable<IntWritable> values,
@@ -137,7 +137,7 @@ public class TestJoinDatamerge {
       for (IntWritable value : values) {
         seen += value.get();
       }
-      assertTrue("Bad count for " + key.get(), verify(key.get(), seen));
+      assertTrue(verify(key.get(), seen), "Bad count for " + key.get());
       context.write(key, new IntWritable(seen));
     }
     
@@ -150,10 +150,10 @@ public class TestJoinDatamerge {
         throws IOException, InterruptedException {
       int k = key.get();
       final String kvstr = "Unexpected tuple: " + stringify(key, val);
-      assertTrue(kvstr, 0 == k % (srcs * srcs));
+      assertTrue(0 == k % (srcs * srcs), kvstr);
       for (int i = 0; i < val.size(); ++i) {
         final int vali = ((IntWritable)val.get(i)).get();
-        assertTrue(kvstr, (vali - i) * srcs == 10 * k);
+        assertTrue((vali - i) * srcs == 10 * k, kvstr);
       }
       context.write(key, one);
       // If the user modifies the key or any of the values in the tuple, it
@@ -181,18 +181,18 @@ public class TestJoinDatamerge {
       final String kvstr = "Unexpected tuple: " + stringify(key, val);
       if (0 == k % (srcs * srcs)) {
         for (int i = 0; i < val.size(); ++i) {
-          assertTrue(kvstr, val.get(i) instanceof IntWritable);
+          assertTrue(val.get(i) instanceof IntWritable, kvstr);
           final int vali = ((IntWritable)val.get(i)).get();
-          assertTrue(kvstr, (vali - i) * srcs == 10 * k);
+          assertTrue((vali - i) * srcs == 10 * k, kvstr);
         }
       } else {
         for (int i = 0; i < val.size(); ++i) {
           if (i == k % srcs) {
-            assertTrue(kvstr, val.get(i) instanceof IntWritable);
+            assertTrue(val.get(i) instanceof IntWritable, kvstr);
             final int vali = ((IntWritable)val.get(i)).get();
-            assertTrue(kvstr, srcs * (vali - i) == 10 * (k - i));
+            assertTrue(srcs * (vali - i) == 10 * (k - i), kvstr);
           } else {
-            assertTrue(kvstr, !val.has(i));
+            assertTrue(!val.has(i), kvstr);
           }
         }
       }
@@ -224,10 +224,10 @@ public class TestJoinDatamerge {
       final int vali = val.get();
       final String kvstr = "Unexpected tuple: " + stringify(key, val);
       if (0 == k % (srcs * srcs)) {
-        assertTrue(kvstr, vali == k * 10 / srcs + srcs - 1);
+        assertTrue(vali == k * 10 / srcs + srcs - 1, kvstr);
       } else {
         final int i = k % srcs;
-        assertTrue(kvstr, srcs * (vali - i) == 10 * (k - i));
+        assertTrue(srcs * (vali - i) == 10 * (k - i), kvstr);
       }
       context.write(key, one);
       //If the user modifies the key or any of the values in the tuple, it
@@ -267,7 +267,7 @@ public class TestJoinDatamerge {
     job.setOutputKeyClass(IntWritable.class);
     job.setOutputValueClass(IntWritable.class);
     job.waitForCompletion(true);
-    assertTrue("Job failed", job.isSuccessful());
+    assertTrue(job.isSuccessful(), "Job failed");
     if ("outer".equals(jointype)) {
       checkOuterConsistency(job, src);
     }
@@ -289,18 +289,18 @@ public class TestJoinDatamerge {
     Path outf = FileOutputFormat.getOutputPath(job);
     FileStatus[] outlist = cluster.getFileSystem().listStatus(outf, new 
                              Utils.OutputFileUtils.OutputFilesFilter());
-    assertEquals("number of part files is more than 1. It is" + outlist.length,
-      1, outlist.length);
-    assertTrue("output file with zero length" + outlist[0].getLen(),
-      0 < outlist[0].getLen());
+    assertEquals(1, outlist.length,
+        "number of part files is more than 1. It is" + outlist.length);
+    assertTrue(0 < outlist[0].getLen(),
+        "output file with zero length" + outlist[0].getLen());
     SequenceFile.Reader r =
       new SequenceFile.Reader(cluster.getFileSystem(),
           outlist[0].getPath(), job.getConfiguration());
     IntWritable k = new IntWritable();
     IntWritable v = new IntWritable();
     while (r.next(k, v)) {
-      assertEquals("counts does not match", v.get(),
-        countProduct(k, src, job.getConfiguration()));
+      assertEquals(v.get(), countProduct(k, src, job.getConfiguration()),
+          "counts does not match");
     }
     r.close();
   }
@@ -394,7 +394,7 @@ public class TestJoinDatamerge {
     job.setOutputValueClass(TupleWritable.class);
     job.setOutputFormatClass(SequenceFileOutputFormat.class);
     job.waitForCompletion(true);
-    assertTrue("Job failed", job.isSuccessful());
+    assertTrue(job.isSuccessful(), "Job failed");
 
     FileStatus[] outlist = cluster.getFileSystem().listStatus(outf, 
                              new Utils.OutputFileUtils.OutputFilesFilter());

+ 33 - 33
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinProperties.java

@@ -31,12 +31,12 @@ import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.*;
 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 import org.apache.hadoop.mapreduce.task.MapContextImpl;
-import org.junit.AfterClass;
-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 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;
 
 public class TestJoinProperties {
 
@@ -47,7 +47,7 @@ public class TestJoinProperties {
   static Path[] src;
   static Path base;
 
-  @BeforeClass
+  @BeforeAll
   public static void setUp() throws Exception {
     Configuration conf = new Configuration();
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
@@ -55,7 +55,7 @@ public class TestJoinProperties {
     src = generateSources(conf);
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() throws Exception {
     if (cluster != null) {
       cluster.shutdown();
@@ -205,7 +205,7 @@ public class TestJoinProperties {
     String joinExpr = constructExpr1(op);
     conf.set(CompositeInputFormat.JOIN_EXPR, joinExpr);
     int count = testFormat(conf, 2, true, false, ttype);
-    assertTrue("not all keys present", count == expectedCount);
+    assertTrue(count == expectedCount, "not all keys present");
   }
 
   private void testExpr2(Configuration conf, String op, TestType ttype,
@@ -213,7 +213,7 @@ public class TestJoinProperties {
     String joinExpr = constructExpr2(op);
     conf.set(CompositeInputFormat.JOIN_EXPR, joinExpr);
     int count = testFormat(conf, 2, false, true, ttype);
-    assertTrue("not all keys present", count == expectedCount);
+    assertTrue(count == expectedCount, "not all keys present");
   }
 
   private void testExpr3(Configuration conf, String op, TestType ttype,
@@ -221,14 +221,14 @@ public class TestJoinProperties {
     String joinExpr = constructExpr3(op);
     conf.set(CompositeInputFormat.JOIN_EXPR, joinExpr);
     int count = testFormat(conf, 3, false, false, ttype);
-    assertTrue("not all keys present", count == expectedCount);
+    assertTrue(count == expectedCount, "not all keys present");
   }
 
   private void testExpr4(Configuration conf) throws Exception {
     String joinExpr = constructExpr4();
     conf.set(CompositeInputFormat.JOIN_EXPR, joinExpr);
     int count = testFormat(conf, 0, false, false, TestType.INNER_IDENTITY);
-    assertTrue("not all keys present", count == ITEMS);
+    assertTrue(count == ITEMS, "not all keys present");
   }
 
   // outer(outer(A, B), C) == outer(A,outer(B, C)) == outer(A, B, C)
@@ -259,7 +259,7 @@ public class TestJoinProperties {
   private void validateOuterKeyValue(IntWritable k, TupleWritable v, 
       int tupleSize, boolean firstTuple, boolean secondTuple) {
 	final String kvstr = "Unexpected tuple: " + stringify(k, v);
-	assertTrue(kvstr, v.size() == tupleSize);
+    assertTrue(v.size() == tupleSize, kvstr);
 	int key = k.get();
 	IntWritable val0 = null;
 	IntWritable val1 = null;
@@ -269,74 +269,74 @@ public class TestJoinProperties {
       if (key % 2 == 0 && key / 2 <= ITEMS) {
         val0 = (IntWritable)v0.get(0);
       } else {
-        assertFalse(kvstr, v0.has(0));
+        assertFalse(v0.has(0), kvstr);
       }
       if (key % 3 == 0 && key / 3 <= ITEMS) {
         val1 = (IntWritable)v0.get(1);
       } else {
-        assertFalse(kvstr, v0.has(1));
+        assertFalse(v0.has(1), kvstr);
       }
       if (key % 4 == 0 && key / 4 <= ITEMS) {
         val2 = (LongWritable)v.get(1);
       } else {
-        assertFalse(kvstr, v.has(2));
+        assertFalse(v.has(2), kvstr);
       }
     } else if (secondTuple) {
       if (key % 2 == 0 && key / 2 <= ITEMS) {
         val0 = (IntWritable)v.get(0);
       } else {
-        assertFalse(kvstr, v.has(0));
+        assertFalse(v.has(0), kvstr);
       }
       TupleWritable v1 = ((TupleWritable)v.get(1));
       if (key % 3 == 0 && key / 3 <= ITEMS) {
         val1 = (IntWritable)v1.get(0);
       } else {
-        assertFalse(kvstr, v1.has(0));
+        assertFalse(v1.has(0), kvstr);
       }
       if (key % 4 == 0 && key / 4 <= ITEMS) {
         val2 = (LongWritable)v1.get(1);
       } else {
-        assertFalse(kvstr, v1.has(1));
+        assertFalse(v1.has(1), kvstr);
       }
     } else {
       if (key % 2 == 0 && key / 2 <= ITEMS) {
         val0 = (IntWritable)v.get(0);
       } else {
-        assertFalse(kvstr, v.has(0));
+        assertFalse(v.has(0), kvstr);
       }
       if (key % 3 == 0 && key / 3 <= ITEMS) {
         val1 = (IntWritable)v.get(1);
       } else {
-        assertFalse(kvstr, v.has(1));
+        assertFalse(v.has(1), kvstr);
       }
       if (key % 4 == 0 && key / 4 <= ITEMS) {
         val2 = (LongWritable)v.get(2);
       } else {
-        assertFalse(kvstr, v.has(2));
+        assertFalse(v.has(2), kvstr);
       }
     }
 	if (val0 != null) {
-      assertTrue(kvstr, val0.get() == 0);
+      assertTrue(val0.get() == 0, kvstr);
     }
 	if (val1 != null) {
-      assertTrue(kvstr, val1.get() == 1);
+      assertTrue(val1.get() == 1, kvstr);
     }
 	if (val2 != null) {
-      assertTrue(kvstr, val2.get() == 2);
+      assertTrue(val2.get() == 2, kvstr);
     }
   }
 
   private void validateInnerKeyValue(IntWritable k, TupleWritable v,
       int tupleSize, boolean firstTuple, boolean secondTuple) {
 	final String kvstr = "Unexpected tuple: " + stringify(k, v);
-	assertTrue(kvstr, v.size() == tupleSize);
+    assertTrue(v.size() == tupleSize, kvstr);
 	int key = k.get();
 	IntWritable val0 = null;
 	IntWritable val1 = null;
 	LongWritable val2 = null;
-	assertTrue(kvstr, key % 2 == 0 && key / 2 <= ITEMS);
-	assertTrue(kvstr, key % 3 == 0 && key / 3 <= ITEMS);
-	assertTrue(kvstr, key % 4 == 0 && key / 4 <= ITEMS);
+    assertTrue(key % 2 == 0 && key / 2 <= ITEMS, kvstr);
+    assertTrue(key % 3 == 0 && key / 3 <= ITEMS, kvstr);
+    assertTrue(key % 4 == 0 && key / 4 <= ITEMS, kvstr);
 	if (firstTuple) {
       TupleWritable v0 = ((TupleWritable)v.get(0));
       val0 = (IntWritable)v0.get(0);
@@ -352,16 +352,16 @@ public class TestJoinProperties {
       val1 = (IntWritable)v.get(1);
       val2 = (LongWritable)v.get(2);
     }
-    assertTrue(kvstr, val0.get() == 0);
-    assertTrue(kvstr, val1.get() == 1);
-    assertTrue(kvstr, val2.get() == 2);
+    assertTrue(val0.get() == 0, kvstr);
+    assertTrue(val1.get() == 1, kvstr);
+    assertTrue(val2.get() == 2, kvstr);
   }
 
   private void validateKeyValue_INNER_IDENTITY(IntWritable k, IntWritable v) {
     final String kvstr = "Unexpected tuple: " + stringify(k, v);
     int key = k.get();
-    assertTrue(kvstr, (key % 2 == 0 && key / 2 <= ITEMS));
-    assertTrue(kvstr, v.get() == 0);
+    assertTrue((key % 2 == 0 && key / 2 <= ITEMS), kvstr);
+    assertTrue(v.get() == 0, kvstr);
   }
   
   @SuppressWarnings("unchecked")

+ 16 - 19
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinTupleWritable.java

@@ -31,12 +31,12 @@ import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class TestJoinTupleWritable {
 
@@ -96,7 +96,7 @@ public class TestJoinTupleWritable {
         i = verifIter(writs, ((TupleWritable)w), i);
         continue;
       }
-      assertTrue("Bad value", w.equals(writs[i++]));
+      assertTrue(w.equals(writs[i++]), "Bad value");
     }
     return i;
   }
@@ -139,7 +139,7 @@ public class TestJoinTupleWritable {
       new IntWritable(r.nextInt())
     };
     TupleWritable sTuple = makeTuple(writs);
-    assertTrue("Bad count", writs.length == verifIter(writs, sTuple, 0));
+    assertTrue(writs.length == verifIter(writs, sTuple, 0), "Bad count");
   }
 
   @Test
@@ -163,7 +163,7 @@ public class TestJoinTupleWritable {
     ByteArrayInputStream in = new ByteArrayInputStream(out.toByteArray());
     TupleWritable dTuple = new TupleWritable();
     dTuple.readFields(new DataInputStream(in));
-    assertTrue("Failed to write/read tuple", sTuple.equals(dTuple));
+    assertTrue(sTuple.equals(dTuple), "Failed to write/read tuple");
   }
 
   @Test
@@ -184,8 +184,8 @@ public class TestJoinTupleWritable {
     dTuple.readFields(new DataInputStream(in));
     assertThat(dTuple).withFailMessage("Failed to write/read tuple")
         .isEqualTo(sTuple);
-    assertEquals("All tuple data has not been read from the stream", 
-      -1, in.read());
+    assertEquals(-1, in.read(),
+        "All tuple data has not been read from the stream");
   }
 
   @Test
@@ -204,8 +204,8 @@ public class TestJoinTupleWritable {
     dTuple.readFields(new DataInputStream(in));
     assertThat(dTuple).withFailMessage("Failed to write/read tuple")
             .isEqualTo(sTuple);
-    assertEquals("All tuple data has not been read from the stream", 
-      -1, in.read());
+    assertEquals(-1, in.read(),
+        "All tuple data has not been read from the stream");
   }
   
   /**
@@ -230,8 +230,8 @@ public class TestJoinTupleWritable {
     dTuple.readFields(new DataInputStream(in));
     assertThat(dTuple).withFailMessage("Failed to write/read tuple")
         .isEqualTo(sTuple);
-    assertEquals("All tuple data has not been read from the stream", 
-      -1, in.read());
+    assertEquals(-1, in.read(),
+        "All tuple data has not been read from the stream");
   }
 
   @Test
@@ -250,8 +250,7 @@ public class TestJoinTupleWritable {
         assertTrue(has);
       }
       else {
-        assertFalse("Tuple position is incorrectly labelled as set: " + pos,
-          has);
+        assertFalse(has, "Tuple position is incorrectly labelled as set: " + pos);
       }
     }
   }
@@ -272,8 +271,7 @@ public class TestJoinTupleWritable {
         assertTrue(has);
       }
       else {
-        assertFalse("Tuple position is incorrectly labelled as set: " + pos,
-          has);
+        assertFalse(has, "Tuple position is incorrectly labelled as set: " + pos);
       }
     }
   }
@@ -297,8 +295,7 @@ public class TestJoinTupleWritable {
         assertTrue(has);
       }
       else {
-        assertFalse("Tuple position is incorrectly labelled as set: " + pos,
-          has);
+        assertFalse(has, "Tuple position is incorrectly labelled as set: " + pos);
       }
     }
   }

+ 5 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestWrappedRRClassloader.java

@@ -31,9 +31,9 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class TestWrappedRRClassloader {
   /**
@@ -87,9 +87,9 @@ public class TestWrappedRRClassloader {
 
     @SuppressWarnings("unchecked")
     public RR_ClassLoaderChecker(Configuration conf) {
-      assertTrue("The class loader has not been inherited from "
-          + CompositeRecordReader.class.getSimpleName(),
-          conf.getClassLoader() instanceof Fake_ClassLoader);
+      assertTrue(conf.getClassLoader() instanceof Fake_ClassLoader,
+          "The class loader has not been inherited from "
+          + CompositeRecordReader.class.getSimpleName());
 
     }
   }

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java

@@ -27,12 +27,12 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.io.IOException;
 
-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;
 
 public class TestMultithreadedMapper extends HadoopTestCase {
 

+ 22 - 21
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java

@@ -33,11 +33,11 @@ import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.OutputCommitter;
 import org.apache.hadoop.mapreduce.OutputFormat;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertFalse;
+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.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertFalse;
 
 /**
  * A JUnit test to test Map-Reduce job committer.
@@ -59,14 +59,14 @@ public class TestJobOutputCommitter extends HadoopTestCase {
   private FileSystem fs;
   private Configuration conf = null;
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     super.setUp();
     conf = createJobConf();
     fs = getFileSystem();
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     fs.delete(new Path(TEST_ROOT_DIR), true);
     super.tearDown();
@@ -151,16 +151,17 @@ public class TestJobOutputCommitter extends HadoopTestCase {
     Job job = MapReduceTestUtil.createJob(conf, inDir, outDir, 1, 0);
     job.setOutputFormatClass(output);
 
-    assertTrue("Job failed!", job.waitForCompletion(true));
+    assertTrue(job.waitForCompletion(true), "Job failed!");
 
     Path testFile = new Path(outDir, filename);
-    assertTrue("Done file missing for job " + job.getJobID(), fs.exists(testFile));
+    assertTrue(fs.exists(testFile), "Done file missing for job " + job.getJobID());
 
     // check if the files from the missing set exists
     for (String ex : exclude) {
       Path file = new Path(outDir, ex);
-      assertFalse("File " + file + " should not be present for successful job "
-          + job.getJobID(), fs.exists(file));
+      assertFalse(fs.exists(file),
+          "File " + file + " should not be present for successful job "
+          + job.getJobID());
     }
   }
 
@@ -171,19 +172,19 @@ public class TestJobOutputCommitter extends HadoopTestCase {
     Job job = MapReduceTestUtil.createFailJob(conf, outDir, inDir);
     job.setOutputFormatClass(output);
 
-    assertFalse("Job did not fail!", job.waitForCompletion(true));
+    assertFalse(job.waitForCompletion(true), "Job did not fail!");
 
     if (fileName != null) {
       Path testFile = new Path(outDir, fileName);
-      assertTrue("File " + testFile + " missing for failed job " + job.getJobID(),
-          fs.exists(testFile));
+      assertTrue(fs.exists(testFile),
+          "File " + testFile + " missing for failed job " + job.getJobID());
     }
 
     // check if the files from the missing set exists
     for (String ex : exclude) {
       Path file = new Path(outDir, ex);
-      assertFalse("File " + file + " should not be present for failed job "
-          + job.getJobID(), fs.exists(file));
+      assertFalse(fs.exists(file), "File " + file + " should not be present for failed job "
+          + job.getJobID());
     }
   }
 
@@ -203,19 +204,19 @@ public class TestJobOutputCommitter extends HadoopTestCase {
 
     job.killJob(); // kill the job
 
-    assertFalse("Job did not get kill", job.waitForCompletion(true));
+    assertFalse(job.waitForCompletion(true), "Job did not get kill");
 
     if (fileName != null) {
       Path testFile = new Path(outDir, fileName);
-      assertTrue("File " + testFile + " missing for job " + job.getJobID(), fs
-          .exists(testFile));
+      assertTrue(fs.exists(testFile),
+          "File " + testFile + " missing for job " + job.getJobID());
     }
 
     // check if the files from the missing set exists
     for (String ex : exclude) {
       Path file = new Path(outDir, ex);
-      assertFalse("File " + file + " should not be present for killed job "
-          + job.getJobID(), fs.exists(file));
+      assertFalse(fs.exists(file), "File " + file + " should not be present for killed job "
+          + job.getJobID());
     }
   }
 

+ 12 - 12
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRCJCFileOutputCommitter.java

@@ -21,14 +21,14 @@ package org.apache.hadoop.mapreduce.lib.output;
 import java.io.*;
 import java.net.URI;
 
-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 static org.assertj.core.api.Assertions.assertThat;
-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.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.*;
@@ -83,12 +83,12 @@ public class TestMRCJCFileOutputCommitter {
     fs.delete(outDir, true);
   }
   
-  @Before
+  @BeforeEach
   public void setUp() throws IOException {
     cleanup();
   }
   
-  @After
+  @AfterEach
   public void tearDown() throws IOException {
     cleanup();
   }
@@ -178,12 +178,12 @@ public class TestMRCJCFileOutputCommitter {
     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());
+    assertFalse(expectedFile.exists(), "job temp dir still exists");
     assertThat(new File(outDir.toString())
         .listFiles()).withFailMessage("Output directory not empty").isEmpty();
     FileUtil.fullyDelete(new File(outDir.toString()));
@@ -241,7 +241,7 @@ public class TestMRCJCFileOutputCommitter {
     File jobTmpDir = new File(committer.getJobAttemptPath(jContext).toUri().getPath());
     File taskTmpDir = new File(committer.getTaskAttemptPath(tContext).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 {
@@ -252,7 +252,7 @@ public class TestMRCJCFileOutputCommitter {
     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()));
   }
 }

+ 21 - 18
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java

@@ -34,9 +34,9 @@ import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.Reducer;
 
-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 java.io.BufferedReader;
 import java.io.IOException;
@@ -44,9 +44,10 @@ import java.io.InputStreamReader;
 import java.util.Arrays;
 import java.util.Map;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -70,17 +71,19 @@ public class TestMRMultipleOutputs extends HadoopTestCase {
   }
 
   @SuppressWarnings("unchecked")
-  @Test(expected = IOException.class)
+  @Test
   public void testParallelCloseIOException() throws IOException, InterruptedException {
-    RecordWriter writer = mock(RecordWriter.class);
-    Map recordWriters = mock(Map.class);
-    when(recordWriters.values()).thenReturn(Arrays.asList(writer, writer));
-    Mapper.Context taskInputOutputContext = mock(Mapper.Context.class);
-    when(taskInputOutputContext.getConfiguration()).thenReturn(createJobConf());
-    doThrow(new IOException("test IO exception")).when(writer).close(taskInputOutputContext);
-    MultipleOutputs<Long, String> mos = new MultipleOutputs<Long, String>(taskInputOutputContext);
-    mos.setRecordWriters(recordWriters);
-    mos.close();
+    assertThrows(IOException.class, () -> {
+      RecordWriter writer = mock(RecordWriter.class);
+      Map recordWriters = mock(Map.class);
+      when(recordWriters.values()).thenReturn(Arrays.asList(writer, writer));
+      Mapper.Context taskInputOutputContext = mock(Mapper.Context.class);
+      when(taskInputOutputContext.getConfiguration()).thenReturn(createJobConf());
+      doThrow(new IOException("test IO exception")).when(writer).close(taskInputOutputContext);
+      MultipleOutputs<Long, String> mos = new MultipleOutputs<Long, String>(taskInputOutputContext);
+      mos.setRecordWriters(recordWriters);
+      mos.close();
+    });
   }
 
   private static String localPathRoot = 
@@ -91,7 +94,7 @@ public class TestMRMultipleOutputs extends HadoopTestCase {
   private static String TEXT = "text";
   private static String SEQUENCE = "sequence";
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     super.setUp();
     Configuration conf = createJobConf();
@@ -99,7 +102,7 @@ public class TestMRMultipleOutputs extends HadoopTestCase {
     fs.delete(ROOT_DIR, true);
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     Configuration conf = createJobConf();
     FileSystem fs = FileSystem.get(conf);

+ 19 - 22
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRSequenceFileAsBinaryOutputFormat.java

@@ -41,7 +41,7 @@ import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 import org.apache.hadoop.mapreduce.task.MapContextImpl;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -49,8 +49,8 @@ import java.io.IOException;
 import java.util.Random;
 
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
 
 public class TestMRSequenceFileAsBinaryOutputFormat {
   private static final Logger LOG =
@@ -138,20 +138,18 @@ public class TestMRSequenceFileAsBinaryOutputFormat {
           sourceDouble = r.nextDouble();
           iwritable = reader.getCurrentKey();
           dwritable = reader.getCurrentValue();
-          assertEquals(
-              "Keys don't match: " + "*" + iwritable.get() + ":" + 
-                                           sourceInt + "*",
-              sourceInt, iwritable.get());
+          assertEquals(sourceInt, iwritable.get(),
+              "Keys don't match: " + "*" + iwritable.get() + ":" + sourceInt + "*");
           assertThat(dwritable.get()).withFailMessage(
               "Vals don't match: " + "*" + dwritable.get() + ":" +
-                  sourceDouble + "*").isEqualTo(sourceDouble);
+              sourceDouble + "*").isEqualTo(sourceDouble);
           ++count;
         }
       } finally {
         reader.close();
       }
     }
-    assertEquals("Some records not found", RECORDS, count);
+    assertEquals(RECORDS, count, "Some records not found");
   }
 
   @Test
@@ -162,25 +160,24 @@ public class TestMRSequenceFileAsBinaryOutputFormat {
     job.setOutputKeyClass(FloatWritable.class);
     job.setOutputValueClass(BooleanWritable.class);
 
-    assertEquals("SequenceFileOutputKeyClass should default to ouputKeyClass", 
-      FloatWritable.class,
-      SequenceFileAsBinaryOutputFormat.getSequenceFileOutputKeyClass(job));
-    assertEquals("SequenceFileOutputValueClass should default to " 
-      + "ouputValueClass", 
-      BooleanWritable.class,
-      SequenceFileAsBinaryOutputFormat.getSequenceFileOutputValueClass(job));
+    assertEquals(FloatWritable.class,
+        SequenceFileAsBinaryOutputFormat.getSequenceFileOutputKeyClass(job),
+        "SequenceFileOutputKeyClass should default to ouputKeyClass");
+    assertEquals(BooleanWritable.class,
+        SequenceFileAsBinaryOutputFormat.getSequenceFileOutputValueClass(job),
+        "SequenceFileOutputValueClass should default to ouputValueClass");
 
     SequenceFileAsBinaryOutputFormat.setSequenceFileOutputKeyClass(job, 
       IntWritable.class );
     SequenceFileAsBinaryOutputFormat.setSequenceFileOutputValueClass(job, 
       DoubleWritable.class ); 
 
-    assertEquals("SequenceFileOutputKeyClass not updated", 
-      IntWritable.class,
-      SequenceFileAsBinaryOutputFormat.getSequenceFileOutputKeyClass(job));
-    assertEquals("SequenceFileOutputValueClass not updated", 
-      DoubleWritable.class,
-      SequenceFileAsBinaryOutputFormat.getSequenceFileOutputValueClass(job));
+    assertEquals(IntWritable.class,
+        SequenceFileAsBinaryOutputFormat.getSequenceFileOutputKeyClass(job),
+        "SequenceFileOutputKeyClass not updated");
+    assertEquals(DoubleWritable.class,
+        SequenceFileAsBinaryOutputFormat.getSequenceFileOutputValueClass(job),
+        "SequenceFileOutputValueClass not updated");
   }
 
   @Test

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

@@ -22,10 +22,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.BinaryComparable;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
-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;
 
 public class TestBinaryPartitioner {
 

+ 7 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestInputSampler.java

@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.mapreduce.lib.partition;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.io.DataInput;
 import java.io.DataOutput;
@@ -36,7 +36,8 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 public class TestInputSampler {
 
@@ -208,7 +209,8 @@ public class TestInputSampler {
    * Verify SplitSampler contract in mapred.lib.InputSampler, which is added
    * back for binary compatibility of M/R 1.x
    */
-  @Test (timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   @SuppressWarnings("unchecked") // IntWritable comparator not typesafe
   public void testMapredSplitSampler() throws Exception {
     final int TOT_SPLITS = 15;
@@ -266,7 +268,8 @@ public class TestInputSampler {
    * Verify IntervalSampler in mapred.lib.InputSampler, which is added back
    * for binary compatibility of M/R 1.x
    */
-  @Test (timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   @SuppressWarnings("unchecked") // IntWritable comparator not typesafe
   public void testMapredIntervalSampler() throws Exception {
     final int TOT_SPLITS = 16;

+ 36 - 36
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestKeyFieldHelper.java

@@ -17,12 +17,12 @@
  */
 package org.apache.hadoop.mapreduce.lib.partition;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-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;
 
 public class TestKeyFieldHelper {
   private static final Logger LOG =
@@ -39,7 +39,7 @@ public class TestKeyFieldHelper {
     String eKeySpecs = keySpecs;
     helper.parseOption(keySpecs);
     String actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     
     // test -k a.b
     keySpecs = "-k 1.2";
@@ -47,28 +47,28 @@ public class TestKeyFieldHelper {
     helper = new KeyFieldHelper();
     helper.parseOption(keySpecs);
     actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     
     keySpecs = "-nr -k1.2,3.4";
     eKeySpecs = "-k1.2,3.4nr";
     helper = new KeyFieldHelper();
     helper.parseOption(keySpecs);
     actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     
     keySpecs = "-nr -k1.2,3.4n";
     eKeySpecs = "-k1.2,3.4n";
     helper = new KeyFieldHelper();
     helper.parseOption(keySpecs);
     actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     
     keySpecs = "-nr -k1.2,3.4r";
     eKeySpecs = "-k1.2,3.4r";
     helper = new KeyFieldHelper();
     helper.parseOption(keySpecs);
     actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     
     keySpecs = "-nr -k1.2,3.4 -k5.6,7.8n -k9.10,11.12r -k13.14,15.16nr";
     //1st
@@ -76,142 +76,142 @@ public class TestKeyFieldHelper {
     helper = new KeyFieldHelper();
     helper.parseOption(keySpecs);
     actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     // 2nd
     eKeySpecs = "-k5.6,7.8n";
     actKeySpecs = helper.keySpecs().get(1).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     //3rd
     eKeySpecs = "-k9.10,11.12r";
     actKeySpecs = helper.keySpecs().get(2).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     //4th
     eKeySpecs = "-k13.14,15.16nr";
     actKeySpecs = helper.keySpecs().get(3).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     
     keySpecs = "-k1.2n,3.4";
     eKeySpecs = "-k1.2,3.4n";
     helper = new KeyFieldHelper();
     helper.parseOption(keySpecs);
     actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     
     keySpecs = "-k1.2r,3.4";
     eKeySpecs = "-k1.2,3.4r";
     helper = new KeyFieldHelper();
     helper.parseOption(keySpecs);
     actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     
     keySpecs = "-k1.2nr,3.4";
     eKeySpecs = "-k1.2,3.4nr";
     helper = new KeyFieldHelper();
     helper.parseOption(keySpecs);
     actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     
     keySpecs = "-k1.2,3.4n";
     eKeySpecs = "-k1.2,3.4n";
     helper = new KeyFieldHelper();
     helper.parseOption(keySpecs);
     actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     
     keySpecs = "-k1.2,3.4r";
     eKeySpecs = "-k1.2,3.4r";
     helper = new KeyFieldHelper();
     helper.parseOption(keySpecs);
     actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     
     keySpecs = "-k1.2,3.4nr";
     eKeySpecs = "-k1.2,3.4nr";
     helper = new KeyFieldHelper();
     helper.parseOption(keySpecs);
     actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     
     keySpecs = "-nr -k1.2,3.4 -k5.6,7.8";
     eKeySpecs = "-k1.2,3.4nr";
     helper = new KeyFieldHelper();
     helper.parseOption(keySpecs);
     actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     eKeySpecs = "-k5.6,7.8nr";
     actKeySpecs = helper.keySpecs().get(1).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     
     keySpecs = "-n -k1.2,3.4 -k5.6,7.8";
     eKeySpecs = "-k1.2,3.4n";
     helper = new KeyFieldHelper();
     helper.parseOption(keySpecs);
     actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     eKeySpecs = "-k5.6,7.8n";
     actKeySpecs = helper.keySpecs().get(1).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     
     keySpecs = "-r -k1.2,3.4 -k5.6,7.8";
     eKeySpecs = "-k1.2,3.4r";
     helper = new KeyFieldHelper();
     helper.parseOption(keySpecs);
     actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     eKeySpecs = "-k5.6,7.8r";
     actKeySpecs = helper.keySpecs().get(1).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     
     keySpecs = "-k1.2,3.4n -k5.6,7.8";
     eKeySpecs = "-k1.2,3.4n";
     helper = new KeyFieldHelper();
     helper.parseOption(keySpecs);
     actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     eKeySpecs = "-k5.6,7.8";
     actKeySpecs = helper.keySpecs().get(1).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     
     keySpecs = "-k1.2,3.4r -k5.6,7.8";
     eKeySpecs = "-k1.2,3.4r";
     helper = new KeyFieldHelper();
     helper.parseOption(keySpecs);
     actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     eKeySpecs = "-k5.6,7.8";
     actKeySpecs = helper.keySpecs().get(1).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     
     keySpecs = "-k1.2,3.4nr -k5.6,7.8";
     eKeySpecs = "-k1.2,3.4nr";
     helper = new KeyFieldHelper();
     helper.parseOption(keySpecs);
     actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     eKeySpecs = "-k5.6,7.8";
     actKeySpecs = helper.keySpecs().get(1).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     
     keySpecs = "-n";
     eKeySpecs = "-k1.1,0.0n";
     helper = new KeyFieldHelper();
     helper.parseOption(keySpecs);
     actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     
     keySpecs = "-r";
     eKeySpecs = "-k1.1,0.0r";
     helper = new KeyFieldHelper();
     helper.parseOption(keySpecs);
     actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
     
     keySpecs = "-nr";
     eKeySpecs = "-k1.1,0.0nr";
     helper = new KeyFieldHelper();
     helper.parseOption(keySpecs);
     actKeySpecs = helper.keySpecs().get(0).toString();
-    assertEquals("KeyFieldHelper's parsing is garbled", eKeySpecs, actKeySpecs);
+    assertEquals(eKeySpecs, actKeySpecs, "KeyFieldHelper's parsing is garbled");
   }
   
   /**
@@ -395,7 +395,7 @@ public class TestKeyFieldHelper {
                                       helper.keySpecs().get(0));
     LOG.info("start : " + start);
     if (expectedOutput == null) {
-      assertEquals("Expected -1 when the start index is invalid", -1, start);
+      assertEquals(-1, start, "Expected -1 when the start index is invalid");
       return;
     }
     // get the end index
@@ -411,8 +411,8 @@ public class TestKeyFieldHelper {
     String output = new String(outputBytes);
     LOG.info("output : " + output);
     LOG.info("expected-output : " + expectedOutput);
-    assertEquals(keySpecs + " failed on input '" + input + "'", 
-                 expectedOutput, output);
+    assertEquals(expectedOutput, output,
+        keySpecs + " failed on input '" + input + "'");
   }
 
   // check for equality of 2 int arrays

+ 19 - 18
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedPartitioner.java

@@ -19,9 +19,9 @@ package org.apache.hadoop.mapreduce.lib.partition;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 public class TestMRKeyFieldBasedPartitioner {
 
@@ -36,8 +36,9 @@ public class TestMRKeyFieldBasedPartitioner {
     Configuration conf = new Configuration();
     conf.setInt("num.key.fields.for.partition", 10);
     kfbp.setConf(conf);
-    assertEquals("Empty key should map to 0th partition", 
-                 0, kfbp.getPartition(new Text(), new Text(), numReducers));
+    assertEquals(0,
+        kfbp.getPartition(new Text(), new Text(), numReducers),
+        "Empty key should map to 0th partition");
     
     // check if the hashcode is correct when no keyspec is specified
     kfbp = new KeyFieldBasedPartitioner<Text, Text>();
@@ -46,8 +47,8 @@ public class TestMRKeyFieldBasedPartitioner {
     String input = "abc\tdef\txyz";
     int hashCode = input.hashCode();
     int expectedPartition = kfbp.getPartition(hashCode, numReducers);
-    assertEquals("Partitioner doesnt work as expected", expectedPartition, 
-                 kfbp.getPartition(new Text(input), new Text(), numReducers));
+    assertEquals(expectedPartition, kfbp.getPartition(new Text(input), new Text(), numReducers),
+        "Partitioner doesnt work as expected");
     
     // check if the hashcode is correct with specified keyspec
     kfbp = new KeyFieldBasedPartitioner<Text, Text>();
@@ -58,8 +59,8 @@ public class TestMRKeyFieldBasedPartitioner {
     byte[] eBytes = expectedOutput.getBytes();
     hashCode = kfbp.hashCode(eBytes, 0, eBytes.length - 1, 0);
     expectedPartition = kfbp.getPartition(hashCode, numReducers);
-    assertEquals("Partitioner doesnt work as expected", expectedPartition, 
-                 kfbp.getPartition(new Text(input), new Text(), numReducers));
+    assertEquals(expectedPartition, kfbp.getPartition(new Text(input), new Text(), numReducers),
+        "Partitioner doesnt work as expected");
     
     // test with invalid end index in keyspecs
     kfbp = new KeyFieldBasedPartitioner<Text, Text>();
@@ -70,8 +71,8 @@ public class TestMRKeyFieldBasedPartitioner {
     eBytes = expectedOutput.getBytes();
     hashCode = kfbp.hashCode(eBytes, 0, eBytes.length - 1, 0);
     expectedPartition = kfbp.getPartition(hashCode, numReducers);
-    assertEquals("Partitioner doesnt work as expected", expectedPartition, 
-                 kfbp.getPartition(new Text(input), new Text(), numReducers));
+    assertEquals(expectedPartition, kfbp.getPartition(new Text(input), new Text(), numReducers),
+        "Partitioner doesnt work as expected");
     
     // test with 0 end index in keyspecs
     kfbp = new KeyFieldBasedPartitioner<Text, Text>();
@@ -82,16 +83,16 @@ public class TestMRKeyFieldBasedPartitioner {
     eBytes = expectedOutput.getBytes();
     hashCode = kfbp.hashCode(eBytes, 0, eBytes.length - 1, 0);
     expectedPartition = kfbp.getPartition(hashCode, numReducers);
-    assertEquals("Partitioner doesnt work as expected", expectedPartition, 
-                 kfbp.getPartition(new Text(input), new Text(), numReducers));
+    assertEquals(expectedPartition, kfbp.getPartition(new Text(input), new Text(), numReducers),
+        "Partitioner doesnt work as expected");
     
     // test with invalid keyspecs
     kfbp = new KeyFieldBasedPartitioner<Text, Text>();
     conf = new Configuration();
     conf.set(KeyFieldBasedPartitioner.PARTITIONER_OPTIONS, "-k10");
     kfbp.setConf(conf);
-    assertEquals("Partitioner doesnt work as expected", 0, 
-                 kfbp.getPartition(new Text(input), new Text(), numReducers));
+    assertEquals(0, kfbp.getPartition(new Text(input), new Text(), numReducers),
+        "Partitioner doesnt work as expected");
     
     // test with multiple keyspecs
     kfbp = new KeyFieldBasedPartitioner<Text, Text>();
@@ -106,8 +107,8 @@ public class TestMRKeyFieldBasedPartitioner {
     eBytes = expectedOutput.getBytes();
     hashCode = kfbp.hashCode(eBytes, 0, eBytes.length - 1, hashCode);
     expectedPartition = kfbp.getPartition(hashCode, numReducers);
-    assertEquals("Partitioner doesnt work as expected", expectedPartition, 
-                 kfbp.getPartition(new Text(input), new Text(), numReducers));
+    assertEquals(expectedPartition, kfbp.getPartition(new Text(input), new Text(), numReducers),
+        "Partitioner doesnt work as expected");
     
     // test with invalid start index in keyspecs
     kfbp = new KeyFieldBasedPartitioner<Text, Text>();
@@ -121,7 +122,7 @@ public class TestMRKeyFieldBasedPartitioner {
     eBytes = expectedOutput.getBytes();
     hashCode = kfbp.hashCode(eBytes, 0, eBytes.length - 1, hashCode);
     expectedPartition = kfbp.getPartition(hashCode, numReducers);
-    assertEquals("Partitioner doesnt work as expected", expectedPartition, 
-                 kfbp.getPartition(new Text(input), new Text(), numReducers));
+    assertEquals(expectedPartition, kfbp.getPartition(new Text(input), new Text(), numReducers),
+        "Partitioner doesnt work as expected");
   }
 }

+ 11 - 10
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestTotalOrderPartitioner.java

@@ -38,9 +38,9 @@ import org.apache.hadoop.io.serializer.JavaSerialization;
 import org.apache.hadoop.io.serializer.JavaSerializationComparator;
 import org.apache.hadoop.io.serializer.WritableSerialization;
 import org.apache.hadoop.mapreduce.MRJobConfig;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 public class TestTotalOrderPartitioner {
 
@@ -158,8 +158,9 @@ public class TestTotalOrderPartitioner {
       partitioner.setConf(conf);
       NullWritable nw = NullWritable.get();
       for (Check<String> chk : testJavaStrings) {
-        assertEquals(chk.data.toString(), chk.part,
-            partitioner.getPartition(chk.data, nw, splitJavaStrings.length + 1));
+        assertEquals(chk.part,
+            partitioner.getPartition(chk.data, nw, splitJavaStrings.length + 1),
+            chk.data.toString());
       }
     } finally {
       p.getFileSystem(conf).delete(p, true);
@@ -178,8 +179,8 @@ public class TestTotalOrderPartitioner {
       partitioner.setConf(conf);
       NullWritable nw = NullWritable.get();
       for (Check<Text> chk : testStrings) {
-        assertEquals(chk.data.toString(), chk.part,
-            partitioner.getPartition(chk.data, nw, splitStrings.length + 1));
+        assertEquals(chk.part,
+            partitioner.getPartition(chk.data, nw, splitStrings.length + 1), chk.data.toString());
       }
     } finally {
       p.getFileSystem(conf).delete(p, true);
@@ -199,8 +200,8 @@ public class TestTotalOrderPartitioner {
       partitioner.setConf(conf);
       NullWritable nw = NullWritable.get();
       for (Check<Text> chk : testStrings) {
-        assertEquals(chk.data.toString(), chk.part,
-            partitioner.getPartition(chk.data, nw, splitStrings.length + 1));
+        assertEquals(chk.part,
+            partitioner.getPartition(chk.data, nw, splitStrings.length + 1), chk.data.toString());
       }
     } finally {
       p.getFileSystem(conf).delete(p, true);
@@ -248,8 +249,8 @@ public class TestTotalOrderPartitioner {
       partitioner.setConf(conf);
       NullWritable nw = NullWritable.get();
       for (Check<Text> chk : revCheck) {
-        assertEquals(chk.data.toString(), chk.part,
-            partitioner.getPartition(chk.data, nw, splitStrings.length + 1));
+        assertEquals(chk.part,
+            partitioner.getPartition(chk.data, nw, splitStrings.length + 1), chk.data.toString());
       }
     } finally {
       p.getFileSystem(conf).delete(p, true);

+ 10 - 11
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestBinaryTokenFile.java

@@ -18,8 +18,8 @@
 package org.apache.hadoop.mapreduce.security;
 
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
@@ -47,10 +47,9 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-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;
 
 public class TestBinaryTokenFile {
 
@@ -175,7 +174,7 @@ public class TestBinaryTokenFile {
 
   private static Path p1;
 
-  @BeforeClass
+  @BeforeAll
   public static void setUp() throws Exception {
     final Configuration conf = new Configuration();
 
@@ -201,7 +200,7 @@ public class TestBinaryTokenFile {
     p1 = fs.makeQualified(p1);
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() throws Exception {
     if(mrCluster != null) {
       mrCluster.stop();
@@ -231,7 +230,7 @@ public class TestBinaryTokenFile {
         os.close();
       }
     } catch (IOException e) {
-      Assert.fail("Exception " + e);
+      fail("Exception " + e);
     }
   }
 
@@ -259,7 +258,7 @@ public class TestBinaryTokenFile {
       e.printStackTrace(System.out);
       fail("Job failed");
     }
-    assertEquals("dist job res is not 0:", 0, res);
+    assertEquals(0, res, "dist job res is not 0:");
   }
 
   /**
@@ -288,6 +287,6 @@ public class TestBinaryTokenFile {
       e.printStackTrace(System.out);
       fail("Job failed");
     }
-    assertEquals("dist job res is not 0:", 0, res);
+    assertEquals(0, res, "dist job res is not 0:");
   }
 }

+ 7 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestJHSSecurity.java

@@ -18,7 +18,8 @@
 
 package org.apache.hadoop.mapreduce.security;
 
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
@@ -27,7 +28,6 @@ import java.security.PrivilegedExceptionAction;
 
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.test.LambdaTestUtils;
-import org.junit.Assert;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@@ -52,7 +52,7 @@ import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -104,8 +104,8 @@ public class TestJHSSecurity {
       // Fake the authentication-method
       UserGroupInformation loggedInUser = UserGroupInformation
           .createRemoteUser("testrenewer@APACHE.ORG");
-      Assert.assertEquals("testrenewer", loggedInUser.getShortUserName());
-   // Default realm is APACHE.ORG
+      assertEquals("testrenewer", loggedInUser.getShortUserName());
+      // Default realm is APACHE.ORG
       loggedInUser.setAuthenticationMethod(AuthenticationMethod.KERBEROS);
 
 
@@ -124,7 +124,7 @@ public class TestJHSSecurity {
       try {
         clientUsingDT.getJobReport(jobReportRequest);
       } catch (IOException e) {
-        Assert.assertEquals("Unknown job job_123456_0001", e.getMessage());
+        assertEquals("Unknown job job_123456_0001", e.getMessage());
       }
       
    // Renew after 50% of token age.
@@ -147,7 +147,7 @@ public class TestJHSSecurity {
       try {
         clientUsingDT.getJobReport(jobReportRequest);
       } catch (IOException e) {
-        Assert.assertEquals("Unknown job job_123456_0001", e.getMessage());
+        assertEquals("Unknown job job_123456_0001", e.getMessage());
       }
       
       // Wait for expiry.

+ 6 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestMRCredentials.java

@@ -18,7 +18,7 @@
 package org.apache.hadoop.mapreduce.security;
 
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.File;
 import java.io.FileNotFoundException;
@@ -36,9 +36,9 @@ import org.apache.hadoop.mapred.MiniMRClientCluster;
 import org.apache.hadoop.mapred.MiniMRClientClusterFactory;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.util.ToolRunner;
-import org.junit.AfterClass;
-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;
 
 /**
  * Tests whether a protected secret passed from JobClient is
@@ -54,7 +54,7 @@ public class TestMRCredentials {
   private static JobConf jConf;
 
   @SuppressWarnings("deprecation")
-  @BeforeClass
+  @BeforeAll
   public static void setUp() throws Exception {
     System.setProperty("hadoop.log.dir", "logs");
     Configuration conf = new Configuration();
@@ -66,7 +66,7 @@ public class TestMRCredentials {
     createKeysAsJson("keys.json");
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() throws Exception {
     if(mrCluster != null)
       mrCluster.stop();

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestUmbilicalProtocolWithJobToken.java

@@ -48,7 +48,7 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.slf4j.Logger;
 import org.slf4j.event.Level;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import static org.slf4j.LoggerFactory.getLogger;
 
 /** Unit tests for using Job Token over RPC. 

+ 11 - 10
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/ssl/TestEncryptedShuffle.java

@@ -34,11 +34,10 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.Assert;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 
 import java.io.File;
 import java.io.FileWriter;
@@ -46,17 +45,19 @@ import java.io.IOException;
 import java.io.OutputStreamWriter;
 import java.io.Writer;
 
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
 public class TestEncryptedShuffle {
 
   private static File testRootDir;
 
-  @BeforeClass
+  @BeforeAll
   public static void setUp() throws Exception {
     testRootDir =
         GenericTestUtils.setupTestRootDir(TestEncryptedShuffle.class);
   }
 
-  @Before
+  @BeforeEach
   public void createCustomYarnClasspath() throws Exception {
     classpathDir = KeyStoreTestUtil.getClasspathDir(TestEncryptedShuffle.class);
     new File(classpathDir, "core-site.xml").delete();
@@ -64,7 +65,7 @@ public class TestEncryptedShuffle {
         Time.monotonicNow()));
   }
 
-  @After
+  @AfterEach
   public void cleanUpMiniClusterSpecialConfig() throws Exception {
     new File(classpathDir, "core-site.xml").delete();
     String keystoresDir = testRootDir.getAbsolutePath();
@@ -151,8 +152,8 @@ public class TestEncryptedShuffle {
       JobClient jobClient = new JobClient(jobConf);
       RunningJob runJob = jobClient.submitJob(jobConf);
       runJob.waitForCompletion();
-      Assert.assertTrue(runJob.isComplete());
-      Assert.assertTrue(runJob.isSuccessful());
+      assertTrue(runJob.isComplete());
+      assertTrue(runJob.isSuccessful());
     } finally {
       stopCluster();
     }

+ 14 - 12
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/token/delegation/TestDelegationToken.java

@@ -26,19 +26,21 @@ import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
 
-import static org.junit.Assert.*;
-@Ignore
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+
+@Disabled
 public class TestDelegationToken {
   private MiniMRCluster cluster;
   private UserGroupInformation user1;
   private UserGroupInformation user2;
   
-  @Before
+  @BeforeEach
   public void setup() throws Exception {
     user1 = UserGroupInformation.createUserForTesting("alice", 
                                                       new String[]{"users"});
@@ -83,8 +85,8 @@ public class TestDelegationToken {
     System.out.println("create time: " + createTime);
     System.out.println("current time: " + currentTime);
     System.out.println("max time: " + maxTime);
-    assertTrue("createTime < current", createTime < currentTime);
-    assertTrue("current < maxTime", currentTime < maxTime);
+    assertTrue(createTime < currentTime, "createTime < current");
+    assertTrue(currentTime < maxTime, "current < maxTime");
 
     // renew should work as user alice
     user1.doAs(new PrivilegedExceptionAction<Void>() {
@@ -102,7 +104,7 @@ public class TestDelegationToken {
       public Void run() throws Exception {
         try {
           bobClient.renewDelegationToken(token);
-          Assert.fail("bob renew");
+          fail("bob renew");
         } catch (AccessControlException ace) {
           // PASS
         }
@@ -116,7 +118,7 @@ public class TestDelegationToken {
       public Void run() throws Exception {
         try {
           bobClient.cancelDelegationToken(token);
-          Assert.fail("bob cancel");
+          fail("bob cancel");
         } catch (AccessControlException ace) {
           // PASS
         }
@@ -131,7 +133,7 @@ public class TestDelegationToken {
         client.cancelDelegationToken(token);
         try {
           client.cancelDelegationToken(token);
-          Assert.fail("second alice cancel");
+          fail("second alice cancel");
         } catch (InvalidToken it) {
           // PASS
         }

+ 14 - 14
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/util/TestMRAsyncDiskService.java

@@ -24,17 +24,17 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileUtil;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.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.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 /**
  * A test for MRAsyncDiskService.
@@ -47,7 +47,7 @@ public class TestMRAsyncDiskService {
   private static String TEST_ROOT_DIR = new Path(System.getProperty(
       "test.build.data", "/tmp")).toString();
   
-  @Before
+  @BeforeEach
   public void setUp() {
     FileUtil.fullyDelete(new File(TEST_ROOT_DIR));
   }
@@ -213,8 +213,8 @@ public class TestMRAsyncDiskService {
     } catch (IOException e) {
       ee = e;
     }
-    assertNotNull("asyncDiskService should not be able to delete files "
-        + "outside all volumes", ee);
+    assertNotNull(ee, "asyncDiskService should not be able to delete files "
+        + "outside all volumes");
     // asyncDiskService is able to automatically find the file in one
     // of the volumes.
     assertTrue(service.moveAndDeleteAbsolutePath(vols[1] + Path.SEPARATOR_CHAR + d));
@@ -325,14 +325,14 @@ public class TestMRAsyncDiskService {
     for (int i = 0; i < vols.length; i++) {
       File subDir = new File(vols[0]);
       String[] subDirContent = subDir.list();
-      assertEquals("Volume should contain a single child: "
-          + MRAsyncDiskService.TOBEDELETED, 1, subDirContent.length);
+      assertEquals(1, subDirContent.length, "Volume should contain a single child: "
+          + MRAsyncDiskService.TOBEDELETED);
       
       File toBeDeletedDir = new File(vols[0], MRAsyncDiskService.TOBEDELETED);
       String[] content = toBeDeletedDir.list();
-      assertNotNull("Cannot find " + toBeDeletedDir, content);
+      assertNotNull(content, "Cannot find " + toBeDeletedDir);
       assertThat(content).withFailMessage(
-          toBeDeletedDir.toString() + " should be empty now.").isEmpty();
+          toBeDeletedDir + " should be empty now.").isEmpty();
     }
   }
   

+ 11 - 10
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRAMWithNonNormalizedCapabilities.java

@@ -21,8 +21,6 @@ package org.apache.hadoop.mapreduce.v2;
 import java.io.File;
 import java.io.IOException;
 
-import org.junit.Assert;
-
 import org.apache.hadoop.mapreduce.SleepJob;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -32,12 +30,15 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobStatus;
 import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
 public class TestMRAMWithNonNormalizedCapabilities {
   private static final Logger LOG =
       LoggerFactory.getLogger(TestMRAMWithNonNormalizedCapabilities.class);
@@ -59,7 +60,7 @@ public class TestMRAMWithNonNormalizedCapabilities {
           .makeQualified(localFs.getUri(), localFs.getWorkingDirectory());
   static Path APP_JAR = new Path(TEST_ROOT_DIR, "MRAppJar.jar");
 
-  @Before
+  @BeforeEach
   public void setup() throws Exception {
     if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {
       LOG.info("MRAppJar " + MiniMRYarnCluster.APPJAR
@@ -102,12 +103,12 @@ public class TestMRAMWithNonNormalizedCapabilities {
     job.addFileToClassPath(APP_JAR); // The AppMaster jar itself.
     job.submit();
     boolean completed = job.waitForCompletion(true);
-    Assert.assertTrue("Job should be completed", completed);
-    Assert.assertEquals("Job should be finished successfully", 
-                    JobStatus.State.SUCCEEDED, job.getJobState());
+    assertTrue(completed, "Job should be completed");
+    assertEquals(JobStatus.State.SUCCEEDED, job.getJobState(),
+        "Job should be finished successfully");
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
     if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {
       LOG.info("MRAppJar " + MiniMRYarnCluster.APPJAR

+ 8 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRAppWithCombiner.java

@@ -41,13 +41,14 @@ import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.hadoop.mapred.lib.IdentityMapper;
 import org.apache.hadoop.mapred.lib.IdentityReducer;
 import org.apache.hadoop.mapreduce.Job;
-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.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.junit.jupiter.api.Assertions.fail;
+
 @SuppressWarnings("deprecation")
 public class TestMRAppWithCombiner {
 
@@ -65,7 +66,7 @@ public class TestMRAppWithCombiner {
     }
   }
 
-  @BeforeClass
+  @BeforeAll
   public static void setup() throws IOException {
 
     if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {
@@ -88,7 +89,7 @@ public class TestMRAppWithCombiner {
     localFs.setPermission(TestMRJobs.APP_JAR, new FsPermission("700"));
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() {
     if (mrCluster != null) {
       mrCluster.stop();
@@ -153,7 +154,7 @@ public class TestMRAppWithCombiner {
     public void reduce(K key, Iterator<V> values, OutputCollector<K, V> output,
         Reporter reporter) throws IOException {
       if (Reporter.NULL == reporter) {
-        Assert.fail("A valid Reporter should have been used but, Reporter.NULL is used");
+        fail("A valid Reporter should have been used but, Reporter.NULL is used");
       }
     }
   }

+ 147 - 131
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java

@@ -100,16 +100,22 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Workflow
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.WorkflowPriorityMappingsManager.WorkflowPriorityMapping;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
 import org.apache.log4j.Level;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Assume;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.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.assertSame;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.junit.jupiter.api.Assumptions.assumeFalse;
 
 public class TestMRJobs {
 
@@ -144,7 +150,7 @@ public class TestMRJobs {
   private static final Path TEST_RESOURCES_DIR = new Path(TEST_ROOT_DIR,
       "localizedResources");
 
-  @BeforeClass
+  @BeforeAll
   public static void setup() throws IOException {
     try {
       dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(2)
@@ -177,7 +183,7 @@ public class TestMRJobs {
     localFs.setPermission(APP_JAR, new FsPermission("700"));
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() throws IOException {
     if (mrCluster != null) {
       mrCluster.stop();
@@ -193,7 +199,7 @@ public class TestMRJobs {
     }
   }
 
-  @After
+  @AfterEach
   public void resetInit() {
     numSleepReducers = DEFAULT_REDUCES;
   }
@@ -231,17 +237,20 @@ public class TestMRJobs {
     localFs.createNewFile(new Path(subDir, "file4.txt"));
   }
 
-  @Test (timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testSleepJob() throws Exception {
     testSleepJobInternal(false);
   }
 
-  @Test (timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testSleepJobWithRemoteJar() throws Exception {
     testSleepJobInternal(true);
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testSleepJobWithLocalResourceUnderLimit() throws Exception {
     Configuration sleepConf = new Configuration(mrCluster.getConfig());
     // set limits to well above what is expected
@@ -252,7 +261,8 @@ public class TestMRJobs {
     testSleepJobInternal(sleepConf, false, true, null);
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testSleepJobWithLocalResourceSizeOverLimit() throws Exception {
     Configuration sleepConf = new Configuration(mrCluster.getConfig());
     // set limits to well below what is expected
@@ -263,7 +273,8 @@ public class TestMRJobs {
         ResourceViolation.TOTAL_RESOURCE_SIZE);
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testSleepJobWithLocalResourceNumberOverLimit() throws Exception {
     Configuration sleepConf = new Configuration(mrCluster.getConfig());
     // set limits to well below what is expected
@@ -274,7 +285,8 @@ public class TestMRJobs {
         ResourceViolation.NUMBER_OF_RESOURCES);
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testSleepJobWithLocalResourceCheckAndRemoteJar()
       throws Exception {
     Configuration sleepConf = new Configuration(mrCluster.getConfig());
@@ -286,7 +298,8 @@ public class TestMRJobs {
     testSleepJobInternal(sleepConf, true, true, null);
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testSleepJobWithLocalIndividualResourceOverLimit()
       throws Exception {
     Configuration sleepConf = new Configuration(mrCluster.getConfig());
@@ -298,7 +311,8 @@ public class TestMRJobs {
         ResourceViolation.SINGLE_RESOURCE_SIZE);
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testSleepJobWithLocalIndividualResourceUnderLimit()
       throws Exception {
     Configuration sleepConf = new Configuration(mrCluster.getConfig());
@@ -324,7 +338,7 @@ public class TestMRJobs {
     LOG.info("\n\n\nStarting testSleepJob: useRemoteJar=" + useRemoteJar);
 
     if (!jobSubmissionShouldSucceed && violation == null) {
-      Assert.fail("Test is misconfigured. jobSubmissionShouldSucceed is set"
+      fail("Test is misconfigured. jobSubmissionShouldSucceed is set"
           + " to false and a ResourceViolation is not specified.");
     }
 
@@ -356,38 +370,37 @@ public class TestMRJobs {
     job.setMaxMapAttempts(1); // speed up failures
     try {
       job.submit();
-      Assert.assertTrue("JobSubmission succeeded when it should have failed.",
-          jobSubmissionShouldSucceed);
+      assertTrue(jobSubmissionShouldSucceed,
+          "JobSubmission succeeded when it should have failed.");
     } catch (IOException e) {
       if (jobSubmissionShouldSucceed) {
-        Assert
-            .fail("Job submission failed when it should have succeeded: " + e);
+        fail("Job submission failed when it should have succeeded: " + e);
       }
       switch (violation) {
       case NUMBER_OF_RESOURCES:
         if (!e.getMessage().contains(
             "This job has exceeded the maximum number of"
                 + " submitted resources")) {
-          Assert.fail("Test failed unexpectedly: " + e);
+          fail("Test failed unexpectedly: " + e);
         }
         break;
 
       case TOTAL_RESOURCE_SIZE:
         if (!e.getMessage().contains(
             "This job has exceeded the maximum size of submitted resources")) {
-          Assert.fail("Test failed unexpectedly: " + e);
+          fail("Test failed unexpectedly: " + e);
         }
         break;
 
       case SINGLE_RESOURCE_SIZE:
         if (!e.getMessage().contains(
             "This job has exceeded the maximum size of a single submitted")) {
-          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;
       }
       // we are done with the test (job submission failed)
@@ -396,11 +409,10 @@ public class TestMRJobs {
     String trackingUrl = job.getTrackingURL();
     String jobId = job.getJobID().toString();
     boolean succeeded = job.waitForCompletion(true);
-    Assert.assertTrue(succeeded);
-    Assert.assertEquals(JobStatus.State.SUCCEEDED, job.getJobState());
-    Assert.assertTrue("Tracking URL was " + trackingUrl +
-                      " but didn't Match Job ID " + jobId ,
-          trackingUrl.endsWith(jobId.substring(jobId.lastIndexOf("_")) + "/"));
+    assertTrue(succeeded);
+    assertEquals(JobStatus.State.SUCCEEDED, job.getJobState());
+    assertTrue(trackingUrl.endsWith(jobId.substring(jobId.lastIndexOf("_")) + "/"),
+        "Tracking URL was " + trackingUrl + " but didn't Match Job ID " + jobId);
     verifySleepJobCounters(job);
     verifyTaskProgress(job);
     
@@ -408,12 +420,13 @@ public class TestMRJobs {
     // JobStatus?)--compare against MRJobConfig.JOB_UBERTASK_ENABLE value
   }
 
-  @Test(timeout = 3000000)
+  @Test
+  @Timeout(value = 3000)
   public void testJobWithChangePriority() throws Exception {
     Configuration sleepConf = new Configuration(mrCluster.getConfig());
     // Assumption can be removed when FS priority support is implemented
-    Assume.assumeFalse(sleepConf.get(YarnConfiguration.RM_SCHEDULER)
-            .equals(FairScheduler.class.getCanonicalName()));
+    assumeFalse(sleepConf.get(YarnConfiguration.RM_SCHEDULER)
+        .equals(FairScheduler.class.getCanonicalName()));
 
     if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {
       LOG.info("MRAppJar " + MiniMRYarnCluster.APPJAR
@@ -452,11 +465,12 @@ public class TestMRJobs {
     assertThat(job.getPriority()).isEqualTo(JobPriority.UNDEFINED_PRIORITY);
 
     boolean succeeded = job.waitForCompletion(true);
-    Assert.assertTrue(succeeded);
-    Assert.assertEquals(JobStatus.State.SUCCEEDED, job.getJobState());
+    assertTrue(succeeded);
+    assertEquals(JobStatus.State.SUCCEEDED, job.getJobState());
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testJobWithWorkflowPriority() throws Exception {
     Configuration sleepConf = new Configuration(mrCluster.getConfig());
     if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {
@@ -496,11 +510,11 @@ public class TestMRJobs {
 
     waitForPriorityToUpdate(job, JobPriority.VERY_LOW);
     // Verify the priority from job itself
-    Assert.assertEquals(JobPriority.VERY_LOW, job.getPriority());
+    assertEquals(JobPriority.VERY_LOW, job.getPriority());
 
     boolean succeeded = job.waitForCompletion(true);
-    Assert.assertTrue(succeeded);
-    Assert.assertEquals(JobStatus.State.SUCCEEDED, job.getJobState());
+    assertTrue(succeeded);
+    assertEquals(JobStatus.State.SUCCEEDED, job.getJobState());
   }
 
   private void waitForPriorityToUpdate(Job job, JobPriority expectedStatus)
@@ -518,13 +532,15 @@ public class TestMRJobs {
     }
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testJobClassloader() throws IOException, InterruptedException,
       ClassNotFoundException {
     testJobClassloader(false);
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testJobClassloaderWithCustomClasses() throws IOException,
       InterruptedException, ClassNotFoundException {
     testJobClassloader(true);
@@ -579,8 +595,7 @@ public class TestMRJobs {
     }
     job.submit();
     boolean succeeded = job.waitForCompletion(true);
-    Assert.assertTrue("Job status: " + job.getStatus().getFailureInfo(),
-        succeeded);
+    assertTrue(succeeded, "Job status: " + job.getStatus().getFailureInfo());
   }
 
   public static class CustomOutputFormat<K,V> extends NullOutputFormat<K,V> {
@@ -631,27 +646,28 @@ public class TestMRJobs {
   protected void verifySleepJobCounters(Job job) throws InterruptedException,
       IOException {
     Counters counters = job.getCounters();
-    Assert.assertEquals(3, counters.findCounter(JobCounter.OTHER_LOCAL_MAPS)
+    assertEquals(3, counters.findCounter(JobCounter.OTHER_LOCAL_MAPS)
         .getValue());
-    Assert.assertEquals(3, counters.findCounter(JobCounter.TOTAL_LAUNCHED_MAPS)
+    assertEquals(3, counters.findCounter(JobCounter.TOTAL_LAUNCHED_MAPS)
         .getValue());
-    Assert.assertEquals(numSleepReducers,
+    assertEquals(numSleepReducers,
         counters.findCounter(JobCounter.TOTAL_LAUNCHED_REDUCES).getValue());
   }
   
   protected void verifyTaskProgress(Job job) throws InterruptedException,
       IOException {
     for (TaskReport taskReport : job.getTaskReports(TaskType.MAP)) {
-      Assert.assertTrue(0.9999f < taskReport.getProgress()
+      assertTrue(0.9999f < taskReport.getProgress()
           && 1.0001f > taskReport.getProgress());
     }
     for (TaskReport taskReport : job.getTaskReports(TaskType.REDUCE)) {
-      Assert.assertTrue(0.9999f < taskReport.getProgress()
+      assertTrue(0.9999f < taskReport.getProgress()
           && 1.0001f > taskReport.getProgress());
     }
   }
 
-  @Test (timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testRandomWriter() throws IOException, InterruptedException,
       ClassNotFoundException {
     
@@ -676,11 +692,10 @@ public class TestMRJobs {
     String trackingUrl = job.getTrackingURL();
     String jobId = job.getJobID().toString();
     boolean succeeded = job.waitForCompletion(true);
-    Assert.assertTrue(succeeded);
-    Assert.assertEquals(JobStatus.State.SUCCEEDED, job.getJobState());
-    Assert.assertTrue("Tracking URL was " + trackingUrl +
-                      " but didn't Match Job ID " + jobId ,
-          trackingUrl.endsWith(jobId.substring(jobId.lastIndexOf("_")) + "/"));
+    assertTrue(succeeded);
+    assertEquals(JobStatus.State.SUCCEEDED, job.getJobState());
+    assertTrue(trackingUrl.endsWith(jobId.substring(jobId.lastIndexOf("_")) + "/"),
+        "Tracking URL was " + trackingUrl + " but didn't Match Job ID " + jobId);
     
     // Make sure there are three files in the output-dir
     
@@ -695,7 +710,7 @@ public class TestMRJobs {
         count++;
       }
     }
-    Assert.assertEquals("Number of part files is wrong!", 3, count);
+    assertEquals(3, count, "Number of part files is wrong!");
     verifyRandomWriterCounters(job);
 
     // TODO later:  add explicit "isUber()" checks of some sort
@@ -704,13 +719,14 @@ public class TestMRJobs {
   protected void verifyRandomWriterCounters(Job job)
       throws InterruptedException, IOException {
     Counters counters = job.getCounters();
-    Assert.assertEquals(3, counters.findCounter(JobCounter.OTHER_LOCAL_MAPS)
+    assertEquals(3, counters.findCounter(JobCounter.OTHER_LOCAL_MAPS)
         .getValue());
-    Assert.assertEquals(3, counters.findCounter(JobCounter.TOTAL_LAUNCHED_MAPS)
+    assertEquals(3, counters.findCounter(JobCounter.TOTAL_LAUNCHED_MAPS)
         .getValue());
   }
 
-  @Test (timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testFailingMapper() throws IOException, InterruptedException,
       ClassNotFoundException {
 
@@ -737,11 +753,11 @@ public class TestMRJobs {
     }
     
     TaskCompletionEvent[] events = job.getTaskCompletionEvents(0, 2);
-    Assert.assertEquals(TaskCompletionEvent.Status.FAILED, 
+    assertEquals(TaskCompletionEvent.Status.FAILED,
         events[0].getStatus());
-    Assert.assertEquals(TaskCompletionEvent.Status.TIPFAILED, 
+    assertEquals(TaskCompletionEvent.Status.TIPFAILED,
         events[1].getStatus());
-    Assert.assertEquals(JobStatus.State.FAILED, job.getJobState());
+    assertEquals(JobStatus.State.FAILED, job.getJobState());
     verifyFailingMapperCounters(job);
 
     // TODO later:  add explicit "isUber()" checks of some sort
@@ -750,15 +766,14 @@ public class TestMRJobs {
   protected void verifyFailingMapperCounters(Job job)
       throws InterruptedException, IOException {
     Counters counters = job.getCounters();
-    Assert.assertEquals(2, counters.findCounter(JobCounter.OTHER_LOCAL_MAPS)
+    assertEquals(2, counters.findCounter(JobCounter.OTHER_LOCAL_MAPS)
         .getValue());
-    Assert.assertEquals(2, counters.findCounter(JobCounter.TOTAL_LAUNCHED_MAPS)
+    assertEquals(2, counters.findCounter(JobCounter.TOTAL_LAUNCHED_MAPS)
         .getValue());
-    Assert.assertEquals(2, counters.findCounter(JobCounter.NUM_FAILED_MAPS)
+    assertEquals(2, counters.findCounter(JobCounter.NUM_FAILED_MAPS)
         .getValue());
-    Assert
-        .assertTrue(counters.findCounter(JobCounter.SLOTS_MILLIS_MAPS) != null
-            && counters.findCounter(JobCounter.SLOTS_MILLIS_MAPS).getValue() != 0);
+    assertTrue(counters.findCounter(JobCounter.SLOTS_MILLIS_MAPS) != null
+        && counters.findCounter(JobCounter.SLOTS_MILLIS_MAPS).getValue() != 0);
   }
 
   protected Job runFailingMapperJob()
@@ -785,10 +800,10 @@ public class TestMRJobs {
     String trackingUrl = job.getTrackingURL();
     String jobId = job.getJobID().toString();
     boolean succeeded = job.waitForCompletion(true);
-    Assert.assertFalse(succeeded);
-    Assert.assertTrue("Tracking URL was " + trackingUrl +
-                      " but didn't Match Job ID " + jobId ,
-          trackingUrl.endsWith(jobId.substring(jobId.lastIndexOf("_")) + "/"));
+    assertFalse(succeeded);
+    assertTrue(trackingUrl.endsWith(jobId.substring(jobId.lastIndexOf("_")) + "/"),
+        "Tracking URL was " + trackingUrl +
+        " but didn't Match Job ID " + jobId);
     return job;
   }
 
@@ -834,10 +849,9 @@ public class TestMRJobs {
         String trackingUrl = job.getTrackingURL();
         String jobId = job.getJobID().toString();
         job.waitForCompletion(true);
-        Assert.assertEquals(JobStatus.State.SUCCEEDED, job.getJobState());
-        Assert.assertTrue("Tracking URL was " + trackingUrl +
-                          " but didn't Match Job ID " + jobId ,
-          trackingUrl.endsWith(jobId.substring(jobId.lastIndexOf("_")) + "/"));
+        assertEquals(JobStatus.State.SUCCEEDED, job.getJobState());
+        assertTrue(trackingUrl.endsWith(jobId.substring(jobId.lastIndexOf("_")) + "/"),
+            "Tracking URL was " + trackingUrl + " but didn't Match Job ID " + jobId);
         return null;
       }
     });
@@ -845,7 +859,8 @@ public class TestMRJobs {
     // TODO later:  add explicit "isUber()" checks of some sort
   }
 
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testContainerRollingLog() throws IOException,
       InterruptedException, ClassNotFoundException {
     if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {
@@ -886,7 +901,7 @@ public class TestMRJobs {
         break;
       }
     }
-    Assert.assertEquals(RMAppState.FINISHED, mrCluster.getResourceManager()
+    assertEquals(RMAppState.FINISHED, mrCluster.getResourceManager()
         .getRMContext().getRMApps().get(appID).getState());
 
     // Job finished, verify logs
@@ -932,28 +947,26 @@ public class TestMRJobs {
           }
 
           if (foundAppMaster) {
-            Assert.assertSame("Unexpected number of AM sylog* files",
-                sleepConf.getInt(MRJobConfig.MR_AM_LOG_BACKUPS, 0) + 1,
-                sysSiblings.length);
-            Assert.assertTrue("AM syslog.1 length kb should be >= " + amLogKb,
-                sysSiblings[1].getLen() >= amLogKb * 1024);
+            assertSame(sleepConf.getInt(MRJobConfig.MR_AM_LOG_BACKUPS, 0) + 1,
+                sysSiblings.length, "Unexpected number of AM sylog* files");
+            assertTrue(sysSiblings[1].getLen() >= amLogKb * 1024,
+                "AM syslog.1 length kb should be >= " + amLogKb);
           } else {
-            Assert.assertSame("Unexpected number of MR task sylog* files",
-                sleepConf.getInt(MRJobConfig.TASK_LOG_BACKUPS, 0) + 1,
-                sysSiblings.length);
-            Assert.assertTrue("MR syslog.1 length kb should be >= " + userLogKb,
-                sysSiblings[1].getLen() >= userLogKb * 1024);
+            assertSame(sleepConf.getInt(MRJobConfig.TASK_LOG_BACKUPS, 0) + 1,
+                sysSiblings.length, "Unexpected number of MR task sylog* files");
+            assertTrue(sysSiblings[1].getLen() >= userLogKb * 1024,
+                "MR syslog.1 length kb should be >= " + userLogKb);
           }
         }
       }
     }
     // Make sure we checked non-empty set
     //
-    Assert.assertEquals("No AppMaster log found!", 1, numAppMasters);
+    assertEquals(1, numAppMasters, "No AppMaster log found!");
     if (sleepConf.getBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false)) {
-      Assert.assertEquals("MapTask log with uber found!", 0, numMapTasks);
+      assertEquals(0, numMapTasks, "MapTask log with uber found!");
     } else {
-      Assert.assertEquals("No MapTask log found!", 1, numMapTasks);
+      assertEquals(1, numMapTasks, "No MapTask log found!");
     }
   }
 
@@ -970,27 +983,27 @@ public class TestMRJobs {
 
       // Check that 4 (2 + appjar + DistrubutedCacheChecker jar) files 
       // and 2 archives are present
-      Assert.assertEquals(4, localFiles.length);
-      Assert.assertEquals(4, files.length);
-      Assert.assertEquals(2, localArchives.length);
-      Assert.assertEquals(2, archives.length);
+      assertEquals(4, localFiles.length);
+      assertEquals(4, files.length);
+      assertEquals(2, localArchives.length);
+      assertEquals(2, archives.length);
 
       // Check lengths of the files
       Map<String, Path> filesMap = pathsToMap(localFiles);
-      Assert.assertTrue(filesMap.containsKey("distributed.first.symlink"));
-      Assert.assertEquals(1, localFs.getFileStatus(
+      assertTrue(filesMap.containsKey("distributed.first.symlink"));
+      assertEquals(1, localFs.getFileStatus(
         filesMap.get("distributed.first.symlink")).getLen());
-      Assert.assertTrue(filesMap.containsKey("distributed.second.jar"));
-      Assert.assertTrue(localFs.getFileStatus(
+      assertTrue(filesMap.containsKey("distributed.second.jar"));
+      assertTrue(localFs.getFileStatus(
         filesMap.get("distributed.second.jar")).getLen() > 1);
 
       // Check extraction of the archive
       Map<String, Path> archivesMap = pathsToMap(localArchives);
-      Assert.assertTrue(archivesMap.containsKey("distributed.third.jar"));
-      Assert.assertTrue(localFs.exists(new Path(
+      assertTrue(archivesMap.containsKey("distributed.third.jar"));
+      assertTrue(localFs.exists(new Path(
         archivesMap.get("distributed.third.jar"), "distributed.jar.inside3")));
-      Assert.assertTrue(archivesMap.containsKey("distributed.fourth.jar"));
-      Assert.assertTrue(localFs.exists(new Path(
+      assertTrue(archivesMap.containsKey("distributed.fourth.jar"));
+      assertTrue(localFs.exists(new Path(
         archivesMap.get("distributed.fourth.jar"), "distributed.jar.inside4")));
 
       // Check the class loaders
@@ -998,29 +1011,29 @@ public class TestMRJobs {
       ClassLoader cl = Thread.currentThread().getContextClassLoader();
       // Both the file and the archive should have been added to classpath, so
       // both should be reachable via the class loader.
-      Assert.assertNotNull(cl.getResource("distributed.jar.inside2"));
-      Assert.assertNotNull(cl.getResource("distributed.jar.inside3"));
-      Assert.assertNotNull(cl.getResource("distributed.jar.inside4"));
+      assertNotNull(cl.getResource("distributed.jar.inside2"));
+      assertNotNull(cl.getResource("distributed.jar.inside3"));
+      assertNotNull(cl.getResource("distributed.jar.inside4"));
       // The Job Jar should have been extracted to a folder named "job.jar" and
       // added to the classpath; the two jar files in the lib folder in the Job
       // Jar should have also been added to the classpath
-      Assert.assertNotNull(cl.getResource("job.jar/"));
-      Assert.assertNotNull(cl.getResource("job.jar/lib/lib1.jar"));
-      Assert.assertNotNull(cl.getResource("job.jar/lib/lib2.jar"));
+      assertNotNull(cl.getResource("job.jar/"));
+      assertNotNull(cl.getResource("job.jar/lib/lib1.jar"));
+      assertNotNull(cl.getResource("job.jar/lib/lib2.jar"));
 
       // Check that the symlink for the renaming was created in the cwd;
       File symlinkFile = new File("distributed.first.symlink");
-      Assert.assertTrue(symlinkFile.exists());
-      Assert.assertEquals(1, symlinkFile.length());
+      assertTrue(symlinkFile.exists());
+      assertEquals(1, symlinkFile.length());
       
       // Check that the symlink for the Job Jar was created in the cwd and
       // points to the extracted directory
       File jobJarDir = new File("job.jar");
       if (Shell.WINDOWS) {
-        Assert.assertTrue(isWindowsSymlinkedDirectory(jobJarDir));
+        assertTrue(isWindowsSymlinkedDirectory(jobJarDir));
       } else {
-        Assert.assertTrue(FileUtils.isSymlink(jobJarDir));
-        Assert.assertTrue(jobJarDir.isDirectory());
+        assertTrue(FileUtils.isSymlink(jobJarDir));
+        assertTrue(jobJarDir.isDirectory());
       }
     }
 
@@ -1136,10 +1149,10 @@ public class TestMRJobs {
     job.submit();
     String trackingUrl = job.getTrackingURL();
     String jobId = job.getJobID().toString();
-    Assert.assertTrue(job.waitForCompletion(false));
-    Assert.assertTrue("Tracking URL was " + trackingUrl +
-                      " but didn't Match Job ID " + jobId ,
-          trackingUrl.endsWith(jobId.substring(jobId.lastIndexOf("_")) + "/"));
+    assertTrue(job.waitForCompletion(false));
+    assertTrue(trackingUrl.endsWith(jobId.substring(jobId.lastIndexOf("_")) + "/"),
+        "Tracking URL was " + trackingUrl +
+        " but didn't Match Job ID " + jobId);
   }
   
   private void testDistributedCache(boolean withWildcard) throws Exception {
@@ -1158,17 +1171,20 @@ public class TestMRJobs {
     testDistributedCache(remoteJobJarPath.toUri().toString(), withWildcard);
   }
 
-  @Test (timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testDistributedCache() throws Exception {
     testDistributedCache(false);
   }
 
-  @Test (timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testDistributedCacheWithWildcards() throws Exception {
     testDistributedCache(true);
   }
 
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testThreadDumpOnTaskTimeout() throws IOException,
       InterruptedException, ClassNotFoundException {
     if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {
@@ -1264,13 +1280,13 @@ public class TestMRJobs {
           if (foundAppMaster) {
             numAppMasters++;
             if (this instanceof TestUberAM) {
-              Assert.assertTrue("No thread dump", foundThreadDump);
+              assertTrue(foundThreadDump, "No thread dump");
             } else {
-              Assert.assertFalse("Unexpected thread dump", foundThreadDump);
+              assertFalse(foundThreadDump, "Unexpected thread dump");
             }
           } else {
             numMapTasks++;
-            Assert.assertTrue("No thread dump", foundThreadDump);
+            assertTrue(foundThreadDump, "No thread dump");
           }
         }
       }
@@ -1278,11 +1294,11 @@ public class TestMRJobs {
 
     // Make sure we checked non-empty set
     //
-    Assert.assertEquals("No AppMaster log found!", 1, numAppMasters);
+    assertEquals(1, numAppMasters, "No AppMaster log found!");
     if (sleepConf.getBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false)) {
-      Assert.assertSame("MapTask log with uber found!", 0, numMapTasks);
+      assertSame(0, numMapTasks, "MapTask log with uber found!");
     } else {
-      Assert.assertSame("No MapTask log found!", 1, numMapTasks);
+      assertSame(1, numMapTasks, "No MapTask log found!");
     }
   }
 
@@ -1394,10 +1410,10 @@ public class TestMRJobs {
     job.submit();
     String trackingUrl = job.getTrackingURL();
     String jobId = job.getJobID().toString();
-    Assert.assertTrue(job.waitForCompletion(true));
-    Assert.assertTrue("Tracking URL was " + trackingUrl
-        + " but didn't Match Job ID " + jobId,
-        trackingUrl.endsWith(jobId.substring(jobId.lastIndexOf("_")) + "/"));
+    assertTrue(job.waitForCompletion(true));
+    assertTrue(trackingUrl.endsWith(jobId.substring(jobId.lastIndexOf("_")) + "/"),
+        "Tracking URL was " + trackingUrl
+        + " but didn't Match Job ID " + jobId);
   }
 
   /**

+ 19 - 16
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithHistoryService.java

@@ -23,8 +23,6 @@ import java.io.IOException;
 import java.util.EnumSet;
 import java.util.List;
 
-import org.junit.Assert;
-
 import org.apache.avro.AvroRemoteException;
 import org.apache.hadoop.mapreduce.SleepJob;
 import org.apache.hadoop.conf.Configuration;
@@ -47,12 +45,16 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.util.Records;
-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 org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
 public class TestMRJobsWithHistoryService {
 
   private static final Logger LOG =
@@ -77,7 +79,7 @@ public class TestMRJobsWithHistoryService {
       new Path("target", TestMRJobs.class.getName() + "-tmpDir"));
   static Path APP_JAR = new Path(TEST_ROOT_DIR, "MRAppJar.jar");
 
-  @Before
+  @BeforeEach
   public void setup() throws InterruptedException, IOException {
 
     if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {
@@ -98,7 +100,7 @@ public class TestMRJobsWithHistoryService {
     localFs.setPermission(APP_JAR, new FsPermission("700"));
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
     if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {
       LOG.info("MRAppJar " + MiniMRYarnCluster.APPJAR
@@ -111,7 +113,8 @@ public class TestMRJobsWithHistoryService {
     }
   }
 
-  @Test (timeout = 90000)
+  @Test
+  @Timeout(value = 90)
   public void testJobHistoryData() throws IOException, InterruptedException,
       AvroRemoteException, ClassNotFoundException {
     if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {
@@ -148,14 +151,14 @@ public class TestMRJobsWithHistoryService {
         break;
       }
     }
-    Assert.assertEquals(RMAppState.FINISHED, mrCluster.getResourceManager()
+    assertEquals(RMAppState.FINISHED, mrCluster.getResourceManager()
       .getRMContext().getRMApps().get(appID).getState());
     Counters counterHS = job.getCounters();
     //TODO the Assert below worked. need to check
     //Should we compare each field or convert to V2 counter and compare
     LOG.info("CounterHS " + counterHS);
     LOG.info("CounterMR " + counterMR);
-    Assert.assertEquals(counterHS, counterMR);
+    assertEquals(counterHS, counterMR);
     
     HSClientProtocol historyClient = instantiateHistoryProxy();
     GetJobReportRequest gjReq = Records.newRecord(GetJobReportRequest.class);
@@ -166,16 +169,16 @@ public class TestMRJobsWithHistoryService {
 
   private void verifyJobReport(JobReport jobReport, JobId jobId) {
     List<AMInfo> amInfos = jobReport.getAMInfos();
-    Assert.assertEquals(1, amInfos.size());
+    assertEquals(1, amInfos.size());
     AMInfo amInfo = amInfos.get(0);
     ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(jobId.getAppId(), 1);
     ContainerId amContainerId = ContainerId.newContainerId(appAttemptId, 1);
-    Assert.assertEquals(appAttemptId, amInfo.getAppAttemptId());
-    Assert.assertEquals(amContainerId, amInfo.getContainerId());
-    Assert.assertTrue(jobReport.getSubmitTime() > 0);
-    Assert.assertTrue(jobReport.getStartTime() > 0
+    assertEquals(appAttemptId, amInfo.getAppAttemptId());
+    assertEquals(amContainerId, amInfo.getContainerId());
+    assertTrue(jobReport.getSubmitTime() > 0);
+    assertTrue(jobReport.getStartTime() > 0
         && jobReport.getStartTime() >= jobReport.getSubmitTime());
-    Assert.assertTrue(jobReport.getFinishTime() > 0
+    assertTrue(jobReport.getFinishTime() > 0
         && jobReport.getFinishTime() >= jobReport.getStartTime());
   }
   

+ 27 - 22
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithProfiler.java

@@ -24,8 +24,7 @@ import java.util.*;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import org.junit.AfterClass;
-import org.junit.Assert;
+import org.junit.jupiter.api.AfterAll;
 
 import org.apache.hadoop.mapreduce.SleepJob;
 import org.apache.hadoop.conf.Configuration;
@@ -39,12 +38,16 @@ import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.junit.Assume.assumeFalse;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assumptions.assumeFalse;
 
 public class TestMRJobsWithProfiler {
 
@@ -74,7 +77,7 @@ public class TestMRJobsWithProfiler {
 
   private static final Path APP_JAR = new Path(TEST_ROOT_DIR, "MRAppJar.jar");
 
-  @BeforeClass
+  @BeforeAll
   public static void setup() throws InterruptedException, IOException {
 
     if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {
@@ -95,7 +98,7 @@ public class TestMRJobsWithProfiler {
     localFs.setPermission(APP_JAR, new FsPermission("700"));
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() {
     if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {
       LOG.info("MRAppJar " + MiniMRYarnCluster.APPJAR
@@ -108,15 +111,17 @@ public class TestMRJobsWithProfiler {
     }
   }
 
-  @Test (timeout = 150000)
+  @Test
+  @Timeout(value = 150)
   public void testDefaultProfiler() throws Exception {
-    assumeFalse("The hprof agent has been removed since Java 9. Skipping.",
-        Shell.isJavaVersionAtLeast(9));
+    assumeFalse(Shell.isJavaVersionAtLeast(9),
+        "The hprof agent has been removed since Java 9. Skipping.");
     LOG.info("Starting testDefaultProfiler");
     testProfilerInternal(true);
   }
 
-  @Test (timeout = 150000)
+  @Test
+  @Timeout(value = 150)
   public void testDifferentProfilers() throws Exception {
     LOG.info("Starting testDefaultProfiler");
     testProfilerInternal(false);
@@ -179,7 +184,7 @@ public class TestMRJobsWithProfiler {
         break;
       }
     }
-    Assert.assertEquals(RMAppState.FINISHED, mrCluster.getResourceManager()
+    assertEquals(RMAppState.FINISHED, mrCluster.getResourceManager()
       .getRMContext().getRMApps().get(appID).getState());
 
     // Job finished, verify logs
@@ -222,7 +227,7 @@ public class TestMRJobsWithProfiler {
       }
     }
 
-    Assert.assertEquals(4, taLogDirs.size());  // all 4 attempts found
+    assertEquals(4, taLogDirs.size());  // all 4 attempts found
 
     // Skip checking the contents because the JFR dumps binary files
     if (Shell.isJavaVersionAtLeast(9)) {
@@ -241,17 +246,17 @@ public class TestMRJobsWithProfiler {
           final BufferedReader br = new BufferedReader(new InputStreamReader(
             localFs.open(profilePath)));
           final String line = br.readLine();
-          Assert.assertTrue("No hprof content found!",
-            line !=null && line.startsWith("JAVA PROFILE"));
+          assertTrue(line !=null && line.startsWith("JAVA PROFILE"),
+              "No hprof content found!");
           br.close();
-          Assert.assertEquals(0L, localFs.getFileStatus(stdoutPath).getLen());
+          assertEquals(0L, localFs.getFileStatus(stdoutPath).getLen());
         } else {
-          Assert.assertFalse("hprof file should not exist",
-            localFs.exists(profilePath));
+          assertFalse(localFs.exists(profilePath),
+              "hprof file should not exist");
         }
       } else {
-        Assert.assertFalse("hprof file should not exist",
-          localFs.exists(profilePath));
+        assertFalse(localFs.exists(profilePath),
+            "hprof file should not exist");
         if (tid.getTaskID().getId() == PROFILED_TASK_ID) {
           // reducer is profiled with Xprof
           final BufferedReader br = new BufferedReader(new InputStreamReader(
@@ -265,9 +270,9 @@ public class TestMRJobsWithProfiler {
             }
           }
           br.close();
-          Assert.assertTrue("Xprof flat profile not found!", flatProfFound);
+          assertTrue(flatProfFound, "Xprof flat profile not found!");
         } else {
-          Assert.assertEquals(0L, localFs.getFileStatus(stdoutPath).getLen());
+          assertEquals(0L, localFs.getFileStatus(stdoutPath).getLen());
         }
       }
     }

+ 22 - 20
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMROldApiJobs.java

@@ -40,13 +40,15 @@ import org.apache.hadoop.mapred.lib.IdentityMapper;
 import org.apache.hadoop.mapred.lib.IdentityReducer;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.filecache.DistributedCache;
-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.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
 public class TestMROldApiJobs {
 
   private static final Logger LOG =
@@ -63,7 +65,7 @@ public class TestMROldApiJobs {
     }
   }
 
-  @BeforeClass
+  @BeforeAll
   public static void setup()  throws IOException {
 
     if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {
@@ -88,7 +90,7 @@ public class TestMROldApiJobs {
     localFs.setPermission(TestMRJobs.APP_JAR, new FsPermission("700"));
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() {
     if (mrCluster != null) {
       mrCluster.stop();
@@ -117,12 +119,12 @@ public class TestMROldApiJobs {
     runJobSucceed(conf, in, out);
     
     FileSystem fs = FileSystem.get(conf);
-    Assert.assertTrue(fs.exists(new Path(out, CustomOutputCommitter.JOB_SETUP_FILE_NAME)));
-    Assert.assertFalse(fs.exists(new Path(out, CustomOutputCommitter.JOB_ABORT_FILE_NAME)));
-    Assert.assertTrue(fs.exists(new Path(out, CustomOutputCommitter.JOB_COMMIT_FILE_NAME)));
-    Assert.assertTrue(fs.exists(new Path(out, CustomOutputCommitter.TASK_SETUP_FILE_NAME)));
-    Assert.assertFalse(fs.exists(new Path(out, CustomOutputCommitter.TASK_ABORT_FILE_NAME)));
-    Assert.assertTrue(fs.exists(new Path(out, CustomOutputCommitter.TASK_COMMIT_FILE_NAME)));
+    assertTrue(fs.exists(new Path(out, CustomOutputCommitter.JOB_SETUP_FILE_NAME)));
+    assertFalse(fs.exists(new Path(out, CustomOutputCommitter.JOB_ABORT_FILE_NAME)));
+    assertTrue(fs.exists(new Path(out, CustomOutputCommitter.JOB_COMMIT_FILE_NAME)));
+    assertTrue(fs.exists(new Path(out, CustomOutputCommitter.TASK_SETUP_FILE_NAME)));
+    assertFalse(fs.exists(new Path(out, CustomOutputCommitter.TASK_ABORT_FILE_NAME)));
+    assertTrue(fs.exists(new Path(out, CustomOutputCommitter.TASK_COMMIT_FILE_NAME)));
   }
 
   @Test
@@ -146,12 +148,12 @@ public class TestMROldApiJobs {
     runJobFail(conf, in, out);
     
     FileSystem fs = FileSystem.get(conf);
-    Assert.assertTrue(fs.exists(new Path(out, CustomOutputCommitter.JOB_SETUP_FILE_NAME)));
-    Assert.assertTrue(fs.exists(new Path(out, CustomOutputCommitter.JOB_ABORT_FILE_NAME)));
-    Assert.assertFalse(fs.exists(new Path(out, CustomOutputCommitter.JOB_COMMIT_FILE_NAME)));
-    Assert.assertTrue(fs.exists(new Path(out, CustomOutputCommitter.TASK_SETUP_FILE_NAME)));
-    Assert.assertTrue(fs.exists(new Path(out, CustomOutputCommitter.TASK_ABORT_FILE_NAME)));
-    Assert.assertFalse(fs.exists(new Path(out, CustomOutputCommitter.TASK_COMMIT_FILE_NAME)));
+    assertTrue(fs.exists(new Path(out, CustomOutputCommitter.JOB_SETUP_FILE_NAME)));
+    assertTrue(fs.exists(new Path(out, CustomOutputCommitter.JOB_ABORT_FILE_NAME)));
+    assertFalse(fs.exists(new Path(out, CustomOutputCommitter.JOB_COMMIT_FILE_NAME)));
+    assertTrue(fs.exists(new Path(out, CustomOutputCommitter.TASK_SETUP_FILE_NAME)));
+    assertTrue(fs.exists(new Path(out, CustomOutputCommitter.TASK_ABORT_FILE_NAME)));
+    assertFalse(fs.exists(new Path(out, CustomOutputCommitter.TASK_COMMIT_FILE_NAME)));
   }
 
   //Run a job that will be failed and wait until it completes
@@ -164,7 +166,7 @@ public class TestMROldApiJobs {
     conf.setMaxMapAttempts(1);
     
     boolean success = runJob(conf, inDir, outDir, 1, 0);
-    Assert.assertFalse("Job expected to fail succeeded", success);
+    assertFalse(success, "Job expected to fail succeeded");
   }
 
   //Run a job that will be succeeded and wait until it completes
@@ -176,7 +178,7 @@ public class TestMROldApiJobs {
     conf.setReducerClass(IdentityReducer.class);
     
     boolean success = runJob(conf, inDir, outDir, 1 , 1);
-    Assert.assertTrue("Job expected to succeed failed", success);
+    assertTrue(success, "Job expected to succeed failed");
   }
 
   static boolean runJob(JobConf conf, Path inDir, Path outDir, int numMaps, 

+ 7 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMiniMRProxyUser.java

@@ -28,24 +28,24 @@ import org.apache.hadoop.mapred.MiniMRCluster;
 import org.apache.hadoop.mapred.RunningJob;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.ProxyUsers;
-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 java.io.OutputStreamWriter;
 import java.io.Writer;
 import java.net.InetAddress;
 import java.security.PrivilegedExceptionAction;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 public class TestMiniMRProxyUser {
 
   private MiniDFSCluster dfsCluster = null;
   private MiniMRCluster mrCluster = null;
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     if (System.getProperty("hadoop.log.dir") == null) {
       System.setProperty("hadoop.log.dir", "/tmp");
@@ -93,7 +93,7 @@ public class TestMiniMRProxyUser {
     return mrCluster.createJobConf();
   }
   
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     if (mrCluster != null) {
       mrCluster.shutdown();

+ 6 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestNonExistentJob.java

@@ -27,20 +27,20 @@ import org.apache.hadoop.mapred.JobID;
 import org.apache.hadoop.mapred.MiniMRCluster;
 import org.apache.hadoop.mapred.RunningJob;
 import org.apache.hadoop.security.authorize.ProxyUsers;
-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 java.net.InetAddress;
 
-import static org.junit.Assert.assertNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
 
 public class TestNonExistentJob {
 
   private MiniDFSCluster dfsCluster = null;
   private MiniMRCluster mrCluster = null;
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     if (System.getProperty("hadoop.log.dir") == null) {
       System.setProperty("hadoop.log.dir", "/tmp");
@@ -81,7 +81,7 @@ public class TestNonExistentJob {
     return mrCluster.createJobConf();
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     if (mrCluster != null) {
       mrCluster.shutdown();

+ 38 - 38
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestRMNMInfo.java

@@ -36,14 +36,18 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMNMInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
-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.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.mockito.Mockito.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 public class TestRMNMInfo {
   private static final Logger LOG = LoggerFactory.getLogger(TestRMNMInfo.class);
@@ -64,7 +68,7 @@ public class TestRMNMInfo {
               .makeQualified(localFs.getUri(), localFs.getWorkingDirectory());
   static Path APP_JAR = new Path(TEST_ROOT_DIR, "MRAppJar.jar");
 
-  @BeforeClass
+  @BeforeAll
   public static void setup() throws IOException {
 
     if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {
@@ -86,7 +90,7 @@ public class TestRMNMInfo {
     localFs.setPermission(APP_JAR, new FsPermission("700"));
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() {
     if (mrCluster != null) {
       mrCluster.stop();
@@ -109,27 +113,24 @@ public class TestRMNMInfo {
     String liveNMs = rmInfo.getLiveNodeManagers();
     ObjectMapper mapper = new ObjectMapper();
     JsonNode jn = mapper.readTree(liveNMs);
-    Assert.assertEquals("Unexpected number of live nodes:",
-                                               NUMNODEMANAGERS, jn.size());
+    assertEquals(NUMNODEMANAGERS, jn.size(), "Unexpected number of live nodes:");
     Iterator<JsonNode> it = jn.iterator();
     while (it.hasNext()) {
       JsonNode n = it.next();
-      Assert.assertNotNull(n.get("HostName"));
-      Assert.assertNotNull(n.get("Rack"));
-      Assert.assertTrue("Node " + n.get("NodeId") + " should be RUNNING",
-              n.get("State").asText().contains("RUNNING"));
-      Assert.assertNotNull(n.get("NodeHTTPAddress"));
-      Assert.assertNotNull(n.get("LastHealthUpdate"));
-      Assert.assertNotNull(n.get("HealthReport"));
-      Assert.assertNotNull(n.get("NodeManagerVersion"));
-      Assert.assertNotNull(n.get("NumContainers"));
-      Assert.assertEquals(
-              n.get("NodeId") + ": Unexpected number of used containers",
-              0, n.get("NumContainers").asInt());
-      Assert.assertEquals(
-              n.get("NodeId") + ": Unexpected amount of used memory",
-              0, n.get("UsedMemoryMB").asInt());
-      Assert.assertNotNull(n.get("AvailableMemoryMB"));
+      assertNotNull(n.get("HostName"));
+      assertNotNull(n.get("Rack"));
+      assertTrue(n.get("State").asText().contains("RUNNING"),
+          "Node " + n.get("NodeId") + " should be RUNNING");
+      assertNotNull(n.get("NodeHTTPAddress"));
+      assertNotNull(n.get("LastHealthUpdate"));
+      assertNotNull(n.get("HealthReport"));
+      assertNotNull(n.get("NodeManagerVersion"));
+      assertNotNull(n.get("NumContainers"));
+      assertEquals(0, n.get("NumContainers").asInt(),
+          n.get("NodeId") + ": Unexpected number of used containers");
+      assertEquals(0, n.get("UsedMemoryMB").asInt(),
+          n.get("NodeId") + ": Unexpected amount of used memory");
+      assertNotNull(n.get("AvailableMemoryMB"));
     }
   }
   
@@ -146,22 +147,21 @@ public class TestRMNMInfo {
     String liveNMs = rmInfo.getLiveNodeManagers();
     ObjectMapper mapper = new ObjectMapper();
     JsonNode jn = mapper.readTree(liveNMs);
-    Assert.assertEquals("Unexpected number of live nodes:",
-                                               1, jn.size());
+    assertEquals(1, jn.size(), "Unexpected number of live nodes:");
     Iterator<JsonNode> it = jn.iterator();
     while (it.hasNext()) {
       JsonNode n = it.next();
-      Assert.assertNotNull(n.get("HostName"));
-      Assert.assertNotNull(n.get("Rack"));
-      Assert.assertTrue("Node " + n.get("NodeId") + " should be RUNNING",
-              n.get("State").asText().contains("RUNNING"));
-      Assert.assertNotNull(n.get("NodeHTTPAddress"));
-      Assert.assertNotNull(n.get("LastHealthUpdate"));
-      Assert.assertNotNull(n.get("HealthReport"));
-      Assert.assertNotNull(n.get("NodeManagerVersion"));
-      Assert.assertNull(n.get("NumContainers"));
-      Assert.assertNull(n.get("UsedMemoryMB"));
-      Assert.assertNull(n.get("AvailableMemoryMB"));
+      assertNotNull(n.get("HostName"));
+      assertNotNull(n.get("Rack"));
+      assertTrue(n.get("State").asText().contains("RUNNING"),
+          "Node " + n.get("NodeId") + " should be RUNNING");
+      assertNotNull(n.get("NodeHTTPAddress"));
+      assertNotNull(n.get("LastHealthUpdate"));
+      assertNotNull(n.get("HealthReport"));
+      assertNotNull(n.get("NodeManagerVersion"));
+      assertNull(n.get("NumContainers"));
+      assertNull(n.get("UsedMemoryMB"));
+      assertNull(n.get("AvailableMemoryMB"));
     }
   }
 }

+ 92 - 83
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestSpeculativeExecOnCluster.java

@@ -57,21 +57,20 @@ import org.apache.hadoop.mapreduce.v2.app.speculate.ExponentiallySmoothedTaskRun
 import org.apache.hadoop.mapreduce.v2.app.speculate.LegacyTaskRuntimeEstimator;
 import org.apache.hadoop.mapreduce.v2.app.speculate.SimpleExponentialTaskRuntimeEstimator;
 import org.apache.hadoop.mapreduce.v2.app.speculate.TaskRuntimeEstimator;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
 /**
  * Test speculation on Mini Cluster.
  */
-@Ignore
-@RunWith(Parameterized.class)
+@Disabled
 public class TestSpeculativeExecOnCluster {
   private static final Logger LOG = LoggerFactory.getLogger(TestSpeculativeExecOnCluster.class);
 
@@ -147,8 +146,6 @@ public class TestSpeculativeExecOnCluster {
    */
   private List<String> ignoredTests;
 
-
-  @Parameterized.Parameters(name = "{index}: TaskEstimator(EstimatorClass {0})")
   public static Collection<Object[]> getTestParameters() {
     List<String> ignoredTests = Arrays.asList(new String[] {
         "stalled_run",
@@ -163,19 +160,18 @@ public class TestSpeculativeExecOnCluster {
     });
   }
 
-  public TestSpeculativeExecOnCluster(
-      Class<? extends TaskRuntimeEstimator> estimatorKlass,
-      List<String> testToIgnore,
-      Integer numMapper,
-      Integer numReduce) {
-    this.ignoredTests = testToIgnore;
-    this.estimatorClass = estimatorKlass;
-    this.myNumMapper = numMapper;
-    this.myNumReduce = numReduce;
-
+  public void initTestSpeculativeExecOnCluster(
+      Class<? extends TaskRuntimeEstimator> pEstimatorKlass,
+      List<String> pTestToIgnore,
+      Integer pNumMapper,
+      Integer pNumReduce) throws IOException {
+    this.ignoredTests = pTestToIgnore;
+    this.estimatorClass = pEstimatorKlass;
+    this.myNumMapper = pNumMapper;
+    this.myNumReduce = pNumReduce;
+    setup();
   }
 
-  @Before
   public void setup() throws IOException {
 
     if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {
@@ -203,7 +199,7 @@ public class TestSpeculativeExecOnCluster {
     chosenSleepCalc = MAP_SLEEP_CALCULATOR_TYPE_DEFAULT;
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
     if (mrCluster != null) {
       mrCluster.stop();
@@ -598,8 +594,12 @@ public class TestSpeculativeExecOnCluster {
     }
   }
 
-  @Test
-  public void testExecDynamicSlowingSpeculative() throws Exception {
+  @ParameterizedTest(name = "{index}: TaskEstimator(EstimatorClass {0})")
+  @MethodSource("getTestParameters")
+  public void testExecDynamicSlowingSpeculative(
+      Class<? extends TaskRuntimeEstimator> pEstimatorKlass,
+      List<String> pTestToIgnore, Integer pNumMapper, Integer pNumReduce) throws Exception {
+    initTestSpeculativeExecOnCluster(pEstimatorKlass, pTestToIgnore, pNumMapper, pNumReduce);
     /*------------------------------------------------------------------
      * Test that Map/Red speculates because:
      * 1- all tasks have same progress rate except for task_0
@@ -641,27 +641,28 @@ public class TestSpeculativeExecOnCluster {
       Job job = runSpecTest();
 
       boolean succeeded = job.waitForCompletion(true);
-      Assert.assertTrue(
-          "Job expected to succeed with estimator " + estimatorClass.getName(),
-          succeeded);
-      Assert.assertEquals(
-          "Job expected to succeed with estimator " + estimatorClass.getName(),
-          JobStatus.State.SUCCEEDED, job.getJobState());
+      assertTrue(succeeded,
+          "Job expected to succeed with estimator " + estimatorClass.getName());
+      assertEquals(JobStatus.State.SUCCEEDED, job.getJobState(),
+          "Job expected to succeed with estimator " + estimatorClass.getName());
       Counters counters = job.getCounters();
 
       String errorMessage = specEstimator.getErrorMessage(counters);
       boolean didSpeculate = specEstimator.didSpeculate(counters);
-      Assert.assertEquals(errorMessage, didSpeculate,
-          specEstimator.speculativeEstimator);
-      Assert
-          .assertEquals("Failed maps higher than 0 " + estimatorClass.getName(),
-              0, counters.findCounter(JobCounter.NUM_FAILED_MAPS).getValue());
+      assertEquals(didSpeculate,
+          specEstimator.speculativeEstimator, errorMessage);
+      assertEquals(0, counters.findCounter(JobCounter.NUM_FAILED_MAPS).getValue(),
+          "Failed maps higher than 0 " + estimatorClass.getName());
     }
   }
 
 
-  @Test
-  public void testExecSlowNonSpeculative() throws Exception {
+  @ParameterizedTest(name = "{index}: TaskEstimator(EstimatorClass {0})")
+  @MethodSource("getTestParameters")
+  public void testExecSlowNonSpeculative(
+      Class<? extends TaskRuntimeEstimator> pEstimatorKlass,
+      List<String> pTestToIgnore, Integer pNumMapper, Integer pNumReduce) throws Exception {
+    initTestSpeculativeExecOnCluster(pEstimatorKlass, pTestToIgnore, pNumMapper, pNumReduce);
     /*------------------------------------------------------------------
      * Test that Map/Red does not speculate because:
      * 1- all tasks have same progress rate except for task_0
@@ -702,26 +703,27 @@ public class TestSpeculativeExecOnCluster {
       Job job = runSpecTest();
 
       boolean succeeded = job.waitForCompletion(true);
-      Assert.assertTrue(
-          "Job expected to succeed with estimator " + estimatorClass.getName(),
-          succeeded);
-      Assert.assertEquals(
-          "Job expected to succeed with estimator " + estimatorClass.getName(),
-          JobStatus.State.SUCCEEDED, job.getJobState());
+      assertTrue(succeeded,
+          "Job expected to succeed with estimator " + estimatorClass.getName());
+      assertEquals(JobStatus.State.SUCCEEDED, job.getJobState(),
+          "Job expected to succeed with estimator " + estimatorClass.getName());
       Counters counters = job.getCounters();
 
       String errorMessage = specEstimator.getErrorMessage(counters);
       boolean didSpeculate = specEstimator.didSpeculate(counters);
-      Assert.assertEquals(errorMessage, didSpeculate,
-          specEstimator.speculativeEstimator);
-      Assert
-          .assertEquals("Failed maps higher than 0 " + estimatorClass.getName(),
-              0, counters.findCounter(JobCounter.NUM_FAILED_MAPS).getValue());
+      assertEquals(didSpeculate,
+          specEstimator.speculativeEstimator, errorMessage);
+      assertEquals(0, counters.findCounter(JobCounter.NUM_FAILED_MAPS).getValue(),
+          "Failed maps higher than 0 " + estimatorClass.getName());
     }
   }
 
-  @Test
-  public void testExecStepStalledSpeculative() throws Exception {
+  @ParameterizedTest(name = "{index}: TaskEstimator(EstimatorClass {0})")
+  @MethodSource("getTestParameters")
+  public void testExecStepStalledSpeculative(
+      Class<? extends TaskRuntimeEstimator> pEstimatorKlass,
+      List<String> pTestToIgnore, Integer pNumMapper, Integer pNumReduce) throws Exception {
+    initTestSpeculativeExecOnCluster(pEstimatorKlass, pTestToIgnore, pNumMapper, pNumReduce);
     /*------------------------------------------------------------------
      * Test that Map/Red speculates because:
      * 1- all tasks have same progress rate except for task_0
@@ -758,26 +760,30 @@ public class TestSpeculativeExecOnCluster {
       Job job = runSpecTest();
 
       boolean succeeded = job.waitForCompletion(true);
-      Assert.assertTrue("Job expected to succeed with estimator "
-          + estimatorClass.getName(), succeeded);
-      Assert.assertEquals("Job expected to succeed with estimator "
-              + estimatorClass.getName(), JobStatus.State.SUCCEEDED,
-          job.getJobState());
+      assertTrue(succeeded, "Job expected to succeed with estimator "
+          + estimatorClass.getName());
+      assertEquals(JobStatus.State.SUCCEEDED,
+          job.getJobState(), "Job expected to succeed with estimator "
+          + estimatorClass.getName());
       Counters counters = job.getCounters();
 
       String errorMessage = specEstimator.getErrorMessage(counters);
       boolean didSpeculate = specEstimator.didSpeculate(counters);
-      Assert.assertEquals(errorMessage, didSpeculate,
-          specEstimator.speculativeEstimator);
-      Assert.assertEquals("Failed maps higher than 0 "
-              + estimatorClass.getName(), 0,
+      assertEquals(didSpeculate,
+          specEstimator.speculativeEstimator, errorMessage);
+      assertEquals(0,
           counters.findCounter(JobCounter.NUM_FAILED_MAPS)
-              .getValue());
+          .getValue(), "Failed maps higher than 0 "
+          + estimatorClass.getName());
     }
   }
 
-  @Test
-  public void testExecStalledSpeculative() throws Exception {
+  @ParameterizedTest(name = "{index}: TaskEstimator(EstimatorClass {0})")
+  @MethodSource("getTestParameters")
+  public void testExecStalledSpeculative(
+      Class<? extends TaskRuntimeEstimator> pEstimatorKlass,
+      List<String> pTestToIgnore, Integer pNumMapper, Integer pNumReduce) throws Exception {
+    initTestSpeculativeExecOnCluster(pEstimatorKlass, pTestToIgnore, pNumMapper, pNumReduce);
     /*------------------------------------------------------------------
      * Test that Map/Red speculates because:
      * 1- all tasks have same progress rate except for task_0
@@ -815,26 +821,29 @@ public class TestSpeculativeExecOnCluster {
       Job job = runSpecTest();
 
       boolean succeeded = job.waitForCompletion(true);
-      Assert.assertTrue("Job expected to succeed with estimator "
-          + estimatorClass.getName(), succeeded);
-      Assert.assertEquals("Job expected to succeed with estimator "
-              + estimatorClass.getName(), JobStatus.State.SUCCEEDED,
-          job.getJobState());
+      assertTrue(succeeded, "Job expected to succeed with estimator "
+          + estimatorClass.getName());
+      assertEquals(JobStatus.State.SUCCEEDED,
+          job.getJobState(), "Job expected to succeed with estimator "
+          + estimatorClass.getName());
       Counters counters = job.getCounters();
 
       String errorMessage = specEstimator.getErrorMessage(counters);
       boolean didSpeculate = specEstimator.didSpeculate(counters);
-      Assert.assertEquals(errorMessage, didSpeculate,
-          specEstimator.speculativeEstimator);
-      Assert.assertEquals("Failed maps higher than 0 "
-              + estimatorClass.getName(), 0,
-          counters.findCounter(JobCounter.NUM_FAILED_MAPS)
-              .getValue());
+      assertEquals(didSpeculate,
+          specEstimator.speculativeEstimator, errorMessage);
+      assertEquals(0, counters.findCounter(JobCounter.NUM_FAILED_MAPS)
+          .getValue(), "Failed maps higher than 0 "
+          + estimatorClass.getName());
     }
   }
 
-  @Test
-  public void testExecNonSpeculative() throws Exception {
+  @ParameterizedTest(name = "{index}: TaskEstimator(EstimatorClass {0})")
+  @MethodSource("getTestParameters")
+  public void testExecNonSpeculative(
+      Class<? extends TaskRuntimeEstimator> pEstimatorKlass,
+      List<String> pTestToIgnore, Integer pNumMapper, Integer pNumReduce) throws Exception {
+    initTestSpeculativeExecOnCluster(pEstimatorKlass, pTestToIgnore, pNumMapper, pNumReduce);
     /*------------------------------------------------------------------
      * Test that Map/Red does not speculate because all tasks progress in the
      *    same rate.
@@ -874,17 +883,17 @@ public class TestSpeculativeExecOnCluster {
       Job job = runSpecTest();
 
       boolean succeeded = job.waitForCompletion(true);
-      Assert.assertTrue("Job expected to succeed with estimator "
-          + estimatorClass.getName(), succeeded);
-      Assert.assertEquals("Job expected to succeed with estimator "
-              + estimatorClass.getName(), JobStatus.State.SUCCEEDED,
-          job.getJobState());
+      assertTrue(succeeded, "Job expected to succeed with estimator "
+          + estimatorClass.getName());
+      assertEquals(JobStatus.State.SUCCEEDED,
+          job.getJobState(), "Job expected to succeed with estimator "
+          + estimatorClass.getName());
       Counters counters = job.getCounters();
 
       String errorMessage = specEstimator.getErrorMessage(counters);
       boolean didSpeculate = specEstimator.didSpeculate(counters);
-      Assert.assertEquals(errorMessage, didSpeculate,
-          specEstimator.speculativeEstimator);
+      assertEquals(didSpeculate,
+          specEstimator.speculativeEstimator, errorMessage);
     }
   }
 

+ 37 - 35
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestSpeculativeExecution.java

@@ -42,13 +42,15 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
 import org.apache.hadoop.mapreduce.v2.app.speculate.LegacyTaskRuntimeEstimator;
 import org.apache.hadoop.mapreduce.v2.app.speculate.TaskRuntimeEstimator;
-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.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
 public class TestSpeculativeExecution {
 
   /*
@@ -103,7 +105,7 @@ public class TestSpeculativeExecution {
   static Path APP_JAR = new Path(TEST_ROOT_DIR, "MRAppJar.jar");
   private static Path TEST_OUT_DIR = new Path(TEST_ROOT_DIR, "test.out.dir");
 
-  @BeforeClass
+  @BeforeAll
   public static void setup() throws IOException {
 
     if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {
@@ -124,7 +126,7 @@ public class TestSpeculativeExecution {
     localFs.setPermission(APP_JAR, new FsPermission("700"));
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() {
     if (mrCluster != null) {
       mrCluster.stop();
@@ -248,15 +250,15 @@ public class TestSpeculativeExecution {
     Job job = runSpecTest(false, false);
 
     boolean succeeded = job.waitForCompletion(true);
-    Assert.assertTrue(succeeded);
-    Assert.assertEquals(JobStatus.State.SUCCEEDED, job.getJobState());
+    assertTrue(succeeded);
+    assertEquals(JobStatus.State.SUCCEEDED, job.getJobState());
     Counters counters = job.getCounters();
-    Assert.assertEquals(2, counters.findCounter(JobCounter.TOTAL_LAUNCHED_MAPS)
-            .getValue());
-    Assert.assertEquals(2, counters.findCounter(JobCounter.TOTAL_LAUNCHED_REDUCES)
-            .getValue());
-    Assert.assertEquals(0, counters.findCounter(JobCounter.NUM_FAILED_MAPS)
-            .getValue());
+    assertEquals(2, counters.findCounter(JobCounter.TOTAL_LAUNCHED_MAPS)
+        .getValue());
+    assertEquals(2, counters.findCounter(JobCounter.TOTAL_LAUNCHED_REDUCES)
+        .getValue());
+    assertEquals(0, counters.findCounter(JobCounter.NUM_FAILED_MAPS)
+        .getValue());
 
 
     /*------------------------------------------------------------------
@@ -268,18 +270,18 @@ public class TestSpeculativeExecution {
     job = runNonSpecFailOnceTest();
 
     succeeded = job.waitForCompletion(true);
-    Assert.assertTrue(succeeded);
-    Assert.assertEquals(JobStatus.State.SUCCEEDED, job.getJobState());
+    assertTrue(succeeded);
+    assertEquals(JobStatus.State.SUCCEEDED, job.getJobState());
     counters = job.getCounters();
     // We will have 4 total since 2 map tasks fail and relaunch attempt once
-    Assert.assertEquals(4,
+    assertEquals(4,
         counters.findCounter(JobCounter.TOTAL_LAUNCHED_MAPS).getValue());
-    Assert.assertEquals(4,
+    assertEquals(4,
         counters.findCounter(JobCounter.TOTAL_LAUNCHED_REDUCES).getValue());
     // Ensure no maps or reduces killed due to accidental speculation
-    Assert.assertEquals(0,
+    assertEquals(0,
         counters.findCounter(JobCounter.NUM_KILLED_MAPS).getValue());
-    Assert.assertEquals(0,
+    assertEquals(0,
         counters.findCounter(JobCounter.NUM_KILLED_REDUCES).getValue());
 
     /*----------------------------------------------------------------------
@@ -290,18 +292,18 @@ public class TestSpeculativeExecution {
     job = runSpecTest(true, false);
 
     succeeded = job.waitForCompletion(true);
-    Assert.assertTrue(succeeded);
-    Assert.assertEquals(JobStatus.State.SUCCEEDED, job.getJobState());
+    assertTrue(succeeded);
+    assertEquals(JobStatus.State.SUCCEEDED, job.getJobState());
     counters = job.getCounters();
 
     // The long-running map will be killed and a new one started.
-    Assert.assertEquals(3, counters.findCounter(JobCounter.TOTAL_LAUNCHED_MAPS)
-            .getValue());
-    Assert.assertEquals(2, counters.findCounter(JobCounter.TOTAL_LAUNCHED_REDUCES)
-            .getValue());
-    Assert.assertEquals(0, counters.findCounter(JobCounter.NUM_FAILED_MAPS)
-            .getValue());
-    Assert.assertEquals(1, counters.findCounter(JobCounter.NUM_KILLED_MAPS)
+    assertEquals(3, counters.findCounter(JobCounter.TOTAL_LAUNCHED_MAPS)
+        .getValue());
+    assertEquals(2, counters.findCounter(JobCounter.TOTAL_LAUNCHED_REDUCES)
+        .getValue());
+    assertEquals(0, counters.findCounter(JobCounter.NUM_FAILED_MAPS)
+        .getValue());
+    assertEquals(1, counters.findCounter(JobCounter.NUM_KILLED_MAPS)
         .getValue());
 
     /*----------------------------------------------------------------------
@@ -312,15 +314,15 @@ public class TestSpeculativeExecution {
     job = runSpecTest(false, true);
 
     succeeded = job.waitForCompletion(true);
-    Assert.assertTrue(succeeded);
-    Assert.assertEquals(JobStatus.State.SUCCEEDED, job.getJobState());
+    assertTrue(succeeded);
+    assertEquals(JobStatus.State.SUCCEEDED, job.getJobState());
     counters = job.getCounters();
 
     // The long-running map will be killed and a new one started.
-    Assert.assertEquals(2, counters.findCounter(JobCounter.TOTAL_LAUNCHED_MAPS)
-            .getValue());
-    Assert.assertEquals(3, counters.findCounter(JobCounter.TOTAL_LAUNCHED_REDUCES)
-            .getValue());
+    assertEquals(2, counters.findCounter(JobCounter.TOTAL_LAUNCHED_MAPS)
+        .getValue());
+    assertEquals(3, counters.findCounter(JobCounter.TOTAL_LAUNCHED_REDUCES)
+        .getValue());
   }
 
   private Path createTempFile(String filename, String contents)

+ 26 - 21
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestSpeculativeExecutionWithMRApp.java

@@ -29,7 +29,6 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.v2.app.speculate.LegacyTaskRuntimeEstimator;
 import org.apache.hadoop.mapreduce.v2.app.speculate.SimpleExponentialTaskRuntimeEstimator;
 import org.apache.hadoop.mapreduce.v2.app.speculate.TaskRuntimeEstimator;
-import org.junit.Assert;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.v2.api.records.JobState;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
@@ -48,18 +47,17 @@ import org.apache.hadoop.service.Service;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.util.ControlledClock;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.Timeout;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 /**
  * The type Test speculative execution with mr app.
  * It test the speculation behavior given a list of estimator classes.
  */
 @SuppressWarnings({ "unchecked", "rawtypes" })
-@RunWith(Parameterized.class)
 public class TestSpeculativeExecutionWithMRApp {
   private static final int NUM_MAPPERS = 5;
   private static final int NUM_REDUCERS = 0;
@@ -69,7 +67,6 @@ public class TestSpeculativeExecutionWithMRApp {
    *
    * @return the test parameters
    */
-  @Parameterized.Parameters(name = "{index}: TaskEstimator(EstimatorClass {0})")
   public static Collection<Object[]> getTestParameters() {
     return Arrays.asList(new Object[][] {
         {SimpleExponentialTaskRuntimeEstimator.class},
@@ -79,19 +76,19 @@ public class TestSpeculativeExecutionWithMRApp {
 
   private Class<? extends TaskRuntimeEstimator> estimatorClass;
 
-  private final ControlledClock controlledClk;
+  private ControlledClock controlledClk;
   /**
    * Instantiates a new Test speculative execution with mr app.
    *
-   * @param estimatorKlass the estimator klass
+   * @param pEstimatorKlass the estimator klass
    */
-  public TestSpeculativeExecutionWithMRApp(
-      Class<? extends TaskRuntimeEstimator>  estimatorKlass) {
-    this.estimatorClass = estimatorKlass;
+  public void initTestSpeculativeExecutionWithMRApp(
+      Class<? extends TaskRuntimeEstimator> pEstimatorKlass) {
+    this.estimatorClass = pEstimatorKlass;
     this.controlledClk = new ControlledClock();
+    setup();
   }
 
-  @Before
   public void setup() {
     this.controlledClk.setTime(System.currentTimeMillis());
   }
@@ -101,8 +98,12 @@ public class TestSpeculativeExecutionWithMRApp {
    *
    * @throws Exception the exception
    */
-  @Test (timeout = 360000)
-  public void testSpeculateSuccessfulWithoutUpdateEvents() throws Exception {
+  @ParameterizedTest(name = "{index}: TaskEstimator(EstimatorClass {0})")
+  @MethodSource("getTestParameters")
+  @Timeout(value = 360)
+  public void testSpeculateSuccessfulWithoutUpdateEvents(
+      Class<? extends TaskRuntimeEstimator> pEstimatorKlass) throws Exception {
+    initTestSpeculativeExecutionWithMRApp(pEstimatorKlass);
     MRApp app =
         new MRApp(NUM_MAPPERS, NUM_REDUCERS, false, "test", true,
             controlledClk);
@@ -110,8 +111,8 @@ public class TestSpeculativeExecutionWithMRApp {
     app.waitForState(job, JobState.RUNNING);
 
     Map<TaskId, Task> tasks = job.getTasks();
-    Assert.assertEquals("Num tasks is not correct", NUM_MAPPERS + NUM_REDUCERS,
-      tasks.size());
+    assertEquals(NUM_MAPPERS + NUM_REDUCERS,
+        tasks.size(), "Num tasks is not correct");
     Iterator<Task> taskIter = tasks.values().iterator();
     while (taskIter.hasNext()) {
       app.waitForState(taskIter.next(), TaskState.RUNNING);
@@ -161,8 +162,12 @@ public class TestSpeculativeExecutionWithMRApp {
    *
    * @throws Exception the exception
    */
-  @Test (timeout = 360000)
-  public void testSpeculateSuccessfulWithUpdateEvents() throws Exception {
+  @ParameterizedTest(name = "{index}: TaskEstimator(EstimatorClass {0})")
+  @MethodSource("getTestParameters")
+  @Timeout(value = 360)
+  public void testSpeculateSuccessfulWithUpdateEvents(
+      Class<? extends TaskRuntimeEstimator> pEstimatorKlass) throws Exception {
+    initTestSpeculativeExecutionWithMRApp(pEstimatorKlass);
     MRApp app =
         new MRApp(NUM_MAPPERS, NUM_REDUCERS, false, "test", true,
             controlledClk);
@@ -170,8 +175,8 @@ public class TestSpeculativeExecutionWithMRApp {
     app.waitForState(job, JobState.RUNNING);
 
     Map<TaskId, Task> tasks = job.getTasks();
-    Assert.assertEquals("Num tasks is not correct", NUM_MAPPERS + NUM_REDUCERS,
-      tasks.size());
+    assertEquals(NUM_MAPPERS + NUM_REDUCERS,
+        tasks.size(), "Num tasks is not correct");
     Iterator<Task> taskIter = tasks.values().iterator();
     while (taskIter.hasNext()) {
       app.waitForState(taskIter.next(), TaskState.RUNNING);

+ 16 - 15
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestUberAM.java

@@ -30,19 +30,20 @@ import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskCompletionEvent;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class TestUberAM extends TestMRJobs {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestUberAM.class);
 
-  @BeforeClass
+  @BeforeAll
   public static void setup() throws IOException {
     TestMRJobs.setup();
     if (mrCluster != null) {
@@ -71,11 +72,11 @@ public class TestUberAM extends TestMRJobs {
       IOException {
     Counters counters = job.getCounters();
     super.verifySleepJobCounters(job);
-    Assert.assertEquals(3,
+    assertEquals(3,
         counters.findCounter(JobCounter.NUM_UBER_SUBMAPS).getValue());
-    Assert.assertEquals(numSleepReducers,
+    assertEquals(numSleepReducers,
         counters.findCounter(JobCounter.NUM_UBER_SUBREDUCES).getValue());
-    Assert.assertEquals(3 + numSleepReducers,
+    assertEquals(3 + numSleepReducers,
         counters.findCounter(JobCounter.TOTAL_LAUNCHED_UBERTASKS).getValue());
   }
 
@@ -91,9 +92,9 @@ public class TestUberAM extends TestMRJobs {
       throws InterruptedException, IOException {
     super.verifyRandomWriterCounters(job);
     Counters counters = job.getCounters();
-    Assert.assertEquals(3, counters.findCounter(JobCounter.NUM_UBER_SUBMAPS)
+    assertEquals(3, counters.findCounter(JobCounter.NUM_UBER_SUBMAPS)
         .getValue());
-    Assert.assertEquals(3,
+    assertEquals(3,
         counters.findCounter(JobCounter.TOTAL_LAUNCHED_UBERTASKS).getValue());
   }
 
@@ -132,12 +133,12 @@ public class TestUberAM extends TestMRJobs {
     assertThat(secondTaskAttemptExists).isFalse();
 
     TaskCompletionEvent[] events = job.getTaskCompletionEvents(0, 2);
-    Assert.assertEquals(1, events.length);
+    assertEquals(1, events.length);
     // TIPFAILED if it comes from the AM, FAILED if it comes from the JHS
     TaskCompletionEvent.Status status = events[0].getStatus();
-    Assert.assertTrue(status == TaskCompletionEvent.Status.FAILED ||
+    assertTrue(status == TaskCompletionEvent.Status.FAILED ||
         status == TaskCompletionEvent.Status.TIPFAILED);
-    Assert.assertEquals(JobStatus.State.FAILED, job.getJobState());
+    assertEquals(JobStatus.State.FAILED, job.getJobState());
     
     //Disabling till UberAM honors MRJobConfig.MAP_MAX_ATTEMPTS
     //verifyFailingMapperCounters(job);
@@ -150,11 +151,11 @@ public class TestUberAM extends TestMRJobs {
       throws InterruptedException, IOException {
     Counters counters = job.getCounters();
     super.verifyFailingMapperCounters(job);
-    Assert.assertEquals(2,
+    assertEquals(2,
         counters.findCounter(JobCounter.TOTAL_LAUNCHED_UBERTASKS).getValue());
-    Assert.assertEquals(2, counters.findCounter(JobCounter.NUM_UBER_SUBMAPS)
+    assertEquals(2, counters.findCounter(JobCounter.NUM_UBER_SUBMAPS)
         .getValue());
-    Assert.assertEquals(2, counters
+    assertEquals(2, counters
         .findCounter(JobCounter.NUM_FAILED_UBERTASKS).getValue());
   }
 

+ 5 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/util/TestMRCJCReflectionUtils.java

@@ -22,16 +22,17 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobConfigurable;
 
-import static org.junit.Assert.*;
-import org.junit.Before;
-import org.junit.Test;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 /**
  * Test for the JobConf-related parts of common's ReflectionUtils
  * class.
  */
 public class TestMRCJCReflectionUtils {
-  @Before
+  @BeforeEach
   public void setUp() {
     ReflectionUtils.clearCache();
   }

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/util/TestMRCJCRunJar.java

@@ -27,8 +27,8 @@ import java.util.jar.JarOutputStream;
 import java.util.zip.ZipEntry;
 
 import org.apache.hadoop.fs.Path;
-import org.junit.Assert;
-import org.junit.Test;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import org.junit.jupiter.api.Test;
 
 /**
  * A test to rest the RunJar class.
@@ -55,7 +55,7 @@ public class TestMRCJCRunJar {
     args[1] = "org.apache.hadoop.util.Hello";
     args[2] = outFile.toString();
     RunJar.main(args);
-    Assert.assertTrue("RunJar failed", outFile.exists());
+    assertTrue(outFile.exists(), "RunJar failed");
   }
 
   private File makeTestJar() throws IOException {