Sfoglia il codice sorgente

HADOOP-19431. [JDK17] Upgrade JUnit from 4 to 5 in hadoop-distcp. (#7368)

Co-authored-by: Chris Nauroth <cnauroth@apache.org>
Reviewed-by: Chris Nauroth <cnauroth@apache.org>
Signed-off-by: Shilun Fan <slfan1989@apache.org>
slfan1989 2 mesi fa
parent
commit
c5fdc3ba6c
33 ha cambiato i file con 1281 aggiunte e 1131 eliminazioni
  1. 13 2
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/SimpleCopyListing.java
  2. 8 8
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestCopyFilter.java
  3. 104 71
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestCopyListing.java
  4. 2 2
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestCopyListingFileStatus.java
  5. 112 102
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpOptions.java
  6. 46 46
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSync.java
  7. 32 28
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSyncReverseBase.java
  8. 26 26
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSystem.java
  9. 9 7
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpViewFs.java
  10. 8 7
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpWithAcls.java
  11. 28 35
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpWithRawXAttrs.java
  12. 5 5
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpWithXAttrs.java
  13. 37 30
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestExternalCall.java
  14. 25 23
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestFileBasedCopyListing.java
  15. 9 8
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestGlobbedCopyListing.java
  16. 122 77
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestIntegration.java
  17. 160 152
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestOptionsParser.java
  18. 9 6
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestRegexCopyFilter.java
  19. 11 11
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestRegexpInConfigurationFilter.java
  20. 5 4
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestTrueCopyFilter.java
  21. 107 90
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java
  22. 2 2
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapperCompositeCrc.java
  23. 23 20
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyOutputFormat.java
  24. 23 20
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestDeletedDirTracker.java
  25. 16 15
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestRetriableFileCopyCommand.java
  26. 12 10
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestUniformSizeInputFormat.java
  27. 32 27
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/lib/TestDynamicInputFormat.java
  28. 6 7
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/util/DistCpTestUtils.java
  29. 233 239
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/util/TestDistCpUtils.java
  30. 17 20
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/util/TestDistCpUtilsWithCombineMode.java
  31. 18 12
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/util/TestProducerConsumer.java
  32. 9 8
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/util/TestRetriableCommand.java
  33. 12 11
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/util/TestThrottledInputStream.java

+ 13 - 2
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/SimpleCopyListing.java

@@ -75,8 +75,8 @@ public class SimpleCopyListing extends CopyListing {
   private long totalDirs = 0;
   private long totalBytesToCopy = 0;
   private int numListstatusThreads = 1;
-  private final int fileStatusLimit;
-  private final boolean randomizeFileListing;
+  private int fileStatusLimit;
+  private boolean randomizeFileListing;
   private final int maxRetries = 3;
   private CopyFilter copyFilter;
   private DistCpSync distCpSync;
@@ -119,6 +119,17 @@ public class SimpleCopyListing extends CopyListing {
     this.randomizeFileListing = randomizeFileListing;
   }
 
+  @VisibleForTesting
+  protected void initSimpleCopyListing(Configuration pConfiguration,
+      Credentials pCredentials, int pNumListstatusThreads, int pFileStatusLimit,
+      boolean pRandomizeFileListing) {
+    setConf(pConfiguration);
+    setCredentials(pCredentials);
+    this.numListstatusThreads = pNumListstatusThreads;
+    this.fileStatusLimit = Math.max(1, pFileStatusLimit);
+    this.randomizeFileListing = pRandomizeFileListing;
+  }
+
   protected SimpleCopyListing(Configuration configuration,
                               Credentials credentials,
                               DistCpSync distCpSync) {

+ 8 - 8
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestCopyFilter.java

@@ -18,12 +18,12 @@
 
 package org.apache.hadoop.tools;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.conf.Configuration;
 
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 /**
  * Test {@link CopyFilter}.
@@ -34,8 +34,8 @@ public class TestCopyFilter {
   public void testGetCopyFilterTrueCopyFilter() {
     Configuration configuration = new Configuration(false);
     CopyFilter copyFilter = CopyFilter.getCopyFilter(configuration);
-    assertTrue("copyFilter should be instance of TrueCopyFilter",
-            copyFilter instanceof TrueCopyFilter);
+    assertTrue(copyFilter instanceof TrueCopyFilter,
+        "copyFilter should be instance of TrueCopyFilter");
   }
 
   @Test
@@ -43,8 +43,8 @@ public class TestCopyFilter {
     Configuration configuration = new Configuration(false);
     configuration.set(DistCpConstants.CONF_LABEL_FILTERS_FILE, "random");
     CopyFilter copyFilter = CopyFilter.getCopyFilter(configuration);
-    assertTrue("copyFilter should be instance of RegexCopyFilter",
-            copyFilter instanceof RegexCopyFilter);
+    assertTrue(copyFilter instanceof RegexCopyFilter,
+        "copyFilter should be instance of RegexCopyFilter");
   }
 
   @Test
@@ -54,8 +54,8 @@ public class TestCopyFilter {
     Configuration configuration = new Configuration(false);
     configuration.set(DistCpConstants.CONF_LABEL_FILTERS_CLASS, filterName);
     CopyFilter copyFilter = CopyFilter.getCopyFilter(configuration);
-    assertTrue("copyFilter should be instance of RegexpInConfigurationFilter",
-            copyFilter instanceof RegexpInConfigurationFilter);
+    assertTrue(copyFilter instanceof RegexpInConfigurationFilter,
+        "copyFilter should be instance of RegexpInConfigurationFilter");
   }
 
   @Test

+ 104 - 71
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestCopyListing.java

@@ -18,7 +18,14 @@
 
 package org.apache.hadoop.tools;
 
-import static org.mockito.Mockito.*;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
 
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.mockito.Mockito;
@@ -33,14 +40,11 @@ import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Text;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-import org.junit.Test;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.AfterClass;
-import static org.assertj.core.api.Assertions.assertThat;
+import org.junit.jupiter.api.Timeout;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 
 import java.io.File;
 import java.io.IOException;
@@ -52,7 +56,6 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Random;
 
-@RunWith(value = Parameterized.class)
 public class TestCopyListing extends SimpleCopyListing {
   private static final Logger LOG = LoggerFactory.getLogger(TestCopyListing.class);
 
@@ -61,31 +64,31 @@ public class TestCopyListing extends SimpleCopyListing {
   private static final Configuration config = new Configuration();
   private static MiniDFSCluster cluster;
 
-  @BeforeClass
+  @BeforeAll
   public static void create() throws IOException {
     cluster = new MiniDFSCluster.Builder(config).numDataNodes(1).format(true)
                                                 .build();
   }
 
-  @AfterClass
+  @AfterAll
   public static void destroy() {
     if (cluster != null) {
       cluster.shutdown();
     }
   }
 
-  @Parameters
   public static Collection<Object[]> data() {
-    Object[][] data = new Object[][] { { 1 }, { 2 }, { 10 }, { 20} };
+    Object[][] data = new Object[][]{{1}, {2}, {10}, {20}};
     return Arrays.asList(data);
   }
 
-  public TestCopyListing(int numListstatusThreads) {
-    super(config, CREDENTIALS, numListstatusThreads, 0, false);
+  public TestCopyListing() {
+    super(config, CREDENTIALS, 1, 0, false);
   }
 
-  protected TestCopyListing(Configuration configuration) {
-    super(configuration, CREDENTIALS);
+  public void initTestCopyListing(int numListstatusThreads) {
+    initSimpleCopyListing(config, CREDENTIALS,
+        numListstatusThreads, 0, false);
   }
 
   @Override
@@ -98,8 +101,12 @@ public class TestCopyListing extends SimpleCopyListing {
     return 0;
   }
 
-  @Test(timeout=10000)
-  public void testMultipleSrcToFile() {
+
+  @Timeout(value = 10)
+  @ParameterizedTest
+  @MethodSource("data")
+  public void testMultipleSrcToFile(int pNumListstatusThreads) {
+    initTestCopyListing(pNumListstatusThreads);
     FileSystem fs = null;
     try {
       fs = FileSystem.get(getConf());
@@ -119,7 +126,7 @@ public class TestCopyListing extends SimpleCopyListing {
       fs.create(target).close();
       try {
         validatePaths(new DistCpContext(options));
-        Assert.fail("Invalid inputs accepted");
+        fail("Invalid inputs accepted");
       } catch (InvalidInputException ignore) { }
       TestDistCpUtils.delete(fs, "/tmp");
 
@@ -129,19 +136,22 @@ public class TestCopyListing extends SimpleCopyListing {
       fs.create(target).close();
       try {
         validatePaths(new DistCpContext(options));
-        Assert.fail("Invalid inputs accepted");
+        fail("Invalid inputs accepted");
       } catch (InvalidInputException ignore) { }
       TestDistCpUtils.delete(fs, "/tmp");
     } catch (IOException e) {
       LOG.error("Exception encountered ", e);
-      Assert.fail("Test input validation failed");
+      fail("Test input validation failed");
     } finally {
       TestDistCpUtils.delete(fs, "/tmp");
     }
   }
 
-  @Test(timeout=10000)
-  public void testDuplicates() {
+  @ParameterizedTest
+  @Timeout(value = 10)
+  @MethodSource("data")
+  public void testDuplicates(int pNumListstatusThreads) {
+    initTestCopyListing(pNumListstatusThreads);
     FileSystem fs = null;
     try {
       fs = FileSystem.get(getConf());
@@ -158,38 +168,47 @@ public class TestCopyListing extends SimpleCopyListing {
           context);
       try {
         listing.buildListing(listingFile, context);
-        Assert.fail("Duplicates not detected");
+        fail("Duplicates not detected");
       } catch (DuplicateFileException ignore) {
       }
     } catch (IOException e) {
       LOG.error("Exception encountered in test", e);
-      Assert.fail("Test failed " + e.getMessage());
+      fail("Test failed " + e.getMessage());
     } finally {
       TestDistCpUtils.delete(fs, "/tmp");
     }
   }
 
-  @Test(expected = DuplicateFileException.class, timeout = 10000)
-  public void testDiffBasedSimpleCopyListing() throws IOException {
-    FileSystem fs = null;
-    Configuration configuration = getConf();
-    DistCpSync distCpSync = Mockito.mock(DistCpSync.class);
-    Path listingFile = new Path("/tmp/list");
-    // Throws DuplicateFileException as it recursively traverses src3 directory
-    // and also adds 3.txt,4.txt twice
-    configuration.setBoolean(
-        DistCpConstants.CONF_LABEL_DIFF_COPY_LISTING_TRAVERSE_DIRECTORY, true);
-    try {
-      fs = FileSystem.get(getConf());
-      buildListingUsingSnapshotDiff(fs, configuration, distCpSync, listingFile);
-    } finally {
-      TestDistCpUtils.delete(fs, "/tmp");
-    }
+  @ParameterizedTest
+  @Timeout(value = 10)
+  @MethodSource("data")
+  public void testDiffBasedSimpleCopyListing(int pNumListstatusThreads)
+      throws IOException {
+    initTestCopyListing(pNumListstatusThreads);
+    assertThrows(DuplicateFileException.class, () -> {
+      FileSystem fs = null;
+      Configuration configuration = getConf();
+      DistCpSync distCpSync = Mockito.mock(DistCpSync.class);
+      Path listingFile = new Path("/tmp/list");
+      // Throws DuplicateFileException as it recursively traverses src3 directory
+      // and also adds 3.txt,4.txt twice
+      configuration.setBoolean(
+          DistCpConstants.CONF_LABEL_DIFF_COPY_LISTING_TRAVERSE_DIRECTORY, true);
+      try {
+        fs = FileSystem.get(getConf());
+        buildListingUsingSnapshotDiff(fs, configuration, distCpSync, listingFile);
+      } finally {
+        TestDistCpUtils.delete(fs, "/tmp");
+      }
+    });
   }
 
-  @Test(timeout=10000)
-  public void testDiffBasedSimpleCopyListingWithoutTraverseDirectory()
-      throws IOException {
+  @ParameterizedTest
+  @Timeout(value = 10)
+  @MethodSource("data")
+  public void testDiffBasedSimpleCopyListingWithoutTraverseDirectory(
+      int pNumListstatusThreads) throws IOException {
+    initTestCopyListing(pNumListstatusThreads);
     FileSystem fs = null;
     Configuration configuration = getConf();
     DistCpSync distCpSync = Mockito.mock(DistCpSync.class);
@@ -235,8 +254,10 @@ public class TestCopyListing extends SimpleCopyListing {
     listing.buildListing(listingFile, context);
   }
 
-  @Test
-  public void testDuplicateSourcePaths() throws Exception {
+  @ParameterizedTest
+  @MethodSource("data")
+  public void testDuplicateSourcePaths(int pNumListstatusThreads) throws Exception {
+    initTestCopyListing(pNumListstatusThreads);
     FileSystem fs = FileSystem.get(getConf());
     List<Path> srcPaths = new ArrayList<Path>();
     try {
@@ -252,14 +273,17 @@ public class TestCopyListing extends SimpleCopyListing {
       CopyListing listing =
           CopyListing.getCopyListing(getConf(), CREDENTIALS, context);
       listing.buildListing(listingFile, context);
-      Assert.assertTrue(fs.exists(listingFile));
+      assertTrue(fs.exists(listingFile));
     } finally {
       TestDistCpUtils.delete(fs, "/tmp");
     }
   }
 
-  @Test(timeout=10000)
-  public void testBuildListing() {
+  @ParameterizedTest
+  @Timeout(value = 10)
+  @MethodSource("data")
+  public void testBuildListing(int pNumListstatusThreads) {
+    initTestCopyListing(pNumListstatusThreads);
     FileSystem fs = null;
     try {
       fs = FileSystem.get(getConf());
@@ -294,7 +318,7 @@ public class TestCopyListing extends SimpleCopyListing {
       CopyListing listing = new SimpleCopyListing(getConf(), CREDENTIALS);
       try {
         listing.buildListing(listingFile, new DistCpContext(options));
-        Assert.fail("Duplicates not detected");
+        fail("Duplicates not detected");
       } catch (DuplicateFileException ignore) {
       }
       assertThat(listing.getBytesToCopy()).isEqualTo(10);
@@ -303,20 +327,24 @@ public class TestCopyListing extends SimpleCopyListing {
 
       try {
         listing.buildListing(listingFile, new DistCpContext(options));
-        Assert.fail("Invalid input not detected");
+        fail("Invalid input not detected");
       } catch (InvalidInputException ignore) {
       }
       TestDistCpUtils.delete(fs, "/tmp");
     } catch (IOException e) {
       LOG.error("Exception encountered ", e);
-      Assert.fail("Test build listing failed");
+      fail("Test build listing failed");
     } finally {
       TestDistCpUtils.delete(fs, "/tmp");
     }
   }
 
-  @Test(timeout=60000)
-  public void testWithRandomFileListing() throws IOException {
+  @ParameterizedTest
+  @Timeout(value = 60)
+  @MethodSource("data")
+  public void testWithRandomFileListing(int pNumListstatusThreads)
+      throws IOException {
+    initTestCopyListing(pNumListstatusThreads);
     FileSystem fs = null;
     try {
       fs = FileSystem.get(getConf());
@@ -346,7 +374,7 @@ public class TestCopyListing extends SimpleCopyListing {
       SimpleCopyListing listing = new SimpleCopyListing(getConf(), CREDENTIALS);
       listing.buildListing(listingFile, new DistCpContext(options));
 
-      Assert.assertEquals(listing.getNumberOfPaths(), pathCount);
+      assertEquals(listing.getNumberOfPaths(), pathCount);
       validateFinalListing(listingFile, srcFiles);
       fs.delete(listingFile, true);
 
@@ -359,7 +387,7 @@ public class TestCopyListing extends SimpleCopyListing {
       long seed = System.nanoTime();
       listing.setSeedForRandomListing(seed);
       listing.buildListing(listingFile, new DistCpContext(options));
-      Assert.assertEquals(listing.getNumberOfPaths(), pathCount);
+      assertEquals(listing.getNumberOfPaths(), pathCount);
 
       // validate randomness
       Collections.shuffle(srcFiles, new Random(seed));
@@ -381,9 +409,9 @@ public class TestCopyListing extends SimpleCopyListing {
       int idx = 0;
       while (reader.next(currentKey)) {
         reader.getCurrentValue(currentVal);
-        Assert.assertEquals("srcFiles.size=" + srcFiles.size()
-                + ", idx=" + idx, fs.makeQualified(srcFiles.get(idx)),
-            currentVal.getPath());
+        assertEquals(fs.makeQualified(srcFiles.get(idx)),
+            currentVal.getPath(), "srcFiles.size=" + srcFiles.size() +
+            ", idx=" + idx);
         if (LOG.isDebugEnabled()) {
           LOG.debug("val=" + fs.makeQualified(srcFiles.get(idx)));
         }
@@ -393,8 +421,11 @@ public class TestCopyListing extends SimpleCopyListing {
   }
 
 
-  @Test(timeout=10000)
-  public void testBuildListingForSingleFile() {
+  @ParameterizedTest
+  @Timeout(value = 10)
+  @MethodSource("data")
+  public void testBuildListingForSingleFile(int pNumListstatusThreads) {
+    initTestCopyListing(pNumListstatusThreads);
     FileSystem fs = null;
     String testRootString = "/singleFileListing";
     Path testRoot = new Path(testRootString);
@@ -426,11 +457,11 @@ public class TestCopyListing extends SimpleCopyListing {
 
       CopyListingFileStatus fileStatus = new CopyListingFileStatus();
       Text relativePath = new Text();
-      Assert.assertTrue(reader.next(relativePath, fileStatus));
-      Assert.assertTrue(relativePath.toString().equals(""));
+      assertTrue(reader.next(relativePath, fileStatus));
+      assertTrue(relativePath.toString().equals(""));
     }
     catch (Exception e) {
-      Assert.fail("Unexpected exception encountered.");
+      fail("Unexpected exception encountered.");
       LOG.error("Unexpected exception: ", e);
     }
     finally {
@@ -438,9 +469,11 @@ public class TestCopyListing extends SimpleCopyListing {
       IOUtils.closeStream(reader);
     }
   }
-  
-  @Test
-  public void testFailOnCloseError() throws IOException {
+
+  @ParameterizedTest
+  @MethodSource("data")
+  public void testFailOnCloseError(int pNumListstatusThreads) throws IOException {
+    initTestCopyListing(pNumListstatusThreads);
     File inFile = File.createTempFile("TestCopyListingIn", null);
     inFile.deleteOnExit();
     File outFile = File.createTempFile("TestCopyListingOut", null);
@@ -461,7 +494,7 @@ public class TestCopyListing extends SimpleCopyListing {
     } catch (Exception e) {
       actualEx = e;
     }
-    Assert.assertNotNull("close writer didn't fail", actualEx);
-    Assert.assertEquals(expectedEx, actualEx);
+    assertNotNull(actualEx, "close writer didn't fail");
+    assertEquals(expectedEx, actualEx);
   }
 }

+ 2 - 2
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestCopyListingFileStatus.java

@@ -23,8 +23,8 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 
-import org.junit.Test;
-import static org.junit.Assert.assertEquals;
+import org.junit.jupiter.api.Test;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 /**
  * Verify CopyListingFileStatus serialization and requirements for distcp.

+ 112 - 102
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpOptions.java

@@ -21,15 +21,20 @@ package org.apache.hadoop.tools;
 import java.util.Collections;
 
 import org.apache.hadoop.conf.Configuration;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.tools.DistCpOptions.FileAttribute;
 
 import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.apache.hadoop.tools.DistCpOptions.MAX_NUM_LISTSTATUS_THREADS;
-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.assertNotSame;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 /**
  * This is to test constructing {@link DistCpOptions} manually with setters.
@@ -46,10 +51,10 @@ public class TestDistCpOptions {
     final DistCpOptions.Builder builder = new DistCpOptions.Builder(
         Collections.singletonList(new Path("hdfs://localhost:8020/source")),
         new Path("hdfs://localhost:8020/target/"));
-    Assert.assertFalse(builder.build().shouldIgnoreFailures());
+    assertFalse(builder.build().shouldIgnoreFailures());
 
     builder.withIgnoreFailures(true);
-    Assert.assertTrue(builder.build().shouldIgnoreFailures());
+    assertTrue(builder.build().shouldIgnoreFailures());
   }
 
   @Test
@@ -57,14 +62,14 @@ public class TestDistCpOptions {
     final DistCpOptions.Builder builder = new DistCpOptions.Builder(
         Collections.singletonList(new Path("hdfs://localhost:8020/source")),
         new Path("hdfs://localhost:8020/target/"));
-    Assert.assertFalse(builder.build().shouldOverwrite());
+    assertFalse(builder.build().shouldOverwrite());
 
     builder.withOverwrite(true);
-    Assert.assertTrue(builder.build().shouldOverwrite());
+    assertTrue(builder.build().shouldOverwrite());
 
     try {
       builder.withSyncFolder(true).build();
-      Assert.fail("Update and overwrite aren't allowed together");
+      fail("Update and overwrite aren't allowed together");
     } catch (IllegalArgumentException ignore) {
     }
   }
@@ -74,11 +79,11 @@ public class TestDistCpOptions {
     final DistCpOptions.Builder builder = new DistCpOptions.Builder(
         Collections.singletonList(new Path("hdfs://localhost:8020/source")),
         new Path("hdfs://localhost:8020/target/"));
-    Assert.assertNull(builder.build().getLogPath());
+    assertNull(builder.build().getLogPath());
 
     final Path logPath = new Path("hdfs://localhost:8020/logs");
     builder.withLogPath(logPath);
-    Assert.assertEquals(logPath, builder.build().getLogPath());
+    assertEquals(logPath, builder.build().getLogPath());
   }
 
   @Test
@@ -86,10 +91,10 @@ public class TestDistCpOptions {
     final DistCpOptions.Builder builder = new DistCpOptions.Builder(
         Collections.singletonList(new Path("hdfs://localhost:8020/source")),
         new Path("hdfs://localhost:8020/target/"));
-    Assert.assertTrue(builder.build().shouldBlock());
+    assertTrue(builder.build().shouldBlock());
 
     builder.withBlocking(false);
-    Assert.assertFalse(builder.build().shouldBlock());
+    assertFalse(builder.build().shouldBlock());
   }
 
   @Test
@@ -97,28 +102,32 @@ public class TestDistCpOptions {
     final DistCpOptions.Builder builder = new DistCpOptions.Builder(
         Collections.singletonList(new Path("hdfs://localhost:8020/source")),
         new Path("hdfs://localhost:8020/target/"));
-    Assert.assertEquals(0, builder.build().getMapBandwidth(), DELTA);
+    assertEquals(0, builder.build().getMapBandwidth(), DELTA);
 
     builder.withMapBandwidth(11);
-    Assert.assertEquals(11, builder.build().getMapBandwidth(), DELTA);
+    assertEquals(11, builder.build().getMapBandwidth(), DELTA);
   }
 
-  @Test(expected = IllegalArgumentException.class)
+  @Test
   public void testSetNonPositiveBandwidth() {
-    new DistCpOptions.Builder(
-        Collections.singletonList(new Path("hdfs://localhost:8020/source")),
-        new Path("hdfs://localhost:8020/target/"))
-        .withMapBandwidth(-11)
-        .build();
+    assertThrows(IllegalArgumentException.class, ()->{
+      new DistCpOptions.Builder(
+              Collections.singletonList(new Path("hdfs://localhost:8020/source")),
+              new Path("hdfs://localhost:8020/target/"))
+              .withMapBandwidth(-11)
+              .build();
+    });
   }
 
-  @Test(expected = IllegalArgumentException.class)
+  @Test
   public void testSetZeroBandwidth() {
-    new DistCpOptions.Builder(
-        Collections.singletonList(new Path("hdfs://localhost:8020/source")),
-        new Path("hdfs://localhost:8020/target/"))
-        .withMapBandwidth(0)
-        .build();
+    assertThrows(IllegalArgumentException.class, () -> {
+      new DistCpOptions.Builder(
+            Collections.singletonList(new Path("hdfs://localhost:8020/source")),
+            new Path("hdfs://localhost:8020/target/"))
+            .withMapBandwidth(0)
+            .build();
+    });
   }
 
   @Test
@@ -126,12 +135,12 @@ public class TestDistCpOptions {
     final DistCpOptions.Builder builder = new DistCpOptions.Builder(
         Collections.singletonList(new Path("hdfs://localhost:8020/source")),
         new Path("hdfs://localhost:8020/target/"));
-    Assert.assertFalse(builder.build().shouldSkipCRC());
+    assertFalse(builder.build().shouldSkipCRC());
 
     final DistCpOptions options = builder.withSyncFolder(true).withSkipCRC(true)
         .build();
-    Assert.assertTrue(options.shouldSyncFolder());
-    Assert.assertTrue(options.shouldSkipCRC());
+    assertTrue(options.shouldSyncFolder());
+    assertTrue(options.shouldSkipCRC());
   }
 
   @Test
@@ -139,14 +148,14 @@ public class TestDistCpOptions {
     final DistCpOptions.Builder builder = new DistCpOptions.Builder(
         Collections.singletonList(new Path("hdfs://localhost:8020/source")),
         new Path("hdfs://localhost:8020/target/"));
-    Assert.assertFalse(builder.build().shouldAtomicCommit());
+    assertFalse(builder.build().shouldAtomicCommit());
 
     builder.withAtomicCommit(true);
-    Assert.assertTrue(builder.build().shouldAtomicCommit());
+    assertTrue(builder.build().shouldAtomicCommit());
 
     try {
       builder.withSyncFolder(true).build();
-      Assert.fail("Atomic and sync folders were mutually exclusive");
+      fail("Atomic and sync folders were mutually exclusive");
     } catch (IllegalArgumentException ignore) {
     }
   }
@@ -156,14 +165,14 @@ public class TestDistCpOptions {
     final DistCpOptions.Builder builder = new DistCpOptions.Builder(
         Collections.singletonList(new Path("hdfs://localhost:8020/source")),
         new Path("hdfs://localhost:8020/target/"));
-    Assert.assertNull(builder.build().getAtomicWorkPath());
+    assertNull(builder.build().getAtomicWorkPath());
 
     builder.withAtomicCommit(true);
-    Assert.assertNull(builder.build().getAtomicWorkPath());
+    assertNull(builder.build().getAtomicWorkPath());
 
     final Path workPath = new Path("hdfs://localhost:8020/work");
     builder.withAtomicWorkPath(workPath);
-    Assert.assertEquals(workPath, builder.build().getAtomicWorkPath());
+    assertEquals(workPath, builder.build().getAtomicWorkPath());
   }
 
   @Test
@@ -171,10 +180,10 @@ public class TestDistCpOptions {
     final DistCpOptions.Builder builder = new DistCpOptions.Builder(
         Collections.singletonList(new Path("hdfs://localhost:8020/source")),
         new Path("hdfs://localhost:8020/target/"));
-    Assert.assertFalse(builder.build().shouldSyncFolder());
+    assertFalse(builder.build().shouldSyncFolder());
 
     builder.withSyncFolder(true);
-    Assert.assertTrue(builder.build().shouldSyncFolder());
+    assertTrue(builder.build().shouldSyncFolder());
   }
 
   @Test
@@ -182,13 +191,13 @@ public class TestDistCpOptions {
     final DistCpOptions.Builder builder = new DistCpOptions.Builder(
         Collections.singletonList(new Path("hdfs://localhost:8020/source")),
         new Path("hdfs://localhost:8020/target/"));
-    Assert.assertFalse(builder.build().shouldDeleteMissing());
+    assertFalse(builder.build().shouldDeleteMissing());
 
     DistCpOptions options = builder.withSyncFolder(true)
         .withDeleteMissing(true)
         .build();
-    Assert.assertTrue(options.shouldSyncFolder());
-    Assert.assertTrue(options.shouldDeleteMissing());
+    assertTrue(options.shouldSyncFolder());
+    assertTrue(options.shouldDeleteMissing());
 
     options = new DistCpOptions.Builder(
         Collections.singletonList(new Path("hdfs://localhost:8020/source")),
@@ -196,8 +205,8 @@ public class TestDistCpOptions {
         .withOverwrite(true)
         .withDeleteMissing(true)
         .build();
-    Assert.assertTrue(options.shouldOverwrite());
-    Assert.assertTrue(options.shouldDeleteMissing());
+    assertTrue(options.shouldOverwrite());
+    assertTrue(options.shouldDeleteMissing());
 
     try {
       new DistCpOptions.Builder(
@@ -230,14 +239,14 @@ public class TestDistCpOptions {
     final DistCpOptions.Builder builder = new DistCpOptions.Builder(
         Collections.singletonList(new Path("hdfs://localhost:8020/source")),
         new Path("hdfs://localhost:8020/target/"));
-    Assert.assertEquals(DistCpConstants.DEFAULT_MAPS,
+    assertEquals(DistCpConstants.DEFAULT_MAPS,
         builder.build().getMaxMaps());
 
     builder.maxMaps(1);
-    Assert.assertEquals(1, builder.build().getMaxMaps());
+    assertEquals(1, builder.build().getMaxMaps());
 
     builder.maxMaps(0);
-    Assert.assertEquals(1, builder.build().getMaxMaps());
+    assertEquals(1, builder.build().getMaxMaps());
   }
 
   @Test
@@ -247,17 +256,17 @@ public class TestDistCpOptions {
         new Path("hdfs://localhost:8020/target/"));
     // If command line argument isn't set, we expect .getNumListstatusThreads
     // option to be zero (so that we know when to override conf properties).
-    Assert.assertEquals(0, builder.build().getNumListstatusThreads());
+    assertEquals(0, builder.build().getNumListstatusThreads());
 
     builder.withNumListstatusThreads(12);
-    Assert.assertEquals(12, builder.build().getNumListstatusThreads());
+    assertEquals(12, builder.build().getNumListstatusThreads());
 
     builder.withNumListstatusThreads(0);
-    Assert.assertEquals(0, builder.build().getNumListstatusThreads());
+    assertEquals(0, builder.build().getNumListstatusThreads());
 
     // Ignore large number of threads.
     builder.withNumListstatusThreads(MAX_NUM_LISTSTATUS_THREADS * 2);
-    Assert.assertEquals(MAX_NUM_LISTSTATUS_THREADS,
+    assertEquals(MAX_NUM_LISTSTATUS_THREADS,
         builder.build().getNumListstatusThreads());
   }
 
@@ -266,14 +275,16 @@ public class TestDistCpOptions {
     final DistCpOptions.Builder builder = new DistCpOptions.Builder(
         new Path("hdfs://localhost:8020/source/first"),
         new Path("hdfs://localhost:8020/target/"));
-    Assert.assertEquals(new Path("hdfs://localhost:8020/source/first"),
+    assertEquals(new Path("hdfs://localhost:8020/source/first"),
         builder.build().getSourceFileListing());
   }
 
-  @Test(expected = IllegalArgumentException.class)
+  @Test
   public void testMissingTarget() {
-    new DistCpOptions.Builder(new Path("hdfs://localhost:8020/source/first"),
-        null);
+    assertThrows(IllegalArgumentException.class, ()->{
+      new DistCpOptions.Builder(new Path("hdfs://localhost:8020/source/first"),
+              null);
+    });
   }
 
   @Test
@@ -291,8 +302,8 @@ public class TestDistCpOptions {
         "blocksPerChunk=0, copyBufferSize=8192, verboseLog=false, " +
         "directWrite=false, useiterator=false, updateRoot=false}";
     String optionString = option.toString();
-    Assert.assertEquals(val, optionString);
-    Assert.assertNotSame(DistCpOptionSwitch.ATOMIC_COMMIT.toString(),
+    assertEquals(val, optionString);
+    assertNotSame(DistCpOptionSwitch.ATOMIC_COMMIT.toString(),
         DistCpOptionSwitch.ATOMIC_COMMIT.name());
   }
 
@@ -301,10 +312,10 @@ public class TestDistCpOptions {
     final DistCpOptions.Builder builder = new DistCpOptions.Builder(
         new Path("hdfs://localhost:8020/source/first"),
         new Path("hdfs://localhost:8020/target/"));
-    Assert.assertEquals(DistCpConstants.UNIFORMSIZE,
+    assertEquals(DistCpConstants.UNIFORMSIZE,
         builder.build().getCopyStrategy());
     builder.withCopyStrategy("dynamic");
-    Assert.assertEquals("dynamic", builder.build().getCopyStrategy());
+    assertEquals("dynamic", builder.build().getCopyStrategy());
   }
 
   @Test
@@ -312,7 +323,7 @@ public class TestDistCpOptions {
     final DistCpOptions options = new DistCpOptions.Builder(
         new Path("hdfs://localhost:8020/source/first"),
         new Path("hdfs://localhost:8020/target/")).build();
-    Assert.assertEquals(new Path("hdfs://localhost:8020/target/"),
+    assertEquals(new Path("hdfs://localhost:8020/target/"),
         options.getTargetPath());
   }
 
@@ -322,25 +333,25 @@ public class TestDistCpOptions {
         new Path("hdfs://localhost:8020/source/first"),
         new Path("hdfs://localhost:8020/target/"))
         .build();
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.BLOCKSIZE));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.REPLICATION));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.PERMISSION));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.USER));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.GROUP));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.CHECKSUMTYPE));
+    assertFalse(options.shouldPreserve(FileAttribute.BLOCKSIZE));
+    assertFalse(options.shouldPreserve(FileAttribute.REPLICATION));
+    assertFalse(options.shouldPreserve(FileAttribute.PERMISSION));
+    assertFalse(options.shouldPreserve(FileAttribute.USER));
+    assertFalse(options.shouldPreserve(FileAttribute.GROUP));
+    assertFalse(options.shouldPreserve(FileAttribute.CHECKSUMTYPE));
 
     options = new DistCpOptions.Builder(
         new Path("hdfs://localhost:8020/source/first"),
         new Path("hdfs://localhost:8020/target/"))
         .preserve(FileAttribute.ACL)
         .build();
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.BLOCKSIZE));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.REPLICATION));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.PERMISSION));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.USER));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.GROUP));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.CHECKSUMTYPE));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.ACL));
+    assertFalse(options.shouldPreserve(FileAttribute.BLOCKSIZE));
+    assertFalse(options.shouldPreserve(FileAttribute.REPLICATION));
+    assertFalse(options.shouldPreserve(FileAttribute.PERMISSION));
+    assertFalse(options.shouldPreserve(FileAttribute.USER));
+    assertFalse(options.shouldPreserve(FileAttribute.GROUP));
+    assertFalse(options.shouldPreserve(FileAttribute.CHECKSUMTYPE));
+    assertTrue(options.shouldPreserve(FileAttribute.ACL));
 
     options = new DistCpOptions.Builder(
         new Path("hdfs://localhost:8020/source/first"),
@@ -353,13 +364,13 @@ public class TestDistCpOptions {
         .preserve(FileAttribute.CHECKSUMTYPE)
         .build();
 
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.BLOCKSIZE));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.REPLICATION));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.PERMISSION));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.USER));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.GROUP));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.CHECKSUMTYPE));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.XATTR));
+    assertTrue(options.shouldPreserve(FileAttribute.BLOCKSIZE));
+    assertTrue(options.shouldPreserve(FileAttribute.REPLICATION));
+    assertTrue(options.shouldPreserve(FileAttribute.PERMISSION));
+    assertTrue(options.shouldPreserve(FileAttribute.USER));
+    assertTrue(options.shouldPreserve(FileAttribute.GROUP));
+    assertTrue(options.shouldPreserve(FileAttribute.CHECKSUMTYPE));
+    assertFalse(options.shouldPreserve(FileAttribute.XATTR));
   }
 
   @Test
@@ -369,7 +380,7 @@ public class TestDistCpOptions {
         new Path("hdfs://localhost:8020/target/"))
         .withSyncFolder(true)
         .withAppend(true);
-    Assert.assertTrue(builder.build().shouldAppend());
+    assertTrue(builder.build().shouldAppend());
 
     try {
       // make sure -append is only valid when -update is specified
@@ -408,9 +419,9 @@ public class TestDistCpOptions {
         .withSyncFolder(true)
         .withUseDiff("s1", "s2")
         .build();
-    Assert.assertTrue(options.shouldUseDiff());
-    Assert.assertEquals("s1", options.getFromSnapshot());
-    Assert.assertEquals("s2", options.getToSnapshot());
+    assertTrue(options.shouldUseDiff());
+    assertEquals("s1", options.getFromSnapshot());
+    assertEquals("s2", options.getToSnapshot());
 
     options = new DistCpOptions.Builder(
         new Path("hdfs://localhost:8020/source/first"),
@@ -418,9 +429,9 @@ public class TestDistCpOptions {
         .withSyncFolder(true)
         .withUseDiff("s1", ".")
         .build();
-    Assert.assertTrue(options.shouldUseDiff());
-    Assert.assertEquals("s1", options.getFromSnapshot());
-    Assert.assertEquals(".", options.getToSnapshot());
+    assertTrue(options.shouldUseDiff());
+    assertEquals("s1", options.getFromSnapshot());
+    assertEquals(".", options.getToSnapshot());
 
     // make sure -diff is only valid when -update is specified
     try {
@@ -478,10 +489,10 @@ public class TestDistCpOptions {
     final DistCpOptions.Builder builder = new DistCpOptions.Builder(
         new Path("hdfs://localhost:8020/source/first"),
         new Path("hdfs://localhost:8020/target/"));
-    Assert.assertNull(builder.build().getFiltersFile());
+    assertNull(builder.build().getFiltersFile());
 
     builder.withFiltersFile("/tmp/filters.txt");
-    Assert.assertEquals("/tmp/filters.txt", builder.build().getFiltersFile());
+    assertEquals("/tmp/filters.txt", builder.build().getFiltersFile());
   }
 
   @Test
@@ -491,12 +502,12 @@ public class TestDistCpOptions {
         new Path("hdfs://localhost:8020/target/"))
         .withAppend(true)
         .withSyncFolder(true);
-    Assert.assertFalse(builder.build().shouldPreserve(FileAttribute.BLOCKSIZE));
-    Assert.assertTrue(builder.build().shouldAppend());
+    assertFalse(builder.build().shouldPreserve(FileAttribute.BLOCKSIZE));
+    assertTrue(builder.build().shouldAppend());
 
     builder.withBlocksPerChunk(5440);
-    Assert.assertTrue(builder.build().shouldPreserve(FileAttribute.BLOCKSIZE));
-    Assert.assertFalse(builder.build().shouldAppend());
+    assertTrue(builder.build().shouldPreserve(FileAttribute.BLOCKSIZE));
+    assertFalse(builder.build().shouldAppend());
   }
 
   @Test
@@ -505,15 +516,15 @@ public class TestDistCpOptions {
         Collections.singletonList(new Path("hdfs://localhost:8020/source")),
         new Path("hdfs://localhost:8020/target/"));
 
-    Assert.assertEquals(DistCpConstants.COPY_BUFFER_SIZE_DEFAULT,
+    assertEquals(DistCpConstants.COPY_BUFFER_SIZE_DEFAULT,
         builder.build().getCopyBufferSize());
 
     builder.withCopyBufferSize(4194304);
-    Assert.assertEquals(4194304,
+    assertEquals(4194304,
         builder.build().getCopyBufferSize());
 
     builder.withCopyBufferSize(-1);
-    Assert.assertEquals(DistCpConstants.COPY_BUFFER_SIZE_DEFAULT,
+    assertEquals(DistCpConstants.COPY_BUFFER_SIZE_DEFAULT,
         builder.build().getCopyBufferSize());
   }
 
@@ -522,7 +533,7 @@ public class TestDistCpOptions {
     final DistCpOptions.Builder builder = new DistCpOptions.Builder(
         Collections.singletonList(new Path("hdfs://localhost:8020/source")),
         new Path("hdfs://localhost:8020/target/"));
-    Assert.assertFalse(builder.build().shouldVerboseLog());
+    assertFalse(builder.build().shouldVerboseLog());
 
     try {
       builder.withVerboseLog(true).build();
@@ -533,7 +544,7 @@ public class TestDistCpOptions {
 
     final Path logPath = new Path("hdfs://localhost:8020/logs");
     builder.withLogPath(logPath).withVerboseLog(true);
-    Assert.assertTrue(builder.build().shouldVerboseLog());
+    assertTrue(builder.build().shouldVerboseLog());
   }
 
   @Test
@@ -552,16 +563,15 @@ public class TestDistCpOptions {
     config.set("", expectedValForEmptyConfigKey);
 
     options.appendToConf(config);
-    Assert.assertEquals(expectedBlocksPerChunk,
+    assertEquals(expectedBlocksPerChunk,
         config.getInt(
             DistCpOptionSwitch
                 .BLOCKS_PER_CHUNK
                 .getConfigLabel(), 0));
-    Assert.assertEquals(
+    assertEquals(expectedValForEmptyConfigKey, config.get(""),
         "Some DistCpOptionSwitch's config label is empty! " +
-            "Pls ensure the config label is provided when apply to config, " +
-            "otherwise it may not be fetched properly",
-            expectedValForEmptyConfigKey, config.get(""));
+        "Pls ensure the config label is provided when apply to config, " +
+        "otherwise it may not be fetched properly");
   }
 
   @Test
@@ -572,6 +582,6 @@ public class TestDistCpOptions {
         new Path("hdfs://localhost:8020/target/"))
         .withUpdateRoot(true)
         .build();
-    Assert.assertTrue(options.shouldUpdateRoot());
+    assertTrue(options.shouldUpdateRoot());
   }
 }

+ 46 - 46
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSync.java

@@ -42,10 +42,10 @@ import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.LambdaTestUtils;
 import org.apache.hadoop.tools.mapred.CopyMapper;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 import java.io.IOException;
 import java.io.FileWriter;
@@ -74,7 +74,7 @@ public class TestDistCpSync {
   private final long BLOCK_SIZE = 1024;
   private final short DATA_NUM = 1;
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(DATA_NUM).build();
     cluster.waitActive();
@@ -99,7 +99,7 @@ public class TestDistCpSync {
     conf.setClass("fs.dummy.impl", DummyFs.class, FileSystem.class);
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     IOUtils.cleanupWithLogger(null, dfs);
     if (cluster != null) {
@@ -116,40 +116,40 @@ public class TestDistCpSync {
   @Test
   public void testFallback() throws Exception {
     // the source/target dir are not snapshottable dir
-    Assert.assertFalse(sync());
+    Assertions.assertFalse(sync());
     // make sure the source path has been updated to the snapshot path
     final Path spath = new Path(source,
         HdfsConstants.DOT_SNAPSHOT_DIR + Path.SEPARATOR + "s2");
-    Assert.assertEquals(spath, context.getSourcePaths().get(0));
+    Assertions.assertEquals(spath, context.getSourcePaths().get(0));
 
     // reset source path in options
     context.setSourcePaths(Collections.singletonList(source));
     // the source/target does not have the given snapshots
     dfs.allowSnapshot(source);
     dfs.allowSnapshot(target);
-    Assert.assertFalse(sync());
-    Assert.assertEquals(spath, context.getSourcePaths().get(0));
+    Assertions.assertFalse(sync());
+    Assertions.assertEquals(spath, context.getSourcePaths().get(0));
 
     // reset source path in options
     context.setSourcePaths(Collections.singletonList(source));
     dfs.createSnapshot(source, "s1");
     dfs.createSnapshot(source, "s2");
     dfs.createSnapshot(target, "s1");
-    Assert.assertTrue(sync());
+    Assertions.assertTrue(sync());
 
     // reset source paths in options
     context.setSourcePaths(Collections.singletonList(source));
     // changes have been made in target
     final Path subTarget = new Path(target, "sub");
     dfs.mkdirs(subTarget);
-    Assert.assertFalse(sync());
+    Assertions.assertFalse(sync());
     // make sure the source path has been updated to the snapshot path
-    Assert.assertEquals(spath, context.getSourcePaths().get(0));
+    Assertions.assertEquals(spath, context.getSourcePaths().get(0));
 
     // reset source paths in options
     context.setSourcePaths(Collections.singletonList(source));
     dfs.delete(subTarget, true);
-    Assert.assertTrue(sync());
+    Assertions.assertTrue(sync());
   }
 
   private void enableAndCreateFirstSnapshot() throws Exception {
@@ -160,7 +160,7 @@ public class TestDistCpSync {
   }
 
   private void syncAndVerify() throws Exception {
-    Assert.assertTrue(sync());
+    Assertions.assertTrue(sync());
     verifyCopy(dfs.getFileStatus(source), dfs.getFileStatus(target), false);
   }
 
@@ -266,12 +266,12 @@ public class TestDistCpSync {
     DistCpSync distCpSync = new DistCpSync(context, conf);
 
     // do the sync
-    Assert.assertTrue(distCpSync.sync());
+    Assertions.assertTrue(distCpSync.sync());
 
     // make sure the source path has been updated to the snapshot path
     final Path spath = new Path(source,
             HdfsConstants.DOT_SNAPSHOT_DIR + Path.SEPARATOR + "s2");
-    Assert.assertEquals(spath, context.getSourcePaths().get(0));
+    Assertions.assertEquals(spath, context.getSourcePaths().get(0));
 
     // build copy listing
     final Path listingPath = new Path("/tmp/META/fileList.seq");
@@ -292,10 +292,10 @@ public class TestDistCpSync {
     }
 
     // verify that we only list modified and created files/directories
-    Assert.assertEquals(numCreatedModified, copyListing.size());
+    Assertions.assertEquals(numCreatedModified, copyListing.size());
 
     // verify that we only copied new appended data of f2 and the new file f1
-    Assert.assertEquals(BLOCK_SIZE * 3, stubContext.getReporter()
+    Assertions.assertEquals(BLOCK_SIZE * 3, stubContext.getReporter()
         .getCounter(CopyMapper.Counter.BYTESCOPIED).getValue());
 
     // verify the source and target now has the same structure
@@ -320,7 +320,7 @@ public class TestDistCpSync {
     DFSTestUtil.createFile(dfs, new Path(source, "encz-mock/datedir/file2"),
         BLOCK_SIZE, DATA_NUM, 0);
     dfs.createSnapshot(source, "s2");
-    Assert.assertTrue(dfs.exists(new Path(source, "encz-mock/datedir/file2")));
+    Assertions.assertTrue(dfs.exists(new Path(source, "encz-mock/datedir/file2")));
 
     SnapshotDiffReport report = dfs.getSnapshotDiffReport(source, "s1", "s2");
     System.out.println(report);
@@ -328,11 +328,11 @@ public class TestDistCpSync {
     DistCpSync distCpSync = new DistCpSync(context, conf);
 
     // do the sync
-    Assert.assertTrue(distCpSync.sync());
+    Assertions.assertTrue(distCpSync.sync());
     // make sure the source path has been updated to the snapshot path
     final Path spath = new Path(source,
         HdfsConstants.DOT_SNAPSHOT_DIR + Path.SEPARATOR + "s2");
-    Assert.assertEquals(spath, context.getSourcePaths().get(0));
+    Assertions.assertEquals(spath, context.getSourcePaths().get(0));
 
     // build copy listing
     final Path listingPath = new Path("/tmp/META/fileList.seq");
@@ -350,7 +350,7 @@ public class TestDistCpSync {
         .entrySet()) {
       copyMapper.map(entry.getKey(), entry.getValue(), mapContext);
     }
-    Assert.assertTrue(dfs.exists(new Path(target, "encz-mock/datedir/file2")));
+    Assertions.assertTrue(dfs.exists(new Path(target, "encz-mock/datedir/file2")));
     // verify the source and target now has the same structure
     verifyCopy(dfs.getFileStatus(spath), dfs.getFileStatus(target), false);
   }
@@ -375,7 +375,7 @@ public class TestDistCpSync {
     DFSTestUtil.createFile(dfs, new Path(source, "encz-mock/datedir/file2"),
         BLOCK_SIZE, DATA_NUM, 0);
     dfs.createSnapshot(source, "s2");
-    Assert.assertTrue(dfs.exists(new Path(source, "encz-mock/datedir/file2")));
+    Assertions.assertTrue(dfs.exists(new Path(source, "encz-mock/datedir/file2")));
 
     SnapshotDiffReport report = dfs.getSnapshotDiffReport(source, "s1", "s2");
     System.out.println(report);
@@ -383,11 +383,11 @@ public class TestDistCpSync {
     DistCpSync distCpSync = new DistCpSync(context, conf);
 
     // do the sync
-    Assert.assertTrue(distCpSync.sync());
+    Assertions.assertTrue(distCpSync.sync());
     // make sure the source path has been updated to the snapshot path
     final Path spath = new Path(source,
         HdfsConstants.DOT_SNAPSHOT_DIR + Path.SEPARATOR + "s2");
-    Assert.assertEquals(spath, context.getSourcePaths().get(0));
+    Assertions.assertEquals(spath, context.getSourcePaths().get(0));
 
     // build copy listing
     final Path listingPath = new Path("/tmp/META/fileList.seq");
@@ -405,8 +405,8 @@ public class TestDistCpSync {
         .entrySet()) {
       copyMapper.map(entry.getKey(), entry.getValue(), mapContext);
     }
-    Assert.assertTrue(dfs.exists(new Path(target, "encz-mock/datedir/file2")));
-    Assert.assertTrue(dfs.exists(new Path(target, "trash/datedir/file1")));
+    Assertions.assertTrue(dfs.exists(new Path(target, "encz-mock/datedir/file2")));
+    Assertions.assertTrue(dfs.exists(new Path(target, "trash/datedir/file1")));
     // verify the source and target now has the same structure
     verifyCopy(dfs.getFileStatus(spath), dfs.getFileStatus(target), false);
   }
@@ -431,7 +431,7 @@ public class TestDistCpSync {
     DFSTestUtil.createFile(dfs, new Path(source, "encz-mock/datedir/file2"),
         BLOCK_SIZE, DATA_NUM, 0);
     dfs.createSnapshot(source, "s2");
-    Assert.assertTrue(dfs.exists(new Path(source, "encz-mock/datedir/file2")));
+    Assertions.assertTrue(dfs.exists(new Path(source, "encz-mock/datedir/file2")));
 
     SnapshotDiffReport report = dfs.getSnapshotDiffReport(source, "s1", "s2");
     System.out.println(report);
@@ -444,11 +444,11 @@ public class TestDistCpSync {
     distCpSync.setCopyFilter(regexCopyFilter);
 
     // do the sync
-    Assert.assertTrue(distCpSync.sync());
+    Assertions.assertTrue(distCpSync.sync());
     // make sure the source path has been updated to the snapshot path
     final Path spath = new Path(source,
         HdfsConstants.DOT_SNAPSHOT_DIR + Path.SEPARATOR + "s2");
-    Assert.assertEquals(spath, context.getSourcePaths().get(0));
+    Assertions.assertEquals(spath, context.getSourcePaths().get(0));
 
     // build copy listing
     final Path listingPath = new Path("/tmp/META/fileList.seq");
@@ -466,9 +466,9 @@ public class TestDistCpSync {
         .entrySet()) {
       copyMapper.map(entry.getKey(), entry.getValue(), mapContext);
     }
-    Assert.assertTrue(dfs.exists(new Path(target, "encz-mock/datedir/file2")));
-    Assert.assertFalse(dfs.exists(new Path(target, "encz-mock/datedir/file1")));
-    Assert.assertFalse(dfs.exists(new Path(target, "trash/datedir/file1")));
+    Assertions.assertTrue(dfs.exists(new Path(target, "encz-mock/datedir/file2")));
+    Assertions.assertFalse(dfs.exists(new Path(target, "encz-mock/datedir/file1")));
+    Assertions.assertFalse(dfs.exists(new Path(target, "trash/datedir/file1")));
   }
 
   private Map<Text, CopyListingFileStatus> getListing(Path listingPath)
@@ -516,19 +516,19 @@ public class TestDistCpSync {
   private void verifyCopy(FileSystem sfs, FileSystem tfs,
                           FileStatus s, FileStatus t, boolean compareName)
           throws Exception {
-    Assert.assertEquals(s.isDirectory(), t.isDirectory());
+    Assertions.assertEquals(s.isDirectory(), t.isDirectory());
     if (compareName) {
-      Assert.assertEquals(s.getPath().getName(), t.getPath().getName());
+      Assertions.assertEquals(s.getPath().getName(), t.getPath().getName());
     }
     if (!s.isDirectory()) {
       // verify the file content is the same
       byte[] sbytes = DFSTestUtil.readFileBuffer(sfs, s.getPath());
       byte[] tbytes = DFSTestUtil.readFileBuffer(tfs, t.getPath());
-      Assert.assertArrayEquals(sbytes, tbytes);
+      Assertions.assertArrayEquals(sbytes, tbytes);
     } else {
       FileStatus[] slist = sfs.listStatus(s.getPath());
       FileStatus[] tlist = tfs.listStatus(t.getPath());
-      Assert.assertEquals(slist.length, tlist.length);
+      Assertions.assertEquals(slist.length, tlist.length);
       for (int i = 0; i < slist.length; i++) {
         verifyCopy(sfs, tfs, slist[i], tlist[i], true);
       }
@@ -557,7 +557,7 @@ public class TestDistCpSync {
     // do the sync
     sync();
     // make sure the source path is still unchanged
-    Assert.assertEquals(source, context.getSourcePaths().get(0));
+    Assertions.assertEquals(source, context.getSourcePaths().get(0));
   }
 
   private void initData2(Path dir) throws Exception {
@@ -737,12 +737,12 @@ public class TestDistCpSync {
 
     DistCpSync distCpSync = new DistCpSync(context, conf);
     // do the sync
-    Assert.assertTrue(distCpSync.sync());
+    Assertions.assertTrue(distCpSync.sync());
 
     // make sure the source path has been updated to the snapshot path
     final Path spath = new Path(source,
             HdfsConstants.DOT_SNAPSHOT_DIR + Path.SEPARATOR + "s2");
-    Assert.assertEquals(spath, context.getSourcePaths().get(0));
+    Assertions.assertEquals(spath, context.getSourcePaths().get(0));
 
     // build copy listing
     final Path listingPath = new Path("/tmp/META/fileList.seq");
@@ -764,7 +764,7 @@ public class TestDistCpSync {
     }
 
     // verify that we only list modified and created files/directories
-    Assert.assertEquals(numCreatedModified, copyListing.size());
+    Assertions.assertEquals(numCreatedModified, copyListing.size());
 
     // verify the source and target now has the same structure
     verifyCopy(dfs.getFileStatus(spath), dfs.getFileStatus(target), false);
@@ -971,7 +971,7 @@ public class TestDistCpSync {
       GenericTestUtils.assertExceptionContains(
           "Snapshot s2 should be newer than s1", e);
     }
-    Assert.assertTrue(threwException);
+    Assertions.assertTrue(threwException);
   }
 
   private void initData10(Path dir) throws Exception {
@@ -1062,15 +1062,15 @@ public class TestDistCpSync {
   private void verifySync(FileStatus s, FileStatus t, boolean compareName,
                           String deletedName)
           throws Exception {
-    Assert.assertEquals(s.isDirectory(), t.isDirectory());
+    Assertions.assertEquals(s.isDirectory(), t.isDirectory());
     if (compareName) {
-      Assert.assertEquals(s.getPath().getName(), t.getPath().getName());
+      Assertions.assertEquals(s.getPath().getName(), t.getPath().getName());
     }
     if (!s.isDirectory()) {
       // verify the file content is the same
       byte[] sbytes = DFSTestUtil.readFileBuffer(dfs, s.getPath());
       byte[] tbytes = DFSTestUtil.readFileBuffer(dfs, t.getPath());
-      Assert.assertArrayEquals(sbytes, tbytes);
+      Assertions.assertArrayEquals(sbytes, tbytes);
     } else {
       FileStatus[] slist = dfs.listStatus(s.getPath());
       FileStatus[] tlist = dfs.listStatus(t.getPath());

+ 32 - 28
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSyncReverseBase.java

@@ -33,10 +33,9 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.tools.mapred.CopyMapper;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 import java.io.ByteArrayOutputStream;
 import java.io.PrintStream;
@@ -48,6 +47,11 @@ import java.util.List;
 import java.util.Map;
 import java.util.StringTokenizer;
 
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
 /**
  * Base class to test "-rdiff s2 s1".
  * Shared by "-rdiff s2 s1 src tgt" and "-rdiff s2 s1 tgt tgt"
@@ -91,7 +95,7 @@ public abstract class TestDistCpSyncReverseBase {
     System.setErr(out);
     final String results;
     try {
-      Assert.assertEquals(0, shell.run(new String[] {"-lsr", dir }));
+      assertEquals(0, shell.run(new String[] {"-lsr", dir }));
       results = bytes.toString();
     } finally {
       IOUtils.closeStream(out);
@@ -127,7 +131,7 @@ public abstract class TestDistCpSyncReverseBase {
     isSrcNotSameAsTgt = srcNotSameAsTgt;
   }
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     initSourcePath();
 
@@ -151,7 +155,7 @@ public abstract class TestDistCpSyncReverseBase {
     conf.set(DistCpConstants.CONF_LABEL_TARGET_FINAL_PATH, target.toString());
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     IOUtils.cleanupWithLogger(null, dfs);
     if (cluster != null) {
@@ -168,39 +172,39 @@ public abstract class TestDistCpSyncReverseBase {
   @Test
   public void testFallback() throws Exception {
     // the source/target dir are not snapshottable dir
-    Assert.assertFalse(sync());
+    assertFalse(sync());
     // make sure the source path has been updated to the snapshot path
     final Path spath = new Path(source,
         HdfsConstants.DOT_SNAPSHOT_DIR + Path.SEPARATOR + "s1");
-    Assert.assertEquals(spath, distCpContext.getSourcePaths().get(0));
+    assertEquals(spath, distCpContext.getSourcePaths().get(0));
 
     // reset source path in options
     optionsBuilder.withSourcePaths(Arrays.asList(source));
     // the source/target does not have the given snapshots
     dfs.allowSnapshot(source);
     dfs.allowSnapshot(target);
-    Assert.assertFalse(sync());
-    Assert.assertEquals(spath, distCpContext.getSourcePaths().get(0));
+    assertFalse(sync());
+    assertEquals(spath, distCpContext.getSourcePaths().get(0));
 
     // reset source path in options
     optionsBuilder.withSourcePaths(Arrays.asList(source));
     this.enableAndCreateFirstSnapshot();
     dfs.createSnapshot(target, "s2");
-    Assert.assertTrue(sync());
+    assertTrue(sync());
 
     // reset source paths in options
     optionsBuilder.withSourcePaths(Arrays.asList(source));
     // changes have been made in target
     final Path subTarget = new Path(target, "sub");
     dfs.mkdirs(subTarget);
-    Assert.assertFalse(sync());
+    assertFalse(sync());
     // make sure the source path has been updated to the snapshot path
-    Assert.assertEquals(spath, distCpContext.getSourcePaths().get(0));
+    assertEquals(spath, distCpContext.getSourcePaths().get(0));
 
     // reset source paths in options
     optionsBuilder.withSourcePaths(Arrays.asList(source));
     dfs.delete(subTarget, true);
-    Assert.assertTrue(sync());
+    assertTrue(sync());
   }
 
   private void syncAndVerify() throws Exception {
@@ -209,7 +213,7 @@ public abstract class TestDistCpSyncReverseBase {
     lsrSource("Before sync source: ", shell, source);
     lsr("Before sync target: ", shell, target);
 
-    Assert.assertTrue(sync());
+    assertTrue(sync());
 
     lsrSource("After sync source: ", shell, source);
     lsr("After sync target: ", shell, target);
@@ -337,14 +341,14 @@ public abstract class TestDistCpSyncReverseBase {
     lsr("Before sync target: ", shell, target);
 
     // do the sync
-    Assert.assertTrue(distCpSync.sync());
+    assertTrue(distCpSync.sync());
 
     lsr("After sync target: ", shell, target);
 
     // make sure the source path has been updated to the snapshot path
     final Path spath = new Path(source,
         HdfsConstants.DOT_SNAPSHOT_DIR + Path.SEPARATOR + "s1");
-    Assert.assertEquals(spath, distCpContext.getSourcePaths().get(0));
+    assertEquals(spath, distCpContext.getSourcePaths().get(0));
 
     // build copy listing
     final Path listingPath = new Path("/tmp/META/fileList.seq");
@@ -370,10 +374,10 @@ public abstract class TestDistCpSyncReverseBase {
     lsr("After mapper target: ", shell, target);
 
     // verify that we only list modified and created files/directories
-    Assert.assertEquals(numDeletedModified, copyListing.size());
+    assertEquals(numDeletedModified, copyListing.size());
 
     // verify that we only copied new appended data of f2 and the new file f1
-    Assert.assertEquals(blockSize * 3, stubContext.getReporter()
+    assertEquals(blockSize * 3, stubContext.getReporter()
         .getCounter(CopyMapper.Counter.BYTESCOPIED).getValue());
 
     // verify the source and target now has the same structure
@@ -404,19 +408,19 @@ public abstract class TestDistCpSyncReverseBase {
 
   private void verifyCopy(FileStatus s, FileStatus t, boolean compareName)
       throws Exception {
-    Assert.assertEquals(s.isDirectory(), t.isDirectory());
+    assertEquals(s.isDirectory(), t.isDirectory());
     if (compareName) {
-      Assert.assertEquals(s.getPath().getName(), t.getPath().getName());
+      assertEquals(s.getPath().getName(), t.getPath().getName());
     }
     if (!s.isDirectory()) {
       // verify the file content is the same
       byte[] sbytes = DFSTestUtil.readFileBuffer(dfs, s.getPath());
       byte[] tbytes = DFSTestUtil.readFileBuffer(dfs, t.getPath());
-      Assert.assertArrayEquals(sbytes, tbytes);
+      assertArrayEquals(sbytes, tbytes);
     } else {
       FileStatus[] slist = dfs.listStatus(s.getPath());
       FileStatus[] tlist = dfs.listStatus(t.getPath());
-      Assert.assertEquals(slist.length, tlist.length);
+      assertEquals(slist.length, tlist.length);
       for (int i = 0; i < slist.length; i++) {
         verifyCopy(slist[i], tlist[i], true);
       }
@@ -440,11 +444,11 @@ public abstract class TestDistCpSyncReverseBase {
     changeData(target);
 
     // do the sync
-    Assert.assertTrue(sync());
+    assertTrue(sync());
     final Path spath = new Path(source,
         HdfsConstants.DOT_SNAPSHOT_DIR + Path.SEPARATOR + "s1");
     // make sure the source path is still unchanged
-    Assert.assertEquals(spath, distCpContext.getSourcePaths().get(0));
+    assertEquals(spath, distCpContext.getSourcePaths().get(0));
   }
 
   private void initData2(Path dir) throws Exception {
@@ -662,7 +666,7 @@ public abstract class TestDistCpSyncReverseBase {
     // make sure the source path has been updated to the snapshot path
     final Path spath = new Path(source,
             HdfsConstants.DOT_SNAPSHOT_DIR + Path.SEPARATOR + "s1");
-    Assert.assertEquals(spath, distCpContext.getSourcePaths().get(0));
+    assertEquals(spath, distCpContext.getSourcePaths().get(0));
 
     // build copy listing
     final Path listingPath = new Path("/tmp/META/fileList.seq");
@@ -684,7 +688,7 @@ public abstract class TestDistCpSyncReverseBase {
     }
 
     // verify that we only list modified and created files/directories
-    Assert.assertEquals(numDeletedAndModified, copyListing.size());
+    assertEquals(numDeletedAndModified, copyListing.size());
 
     lsr("After Copy target: ", shell, target);
 

+ 26 - 26
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpSystem.java

@@ -19,10 +19,11 @@
 package org.apache.hadoop.tools;
 
 import static org.apache.hadoop.test.GenericTestUtils.getMethodName;
-import static org.hamcrest.core.Is.is;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertTrue;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
@@ -48,10 +49,9 @@ import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.tools.util.DistCpTestUtils;
 import org.apache.hadoop.util.ToolRunner;
-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;
 
 /**
  * A JUnit test for copying files recursively.
@@ -85,7 +85,7 @@ public class TestDistCpSystem {
     }
   }
 
-  @BeforeClass
+  @BeforeAll
   public static void beforeClass() throws IOException {
     conf = new Configuration();
     conf.setLong(DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_KEY, BLOCK_SIZE);
@@ -94,7 +94,7 @@ public class TestDistCpSystem {
     cluster.waitActive();
   }
 
-  @AfterClass
+  @AfterAll
   public static void afterClass() throws IOException {
     if (cluster != null) {
       cluster.shutdown();
@@ -202,7 +202,7 @@ public class TestDistCpSystem {
     }
     FileStatus[] dststat = getFileStatus(fs, realTgtPath, dstEntries);
     for(int i = 0; i < dststat.length; i++) {
-      assertEquals("i=" + i, "u" + i, dststat[i].getOwner());
+      assertEquals("u" + i, dststat[i].getOwner(), "i=" + i);
     }
     deldir(fs, testRoot);
   }
@@ -212,20 +212,20 @@ public class TestDistCpSystem {
     LOG.info("Comparing " + srcStat + " and " + dstStat);
     assertEquals(srcStat.isDirectory(), dstStat.isDirectory());
     assertEquals(srcStat.getReplication(), dstStat.getReplication());
-    assertEquals("File POSIX permission should match",
-        srcStat.getPermission(), dstStat.getPermission());
-    assertEquals("File user ownership should match",
-        srcStat.getOwner(), dstStat.getOwner());
-    assertEquals("File group ownership should match",
-        srcStat.getGroup(), dstStat.getGroup());
+    assertEquals(srcStat.getPermission(), dstStat.getPermission(),
+        "File POSIX permission should match");
+    assertEquals(srcStat.getOwner(), dstStat.getOwner(),
+        "File user ownership should match");
+    assertEquals(srcStat.getGroup(), dstStat.getGroup(),
+        "File group ownership should match");
     // TODO; check ACL attributes
 
     if (srcStat.isDirectory()) {
       return;
     }
 
-    assertEquals("File length should match (" + srcStat.getPath() + ")",
-        srcStat.getLen(), dstStat.getLen());
+    assertEquals(srcStat.getLen(), dstStat.getLen(),
+        "File length should match (" + srcStat.getPath() + ")");
 
     FSDataInputStream srcIn = fs.open(srcStat.getPath());
     FSDataInputStream dstIn = fs.open(dstStat.getPath());
@@ -251,12 +251,12 @@ public class TestDistCpSystem {
           LOG.info("______ compared src and dst files for "
               + totalComparedBytes + " bytes, content match.");
           if (srcBytesRead != tgtBytesRead) {
-            Assert.fail("Read mismatching size, compared "
+            fail("Read mismatching size, compared "
                 + totalComparedBytes + " bytes between src and dst file "
                 + srcStat + " and " + dstStat);
           }
           if (totalComparedBytes != srcStat.getLen()) {
-            Assert.fail("Only read/compared " + totalComparedBytes +
+            fail("Only read/compared " + totalComparedBytes +
                 " bytes between src and dst file " + srcStat +
                 " and " + dstStat);
           } else {
@@ -267,7 +267,7 @@ public class TestDistCpSystem {
         for (; srcIdx < srcBytesRead && tgtIdx < tgtBytesRead;
             ++srcIdx, ++tgtIdx) {
           if (readSrc[srcIdx] != readDst[tgtIdx]) {
-            Assert.fail("src and dst file does not match at "
+            fail("src and dst file does not match at "
                 + totalComparedBytes + " between "
                 + srcStat + " and " + dstStat);
           }
@@ -456,8 +456,8 @@ public class TestDistCpSystem {
 
     String realTgtPath = testDst;
     FileStatus[] dststat = getFileStatus(fs, realTgtPath, srcfiles);
-    assertEquals("File length should match", srcLen,
-        dststat[dststat.length - 1].getLen());
+    assertEquals(srcLen, dststat[dststat.length - 1].getLen(),
+        "File length should match");
 
     this.compareFiles(fs,  srcstats[srcstats.length-1],
         dststat[dststat.length-1]);
@@ -544,7 +544,7 @@ public class TestDistCpSystem {
     Path tgtPath = new Path(testRoot + "/nodir");
     String tgtStr = fs.makeQualified(tgtPath).toString();
     String[] args = new String[]{rootStr, tgtStr};
-    Assert.assertThat(ToolRunner.run(conf, new DistCp(), args), is(0));
+    assertThat(ToolRunner.run(conf, new DistCp(), args)).isEqualTo(0);
 
     // Case 2. The target exists.
 
@@ -552,7 +552,7 @@ public class TestDistCpSystem {
     assertTrue(fs.mkdirs(tgtPath2));
     String tgtStr2 = fs.makeQualified(tgtPath2).toString();
     String[] args2 = new String[]{rootStr, tgtStr2};
-    Assert.assertThat(ToolRunner.run(conf, new DistCp(), args2), is(0));
+    assertThat(ToolRunner.run(conf, new DistCp(), args2)).isEqualTo(0);
   }
 
   @Test

+ 9 - 7
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpViewFs.java

@@ -27,15 +27,17 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.tools.util.TestDistCpUtils;
 import org.apache.hadoop.fs.FsConstants;
 
-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 java.io.IOException;
 import java.io.OutputStream;
 import java.net.URI;
 import java.net.URISyntaxException;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
 public class TestDistCpViewFs {
   private static final Logger LOG = LoggerFactory.getLogger(TestDistCpViewFs.class);
 
@@ -52,7 +54,7 @@ public class TestDistCpViewFs {
     return conf;
   }
 
-  @BeforeClass
+  @BeforeAll
   public static void setup() throws URISyntaxException{
     try {
       Path fswd = FileSystem.get(getConf()).getWorkingDirectory();
@@ -427,13 +429,13 @@ public class TestDistCpViewFs {
   }
 
   private void checkResult(Path target, int count, String... relPaths) throws IOException {
-    Assert.assertEquals(count, fs.listStatus(target).length);
+    assertEquals(count, fs.listStatus(target).length);
     if (relPaths == null || relPaths.length == 0) {
-      Assert.assertTrue(target.toString(), fs.exists(target));
+      assertTrue(fs.exists(target), target.toString());
       return;
     }
     for (String relPath : relPaths) {
-      Assert.assertTrue(new Path(target, relPath).toString(), fs.exists(new Path(target, relPath)));
+      assertTrue(fs.exists(new Path(target, relPath)), new Path(target, relPath).toString());
     }
   }
 

+ 8 - 7
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpWithAcls.java

@@ -21,7 +21,8 @@ package org.apache.hadoop.tools;
 import static org.apache.hadoop.fs.permission.AclEntryScope.*;
 import static org.apache.hadoop.fs.permission.AclEntryType.*;
 import static org.apache.hadoop.fs.permission.FsAction.*;
-import static org.junit.Assert.*;
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.io.IOException;
 import java.net.URI;
@@ -45,9 +46,9 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.Progressable;
 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 distcp in combination with HDFS ACLs.
@@ -58,7 +59,7 @@ public class TestDistCpWithAcls {
   private static Configuration conf;
   private static FileSystem fs;
 
-  @BeforeClass
+  @BeforeAll
   public static void init() throws Exception {
     initCluster(true, true);
     // Create this directory structure:
@@ -94,7 +95,7 @@ public class TestDistCpWithAcls {
       new FsPermission((short)01777));
   }
 
-  @AfterClass
+  @AfterAll
   public static void shutdown() {
     IOUtils.cleanupWithLogger(null, fs);
     if (cluster != null) {
@@ -286,7 +287,7 @@ public class TestDistCpWithAcls {
    * Runs distcp from /src to specified destination, preserving ACLs.  Asserts
    * expected exit code.
    *
-   * @param int exitCode expected exit code
+   * @param exitCode expected exit code
    * @param dst String distcp destination
    * @throws Exception if there is any error
    */

+ 28 - 35
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpWithRawXAttrs.java

@@ -43,18 +43,18 @@ import org.apache.hadoop.tools.util.DistCpTestUtils;
 
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.util.functional.RemoteIterators;
-import org.assertj.core.api.Assertions;
-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.apache.hadoop.thirdparty.com.google.common.collect.Maps;
 
 import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
-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.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 /**
  * Tests distcp in combination with HDFS raw.* XAttrs.
@@ -84,7 +84,7 @@ public class TestDistCpWithRawXAttrs {
 
   private static final String TEST_ROOT_DIR = base.getAbsolutePath();
 
-  @BeforeClass
+  @BeforeAll
   public static void init() throws Exception {
     conf = new Configuration();
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_KEY, true);
@@ -96,7 +96,7 @@ public class TestDistCpWithRawXAttrs {
     fs = cluster.getFileSystem();
   }
 
-  @AfterClass
+  @AfterAll
   public static void shutdown() {
     IOUtils.cleanupWithLogger(null, fs);
     if (cluster != null) {
@@ -213,7 +213,7 @@ public class TestDistCpWithRawXAttrs {
     fs.create(file1).close();
     fs.create(FILE_2).close();
     int res = ToolRunner.run(conf, new ECAdmin(conf), args);
-    assertEquals("Unable to set EC policy on " + subDir1.toString(), res, 0);
+    assertEquals(res, 0, "Unable to set EC policy on " + subDir1.toString());
 
     // preserve all attributes
     DistCpTestUtils.assertRunDistCp(DistCpConstants.SUCCESS, src, dest,
@@ -228,20 +228,13 @@ public class TestDistCpWithRawXAttrs {
     FileStatus destDir1Status = fs.getFileStatus(destDir1);
     FileStatus destSubDir1Status = fs.getFileStatus(destSubDir1);
 
-    assertFalse("/src is erasure coded!",
-        srcStatus.isErasureCoded());
-    assertFalse("/dest is erasure coded!",
-        destStatus.isErasureCoded());
-    assertTrue("/src/dir1 is not erasure coded!",
-        srcDir1Status.isErasureCoded());
-    assertTrue("/src/dir1/file2 is not erasure coded",
-        srcFile2Status.isErasureCoded());
-    assertTrue("/dest/dir1 is not erasure coded!",
-        destDir1Status.isErasureCoded());
-    assertTrue("/src/dir1/subdir1 is not erasure coded!",
-        srcSubDir1Status.isErasureCoded());
-    assertTrue("/dest/dir1/subdir1 is not erasure coded!",
-        destSubDir1Status.isErasureCoded());
+    assertFalse(srcStatus.isErasureCoded(), "/src is erasure coded!");
+    assertFalse(destStatus.isErasureCoded(), "/dest is erasure coded!");
+    assertTrue(srcDir1Status.isErasureCoded(), "/src/dir1 is not erasure coded!");
+    assertTrue(srcFile2Status.isErasureCoded(), "/src/dir1/file2 is not erasure coded");
+    assertTrue(destDir1Status.isErasureCoded(), "/dest/dir1 is not erasure coded!");
+    assertTrue(srcSubDir1Status.isErasureCoded(), "/src/dir1/subdir1 is not erasure coded!");
+    assertTrue(destSubDir1Status.isErasureCoded(), "/dest/dir1/subdir1 is not erasure coded!");
 
     // test without -p to check if src is EC then target FS default replication
     // is obeyed on the target file.
@@ -269,7 +262,7 @@ public class TestDistCpWithRawXAttrs {
     dfs.setErasureCodingPolicy(dir1, "XOR-2-1-1024k");
     fs.create(file1).close();
     int res = ToolRunner.run(conf, new ECAdmin(conf), args);
-    assertEquals("Unable to set EC policy on " + subDir1.toString(), 0, res);
+    assertEquals(0, res, "Unable to set EC policy on " + subDir1.toString());
     String src = "/src/*";
     Path dest = new Path(TEST_ROOT_DIR, "dest");
     final Path dest2Dir1 = new Path(dest, "dir1");
@@ -284,13 +277,13 @@ public class TestDistCpWithRawXAttrs {
       try {
         FileStatus destDir1Status = dummyEcFs.getFileStatus(dest2Dir1);
         FileStatus destSubDir1Status = dummyEcFs.getFileStatus(dest2SubDir1);
-        assertNotNull("FileStatus for path: " + dest2Dir1 + " is null", destDir1Status);
-        assertNotNull("FileStatus for path: " + dest2SubDir1 + " is null", destSubDir1Status);
+        assertNotNull(destDir1Status, "FileStatus for path: " + dest2Dir1 + " is null");
+        assertNotNull(destSubDir1Status, "FileStatus for path: " + dest2SubDir1 + " is null");
         // check if target paths are erasure coded.
-        assertTrue("Path is not erasure coded : " + dest2Dir1,
-            dummyEcFs.isPathErasureCoded(destDir1Status.getPath()));
-        assertTrue("Path is not erasure coded : " + dest2SubDir1,
-            dummyEcFs.isPathErasureCoded(destSubDir1Status.getPath()));
+        assertTrue(dummyEcFs.isPathErasureCoded(destDir1Status.getPath()),
+            "Path is not erasure coded : " + dest2Dir1);
+        assertTrue(dummyEcFs.isPathErasureCoded(destSubDir1Status.getPath()),
+            "Path is not erasure coded : " + dest2SubDir1);
 
         // copy source(DummyECFS) to target (HDFS)
         String dfsTarget = "/dest";
@@ -303,8 +296,8 @@ public class TestDistCpWithRawXAttrs {
         ContractTestUtils.assertPathExists(fs,
             "Path  doesn't exist:" + dfsTargetDir1, dfsTargetDir1);
         FileStatus targetDir1Status = fs.getFileStatus(dfsTargetDir1);
-        assertTrue("Path is not erasure coded : " + targetDir1Status,
-            targetDir1Status.isErasureCoded());
+        assertTrue(targetDir1Status.isErasureCoded(),
+            "Path is not erasure coded : " + targetDir1Status);
         fs.delete(dfsTargetPath, true);
       } finally {
         dummyEcFs.delete(new Path(base.getAbsolutePath()),true);
@@ -385,7 +378,7 @@ public class TestDistCpWithRawXAttrs {
     DistCpTestUtils.assertRunDistCp(DistCpConstants.SUCCESS, source.toString(),
         dest.toString(), "-useiterator", conf);
 
-    Assertions.assertThat(RemoteIterators.toList(fs.listFiles(dest, true)))
+    assertThat(RemoteIterators.toList(fs.listFiles(dest, true)))
         .describedAs("files").hasSize(1110);
   }
 }

+ 5 - 5
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpWithXAttrs.java

@@ -36,9 +36,9 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.tools.util.DistCpTestUtils;
 import org.apache.hadoop.util.Progressable;
 
-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.apache.hadoop.thirdparty.com.google.common.collect.Maps;
 
@@ -77,7 +77,7 @@ public class TestDistCpWithXAttrs {
   private static final Path dstFile4 = new Path(dstDir2, "file4");
   private static final String rootedSrcName = "/src";
 
-  @BeforeClass
+  @BeforeAll
   public static void init() throws Exception {
     initCluster(true, true);
     fs.mkdirs(subDir1);
@@ -112,7 +112,7 @@ public class TestDistCpWithXAttrs {
     fs.setXAttr(file3, name4, value4);
   }
 
-  @AfterClass
+  @AfterAll
   public static void shutdown() {
     IOUtils.cleanupWithLogger(null, fs);
     if (cluster != null) {

+ 37 - 30
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestExternalCall.java

@@ -19,7 +19,6 @@
 package org.apache.hadoop.tools;
 
 import org.apache.hadoop.mapreduce.Job;
-import org.mockito.Mockito;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -28,16 +27,25 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.Cluster;
 import org.apache.hadoop.mapreduce.JobSubmissionFiles;
 import org.apache.hadoop.tools.util.TestDistCpUtils;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 import java.io.IOException;
 import java.io.OutputStream;
 import java.security.Permission;
 
-import static org.mockito.Mockito.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.anyBoolean;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 public class TestExternalCall {
 
@@ -54,7 +62,7 @@ public class TestExternalCall {
     return conf;
   }
 
-  @Before
+  @BeforeEach
   public void setup() {
 
     securityManager = System.getSecurityManager();
@@ -69,7 +77,7 @@ public class TestExternalCall {
     }
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
     System.setSecurityManager(securityManager);
   }
@@ -80,21 +88,20 @@ public class TestExternalCall {
   @Test
   public void testCleanup() throws Exception {
 
-      Configuration conf = getConf();
+    Configuration conf = getConf();
 
-      Path stagingDir = JobSubmissionFiles.getStagingDir(new Cluster(conf),
-          conf);
-      stagingDir.getFileSystem(conf).mkdirs(stagingDir);
-      Path soure = createFile("tmp.txt");
-      Path target = createFile("target.txt");
+    Path stagingDir = JobSubmissionFiles.getStagingDir(new Cluster(conf),
+        conf);
+    stagingDir.getFileSystem(conf).mkdirs(stagingDir);
+    Path soure = createFile("tmp.txt");
+    Path target = createFile("target.txt");
 
-      DistCp distcp = new DistCp(conf, null);
-      String[] arg = { soure.toString(), target.toString() };
+    DistCp distcp = new DistCp(conf, null);
+    String[] arg = {soure.toString(), target.toString()};
 
-      distcp.run(arg);
-      Assert.assertTrue(fs.exists(target));
+    distcp.run(arg);
+    assertTrue(fs.exists(target));
 
-  
   }
 
   private Path createFile(String fname) throws IOException {
@@ -127,12 +134,12 @@ public class TestExternalCall {
 
       String[] arg = {target.toString(),soure.toString()};
       DistCp.main(arg);
-      Assert.fail();
+      fail();
 
     } catch (ExitException t) {
-      Assert.assertTrue(fs.exists(target));
-      Assert.assertEquals(t.status, 0);
-      Assert.assertEquals(
+      assertTrue(fs.exists(target));
+      assertEquals(t.status, 0);
+      assertEquals(
           stagingDir.getFileSystem(conf).listStatus(stagingDir).length, 0);
     }
 
@@ -155,16 +162,16 @@ public class TestExternalCall {
 
     DistCp distcp = mock(DistCp.class);
     Job job = spy(Job.class);
-    Mockito.when(distcp.getConf()).thenReturn(conf);
-    Mockito.when(distcp.createAndSubmitJob()).thenReturn(job);
-    Mockito.when(distcp.execute()).thenCallRealMethod();
-    Mockito.when(distcp.execute(Mockito.anyBoolean())).thenCallRealMethod();
-    Mockito.doReturn(true).when(job).waitForCompletion(Mockito.anyBoolean());
-    Mockito.when(distcp.run(Mockito.any())).thenCallRealMethod();
+    when(distcp.getConf()).thenReturn(conf);
+    when(distcp.createAndSubmitJob()).thenReturn(job);
+    when(distcp.execute()).thenCallRealMethod();
+    when(distcp.execute(anyBoolean())).thenCallRealMethod();
+    doReturn(true).when(job).waitForCompletion(anyBoolean());
+    when(distcp.run(any())).thenCallRealMethod();
     String[] arg = { soure.toString(), target.toString() };
 
     distcp.run(arg);
-    Mockito.verify(job, times(1)).close();
+    verify(job, times(1)).close();
   }
 
 

+ 25 - 23
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestFileBasedCopyListing.java

@@ -29,16 +29,18 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.tools.util.TestDistCpUtils;
 import org.apache.hadoop.security.Credentials;
-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 java.io.IOException;
 import java.io.OutputStream;
 import java.util.HashMap;
 import java.util.Map;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
 public class TestFileBasedCopyListing {
   private static final Logger LOG = LoggerFactory.getLogger(TestFileBasedCopyListing.class);
 
@@ -48,7 +50,7 @@ public class TestFileBasedCopyListing {
   private static MiniDFSCluster cluster;
   private static FileSystem fs;
 
-  @BeforeClass
+  @BeforeAll
   public static void create() throws IOException {
     cluster = new MiniDFSCluster.Builder(config).numDataNodes(1).format(true)
                                                 .build();
@@ -56,7 +58,7 @@ public class TestFileBasedCopyListing {
     buildExpectedValuesMap();
   }
 
-  @AfterClass
+  @AfterAll
   public static void destroy() {
     if (cluster != null) {
       cluster.shutdown();
@@ -116,7 +118,7 @@ public class TestFileBasedCopyListing {
       checkResult(listFile, 0);
     } catch (IOException e) {
       LOG.error("Exception encountered while testing build listing", e);
-      Assert.fail("build listing failure");
+      fail("build listing failure");
     } finally {
       TestDistCpUtils.delete(fs, "/tmp");
     }
@@ -142,7 +144,7 @@ public class TestFileBasedCopyListing {
       checkResult(listFile, 0);
     } catch (IOException e) {
       LOG.error("Exception encountered while testing build listing", e);
-      Assert.fail("build listing failure");
+      fail("build listing failure");
     } finally {
       TestDistCpUtils.delete(fs, "/tmp");
     }
@@ -169,7 +171,7 @@ public class TestFileBasedCopyListing {
       checkResult(listFile, 1);
     } catch (IOException e) {
       LOG.error("Exception encountered while testing build listing", e);
-      Assert.fail("build listing failure");
+      fail("build listing failure");
     } finally {
       TestDistCpUtils.delete(fs, "/tmp");
     }
@@ -195,7 +197,7 @@ public class TestFileBasedCopyListing {
       checkResult(listFile, 1);
     } catch (IOException e) {
       LOG.error("Exception encountered while testing build listing", e);
-      Assert.fail("build listing failure");
+      fail("build listing failure");
     } finally {
       TestDistCpUtils.delete(fs, "/tmp");
     }
@@ -217,7 +219,7 @@ public class TestFileBasedCopyListing {
       checkResult(listFile, 1);
     } catch (IOException e) {
       LOG.error("Exception encountered while testing build listing", e);
-      Assert.fail("build listing failure");
+      fail("build listing failure");
     } finally {
       TestDistCpUtils.delete(fs, "/tmp");
     }
@@ -239,7 +241,7 @@ public class TestFileBasedCopyListing {
       checkResult(listFile, 1);
     } catch (IOException e) {
       LOG.error("Exception encountered while testing build listing", e);
-      Assert.fail("build listing failure");
+      fail("build listing failure");
     } finally {
       TestDistCpUtils.delete(fs, "/tmp");
     }
@@ -266,7 +268,7 @@ public class TestFileBasedCopyListing {
       checkResult(listFile, 3);
     } catch (IOException e) {
       LOG.error("Exception encountered while testing build listing", e);
-      Assert.fail("build listing failure");
+      fail("build listing failure");
     } finally {
       TestDistCpUtils.delete(fs, "/tmp");
     }
@@ -292,7 +294,7 @@ public class TestFileBasedCopyListing {
       checkResult(listFile, 3);
     } catch (IOException e) {
       LOG.error("Exception encountered while testing build listing", e);
-      Assert.fail("build listing failure");
+      fail("build listing failure");
     } finally {
       TestDistCpUtils.delete(fs, "/tmp");
     }
@@ -314,7 +316,7 @@ public class TestFileBasedCopyListing {
       checkResult(listFile, 4);
     } catch (IOException e) {
       LOG.error("Exception encountered while testing build listing", e);
-      Assert.fail("build listing failure");
+      fail("build listing failure");
     } finally {
       TestDistCpUtils.delete(fs, "/tmp");
     }
@@ -336,7 +338,7 @@ public class TestFileBasedCopyListing {
       checkResult(listFile, 4);
     } catch (IOException e) {
       LOG.error("Exception encountered while testing build listing", e);
-      Assert.fail("build listing failure");
+      fail("build listing failure");
     } finally {
       TestDistCpUtils.delete(fs, "/tmp");
     }
@@ -363,7 +365,7 @@ public class TestFileBasedCopyListing {
       checkResult(listFile, 4);
     } catch (IOException e) {
       LOG.error("Exception encountered while testing build listing", e);
-      Assert.fail("build listing failure");
+      fail("build listing failure");
     } finally {
       TestDistCpUtils.delete(fs, "/tmp");
     }
@@ -390,7 +392,7 @@ public class TestFileBasedCopyListing {
       checkResult(listFile, 5);
     } catch (IOException e) {
       LOG.error("Exception encountered while testing build listing", e);
-      Assert.fail("build listing failure");
+      fail("build listing failure");
     } finally {
       TestDistCpUtils.delete(fs, "/tmp");
       TestDistCpUtils.delete(fs, "/tmp1");
@@ -419,7 +421,7 @@ public class TestFileBasedCopyListing {
       checkResult(listFile, 6);
     } catch (IOException e) {
       LOG.error("Exception encountered while testing build listing", e);
-      Assert.fail("build listing failure");
+      fail("build listing failure");
     } finally {
       TestDistCpUtils.delete(fs, "/tmp");
       TestDistCpUtils.delete(fs, "/tmp1");
@@ -444,7 +446,7 @@ public class TestFileBasedCopyListing {
       checkResult(listFile, 6);
     } catch (IOException e) {
       LOG.error("Exception encountered while testing build listing", e);
-      Assert.fail("build listing failure");
+      fail("build listing failure");
     } finally {
       TestDistCpUtils.delete(fs, "/tmp");
       TestDistCpUtils.delete(fs, "/tmp1");
@@ -469,7 +471,7 @@ public class TestFileBasedCopyListing {
       checkResult(listFile, 6);
     } catch (IOException e) {
       LOG.error("Exception encountered while testing build listing", e);
-      Assert.fail("build listing failure");
+      fail("build listing failure");
     } finally {
       TestDistCpUtils.delete(fs, "/tmp");
       TestDistCpUtils.delete(fs, "/tmp1");
@@ -538,13 +540,13 @@ public class TestFileBasedCopyListing {
           // used for preserving root attributes etc.
           continue;
         }
-        Assert.assertEquals(fileStatus.getPath().toUri().getPath(), map.get(relPath.toString()));
+        assertEquals(fileStatus.getPath().toUri().getPath(), map.get(relPath.toString()));
         recCount++;
       }
     } finally {
       IOUtils.closeStream(reader);
     }
-    Assert.assertEquals(recCount, count);
+    assertEquals(recCount, count);
   }
 
 }

+ 9 - 8
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestGlobbedCopyListing.java

@@ -27,10 +27,9 @@ import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.tools.util.DistCpUtils;
 import org.apache.hadoop.security.Credentials;
-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 java.io.DataOutputStream;
 import java.net.URI;
@@ -38,6 +37,8 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
 public class TestGlobbedCopyListing {
 
   private static MiniDFSCluster cluster;
@@ -46,7 +47,7 @@ public class TestGlobbedCopyListing {
 
   public static Map<String, String> expectedValues = new HashMap<String, String>();
 
-  @BeforeClass
+  @BeforeAll
   public static void setup() throws Exception {
     cluster = new MiniDFSCluster.Builder(new Configuration()).build();
     createSourceData();
@@ -96,7 +97,7 @@ public class TestGlobbedCopyListing {
         new Path("/tmp/source"), sourcePath));
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() {
     cluster.shutdown();
   }
@@ -134,9 +135,9 @@ public class TestGlobbedCopyListing {
       actualValues.put(value.getPath().toString(), key.toString());
     }
 
-    Assert.assertEquals(expectedValues.size(), actualValues.size());
+    assertEquals(expectedValues.size(), actualValues.size());
     for (Map.Entry<String, String> entry : actualValues.entrySet()) {
-      Assert.assertEquals(entry.getValue(), expectedValues.get(entry.getKey()));
+      assertEquals(entry.getValue(), expectedValues.get(entry.getKey()));
     }
   }
 }

+ 122 - 77
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestIntegration.java

@@ -27,12 +27,10 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.Cluster;
 import org.apache.hadoop.mapreduce.JobSubmissionFiles;
 import org.apache.hadoop.tools.util.TestDistCpUtils;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Timeout;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 
 import java.io.IOException;
 import java.io.OutputStream;
@@ -41,7 +39,11 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
 
-@RunWith(value = Parameterized.class)
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+
 public class TestIntegration {
   private static final Logger LOG = LoggerFactory.getLogger(TestIntegration.class);
 
@@ -52,13 +54,12 @@ public class TestIntegration {
   private static String root;
   private int numListstatusThreads;
 
-  public TestIntegration(int numListstatusThreads) {
-    this.numListstatusThreads = numListstatusThreads;
+  public void initTestIntegration(int pNumListstatusThreads) {
+    this.numListstatusThreads = pNumListstatusThreads;
   }
 
-  @Parameters
   public static Collection<Object[]> data() {
-    Object[][] data = new Object[][] { { 1 }, { 2 }, { 10 } };
+    Object[][] data = new Object[][]{{1}, {2}, {10}};
     return Arrays.asList(data);
   }
 
@@ -69,7 +70,7 @@ public class TestIntegration {
     return conf;
   }
 
-  @BeforeClass
+  @BeforeAll
   public static void setup() {
     try {
       fs = FileSystem.get(getConf());
@@ -85,8 +86,11 @@ public class TestIntegration {
     }
   }
 
-  @Test(timeout=100000)
-  public void testSingleFileMissingTarget() {
+  @Timeout(value = 100)
+  @ParameterizedTest
+  @MethodSource("data")
+  public void testSingleFileMissingTarget(int pNumListStatusThreads) {
+    initTestIntegration(pNumListStatusThreads);
     caseSingleFileMissingTarget(false);
     caseSingleFileMissingTarget(true);
   }
@@ -102,14 +106,17 @@ public class TestIntegration {
       checkResult(target, 1);
     } catch (IOException e) {
       LOG.error("Exception encountered while testing distcp", e);
-      Assert.fail("distcp failure");
+      fail("distcp failure");
     } finally {
       TestDistCpUtils.delete(fs, root);
     }
   }
 
-  @Test(timeout=100000)
-  public void testSingleFileTargetFile() {
+  @ParameterizedTest
+  @Timeout(value = 100)
+  @MethodSource("data")
+  public void testSingleFileTargetFile(int pNumListStatusThreads) {
+    initTestIntegration(pNumListStatusThreads);
     caseSingleFileTargetFile(false);
     caseSingleFileTargetFile(true);
   }
@@ -125,14 +132,17 @@ public class TestIntegration {
       checkResult(target, 1);
     } catch (IOException e) {
       LOG.error("Exception encountered while testing distcp", e);
-      Assert.fail("distcp failure");
+      fail("distcp failure");
     } finally {
       TestDistCpUtils.delete(fs, root);
     }
   }
 
-  @Test(timeout=100000)
-  public void testSingleFileTargetDir() {
+  @ParameterizedTest
+  @Timeout(value = 100)
+  @MethodSource("data")
+  public void testSingleFileTargetDir(int pNumListStatusThreads) {
+    initTestIntegration(pNumListStatusThreads);
     caseSingleFileTargetDir(false);
     caseSingleFileTargetDir(true);
   }
@@ -149,14 +159,17 @@ public class TestIntegration {
       checkResult(target, 1, "file2");
     } catch (IOException e) {
       LOG.error("Exception encountered while testing distcp", e);
-      Assert.fail("distcp failure");
+      fail("distcp failure");
     } finally {
       TestDistCpUtils.delete(fs, root);
     }
   }
 
-  @Test(timeout=100000)
-  public void testSingleDirTargetMissing() {
+  @ParameterizedTest
+  @Timeout(value = 100)
+  @MethodSource("data")
+  public void testSingleDirTargetMissing(int pNumListStatusThreads) {
+    initTestIntegration(pNumListStatusThreads);
     caseSingleDirTargetMissing(false);
     caseSingleDirTargetMissing(true);
   }
@@ -172,15 +185,17 @@ public class TestIntegration {
       checkResult(target, 1, "dir1");
     } catch (IOException e) {
       LOG.error("Exception encountered while testing distcp", e);
-      Assert.fail("distcp failure");
+      fail("distcp failure");
     } finally {
       TestDistCpUtils.delete(fs, root);
     }
   }
 
-  @Test(timeout=100000)
-  public void testSingleDirTargetPresent() {
-
+  @ParameterizedTest
+  @Timeout(value = 100)
+  @MethodSource("data")
+  public void testSingleDirTargetPresent(int pNumListStatusThreads) {
+    initTestIntegration(pNumListStatusThreads);
     try {
       addEntries(listFile, "singledir");
       mkdirs(root + "/singledir/dir1");
@@ -191,15 +206,17 @@ public class TestIntegration {
       checkResult(target, 1, "singledir/dir1");
     } catch (IOException e) {
       LOG.error("Exception encountered while testing distcp", e);
-      Assert.fail("distcp failure");
+      fail("distcp failure");
     } finally {
       TestDistCpUtils.delete(fs, root);
     }
   }
 
-  @Test(timeout=100000)
-  public void testUpdateSingleDirTargetPresent() {
-
+  @ParameterizedTest
+  @Timeout(value = 100)
+  @MethodSource("data")
+  public void testUpdateSingleDirTargetPresent(int pNumListStatusThreads) {
+    initTestIntegration(pNumListStatusThreads);
     try {
       addEntries(listFile, "Usingledir");
       mkdirs(root + "/Usingledir/Udir1");
@@ -210,14 +227,17 @@ public class TestIntegration {
       checkResult(target, 1, "Udir1");
     } catch (IOException e) {
       LOG.error("Exception encountered while testing distcp", e);
-      Assert.fail("distcp failure");
+      fail("distcp failure");
     } finally {
       TestDistCpUtils.delete(fs, root);
     }
   }
 
-  @Test(timeout=100000)
-  public void testMultiFileTargetPresent() {
+  @ParameterizedTest
+  @Timeout(value = 100)
+  @MethodSource("data")
+  public void testMultiFileTargetPresent(int pNumListStatusThreads) {
+    initTestIntegration(pNumListStatusThreads);
     caseMultiFileTargetPresent(false);
     caseMultiFileTargetPresent(true);
   }
@@ -234,14 +254,17 @@ public class TestIntegration {
       checkResult(target, 3, "file3", "file4", "file5");
     } catch (IOException e) {
       LOG.error("Exception encountered while testing distcp", e);
-      Assert.fail("distcp failure");
+      fail("distcp failure");
     } finally {
       TestDistCpUtils.delete(fs, root);
     }
   }
 
-  @Test(timeout=100000)
-  public void testMultiFileTargetMissing() {
+  @ParameterizedTest
+  @Timeout(value = 100)
+  @MethodSource("data")
+  public void testMultiFileTargetMissing(int pNumListStatusThreads) {
+    initTestIntegration(pNumListStatusThreads);
     caseMultiFileTargetMissing(false);
     caseMultiFileTargetMissing(true);
   }
@@ -257,15 +280,17 @@ public class TestIntegration {
       checkResult(target, 3, "file3", "file4", "file5");
     } catch (IOException e) {
       LOG.error("Exception encountered while testing distcp", e);
-      Assert.fail("distcp failure");
+      fail("distcp failure");
     } finally {
       TestDistCpUtils.delete(fs, root);
     }
   }
 
-  @Test(timeout=100000)
-  public void testMultiDirTargetPresent() {
-
+  @ParameterizedTest
+  @Timeout(value = 100)
+  @MethodSource("data")
+  public void testMultiDirTargetPresent(int pNumListStatusThreads) {
+    initTestIntegration(pNumListStatusThreads);
     try {
       addEntries(listFile, "multifile", "singledir");
       createFiles("multifile/file3", "multifile/file4", "multifile/file5");
@@ -276,15 +301,17 @@ public class TestIntegration {
       checkResult(target, 2, "multifile/file3", "multifile/file4", "multifile/file5", "singledir/dir1");
     } catch (IOException e) {
       LOG.error("Exception encountered while testing distcp", e);
-      Assert.fail("distcp failure");
+      fail("distcp failure");
     } finally {
       TestDistCpUtils.delete(fs, root);
     }
   }
 
-  @Test(timeout=100000)
-  public void testUpdateMultiDirTargetPresent() {
-
+  @ParameterizedTest
+  @Timeout(value = 100)
+  @MethodSource("data")
+  public void testUpdateMultiDirTargetPresent(int pNumListStatusThreads) {
+    initTestIntegration(pNumListStatusThreads);
     try {
       addEntries(listFile, "Umultifile", "Usingledir");
       createFiles("Umultifile/Ufile3", "Umultifile/Ufile4", "Umultifile/Ufile5");
@@ -295,15 +322,17 @@ public class TestIntegration {
       checkResult(target, 4, "Ufile3", "Ufile4", "Ufile5", "Udir1");
     } catch (IOException e) {
       LOG.error("Exception encountered while testing distcp", e);
-      Assert.fail("distcp failure");
+      fail("distcp failure");
     } finally {
       TestDistCpUtils.delete(fs, root);
     }
   }
 
-  @Test(timeout=100000)
-  public void testMultiDirTargetMissing() {
-
+  @ParameterizedTest
+  @Timeout(value = 100)
+  @MethodSource("data")
+  public void testMultiDirTargetMissing(int pNumListStatusThreads) {
+    initTestIntegration(pNumListStatusThreads);
     try {
       addEntries(listFile, "multifile", "singledir");
       createFiles("multifile/file3", "multifile/file4", "multifile/file5");
@@ -315,15 +344,17 @@ public class TestIntegration {
           "multifile/file5", "singledir/dir1");
     } catch (IOException e) {
       LOG.error("Exception encountered while testing distcp", e);
-      Assert.fail("distcp failure");
+      fail("distcp failure");
     } finally {
       TestDistCpUtils.delete(fs, root);
     }
   }
 
-  @Test(timeout=100000)
-  public void testUpdateMultiDirTargetMissing() {
-
+  @ParameterizedTest
+  @Timeout(value = 100)
+  @MethodSource("data")
+  public void testUpdateMultiDirTargetMissing(int pNumListStatusThreads) {
+    initTestIntegration(pNumListStatusThreads);
     try {
       addEntries(listFile, "multifile", "singledir");
       createFiles("multifile/file3", "multifile/file4", "multifile/file5");
@@ -334,13 +365,15 @@ public class TestIntegration {
       checkResult(target, 4, "file3", "file4", "file5", "dir1");
     } catch (IOException e) {
       LOG.error("Exception encountered while testing distcp", e);
-      Assert.fail("distcp failure");
+      fail("distcp failure");
     } finally {
       TestDistCpUtils.delete(fs, root);
     }
   }
-  
-  @Test(timeout=100000)
+
+  @ParameterizedTest
+  @Timeout(value = 100)
+  @MethodSource("data")
   public void testDeleteMissingInDestination() {
     
     try {
@@ -353,18 +386,20 @@ public class TestIntegration {
       checkResult(target, 1, "file1");
     } catch (IOException e) {
       LOG.error("Exception encountered while running distcp", e);
-      Assert.fail("distcp failure");
+      fail("distcp failure");
     } finally {
       TestDistCpUtils.delete(fs, root);
       TestDistCpUtils.delete(fs, "target/tmp1");
     }
   }
-  
-  @Test(timeout=100000)
+
+  @ParameterizedTest
+  @Timeout(value = 100)
+  @MethodSource("data")
   public void testOverwrite() {
     byte[] contents1 = "contents1".getBytes();
     byte[] contents2 = "contents2".getBytes();
-    Assert.assertEquals(contents1.length, contents2.length);
+    assertEquals(contents1.length, contents2.length);
     
     try {
       addEntries(listFile, "srcdir");
@@ -382,17 +417,19 @@ public class TestIntegration {
       byte[] dstContents = new byte[contents1.length];
       is.readFully(dstContents);
       is.close();
-      Assert.assertArrayEquals(contents1, dstContents);
+      assertArrayEquals(contents1, dstContents);
     } catch (IOException e) {
       LOG.error("Exception encountered while running distcp", e);
-      Assert.fail("distcp failure");
+      fail("distcp failure");
     } finally {
       TestDistCpUtils.delete(fs, root);
       TestDistCpUtils.delete(fs, "target/tmp1");
     }
   }
 
-  @Test(timeout=100000)
+  @ParameterizedTest
+  @Timeout(value = 100)
+  @MethodSource("data")
   public void testGlobTargetMissingSingleLevel() {
 
     try {
@@ -408,14 +445,16 @@ public class TestIntegration {
           "singledir/dir2/file6");
     } catch (IOException e) {
       LOG.error("Exception encountered while testing distcp", e);
-      Assert.fail("distcp failure");
+      fail("distcp failure");
     } finally {
       TestDistCpUtils.delete(fs, root);
       TestDistCpUtils.delete(fs, "target/tmp1");
     }
   }
 
-  @Test(timeout=100000)
+  @ParameterizedTest
+  @Timeout(value = 100)
+  @MethodSource("data")
   public void testUpdateGlobTargetMissingSingleLevel() {
 
     try {
@@ -430,14 +469,16 @@ public class TestIntegration {
       checkResult(target, 4, "file3", "file4", "file5", "dir2/file6");
     } catch (IOException e) {
       LOG.error("Exception encountered while running distcp", e);
-      Assert.fail("distcp failure");
+      fail("distcp failure");
     } finally {
       TestDistCpUtils.delete(fs, root);
       TestDistCpUtils.delete(fs, "target/tmp1");
     }
   }
 
-  @Test(timeout=100000)
+  @ParameterizedTest
+  @Timeout(value = 100)
+  @MethodSource("data")
   public void testGlobTargetMissingMultiLevel() {
 
     try {
@@ -454,14 +495,16 @@ public class TestIntegration {
           "dir3/file7", "dir3/file8", "dir3/file9");
     } catch (IOException e) {
       LOG.error("Exception encountered while running distcp", e);
-      Assert.fail("distcp failure");
+      fail("distcp failure");
     } finally {
       TestDistCpUtils.delete(fs, root);
       TestDistCpUtils.delete(fs, "target/tmp1");
     }
   }
 
-  @Test(timeout=100000)
+  @ParameterizedTest
+  @Timeout(value = 100)
+  @MethodSource("data")
   public void testUpdateGlobTargetMissingMultiLevel() {
 
     try {
@@ -478,14 +521,16 @@ public class TestIntegration {
           "file7", "file8", "file9");
     } catch (IOException e) {
       LOG.error("Exception encountered while running distcp", e);
-      Assert.fail("distcp failure");
+      fail("distcp failure");
     } finally {
       TestDistCpUtils.delete(fs, root);
       TestDistCpUtils.delete(fs, "target/tmp1");
     }
   }
-  
-  @Test(timeout=100000)
+
+  @ParameterizedTest
+  @Timeout(value = 100)
+  @MethodSource("data")
   public void testCleanup() {
     try {
       Path sourcePath = new Path("noscheme:///file");
@@ -503,12 +548,12 @@ public class TestIntegration {
       try {
         new DistCp(conf, options).execute();
       } catch (Throwable t) {
-        Assert.assertEquals(stagingDir.getFileSystem(conf).
+        assertEquals(stagingDir.getFileSystem(conf).
             listStatus(stagingDir).length, 0);
       }
     } catch (Exception e) {
       LOG.error("Exception encountered ", e);
-      Assert.fail("testCleanup failed " + e.getMessage());
+      fail("testCleanup failed " + e.getMessage());
     }
   }
   
@@ -576,13 +621,13 @@ public class TestIntegration {
   }
 
   private void checkResult(Path target, int count, String... relPaths) throws IOException {
-    Assert.assertEquals(count, fs.listStatus(target).length);
+    assertEquals(count, fs.listStatus(target).length);
     if (relPaths == null || relPaths.length == 0) {
-      Assert.assertTrue(target.toString(), fs.exists(target));
+      assertTrue(fs.exists(target), target.toString());
       return;
     }
     for (String relPath : relPaths) {
-      Assert.assertTrue(new Path(target, relPath).toString(), fs.exists(new Path(target, relPath)));
+      assertTrue(fs.exists(new Path(target, relPath)), new Path(target, relPath).toString());
     }
   }
 

+ 160 - 152
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestOptionsParser.java

@@ -19,12 +19,16 @@
 package org.apache.hadoop.tools;
 
 import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
-import static org.assertj.core.api.Assertions.within;
-import static org.junit.Assert.fail;
 import static org.assertj.core.api.Assertions.assertThat;
-
-import org.junit.Assert;
-import org.junit.Test;
+import static org.assertj.core.api.Assertions.within;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+
+import org.junit.jupiter.api.Test;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.tools.DistCpOptions.*;
@@ -41,13 +45,13 @@ public class TestOptionsParser {
     DistCpOptions options = OptionsParser.parse(new String[] {
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertFalse(options.shouldIgnoreFailures());
+    assertFalse(options.shouldIgnoreFailures());
 
     options = OptionsParser.parse(new String[] {
         "-i",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertTrue(options.shouldIgnoreFailures());
+    assertTrue(options.shouldIgnoreFailures());
   }
 
   @Test
@@ -55,13 +59,13 @@ public class TestOptionsParser {
     DistCpOptions options = OptionsParser.parse(new String[] {
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertFalse(options.shouldOverwrite());
+    assertFalse(options.shouldOverwrite());
 
     options = OptionsParser.parse(new String[] {
         "-overwrite",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertTrue(options.shouldOverwrite());
+    assertTrue(options.shouldOverwrite());
 
     try {
       OptionsParser.parse(new String[] {
@@ -69,7 +73,7 @@ public class TestOptionsParser {
           "-overwrite",
           "hdfs://localhost:8020/source/first",
           "hdfs://localhost:8020/target/"});
-      Assert.fail("Update and overwrite aren't allowed together");
+      fail("Update and overwrite aren't allowed together");
     } catch (IllegalArgumentException ignore) {
     }
   }
@@ -79,14 +83,14 @@ public class TestOptionsParser {
     DistCpOptions options = OptionsParser.parse(new String[] {
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertNull(options.getLogPath());
+    assertNull(options.getLogPath());
 
     options = OptionsParser.parse(new String[] {
         "-log",
         "hdfs://localhost:8020/logs",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertEquals(options.getLogPath(), new Path("hdfs://localhost:8020/logs"));
+    assertEquals(options.getLogPath(), new Path("hdfs://localhost:8020/logs"));
   }
 
   @Test
@@ -94,13 +98,13 @@ public class TestOptionsParser {
     DistCpOptions options = OptionsParser.parse(new String[] {
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertTrue(options.shouldBlock());
+    assertTrue(options.shouldBlock());
 
     options = OptionsParser.parse(new String[] {
         "-async",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertFalse(options.shouldBlock());
+    assertFalse(options.shouldBlock());
   }
 
   @Test
@@ -118,22 +122,26 @@ public class TestOptionsParser {
     assertThat(options.getMapBandwidth()).isCloseTo(11.2f, within(DELTA));
   }
 
-  @Test(expected=IllegalArgumentException.class)
+  @Test
   public void testParseNonPositiveBandwidth() {
-    OptionsParser.parse(new String[] {
-        "-bandwidth",
-        "-11",
-        "hdfs://localhost:8020/source/first",
-        "hdfs://localhost:8020/target/"});
+    assertThrows(IllegalArgumentException.class, () -> {
+      OptionsParser.parse(new String[]{
+          "-bandwidth",
+          "-11",
+          "hdfs://localhost:8020/source/first",
+          "hdfs://localhost:8020/target/"});
+    });
   }
 
-  @Test(expected=IllegalArgumentException.class)
+  @Test
   public void testParseZeroBandwidth() {
-    OptionsParser.parse(new String[] {
-        "-bandwidth",
-        "0",
-        "hdfs://localhost:8020/source/first",
-        "hdfs://localhost:8020/target/"});
+    assertThrows(IllegalArgumentException.class, () -> {
+      OptionsParser.parse(new String[]{
+          "-bandwidth",
+          "0",
+          "hdfs://localhost:8020/source/first",
+          "hdfs://localhost:8020/target/"});
+    });
   }
 
   @Test
@@ -141,15 +149,15 @@ public class TestOptionsParser {
     DistCpOptions options = OptionsParser.parse(new String[] {
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertFalse(options.shouldSkipCRC());
+    assertFalse(options.shouldSkipCRC());
 
     options = OptionsParser.parse(new String[] {
         "-update",
         "-skipcrccheck",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertTrue(options.shouldSyncFolder());
-    Assert.assertTrue(options.shouldSkipCRC());
+    assertTrue(options.shouldSyncFolder());
+    assertTrue(options.shouldSkipCRC());
   }
 
   @Test
@@ -157,13 +165,13 @@ public class TestOptionsParser {
     DistCpOptions options = OptionsParser.parse(new String[] {
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertFalse(options.shouldAtomicCommit());
+    assertFalse(options.shouldAtomicCommit());
 
     options = OptionsParser.parse(new String[] {
         "-atomic",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertTrue(options.shouldAtomicCommit());
+    assertTrue(options.shouldAtomicCommit());
 
     try {
       OptionsParser.parse(new String[] {
@@ -171,7 +179,7 @@ public class TestOptionsParser {
           "-update",
           "hdfs://localhost:8020/source/first",
           "hdfs://localhost:8020/target/"});
-      Assert.fail("Atomic and sync folders were allowed");
+      fail("Atomic and sync folders were allowed");
     } catch (IllegalArgumentException ignore) { }
   }
 
@@ -180,13 +188,13 @@ public class TestOptionsParser {
     DistCpOptions options = OptionsParser.parse(new String[] {
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertNull(options.getAtomicWorkPath());
+    assertNull(options.getAtomicWorkPath());
 
     options = OptionsParser.parse(new String[] {
         "-atomic",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertNull(options.getAtomicWorkPath());
+    assertNull(options.getAtomicWorkPath());
 
     options = OptionsParser.parse(new String[] {
         "-atomic",
@@ -194,7 +202,7 @@ public class TestOptionsParser {
         "hdfs://localhost:8020/work",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertEquals(options.getAtomicWorkPath(), new Path("hdfs://localhost:8020/work"));
+    assertEquals(options.getAtomicWorkPath(), new Path("hdfs://localhost:8020/work"));
 
     try {
       OptionsParser.parse(new String[] {
@@ -202,7 +210,7 @@ public class TestOptionsParser {
           "hdfs://localhost:8020/work",
           "hdfs://localhost:8020/source/first",
           "hdfs://localhost:8020/target/"});
-      Assert.fail("work path was allowed without -atomic switch");
+      fail("work path was allowed without -atomic switch");
     } catch (IllegalArgumentException ignore) {}
   }
 
@@ -211,13 +219,13 @@ public class TestOptionsParser {
     DistCpOptions options = OptionsParser.parse(new String[] {
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertFalse(options.shouldSyncFolder());
+    assertFalse(options.shouldSyncFolder());
 
     options = OptionsParser.parse(new String[] {
         "-update",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertTrue(options.shouldSyncFolder());
+    assertTrue(options.shouldSyncFolder());
   }
 
   @Test
@@ -225,23 +233,23 @@ public class TestOptionsParser {
     DistCpOptions options = OptionsParser.parse(new String[] {
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertFalse(options.shouldDeleteMissing());
+    assertFalse(options.shouldDeleteMissing());
 
     options = OptionsParser.parse(new String[] {
         "-update",
         "-delete",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertTrue(options.shouldSyncFolder());
-    Assert.assertTrue(options.shouldDeleteMissing());
+    assertTrue(options.shouldSyncFolder());
+    assertTrue(options.shouldDeleteMissing());
 
     options = OptionsParser.parse(new String[] {
         "-overwrite",
         "-delete",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertTrue(options.shouldOverwrite());
-    Assert.assertTrue(options.shouldDeleteMissing());
+    assertTrue(options.shouldOverwrite());
+    assertTrue(options.shouldDeleteMissing());
 
     try {
       OptionsParser.parse(new String[] {
@@ -249,7 +257,7 @@ public class TestOptionsParser {
           "-delete",
           "hdfs://localhost:8020/source/first",
           "hdfs://localhost:8020/target/"});
-      Assert.fail("Atomic and delete folders were allowed");
+      fail("Atomic and delete folders were allowed");
     } catch (IllegalArgumentException ignore) { }
   }
 
@@ -280,7 +288,7 @@ public class TestOptionsParser {
           "hello",
           "hdfs://localhost:8020/source/first",
           "hdfs://localhost:8020/target/"});
-      Assert.fail("Non numberic map parsed");
+      fail("Non numberic map parsed");
     } catch (IllegalArgumentException ignore) { }
 
     try {
@@ -288,7 +296,7 @@ public class TestOptionsParser {
           "-mapredXslConf",
           "hdfs://localhost:8020/source/first",
           "hdfs://localhost:8020/target/"});
-      Assert.fail("Non numberic map parsed");
+      fail("Non numberic map parsed");
     } catch (IllegalArgumentException ignore) { }
   }
 
@@ -299,21 +307,21 @@ public class TestOptionsParser {
         "hdfs://localhost:8020/target/"});
     // If command line argument isn't set, we expect .getNumListstatusThreads
     // option to be zero (so that we know when to override conf properties).
-    Assert.assertEquals(0, options.getNumListstatusThreads());
+    assertEquals(0, options.getNumListstatusThreads());
 
     options = OptionsParser.parse(new String[] {
         "--numListstatusThreads",
         "12",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertEquals(12, options.getNumListstatusThreads());
+    assertEquals(12, options.getNumListstatusThreads());
 
     options = OptionsParser.parse(new String[] {
         "--numListstatusThreads",
         "0",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertEquals(0, options.getNumListstatusThreads());
+    assertEquals(0, options.getNumListstatusThreads());
 
     try {
       OptionsParser.parse(new String[] {
@@ -321,7 +329,7 @@ public class TestOptionsParser {
           "hello",
           "hdfs://localhost:8020/source/first",
           "hdfs://localhost:8020/target/"});
-      Assert.fail("Non numberic numListstatusThreads parsed");
+      fail("Non numberic numListstatusThreads parsed");
     } catch (IllegalArgumentException ignore) { }
 
     // Ignore large number of threads.
@@ -330,7 +338,7 @@ public class TestOptionsParser {
         "100",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertEquals(DistCpOptions.MAX_NUM_LISTSTATUS_THREADS,
+    assertEquals(DistCpOptions.MAX_NUM_LISTSTATUS_THREADS,
                         options.getNumListstatusThreads());
   }
 
@@ -340,7 +348,7 @@ public class TestOptionsParser {
         "-f",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertEquals(options.getSourceFileListing(),
+    assertEquals(options.getSourceFileListing(),
         new Path("hdfs://localhost:8020/source/first"));
   }
 
@@ -352,7 +360,7 @@ public class TestOptionsParser {
           "hdfs://localhost:8020/source/first",
           "hdfs://localhost:8020/source/first",
           "hdfs://localhost:8020/target/"});
-      Assert.fail("Both source listing & source paths allowed");
+      fail("Both source listing & source paths allowed");
     } catch (IllegalArgumentException ignore) {}
   }
 
@@ -361,7 +369,7 @@ public class TestOptionsParser {
     try {
       OptionsParser.parse(new String[] {
           "hdfs://localhost:8020/target/"});
-      Assert.fail("Neither source listing not source paths present");
+      fail("Neither source listing not source paths present");
     } catch (IllegalArgumentException ignore) {}
   }
 
@@ -370,7 +378,7 @@ public class TestOptionsParser {
     try {
       OptionsParser.parse(new String[] {
           "-f", "hdfs://localhost:8020/source"});
-      Assert.fail("Missing target allowed");
+      fail("Missing target allowed");
     } catch (IllegalArgumentException ignore) {}
   }
 
@@ -379,7 +387,7 @@ public class TestOptionsParser {
     try {
       OptionsParser.parse(new String[] {
           "-m", "-f", "hdfs://localhost:8020/source"});
-      Assert.fail("Missing map value");
+      fail("Missing map value");
     } catch (IllegalArgumentException ignore) {}
   }
 
@@ -406,7 +414,7 @@ public class TestOptionsParser {
         "-f",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertEquals(options.getTargetPath(), new Path("hdfs://localhost:8020/target/"));
+    assertEquals(options.getTargetPath(), new Path("hdfs://localhost:8020/target/"));
   }
 
   @Test
@@ -415,103 +423,103 @@ public class TestOptionsParser {
         "-f",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.BLOCKSIZE));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.REPLICATION));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.PERMISSION));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.USER));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.GROUP));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.CHECKSUMTYPE));
+    assertFalse(options.shouldPreserve(FileAttribute.BLOCKSIZE));
+    assertFalse(options.shouldPreserve(FileAttribute.REPLICATION));
+    assertFalse(options.shouldPreserve(FileAttribute.PERMISSION));
+    assertFalse(options.shouldPreserve(FileAttribute.USER));
+    assertFalse(options.shouldPreserve(FileAttribute.GROUP));
+    assertFalse(options.shouldPreserve(FileAttribute.CHECKSUMTYPE));
 
     options = OptionsParser.parse(new String[] {
         "-p",
         "-f",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.BLOCKSIZE));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.REPLICATION));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.PERMISSION));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.USER));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.GROUP));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.CHECKSUMTYPE));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.ACL));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.XATTR));
+    assertTrue(options.shouldPreserve(FileAttribute.BLOCKSIZE));
+    assertTrue(options.shouldPreserve(FileAttribute.REPLICATION));
+    assertTrue(options.shouldPreserve(FileAttribute.PERMISSION));
+    assertTrue(options.shouldPreserve(FileAttribute.USER));
+    assertTrue(options.shouldPreserve(FileAttribute.GROUP));
+    assertTrue(options.shouldPreserve(FileAttribute.CHECKSUMTYPE));
+    assertFalse(options.shouldPreserve(FileAttribute.ACL));
+    assertFalse(options.shouldPreserve(FileAttribute.XATTR));
 
     options = OptionsParser.parse(new String[] {
         "-p",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.BLOCKSIZE));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.REPLICATION));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.PERMISSION));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.USER));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.GROUP));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.CHECKSUMTYPE));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.ACL));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.XATTR));
+    assertTrue(options.shouldPreserve(FileAttribute.BLOCKSIZE));
+    assertTrue(options.shouldPreserve(FileAttribute.REPLICATION));
+    assertTrue(options.shouldPreserve(FileAttribute.PERMISSION));
+    assertTrue(options.shouldPreserve(FileAttribute.USER));
+    assertTrue(options.shouldPreserve(FileAttribute.GROUP));
+    assertTrue(options.shouldPreserve(FileAttribute.CHECKSUMTYPE));
+    assertFalse(options.shouldPreserve(FileAttribute.ACL));
+    assertFalse(options.shouldPreserve(FileAttribute.XATTR));
 
     options = OptionsParser.parse(new String[] {
         "-pbr",
         "-f",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.BLOCKSIZE));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.REPLICATION));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.PERMISSION));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.USER));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.GROUP));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.CHECKSUMTYPE));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.ACL));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.XATTR));
+    assertTrue(options.shouldPreserve(FileAttribute.BLOCKSIZE));
+    assertTrue(options.shouldPreserve(FileAttribute.REPLICATION));
+    assertFalse(options.shouldPreserve(FileAttribute.PERMISSION));
+    assertFalse(options.shouldPreserve(FileAttribute.USER));
+    assertFalse(options.shouldPreserve(FileAttribute.GROUP));
+    assertFalse(options.shouldPreserve(FileAttribute.CHECKSUMTYPE));
+    assertFalse(options.shouldPreserve(FileAttribute.ACL));
+    assertFalse(options.shouldPreserve(FileAttribute.XATTR));
 
     options = OptionsParser.parse(new String[] {
         "-pbrgup",
         "-f",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.BLOCKSIZE));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.REPLICATION));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.PERMISSION));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.USER));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.GROUP));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.CHECKSUMTYPE));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.ACL));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.XATTR));
+    assertTrue(options.shouldPreserve(FileAttribute.BLOCKSIZE));
+    assertTrue(options.shouldPreserve(FileAttribute.REPLICATION));
+    assertTrue(options.shouldPreserve(FileAttribute.PERMISSION));
+    assertTrue(options.shouldPreserve(FileAttribute.USER));
+    assertTrue(options.shouldPreserve(FileAttribute.GROUP));
+    assertFalse(options.shouldPreserve(FileAttribute.CHECKSUMTYPE));
+    assertFalse(options.shouldPreserve(FileAttribute.ACL));
+    assertFalse(options.shouldPreserve(FileAttribute.XATTR));
 
     options = OptionsParser.parse(new String[] {
         "-pbrgupcaxt",
         "-f",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.BLOCKSIZE));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.REPLICATION));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.PERMISSION));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.USER));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.GROUP));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.CHECKSUMTYPE));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.ACL));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.XATTR));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.TIMES));
+    assertTrue(options.shouldPreserve(FileAttribute.BLOCKSIZE));
+    assertTrue(options.shouldPreserve(FileAttribute.REPLICATION));
+    assertTrue(options.shouldPreserve(FileAttribute.PERMISSION));
+    assertTrue(options.shouldPreserve(FileAttribute.USER));
+    assertTrue(options.shouldPreserve(FileAttribute.GROUP));
+    assertTrue(options.shouldPreserve(FileAttribute.CHECKSUMTYPE));
+    assertTrue(options.shouldPreserve(FileAttribute.ACL));
+    assertTrue(options.shouldPreserve(FileAttribute.XATTR));
+    assertTrue(options.shouldPreserve(FileAttribute.TIMES));
 
     options = OptionsParser.parse(new String[] {
         "-pc",
         "-f",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.BLOCKSIZE));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.REPLICATION));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.PERMISSION));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.USER));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.GROUP));
-    Assert.assertTrue(options.shouldPreserve(FileAttribute.CHECKSUMTYPE));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.ACL));
-    Assert.assertFalse(options.shouldPreserve(FileAttribute.XATTR));
+    assertFalse(options.shouldPreserve(FileAttribute.BLOCKSIZE));
+    assertFalse(options.shouldPreserve(FileAttribute.REPLICATION));
+    assertFalse(options.shouldPreserve(FileAttribute.PERMISSION));
+    assertFalse(options.shouldPreserve(FileAttribute.USER));
+    assertFalse(options.shouldPreserve(FileAttribute.GROUP));
+    assertTrue(options.shouldPreserve(FileAttribute.CHECKSUMTYPE));
+    assertFalse(options.shouldPreserve(FileAttribute.ACL));
+    assertFalse(options.shouldPreserve(FileAttribute.XATTR));
 
     options = OptionsParser.parse(new String[] {
         "-p",
         "-f",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertEquals(DistCpOptionSwitch.PRESERVE_STATUS_DEFAULT.length() - 2,
+    assertEquals(DistCpOptionSwitch.PRESERVE_STATUS_DEFAULT.length() - 2,
         options.getPreserveAttributes().size());
 
     try {
@@ -520,36 +528,36 @@ public class TestOptionsParser {
           "-f",
           "hdfs://localhost:8020/source/first",
           "hdfs://localhost:8020/target"});
-      Assert.fail("Invalid preserve attribute");
+      fail("Invalid preserve attribute");
     }
     catch (NoSuchElementException ignore) {}
 
     Builder builder = new DistCpOptions.Builder(
         new Path("hdfs://localhost:8020/source/first"),
         new Path("hdfs://localhost:8020/target/"));
-    Assert.assertFalse(
+    assertFalse(
         builder.build().shouldPreserve(FileAttribute.PERMISSION));
     builder.preserve(FileAttribute.PERMISSION);
-    Assert.assertTrue(builder.build().shouldPreserve(FileAttribute.PERMISSION));
+    assertTrue(builder.build().shouldPreserve(FileAttribute.PERMISSION));
 
     builder.preserve(FileAttribute.PERMISSION);
-    Assert.assertTrue(builder.build().shouldPreserve(FileAttribute.PERMISSION));
+    assertTrue(builder.build().shouldPreserve(FileAttribute.PERMISSION));
   }
 
   @Test
   public void testOptionsSwitchAddToConf() {
     Configuration conf = new Configuration();
-    Assert.assertNull(conf.get(DistCpOptionSwitch.ATOMIC_COMMIT.getConfigLabel()));
+    assertNull(conf.get(DistCpOptionSwitch.ATOMIC_COMMIT.getConfigLabel()));
     DistCpOptionSwitch.addToConf(conf, DistCpOptionSwitch.ATOMIC_COMMIT);
-    Assert.assertTrue(conf.getBoolean(DistCpOptionSwitch.ATOMIC_COMMIT.getConfigLabel(), false));
+    assertTrue(conf.getBoolean(DistCpOptionSwitch.ATOMIC_COMMIT.getConfigLabel(), false));
   }
 
   @Test
   public void testOptionsAppendToConf() {
     Configuration conf = new Configuration();
-    Assert.assertFalse(conf.getBoolean(DistCpOptionSwitch.IGNORE_FAILURES.getConfigLabel(), false));
-    Assert.assertFalse(conf.getBoolean(DistCpOptionSwitch.ATOMIC_COMMIT.getConfigLabel(), false));
-    Assert.assertEquals(
+    assertFalse(conf.getBoolean(DistCpOptionSwitch.IGNORE_FAILURES.getConfigLabel(), false));
+    assertFalse(conf.getBoolean(DistCpOptionSwitch.ATOMIC_COMMIT.getConfigLabel(), false));
+    assertEquals(
         conf.getRaw(DistCpOptionSwitch.BANDWIDTH.getConfigLabel()), null);
     DistCpOptions options = OptionsParser.parse(new String[] {
         "-atomic",
@@ -557,14 +565,14 @@ public class TestOptionsParser {
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
     options.appendToConf(conf);
-    Assert.assertTrue(conf.getBoolean(DistCpOptionSwitch.IGNORE_FAILURES.getConfigLabel(), false));
-    Assert.assertTrue(conf.getBoolean(DistCpOptionSwitch.ATOMIC_COMMIT.getConfigLabel(), false));
-    Assert.assertEquals(conf.getFloat(DistCpOptionSwitch.BANDWIDTH.getConfigLabel(), -1),
+    assertTrue(conf.getBoolean(DistCpOptionSwitch.IGNORE_FAILURES.getConfigLabel(), false));
+    assertTrue(conf.getBoolean(DistCpOptionSwitch.ATOMIC_COMMIT.getConfigLabel(), false));
+    assertEquals(conf.getFloat(DistCpOptionSwitch.BANDWIDTH.getConfigLabel(), -1),
         -1.0, DELTA);
 
     conf = new Configuration();
-    Assert.assertFalse(conf.getBoolean(DistCpOptionSwitch.SYNC_FOLDERS.getConfigLabel(), false));
-    Assert.assertFalse(conf.getBoolean(DistCpOptionSwitch.DELETE_MISSING.getConfigLabel(), false));
+    assertFalse(conf.getBoolean(DistCpOptionSwitch.SYNC_FOLDERS.getConfigLabel(), false));
+    assertFalse(conf.getBoolean(DistCpOptionSwitch.DELETE_MISSING.getConfigLabel(), false));
     assertThat(conf.get(DistCpOptionSwitch.PRESERVE_STATUS.getConfigLabel())).isNull();
     options = OptionsParser.parse(new String[] {
         "-update",
@@ -575,8 +583,8 @@ public class TestOptionsParser {
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
     options.appendToConf(conf);
-    Assert.assertTrue(conf.getBoolean(DistCpOptionSwitch.SYNC_FOLDERS.getConfigLabel(), false));
-    Assert.assertTrue(conf.getBoolean(DistCpOptionSwitch.DELETE_MISSING.getConfigLabel(), false));
+    assertTrue(conf.getBoolean(DistCpOptionSwitch.SYNC_FOLDERS.getConfigLabel(), false));
+    assertTrue(conf.getBoolean(DistCpOptionSwitch.DELETE_MISSING.getConfigLabel(), false));
     assertThat(conf.get(DistCpOptionSwitch.PRESERVE_STATUS.getConfigLabel())).isEqualTo("U");
     assertThat(conf.getFloat(DistCpOptionSwitch.BANDWIDTH.getConfigLabel(), -1))
         .isCloseTo(11.2f, within(DELTA));
@@ -585,7 +593,7 @@ public class TestOptionsParser {
   @Test
   public void testOptionsAppendToConfDoesntOverwriteBandwidth() {
     Configuration conf = new Configuration();
-    Assert.assertEquals(
+    assertEquals(
         conf.getRaw(DistCpOptionSwitch.BANDWIDTH.getConfigLabel()), null);
     DistCpOptions options = OptionsParser.parse(new String[] {
         "hdfs://localhost:8020/source/first",
@@ -595,7 +603,7 @@ public class TestOptionsParser {
             .isCloseTo(-1.0f,within(DELTA));
 
     conf = new Configuration();
-    Assert.assertEquals(
+    assertEquals(
         conf.getRaw(DistCpOptionSwitch.BANDWIDTH.getConfigLabel()), null);
     options = OptionsParser.parse(new String[] {
         "-update",
@@ -606,25 +614,25 @@ public class TestOptionsParser {
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
     options.appendToConf(conf);
-    Assert.assertEquals(
+    assertEquals(
         conf.getFloat(DistCpOptionSwitch.BANDWIDTH.getConfigLabel(), -1), 77.0,
         DELTA);
 
     conf = new Configuration();
     conf.set(DistCpOptionSwitch.BANDWIDTH.getConfigLabel(), "88");
-    Assert.assertEquals(
+    assertEquals(
         conf.getRaw(DistCpOptionSwitch.BANDWIDTH.getConfigLabel()), "88");
     options = OptionsParser.parse(new String[] {
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
     options.appendToConf(conf);
-    Assert.assertEquals(
+    assertEquals(
         conf.getFloat(DistCpOptionSwitch.BANDWIDTH.getConfigLabel(), -1), 88.0,
         DELTA);
 
     conf = new Configuration();
     conf.set(DistCpOptionSwitch.BANDWIDTH.getConfigLabel(), "88.0");
-    Assert.assertEquals(
+    assertEquals(
         conf.getRaw(DistCpOptionSwitch.BANDWIDTH.getConfigLabel()), "88.0");
     options = OptionsParser.parse(new String[] {
         "-bandwidth",
@@ -632,7 +640,7 @@ public class TestOptionsParser {
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
     options.appendToConf(conf);
-    Assert.assertEquals(
+    assertEquals(
         conf.getFloat(DistCpOptionSwitch.BANDWIDTH.getConfigLabel(), -1), 99.0,
         DELTA);
   }
@@ -640,18 +648,18 @@ public class TestOptionsParser {
   @Test
   public void testAppendOption() {
     Configuration conf = new Configuration();
-    Assert.assertFalse(conf.getBoolean(
+    assertFalse(conf.getBoolean(
         DistCpOptionSwitch.APPEND.getConfigLabel(), false));
-    Assert.assertFalse(conf.getBoolean(
+    assertFalse(conf.getBoolean(
         DistCpOptionSwitch.SYNC_FOLDERS.getConfigLabel(), false));
 
     DistCpOptions options = OptionsParser.parse(new String[] { "-update",
         "-append", "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/" });
     options.appendToConf(conf);
-    Assert.assertTrue(conf.getBoolean(
+    assertTrue(conf.getBoolean(
         DistCpOptionSwitch.APPEND.getConfigLabel(), false));
-    Assert.assertTrue(conf.getBoolean(
+    assertTrue(conf.getBoolean(
         DistCpOptionSwitch.SYNC_FOLDERS.getConfigLabel(), false));
 
     // make sure -append is only valid when -update is specified
@@ -685,29 +693,29 @@ public class TestOptionsParser {
         DistCpOptionSwitch.DIFF.getConfigLabel() :
           DistCpOptionSwitch.RDIFF.getConfigLabel();
     Configuration conf = new Configuration();
-    Assert.assertFalse(conf.getBoolean(optionLabel, false));
+    assertFalse(conf.getBoolean(optionLabel, false));
 
     DistCpOptions options = OptionsParser.parse(new String[] { "-update",
         optionStr, "s1", "s2",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/" });
     options.appendToConf(conf);
-    Assert.assertTrue(conf.getBoolean(optionLabel, false));
-    Assert.assertTrue(isDiff?
+    assertTrue(conf.getBoolean(optionLabel, false));
+    assertTrue(isDiff?
         options.shouldUseDiff() : options.shouldUseRdiff());
-    Assert.assertEquals("s1", options.getFromSnapshot());
-    Assert.assertEquals("s2", options.getToSnapshot());
+    assertEquals("s1", options.getFromSnapshot());
+    assertEquals("s2", options.getToSnapshot());
 
     options = OptionsParser.parse(new String[] {
         optionStr, "s1", ".", "-update",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/" });
     options.appendToConf(conf);
-    Assert.assertTrue(conf.getBoolean(optionLabel, false));
-    Assert.assertTrue(isDiff?
+    assertTrue(conf.getBoolean(optionLabel, false));
+    assertTrue(isDiff?
         options.shouldUseDiff() : options.shouldUseRdiff());
-    Assert.assertEquals("s1", options.getFromSnapshot());
-    Assert.assertEquals(".", options.getToSnapshot());
+    assertEquals("s1", options.getFromSnapshot());
+    assertEquals(".", options.getToSnapshot());
 
     // -diff/-rdiff requires two option values
     try {
@@ -795,7 +803,7 @@ public class TestOptionsParser {
     DistCpOptions options = OptionsParser.parse(new String[] {
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertNull(options.getFiltersFile());
+    assertNull(options.getFiltersFile());
 
     options = OptionsParser.parse(new String[] {
         "-filters",
@@ -810,12 +818,12 @@ public class TestOptionsParser {
     DistCpOptions options = OptionsParser.parse(new String[] {
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertFalse(options.shouldUpdateRoot());
+    assertFalse(options.shouldUpdateRoot());
 
     options = OptionsParser.parse(new String[] {
         "-updateRoot",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertTrue(options.shouldUpdateRoot());
+    assertTrue(options.shouldUpdateRoot());
   }
 }

+ 9 - 6
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestRegexCopyFilter.java

@@ -19,14 +19,17 @@
 package org.apache.hadoop.tools;
 
 import org.apache.hadoop.fs.Path;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.io.File;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.regex.Pattern;
 
+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 TestRegexCopyFilter {
 
   @Test
@@ -38,7 +41,7 @@ public class TestRegexCopyFilter {
     regexCopyFilter.setFilters(filters);
 
     Path shouldCopyPath = new Path("/user/bar");
-    Assert.assertTrue(regexCopyFilter.shouldCopy(shouldCopyPath));
+    assertTrue(regexCopyFilter.shouldCopy(shouldCopyPath));
   }
 
   @Test
@@ -50,7 +53,7 @@ public class TestRegexCopyFilter {
     regexCopyFilter.setFilters(filters);
 
     Path shouldNotCopyPath = new Path("/user/testing");
-    Assert.assertFalse(regexCopyFilter.shouldCopy(shouldNotCopyPath));
+    assertFalse(regexCopyFilter.shouldCopy(shouldNotCopyPath));
   }
 
   @Test
@@ -73,7 +76,7 @@ public class TestRegexCopyFilter {
       }
     }
 
-    Assert.assertEquals(2, shouldCopyCount);
+    assertEquals(2, shouldCopyCount);
   }
 
   @Test
@@ -96,7 +99,7 @@ public class TestRegexCopyFilter {
       }
     }
 
-    Assert.assertEquals(0, shouldCopyCount);
+    assertEquals(0, shouldCopyCount);
   }
 
   private List<Path> getTestPaths() {

+ 11 - 11
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestRegexpInConfigurationFilter.java

@@ -18,13 +18,13 @@
 
 package org.apache.hadoop.tools;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 
-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;
 
 /**
  * Test {@link RegexpInConfigurationFilter}.
@@ -40,16 +40,16 @@ public class TestRegexpInConfigurationFilter {
     RegexpInConfigurationFilter defaultCopyFilter =
             new RegexpInConfigurationFilter(configuration);
     Path shouldCopyPath = new Path("/user/bar");
-    assertTrue(shouldCopyPath.toString() + " should be copied",
-            defaultCopyFilter.shouldCopy(shouldCopyPath));
+    assertTrue(defaultCopyFilter.shouldCopy(shouldCopyPath),
+        shouldCopyPath.toString() + " should be copied");
     shouldCopyPath = new Path("/user/bar/_COPYING");
-    assertFalse(shouldCopyPath.toString() + " shouldn't be copied",
-            defaultCopyFilter.shouldCopy(shouldCopyPath));
+    assertFalse(defaultCopyFilter.shouldCopy(shouldCopyPath),
+        shouldCopyPath.toString() + " shouldn't be copied");
     shouldCopyPath = new Path("/user/bar/_COPYING_");
-    assertFalse(shouldCopyPath.toString() + " shouldn't be copied",
-            defaultCopyFilter.shouldCopy(shouldCopyPath));
+    assertFalse(defaultCopyFilter.shouldCopy(shouldCopyPath),
+        shouldCopyPath.toString() + " shouldn't be copied");
     shouldCopyPath = new Path("/temp/");
-    assertTrue(shouldCopyPath.toString() + " should be copied",
-            defaultCopyFilter.shouldCopy(shouldCopyPath));
+    assertTrue(defaultCopyFilter.shouldCopy(shouldCopyPath),
+        shouldCopyPath.toString() + " should be copied");
   }
 }

+ 5 - 4
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestTrueCopyFilter.java

@@ -19,18 +19,19 @@
 package org.apache.hadoop.tools;
 
 import org.apache.hadoop.fs.Path;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class TestTrueCopyFilter {
 
   @Test
   public void testShouldCopy() {
-    Assert.assertTrue(new TrueCopyFilter().shouldCopy(new Path("fake")));
+    assertTrue(new TrueCopyFilter().shouldCopy(new Path("fake")));
   }
 
   @Test
   public void testShouldCopyWithNull() {
-    Assert.assertTrue(new TrueCopyFilter().shouldCopy(new Path("fake")));
+    assertTrue(new TrueCopyFilter().shouldCopy(new Path("fake")));
   }
 }

+ 107 - 90
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java

@@ -60,14 +60,18 @@ import org.apache.hadoop.tools.util.DistCpUtils;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.StringUtils;
 
-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.junit.jupiter.api.Timeout;
 
 import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
 import static org.apache.hadoop.test.MetricsAsserts.getLongCounter;
 import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
 import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 public class TestCopyMapper {
   private static final Logger LOG = LoggerFactory.getLogger(TestCopyMapper.class);
@@ -81,7 +85,7 @@ public class TestCopyMapper {
   private static final String SOURCE_PATH = "/tmp/source";
   private static final String TARGET_PATH = "/tmp/target";
 
-  @BeforeClass
+  @BeforeAll
   public static void setup() throws Exception {
     Configuration configuration = getConfigurationForCluster();
     setCluster(new MiniDFSCluster.Builder(configuration)
@@ -285,7 +289,8 @@ public class TestCopyMapper {
     testCopy(true);
   }
 
-  @Test(timeout=40000)
+  @Test
+  @Timeout(value = 40)
   public void testRun() throws Exception {
     testCopy(false);
   }
@@ -329,10 +334,10 @@ public class TestCopyMapper {
 
     verifyCopy(fs, false, true);
     // verify that we only copied new appended data
-    Assert.assertEquals(nFiles * DEFAULT_FILE_SIZE * 2, stubContext
+    assertEquals(nFiles * DEFAULT_FILE_SIZE * 2, stubContext
         .getReporter().getCounter(CopyMapper.Counter.BYTESCOPIED)
         .getValue());
-    Assert.assertEquals(numFiles, stubContext.getReporter().
+    assertEquals(numFiles, stubContext.getReporter().
         getCounter(CopyMapper.Counter.COPY).getValue());
     rb = getMetrics(cluster.getDataNodes().get(0).getMetrics().name());
     /*
@@ -391,24 +396,24 @@ public class TestCopyMapper {
 
     // Check that the maps worked.
     verifyCopy(fs, preserveChecksum, true);
-    Assert.assertEquals(numFiles, stubContext.getReporter()
+    assertEquals(numFiles, stubContext.getReporter()
         .getCounter(CopyMapper.Counter.COPY).getValue());
-    Assert.assertEquals(numDirs, stubContext.getReporter()
+    assertEquals(numDirs, stubContext.getReporter()
         .getCounter(CopyMapper.Counter.DIR_COPY).getValue());
     if (!preserveChecksum) {
-      Assert.assertEquals(nFiles * DEFAULT_FILE_SIZE, stubContext
+      assertEquals(nFiles * DEFAULT_FILE_SIZE, stubContext
           .getReporter().getCounter(CopyMapper.Counter.BYTESCOPIED)
           .getValue());
     } else {
-      Assert.assertEquals(nFiles * NON_DEFAULT_BLOCK_SIZE * 2, stubContext
+      assertEquals(nFiles * NON_DEFAULT_BLOCK_SIZE * 2, stubContext
           .getReporter().getCounter(CopyMapper.Counter.BYTESCOPIED)
           .getValue());
     }
 
     testCopyingExistingFiles(fs, copyMapper, context);
     for (Text value : stubContext.getWriter().values()) {
-      Assert.assertTrue(value.toString() + " is not skipped", value
-          .toString().startsWith("SKIP:"));
+      assertTrue(value.toString().startsWith("SKIP:"),
+          value.toString() + " is not skipped");
     }
   }
 
@@ -418,19 +423,19 @@ public class TestCopyMapper {
     for (Path path : pathList) {
       final Path targetPath = new Path(path.toString().replaceAll(SOURCE_PATH,
           TARGET_PATH));
-      Assert.assertTrue(fs.exists(targetPath));
-      Assert.assertTrue(fs.isFile(targetPath) == fs.isFile(path));
+      assertTrue(fs.exists(targetPath));
+      assertTrue(fs.isFile(targetPath) == fs.isFile(path));
       FileStatus sourceStatus = fs.getFileStatus(path);
       FileStatus targetStatus = fs.getFileStatus(targetPath);
       if (preserveReplication) {
-        Assert.assertEquals(sourceStatus.getReplication(),
+        assertEquals(sourceStatus.getReplication(),
             targetStatus.getReplication());
       }
       if (preserveChecksum) {
-        Assert.assertEquals(sourceStatus.getBlockSize(),
+        assertEquals(sourceStatus.getBlockSize(),
             targetStatus.getBlockSize());
       }
-      Assert.assertTrue(!fs.isFile(targetPath)
+      assertTrue(!fs.isFile(targetPath)
           || fs.getFileChecksum(targetPath).equals(fs.getFileChecksum(path)));
     }
   }
@@ -443,16 +448,16 @@ public class TestCopyMapper {
                 new CopyListingFileStatus(fs.getFileStatus(path)), context);
       }
 
-      Assert.assertEquals(nFiles,
+      assertEquals(nFiles,
               context.getCounter(CopyMapper.Counter.SKIP).getValue());
     }
     catch (Exception exception) {
-      Assert.assertTrue("Caught unexpected exception:" + exception.getMessage(),
-              false);
+      assertTrue(false, "Caught unexpected exception:" + exception.getMessage());
     }
   }
 
-  @Test(timeout = 40000)
+  @Test
+  @Timeout(value = 40)
   public void testCopyWhileAppend() throws Exception {
     deleteState();
     mkdirs(SOURCE_PATH + "/1");
@@ -479,7 +484,7 @@ public class TestCopyMapper {
           }
         } catch (IOException | InterruptedException e) {
             LOG.error("Exception encountered ", e);
-            Assert.fail("Test failed: " + e.getMessage());
+            fail("Test failed: " + e.getMessage());
         }
       }
     };
@@ -494,14 +499,15 @@ public class TestCopyMapper {
       String exceptionAsString = StringUtils.stringifyException(ex);
       if (exceptionAsString.contains(DistCpConstants.LENGTH_MISMATCH_ERROR_MSG) ||
               exceptionAsString.contains(DistCpConstants.CHECKSUM_MISMATCH_ERROR_MSG)) {
-        Assert.fail("Test failed: " + exceptionAsString);
+        fail("Test failed: " + exceptionAsString);
       }
     } finally {
       scheduledExecutorService.shutdown();
     }
   }
 
-  @Test(timeout=40000)
+  @Test
+  @Timeout(value = 40)
   public void testMakeDirFailure() {
     try {
       deleteState();
@@ -524,13 +530,14 @@ public class TestCopyMapper {
           pathList.get(0))),
           new CopyListingFileStatus(fs.getFileStatus(pathList.get(0))), context);
 
-      Assert.assertTrue("There should have been an exception.", false);
+      assertTrue(false, "There should have been an exception.");
     }
     catch (Exception ignore) {
     }
   }
 
-  @Test(timeout=40000)
+  @Test
+  @Timeout(value = 40)
   public void testIgnoreFailures() {
     doTestIgnoreFailures(true);
     doTestIgnoreFailures(false);
@@ -538,7 +545,8 @@ public class TestCopyMapper {
     doTestIgnoreFailuresDoubleWrapped(false);
   }
 
-  @Test(timeout=40000)
+  @Test
+  @Timeout(value = 40)
   public void testDirToFile() {
     try {
       deleteState();
@@ -559,15 +567,16 @@ public class TestCopyMapper {
               new Path(SOURCE_PATH + "/src/file"))),
             context);
       } catch (IOException e) {
-        Assert.assertTrue(e.getMessage().startsWith("Can't replace"));
+        assertTrue(e.getMessage().startsWith("Can't replace"));
       }
     } catch (Exception e) {
       LOG.error("Exception encountered ", e);
-      Assert.fail("Test failed: " + e.getMessage());
+      fail("Test failed: " + e.getMessage());
     }
   }
 
-  @Test(timeout=40000)
+  @Test
+  @Timeout(value = 40)
   public void testPreserve() {
     try {
       deleteState();
@@ -611,7 +620,7 @@ public class TestCopyMapper {
             return FileSystem.get(cluster.getConfiguration(0));
           } catch (IOException e) {
             LOG.error("Exception encountered ", e);
-            Assert.fail("Test failed: " + e.getMessage());
+            fail("Test failed: " + e.getMessage());
             throw new RuntimeException("Test ought to fail here");
           }
         }
@@ -626,9 +635,9 @@ public class TestCopyMapper {
                 new CopyListingFileStatus(tmpFS.getFileStatus(
                   new Path(SOURCE_PATH + "/src/file"))),
                 context);
-            Assert.fail("Expected copy to fail");
+            fail("Expected copy to fail");
           } catch (AccessControlException e) {
-            Assert.assertTrue("Got exception: " + e.getMessage(), true);
+            assertTrue(true, "Got exception: " + e.getMessage());
           } catch (Exception e) {
             throw new RuntimeException(e);
           }
@@ -637,11 +646,12 @@ public class TestCopyMapper {
       });
     } catch (Exception e) {
       LOG.error("Exception encountered ", e);
-      Assert.fail("Test failed: " + e.getMessage());
+      fail("Test failed: " + e.getMessage());
     }
   }
 
-  @Test(timeout=40000)
+  @Test
+  @Timeout(value = 40)
   public void testCopyReadableFiles() {
     try {
       deleteState();
@@ -680,7 +690,7 @@ public class TestCopyMapper {
             return FileSystem.get(cluster.getConfiguration(0));
           } catch (IOException e) {
             LOG.error("Exception encountered ", e);
-            Assert.fail("Test failed: " + e.getMessage());
+            fail("Test failed: " + e.getMessage());
             throw new RuntimeException("Test ought to fail here");
           }
         }
@@ -703,11 +713,12 @@ public class TestCopyMapper {
       });
     } catch (Exception e) {
       LOG.error("Exception encountered ", e);
-      Assert.fail("Test failed: " + e.getMessage());
+      fail("Test failed: " + e.getMessage());
     }
   }
 
-  @Test(timeout=40000)
+  @Test
+  @Timeout(value = 40)
   public void testSkipCopyNoPerms() {
     try {
       deleteState();
@@ -755,7 +766,7 @@ public class TestCopyMapper {
             return FileSystem.get(cluster.getConfiguration(0));
           } catch (IOException e) {
             LOG.error("Exception encountered ", e);
-            Assert.fail("Test failed: " + e.getMessage());
+            fail("Test failed: " + e.getMessage());
             throw new RuntimeException("Test ought to fail here");
           }
         }
@@ -771,8 +782,8 @@ public class TestCopyMapper {
                   new Path(SOURCE_PATH + "/src/file"))),
                 context);
             assertThat(stubContext.getWriter().values().size()).isEqualTo(1);
-            Assert.assertTrue(stubContext.getWriter().values().get(0).toString().startsWith("SKIP"));
-            Assert.assertTrue(stubContext.getWriter().values().get(0).toString().
+            assertTrue(stubContext.getWriter().values().get(0).toString().startsWith("SKIP"));
+            assertTrue(stubContext.getWriter().values().get(0).toString().
                 contains(SOURCE_PATH + "/src/file"));
           } catch (Exception e) {
             throw new RuntimeException(e);
@@ -782,11 +793,12 @@ public class TestCopyMapper {
       });
     } catch (Exception e) {
       LOG.error("Exception encountered ", e);
-      Assert.fail("Test failed: " + e.getMessage());
+      fail("Test failed: " + e.getMessage());
     }
   }
 
-  @Test(timeout=40000)
+  @Test
+  @Timeout(value = 40)
   public void testFailCopyWithAccessControlException() {
     try {
       deleteState();
@@ -836,7 +848,7 @@ public class TestCopyMapper {
             return FileSystem.get(cluster.getConfiguration(0));
           } catch (IOException e) {
             LOG.error("Exception encountered ", e);
-            Assert.fail("Test failed: " + e.getMessage());
+            fail("Test failed: " + e.getMessage());
             throw new RuntimeException("Test ought to fail here");
           }
         }
@@ -851,7 +863,7 @@ public class TestCopyMapper {
                 new CopyListingFileStatus(tmpFS.getFileStatus(
                   new Path(SOURCE_PATH + "/src/file"))),
                 context);
-            Assert.fail("Didn't expect the file to be copied");
+            fail("Didn't expect the file to be copied");
           } catch (AccessControlException ignore) {
           } catch (Exception e) {
             // We want to make sure the underlying cause of the exception is
@@ -868,11 +880,12 @@ public class TestCopyMapper {
       });
     } catch (Exception e) {
       LOG.error("Exception encountered ", e);
-      Assert.fail("Test failed: " + e.getMessage());
+      fail("Test failed: " + e.getMessage());
     }
   }
 
-  @Test(timeout=40000)
+  @Test
+  @Timeout(value = 40)
   public void testFileToDir() {
     try {
       deleteState();
@@ -893,11 +906,11 @@ public class TestCopyMapper {
               new Path(SOURCE_PATH + "/src/file"))),
             context);
       } catch (IOException e) {
-        Assert.assertTrue(e.getMessage().startsWith("Can't replace"));
+        assertTrue(e.getMessage().startsWith("Can't replace"));
       }
     } catch (Exception e) {
       LOG.error("Exception encountered ", e);
-      Assert.fail("Test failed: " + e.getMessage());
+      fail("Test failed: " + e.getMessage());
     }
   }
 
@@ -931,15 +944,14 @@ public class TestCopyMapper {
       }
       if (ignoreFailures) {
         for (Text value : stubContext.getWriter().values()) {
-          Assert.assertTrue(value.toString() + " is not skipped",
-              value.toString().startsWith("FAIL:"));
+          assertTrue(value.toString().startsWith("FAIL:"),
+              value.toString() + " is not skipped");
         }
       }
-      Assert.assertTrue("There should have been an exception.", ignoreFailures);
+      assertTrue(ignoreFailures, "There should have been an exception.");
     }
     catch (Exception e) {
-      Assert.assertTrue("Unexpected exception: " + e.getMessage(),
-              !ignoreFailures);
+      assertTrue(!ignoreFailures, "Unexpected exception: " + e.getMessage());
       e.printStackTrace();
     }
   }
@@ -1007,12 +1019,12 @@ public class TestCopyMapper {
                 new CopyListingFileStatus(tmpFS.getFileStatus(
                     new Path(SOURCE_PATH + "/src/file"))),
                 context);
-            Assert.assertTrue("Should have thrown an IOException if not " +
-                "ignoring failures", ignoreFailures);
+            assertTrue(ignoreFailures, "Should have thrown an IOException if not " +
+                "ignoring failures");
           } catch (IOException e) {
             LOG.error("Unexpected exception encountered. ", e);
-            Assert.assertFalse("Should not have thrown an IOException if " +
-                "ignoring failures", ignoreFailures);
+            assertFalse(ignoreFailures, "Should not have thrown an IOException if " +
+                "ignoring failures");
             // the IOException is not thrown again as it's expected
           } catch (Exception e) {
             LOG.error("Exception encountered when the mapper copies file.", e);
@@ -1023,7 +1035,7 @@ public class TestCopyMapper {
       });
     } catch (Exception e) {
       LOG.error("Unexpected exception encountered. ", e);
-      Assert.fail("Test failed: " + e.getMessage());
+      fail("Test failed: " + e.getMessage());
     }
   }
 
@@ -1034,13 +1046,15 @@ public class TestCopyMapper {
     cluster.getFileSystem().delete(new Path(TARGET_PATH), true);
   }
 
-  @Test(timeout=40000)
+  @Test
+  @Timeout(value = 40)
   public void testPreserveBlockSizeAndReplication() {
     testPreserveBlockSizeAndReplicationImpl(true);
     testPreserveBlockSizeAndReplicationImpl(false);
   }
 
-  @Test(timeout=40000)
+  @Test
+  @Timeout(value = 40)
   public void testCopyWithDifferentBlockSizes() throws Exception {
     try {
       deleteState();
@@ -1072,7 +1086,7 @@ public class TestCopyMapper {
       if (expectDifferentBlockSizesMultipleBlocksToSucceed()) {
         verifyCopy(fs, false, false);
       } else {
-        Assert.fail(
+        fail(
             "Copy should have failed because of block-size difference.");
       }
     } catch (Exception exception) {
@@ -1091,7 +1105,8 @@ public class TestCopyMapper {
     }
   }
 
-  @Test(timeout=40000)
+  @Test
+  @Timeout(value = 40)
   public void testCopyWithDifferentBytesPerCrc() throws Exception {
     try {
       deleteState();
@@ -1123,7 +1138,7 @@ public class TestCopyMapper {
       if (expectDifferentBytesPerCrcToSucceed()) {
         verifyCopy(fs, false, false);
       } else {
-        Assert.fail(
+        fail(
             "Copy should have failed because of bytes-per-crc difference.");
       }
     } catch (Exception exception) {
@@ -1183,18 +1198,18 @@ public class TestCopyMapper {
           // in the FileChecksum algorithmName. If we had instead written
           // a large enough file to exceed the blocksize, then the copy
           // would not have succeeded.
-          Assert.assertTrue(preserve ||
+          assertTrue(preserve ||
                   source.getBlockSize() != target.getBlockSize());
-          Assert.assertTrue(preserve ||
+          assertTrue(preserve ||
                   source.getReplication() != target.getReplication());
-          Assert.assertTrue(!preserve ||
+          assertTrue(!preserve ||
                   source.getBlockSize() == target.getBlockSize());
-          Assert.assertTrue(!preserve ||
+          assertTrue(!preserve ||
                   source.getReplication() == target.getReplication());
         }
       }
     } catch (Exception e) {
-      Assert.assertTrue("Unexpected exception: " + e.getMessage(), false);
+      assertTrue(false, "Unexpected exception: " + e.getMessage());
       e.printStackTrace();
     }
   }
@@ -1216,7 +1231,8 @@ public class TestCopyMapper {
    * If a single file is being copied to a location where the file (of the same
    * name) already exists, then the file shouldn't be skipped.
    */
-  @Test(timeout=40000)
+  @Test
+  @Timeout(value = 40)
   public void testSingleFileCopy() {
     try {
       deleteState();
@@ -1245,7 +1261,7 @@ public class TestCopyMapper {
               new Path(SOURCE_PATH), sourceFilePath)), sourceFileStatus, context);
       long after = fs.getFileStatus(targetFilePath).getModificationTime();
 
-      Assert.assertTrue("File should have been skipped", before == after);
+      assertTrue(before == after, "File should have been skipped");
 
       context.getConfiguration().set(
               DistCpConstants.CONF_LABEL_TARGET_FINAL_PATH,
@@ -1258,15 +1274,16 @@ public class TestCopyMapper {
               new Path(SOURCE_PATH), sourceFilePath)), sourceFileStatus, context);
       after = fs.getFileStatus(targetFilePath).getModificationTime();
 
-      Assert.assertTrue("File should have been overwritten.", before < after);
+      assertTrue(before < after, "File should have been overwritten.");
 
     } catch (Exception exception) {
-      Assert.fail("Unexpected exception: " + exception.getMessage());
+      fail("Unexpected exception: " + exception.getMessage());
       exception.printStackTrace();
     }
   }
 
