浏览代码

HADOOP-19436. [JDK17] Upgrade JUnit from 4 to 5 in hadoop-gridmix. (#7578)

* HADOOP-19436. [JDK17] Upgrade JUnit from 4 to 5 in hadoop-gridmix.

Co-authored-by: Chris Nauroth <cnauroth@apache.org>
Reviewed-by: Chris Nauroth <cnauroth@apache.org>
Signed-off-by: Shilun Fan <slfan1989@apache.org>
slfan1989 3 周之前
父节点
当前提交
a38294da09
共有 19 个文件被更改,包括 413 次插入367 次删除
  1. 28 35
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/CommonJobTest.java
  2. 13 11
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestCompressionEmulationUtils.java
  3. 56 52
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestDistCacheEmulation.java
  4. 12 8
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestFilePool.java
  5. 7 6
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestFileQueue.java
  6. 33 18
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestGridMixClasses.java
  7. 37 34
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestGridmixMemoryEmulation.java
  8. 7 4
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestGridmixRecord.java
  9. 19 12
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestGridmixSubmission.java
  10. 81 88
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestGridmixSummary.java
  11. 10 8
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestHighRamJob.java
  12. 10 7
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestLoadJob.java
  13. 31 24
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestPseudoLocalFs.java
  14. 2 2
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestRandomAlgorithm.java
  15. 9 7
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestRandomTextDataGenerator.java
  16. 3 2
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestRecordFactory.java
  17. 40 36
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestResourceUsageEmulators.java
  18. 6 6
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestSleepJob.java
  19. 9 7
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestUserResolve.java

+ 28 - 35
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/CommonJobTest.java

@@ -17,10 +17,10 @@
  */
 package org.apache.hadoop.mapred.gridmix;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.File;
 import java.io.IOException;
@@ -114,10 +114,10 @@ public class CommonJobTest {
     }
 
     public void verify(ArrayList<JobStory> submitted) throws Exception {
-      assertEquals("Bad job count", expected, retiredJobs.size());
+      assertEquals(expected, retiredJobs.size(), "Bad job count");
 
       final ArrayList<Job> succeeded = new ArrayList<Job>();
-      assertEquals("Bad job count", expected, retiredJobs.drainTo(succeeded));
+      assertEquals(expected, retiredJobs.drainTo(succeeded), "Bad job count");
       final HashMap<String, JobStory> sub = new HashMap<String, JobStory>();
       for (JobStory spec : submitted) {
         sub.put(spec.getJobID().toString(), spec);
@@ -152,13 +152,13 @@ public class CommonJobTest {
 
         final String originalJobId = configuration.get(Gridmix.ORIGINAL_JOB_ID);
         final JobStory spec = sub.get(originalJobId);
-        assertNotNull("No spec for " + jobName, spec);
-        assertNotNull("No counters for " + jobName, job.getCounters());
+        assertNotNull(spec, "No spec for " + jobName);
+        assertNotNull(job.getCounters(), "No counters for " + jobName);
         final String originalJobName = spec.getName();
         System.out.println("originalJobName=" + originalJobName
                 + ";GridmixJobName=" + jobName + ";originalJobID=" + originalJobId);
-        assertTrue("Original job name is wrong.",
-                originalJobName.equals(configuration.get(Gridmix.ORIGINAL_JOB_NAME)));
+        assertTrue(originalJobName.equals(configuration.get(Gridmix.ORIGINAL_JOB_NAME)),
+            "Original job name is wrong.");
 
         // Gridmix job seqNum contains 6 digits
         int seqNumLength = 6;
@@ -169,12 +169,11 @@ public class CommonJobTest {
         assertTrue(originalJobName.substring(
                 originalJobName.length() - seqNumLength).equals(jobSeqNum));
 
-        assertTrue("Gridmix job name is not in the expected format.",
-                jobName.equals(GridmixJob.JOB_NAME_PREFIX + jobSeqNum));
+        assertTrue(jobName.equals(GridmixJob.JOB_NAME_PREFIX + jobSeqNum),
+            "Gridmix job name is not in the expected format.");
         final FileStatus stat = GridmixTestUtils.dfs.getFileStatus(new Path(
                 GridmixTestUtils.DEST, "" + Integer.parseInt(jobSeqNum)));
-        assertEquals("Wrong owner for " + jobName, spec.getUser(),
-                stat.getOwner());
+        assertEquals(spec.getUser(), stat.getOwner(), "Wrong owner for " + jobName);
         final int nMaps = spec.getNumberMaps();
         final int nReds = spec.getNumberReduces();
 
@@ -182,12 +181,12 @@ public class CommonJobTest {
                 GridmixTestUtils.mrvl.getConfig());
         final TaskReport[] mReports = client.getMapTaskReports(JobID
                 .downgrade(job.getJobID()));
-        assertEquals("Mismatched map count", nMaps, mReports.length);
+        assertEquals(nMaps, mReports.length, "Mismatched map count");
         check(TaskType.MAP, spec, mReports, 0, 0, SLOPBYTES, nReds);
 
         final TaskReport[] rReports = client.getReduceTaskReports(JobID
                 .downgrade(job.getJobID()));
-        assertEquals("Mismatched reduce count", nReds, rReports.length);
+        assertEquals(nReds, rReports.length, "Mismatched reduce count");
         check(TaskType.REDUCE, spec, rReports, nMaps * SLOPBYTES, 2 * nMaps, 0,
                 0);
 
@@ -273,43 +272,37 @@ public class CommonJobTest {
       Arrays.sort(specInputBytes);
       Arrays.sort(runInputBytes);
       for (int i = 0; i < runTasks.length; ++i) {
-        assertTrue("Mismatched " + type + " input bytes " + specInputBytes[i]
-                + "/" + runInputBytes[i],
-                eqPlusMinus(runInputBytes[i], specInputBytes[i], extraInputBytes));
+        assertTrue(eqPlusMinus(runInputBytes[i], specInputBytes[i], extraInputBytes),
+            "Mismatched " + type + " input bytes " + specInputBytes[i]
+            + "/" + runInputBytes[i]);
       }
 
       // Check input records
       Arrays.sort(specInputRecords);
       Arrays.sort(runInputRecords);
       for (int i = 0; i < runTasks.length; ++i) {
-        assertTrue(
-                "Mismatched " + type + " input records " + specInputRecords[i]
-                        + "/" + runInputRecords[i],
-                eqPlusMinus(runInputRecords[i], specInputRecords[i],
-                        extraInputRecords));
+        assertTrue(eqPlusMinus(runInputRecords[i], specInputRecords[i],
+            extraInputRecords), "Mismatched " + type + " input records " + specInputRecords[i]
+            + "/" + runInputRecords[i]);
       }
 
       // Check output bytes
       Arrays.sort(specOutputBytes);
       Arrays.sort(runOutputBytes);
       for (int i = 0; i < runTasks.length; ++i) {
-        assertTrue(
-                "Mismatched " + type + " output bytes " + specOutputBytes[i] + "/"
-                        + runOutputBytes[i],
-                eqPlusMinus(runOutputBytes[i], specOutputBytes[i], extraOutputBytes));
+        assertTrue(eqPlusMinus(runOutputBytes[i], specOutputBytes[i], extraOutputBytes),
+            "Mismatched " + type + " output bytes " + specOutputBytes[i] + "/"
+            + runOutputBytes[i]);
       }
 
       // Check output records
       Arrays.sort(specOutputRecords);
       Arrays.sort(runOutputRecords);
       for (int i = 0; i < runTasks.length; ++i) {
-        assertTrue(
-                "Mismatched " + type + " output records " + specOutputRecords[i]
-                        + "/" + runOutputRecords[i],
-                eqPlusMinus(runOutputRecords[i], specOutputRecords[i],
-                        extraOutputRecords));
+        assertTrue(eqPlusMinus(runOutputRecords[i], specOutputRecords[i],
+            extraOutputRecords), "Mismatched " + type + " output records " + specOutputRecords[i]
+            + "/" + runOutputRecords[i]);
       }
-
     }
 
     private static boolean eqPlusMinus(long a, long b, long x) {
@@ -372,7 +365,7 @@ public class CommonJobTest {
       GridmixTestUtils.dfs.setPermission(root, new FsPermission((short) 777));
 
       int res = ToolRunner.run(conf, client, argv);
-      assertEquals("Client exited with nonzero status", 0, res);
+      assertEquals(0, res, "Client exited with nonzero status");
       client.checkMonitor();
     } catch (Exception e) {
       e.printStackTrace();

+ 13 - 11
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestCompressionEmulationUtils.java

@@ -47,8 +47,10 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 
-import static org.junit.Assert.*;
-import org.junit.Test;
+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 org.junit.jupiter.api.Test;
 
 /**
  * Test {@link CompressionEmulationUtil}
@@ -169,7 +171,7 @@ public class TestCompressionEmulationUtils {
     job.submit();
     int ret = job.waitForCompletion(true) ? 0 : 1;
 
-    assertEquals("Job Failed", 0, ret);
+    assertEquals(0, ret, "Job Failed");
   }
   
   /**
@@ -260,7 +262,7 @@ public class TestCompressionEmulationUtils {
     } catch (RuntimeException re) {
       failed = true;
     }
-    assertTrue("Compression ratio min value (0.07) check failed!", failed);
+    assertTrue(failed, "Compression ratio min value (0.07) check failed!");
     
     // test with a compression ratio of 0.01 which less than the max supported
     // value of 0.68
@@ -270,7 +272,7 @@ public class TestCompressionEmulationUtils {
     } catch (RuntimeException re) {
       failed = true;
     }
-    assertTrue("Compression ratio max value (0.68) check failed!", failed);
+    assertTrue(failed, "Compression ratio max value (0.68) check failed!");
   }
   
   /**
@@ -380,10 +382,10 @@ public class TestCompressionEmulationUtils {
     GridmixRecord recordRead = new GridmixRecord();
     recordRead.readFields(new DataInputStream(in));
     
-    assertEquals("Record size mismatch in a compressible GridmixRecord",
-                 dataSize, recordRead.getSize());
-    assertTrue("Failed to generate a compressible GridmixRecord",
-               recordRead.getSize() > compressedFileSize);
+    assertEquals(dataSize, recordRead.getSize(),
+        "Record size mismatch in a compressible GridmixRecord");
+    assertTrue(recordRead.getSize() > compressedFileSize,
+        "Failed to generate a compressible GridmixRecord");
     
     // check if the record can generate data with the desired compression ratio
     float seenRatio = ((float)compressedFileSize)/dataSize;
@@ -456,7 +458,7 @@ public class TestCompressionEmulationUtils {
         .getPossiblyDecompressedInputStream(compressedFile, conf, 0);
     BufferedReader reader = new BufferedReader(new InputStreamReader(in));
     String readLine = reader.readLine();
-    assertEquals("Compression/Decompression error", inputLine, readLine);
+    assertEquals(inputLine, readLine, "Compression/Decompression error");
     reader.close();
   }
   
@@ -555,7 +557,7 @@ public class TestCompressionEmulationUtils {
     queue.read(bytes);
     queue.close();
     String readLine = new String(bytes);
-    assertEquals("Compression/Decompression error", inputLine, readLine);
+    assertEquals(inputLine, readLine, "Compression/Decompression error");
   }
 
   /**

+ 56 - 52
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestDistCacheEmulation.java

@@ -17,7 +17,9 @@
  */
 package org.apache.hadoop.mapred.gridmix;
 
-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;
 
 import java.io.File;
 import java.io.FileNotFoundException;
@@ -46,9 +48,10 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.lib.input.FileSplit;
 import org.apache.hadoop.mapreduce.task.MapContextImpl;
 import org.apache.hadoop.security.UserGroupInformation;
-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 org.junit.jupiter.api.Timeout;
 
 /**
  * Validate emulation of distributed cache load in gridmix simulated jobs.
@@ -58,7 +61,7 @@ public class TestDistCacheEmulation {
 
   private DistributedCacheEmulator dce = null;
 
-  @BeforeClass
+  @BeforeAll
   public static void init() throws IOException {
     GridmixTestUtils.initCluster(TestDistCacheEmulation.class);
     File target=new File("target"+File.separator+TestDistCacheEmulation.class.getName());
@@ -68,7 +71,7 @@ public class TestDistCacheEmulation {
     
   }
 
-  @AfterClass
+  @AfterAll
   public static void shutDown() throws IOException {
     GridmixTestUtils.shutdownCluster();
   }
@@ -93,13 +96,13 @@ public class TestDistCacheEmulation {
     // Validate the existence of Distributed Cache files list file directly
     // under distributed cache directory
     Path listFile = new Path(filesListFile);
-    assertTrue("Path of Distributed Cache files list file is wrong.",
-        distCachePath.equals(listFile.getParent().makeQualified(fs.getUri(), fs.getWorkingDirectory())));
+    assertTrue(distCachePath.equals(
+        listFile.getParent().makeQualified(fs.getUri(), fs.getWorkingDirectory())),
+        "Path of Distributed Cache files list file is wrong.");
 
     // Delete the dist cache files list file
-    assertTrue(
-        "Failed to delete distributed Cache files list file " + listFile,
-        fs.delete(listFile,true));
+    assertTrue(fs.delete(listFile, true),
+        "Failed to delete distributed Cache files list file " + listFile);
 
     List<Long> fileSizes = new ArrayList<Long>();
     for (long size : sortedFileSizes) {
@@ -124,18 +127,17 @@ public class TestDistCacheEmulation {
     // RemoteIterator<LocatedFileStatus> iter =
     FileStatus[] statuses = GridmixTestUtils.dfs.listStatus(distCacheDir);
     int numFiles = filesSizesExpected.size();
-    assertEquals("Number of files under distributed cache dir is wrong.",
-        numFiles, statuses.length);
+    assertEquals(numFiles, statuses.length,
+        "Number of files under distributed cache dir is wrong.");
     for (int i = 0; i < numFiles; i++) {
       FileStatus stat = statuses[i];
-      assertTrue("File size of distributed cache file "
-          + stat.getPath().toUri().getPath() + " is wrong.",
-          filesSizesExpected.remove(stat.getLen()));
+      assertTrue(filesSizesExpected.remove(stat.getLen()), "File size of distributed cache file "
+          + stat.getPath().toUri().getPath() + " is wrong.");
 
       FsPermission perm = stat.getPermission();
-      assertEquals("Wrong permissions for distributed cache file "
-          + stat.getPath().toUri().getPath(), new FsPermission(
-          GenerateDistCacheData.GRIDMIX_DISTCACHE_FILE_PERM), perm);
+      assertEquals(new FsPermission(GenerateDistCacheData.GRIDMIX_DISTCACHE_FILE_PERM), perm,
+          "Wrong permissions for distributed cache file "
+          + stat.getPath().toUri().getPath());
     }
   }
 
@@ -214,8 +216,7 @@ public class TestDistCacheEmulation {
     int exitCode = dce.setupGenerateDistCacheData(jobProducer);
     int expectedExitCode = generate ? 0
         : Gridmix.MISSING_DIST_CACHE_FILES_ERROR;
-    assertEquals("setupGenerateDistCacheData failed.", expectedExitCode,
-        exitCode);
+    assertEquals(expectedExitCode, exitCode, "setupGenerateDistCacheData failed.");
 
     // reset back
     resetDistCacheConfigProperties(jobConf);
@@ -247,16 +248,16 @@ public class TestDistCacheEmulation {
    * 
    * @throws Exception
    */
-  @Test (timeout=200000)
+  @Test
+  @Timeout(value = 200)
   public void testGenerateDistCacheData() throws Exception {
     long[] sortedFileSizes = new long[5];
     Configuration jobConf = runSetupGenerateDistCacheData(true, sortedFileSizes);
     GridmixJob gridmixJob = new GenerateDistCacheData(jobConf);
     Job job = gridmixJob.call();
-    assertEquals("Number of reduce tasks in GenerateDistCacheData is not 0.",
-        0, job.getNumReduceTasks());
-    assertTrue("GenerateDistCacheData job failed.",
-        job.waitForCompletion(false));
+    assertEquals(0, job.getNumReduceTasks(),
+        "Number of reduce tasks in GenerateDistCacheData is not 0.");
+    assertTrue(job.waitForCompletion(false), "GenerateDistCacheData job failed.");
     validateDistCacheData(jobConf, sortedFileSizes);
   }
 
@@ -275,17 +276,18 @@ public class TestDistCacheEmulation {
     }
 
     FileSystem fs = FileSystem.get(jobConf);
-    assertEquals("Number of distributed cache files to be generated is wrong.",
-        sortedFileSizes.length,
-        jobConf.getInt(GenerateDistCacheData.GRIDMIX_DISTCACHE_FILE_COUNT, -1));
-    assertEquals("Total size of dist cache files to be generated is wrong.",
-        sumOfFileSizes,
-        jobConf.getLong(GenerateDistCacheData.GRIDMIX_DISTCACHE_BYTE_COUNT, -1));
+    assertEquals(sortedFileSizes.length,
+        jobConf.getInt(GenerateDistCacheData.GRIDMIX_DISTCACHE_FILE_COUNT, -1),
+        "Number of distributed cache files to be generated is wrong.");
+    assertEquals(sumOfFileSizes,
+        jobConf.getLong(GenerateDistCacheData.GRIDMIX_DISTCACHE_BYTE_COUNT, -1),
+        "Total size of dist cache files to be generated is wrong.");
     Path filesListFile = new Path(
         jobConf.get(GenerateDistCacheData.GRIDMIX_DISTCACHE_FILE_LIST));
     FileStatus stat = fs.getFileStatus(filesListFile);
-    assertEquals("Wrong permissions of dist Cache files list file "
-        + filesListFile, new FsPermission((short) 0644), stat.getPermission());
+    assertEquals(new FsPermission((short) 0644), stat.getPermission(),
+        "Wrong permissions of dist Cache files list file "
+        + filesListFile);
 
     InputSplit split = new FileSplit(filesListFile, 0, stat.getLen(),
         (String[]) null);
@@ -313,35 +315,33 @@ public class TestDistCacheEmulation {
 
     // Validate permissions of dist cache directory
     Path distCacheDir = dce.getDistributedCacheDir();
-    assertEquals(
-        "Wrong permissions for distributed cache dir " + distCacheDir,
-        fs.getFileStatus(distCacheDir).getPermission().getOtherAction()
-            .and(FsAction.EXECUTE), FsAction.EXECUTE);
+    assertEquals(fs.getFileStatus(distCacheDir).getPermission().getOtherAction()
+        .and(FsAction.EXECUTE), FsAction.EXECUTE,
+        "Wrong permissions for distributed cache dir " + distCacheDir);
 
     // Validate the content of the sequence file generated by
     // dce.setupGenerateDistCacheData().
     LongWritable key = new LongWritable();
     BytesWritable val = new BytesWritable();
     for (int i = 0; i < sortedFileSizes.length; i++) {
-      assertTrue("Number of files written to the sequence file by "
-          + "setupGenerateDistCacheData is less than the expected.",
-          reader.nextKeyValue());
+      assertTrue(reader.nextKeyValue(),
+          "Number of files written to the sequence file by "
+          + "setupGenerateDistCacheData is less than the expected.");
       key = reader.getCurrentKey();
       val = reader.getCurrentValue();
       long fileSize = key.get();
       String file = new String(val.getBytes(), 0, val.getLength());
 
       // Dist Cache files should be sorted based on file size.
-      assertEquals("Dist cache file size is wrong.", sortedFileSizes[i],
-          fileSize);
+      assertEquals(sortedFileSizes[i], fileSize, "Dist cache file size is wrong.");
 
       // Validate dist cache file path.
 
       // parent dir of dist cache file
       Path parent = new Path(file).getParent().makeQualified(fs.getUri(),fs.getWorkingDirectory());
       // should exist in dist cache dir
-      assertTrue("Public dist cache file path is wrong.",
-          distCacheDir.equals(parent));
+      assertTrue(distCacheDir.equals(parent),
+          "Public dist cache file path is wrong.");
     }
   }
 
@@ -352,7 +352,8 @@ public class TestDistCacheEmulation {
    * @throws IOException
    * @throws InterruptedException
    */
-  @Test  (timeout=20000)
+  @Test
+  @Timeout(value = 20)
   public void testSetupGenerateDistCacheData() throws IOException,
       InterruptedException {
     long[] sortedFileSizes = new long[5];
@@ -381,7 +382,8 @@ public class TestDistCacheEmulation {
    * Test the configuration property for disabling/enabling emulation of
    * distributed cache load.
    */
-  @Test  (timeout=2000)
+  @Test
+  @Timeout(value = 2)
   public void testDistCacheEmulationConfigurability() throws IOException {
     Configuration jobConf = GridmixTestUtils.mrvl.getConfig();
     Path ioPath = new Path("testDistCacheEmulationConfigurability")
@@ -391,23 +393,25 @@ public class TestDistCacheEmulation {
 
     // default config
     dce = createDistributedCacheEmulator(jobConf, ioPath, false);
-    assertTrue("Default configuration of "
+    assertTrue(dce.shouldEmulateDistCacheLoad(), "Default configuration of "
         + DistributedCacheEmulator.GRIDMIX_EMULATE_DISTRIBUTEDCACHE
-        + " is wrong.", dce.shouldEmulateDistCacheLoad());
+        + " is wrong.");
 
     // config property set to false
     jobConf.setBoolean(
         DistributedCacheEmulator.GRIDMIX_EMULATE_DISTRIBUTEDCACHE, false);
     dce = createDistributedCacheEmulator(jobConf, ioPath, false);
-    assertFalse("Disabling of emulation of distributed cache load by setting "
+    assertFalse(dce.shouldEmulateDistCacheLoad(),
+        "Disabling of emulation of distributed cache load by setting "
         + DistributedCacheEmulator.GRIDMIX_EMULATE_DISTRIBUTEDCACHE
-        + " to false is not working.", dce.shouldEmulateDistCacheLoad());
+        + " to false is not working.");
   }
 /** 
  * test method configureDistCacheFiles
  * 
  */
-  @Test  (timeout=2000)
+  @Test
+  @Timeout(value = 2)
   public void testDistCacheEmulator() throws Exception {
 
     Configuration conf = new Configuration();

+ 12 - 8
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestFilePool.java

@@ -17,6 +17,11 @@
  */
 package org.apache.hadoop.mapred.gridmix;
 
+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 java.io.IOException;
 import java.io.OutputStream;
 import java.util.ArrayList;
@@ -24,10 +29,9 @@ import java.util.Arrays;
 import java.util.HashSet;
 import java.util.Random;
 
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import static org.junit.Assert.*;
+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;
 
@@ -56,7 +60,7 @@ public class TestFilePool {
     return null;
   }
 
-  @BeforeClass
+  @BeforeAll
   public static void setup() throws IOException {
     final Configuration conf = new Configuration();
     final FileSystem fs = FileSystem.getLocal(conf).getRaw();
@@ -91,7 +95,7 @@ public class TestFilePool {
     }
   }
 
-  @AfterClass
+  @AfterAll
   public static void cleanup() throws IOException {
     final Configuration conf = new Configuration();
     final FileSystem fs = FileSystem.getLocal(conf).getRaw();
@@ -133,8 +137,8 @@ public class TestFilePool {
     // match random within 12k
     files.clear();
     final long rand = r.nextInt(expectedPoolSize);
-    assertTrue("Missed: " + rand,
-        (NFILES / 2) * 1024 > rand - pool.getInputFiles(rand, files));
+    assertTrue((NFILES / 2) * 1024 > rand - pool.getInputFiles(rand, files),
+        "Missed: " + rand);
 
     // all files
     conf.setLong(FilePool.GRIDMIX_MIN_FILE, 0);

+ 7 - 6
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestFileQueue.java

@@ -17,15 +17,16 @@
  */
 package org.apache.hadoop.mapred.gridmix;
 
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.util.Arrays;
 
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import static org.junit.Assert.*;
+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;
 
@@ -44,7 +45,7 @@ public class TestFileQueue {
   static final long[] start = new long[NFILES];
   static final long[] len = new long[NFILES];
 
-  @BeforeClass
+  @BeforeAll
   public static void setup() throws IOException {
     final Configuration conf = new Configuration();
     final FileSystem fs = FileSystem.getLocal(conf).getRaw();
@@ -67,7 +68,7 @@ public class TestFileQueue {
     }
   }
 
-  @AfterClass
+  @AfterAll
   public static void cleanup() throws IOException {
     final Configuration conf = new Configuration();
     final FileSystem fs = FileSystem.getLocal(conf).getRaw();

+ 33 - 18
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestGridMixClasses.java

@@ -77,12 +77,15 @@ import org.apache.hadoop.tools.rumen.JobStoryProducer;
 import org.apache.hadoop.tools.rumen.ResourceUsageMetrics;
 import org.apache.hadoop.tools.rumen.ZombieJobProducer;
 import org.apache.hadoop.util.Progress;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
-import static org.mockito.Mockito.*;
-
-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;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 public class TestGridMixClasses {
   private static final Logger LOG = LoggerFactory.getLogger(TestGridMixClasses.class);
@@ -90,7 +93,8 @@ public class TestGridMixClasses {
   /*
    * simple test LoadSplit (getters,copy, write, read...)
    */
-  @Test (timeout=1000)
+  @Test
+  @Timeout(value = 1)
   public void testLoadSplit() throws Exception {
 
     LoadSplit test = getLoadSplit();
@@ -121,7 +125,8 @@ public class TestGridMixClasses {
   /*
    * simple test GridmixSplit (copy, getters, write, read..)
    */
-  @Test (timeout=1000)
+  @Test
+  @Timeout(value = 1)
   public void testGridmixSplit() throws Exception {
     Path[] files = {new Path("one"), new Path("two")};
     long[] start = {1, 2};
@@ -164,7 +169,8 @@ public class TestGridMixClasses {
    * test LoadMapper loadMapper should write to writer record for each reduce
    */
   @SuppressWarnings({"rawtypes", "unchecked"})
-  @Test (timeout=10000)
+  @Test
+  @Timeout(value = 10)
   public void testLoadMapper() throws Exception {
 
     Configuration conf = new Configuration();
@@ -376,7 +382,8 @@ public class TestGridMixClasses {
   /*
    * test LoadSortComparator
    */
-  @Test (timeout=3000)
+  @Test
+  @Timeout(value = 3)
   public void testLoadJobLoadSortComparator() throws Exception {
     LoadJob.LoadSortComparator test = new LoadJob.LoadSortComparator();
 
@@ -409,7 +416,8 @@ public class TestGridMixClasses {
   /*
    * test SpecGroupingComparator
    */
-  @Test (timeout=3000)
+  @Test
+  @Timeout(value = 3)
   public void testGridmixJobSpecGroupingComparator() throws Exception {
     GridmixJob.SpecGroupingComparator test = new GridmixJob.SpecGroupingComparator();
 
@@ -452,7 +460,8 @@ public class TestGridMixClasses {
   /*
    * test CompareGridmixJob only equals and compare
    */
-  @Test (timeout=30000)
+  @Test
+  @Timeout(value = 30)
   public void testCompareGridmixJob() throws Exception {
     Configuration conf = new Configuration();
     Path outRoot = new Path("target");
@@ -478,7 +487,8 @@ public class TestGridMixClasses {
   /*
    * test ReadRecordFactory. should read all data from inputstream
    */
-  @Test (timeout=3000)
+  @Test
+  @Timeout(value = 3)
   public void testReadRecordFactory() throws Exception {
 
     // RecordFactory factory, InputStream src, Configuration conf
@@ -589,7 +599,8 @@ public class TestGridMixClasses {
   /*
    * test LoadRecordReader. It class reads data from some files.
    */
-  @Test (timeout=6100)
+  @Test
+  @Timeout(value = 6)
   public void testLoadJobLoadRecordReader() throws Exception {
     LoadJob.LoadRecordReader test = new LoadJob.LoadRecordReader();
     Configuration conf = new Configuration();
@@ -652,7 +663,8 @@ public class TestGridMixClasses {
    * test LoadReducer
    */
 
-  @Test (timeout=3000)
+  @Test
+  @Timeout(value = 3)
   public void testLoadJobLoadReducer() throws Exception {
     LoadJob.LoadReducer test = new LoadJob.LoadReducer();
 
@@ -772,7 +784,8 @@ public class TestGridMixClasses {
   /*
    * test SerialJobFactory
    */
-  @Test (timeout=120000)
+  @Test
+  @Timeout(value = 120)
   public void testSerialReaderThread() throws Exception {
 
     Configuration conf = new Configuration();
@@ -787,7 +800,7 @@ public class TestGridMixClasses {
     FakeJobSubmitter submitter = new FakeJobSubmitter();
     File ws = new File("target" + File.separator + this.getClass().getName());
     if (!ws.exists()) {
-      Assert.assertTrue(ws.mkdirs());
+      assertTrue(ws.mkdirs());
     }
 
     SerialJobFactory jobFactory = new SerialJobFactory(submitter, jobProducer,
@@ -833,7 +846,8 @@ public class TestGridMixClasses {
    * test SleepMapper
    */
   @SuppressWarnings({"unchecked", "rawtypes"})
-  @Test (timeout=30000)
+  @Test
+  @Timeout(value = 30)
   public void testSleepMapper() throws Exception {
     SleepJob.SleepMapper test = new SleepJob.SleepMapper();
 
@@ -878,7 +892,8 @@ public class TestGridMixClasses {
   /*
    * test SleepReducer
    */
-  @Test (timeout=3000)
+  @Test
+  @Timeout(value = 3)
   public void testSleepReducer() throws Exception {
     Configuration conf = new Configuration();
     conf.setInt(JobContext.NUM_REDUCES, 2);

+ 37 - 34
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestGridmixMemoryEmulation.java

@@ -17,8 +17,10 @@
  */
 package org.apache.hadoop.mapred.gridmix;
 
-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.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.IOException;
 
@@ -112,13 +114,13 @@ public class TestGridmixMemoryEmulation {
     long currentHeap = heapEmulator.getHeapUsageInMB();
     
     // check if the heap has increased by expected value
-    assertEquals("Default heap emulator failed to load 10mb", 
-                 previousHeap + testSizeInMB, currentHeap);
+    assertEquals(previousHeap + testSizeInMB, currentHeap,
+        "Default heap emulator failed to load 10mb");
     
     // test reset
     heapEmulator.resetFake();
-    assertEquals("Default heap emulator failed to reset", 
-                 0, heapEmulator.getHeapUsageInMB());
+    assertEquals(0, heapEmulator.getHeapUsageInMB(),
+        "Default heap emulator failed to reset");
   }
 
   /**
@@ -164,16 +166,17 @@ public class TestGridmixMemoryEmulation {
     long heapUsagePost = fakeCore.getHeapUsageInMB();
     
     //  test if no calls are made heap usage emulator core
-    assertEquals("Disabled heap usage emulation plugin works!", 
-                 numCallsPre, numCallsPost);
+    assertEquals(numCallsPre, numCallsPost,
+        "Disabled heap usage emulation plugin works!");
     //  test if no calls are made heap usage emulator core
-    assertEquals("Disabled heap usage emulation plugin works!", 
-                 heapUsagePre, heapUsagePost);
+    assertEquals(heapUsagePre, heapUsagePost,
+        "Disabled heap usage emulation plugin works!");
     
     // test with get progress
     float progress = heapPlugin.getProgress();
-    assertEquals("Invalid progress of disabled cumulative heap usage emulation "
-                 + "plugin!", 1.0f, progress, 0f);
+    assertEquals(1.0f, progress, 0f,
+        "Invalid progress of disabled cumulative heap usage emulation "
+        + "plugin!");
     
     // test with wrong/invalid configuration
     Boolean failed = null;
@@ -186,8 +189,8 @@ public class TestGridmixMemoryEmulation {
     } catch (Exception e) {
       failed = true;
     }
-    assertNotNull("Fail case failure!", failed);
-    assertTrue("Expected failure!", failed); 
+    assertNotNull(failed, "Fail case failure!");
+    assertTrue(failed, "Expected failure!");
     
     // test with valid resource usage value
     ResourceUsageMetrics metrics = 
@@ -282,11 +285,11 @@ public class TestGridmixMemoryEmulation {
     }
     
     // test if the resource plugin shows the expected usage
-    assertEquals("Cumulative heap usage emulator plugin failed (total usage)!", 
-                 expectedTotalHeapUsageInMB, fakeCore.getHeapUsageInMB(), 1L);
+    assertEquals(expectedTotalHeapUsageInMB, fakeCore.getHeapUsageInMB(), 1L,
+        "Cumulative heap usage emulator plugin failed (total usage)!");
     // test if the resource plugin shows the expected num calls
-    assertEquals("Cumulative heap usage emulator plugin failed (num calls)!", 
-                 expectedTotalNumCalls, fakeCore.getNumCalls(), 0L);
+    assertEquals(expectedTotalNumCalls, fakeCore.getNumCalls(), 0L,
+        "Cumulative heap usage emulator plugin failed (num calls)!");
   }
 
   // tests if the heap usage emulation plugin emulates only at the expected
@@ -298,11 +301,11 @@ public class TestGridmixMemoryEmulation {
     fakeProgress.setProgress(progress);
     heapPlugin.emulate();
     // test heap usage
-    assertEquals("Emulation interval test for heap usage failed " + info + "!", 
-                 expectedTotalHeapUsageInMB, fakeCore.getHeapUsageInMB(), 0L);
+    assertEquals(expectedTotalHeapUsageInMB, fakeCore.getHeapUsageInMB(), 0L,
+        "Emulation interval test for heap usage failed " + info + "!");
     // test num calls
-    assertEquals("Emulation interval test for heap usage failed " + info + "!", 
-                 expectedTotalNumCalls, fakeCore.getNumCalls(), 0L);
+    assertEquals(expectedTotalNumCalls, fakeCore.getNumCalls(), 0L,
+        "Emulation interval test for heap usage failed " + info + "!");
   }
   
   /**
@@ -355,12 +358,12 @@ public class TestGridmixMemoryEmulation {
     // configure the task jvm's heap options
     GridmixJob.configureTaskJVMOptions(originalConf, simulatedConf);
     
-    assertEquals("Map heap options mismatch!", expectedMapOptions, 
-                 simulatedConf.get(MRJobConfig.MAP_JAVA_OPTS));
-    assertEquals("Reduce heap options mismatch!", expectedReduceOptions, 
-                 simulatedConf.get(MRJobConfig.REDUCE_JAVA_OPTS));
-    assertEquals("Task heap options mismatch!", expectedTaskOptions, 
-                 simulatedConf.get(JobConf.MAPRED_TASK_JAVA_OPTS));
+    assertEquals(expectedMapOptions, simulatedConf.get(MRJobConfig.MAP_JAVA_OPTS),
+        "Map heap options mismatch!");
+    assertEquals(expectedReduceOptions, simulatedConf.get(MRJobConfig.REDUCE_JAVA_OPTS),
+        "Reduce heap options mismatch!");
+    assertEquals(expectedTaskOptions, simulatedConf.get(JobConf.MAPRED_TASK_JAVA_OPTS),
+        "Task heap options mismatch!");
   }
   
   /**
@@ -447,11 +450,11 @@ public class TestGridmixMemoryEmulation {
     Job simulatedJob = job.getJob();
     Configuration simulatedConf = simulatedJob.getConfiguration();
     
-    assertEquals("Map heap options works when disabled!", "-Xmx1m", 
-                 simulatedConf.get(MRJobConfig.MAP_JAVA_OPTS));
-    assertEquals("Reduce heap options works when disabled!", "-Xmx2m", 
-                 simulatedConf.get(MRJobConfig.REDUCE_JAVA_OPTS));
-    assertEquals("Task heap options works when disabled!", "-Xmx3m", 
-                 simulatedConf.get(JobConf.MAPRED_TASK_JAVA_OPTS));
+    assertEquals("-Xmx1m", simulatedConf.get(MRJobConfig.MAP_JAVA_OPTS),
+        "Map heap options works when disabled!");
+    assertEquals("-Xmx2m", simulatedConf.get(MRJobConfig.REDUCE_JAVA_OPTS),
+        "Reduce heap options works when disabled!");
+    assertEquals("-Xmx3m", simulatedConf.get(JobConf.MAPRED_TASK_JAVA_OPTS),
+        "Task heap options works when disabled!");
   }
 }

+ 7 - 4
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestGridmixRecord.java

@@ -17,12 +17,15 @@
  */
 package org.apache.hadoop.mapred.gridmix;
 
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Random;
 
-import org.junit.Test;
-import static org.junit.Assert.*;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -90,7 +93,7 @@ public class TestGridmixRecord {
     // verify written contents match
     assertEquals(out1.getLength(), out2.getLength());
     // assumes that writes will grow buffer deterministically
-    assertEquals("Bad test", out1.getData().length, out2.getData().length);
+    assertEquals(out1.getData().length, out2.getData().length, "Bad test");
     assertArrayEquals(out1.getData(), out2.getData());
   }
 
@@ -114,7 +117,7 @@ public class TestGridmixRecord {
       // verify written contents match
       assertEquals(out1.getLength(), out2.getLength());
       // assumes that writes will grow buffer deterministically
-      assertEquals("Bad test", out1.getData().length, out2.getData().length);
+      assertEquals(out1.getData().length, out2.getData().length, "Bad test");
       assertArrayEquals(out1.getData(), out2.getData());
     }
   }

+ 19 - 12
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestGridmixSubmission.java

@@ -25,9 +25,10 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.tools.rumen.JobStory;
 import org.apache.hadoop.tools.rumen.JobStoryProducer;
 import org.apache.hadoop.util.ExitUtil;
-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 org.junit.jupiter.api.Timeout;
 import org.slf4j.event.Level;
 
 
@@ -39,7 +40,9 @@ import java.io.PrintStream;
 import java.util.zip.GZIPInputStream;
 
 import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
-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;
 import static org.slf4j.LoggerFactory.getLogger;
 
 public class TestGridmixSubmission extends CommonJobTest {
@@ -53,14 +56,14 @@ public class TestGridmixSubmission extends CommonJobTest {
   }
 
 
-  @BeforeClass
+  @BeforeAll
   public static void init() throws IOException {
     GridmixTestUtils.initCluster(TestGridmixSubmission.class);
 
     System.setProperty("src.test.data", inSpace.getAbsolutePath());
   }
 
-  @AfterClass
+  @AfterAll
   public static void shutDown() throws IOException {
     GridmixTestUtils.shutdownCluster();
   }
@@ -73,12 +76,12 @@ public class TestGridmixSubmission extends CommonJobTest {
    * @param js the candidate JobStory.
    */
   private void verifyWordCountJobStory(JobStory js) {
-    assertNotNull("Null JobStory", js);
+    assertNotNull(js, "Null JobStory");
     String expectedJobStory = "WordCount:johndoe:default:1285322645148:3:1";
     String actualJobStory = js.getName() + ":" + js.getUser() + ":"
             + js.getQueueName() + ":" + js.getSubmissionTime() + ":"
             + js.getNumberMaps() + ":" + js.getNumberReduces();
-    assertEquals("Unexpected JobStory", expectedJobStory, actualJobStory);
+    assertEquals(expectedJobStory, actualJobStory, "Unexpected JobStory");
   }
 
   /**
@@ -110,7 +113,8 @@ public class TestGridmixSubmission extends CommonJobTest {
    *
    * @throws Exception if there was an error.
    */
-  @Test (timeout=20000)
+  @Test
+  @Timeout(value = 20)
   public void testTraceReader() throws Exception {
     Configuration conf = new Configuration();
     FileSystem lfs = FileSystem.getLocal(conf);
@@ -153,7 +157,8 @@ public class TestGridmixSubmission extends CommonJobTest {
     }
   }
 
-  @Test (timeout=500000)
+  @Test
+  @Timeout(value = 500)
   public void testReplaySubmit() throws Exception {
     policy = GridmixJobSubmissionPolicy.REPLAY;
     LOG.info(" Replay started at " + System.currentTimeMillis());
@@ -162,7 +167,8 @@ public class TestGridmixSubmission extends CommonJobTest {
 
   }
 
-  @Test (timeout=500000)
+  @Test
+  @Timeout(value = 500)
   public void testStressSubmit() throws Exception {
     policy = GridmixJobSubmissionPolicy.STRESS;
     LOG.info(" Stress started at " + System.currentTimeMillis());
@@ -171,7 +177,8 @@ public class TestGridmixSubmission extends CommonJobTest {
   }
 
   // test empty request should be hint message
-  @Test (timeout=100000)
+  @Test
+  @Timeout(value = 100)
   public void testMain() throws Exception {
 
     SecurityManager securityManager = System.getSecurityManager();

+ 81 - 88
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestGridmixSummary.java

@@ -17,7 +17,10 @@
  */
 package org.apache.hadoop.mapred.gridmix;
 
-import static org.junit.Assert.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.IOException;
 
@@ -35,7 +38,7 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
 import org.apache.hadoop.tools.rumen.JobStory;
 import org.apache.hadoop.tools.rumen.JobStoryProducer;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 /**
  * Test {@link ExecutionSummarizer} and {@link ClusterSummarizer}.
@@ -49,15 +52,15 @@ public class TestGridmixSummary {
   public void testDataStatistics() throws Exception {
     // test data-statistics getters with compression enabled
     DataStatistics stats = new DataStatistics(10, 2, true);
-    assertEquals("Data size mismatch", 10, stats.getDataSize());
-    assertEquals("Num files mismatch", 2, stats.getNumFiles());
-    assertTrue("Compression configuration mismatch", stats.isDataCompressed());
+    assertEquals(10, stats.getDataSize(), "Data size mismatch");
+    assertEquals(2, stats.getNumFiles(), "Num files mismatch");
+    assertTrue(stats.isDataCompressed(), "Compression configuration mismatch");
     
     // test data-statistics getters with compression disabled
     stats = new DataStatistics(100, 5, false);
-    assertEquals("Data size mismatch", 100, stats.getDataSize());
-    assertEquals("Num files mismatch", 5, stats.getNumFiles());
-    assertFalse("Compression configuration mismatch", stats.isDataCompressed());
+    assertEquals(100, stats.getDataSize(), "Data size mismatch");
+    assertEquals(5, stats.getNumFiles(), "Num files mismatch");
+    assertFalse(stats.isDataCompressed(), "Compression configuration mismatch");
     
     // test publish data stats
     Configuration conf = new Configuration();
@@ -77,15 +80,15 @@ public class TestGridmixSummary {
     } catch (RuntimeException e) {
       failed = true;
     }
-    assertNotNull("Expected failure!", failed);
-    assertTrue("Compression data publishing error", failed);
+    assertNotNull(failed, "Expected failure!");
+    assertTrue(failed, "Compression data publishing error");
     
     // test with empty folder (compression = off)
     CompressionEmulationUtil.setCompressionEmulationEnabled(conf, false);
     stats = GenerateData.publishDataStatistics(testInputDir, 1024L, conf);
-    assertEquals("Data size mismatch", 0, stats.getDataSize());
-    assertEquals("Num files mismatch", 0, stats.getNumFiles());
-    assertFalse("Compression configuration mismatch", stats.isDataCompressed());
+    assertEquals(0, stats.getDataSize(), "Data size mismatch");
+    assertEquals(0, stats.getNumFiles(), "Num files mismatch");
+    assertFalse(stats.isDataCompressed(), "Compression configuration mismatch");
     
     // test with some plain input data (compression = off)
     CompressionEmulationUtil.setCompressionEmulationEnabled(conf, false);
@@ -94,9 +97,9 @@ public class TestGridmixSummary {
       UtilsForTests.createTmpFileDFS(fs, inputDataFile, 
           FsPermission.createImmutable((short)777), "hi hello bye").size();
     stats = GenerateData.publishDataStatistics(testInputDir, -1, conf);
-    assertEquals("Data size mismatch", size, stats.getDataSize());
-    assertEquals("Num files mismatch", 1, stats.getNumFiles());
-    assertFalse("Compression configuration mismatch", stats.isDataCompressed());
+    assertEquals(size, stats.getDataSize(), "Data size mismatch");
+    assertEquals(1, stats.getNumFiles(), "Num files mismatch");
+    assertFalse(stats.isDataCompressed(), "Compression configuration mismatch");
     
     // test with some plain input data (compression = on)
     CompressionEmulationUtil.setCompressionEmulationEnabled(conf, true);
@@ -107,8 +110,8 @@ public class TestGridmixSummary {
     } catch (RuntimeException e) {
       failed = true;
     }
-    assertNotNull("Expected failure!", failed);
-    assertTrue("Compression data publishing error", failed);
+    assertNotNull(failed, "Expected failure!");
+    assertTrue(failed, "Compression data publishing error");
     
     // test with some compressed input data (compression = off)
     CompressionEmulationUtil.setCompressionEmulationEnabled(conf, false);
@@ -118,16 +121,16 @@ public class TestGridmixSummary {
       UtilsForTests.createTmpFileDFS(fs, inputDataFile, 
           FsPermission.createImmutable((short)777), "hi hello").size();
     stats =  GenerateData.publishDataStatistics(testInputDir, 1234L, conf);
-    assertEquals("Data size mismatch", size, stats.getDataSize());
-    assertEquals("Num files mismatch", 1, stats.getNumFiles());
-    assertFalse("Compression configuration mismatch", stats.isDataCompressed());
+    assertEquals(size, stats.getDataSize(), "Data size mismatch");
+    assertEquals(1, stats.getNumFiles(), "Num files mismatch");
+    assertFalse(stats.isDataCompressed(), "Compression configuration mismatch");
     
     // test with some compressed input data (compression = on)
     CompressionEmulationUtil.setCompressionEmulationEnabled(conf, true);
     stats = GenerateData.publishDataStatistics(testInputDir, 1234L, conf);
-    assertEquals("Data size mismatch", size, stats.getDataSize());
-    assertEquals("Num files mismatch", 1, stats.getNumFiles());
-    assertTrue("Compression configuration mismatch", stats.isDataCompressed());
+    assertEquals(size, stats.getDataSize(), "Data size mismatch");
+    assertEquals(1, stats.getNumFiles(), "Num files mismatch");
+    assertTrue(stats.isDataCompressed(), "Compression configuration mismatch");
   }
   
   /**
@@ -172,35 +175,33 @@ public class TestGridmixSummary {
     Configuration conf = new Configuration();
     
     ExecutionSummarizer es = new ExecutionSummarizer();
-    assertEquals("ExecutionSummarizer init failed", 
-                 Summarizer.NA, es.getCommandLineArgsString());
+    assertEquals(Summarizer.NA, es.getCommandLineArgsString(),
+        "ExecutionSummarizer init failed");
     
     long startTime = System.currentTimeMillis();
     // test configuration parameters
     String[] initArgs = new String[] {"-Xmx20m", "-Dtest.args='test'"};
     es = new ExecutionSummarizer(initArgs);
     
-    assertEquals("ExecutionSummarizer init failed", 
-                 "-Xmx20m -Dtest.args='test'", 
-                 es.getCommandLineArgsString());
+    assertEquals("-Xmx20m -Dtest.args='test'",
+        es.getCommandLineArgsString(), "ExecutionSummarizer init failed");
     
     // test start time
-    assertTrue("Start time mismatch", es.getStartTime() >= startTime);
-    assertTrue("Start time mismatch", 
-               es.getStartTime() <= System.currentTimeMillis());
+    assertTrue(es.getStartTime() >= startTime, "Start time mismatch");
+    assertTrue(es.getStartTime() <= System.currentTimeMillis(), "Start time mismatch");
     
     // test start() of ExecutionSummarizer
     es.update(null);
-    assertEquals("ExecutionSummarizer init failed", 0, 
-                 es.getSimulationStartTime());
+    assertEquals(0, es.getSimulationStartTime(),
+        "ExecutionSummarizer init failed");
     testExecutionSummarizer(0, 0, 0, 0, 0, 0, 0, es);
     
     long simStartTime = System.currentTimeMillis();
     es.start(null);
-    assertTrue("Simulation start time mismatch", 
-               es.getSimulationStartTime() >= simStartTime);
-    assertTrue("Simulation start time mismatch", 
-               es.getSimulationStartTime() <= System.currentTimeMillis());
+    assertTrue(es.getSimulationStartTime() >= simStartTime,
+        "Simulation start time mismatch");
+    assertTrue(es.getSimulationStartTime() <= System.currentTimeMillis(),
+        "Simulation start time mismatch");
     
     // test with job stats
     JobStats stats = generateFakeJobStats(1, 10, true, false);
@@ -244,35 +245,30 @@ public class TestGridmixSummary {
                 conf);
     
     // test num jobs in trace
-    assertEquals("Mismtach in num jobs in trace", 3, es.getNumJobsInTrace());
+    assertEquals(3, es.getNumJobsInTrace(), "Mismtach in num jobs in trace");
     
     // test trace signature
     String tid = 
       ExecutionSummarizer.getTraceSignature(testTraceFile.toString());
-    assertEquals("Mismatch in trace signature", 
-                 tid, es.getInputTraceSignature());
+    assertEquals(tid, es.getInputTraceSignature(), "Mismatch in trace signature");
     // test trace location
     Path qPath = fs.makeQualified(testTraceFile);
-    assertEquals("Mismatch in trace filename", 
-                 qPath.toString(), es.getInputTraceLocation());
+    assertEquals(qPath.toString(), es.getInputTraceLocation(), "Mismatch in trace filename");
     // test expected data size
-    assertEquals("Mismatch in expected data size", 
-                 "1 K", es.getExpectedDataSize());
+    assertEquals("1 K", es.getExpectedDataSize(), "Mismatch in expected data size");
     // test input data statistics
-    assertEquals("Mismatch in input data statistics", 
-                 ExecutionSummarizer.stringifyDataStatistics(dataStats), 
-                 es.getInputDataStatistics());
+    assertEquals(ExecutionSummarizer.stringifyDataStatistics(dataStats),
+        es.getInputDataStatistics(), "Mismatch in input data statistics");
     // test user resolver
-    assertEquals("Mismatch in user resolver", 
-                 resolver.getClass().getName(), es.getUserResolver());
+    assertEquals(resolver.getClass().getName(), es.getUserResolver(),
+        "Mismatch in user resolver");
     // test policy
-    assertEquals("Mismatch in policy", policy, es.getJobSubmissionPolicy());
+    assertEquals(policy, es.getJobSubmissionPolicy(), "Mismatch in policy");
     
     // test data stringification using large data
     es.finalize(factory, testTraceFile.toString(), 1024*1024*1024*10L, resolver,
                 dataStats, conf);
-    assertEquals("Mismatch in expected data size", 
-                 "10 G", es.getExpectedDataSize());
+    assertEquals("10 G", es.getExpectedDataSize(), "Mismatch in expected data size");
     
     // test trace signature uniqueness
     //  touch the trace file
@@ -285,52 +281,50 @@ public class TestGridmixSummary {
     es.finalize(factory, testTraceFile.toString(), 0L, resolver, dataStats, 
                 conf);
     // test missing expected data size
-    assertEquals("Mismatch in trace data size", 
-                 Summarizer.NA, es.getExpectedDataSize());
-    assertFalse("Mismatch in trace signature", 
-                tid.equals(es.getInputTraceSignature()));
+    assertEquals(Summarizer.NA, es.getExpectedDataSize(),
+        "Mismatch in trace data size");
+    assertFalse(tid.equals(es.getInputTraceSignature()),
+        "Mismatch in trace signature");
     // get the new identifier
     tid = ExecutionSummarizer.getTraceSignature(testTraceFile.toString());
-    assertEquals("Mismatch in trace signature", 
-                 tid, es.getInputTraceSignature());
+    assertEquals(tid, es.getInputTraceSignature(), "Mismatch in trace signature");
     
     testTraceFile = new Path(testDir, "test-trace2.json");
     fs.create(testTraceFile).close();
     es.finalize(factory, testTraceFile.toString(), 0L, resolver, dataStats, 
                 conf);
-    assertFalse("Mismatch in trace signature", 
-                tid.equals(es.getInputTraceSignature()));
+    assertFalse(tid.equals(es.getInputTraceSignature()),
+        "Mismatch in trace signature");
     // get the new identifier
     tid = ExecutionSummarizer.getTraceSignature(testTraceFile.toString());
-    assertEquals("Mismatch in trace signature", 
-                 tid, es.getInputTraceSignature());
+    assertEquals(tid, es.getInputTraceSignature(), "Mismatch in trace signature");
     
     // finalize trace identifier '-' input
     es.finalize(factory, "-", 0L, resolver, dataStats, conf);
-    assertEquals("Mismatch in trace signature",
-                 Summarizer.NA, es.getInputTraceSignature());
-    assertEquals("Mismatch in trace file location", 
-                 Summarizer.NA, es.getInputTraceLocation());
+    assertEquals(Summarizer.NA, es.getInputTraceSignature(),
+        "Mismatch in trace signature");
+    assertEquals(Summarizer.NA, es.getInputTraceLocation(),
+        "Mismatch in trace file location");
   }
   
   // test the ExecutionSummarizer
   private static void testExecutionSummarizer(int numMaps, int numReds,
       int totalJobsInTrace, int totalJobSubmitted, int numSuccessfulJob, 
       int numFailedJobs, int numLostJobs, ExecutionSummarizer es) {
-    assertEquals("ExecutionSummarizer test failed [num-maps]", 
-                 numMaps, es.getNumMapTasksLaunched());
-    assertEquals("ExecutionSummarizer test failed [num-reducers]", 
-                 numReds, es.getNumReduceTasksLaunched());
-    assertEquals("ExecutionSummarizer test failed [num-jobs-in-trace]", 
-                 totalJobsInTrace, es.getNumJobsInTrace());
-    assertEquals("ExecutionSummarizer test failed [num-submitted jobs]", 
-                 totalJobSubmitted, es.getNumSubmittedJobs());
-    assertEquals("ExecutionSummarizer test failed [num-successful-jobs]", 
-                 numSuccessfulJob, es.getNumSuccessfulJobs());
-    assertEquals("ExecutionSummarizer test failed [num-failed jobs]", 
-                 numFailedJobs, es.getNumFailedJobs());
-    assertEquals("ExecutionSummarizer test failed [num-lost jobs]", 
-                 numLostJobs, es.getNumLostJobs());
+    assertEquals(numMaps, es.getNumMapTasksLaunched(),
+        "ExecutionSummarizer test failed [num-maps]");
+    assertEquals(numReds, es.getNumReduceTasksLaunched(),
+        "ExecutionSummarizer test failed [num-reducers]");
+    assertEquals(totalJobsInTrace, es.getNumJobsInTrace(),
+        "ExecutionSummarizer test failed [num-jobs-in-trace]");
+    assertEquals(totalJobSubmitted, es.getNumSubmittedJobs(),
+        "ExecutionSummarizer test failed [num-submitted jobs]");
+    assertEquals(numSuccessfulJob, es.getNumSuccessfulJobs(),
+        "ExecutionSummarizer test failed [num-successful-jobs]");
+    assertEquals(numFailedJobs, es.getNumFailedJobs(),
+        "ExecutionSummarizer test failed [num-failed jobs]");
+    assertEquals(numLostJobs, es.getNumLostJobs(),
+        "ExecutionSummarizer test failed [num-lost jobs]");
   }
   
   // generate fake job stats
@@ -370,8 +364,8 @@ public class TestGridmixSummary {
     conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, nn);
     cs.start(conf);
     
-    assertEquals("JT name mismatch", jt, cs.getJobTrackerInfo());
-    assertEquals("NN name mismatch", nn, cs.getNamenodeInfo());
+    assertEquals(jt, cs.getJobTrackerInfo(), "JT name mismatch");
+    assertEquals(nn, cs.getNamenodeInfo(), "NN name mismatch");
     
     ClusterStats cStats = ClusterStats.getClusterStats();
     conf.set(JTConfig.JT_IPC_ADDRESS, "local");
@@ -382,10 +376,9 @@ public class TestGridmixSummary {
     cs.update(cStats);
     
     // test
-    assertEquals("Cluster summary test failed!", 1, cs.getMaxMapTasks());
-    assertEquals("Cluster summary test failed!", 1, cs.getMaxReduceTasks());
-    assertEquals("Cluster summary test failed!", 1, cs.getNumActiveTrackers());
-    assertEquals("Cluster summary test failed!", 0, 
-                 cs.getNumBlacklistedTrackers());
+    assertEquals(1, cs.getMaxMapTasks(), "Cluster summary test failed!");
+    assertEquals(1, cs.getMaxReduceTasks(), "Cluster summary test failed!");
+    assertEquals(1, cs.getNumActiveTrackers(), "Cluster summary test failed!");
+    assertEquals(0, cs.getNumBlacklistedTrackers(), "Cluster summary test failed!");
   }
 }

+ 10 - 8
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestHighRamJob.java

@@ -17,7 +17,9 @@
  */
 package org.apache.hadoop.mapred.gridmix;
 
-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;
 
 import java.io.IOException;
 
@@ -32,7 +34,7 @@ import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.tools.rumen.JobStory;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 /**
  * Test if Gridmix correctly configures the simulated job's configuration for
@@ -145,8 +147,8 @@ public class TestHighRamJob {
       failed = true;
     }
     assertNotNull(failed);
-    assertTrue("Exception expected for exceeding map memory limit "
-               + "(deprecation)!", failed);
+    assertTrue(failed, "Exception expected for exceeding map memory limit "
+         + "(deprecation)!");
     
     // test : check with high ram enabled and reduce memory scaling mismatch 
     //        (deprecated)
@@ -161,8 +163,8 @@ public class TestHighRamJob {
       failed = true;
     }
     assertNotNull(failed);
-    assertTrue("Exception expected for exceeding reduce memory limit "
-               + "(deprecation)!", failed);
+    assertTrue(failed, "Exception expected for exceeding reduce memory limit "
+        + "(deprecation)!");
     
     // test : check with high ram enabled and scaling mismatch on map limits
     gridmixConf = new Configuration();
@@ -175,7 +177,7 @@ public class TestHighRamJob {
       failed = true;
     }
     assertNotNull(failed);
-    assertTrue("Exception expected for exceeding map memory limit!", failed);
+    assertTrue(failed, "Exception expected for exceeding map memory limit!");
     
     // test : check with high ram enabled and scaling mismatch on reduce 
     //        limits
@@ -189,6 +191,6 @@ public class TestHighRamJob {
       failed = true;
     }
     assertNotNull(failed);
-    assertTrue("Exception expected for exceeding reduce memory limit!", failed);
+    assertTrue(failed, "Exception expected for exceeding reduce memory limit!");
   }
 }

+ 10 - 7
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestLoadJob.java

@@ -18,9 +18,10 @@
 package org.apache.hadoop.mapred.gridmix;
 
 import org.apache.hadoop.test.GenericTestUtils;
-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 org.junit.jupiter.api.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.event.Level;
 
@@ -43,12 +44,12 @@ public class TestLoadJob extends CommonJobTest {
   }
 
 
-  @BeforeClass
+  @BeforeAll
   public static void init() throws IOException {
     GridmixTestUtils.initCluster(TestLoadJob.class);
   }
 
-  @AfterClass
+  @AfterAll
   public static void shutDown() throws IOException {
     GridmixTestUtils.shutdownCluster();
   }
@@ -57,7 +58,8 @@ public class TestLoadJob extends CommonJobTest {
   /*
   * test serial policy  with LoadJob. Task should execute without exceptions
   */
-  @Test (timeout=500000)
+  @Test
+  @Timeout(value = 500)
   public void testSerialSubmit() throws Exception {
     policy = GridmixJobSubmissionPolicy.SERIAL;
     LOG.info("Serial started at " + System.currentTimeMillis());
@@ -69,7 +71,8 @@ public class TestLoadJob extends CommonJobTest {
   /*
    * test reply policy with LoadJob
    */
-  @Test  (timeout=500000)
+  @Test
+  @Timeout(value = 500)
   public void testReplaySubmit() throws Exception {
     policy = GridmixJobSubmissionPolicy.REPLAY;
     LOG.info(" Replay started at " + System.currentTimeMillis());

+ 31 - 24
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestPseudoLocalFs.java

@@ -17,7 +17,10 @@
  */
 package org.apache.hadoop.mapred.gridmix;
 
-import static org.junit.Assert.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -27,7 +30,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 /**
  * Test the basic functionality of PseudoLocalFs
@@ -55,7 +58,7 @@ public class TestPseudoLocalFs {
       ++totalSize;
     }
     in.close();
-    assertEquals("File size mismatch with read().", fileSize, totalSize);
+    assertEquals(fileSize, totalSize, "File size mismatch with read().");
 
     // Read data from PseudoLocalFs-based file into buffer to
     // validate read(byte[]) and file size.
@@ -67,7 +70,7 @@ public class TestPseudoLocalFs {
       totalSize += bytesRead;
       bytesRead = in.read(b);
     }
-    assertEquals("File size mismatch with read(byte[]).", fileSize, totalSize);
+    assertEquals(fileSize, totalSize, "File size mismatch with read(byte[]).");
   }
 
   /**
@@ -89,17 +92,18 @@ public class TestPseudoLocalFs {
       expectedExceptionSeen = true;
     }
     if (shouldSucceed) {
-      assertFalse("getFileStatus() has thrown Exception for valid file name "
-                  + path, expectedExceptionSeen);
-      assertNotNull("Missing file status for a valid file.", stat);
+      assertFalse(expectedExceptionSeen,
+          "getFileStatus() has thrown Exception for valid file name " + path);
+      assertNotNull(stat, "Missing file status for a valid file.");
 
       // validate fileSize
       String[] parts = path.toUri().getPath().split("\\.");
       long expectedFileSize = Long.parseLong(parts[parts.length - 1]);
-      assertEquals("Invalid file size.", expectedFileSize, stat.getLen());
+      assertEquals(expectedFileSize, stat.getLen(), "Invalid file size.");
     } else {
-      assertTrue("getFileStatus() did not throw Exception for invalid file "
-                 + " name " + path, expectedExceptionSeen);
+      assertTrue(expectedExceptionSeen,
+          "getFileStatus() did not throw Exception for invalid file "
+          + " name " + path);
     }
   }
 
@@ -121,11 +125,13 @@ public class TestPseudoLocalFs {
       expectedExceptionSeen = true;
     }
     if (shouldSucceed) {
-      assertFalse("create() has thrown Exception for valid file name "
-                  + path, expectedExceptionSeen);
+      assertFalse(expectedExceptionSeen,
+          "create() has thrown Exception for valid file name "
+          + path);
     } else {
-      assertTrue("create() did not throw Exception for invalid file name "
-                 + path, expectedExceptionSeen);
+      assertTrue(expectedExceptionSeen,
+          "create() did not throw Exception for invalid file name "
+          + path);
     }
   }
 
@@ -147,11 +153,13 @@ public class TestPseudoLocalFs {
       expectedExceptionSeen = true;
     }
     if (shouldSucceed) {
-      assertFalse("open() has thrown Exception for valid file name "
-                  + path, expectedExceptionSeen);
+      assertFalse(expectedExceptionSeen,
+          "open() has thrown Exception for valid file name "
+          + path);
     } else {
-      assertTrue("open() did not throw Exception for invalid file name "
-                 + path, expectedExceptionSeen);
+      assertTrue(expectedExceptionSeen,
+          "open() did not throw Exception for invalid file name "
+          + path);
     }
   }
 
@@ -168,9 +176,9 @@ public class TestPseudoLocalFs {
       boolean shouldSucceed) throws IOException {
     boolean ret = pfs.exists(path);
     if (shouldSucceed) {
-      assertTrue("exists() returned false for valid file name " + path, ret);
+      assertTrue(ret, "exists() returned false for valid file name " + path);
     } else {
-      assertFalse("exists() returned true for invalid file name " + path, ret);
+      assertFalse(ret, "exists() returned true for invalid file name " + path);
     }
   }
 
@@ -187,8 +195,8 @@ public class TestPseudoLocalFs {
 
     Path path = new Path("pseudo:///myPsedoFile.1234");
     FileSystem testFs = path.getFileSystem(conf);
-    assertEquals("Failed to obtain a pseudo local file system object from path",
-                 pfs.getUri().getScheme(), testFs.getUri().getScheme());
+    assertEquals(pfs.getUri().getScheme(), testFs.getUri().getScheme(),
+        "Failed to obtain a pseudo local file system object from path");
 
     // Validate PseudoLocalFS operations on URI of some other file system
     path = new Path("file:///myPsedoFile.12345");
@@ -214,8 +222,7 @@ public class TestPseudoLocalFs {
     long fileSize = 231456;
     path = PseudoLocalFs.generateFilePath("my.Psedo.File", fileSize);
     // Validate the above generateFilePath()
-    assertEquals("generateFilePath() failed.", fileSize,
-                 pfs.validateFileNameFormat(path));
+    assertEquals(fileSize, pfs.validateFileNameFormat(path), "generateFilePath() failed.");
 
     validateGetFileStatus(pfs, path, true);
     validateCreate(pfs, path, true);

+ 2 - 2
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestRandomAlgorithm.java

@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.mapred.gridmix;
 
-import static org.junit.Assert.*;
+import static org.junit.jupiter.api.Assertions.*;
 
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -28,7 +28,7 @@ import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 public class TestRandomAlgorithm {
   private static final int[][] parameters = new int[][] {

+ 9 - 7
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestRandomTextDataGenerator.java

@@ -23,8 +23,10 @@ import java.util.Set;
 
 import org.apache.hadoop.mapred.gridmix.RandomTextDataGenerator;
 
-import static org.junit.Assert.*;
-import org.junit.Test;
+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 org.junit.jupiter.api.Test;
 
 /**
  * Test {@link RandomTextDataGenerator}.
@@ -40,15 +42,15 @@ public class TestRandomTextDataGenerator {
     List<String> words = rtdg.getRandomWords();
 
     // check the size
-    assertEquals("List size mismatch", 10, words.size());
+    assertEquals(10, words.size(), "List size mismatch");
 
     // check the words
     Set<String> wordsSet = new HashSet<String>(words);
-    assertEquals("List size mismatch due to duplicates", 10, wordsSet.size());
+    assertEquals(10, wordsSet.size(), "List size mismatch due to duplicates");
 
     // check the word lengths
     for (String word : wordsSet) {
-      assertEquals("Word size mismatch", 5, word.length());
+      assertEquals(5, word.length(), "Word size mismatch");
     }
   }
   
@@ -64,7 +66,7 @@ public class TestRandomTextDataGenerator {
     RandomTextDataGenerator rtdg2 = new RandomTextDataGenerator(10, 0L, 5);
     List<String> words2 = rtdg2.getRandomWords();
     
-    assertTrue("List mismatch", words1.equals(words2));
+    assertTrue(words1.equals(words2), "List mismatch");
   }
   
   /**
@@ -79,6 +81,6 @@ public class TestRandomTextDataGenerator {
     RandomTextDataGenerator rtdg2 = new RandomTextDataGenerator(10, 0L, 5);
     Set<String> words2 = new HashSet(rtdg2.getRandomWords());
     
-    assertFalse("List size mismatch across lists", words1.equals(words2));
+    assertFalse(words1.equals(words2), "List size mismatch across lists");
   }
 }

+ 3 - 2
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestRecordFactory.java

@@ -18,10 +18,11 @@
  */
 package org.apache.hadoop.mapred.gridmix;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
 import java.util.Random;
 
-import org.junit.Test;
-import static org.junit.Assert.*;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

+ 40 - 36
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestResourceUsageEmulators.java

@@ -19,8 +19,9 @@ package org.apache.hadoop.mapred.gridmix;
 
 import java.io.IOException;
 
-import org.junit.Test;
-import static org.junit.Assert.*;
+import org.junit.jupiter.api.Test;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -188,11 +189,13 @@ public class TestResourceUsageEmulators {
     String id = TestResourceUsageEmulatorPlugin.DEFAULT_IDENTIFIER;
     long result = 
       TestResourceUsageEmulatorPlugin.testInitialization(id, conf);
-    assertTrue("Resource usage matcher failed to initialize the configured"
-               + " plugin", result > currentTime);
+    assertTrue(result > currentTime,
+        "Resource usage matcher failed to initialize the configured"
+        + " plugin");
     result = TestResourceUsageEmulatorPlugin.testEmulation(id, conf);
-    assertTrue("Resource usage matcher failed to load and emulate the"
-               + " configured plugin", result > currentTime);
+    assertTrue(result > currentTime,
+        "Resource usage matcher failed to load and emulate the"
+        + " configured plugin");
     
     // test plugin order to first emulate cpu and then others
     conf.setStrings(ResourceUsageMatcher.RESOURCE_USAGE_EMULATION_PLUGINS, 
@@ -206,8 +209,9 @@ public class TestResourceUsageEmulators {
     long time2 = 
            TestResourceUsageEmulatorPlugin.testInitialization(TestOthers.ID, 
                                                               conf);
-    assertTrue("Resource usage matcher failed to initialize the configured"
-               + " plugins in order", time1 < time2);
+    assertTrue(time1 < time2,
+        "Resource usage matcher failed to initialize the configured"
+        + " plugins in order");
     
     matcher.matchResourceUsage();
 
@@ -218,8 +222,8 @@ public class TestResourceUsageEmulators {
     time2 = 
       TestResourceUsageEmulatorPlugin.testInitialization(TestOthers.ID, 
                                                          conf);
-    assertTrue("Resource usage matcher failed to load the configured plugins", 
-               time1 < time2);
+    assertTrue(time1 < time2,
+        "Resource usage matcher failed to load the configured plugins");
   }
   
   /**
@@ -344,12 +348,13 @@ public class TestResourceUsageEmulators {
     String identifier = TestResourceUsageEmulatorPlugin.DEFAULT_IDENTIFIER;
     long initTime = 
       TestResourceUsageEmulatorPlugin.testInitialization(identifier, conf);
-    assertTrue("ResourceUsageMatcherRunner failed to initialize the"
-               + " configured plugin", initTime > currentTime);
+    assertTrue(initTime > currentTime,
+        "ResourceUsageMatcherRunner failed to initialize the"
+        + " configured plugin");
     
     // check the progress
-    assertEquals("Progress mismatch in ResourceUsageMatcherRunner", 
-                 0, progress.getProgress(), 0D);
+    assertEquals(0, progress.getProgress(), 0D,
+        "Progress mismatch in ResourceUsageMatcherRunner");
     
     // call match() and check progress
     progress.setProgress(0.01f);
@@ -357,8 +362,9 @@ public class TestResourceUsageEmulators {
     matcher.test();
     long emulateTime = 
       TestResourceUsageEmulatorPlugin.testEmulation(identifier, conf);
-    assertTrue("ProgressBasedResourceUsageMatcher failed to load and emulate"
-               + " the configured plugin", emulateTime > currentTime);
+    assertTrue(emulateTime > currentTime,
+        "ProgressBasedResourceUsageMatcher failed to load and emulate"
+        + " the configured plugin");
   }
   
   /**
@@ -379,14 +385,11 @@ public class TestResourceUsageEmulators {
     fakeCpuEmulator.calibrate(fakeMonitor, target);
     
     // by default, CpuUsageEmulator.calibrate() will consume 100ms of CPU usage
-    assertEquals("Fake calibration failed", 
-                 100, fakeMonitor.getCumulativeCpuTime());
-    assertEquals("Fake calibration failed", 
-                 100, fakeCpuEmulator.getCpuUsage());
+    assertEquals(100, fakeMonitor.getCumulativeCpuTime(), "Fake calibration failed");
+    assertEquals(100, fakeCpuEmulator.getCpuUsage(), "Fake calibration failed");
     // by default, CpuUsageEmulator.performUnitComputation() will be called 
     // twice
-    assertEquals("Fake calibration failed", 
-                 2, fakeCpuEmulator.getNumCalls());
+    assertEquals(2, fakeCpuEmulator.getNumCalls(), "Fake calibration failed");
   }
   
   /**
@@ -465,17 +468,18 @@ public class TestResourceUsageEmulators {
     long cpuUsagePost = fakeCore.getCpuUsage();
     
     //  test if no calls are made cpu usage emulator core
-    assertEquals("Disabled cumulative CPU usage emulation plugin works!", 
-                 numCallsPre, numCallsPost);
+    assertEquals(numCallsPre, numCallsPost,
+        "Disabled cumulative CPU usage emulation plugin works!");
     
     //  test if no calls are made cpu usage emulator core
-    assertEquals("Disabled cumulative CPU usage emulation plugin works!", 
-                 cpuUsagePre, cpuUsagePost);
+    assertEquals(cpuUsagePre, cpuUsagePost,
+        "Disabled cumulative CPU usage emulation plugin works!");
     
     // test with get progress
     float progress = cpuPlugin.getProgress();
-    assertEquals("Invalid progress of disabled cumulative CPU usage emulation " 
-                 + "plugin!", 1.0f, progress, 0f);
+    assertEquals(1.0f, progress, 0f,
+        "Invalid progress of disabled cumulative CPU usage emulation "
+        + "plugin!");
     
     // test with valid resource usage value
     ResourceUsageMetrics metrics = createMetrics(targetCpuUsage);
@@ -590,11 +594,11 @@ public class TestResourceUsageEmulators {
     }
     
     // test if the resource plugin shows the expected invocations
-    assertEquals("Cumulative cpu usage emulator plugin failed (num calls)!", 
-                 expectedTotalNumCalls, fakeCore.getNumCalls(), 0L);
+    assertEquals(expectedTotalNumCalls, fakeCore.getNumCalls(), 0L,
+        "Cumulative cpu usage emulator plugin failed (num calls)!");
     // test if the resource plugin shows the expected usage
-    assertEquals("Cumulative cpu usage emulator plugin failed (total usage)!", 
-                 expectedTotalCpuUsage, fakeCore.getCpuUsage(), 0L);
+    assertEquals(expectedTotalCpuUsage, fakeCore.getCpuUsage(), 0L,
+        "Cumulative cpu usage emulator plugin failed (total usage)!");
   }
   
   // tests if the CPU usage emulation plugin emulates only at the expected
@@ -606,9 +610,9 @@ public class TestResourceUsageEmulators {
     fakeProgress.setProgress(progress);
     cpuPlugin.emulate();
     
-    assertEquals("Emulation interval test for cpu usage failed " + info + "!", 
-                 expectedTotalCpuUsage, fakeCore.getCpuUsage(), 0L);
-    assertEquals("Emulation interval test for num calls failed " + info + "!", 
-                 expectedTotalNumCalls, fakeCore.getNumCalls(), 0L);
+    assertEquals(expectedTotalCpuUsage, fakeCore.getCpuUsage(), 0L,
+        "Emulation interval test for cpu usage failed " + info + "!");
+    assertEquals(expectedTotalNumCalls, fakeCore.getNumCalls(), 0L,
+        "Emulation interval test for num calls failed " + info + "!");
   }
 }

+ 6 - 6
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestSleepJob.java

@@ -24,16 +24,16 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.tools.rumen.JobStory;
-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 org.slf4j.Logger;
 import org.slf4j.event.Level;
 
 import java.io.IOException;
 import java.util.List;
 
-import static org.junit.Assert.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.slf4j.LoggerFactory.getLogger;
 
 public class TestSleepJob extends CommonJobTest {
@@ -47,12 +47,12 @@ public class TestSleepJob extends CommonJobTest {
 
   static GridmixJobSubmissionPolicy policy = GridmixJobSubmissionPolicy.REPLAY;
 
-  @BeforeClass
+  @BeforeAll
   public static void init() throws IOException {
     GridmixTestUtils.initCluster(TestSleepJob.class);
   }
 
-  @AfterClass
+  @AfterAll
   public static void shutDown() throws IOException {
     GridmixTestUtils.shutdownCluster();
   }

+ 9 - 7
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestUserResolve.java

@@ -20,9 +20,11 @@ package org.apache.hadoop.mapred.gridmix;
 import java.io.IOException;
 import java.net.URI;
 
-import org.junit.BeforeClass;
-import org.junit.Test;
-import static org.junit.Assert.*;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -36,7 +38,7 @@ public class TestUserResolve {
   private static Configuration conf = null;
   private static FileSystem fs = null;
 
-  @BeforeClass
+  @BeforeAll
   public static void createRootDir() throws IOException {
     conf = new Configuration();
     fs = FileSystem.getLocal(conf);
@@ -81,11 +83,11 @@ public class TestUserResolve {
     try {
       rslv.setTargetUsers(userRsrc, conf);
     } catch (IOException e) {
-      assertTrue("Exception message from RoundRobinUserResolver is wrong",
-          e.getMessage().equals(expectedErrorMsg));
+      assertTrue(e.getMessage().equals(expectedErrorMsg),
+          "Exception message from RoundRobinUserResolver is wrong");
       fail = true;
     }
-    assertTrue("User list required for RoundRobinUserResolver", fail);
+    assertTrue(fail, "User list required for RoundRobinUserResolver");
   }
 
   /**