-  @Test(timeout=40000)
+  @Test
+  @Timeout(value = 40)
   public void testPreserveUserGroup() {
     testPreserveUserGroupImpl(true);
     testPreserveUserGroupImpl(false);
@@ -1312,19 +1329,19 @@ public class TestCopyMapper {
         final FileStatus source = fs.getFileStatus(path);
         final FileStatus target = fs.getFileStatus(targetPath);
         if (!source.isDirectory()) {
-          Assert.assertTrue(!preserve || source.getOwner().equals(target.getOwner()));
-          Assert.assertTrue(!preserve || source.getGroup().equals(target.getGroup()));
-          Assert.assertTrue(!preserve || source.getPermission().equals(target.getPermission()));
-          Assert.assertTrue( preserve || !source.getOwner().equals(target.getOwner()));
-          Assert.assertTrue( preserve || !source.getGroup().equals(target.getGroup()));
-          Assert.assertTrue( preserve || !source.getPermission().equals(target.getPermission()));
-          Assert.assertTrue(source.isDirectory() ||
-                  source.getReplication() != target.getReplication());
+          assertTrue(!preserve || source.getOwner().equals(target.getOwner()));
+          assertTrue(!preserve || source.getGroup().equals(target.getGroup()));
+          assertTrue(!preserve || source.getPermission().equals(target.getPermission()));
+          assertTrue(preserve || !source.getOwner().equals(target.getOwner()));
+          assertTrue(preserve || !source.getGroup().equals(target.getGroup()));
+          assertTrue(preserve || !source.getPermission().equals(target.getPermission()));
+          assertTrue(source.isDirectory() ||
+              source.getReplication() != target.getReplication());
         }
       }
     }
     catch (Exception e) {
-      Assert.assertTrue("Unexpected exception: " + e.getMessage(), false);
+      assertTrue(false, "Unexpected exception: " + e.getMessage());
       e.printStackTrace();
     }
   }
@@ -1353,15 +1370,15 @@ public class TestCopyMapper {
     }
 
     // Check that the maps worked.
-    Assert.assertEquals(numFiles, stubContext.getReporter()
+    assertEquals(numFiles, stubContext.getReporter()
         .getCounter(CopyMapper.Counter.COPY).getValue());
 
     testCopyingExistingFiles(fs, copyMapper, context);
     // verify the verbose log
     // we shouldn't print verbose log since this option is disabled
     for (Text value : stubContext.getWriter().values()) {
-      Assert.assertTrue(!value.toString().startsWith("FILE_COPIED:"));
-      Assert.assertTrue(!value.toString().startsWith("FILE_SKIPPED:"));
+      assertTrue(!value.toString().startsWith("FILE_COPIED:"));
+      assertTrue(!value.toString().startsWith("FILE_SKIPPED:"));
     }
 
     // test with verbose logging
@@ -1383,7 +1400,7 @@ public class TestCopyMapper {
           new CopyListingFileStatus(fs.getFileStatus(path)), context);
     }
 
-    Assert.assertEquals(numFiles, stubContext.getReporter()
+    assertEquals(numFiles, stubContext.getReporter()
         .getCounter(CopyMapper.Counter.COPY).getValue());
 
     // verify the verbose log of COPY log
@@ -1393,7 +1410,7 @@ public class TestCopyMapper {
         numFileCopied++;
       }
     }
-    Assert.assertEquals(numFiles, numFileCopied);
+    assertEquals(numFiles, numFileCopied);
 
     // verify the verbose log of SKIP log
     int numFileSkipped = 0;
@@ -1403,6 +1420,6 @@ public class TestCopyMapper {
         numFileSkipped++;
       }
     }
-    Assert.assertEquals(numFiles, numFileSkipped);
+    assertEquals(numFiles, numFileSkipped);
   }
 }

+ 2 - 2
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapperCompositeCrc.java

@@ -21,13 +21,13 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 
-import org.junit.BeforeClass;
+import org.junit.jupiter.api.BeforeAll;
 
 /**
  * End-to-end tests for COMPOSITE_CRC combine mode.
  */
 public class TestCopyMapperCompositeCrc extends TestCopyMapper {
-  @BeforeClass
+  @BeforeAll
   public static void setup() throws Exception {
     Configuration configuration = TestCopyMapper.getConfigurationForCluster();
     configuration.set(

+ 23 - 20
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyOutputFormat.java

@@ -26,8 +26,11 @@ import org.apache.hadoop.mapreduce.task.JobContextImpl;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.tools.DistCpConstants;
-import org.junit.Test;
-import 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 static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.IOException;
 
@@ -38,19 +41,19 @@ public class TestCopyOutputFormat {
   public void testSetCommitDirectory() {
     try {
       Job job = Job.getInstance(new Configuration());
-      Assert.assertEquals(null, CopyOutputFormat.getCommitDirectory(job));
+      assertEquals(null, CopyOutputFormat.getCommitDirectory(job));
 
       job.getConfiguration().set(DistCpConstants.CONF_LABEL_TARGET_FINAL_PATH, "");
-      Assert.assertEquals(null, CopyOutputFormat.getCommitDirectory(job));
+      assertEquals(null, CopyOutputFormat.getCommitDirectory(job));
 
       Path directory = new Path("/tmp/test");
       CopyOutputFormat.setCommitDirectory(job, directory);
-      Assert.assertEquals(directory, CopyOutputFormat.getCommitDirectory(job));
-      Assert.assertEquals(directory.toString(), job.getConfiguration().
+      assertEquals(directory, CopyOutputFormat.getCommitDirectory(job));
+      assertEquals(directory.toString(), job.getConfiguration().
           get(DistCpConstants.CONF_LABEL_TARGET_FINAL_PATH));
     } catch (IOException e) {
       LOG.error("Exception encountered while running test", e);
-      Assert.fail("Failed while testing for set Commit Directory");
+      fail("Failed while testing for set Commit Directory");
     }
   }
 
@@ -58,19 +61,19 @@ public class TestCopyOutputFormat {
   public void testSetWorkingDirectory() {
     try {
       Job job = Job.getInstance(new Configuration());
-      Assert.assertEquals(null, CopyOutputFormat.getWorkingDirectory(job));
+      assertEquals(null, CopyOutputFormat.getWorkingDirectory(job));
 
       job.getConfiguration().set(DistCpConstants.CONF_LABEL_TARGET_WORK_PATH, "");
-      Assert.assertEquals(null, CopyOutputFormat.getWorkingDirectory(job));
+      assertEquals(null, CopyOutputFormat.getWorkingDirectory(job));
 
       Path directory = new Path("/tmp/test");
       CopyOutputFormat.setWorkingDirectory(job, directory);
-      Assert.assertEquals(directory, CopyOutputFormat.getWorkingDirectory(job));
-      Assert.assertEquals(directory.toString(), job.getConfiguration().
+      assertEquals(directory, CopyOutputFormat.getWorkingDirectory(job));
+      assertEquals(directory.toString(), job.getConfiguration().
           get(DistCpConstants.CONF_LABEL_TARGET_WORK_PATH));
     } catch (IOException e) {
       LOG.error("Exception encountered while running test", e);
-      Assert.fail("Failed while testing for set Working Directory");
+      fail("Failed while testing for set Working Directory");
     }
   }
 
@@ -80,10 +83,10 @@ public class TestCopyOutputFormat {
       TaskAttemptContext context = new TaskAttemptContextImpl(new Configuration(),
         new TaskAttemptID("200707121733", 1, TaskType.MAP, 1, 1));
       context.getConfiguration().set("mapred.output.dir", "/out");
-      Assert.assertTrue(new CopyOutputFormat().getOutputCommitter(context) instanceof CopyCommitter);
+      assertTrue(new CopyOutputFormat().getOutputCommitter(context) instanceof CopyCommitter);
     } catch (IOException e) {
       LOG.error("Exception encountered ", e);
-      Assert.fail("Unable to get output committer");
+      fail("Unable to get output committer");
     }
   }
 
@@ -97,14 +100,14 @@ public class TestCopyOutputFormat {
       try {
         JobContext context = new JobContextImpl(job.getConfiguration(), jobID);
         outputFormat.checkOutputSpecs(context);
-        Assert.fail("No checking for invalid work/commit path");
+        fail("No checking for invalid work/commit path");
       } catch (IllegalStateException ignore) { }
 
       CopyOutputFormat.setWorkingDirectory(job, new Path("/tmp/work"));
       try {
         JobContext context = new JobContextImpl(job.getConfiguration(), jobID);
         outputFormat.checkOutputSpecs(context);
-        Assert.fail("No checking for invalid commit path");
+        fail("No checking for invalid commit path");
       } catch (IllegalStateException ignore) { }
 
       job.getConfiguration().set(DistCpConstants.CONF_LABEL_TARGET_WORK_PATH, "");
@@ -112,7 +115,7 @@ public class TestCopyOutputFormat {
       try {
         JobContext context = new JobContextImpl(job.getConfiguration(), jobID);
         outputFormat.checkOutputSpecs(context);
-        Assert.fail("No checking for invalid work path");
+        fail("No checking for invalid work path");
       } catch (IllegalStateException ignore) { }
 
       CopyOutputFormat.setWorkingDirectory(job, new Path("/tmp/work"));
@@ -121,15 +124,15 @@ public class TestCopyOutputFormat {
         JobContext context = new JobContextImpl(job.getConfiguration(), jobID);
         outputFormat.checkOutputSpecs(context);
       } catch (IllegalStateException ignore) {
-        Assert.fail("Output spec check failed.");
+        fail("Output spec check failed.");
       }
 
     } catch (IOException e) {
       LOG.error("Exception encountered while testing checkoutput specs", e);
-      Assert.fail("Checkoutput Spec failure");
+      fail("Checkoutput Spec failure");
     } catch (InterruptedException e) {
       LOG.error("Exception encountered while testing checkoutput specs", e);
-      Assert.fail("Checkoutput Spec failure");
+      fail("Checkoutput Spec failure");
     }
   }
 }

+ 23 - 20
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestDeletedDirTracker.java

@@ -23,10 +23,10 @@ import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -38,7 +38,7 @@ import org.apache.hadoop.tools.CopyListingFileStatus;
  * Unit tests of the deleted directory tracker.
  */
 @SuppressWarnings("RedundantThrows")
-public class TestDeletedDirTracker extends Assert {
+public class TestDeletedDirTracker extends Assertions {
 
   private static final Logger LOG =
       LoggerFactory.getLogger(TestDeletedDirTracker.class);
@@ -63,24 +63,28 @@ public class TestDeletedDirTracker extends Assert {
 
   private DeletedDirTracker tracker;
 
-  @Before
+  @BeforeEach
   public void setup() {
     tracker = new DeletedDirTracker(1000);
   }
 
-  @After
+  @AfterEach
   public void teardown() {
     LOG.info(tracker.toString());
   }
 
-  @Test(expected = IllegalArgumentException.class)
+  @Test
   public void testNoRootDir() throws Throwable {
-    shouldDelete(ROOT, true);
+    assertThrows(IllegalArgumentException.class, () -> {
+      shouldDelete(ROOT, true);
+    });
   }
 
-  @Test(expected = IllegalArgumentException.class)
+  @Test
   public void testNoRootFile() throws Throwable {
-    shouldDelete(dirStatus(ROOT));
+    assertThrows(IllegalArgumentException.class, () -> {
+      shouldDelete(dirStatus(ROOT));
+    });
   }
 
   @Test
@@ -202,8 +206,8 @@ public class TestDeletedDirTracker extends Assert {
   }
 
   private void expectShouldDelete(CopyListingFileStatus status) {
-    assertTrue("Expected shouldDelete of " + status.getPath(),
-        shouldDelete(status));
+    assertTrue(shouldDelete(status),
+        "Expected shouldDelete of " + status.getPath());
   }
 
   private boolean shouldDelete(final Path path, final boolean isDir) {
@@ -219,9 +223,8 @@ public class TestDeletedDirTracker extends Assert {
   }
 
   private void expectShouldNotDelete(CopyListingFileStatus status) {
-    assertFalse("Expected !shouldDelete of " + status.getPath()
-            + " but got true",
-        shouldDelete(status));
+    assertFalse(shouldDelete(status),
+        "Expected !shouldDelete of " + status.getPath() + " but got true");
   }
 
   private CopyListingFileStatus newStatus(final Path path,
@@ -238,13 +241,13 @@ public class TestDeletedDirTracker extends Assert {
   }
 
   private void expectCached(final Path path) {
-    assertTrue("Path " + path + " is not in the cache of " + tracker,
-        tracker.isContained(path));
+    assertTrue(tracker.isContained(path),
+        "Path " + path + " is not in the cache of " + tracker);
   }
 
   private void expectNotCached(final Path path) {
-    assertFalse("Path " + path + " is in the cache of " + tracker,
-        tracker.isContained(path));
+    assertFalse(tracker.isContained(path),
+        "Path " + path + " is in the cache of " + tracker);
   }
 
 }

+ 16 - 15
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestRetriableFileCopyCommand.java

@@ -25,10 +25,13 @@ import org.apache.hadoop.mapreduce.*;
 import org.apache.hadoop.tools.CopyListingFileStatus;
 import org.apache.hadoop.tools.mapred.CopyMapper.FileAction;
 
-import org.junit.Assert;
-import org.junit.Test;
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
 
 import java.io.File;
 import java.io.IOException;
@@ -57,11 +60,12 @@ public class TestRetriableFileCopyCommand {
     } catch (Exception e) {
       actualEx = e;
     }
-    assertNotNull("close didn't fail", actualEx);
+    assertNotNull(actualEx, "close didn't fail");
     assertEquals(expectedEx, actualEx);
   }
 
-  @Test(timeout = 40000)
+  @Test
+  @Timeout(value = 40)
   public void testGetNumBytesToRead() {
     long pos = 100;
     long buffLength = 1024;
@@ -69,16 +73,13 @@ public class TestRetriableFileCopyCommand {
     RetriableFileCopyCommand retriableFileCopyCommand =
             new RetriableFileCopyCommand("Testing NumBytesToRead ",
                     FileAction.OVERWRITE);
-    long numBytes = retriableFileCopyCommand
-            .getNumBytesToRead(fileLength, pos, buffLength);
-    Assert.assertEquals(1024, numBytes);
+    long numBytes = retriableFileCopyCommand.getNumBytesToRead(fileLength, pos, buffLength);
+    assertEquals(1024, numBytes);
     pos += numBytes;
-    numBytes = retriableFileCopyCommand
-            .getNumBytesToRead(fileLength, pos, buffLength);
-    Assert.assertEquals(934, numBytes);
+    numBytes = retriableFileCopyCommand.getNumBytesToRead(fileLength, pos, buffLength);
+    assertEquals(934, numBytes);
     pos += numBytes;
-    numBytes = retriableFileCopyCommand
-            .getNumBytesToRead(fileLength, pos, buffLength);
-    Assert.assertEquals(0, numBytes);
+    numBytes = retriableFileCopyCommand.getNumBytesToRead(fileLength, pos, buffLength);
+    assertEquals(0, numBytes);
   }
 }

+ 12 - 10
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestUniformSizeInputFormat.java

@@ -35,10 +35,9 @@ import org.apache.hadoop.tools.DistCpContext;
 import org.apache.hadoop.tools.DistCpOptions;
 import org.apache.hadoop.tools.StubContext;
 import org.apache.hadoop.security.Credentials;
-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 java.io.DataOutputStream;
 import java.io.IOException;
@@ -46,6 +45,9 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Random;
 
+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 TestUniformSizeInputFormat {
   private static MiniDFSCluster cluster;
@@ -57,7 +59,7 @@ public class TestUniformSizeInputFormat {
   private static final Credentials CREDENTIALS = new Credentials();
 
 
-  @BeforeClass
+  @BeforeAll
   public static void setup() throws Exception {
     cluster = new MiniDFSCluster.Builder(new Configuration()).numDataNodes(1)
                                           .format(true).build();
@@ -95,7 +97,7 @@ public class TestUniformSizeInputFormat {
     }
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() {
     cluster.shutdown();
   }
@@ -140,7 +142,7 @@ public class TestUniformSizeInputFormat {
         }
         currentSplitSize += fileStatus[0].getLen();
       }
-      Assert.assertTrue(
+      assertTrue(
            previousSplitSize == -1
                || Math.abs(currentSplitSize - previousSplitSize) < 0.1*sizePerMap
                || i == splits.size()-1);
@@ -148,7 +150,7 @@ public class TestUniformSizeInputFormat {
       doubleCheckedTotalSize += currentSplitSize;
     }
 
-    Assert.assertEquals(totalFileSize, doubleCheckedTotalSize);
+    assertEquals(totalFileSize, doubleCheckedTotalSize);
   }
 
   private void checkSplits(Path listFile, List<InputSplit> splits) throws IOException {
@@ -159,7 +161,7 @@ public class TestUniformSizeInputFormat {
     for (InputSplit split : splits) {
       FileSplit fileSplit = (FileSplit) split;
       long start = fileSplit.getStart();
-      Assert.assertEquals(lastEnd, start);
+      assertEquals(lastEnd, start);
       lastEnd = start + fileSplit.getLength();
     }
 
@@ -172,7 +174,7 @@ public class TestUniformSizeInputFormat {
       reader.seek(lastEnd);
       CopyListingFileStatus srcFileStatus = new CopyListingFileStatus();
       Text srcRelPath = new Text();
-      Assert.assertFalse(reader.next(srcRelPath, srcFileStatus));
+      assertFalse(reader.next(srcRelPath, srcFileStatus));
     } finally {
       IOUtils.closeStream(reader);
     }

+ 32 - 27
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/lib/TestDynamicInputFormat.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.tools.mapred.lib;
 
 import org.apache.hadoop.tools.DistCpConstants;
 import org.apache.hadoop.tools.DistCpContext;
-import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -36,15 +35,18 @@ import org.apache.hadoop.tools.CopyListingFileStatus;
 import org.apache.hadoop.tools.DistCpOptions;
 import org.apache.hadoop.tools.StubContext;
 import org.apache.hadoop.security.Credentials;
-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 java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
 public class TestDynamicInputFormat {
   private static final Logger LOG = LoggerFactory.getLogger(TestDynamicInputFormat.class);
   private static MiniDFSCluster cluster;
@@ -55,7 +57,7 @@ public class TestDynamicInputFormat {
 
   private static List<String> expectedFilePaths = new ArrayList<String>(N_FILES);
 
-  @BeforeClass
+  @BeforeAll
   public static void setup() throws Exception {
     cluster = new MiniDFSCluster.Builder(getConfigurationForCluster())
                   .numDataNodes(1).format(true).build();
@@ -104,7 +106,7 @@ public class TestDynamicInputFormat {
     }
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() {
     cluster.shutdown();
   }
@@ -142,28 +144,28 @@ public class TestDynamicInputFormat {
         CopyListingFileStatus fileStatus = recordReader.getCurrentValue();
         String source = fileStatus.getPath().toString();
         System.out.println(source);
-        Assert.assertTrue(expectedFilePaths.contains(source));
+        assertTrue(expectedFilePaths.contains(source));
         final float progress = recordReader.getProgress();
-        Assert.assertTrue(progress >= previousProgressValue);
-        Assert.assertTrue(progress >= 0.0f);
-        Assert.assertTrue(progress <= 1.0f);
+        assertTrue(progress >= previousProgressValue);
+        assertTrue(progress >= 0.0f);
+        assertTrue(progress <= 1.0f);
         previousProgressValue = progress;
         ++nFiles;
       }
-      Assert.assertTrue(recordReader.getProgress() == 1.0f);
+      assertTrue(recordReader.getProgress() == 1.0f);
 
       ++taskId;
     }
 
-    Assert.assertEquals(expectedFilePaths.size(), nFiles);
+    assertEquals(expectedFilePaths.size(), nFiles);
   }
 
   @Test
   public void testGetSplitRatio() throws Exception {
-    Assert.assertEquals(1, DynamicInputFormat.getSplitRatio(1, 1000000000));
-    Assert.assertEquals(2, DynamicInputFormat.getSplitRatio(11000000, 10));
-    Assert.assertEquals(4, DynamicInputFormat.getSplitRatio(30, 700));
-    Assert.assertEquals(2, DynamicInputFormat.getSplitRatio(30, 200));
+    assertEquals(1, DynamicInputFormat.getSplitRatio(1, 1000000000));
+    assertEquals(2, DynamicInputFormat.getSplitRatio(11000000, 10));
+    assertEquals(4, DynamicInputFormat.getSplitRatio(30, 700));
+    assertEquals(2, DynamicInputFormat.getSplitRatio(30, 200));
 
     // Tests with negative value configuration
     Configuration conf = new Configuration();
@@ -171,19 +173,19 @@ public class TestDynamicInputFormat {
     conf.setInt(DistCpConstants.CONF_LABEL_MAX_CHUNKS_IDEAL, -1);
     conf.setInt(DistCpConstants.CONF_LABEL_MIN_RECORDS_PER_CHUNK, -1);
     conf.setInt(DistCpConstants.CONF_LABEL_SPLIT_RATIO, -1);
-    Assert.assertEquals(1,
+    assertEquals(1,
         DynamicInputFormat.getSplitRatio(1, 1000000000, conf));
-    Assert.assertEquals(2,
+    assertEquals(2,
         DynamicInputFormat.getSplitRatio(11000000, 10, conf));
-    Assert.assertEquals(4, DynamicInputFormat.getSplitRatio(30, 700, conf));
-    Assert.assertEquals(2, DynamicInputFormat.getSplitRatio(30, 200, conf));
+    assertEquals(4, DynamicInputFormat.getSplitRatio(30, 700, conf));
+    assertEquals(2, DynamicInputFormat.getSplitRatio(30, 200, conf));
 
     // Tests with valid configuration
     conf.setInt(DistCpConstants.CONF_LABEL_MAX_CHUNKS_TOLERABLE, 100);
     conf.setInt(DistCpConstants.CONF_LABEL_MAX_CHUNKS_IDEAL, 30);
     conf.setInt(DistCpConstants.CONF_LABEL_MIN_RECORDS_PER_CHUNK, 10);
     conf.setInt(DistCpConstants.CONF_LABEL_SPLIT_RATIO, 53);
-    Assert.assertEquals(53, DynamicInputFormat.getSplitRatio(3, 200, conf));
+    assertEquals(53, DynamicInputFormat.getSplitRatio(3, 200, conf));
   }
 
   @Test
@@ -201,11 +203,14 @@ public class TestDynamicInputFormat {
         secondInputFormat.getChunkContext(configuration);
     DynamicInputChunkContext fourthContext =
         secondInputFormat.getChunkContext(configuration);
-    Assert.assertTrue("Chunk contexts from the same DynamicInputFormat " +
-        "object should be the same.",firstContext.equals(secondContext));
-    Assert.assertTrue("Chunk contexts from the same DynamicInputFormat " +
-        "object should be the same.",thirdContext.equals(fourthContext));
-    Assert.assertTrue("Contexts from different DynamicInputFormat " +
-        "objects should be different.",!firstContext.equals(thirdContext));
+    assertTrue(firstContext.equals(secondContext),
+        "Chunk contexts from the same DynamicInputFormat " +
+        "object should be the same.");
+    assertTrue(thirdContext.equals(fourthContext),
+        "Chunk contexts from the same DynamicInputFormat " +
+        "object should be the same.");
+    assertTrue(!firstContext.equals(thirdContext),
+        "Contexts from different DynamicInputFormat " +
+        "objects should be different.");
   }
 }

+ 6 - 7
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/util/DistCpTestUtils.java

@@ -30,11 +30,10 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.stream.Collectors;
 
-import org.assertj.core.api.Assertions;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 /**
  * Utility class for DistCpTests
@@ -54,7 +53,7 @@ public class DistCpTestUtils {
       Map<String, byte[]> expectedXAttrs)
       throws Exception {
     Map<String, byte[]> xAttrs = fs.getXAttrs(path);
-    assertEquals(path.toString(), expectedXAttrs.size(), xAttrs.size());
+    assertEquals(expectedXAttrs.size(), xAttrs.size(), path.toString());
     Iterator<Entry<String, byte[]>> i = expectedXAttrs.entrySet().iterator();
     while (i.hasNext()) {
       Entry<String, byte[]> e = i.next();
@@ -95,7 +94,7 @@ public class DistCpTestUtils {
     optsArr[optsArr.length - 2] = src;
     optsArr[optsArr.length - 1] = dst;
 
-    Assertions.assertThat(ToolRunner.run(conf, distCp, optsArr))
+    assertThat(ToolRunner.run(conf, distCp, optsArr))
         .describedAs("Exit code of distcp %s",
             Arrays.stream(optsArr).collect(Collectors.joining(" ")))
         .isEqualTo(exitCode);

+ 233 - 239
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/util/TestDistCpUtils.java

@@ -38,10 +38,10 @@ import org.apache.hadoop.tools.DistCpConstants;
 import org.apache.hadoop.tools.DistCpOptionSwitch;
 import org.apache.hadoop.tools.DistCpOptions.FileAttribute;
 import org.apache.hadoop.util.ToolRunner;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import org.apache.hadoop.util.Lists;
 
@@ -65,11 +65,12 @@ import static org.apache.hadoop.fs.permission.FsAction.READ_EXECUTE;
 import static org.apache.hadoop.fs.permission.FsAction.READ_WRITE;
 import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.aclEntry;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertTrue;
 import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class TestDistCpUtils {
   private static final Logger LOG = LoggerFactory.getLogger(TestDistCpUtils.class);
@@ -80,7 +81,7 @@ public class TestDistCpUtils {
   private static final FsPermission almostFullPerm = new FsPermission((short) 666);
   private static final FsPermission noPerm = new FsPermission((short) 0);
   
-  @BeforeClass
+  @BeforeAll
   public static void create() throws IOException {
     config.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
     cluster = new MiniDFSCluster.Builder(config)
@@ -90,7 +91,7 @@ public class TestDistCpUtils {
     cluster.getFileSystem().enableErasureCodingPolicy("XOR-2-1-1024k");
   }
 
-  @AfterClass
+  @AfterAll
   public static void destroy() {
     if (cluster != null) {
       cluster.shutdown();
@@ -139,29 +140,29 @@ public class TestDistCpUtils {
   @Test
   public void testUnpackAttributes() {
     EnumSet<FileAttribute> attributes = EnumSet.allOf(FileAttribute.class);
-    Assert.assertEquals(attributes, DistCpUtils.unpackAttributes("RCBUGPAXTE"));
+    assertEquals(attributes, DistCpUtils.unpackAttributes("RCBUGPAXTE"));
 
     attributes.remove(FileAttribute.REPLICATION);
     attributes.remove(FileAttribute.CHECKSUMTYPE);
     attributes.remove(FileAttribute.ACL);
     attributes.remove(FileAttribute.XATTR);
     attributes.remove(FileAttribute.ERASURECODINGPOLICY);
-    Assert.assertEquals(attributes, DistCpUtils.unpackAttributes("BUGPT"));
+    assertEquals(attributes, DistCpUtils.unpackAttributes("BUGPT"));
 
     attributes.remove(FileAttribute.TIMES);
-    Assert.assertEquals(attributes, DistCpUtils.unpackAttributes("BUGP"));
+    assertEquals(attributes, DistCpUtils.unpackAttributes("BUGP"));
 
     attributes.remove(FileAttribute.BLOCKSIZE);
-    Assert.assertEquals(attributes, DistCpUtils.unpackAttributes("UGP"));
+    assertEquals(attributes, DistCpUtils.unpackAttributes("UGP"));
 
     attributes.remove(FileAttribute.GROUP);
-    Assert.assertEquals(attributes, DistCpUtils.unpackAttributes("UP"));
+    assertEquals(attributes, DistCpUtils.unpackAttributes("UP"));
 
     attributes.remove(FileAttribute.USER);
-    Assert.assertEquals(attributes, DistCpUtils.unpackAttributes("P"));
+    assertEquals(attributes, DistCpUtils.unpackAttributes("P"));
 
     attributes.remove(FileAttribute.PERMISSION);
-    Assert.assertEquals(attributes, DistCpUtils.unpackAttributes(""));
+    assertEquals(attributes, DistCpUtils.unpackAttributes(""));
   }
 
   @Test
@@ -208,18 +209,17 @@ public class TestDistCpUtils {
         destStatus);
 
     // FileStatus.equals only compares path field, must explicitly compare all fields
-    assertEquals(text + "permission",
-        srcStatus.getPermission(), dstStatus.getPermission());
-    assertEquals(text + "owner",
-        srcStatus.getOwner(), dstStatus.getOwner());
-    assertEquals(text + "group",
-        srcStatus.getGroup(), dstStatus.getGroup());
-    assertEquals(text + "accessTime",
-        srcStatus.getAccessTime(), dstStatus.getAccessTime());
-    assertEquals(text + "modificationTime",
-        srcStatus.getModificationTime(), dstStatus.getModificationTime());
-    assertEquals(text + "replication",
-        srcStatus.getReplication(), dstStatus.getReplication());
+    assertEquals(srcStatus.getPermission(), dstStatus.getPermission(),
+        text + "permission");
+    assertEquals(srcStatus.getOwner(), dstStatus.getOwner(),
+        text + "owner");
+    assertEquals(srcStatus.getGroup(), dstStatus.getGroup(), text + "group");
+    assertEquals(srcStatus.getAccessTime(),
+        dstStatus.getAccessTime(), text + "accessTime");
+    assertEquals(srcStatus.getModificationTime(),
+        dstStatus.getModificationTime(), text + "modificationTime");
+    assertEquals(srcStatus.getReplication(),
+        dstStatus.getReplication(), text + "replication");
   }
 
   private void assertStatusNotEqual(final FileSystem fs,
@@ -233,18 +233,15 @@ public class TestDistCpUtils {
         srcStatus, destStatus);
     // FileStatus.equals only compares path field,
     // must explicitly compare all fields
-    assertNotEquals(text + "permission",
-        srcStatus.getPermission(), dstStatus.getPermission());
-    assertNotEquals(text + "owner",
-        srcStatus.getOwner(), dstStatus.getOwner());
-    assertNotEquals(text + "group",
-        srcStatus.getGroup(), dstStatus.getGroup());
-    assertNotEquals(text + "accessTime",
-        srcStatus.getAccessTime(), dstStatus.getAccessTime());
-    assertNotEquals(text + "modificationTime",
-        srcStatus.getModificationTime(), dstStatus.getModificationTime());
-    assertNotEquals(text + "replication",
-        srcStatus.getReplication(), dstStatus.getReplication());
+    assertNotEquals(srcStatus.getPermission(), dstStatus.getPermission(),
+        text + "permission");
+    assertNotEquals(srcStatus.getOwner(), dstStatus.getOwner(), text + "owner");
+    assertNotEquals(srcStatus.getGroup(), dstStatus.getGroup(), text + "group");
+    assertNotEquals(srcStatus.getAccessTime(), dstStatus.getAccessTime(),
+        text + "accessTime");
+    assertNotEquals(srcStatus.getModificationTime(),
+        dstStatus.getModificationTime(), text + "modificationTime");
+    assertNotEquals(srcStatus.getReplication(), dstStatus.getReplication(), text + "replication");
   }
 
 
@@ -324,7 +321,7 @@ public class TestDistCpUtils {
     List<AclEntry> en1 = fs.getAclStatus(src).getEntries();
     List<AclEntry> dd2 = fs.getAclStatus(dest).getEntries();
 
-    Assert.assertNotEquals(en1, dd2);
+    assertNotEquals(en1, dd2);
 
     CopyListingFileStatus srcStatus = new CopyListingFileStatus(
         fs.getFileStatus(src));
@@ -343,7 +340,7 @@ public class TestDistCpUtils {
     // fields
     assertStatusEqual(fs, dest, srcStatus);
 
-    Assert.assertArrayEquals(en1.toArray(), dd2.toArray());
+    assertArrayEquals(en1.toArray(), dd2.toArray());
   }
 
   @Test
@@ -372,12 +369,12 @@ public class TestDistCpUtils {
     CopyListingFileStatus dstStatus = new CopyListingFileStatus(fs.getFileStatus(dst));
 
     // FileStatus.equals only compares path field, must explicitly compare all fields
-    Assert.assertFalse(srcStatus.getPermission().equals(dstStatus.getPermission()));
-    Assert.assertFalse(srcStatus.getOwner().equals(dstStatus.getOwner()));
-    Assert.assertFalse(srcStatus.getGroup().equals(dstStatus.getGroup()));
-    Assert.assertTrue(dstStatus.getAccessTime() == 100);
-    Assert.assertTrue(dstStatus.getModificationTime() == 100);
-    Assert.assertTrue(dstStatus.getReplication() == 0);
+    assertFalse(srcStatus.getPermission().equals(dstStatus.getPermission()));
+    assertFalse(srcStatus.getOwner().equals(dstStatus.getOwner()));
+    assertFalse(srcStatus.getGroup().equals(dstStatus.getGroup()));
+    assertTrue(dstStatus.getAccessTime() == 100);
+    assertTrue(dstStatus.getModificationTime() == 100);
+    assertTrue(dstStatus.getReplication() == 0);
   }
 
   @Test
@@ -404,9 +401,9 @@ public class TestDistCpUtils {
     CopyListingFileStatus dstStatus = new CopyListingFileStatus(fs.getFileStatus(dst));
 
     // FileStatus.equals only compares path field, must explicitly compare all fields
-    Assert.assertTrue(srcStatus.getPermission().equals(dstStatus.getPermission()));
-    Assert.assertFalse(srcStatus.getOwner().equals(dstStatus.getOwner()));
-    Assert.assertFalse(srcStatus.getGroup().equals(dstStatus.getGroup()));
+    assertTrue(srcStatus.getPermission().equals(dstStatus.getPermission()));
+    assertFalse(srcStatus.getOwner().equals(dstStatus.getOwner()));
+    assertFalse(srcStatus.getGroup().equals(dstStatus.getGroup()));
   }
 
   @Test
@@ -433,9 +430,9 @@ public class TestDistCpUtils {
     CopyListingFileStatus dstStatus = new CopyListingFileStatus(fs.getFileStatus(dst));
 
     // FileStatus.equals only compares path field, must explicitly compare all fields
-    Assert.assertFalse(srcStatus.getPermission().equals(dstStatus.getPermission()));
-    Assert.assertFalse(srcStatus.getOwner().equals(dstStatus.getOwner()));
-    Assert.assertTrue(srcStatus.getGroup().equals(dstStatus.getGroup()));
+    assertFalse(srcStatus.getPermission().equals(dstStatus.getPermission()));
+    assertFalse(srcStatus.getOwner().equals(dstStatus.getOwner()));
+    assertTrue(srcStatus.getGroup().equals(dstStatus.getGroup()));
   }
 
   @Test
@@ -462,9 +459,9 @@ public class TestDistCpUtils {
     CopyListingFileStatus dstStatus = new CopyListingFileStatus(fs.getFileStatus(dst));
 
     // FileStatus.equals only compares path field, must explicitly compare all fields
-    Assert.assertFalse(srcStatus.getPermission().equals(dstStatus.getPermission()));
-    Assert.assertTrue(srcStatus.getOwner().equals(dstStatus.getOwner()));
-    Assert.assertFalse(srcStatus.getGroup().equals(dstStatus.getGroup()));
+    assertFalse(srcStatus.getPermission().equals(dstStatus.getPermission()));
+    assertTrue(srcStatus.getOwner().equals(dstStatus.getOwner()));
+    assertFalse(srcStatus.getGroup().equals(dstStatus.getGroup()));
   }
 
   @Test
@@ -493,11 +490,11 @@ public class TestDistCpUtils {
     CopyListingFileStatus dstStatus = new CopyListingFileStatus(fs.getFileStatus(dst));
 
     // FileStatus.equals only compares path field, must explicitly compare all fields
-    Assert.assertFalse(srcStatus.getPermission().equals(dstStatus.getPermission()));
-    Assert.assertFalse(srcStatus.getOwner().equals(dstStatus.getOwner()));
-    Assert.assertFalse(srcStatus.getGroup().equals(dstStatus.getGroup()));
+    assertFalse(srcStatus.getPermission().equals(dstStatus.getPermission()));
+    assertFalse(srcStatus.getOwner().equals(dstStatus.getOwner()));
+    assertFalse(srcStatus.getGroup().equals(dstStatus.getGroup()));
     // Replication shouldn't apply to dirs so this should still be 0 == 0
-    Assert.assertTrue(srcStatus.getReplication() == dstStatus.getReplication());
+    assertTrue(srcStatus.getReplication() == dstStatus.getReplication());
   }
 
   @Test
@@ -526,11 +523,11 @@ public class TestDistCpUtils {
     CopyListingFileStatus dstStatus = new CopyListingFileStatus(fs.getFileStatus(dst));
 
     // FileStatus.equals only compares path field, must explicitly compare all fields
-    Assert.assertFalse(srcStatus.getPermission().equals(dstStatus.getPermission()));
-    Assert.assertFalse(srcStatus.getOwner().equals(dstStatus.getOwner()));
-    Assert.assertFalse(srcStatus.getGroup().equals(dstStatus.getGroup()));
-    Assert.assertTrue(srcStatus.getAccessTime() == dstStatus.getAccessTime());
-    Assert.assertTrue(srcStatus.getModificationTime() == dstStatus.getModificationTime());
+    assertFalse(srcStatus.getPermission().equals(dstStatus.getPermission()));
+    assertFalse(srcStatus.getOwner().equals(dstStatus.getOwner()));
+    assertFalse(srcStatus.getGroup().equals(dstStatus.getGroup()));
+    assertTrue(srcStatus.getAccessTime() == dstStatus.getAccessTime());
+    assertTrue(srcStatus.getModificationTime() == dstStatus.getModificationTime());
   }
 
   @Test
@@ -592,12 +589,12 @@ public class TestDistCpUtils {
     CopyListingFileStatus dstStatus = new CopyListingFileStatus(fs.getFileStatus(dst));
 
     // FileStatus.equals only compares path field, must explicitly compare all fields
-    Assert.assertTrue(srcStatus.getPermission().equals(dstStatus.getPermission()));
-    Assert.assertFalse(srcStatus.getOwner().equals(dstStatus.getOwner()));
-    Assert.assertFalse(srcStatus.getGroup().equals(dstStatus.getGroup()));
-    Assert.assertFalse(srcStatus.getAccessTime() == dstStatus.getAccessTime());
-    Assert.assertFalse(srcStatus.getModificationTime() == dstStatus.getModificationTime());
-    Assert.assertFalse(srcStatus.getReplication() == dstStatus.getReplication());
+    assertTrue(srcStatus.getPermission().equals(dstStatus.getPermission()));
+    assertFalse(srcStatus.getOwner().equals(dstStatus.getOwner()));
+    assertFalse(srcStatus.getGroup().equals(dstStatus.getGroup()));
+    assertFalse(srcStatus.getAccessTime() == dstStatus.getAccessTime());
+    assertFalse(srcStatus.getModificationTime() == dstStatus.getModificationTime());
+    assertFalse(srcStatus.getReplication() == dstStatus.getReplication());
   }
 
   @Test
@@ -628,12 +625,12 @@ public class TestDistCpUtils {
     CopyListingFileStatus dstStatus = new CopyListingFileStatus(fs.getFileStatus(dst));
 
     // FileStatus.equals only compares path field, must explicitly compare all fields
-    Assert.assertFalse(srcStatus.getPermission().equals(dstStatus.getPermission()));
-    Assert.assertFalse(srcStatus.getOwner().equals(dstStatus.getOwner()));
-    Assert.assertTrue(srcStatus.getGroup().equals(dstStatus.getGroup()));
-    Assert.assertFalse(srcStatus.getAccessTime() == dstStatus.getAccessTime());
-    Assert.assertFalse(srcStatus.getModificationTime() == dstStatus.getModificationTime());
-    Assert.assertFalse(srcStatus.getReplication() == dstStatus.getReplication());
+    assertFalse(srcStatus.getPermission().equals(dstStatus.getPermission()));
+    assertFalse(srcStatus.getOwner().equals(dstStatus.getOwner()));
+    assertTrue(srcStatus.getGroup().equals(dstStatus.getGroup()));
+    assertFalse(srcStatus.getAccessTime() == dstStatus.getAccessTime());
+    assertFalse(srcStatus.getModificationTime() == dstStatus.getModificationTime());
+    assertFalse(srcStatus.getReplication() == dstStatus.getReplication());
   }
 
   @Test
@@ -664,12 +661,12 @@ public class TestDistCpUtils {
     CopyListingFileStatus dstStatus = new CopyListingFileStatus(fs.getFileStatus(dst));
 
     // FileStatus.equals only compares path field, must explicitly compare all fields
-    Assert.assertFalse(srcStatus.getPermission().equals(dstStatus.getPermission()));
-    Assert.assertTrue(srcStatus.getOwner().equals(dstStatus.getOwner()));
-    Assert.assertFalse(srcStatus.getGroup().equals(dstStatus.getGroup()));
-    Assert.assertFalse(srcStatus.getAccessTime() == dstStatus.getAccessTime());
-    Assert.assertFalse(srcStatus.getModificationTime() == dstStatus.getModificationTime());
-    Assert.assertFalse(srcStatus.getReplication() == dstStatus.getReplication());
+    assertFalse(srcStatus.getPermission().equals(dstStatus.getPermission()));
+    assertTrue(srcStatus.getOwner().equals(dstStatus.getOwner()));
+    assertFalse(srcStatus.getGroup().equals(dstStatus.getGroup()));
+    assertFalse(srcStatus.getAccessTime() == dstStatus.getAccessTime());
+    assertFalse(srcStatus.getModificationTime() == dstStatus.getModificationTime());
+    assertFalse(srcStatus.getReplication() == dstStatus.getReplication());
   }
 
   @Test
@@ -700,15 +697,16 @@ public class TestDistCpUtils {
     CopyListingFileStatus dstStatus = new CopyListingFileStatus(fs.getFileStatus(dst));
 
     // FileStatus.equals only compares path field, must explicitly compare all fields
-    Assert.assertFalse(srcStatus.getPermission().equals(dstStatus.getPermission()));
-    Assert.assertFalse(srcStatus.getOwner().equals(dstStatus.getOwner()));
-    Assert.assertFalse(srcStatus.getGroup().equals(dstStatus.getGroup()));
-    Assert.assertFalse(srcStatus.getAccessTime() == dstStatus.getAccessTime());
-    Assert.assertFalse(srcStatus.getModificationTime() == dstStatus.getModificationTime());
-    Assert.assertTrue(srcStatus.getReplication() == dstStatus.getReplication());
+    assertFalse(srcStatus.getPermission().equals(dstStatus.getPermission()));
+    assertFalse(srcStatus.getOwner().equals(dstStatus.getOwner()));
+    assertFalse(srcStatus.getGroup().equals(dstStatus.getGroup()));
+    assertFalse(srcStatus.getAccessTime() == dstStatus.getAccessTime());
+    assertFalse(srcStatus.getModificationTime() == dstStatus.getModificationTime());
+    assertTrue(srcStatus.getReplication() == dstStatus.getReplication());
   }
 
-  @Test (timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testReplFactorNotPreservedOnErasureCodedFile() throws Exception {
     FileSystem fs = FileSystem.get(config);
 
@@ -723,7 +721,7 @@ public class TestDistCpUtils {
     String[] args = {"-setPolicy", "-path", "/tmp/srcECDir",
         "-policy", "XOR-2-1-1024k"};
     int res = ToolRunner.run(config, new ECAdmin(config), args);
-    assertEquals("Setting EC policy should succeed!", 0, res);
+    assertEquals(0, res, "Setting EC policy should succeed!");
     verifyReplFactorNotPreservedOnErasureCodedFile(srcECFile, true,
         dstReplFile, false);
 
@@ -738,7 +736,7 @@ public class TestDistCpUtils {
     args = new String[]{"-setPolicy", "-path", "/tmp/dstECDir",
         "-policy", "XOR-2-1-1024k"};
     res = ToolRunner.run(config, new ECAdmin(config), args);
-    assertEquals("Setting EC policy should succeed!", 0, res);
+    assertEquals(0, res, "Setting EC policy should succeed!");
     verifyReplFactorNotPreservedOnErasureCodedFile(srcReplFile,
         false, dstECFile, true);
 
@@ -756,27 +754,24 @@ public class TestDistCpUtils {
     CopyListingFileStatus srcStatus = new CopyListingFileStatus(
         fs.getFileStatus(srcFile));
     if (isSrcEC) {
-      assertTrue(srcFile + "should be erasure coded!",
-          srcStatus.isErasureCoded());
+      assertTrue(srcStatus.isErasureCoded(), srcFile + "should be erasure coded!");
       assertEquals(INodeFile.DEFAULT_REPL_FOR_STRIPED_BLOCKS,
           srcStatus.getReplication());
     } else {
-      assertEquals("Unexpected replication factor for " + srcFile,
-          fs.getDefaultReplication(srcFile), srcStatus.getReplication());
+      assertEquals(fs.getDefaultReplication(srcFile), srcStatus.getReplication(),
+          "Unexpected replication factor for " + srcFile);
     }
 
     createFile(fs, dstFile);
     CopyListingFileStatus dstStatus = new CopyListingFileStatus(
         fs.getFileStatus(dstFile));
     if (isDstEC) {
-      assertTrue(dstFile + "should be erasure coded!",
-          dstStatus.isErasureCoded());
-      assertEquals("Unexpected replication factor for erasure coded file!",
-          INodeFile.DEFAULT_REPL_FOR_STRIPED_BLOCKS,
-          dstStatus.getReplication());
+      assertTrue(dstStatus.isErasureCoded(), dstFile + "should be erasure coded!");
+      assertEquals(INodeFile.DEFAULT_REPL_FOR_STRIPED_BLOCKS,
+          dstStatus.getReplication(), "Unexpected replication factor for erasure coded file!");
     } else {
-      assertEquals("Unexpected replication factor for " + dstFile,
-          fs.getDefaultReplication(dstFile), dstStatus.getReplication());
+      assertEquals(fs.getDefaultReplication(dstFile), dstStatus.getReplication(),
+          "Unexpected replication factor for " + dstFile);
     }
 
     // Let srcFile and dstFile differ on their FileAttribute
@@ -794,28 +789,27 @@ public class TestDistCpUtils {
     DistCpUtils.preserve(fs, dstFile, srcStatus, attributes, false);
     dstStatus = new CopyListingFileStatus(fs.getFileStatus(dstFile));
 
-    assertFalse("Permission for " + srcFile + " and " + dstFile +
-            " should not be same after preserve only for replication attr!",
-        srcStatus.getPermission().equals(dstStatus.getPermission()));
-    assertFalse("File ownership should not match!",
-        srcStatus.getOwner().equals(dstStatus.getOwner()));
+    assertFalse(srcStatus.getPermission().equals(dstStatus.getPermission()),
+        "Permission for " + srcFile + " and " + dstFile +
+        " should not be same after preserve only for replication attr!");
+    assertFalse(srcStatus.getOwner().equals(dstStatus.getOwner()),
+        "File ownership should not match!");
     assertFalse(srcStatus.getGroup().equals(dstStatus.getGroup()));
     assertFalse(srcStatus.getAccessTime() == dstStatus.getAccessTime());
     assertFalse(
         srcStatus.getModificationTime() == dstStatus.getModificationTime());
     if (isDstEC) {
-      assertEquals("Unexpected replication factor for erasure coded file!",
-          INodeFile.DEFAULT_REPL_FOR_STRIPED_BLOCKS,
-          dstStatus.getReplication());
+      assertEquals(INodeFile.DEFAULT_REPL_FOR_STRIPED_BLOCKS,
+          dstStatus.getReplication(), "Unexpected replication factor for erasure coded file!");
     } else {
-      assertEquals(dstFile + " replication factor should be same as dst " +
-              "filesystem!", fs.getDefaultReplication(dstFile),
-          dstStatus.getReplication());
+      assertEquals(fs.getDefaultReplication(dstFile),
+          dstStatus.getReplication(), dstFile + " replication factor should be same as dst " +
+          "filesystem!");
     }
     if (!isSrcEC || !isDstEC) {
-      assertFalse(dstFile + " replication factor should not be " +
-              "same as " + srcFile,
-          srcStatus.getReplication() == dstStatus.getReplication());
+      assertFalse(srcStatus.getReplication() == dstStatus.getReplication(),
+          dstFile + " replication factor should not be " +
+          "same as " + srcFile);
     }
   }
 
@@ -847,12 +841,12 @@ public class TestDistCpUtils {
     CopyListingFileStatus dstStatus = new CopyListingFileStatus(fs.getFileStatus(dst));
 
     // FileStatus.equals only compares path field, must explicitly compare all fields
-    Assert.assertFalse(srcStatus.getPermission().equals(dstStatus.getPermission()));
-    Assert.assertFalse(srcStatus.getOwner().equals(dstStatus.getOwner()));
-    Assert.assertFalse(srcStatus.getGroup().equals(dstStatus.getGroup()));
-    Assert.assertTrue(srcStatus.getAccessTime() == dstStatus.getAccessTime());
-    Assert.assertTrue(srcStatus.getModificationTime() == dstStatus.getModificationTime());
-    Assert.assertFalse(srcStatus.getReplication() == dstStatus.getReplication());
+    assertFalse(srcStatus.getPermission().equals(dstStatus.getPermission()));
+    assertFalse(srcStatus.getOwner().equals(dstStatus.getOwner()));
+    assertFalse(srcStatus.getGroup().equals(dstStatus.getGroup()));
+    assertTrue(srcStatus.getAccessTime() == dstStatus.getAccessTime());
+    assertTrue(srcStatus.getModificationTime() == dstStatus.getModificationTime());
+    assertFalse(srcStatus.getReplication() == dstStatus.getReplication());
   }
 
   @Test
@@ -916,39 +910,39 @@ public class TestDistCpUtils {
 
     // attributes of src -> f1 ? should be no
     CopyListingFileStatus f1Status = new CopyListingFileStatus(fs.getFileStatus(f1));
-    Assert.assertFalse(srcStatus.getPermission().equals(f1Status.getPermission()));
-    Assert.assertFalse(srcStatus.getOwner().equals(f1Status.getOwner()));
-    Assert.assertFalse(srcStatus.getGroup().equals(f1Status.getGroup()));
-    Assert.assertFalse(srcStatus.getAccessTime() == f1Status.getAccessTime());
-    Assert.assertFalse(srcStatus.getModificationTime() == f1Status.getModificationTime());
-    Assert.assertFalse(srcStatus.getReplication() == f1Status.getReplication());
+    assertFalse(srcStatus.getPermission().equals(f1Status.getPermission()));
+    assertFalse(srcStatus.getOwner().equals(f1Status.getOwner()));
+    assertFalse(srcStatus.getGroup().equals(f1Status.getGroup()));
+    assertFalse(srcStatus.getAccessTime() == f1Status.getAccessTime());
+    assertFalse(srcStatus.getModificationTime() == f1Status.getModificationTime());
+    assertFalse(srcStatus.getReplication() == f1Status.getReplication());
 
     // attributes of src -> f0 ? should be no
     CopyListingFileStatus f0Status = new CopyListingFileStatus(fs.getFileStatus(f0));
-    Assert.assertFalse(srcStatus.getPermission().equals(f0Status.getPermission()));
-    Assert.assertFalse(srcStatus.getOwner().equals(f0Status.getOwner()));
-    Assert.assertFalse(srcStatus.getGroup().equals(f0Status.getGroup()));
-    Assert.assertFalse(srcStatus.getAccessTime() == f0Status.getAccessTime());
-    Assert.assertFalse(srcStatus.getModificationTime() == f0Status.getModificationTime());
-    Assert.assertFalse(srcStatus.getReplication() == f0Status.getReplication());
+    assertFalse(srcStatus.getPermission().equals(f0Status.getPermission()));
+    assertFalse(srcStatus.getOwner().equals(f0Status.getOwner()));
+    assertFalse(srcStatus.getGroup().equals(f0Status.getGroup()));
+    assertFalse(srcStatus.getAccessTime() == f0Status.getAccessTime());
+    assertFalse(srcStatus.getModificationTime() == f0Status.getModificationTime());
+    assertFalse(srcStatus.getReplication() == f0Status.getReplication());
 
     // attributes of src -> d2 ? should be no
     CopyListingFileStatus d2Status = new CopyListingFileStatus(fs.getFileStatus(d2));
-    Assert.assertFalse(srcStatus.getPermission().equals(d2Status.getPermission()));
-    Assert.assertFalse(srcStatus.getOwner().equals(d2Status.getOwner()));
-    Assert.assertFalse(srcStatus.getGroup().equals(d2Status.getGroup()));
-    Assert.assertTrue(d2Status.getAccessTime() == 300);
-    Assert.assertTrue(d2Status.getModificationTime() == 300);
-    Assert.assertFalse(srcStatus.getReplication() == d2Status.getReplication());
+    assertFalse(srcStatus.getPermission().equals(d2Status.getPermission()));
+    assertFalse(srcStatus.getOwner().equals(d2Status.getOwner()));
+    assertFalse(srcStatus.getGroup().equals(d2Status.getGroup()));
+    assertTrue(d2Status.getAccessTime() == 300);
+    assertTrue(d2Status.getModificationTime() == 300);
+    assertFalse(srcStatus.getReplication() == d2Status.getReplication());
 
     // attributes of src -> d1 ? should be no
     CopyListingFileStatus d1Status = new CopyListingFileStatus(fs.getFileStatus(d1));
-    Assert.assertFalse(srcStatus.getPermission().equals(d1Status.getPermission()));
-    Assert.assertFalse(srcStatus.getOwner().equals(d1Status.getOwner()));
-    Assert.assertFalse(srcStatus.getGroup().equals(d1Status.getGroup()));
-    Assert.assertTrue(d1Status.getAccessTime() == 400);
-    Assert.assertTrue(d1Status.getModificationTime() == 400);
-    Assert.assertFalse(srcStatus.getReplication() == d1Status.getReplication());
+    assertFalse(srcStatus.getPermission().equals(d1Status.getPermission()));
+    assertFalse(srcStatus.getOwner().equals(d1Status.getOwner()));
+    assertFalse(srcStatus.getGroup().equals(d1Status.getGroup()));
+    assertTrue(d1Status.getAccessTime() == 400);
+    assertTrue(d1Status.getModificationTime() == 400);
+    assertFalse(srcStatus.getReplication() == d1Status.getReplication());
   }
 
   @Test
@@ -1010,48 +1004,48 @@ public class TestDistCpUtils {
     // FileStatus.equals only compares path field, must explicitly compare all fields
     // attributes of src -> d2 ? should be yes
     CopyListingFileStatus d2Status = new CopyListingFileStatus(fs.getFileStatus(d2));
-    Assert.assertTrue(srcStatus.getPermission().equals(d2Status.getPermission()));
-    Assert.assertTrue(srcStatus.getOwner().equals(d2Status.getOwner()));
-    Assert.assertTrue(srcStatus.getGroup().equals(d2Status.getGroup()));
-    Assert.assertTrue(srcStatus.getAccessTime() == d2Status.getAccessTime());
-    Assert.assertTrue(srcStatus.getModificationTime() == d2Status.getModificationTime());
-    Assert.assertTrue(srcStatus.getReplication() != d2Status.getReplication());
+    assertTrue(srcStatus.getPermission().equals(d2Status.getPermission()));
+    assertTrue(srcStatus.getOwner().equals(d2Status.getOwner()));
+    assertTrue(srcStatus.getGroup().equals(d2Status.getGroup()));
+    assertTrue(srcStatus.getAccessTime() == d2Status.getAccessTime());
+    assertTrue(srcStatus.getModificationTime() == d2Status.getModificationTime());
+    assertTrue(srcStatus.getReplication() != d2Status.getReplication());
 
     // attributes of src -> d1 ? should be no
     CopyListingFileStatus d1Status = new CopyListingFileStatus(fs.getFileStatus(d1));
-    Assert.assertFalse(srcStatus.getPermission().equals(d1Status.getPermission()));
-    Assert.assertFalse(srcStatus.getOwner().equals(d1Status.getOwner()));
-    Assert.assertFalse(srcStatus.getGroup().equals(d1Status.getGroup()));
-    Assert.assertFalse(srcStatus.getAccessTime() == d1Status.getAccessTime());
-    Assert.assertFalse(srcStatus.getModificationTime() == d1Status.getModificationTime());
-    Assert.assertTrue(srcStatus.getReplication() != d1Status.getReplication());
+    assertFalse(srcStatus.getPermission().equals(d1Status.getPermission()));
+    assertFalse(srcStatus.getOwner().equals(d1Status.getOwner()));
+    assertFalse(srcStatus.getGroup().equals(d1Status.getGroup()));
+    assertFalse(srcStatus.getAccessTime() == d1Status.getAccessTime());
+    assertFalse(srcStatus.getModificationTime() == d1Status.getModificationTime());
+    assertTrue(srcStatus.getReplication() != d1Status.getReplication());
 
     // attributes of src -> f2 ? should be no
     CopyListingFileStatus f2Status = new CopyListingFileStatus(fs.getFileStatus(f2));
-    Assert.assertFalse(srcStatus.getPermission().equals(f2Status.getPermission()));
-    Assert.assertFalse(srcStatus.getOwner().equals(f2Status.getOwner()));
-    Assert.assertFalse(srcStatus.getGroup().equals(f2Status.getGroup()));
-    Assert.assertFalse(srcStatus.getAccessTime() == f2Status.getAccessTime());
-    Assert.assertFalse(srcStatus.getModificationTime() == f2Status.getModificationTime());
-    Assert.assertFalse(srcStatus.getReplication() == f2Status.getReplication());
+    assertFalse(srcStatus.getPermission().equals(f2Status.getPermission()));
+    assertFalse(srcStatus.getOwner().equals(f2Status.getOwner()));
+    assertFalse(srcStatus.getGroup().equals(f2Status.getGroup()));
+    assertFalse(srcStatus.getAccessTime() == f2Status.getAccessTime());
+    assertFalse(srcStatus.getModificationTime() == f2Status.getModificationTime());
+    assertFalse(srcStatus.getReplication() == f2Status.getReplication());
 
     // attributes of src -> f1 ? should be no
     CopyListingFileStatus f1Status = new CopyListingFileStatus(fs.getFileStatus(f1));
-    Assert.assertFalse(srcStatus.getPermission().equals(f1Status.getPermission()));
-    Assert.assertFalse(srcStatus.getOwner().equals(f1Status.getOwner()));
-    Assert.assertFalse(srcStatus.getGroup().equals(f1Status.getGroup()));
-    Assert.assertFalse(srcStatus.getAccessTime() == f1Status.getAccessTime());
-    Assert.assertFalse(srcStatus.getModificationTime() == f1Status.getModificationTime());
-    Assert.assertFalse(srcStatus.getReplication() == f1Status.getReplication());
+    assertFalse(srcStatus.getPermission().equals(f1Status.getPermission()));
+    assertFalse(srcStatus.getOwner().equals(f1Status.getOwner()));
+    assertFalse(srcStatus.getGroup().equals(f1Status.getGroup()));
+    assertFalse(srcStatus.getAccessTime() == f1Status.getAccessTime());
+    assertFalse(srcStatus.getModificationTime() == f1Status.getModificationTime());
+    assertFalse(srcStatus.getReplication() == f1Status.getReplication());
 
     // attributes of src -> f0 ? should be no
     CopyListingFileStatus f0Status = new CopyListingFileStatus(fs.getFileStatus(f0));
-    Assert.assertFalse(srcStatus.getPermission().equals(f0Status.getPermission()));
-    Assert.assertFalse(srcStatus.getOwner().equals(f0Status.getOwner()));
-    Assert.assertFalse(srcStatus.getGroup().equals(f0Status.getGroup()));
-    Assert.assertFalse(srcStatus.getAccessTime() == f0Status.getAccessTime());
-    Assert.assertFalse(srcStatus.getModificationTime() == f0Status.getModificationTime());
-    Assert.assertFalse(srcStatus.getReplication() == f0Status.getReplication());
+    assertFalse(srcStatus.getPermission().equals(f0Status.getPermission()));
+    assertFalse(srcStatus.getOwner().equals(f0Status.getOwner()));
+    assertFalse(srcStatus.getGroup().equals(f0Status.getGroup()));
+    assertFalse(srcStatus.getAccessTime() == f0Status.getAccessTime());
+    assertFalse(srcStatus.getModificationTime() == f0Status.getModificationTime());
+    assertFalse(srcStatus.getReplication() == f0Status.getReplication());
   }
 
   @Test
@@ -1115,39 +1109,39 @@ public class TestDistCpUtils {
 
     // attributes of src -> f1 ? should be no
     CopyListingFileStatus f1Status = new CopyListingFileStatus(fs.getFileStatus(f1));
-    Assert.assertFalse(srcStatus.getPermission().equals(f1Status.getPermission()));
-    Assert.assertFalse(srcStatus.getOwner().equals(f1Status.getOwner()));
-    Assert.assertFalse(srcStatus.getGroup().equals(f1Status.getGroup()));
-    Assert.assertFalse(srcStatus.getAccessTime() == f1Status.getAccessTime());
-    Assert.assertFalse(srcStatus.getModificationTime() == f1Status.getModificationTime());
-    Assert.assertFalse(srcStatus.getReplication() == f1Status.getReplication());
+    assertFalse(srcStatus.getPermission().equals(f1Status.getPermission()));
+    assertFalse(srcStatus.getOwner().equals(f1Status.getOwner()));
+    assertFalse(srcStatus.getGroup().equals(f1Status.getGroup()));
+    assertFalse(srcStatus.getAccessTime() == f1Status.getAccessTime());
+    assertFalse(srcStatus.getModificationTime() == f1Status.getModificationTime());
+    assertFalse(srcStatus.getReplication() == f1Status.getReplication());
 
     // attributes of src -> f2 ? should be no
     CopyListingFileStatus f2Status = new CopyListingFileStatus(fs.getFileStatus(f2));
-    Assert.assertFalse(srcStatus.getPermission().equals(f2Status.getPermission()));
-    Assert.assertFalse(srcStatus.getOwner().equals(f2Status.getOwner()));
-    Assert.assertFalse(srcStatus.getGroup().equals(f2Status.getGroup()));
-    Assert.assertFalse(srcStatus.getAccessTime() == f2Status.getAccessTime());
-    Assert.assertFalse(srcStatus.getModificationTime() == f2Status.getModificationTime());
-    Assert.assertFalse(srcStatus.getReplication() == f2Status.getReplication());
+    assertFalse(srcStatus.getPermission().equals(f2Status.getPermission()));
+    assertFalse(srcStatus.getOwner().equals(f2Status.getOwner()));
+    assertFalse(srcStatus.getGroup().equals(f2Status.getGroup()));
+    assertFalse(srcStatus.getAccessTime() == f2Status.getAccessTime());
+    assertFalse(srcStatus.getModificationTime() == f2Status.getModificationTime());
+    assertFalse(srcStatus.getReplication() == f2Status.getReplication());
 
     // attributes of src -> d1 ? should be no
     CopyListingFileStatus d1Status = new CopyListingFileStatus(fs.getFileStatus(d1));
-    Assert.assertFalse(srcStatus.getPermission().equals(d1Status.getPermission()));
-    Assert.assertFalse(srcStatus.getOwner().equals(d1Status.getOwner()));
-    Assert.assertFalse(srcStatus.getGroup().equals(d1Status.getGroup()));
-    Assert.assertTrue(d1Status.getAccessTime() == 400);
-    Assert.assertTrue(d1Status.getModificationTime() == 400);
-    Assert.assertFalse(srcStatus.getReplication() == d1Status.getReplication());
+    assertFalse(srcStatus.getPermission().equals(d1Status.getPermission()));
+    assertFalse(srcStatus.getOwner().equals(d1Status.getOwner()));
+    assertFalse(srcStatus.getGroup().equals(d1Status.getGroup()));
+    assertTrue(d1Status.getAccessTime() == 400);
+    assertTrue(d1Status.getModificationTime() == 400);
+    assertFalse(srcStatus.getReplication() == d1Status.getReplication());
 
     // attributes of src -> d2 ? should be no
     CopyListingFileStatus d2Status = new CopyListingFileStatus(fs.getFileStatus(d2));
-    Assert.assertFalse(srcStatus.getPermission().equals(d2Status.getPermission()));
-    Assert.assertFalse(srcStatus.getOwner().equals(d2Status.getOwner()));
-    Assert.assertFalse(srcStatus.getGroup().equals(d2Status.getGroup()));
-    Assert.assertTrue(d2Status.getAccessTime() == 300);
-    Assert.assertTrue(d2Status.getModificationTime() == 300);
-    Assert.assertFalse(srcStatus.getReplication() == d2Status.getReplication());
+    assertFalse(srcStatus.getPermission().equals(d2Status.getPermission()));
+    assertFalse(srcStatus.getOwner().equals(d2Status.getOwner()));
+    assertFalse(srcStatus.getGroup().equals(d2Status.getGroup()));
+    assertTrue(d2Status.getAccessTime() == 300);
+    assertTrue(d2Status.getModificationTime() == 300);
+    assertFalse(srcStatus.getReplication() == d2Status.getReplication());
   }
 
   @Test
@@ -1214,57 +1208,57 @@ public class TestDistCpUtils {
     // FileStatus.equals only compares path field, must explicitly compare all fields
     // attributes of src -> root ? should be yes
     CopyListingFileStatus rootStatus = new CopyListingFileStatus(fs.getFileStatus(root));
-    Assert.assertTrue(srcStatus.getPermission().equals(rootStatus.getPermission()));
-    Assert.assertTrue(srcStatus.getOwner().equals(rootStatus.getOwner()));
-    Assert.assertTrue(srcStatus.getGroup().equals(rootStatus.getGroup()));
-    Assert.assertTrue(srcStatus.getAccessTime() == rootStatus.getAccessTime());
-    Assert.assertTrue(srcStatus.getModificationTime() == rootStatus.getModificationTime());
-    Assert.assertTrue(srcStatus.getReplication() != rootStatus.getReplication());
+    assertTrue(srcStatus.getPermission().equals(rootStatus.getPermission()));
+    assertTrue(srcStatus.getOwner().equals(rootStatus.getOwner()));
+    assertTrue(srcStatus.getGroup().equals(rootStatus.getGroup()));
+    assertTrue(srcStatus.getAccessTime() == rootStatus.getAccessTime());
+    assertTrue(srcStatus.getModificationTime() == rootStatus.getModificationTime());
+    assertTrue(srcStatus.getReplication() != rootStatus.getReplication());
 
     // attributes of src -> d1 ? should be no
     CopyListingFileStatus d1Status = new CopyListingFileStatus(fs.getFileStatus(d1));
-    Assert.assertFalse(srcStatus.getPermission().equals(d1Status.getPermission()));
-    Assert.assertFalse(srcStatus.getOwner().equals(d1Status.getOwner()));
-    Assert.assertFalse(srcStatus.getGroup().equals(d1Status.getGroup()));
-    Assert.assertFalse(srcStatus.getAccessTime() == d1Status.getAccessTime());
-    Assert.assertFalse(srcStatus.getModificationTime() == d1Status.getModificationTime());
-    Assert.assertTrue(srcStatus.getReplication() != d1Status.getReplication());
+    assertFalse(srcStatus.getPermission().equals(d1Status.getPermission()));
+    assertFalse(srcStatus.getOwner().equals(d1Status.getOwner()));
+    assertFalse(srcStatus.getGroup().equals(d1Status.getGroup()));
+    assertFalse(srcStatus.getAccessTime() == d1Status.getAccessTime());
+    assertFalse(srcStatus.getModificationTime() == d1Status.getModificationTime());
+    assertTrue(srcStatus.getReplication() != d1Status.getReplication());
 
     // attributes of src -> d2 ? should be no
     CopyListingFileStatus d2Status = new CopyListingFileStatus(fs.getFileStatus(d2));
-    Assert.assertFalse(srcStatus.getPermission().equals(d2Status.getPermission()));
-    Assert.assertFalse(srcStatus.getOwner().equals(d2Status.getOwner()));
-    Assert.assertFalse(srcStatus.getGroup().equals(d2Status.getGroup()));
-    Assert.assertFalse(srcStatus.getAccessTime() == d2Status.getAccessTime());
-    Assert.assertFalse(srcStatus.getModificationTime() == d2Status.getModificationTime());
-    Assert.assertTrue(srcStatus.getReplication() != d2Status.getReplication());
+    assertFalse(srcStatus.getPermission().equals(d2Status.getPermission()));
+    assertFalse(srcStatus.getOwner().equals(d2Status.getOwner()));
+    assertFalse(srcStatus.getGroup().equals(d2Status.getGroup()));
+    assertFalse(srcStatus.getAccessTime() == d2Status.getAccessTime());
+    assertFalse(srcStatus.getModificationTime() == d2Status.getModificationTime());
+    assertTrue(srcStatus.getReplication() != d2Status.getReplication());
 
     // attributes of src -> f0 ? should be no
     CopyListingFileStatus f0Status = new CopyListingFileStatus(fs.getFileStatus(f0));
-    Assert.assertFalse(srcStatus.getPermission().equals(f0Status.getPermission()));
-    Assert.assertFalse(srcStatus.getOwner().equals(f0Status.getOwner()));
-    Assert.assertFalse(srcStatus.getGroup().equals(f0Status.getGroup()));
-    Assert.assertFalse(srcStatus.getAccessTime() == f0Status.getAccessTime());
-    Assert.assertFalse(srcStatus.getModificationTime() == f0Status.getModificationTime());
-    Assert.assertFalse(srcStatus.getReplication() == f0Status.getReplication());
+    assertFalse(srcStatus.getPermission().equals(f0Status.getPermission()));
+    assertFalse(srcStatus.getOwner().equals(f0Status.getOwner()));
+    assertFalse(srcStatus.getGroup().equals(f0Status.getGroup()));
+    assertFalse(srcStatus.getAccessTime() == f0Status.getAccessTime());
+    assertFalse(srcStatus.getModificationTime() == f0Status.getModificationTime());
+    assertFalse(srcStatus.getReplication() == f0Status.getReplication());
 
     // attributes of src -> f1 ? should be no
     CopyListingFileStatus f1Status = new CopyListingFileStatus(fs.getFileStatus(f1));
-    Assert.assertFalse(srcStatus.getPermission().equals(f1Status.getPermission()));
-    Assert.assertFalse(srcStatus.getOwner().equals(f1Status.getOwner()));
-    Assert.assertFalse(srcStatus.getGroup().equals(f1Status.getGroup()));
-    Assert.assertFalse(srcStatus.getAccessTime() == f1Status.getAccessTime());
-    Assert.assertFalse(srcStatus.getModificationTime() == f1Status.getModificationTime());
-    Assert.assertFalse(srcStatus.getReplication() == f1Status.getReplication());
+    assertFalse(srcStatus.getPermission().equals(f1Status.getPermission()));
+    assertFalse(srcStatus.getOwner().equals(f1Status.getOwner()));
+    assertFalse(srcStatus.getGroup().equals(f1Status.getGroup()));
+    assertFalse(srcStatus.getAccessTime() == f1Status.getAccessTime());
+    assertFalse(srcStatus.getModificationTime() == f1Status.getModificationTime());
+    assertFalse(srcStatus.getReplication() == f1Status.getReplication());
 
     // attributes of src -> f2 ? should be no
     CopyListingFileStatus f2Status = new CopyListingFileStatus(fs.getFileStatus(f2));
-    Assert.assertFalse(srcStatus.getPermission().equals(f2Status.getPermission()));
-    Assert.assertFalse(srcStatus.getOwner().equals(f2Status.getOwner()));
-    Assert.assertFalse(srcStatus.getGroup().equals(f2Status.getGroup()));
-    Assert.assertFalse(srcStatus.getAccessTime() == f2Status.getAccessTime());
-    Assert.assertFalse(srcStatus.getModificationTime() == f2Status.getModificationTime());
-    Assert.assertFalse(srcStatus.getReplication() == f2Status.getReplication());
+    assertFalse(srcStatus.getPermission().equals(f2Status.getPermission()));
+    assertFalse(srcStatus.getOwner().equals(f2Status.getOwner()));
+    assertFalse(srcStatus.getGroup().equals(f2Status.getGroup()));
+    assertFalse(srcStatus.getAccessTime() == f2Status.getAccessTime());
+    assertFalse(srcStatus.getModificationTime() == f2Status.getModificationTime());
+    assertFalse(srcStatus.getReplication() == f2Status.getReplication());
   }
 
   @Test

+ 17 - 20
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/util/TestDistCpUtilsWithCombineMode.java

@@ -25,17 +25,17 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInfo;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 
+import static org.junit.jupiter.api.Assertions.fail;
+
 /**
  * Test length and checksums comparison with checksum combine mode.
  * When the combine mode is COMPOSITE_CRC, it should tolerate different file
@@ -47,13 +47,10 @@ public class TestDistCpUtilsWithCombineMode {
   private Configuration config;
   private MiniDFSCluster cluster;
 
-  @Rule
-  public TestName testName = new TestName();
-
-  @Before
-  public void create() throws IOException {
+  @BeforeEach
+  public void create(TestInfo testInfo) throws IOException {
     config = new Configuration();
-    if (testName.getMethodName().contains("WithCombineMode")) {
+    if (testInfo.getDisplayName().contains("WithCombineMode")) {
       config.set("dfs.checksum.combine.mode", "COMPOSITE_CRC");
     }
     config.setLong(
@@ -64,7 +61,7 @@ public class TestDistCpUtilsWithCombineMode {
         .build();
   }
 
-  @After
+  @AfterEach
   public void destroy() {
     if (cluster != null) {
       cluster.shutdown();
@@ -72,9 +69,9 @@ public class TestDistCpUtilsWithCombineMode {
   }
 
   @Test
-  public void testChecksumsComparisonWithCombineMode() throws IOException {
+  public void testChecksumsComparisonWithCombineMode(TestInfo testInfo) throws IOException {
     try {
-      compareSameContentButDiffBlockSizes();
+      compareSameContentButDiffBlockSizes(testInfo.getDisplayName());
     } catch (IOException e) {
       LOG.error("Unexpected exception is found", e);
       throw e;
@@ -82,18 +79,18 @@ public class TestDistCpUtilsWithCombineMode {
   }
 
   @Test
-  public void testChecksumsComparisonWithoutCombineMode() {
+  public void testChecksumsComparisonWithoutCombineMode(TestInfo testInfo) {
     try {
-      compareSameContentButDiffBlockSizes();
-      Assert.fail("Expected comparison to fail");
+      compareSameContentButDiffBlockSizes(testInfo.getDisplayName());
+      fail("Expected comparison to fail");
     } catch (IOException e) {
       GenericTestUtils.assertExceptionContains(
           "Checksum mismatch", e);
     }
   }
 
-  private void compareSameContentButDiffBlockSizes() throws IOException {
-    String base = "/tmp/verify-checksum-" + testName.getMethodName() + "/";
+  private void compareSameContentButDiffBlockSizes(String name) throws IOException {
+    String base = "/tmp/verify-checksum-" + name + "/";
     long seed = System.currentTimeMillis();
     short rf = 2;
 

+ 18 - 12
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/util/TestProducerConsumer.java

@@ -23,13 +23,18 @@ import org.apache.hadoop.tools.util.ProducerConsumer;
 import org.apache.hadoop.tools.util.WorkReport;
 import org.apache.hadoop.tools.util.WorkRequest;
 import org.apache.hadoop.tools.util.WorkRequestProcessor;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import java.lang.Exception;
 import java.lang.Integer;
 import java.util.concurrent.TimeoutException;
 
+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;
+
 public class TestProducerConsumer {
   public class CopyProcessor implements WorkRequestProcessor<Integer, Integer> {
     public WorkReport<Integer> processItem(WorkRequest<Integer> workRequest) {
@@ -62,9 +67,9 @@ public class TestProducerConsumer {
     worker.put(new WorkRequest<Integer>(42));
     try {
       WorkReport<Integer> report = worker.take();
-      Assert.assertEquals(42, report.getItem().intValue());
+      assertEquals(42, report.getItem().intValue());
     } catch (InterruptedException ie) {
-      Assert.assertTrue(false);
+      assertTrue(false);
     }
     worker.shutdown();
   }
@@ -90,8 +95,8 @@ public class TestProducerConsumer {
       sum -= report.getItem().intValue();
       numReports++;
     }
-    Assert.assertEquals(0, sum);
-    Assert.assertEquals(numRequests, numReports);
+    assertEquals(0, sum);
+    assertEquals(numRequests, numReports);
     workers.shutdown();
   }
 
@@ -103,11 +108,11 @@ public class TestProducerConsumer {
     worker.put(new WorkRequest<Integer>(42));
     try {
       WorkReport<Integer> report = worker.take();
-      Assert.assertEquals(42, report.getItem().intValue());
-      Assert.assertFalse(report.getSuccess());
-      Assert.assertNotNull(report.getException());
+      assertEquals(42, report.getItem().intValue());
+      assertFalse(report.getSuccess());
+      assertNotNull(report.getException());
     } catch (InterruptedException ie) {
-      Assert.assertTrue(false);
+      assertTrue(false);
     }
     worker.shutdown();
   }
@@ -127,7 +132,8 @@ public class TestProducerConsumer {
     GenericTestUtils.waitForThreadTermination("pool-.*-thread.*",100,10000);
   }
 
-  @Test(timeout=10000)
+  @Test
+  @Timeout(value = 10)
   public void testMultipleProducerConsumerShutdown()
       throws InterruptedException, TimeoutException {
     int numWorkers = 10;
@@ -160,7 +166,7 @@ public class TestProducerConsumer {
         try {
           while (true) {
             WorkReport<Integer> report = worker.take();
-            Assert.assertEquals(42, report.getItem().intValue());
+            assertEquals(42, report.getItem().intValue());
           }
         } catch (InterruptedException ie) {
           return;

+ 9 - 8
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/util/TestRetriableCommand.java

@@ -20,11 +20,12 @@ package org.apache.hadoop.tools.util;
 
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.io.retry.RetryPolicies;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.util.concurrent.TimeUnit;
 
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
 public class TestRetriableCommand {
 
   private static class MyRetriableCommand extends RetriableCommand {
@@ -54,28 +55,28 @@ public class TestRetriableCommand {
   public void testRetriableCommand() {
     try {
       new MyRetriableCommand(5).execute(0);
-      Assert.assertTrue(false);
+      assertTrue(false);
     }
     catch (Exception e) {
-      Assert.assertTrue(true);
+      assertTrue(true);
     }
 
 
     try {
       new MyRetriableCommand(3).execute(0);
-      Assert.assertTrue(true);
+      assertTrue(true);
     }
     catch (Exception e) {
-      Assert.assertTrue(false);
+      assertTrue(false);
     }
 
     try {
       new MyRetriableCommand(5, RetryPolicies.
           retryUpToMaximumCountWithFixedSleep(5, 0, TimeUnit.MILLISECONDS)).execute(0);
-      Assert.assertTrue(true);
+      assertTrue(true);
     }
     catch (Exception e) {
-      Assert.assertTrue(false);
+      assertTrue(false);
     }
   }
 }

+ 12 - 11
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/util/TestThrottledInputStream.java

@@ -21,13 +21,14 @@ package org.apache.hadoop.tools.util;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.io.IOUtils;
-import org.junit.Assert;
-import static org.hamcrest.Matchers.greaterThanOrEqualTo;
-import static org.junit.Assert.assertThat;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.io.*;
 
+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 TestThrottledInputStream {
   private static final Logger LOG = LoggerFactory.getLogger(TestThrottledInputStream.class);
   private static final int BUFF_SIZE = 1024;
@@ -101,9 +102,9 @@ public class TestThrottledInputStream {
         which magnifies the error of getBytesPerSec()
       */
       bandwidth = in.getBytesPerSec();
-      Assert.assertEquals(in.getTotalBytesRead(), tmpFile.length());
-      Assert.assertTrue(bandwidth > maxBandwidth / (factor * 1.2));
-      Assert.assertTrue(in.getTotalSleepTime() >  sleepTime || bandwidth <= maxBPS);
+      assertEquals(in.getTotalBytesRead(), tmpFile.length());
+      assertTrue(bandwidth > maxBandwidth / (factor * 1.2));
+      assertTrue(in.getTotalSleepTime() >  sleepTime || bandwidth <= maxBPS);
     } finally {
       IOUtils.closeStream(in);
       IOUtils.closeStream(out);
@@ -198,8 +199,8 @@ public class TestThrottledInputStream {
       // Check whether the speed limit is successfully limited
       long end = System.currentTimeMillis();
       LOG.info("end: " + end);
-      assertThat((int) (end - begin) / 1000,
-          greaterThanOrEqualTo(testFileCnt * fileSize / bandwidth));
+      assertThat((int) (end - begin) / 1000).
+          isGreaterThanOrEqualTo(testFileCnt * fileSize / bandwidth);
     } catch (IOException e) {
       LOG.error("Exception encountered ", e);
     }
@@ -212,10 +213,10 @@ public class TestThrottledInputStream {
     try {
       copyBytes(in, out, BUFF_SIZE);
       LOG.info("{}", in);
-      Assert.assertEquals(in.getTotalBytesRead(), tmpFile.length());
+      assertEquals(in.getTotalBytesRead(), tmpFile.length());
 
       long bytesPerSec = in.getBytesPerSec();
-      Assert.assertTrue(bytesPerSec < maxBPS);
+      assertTrue(bytesPerSec < maxBPS);
     } finally {
       IOUtils.closeStream(in);
       IOUtils.closeStream(out);