Переглянути джерело

HDFS-12431. [JDK17] Upgrade JUnit from 4 to 5 in hadoop-hdfs Part6. (#7739)

* HDFS-12431. [JDK17] Upgrade JUnit from 4 to 5 in hadoop-hdfs Part5.

Co-authored-by: Shilun Fan <slfan1989@apache.org>
Reviewed-by: Shilun Fan <slfan1989@apache.org>
Signed-off-by: Shilun Fan <slfan1989@apache.org>
zhtttylz 1 тиждень тому
батько
коміт
6eae1589ae
45 змінених файлів з 1416 додано та 1266 видалено
  1. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRemove.java
  2. 10 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRename.java
  3. 10 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java
  4. 209 194
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
  5. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShellGenericOptions.java
  6. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShellTouch.java
  7. 8 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStartupVersions.java
  8. 9 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStorageStateRecovery.java
  9. 19 23
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
  10. 21 27
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
  11. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamUpdatePipeline.java
  12. 29 26
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java
  13. 18 15
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
  14. 196 190
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
  15. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDisableConnCache.java
  16. 59 53
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
  17. 17 17
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystemWithECFile.java
  18. 77 57
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptedTransfer.java
  19. 16 13
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesWithHA.java
  20. 14 15
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodeBenchmarkThroughput.java
  21. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingAddConfig.java
  22. 13 10
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingExerciseAPIs.java
  23. 9 12
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingMultipleRacks.java
  24. 84 96
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
  25. 50 61
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicyWithSnapshot.java
  26. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestExtendedAcls.java
  27. 27 23
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestExternalBlockReader.java
  28. 7 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSInputChecker.java
  29. 11 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFetchImage.java
  30. 40 35
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
  31. 11 13
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend2.java
  32. 15 16
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java
  33. 16 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
  34. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java
  35. 209 109
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileChecksum.java
  36. 30 25
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileConcurrentReader.java
  37. 12 12
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java
  38. 63 61
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
  39. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationClient.java
  40. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationDelete.java
  41. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationEmpty.java
  42. 11 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileLengthOnClusterRestart.java
  43. 41 41
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java
  44. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusSerialization.java
  45. 12 16
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithDefaultECPolicy.java

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRemove.java

@@ -16,8 +16,8 @@
  * limitations under the License.
  */
 package org.apache.hadoop.hdfs;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
@@ -28,7 +28,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 public class TestDFSRemove {
   final Path dir = new Path("/test/remove/");
@@ -80,8 +80,8 @@ public class TestDFSRemove {
         Thread.sleep(3 * DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT * 1000);
         // all blocks should be gone now.
         long dfsUsedFinal = getTotalDfsUsed(cluster);
-        assertEquals("All blocks should be gone. start=" + dfsUsedStart
-            + " max=" + dfsUsedMax + " final=" + dfsUsedFinal, dfsUsedStart, dfsUsedFinal);
+        assertEquals(dfsUsedStart, dfsUsedFinal, "All blocks should be gone. start=" + dfsUsedStart
+            + " max=" + dfsUsedMax + " final=" + dfsUsedFinal);
       }
 
       fs.delete(dir, true);

+ 10 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRename.java

@@ -16,9 +16,9 @@
  * limitations under the License.
  */
 package org.apache.hadoop.hdfs;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
@@ -34,7 +34,8 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 public class TestDFSRename {
   static int countLease(MiniDFSCluster cluster) {
@@ -136,7 +137,8 @@ public class TestDFSRename {
    * Check the blocks of dst file are cleaned after rename with overwrite
    * Restart NN to check the rename successfully
    */
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testRenameWithOverwrite() throws Exception {
     final short replFactor = 2;
     final long blockSize = 512;
@@ -194,9 +196,9 @@ public class TestDFSRename {
       dfs.rename(path, new Path("/dir1"),
           new Rename[] {Rename.OVERWRITE, Rename.TO_TRASH});
       String auditOut = auditLog.getOutput();
-      assertTrue("Rename should have both OVERWRITE and TO_TRASH "
-              + "flags at namenode but had only " + auditOut,
-          auditOut.contains("options=[OVERWRITE, TO_TRASH]"));
+      assertTrue(auditOut.contains("options=[OVERWRITE, TO_TRASH]"),
+          "Rename should have both OVERWRITE and TO_TRASH "
+              + "flags at namenode but had only " + auditOut);
     }
   }
 }

+ 10 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java

@@ -19,7 +19,10 @@ package org.apache.hadoop.hdfs;
 
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.DATA_NODE;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.NAME_NODE;
-import static org.junit.Assert.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.File;
 import java.io.IOException;
@@ -39,8 +42,8 @@ import org.apache.hadoop.hdfs.server.datanode.DataNodeLayoutVersion;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.util.StringUtils;
-import org.junit.After;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
 
 
 /**
@@ -80,8 +83,7 @@ public class TestDFSRollback {
         FSImageTestUtil.assertReasonableNameCurrentDir(curDir);
         break;
       case DATA_NODE:
-        assertEquals(
-            UpgradeUtilities.checksumContents(nodeType, curDir, false),
+        assertEquals(UpgradeUtilities.checksumContents(nodeType, curDir, false),
             UpgradeUtilities.checksumMasterDataNodeContents());
         break;
       }
@@ -127,8 +129,8 @@ public class TestDFSRollback {
   void startBlockPoolShouldFail(StartupOption operation, String bpid)
       throws IOException {
     cluster.startDataNodes(conf, 1, false, operation, null); // should fail
-    assertFalse("Block pool " + bpid + " should have failed to start", 
-        cluster.getDataNodes().get(0).isBPServiceAlive(bpid));
+    assertFalse(cluster.getDataNodes().get(0).isBPServiceAlive(bpid),
+        "Block pool " + bpid + " should have failed to start");
   }
  
   /**
@@ -344,7 +346,7 @@ public class TestDFSRollback {
     }
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     LOG.info("Shutting down MiniDFSCluster");
     if (cluster != null) {

Різницю між файлами не показано, бо вона завелика
+ 209 - 194
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java


+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShellGenericOptions.java

@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.File;
 import java.io.FileNotFoundException;
@@ -29,7 +29,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.util.ToolRunner;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 public class TestDFSShellGenericOptions {
 
@@ -103,8 +103,8 @@ public class TestDFSShellGenericOptions {
       ToolRunner.run(shell, args);
       fs = FileSystem.get(DFSUtilClient.getNNUri(
           DFSUtilClient.getNNAddress(namenode)), shell.getConf());
-      assertTrue("Directory does not get created",
-                 fs.isDirectory(new Path("/data")));
+      assertTrue(fs.isDirectory(new Path("/data")),
+          "Directory does not get created");
       fs.delete(new Path("/data"), true);
     } catch (Exception e) {
       System.err.println(e.getMessage());

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShellTouch.java

@@ -22,9 +22,9 @@ import java.io.IOException;
 import java.text.ParseException;
 import java.util.Date;
 
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -49,7 +49,7 @@ public class TestDFSShellTouch {
   private static DistributedFileSystem dfs;
   private static FsShell shell;
 
-  @BeforeClass
+  @BeforeAll
   public static void setup() throws IOException {
     final Configuration conf = new Configuration();
     conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR,
@@ -61,7 +61,7 @@ public class TestDFSShellTouch {
     shell = new FsShell(dfs.getConf());
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() {
     if (miniCluster != null) {
       miniCluster.shutdown(true, true);

+ 8 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStartupVersions.java

@@ -19,8 +19,8 @@ package org.apache.hadoop.hdfs;
 
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.DATA_NODE;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.NAME_NODE;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.File;
 
@@ -33,8 +33,9 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeLayoutVersion;
-import org.junit.After;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 /**
  * This test ensures the appropriate response (successful or failure) from 
@@ -238,7 +239,8 @@ public class TestDFSStartupVersions {
    *         this iterations version 3-tuple
    * </pre>
    */
-  @Test (timeout=300000)
+  @Test
+  @Timeout(value = 300)
   public void testVersions() throws Exception {
     UpgradeUtilities.initialize();
     Configuration conf = UpgradeUtilities.initializeStorageStateConf(1, 
@@ -280,7 +282,7 @@ public class TestDFSStartupVersions {
     }
   }
   
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     LOG.info("Shutting down MiniDFSCluster");
     if (cluster != null) {

+ 9 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStorageStateRecovery.java

@@ -19,10 +19,10 @@ package org.apache.hadoop.hdfs;
 
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.DATA_NODE;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.NAME_NODE;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.File;
 import java.io.IOException;
@@ -33,9 +33,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 /**
 * This test ensures the appropriate response (successful or failure) from
@@ -443,13 +443,13 @@ public class TestDFSStorageStateRecovery {
     } // end numDirs loop
   }
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     LOG.info("Setting up the directory structures.");
     UpgradeUtilities.initialize();
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     LOG.info("Shutting down MiniDFSCluster");
     if (cluster != null) {

+ 19 - 23
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java

@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hdfs;
 
-import org.junit.rules.TemporaryFolder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.HadoopIllegalArgumentException;
@@ -41,12 +40,9 @@ import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
 import org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawErasureCoderFactory;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
 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.Timeout;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -57,15 +53,18 @@ import java.util.List;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.ArgumentMatchers.anyLong;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.spy;
+import org.junit.jupiter.api.Timeout;
+import org.junit.jupiter.api.io.TempDir;
 
+@Timeout(300)
 public class TestDFSStripedInputStream {
 
   public static final Logger LOG =
@@ -84,17 +83,15 @@ public class TestDFSStripedInputStream {
   private int blockSize;
   private int blockGroupSize;
 
-  @Rule
-  public Timeout globalTimeout = new Timeout(300000);
-
-  @Rule
-  public TemporaryFolder baseDir = new TemporaryFolder();
+  @SuppressWarnings("checkstyle:VisibilityModifier")
+  @TempDir
+  java.nio.file.Path baseDir;
 
   public ErasureCodingPolicy getEcPolicy() {
     return StripedFileTestUtil.getDefaultECPolicy();
   }
 
-  @Before
+  @BeforeEach
   public void setup() throws IOException {
     /*
      * Initialize erasure coding policy.
@@ -119,7 +116,7 @@ public class TestDFSStripedInputStream {
   }
 
   private void startUp() throws IOException {
-    cluster = new MiniDFSCluster.Builder(conf, baseDir.getRoot()).numDataNodes(
+    cluster = new MiniDFSCluster.Builder(conf, baseDir.toFile()).numDataNodes(
         dataBlocks + parityBlocks).build();
     cluster.waitActive();
     for (DataNode dn : cluster.getDataNodes()) {
@@ -132,7 +129,7 @@ public class TestDFSStripedInputStream {
         .setErasureCodingPolicy(dirPath.toString(), ecPolicy.getName());
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
     if (cluster != null) {
       cluster.shutdown();
@@ -218,9 +215,8 @@ public class TestDFSStripedInputStream {
       int ret = in.read(startOffset, buf, 0, fileLen);
       assertEquals(remaining, ret);
       for (int i = 0; i < remaining; i++) {
-        Assert.assertEquals("Byte at " + (startOffset + i) + " should be the " +
-                "same",
-            expected[startOffset + i], buf[i]);
+        assertEquals(expected[startOffset + i], buf[i],
+            "Byte at " + (startOffset + i) + " should be the " + "same");
       }
     }
     in.close();

+ 21 - 27
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java

@@ -18,9 +18,9 @@
 package org.apache.hadoop.hdfs;
 
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Write.RECOVER_LEASE_ON_CLOSE_EXCEPTION_KEY;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.Mockito.doThrow;
 
 import java.io.ByteArrayInputStream;
@@ -47,13 +47,13 @@ import org.apache.hadoop.io.erasurecode.CodecUtil;
 import org.apache.hadoop.io.erasurecode.ErasureCodeNative;
 import org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawErasureCoderFactory;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.slf4j.event.Level;
+import org.junit.jupiter.api.Timeout;
 
+@Timeout(300)
 public class TestDFSStripedOutputStream {
   public static final Logger LOG = LoggerFactory.getLogger(
       TestDFSStripedOutputStream.class);
@@ -74,14 +74,11 @@ public class TestDFSStripedOutputStream {
   private final int stripesPerBlock = 4;
   private int blockSize;
 
-  @Rule
-  public Timeout globalTimeout = new Timeout(300000);
-
   public ErasureCodingPolicy getEcPolicy() {
     return StripedFileTestUtil.getDefaultECPolicy();
   }
 
-  @Before
+  @BeforeEach
   public void setup() throws IOException {
     /*
      * Initialize erasure coding policy.
@@ -110,7 +107,7 @@ public class TestDFSStripedOutputStream {
     fs.getClient().setErasureCodingPolicy("/", ecPolicy.getName());
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
     if (cluster != null) {
       cluster.shutdown();
@@ -206,12 +203,10 @@ public class TestDFSStripedOutputStream {
     final byte[] bytes = StripedFileTestUtil.generateBytes(blockSize *
         dataBlocks * 3 + cellSize * dataBlocks + cellSize + 123);
     try (FSDataOutputStream os = fs.create(new Path("/ec-file-1"))) {
-      assertFalse(
-          "DFSStripedOutputStream should not have hflush() capability yet!",
-          os.hasCapability(StreamCapability.HFLUSH.getValue()));
-      assertFalse(
-          "DFSStripedOutputStream should not have hsync() capability yet!",
-          os.hasCapability(StreamCapability.HSYNC.getValue()));
+      assertFalse(os.hasCapability(StreamCapability.HFLUSH.getValue()),
+          "DFSStripedOutputStream should not have hflush() capability yet!");
+      assertFalse(os.hasCapability(StreamCapability.HSYNC.getValue()),
+          "DFSStripedOutputStream should not have hsync() capability yet!");
       try (InputStream is = new ByteArrayInputStream(bytes)) {
         IOUtils.copyBytes(is, os, bytes.length);
         os.hflush();
@@ -219,8 +214,8 @@ public class TestDFSStripedOutputStream {
         os.hsync();
         IOUtils.copyBytes(is, os, bytes.length);
       }
-      assertTrue("stream is not a DFSStripedOutputStream",
-          os.getWrappedStream() instanceof DFSStripedOutputStream);
+      assertTrue(os.getWrappedStream() instanceof DFSStripedOutputStream,
+          "stream is not a DFSStripedOutputStream");
       final DFSStripedOutputStream dfssos =
           (DFSStripedOutputStream) os.getWrappedStream();
       dfssos.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
@@ -265,8 +260,8 @@ public class TestDFSStripedOutputStream {
         spyClient.create("/testExceptionInCloseECFileWithRecoverLease",
             FsPermission.getFileDefault(), EnumSet.of(CreateFlag.CREATE),
             (short) 3, 1024*1024, null, 1024, null);
-    assertTrue("stream should be a DFSStripedOutputStream",
-        dfsOutputStream instanceof DFSStripedOutputStream);
+    assertTrue(dfsOutputStream instanceof DFSStripedOutputStream,
+        "stream should be a DFSStripedOutputStream");
     DFSOutputStream spyDFSOutputStream = Mockito.spy(dfsOutputStream);
     doThrow(new IOException("Emulated IOException in close"))
         .when(spyDFSOutputStream).completeFile(Mockito.any());
@@ -290,8 +285,8 @@ public class TestDFSStripedOutputStream {
         spyClient.create("/testExceptionInCloseECFileWithoutRecoverLease",
             FsPermission.getFileDefault(), EnumSet.of(CreateFlag.CREATE),
             (short) 3, 1024*1024, null, 1024, null);
-    assertTrue("stream should be a DFSStripedOutputStream",
-        dfsOutputStream instanceof DFSStripedOutputStream);
+    assertTrue(dfsOutputStream instanceof DFSStripedOutputStream,
+        "stream should be a DFSStripedOutputStream");
     DFSOutputStream spyDFSOutputStream = Mockito.spy(dfsOutputStream);
     doThrow(new IOException("Emulated IOException in close"))
         .when(spyDFSOutputStream).completeFile(Mockito.any());
@@ -303,8 +298,7 @@ public class TestDFSStripedOutputStream {
       try {
         waitForFileClosed("/testExceptionInCloseECFileWithoutRecoverLease");
       } catch (TimeoutException e) {
-        assertFalse(
-            isFileClosed("/testExceptionInCloseECFileWithoutRecoverLease"));
+        assertFalse(isFileClosed("/testExceptionInCloseECFileWithoutRecoverLease"));
       }
     }
   }

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamUpdatePipeline.java

@@ -21,7 +21,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
@@ -66,7 +67,8 @@ public class TestDFSStripedOutputStreamUpdatePipeline {
    * Test writing ec file hang when applying the second block group occurs
    * an addBlock exception (e.g. quota exception).
    */
-  @Test(timeout = 90000)
+  @Test
+  @Timeout(value = 90)
   public void testECWriteHangWhenAddBlockWithException() throws Exception {
     Configuration conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1 * 1024 * 1024);

+ 29 - 26
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java

@@ -22,16 +22,18 @@ import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.
 import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getImageFileName;
 import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getInProgressEditsFileName;
 import static org.apache.hadoop.test.GenericTestUtils.assertExists;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.File;
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 import java.util.regex.Pattern;
 
+import org.junit.jupiter.api.Disabled;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -46,9 +48,9 @@ import org.apache.hadoop.hdfs.server.datanode.DataNodeLayoutVersion;
 import org.apache.hadoop.hdfs.server.namenode.TestParallelImageWrite;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.util.StringUtils;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import org.apache.hadoop.thirdparty.com.google.common.base.Joiner;
 
@@ -171,16 +173,14 @@ public class TestDFSUpgrade {
     } catch (Exception e) {
       // expect exception
       if (exceptionClass != null) {
-        assertTrue("Caught exception is not of expected class "
-            + exceptionClass.getSimpleName() + ": "
-            + StringUtils.stringifyException(e), 
-            exceptionClass.isInstance(e));
+        assertTrue(exceptionClass.isInstance(e),
+            "Caught exception is not of expected class "
+                + exceptionClass.getSimpleName() + ": " + StringUtils.stringifyException(e));
       }
       if (messagePattern != null) {
-        assertTrue("Caught exception message string does not match expected pattern \""
-            + messagePattern.pattern() + "\" : "
-            + StringUtils.stringifyException(e), 
-            messagePattern.matcher(e.getMessage()).find());
+        assertTrue(messagePattern.matcher(e.getMessage()).find(),
+            "Caught exception message string does not match expected pattern \""
+                + messagePattern.pattern() + "\" : " + StringUtils.stringifyException(e));
       }
       LOG.info("Successfully detected expected NameNode startup failure.");
     }
@@ -195,8 +195,8 @@ public class TestDFSUpgrade {
    */
   void startBlockPoolShouldFail(StartupOption operation, String bpid) throws IOException {
     cluster.startDataNodes(conf, 1, false, operation, null); // should fail
-    assertFalse("Block pool " + bpid + " should have failed to start",
-        cluster.getDataNodes().get(0).isBPServiceAlive(bpid));
+    assertFalse(cluster.getDataNodes().get(0).isBPServiceAlive(bpid),
+        "Block pool " + bpid + " should have failed to start");
   }
  
   /**
@@ -212,7 +212,7 @@ public class TestDFSUpgrade {
                                            .build();
   }
   
-  @BeforeClass
+  @BeforeAll
   public static void initialize() throws Exception {
     UpgradeUtilities.initialize();
   }
@@ -221,7 +221,9 @@ public class TestDFSUpgrade {
    * This test attempts to upgrade the NameNode and DataNode under
    * a number of valid and invalid conditions.
    */
-  @Test(timeout = 60000)
+  @SuppressWarnings("checkstyle:MethodLength")
+  @Test
+  @Timeout(value = 60)
   public void testUpgrade() throws Exception {
     File[] baseDirs;
     StorageInfo storageInfo = null;
@@ -404,7 +406,7 @@ public class TestDFSUpgrade {
    * Stand-alone test to detect failure of one SD during parallel upgrade.
    * At this time, can only be done with manual hack of {@link FSImage.doUpgrade()}
    */
-  @Ignore
+  @Disabled
   public void testUpgrade4() throws Exception {
     int numDirs = 4;
     conf = new HdfsConfiguration();
@@ -432,21 +434,22 @@ public class TestDFSUpgrade {
       File currentDir = new File(baseDir, "current");
       for (File f : currentDir.listFiles()) {
         if (f.getName().startsWith(prefix)) {
-          assertTrue("Deleting " + f, f.delete());
+          assertTrue(f.delete(), "Deleting " + f);
         }
       }
     }
   }
 
-  @Test(expected=IOException.class)
+  @Test
   public void testUpgradeFromPreUpgradeLVFails() throws IOException {
+    assertThrows(IOException.class, () -> {
+      Storage.checkVersionUpgradable(Storage.LAST_PRE_UPGRADE_LAYOUT_VERSION + 1);
+      fail("Expected IOException is not thrown");
+    });
     // Upgrade from versions prior to Storage#LAST_UPGRADABLE_LAYOUT_VERSION
-    // is not allowed
-    Storage.checkVersionUpgradable(Storage.LAST_PRE_UPGRADE_LAYOUT_VERSION + 1);
-    fail("Expected IOException is not thrown");
   }
   
-  @Ignore
+  @Disabled
   public void test203LayoutVersion() {
     for (int lv : Storage.LAYOUT_VERSIONS_203) {
       assertTrue(Storage.is203LayoutVersion(lv));

+ 18 - 15
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java

@@ -48,9 +48,12 @@ import org.apache.hadoop.hdfs.server.namenode.IllegalReservedPathException;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.log4j.Logger;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
-import static org.junit.Assert.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 /**
  * This tests data transfer protocol handling in the Datanode. It sends
@@ -161,7 +164,7 @@ public class TestDFSUpgradeFromImage {
       // The paths are expected to be listed in the same order 
       // as they are traversed here.
       assertEquals(info.path, path);
-      assertEquals("Checking checksum for " + path, info.checksum, checksum);
+      assertEquals(info.checksum, checksum, "Checking checksum for " + path);
     }
   }
   
@@ -251,9 +254,9 @@ public class TestDFSUpgradeFromImage {
 
     // Set up a fake NN storage that looks like an ancient Hadoop dir circa 0.3.0
     FileUtil.fullyDelete(namenodeStorage);
-    assertTrue("Make " + namenodeStorage, namenodeStorage.mkdirs());
+    assertTrue(namenodeStorage.mkdirs(), "Make " + namenodeStorage);
     File imageDir = new File(namenodeStorage, "image");
-    assertTrue("Make " + imageDir, imageDir.mkdirs());
+    assertTrue(imageDir.mkdirs(), "Make " + imageDir);
 
     // Hex dump of a formatted image from Hadoop 0.3.0
     File imageFile = new File(imageDir, "fsimage");
@@ -333,7 +336,7 @@ public class TestDFSUpgradeFromImage {
       }
       int md5failures = appender.countExceptionsWithMessage(
           " is corrupt with MD5 checksum of ");
-      assertEquals("Upgrade did not fail with bad MD5", 1, md5failures);
+      assertEquals(1, md5failures, "Upgrade did not fail with bad MD5");
     }
   }
 
@@ -395,10 +398,10 @@ public class TestDFSUpgradeFromImage {
           }
         }
         for (String s: expected) {
-          assertTrue("Did not find expected path " + s, found.contains(s));
+          assertTrue(found.contains(s), "Did not find expected path " + s);
         }
-        assertEquals("Found an unexpected path while listing filesystem",
-            found.size(), expected.length);
+        assertEquals(found.size(), expected.length,
+            "Found an unexpected path while listing filesystem");
       }
     } finally {
       if (cluster != null) {
@@ -459,10 +462,10 @@ public class TestDFSUpgradeFromImage {
           }
         }
         for (String s: expected) {
-          assertTrue("Did not find expected path " + s, found.contains(s));
+          assertTrue(found.contains(s), "Did not find expected path " + s);
         }
-        assertEquals("Found an unexpected path while listing filesystem",
-            found.size(), expected.length);
+        assertEquals(found.size(), expected.length,
+            "Found an unexpected path while listing filesystem");
       }
     } finally {
       if (cluster != null) {
@@ -554,10 +557,10 @@ public class TestDFSUpgradeFromImage {
           }
         }
         for (String s: expected) {
-          assertTrue("Did not find expected path " + s, found.contains(s));
+          assertTrue(found.contains(s), "Did not find expected path " + s);
         }
-        assertEquals("Found an unexpected path while listing filesystem",
-            found.size(), expected.length);
+        assertEquals(found.size(), expected.length,
+            "Found an unexpected path while listing filesystem");
       }
     } finally {
       if (cluster != null) {

+ 196 - 190
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java

@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -37,15 +37,15 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_PAS
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_TRUSTSTORE_PASSWORD_KEY;
 import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.apache.hadoop.test.PlatformAssumptions.assumeNotWindows;
-import static org.hamcrest.CoreMatchers.not;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+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.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 static org.mockito.Mockito.*;
+import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.File;
 import java.io.IOException;
@@ -84,26 +84,26 @@ import org.apache.hadoop.security.alias.JavaKeyStoreProvider;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.LambdaTestUtils;
 import org.apache.hadoop.util.Shell;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 public class TestDFSUtil {
 
-  static final String NS1_NN_ADDR    = "ns1-nn.example.com:8020";
-  static final String NS1_NN1_ADDR   = "ns1-nn1.example.com:8020";
-  static final String NS1_NN2_ADDR   = "ns1-nn2.example.com:8020";
-  static final String NS1_NN1_HTTPS_ADDR   = "ns1-nn1.example.com:50740";
-  static final String NS1_NN1_HTTP_ADDR    = "ns1-nn1.example.com:50070";
+  static final String NS1_NN_ADDR = "ns1-nn.example.com:8020";
+  static final String NS1_NN1_ADDR = "ns1-nn1.example.com:8020";
+  static final String NS1_NN2_ADDR = "ns1-nn2.example.com:8020";
+  static final String NS1_NN1_HTTPS_ADDR = "ns1-nn1.example.com:50740";
+  static final String NS1_NN1_HTTP_ADDR = "ns1-nn1.example.com:50070";
 
   /**
    * Reset to default UGI settings since some tests change them.
    */
-  @Before
+  @BeforeEach
   public void resetUGI() {
     UserGroupInformation.setConfiguration(new Configuration());
   }
-  
+
   /**
    * Test conversion of LocatedBlock to BlockLocation
    */
@@ -130,18 +130,17 @@ public class TestDFSUtil {
 
     BlockLocation[] bs = DFSUtilClient.locatedBlocks2Locations(lbs);
 
-    assertTrue("expected 2 blocks but got " + bs.length,
-               bs.length == 2);
+    assertTrue(bs.length == 2, "expected 2 blocks but got " + bs.length);
 
     int corruptCount = 0;
-    for (BlockLocation b: bs) {
+    for (BlockLocation b : bs) {
       if (b.isCorrupt()) {
         corruptCount++;
       }
     }
 
-    assertTrue("expected 1 corrupt files but got " + corruptCount,
-        corruptCount == 1);
+    assertTrue(corruptCount == 1,
+        "expected 1 corrupt files but got " + corruptCount);
 
     // test an empty location
     bs = DFSUtilClient.locatedBlocks2Locations(new LocatedBlocks());
@@ -156,7 +155,7 @@ public class TestDFSUtil {
     DatanodeInfo d = DFSTestUtil.getLocalDatanodeInfo();
     DatanodeInfo[] ds = new DatanodeInfo[1];
     ds[0] = d;
-    
+
     ExtendedBlock b1 = new ExtendedBlock("bpid", 1, 1, 1);
     LocatedBlock l1 = new LocatedBlock(b1, ds, null, null, 0, false, null);
     final DatanodeInfo[] cachedLocs = l1.getCachedLocations();
@@ -180,7 +179,7 @@ public class TestDFSUtil {
     conf.set(DFS_NAMESERVICE_ID, "nn1");
     assertEquals("nn1", DFSUtil.getNamenodeNameServiceId(conf));
   }
-  
+
   /**
    * Test {@link DFSUtil#getNamenodeNameServiceId(Configuration)} to ensure
    * nameserviceId for namenode is determined based on matching the address with
@@ -219,16 +218,18 @@ public class TestDFSUtil {
    * exception is thrown when multiple rpc addresses match the local node's
    * address
    */
-  @Test(expected = HadoopIllegalArgumentException.class)
+  @Test
   public void testGetNameServiceIdException() {
-    HdfsConfiguration conf = new HdfsConfiguration();
-    conf.set(DFS_NAMESERVICES, "nn1,nn2");
-    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY, "nn1"),
-        "localhost:9000");
-    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY, "nn2"),
-        "localhost:9001");
-    DFSUtil.getNamenodeNameServiceId(conf);
-    fail("Expected exception is not thrown");
+    assertThrows(HadoopIllegalArgumentException.class, () -> {
+      HdfsConfiguration conf = new HdfsConfiguration();
+      conf.set(DFS_NAMESERVICES, "nn1,nn2");
+      conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY, "nn1"),
+          "localhost:9000");
+      conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY, "nn2"),
+          "localhost:9001");
+      DFSUtil.getNamenodeNameServiceId(conf);
+      fail("Expected exception is not thrown");
+    });
   }
 
   /**
@@ -244,7 +245,7 @@ public class TestDFSUtil {
     assertEquals("nn1", it.next().toString());
     assertEquals("nn2", it.next().toString());
   }
-  
+
   @Test
   public void testGetOnlyNameServiceIdOrNull() {
     HdfsConfiguration conf = new HdfsConfiguration();
@@ -277,13 +278,13 @@ public class TestDFSUtil {
     Map<String, Map<String, InetSocketAddress>> nnMap = DFSUtil
         .getNNServiceRpcAddresses(conf);
     assertEquals(2, nnMap.size());
-    
+
     Map<String, InetSocketAddress> nn1Map = nnMap.get("nn1");
     assertEquals(1, nn1Map.size());
     InetSocketAddress addr = nn1Map.get(null);
     assertEquals("localhost", addr.getHostName());
     assertEquals(9000, addr.getPort());
-    
+
     Map<String, InetSocketAddress> nn2Map = nnMap.get("nn2");
     assertEquals(1, nn2Map.size());
     addr = nn2Map.get(null);
@@ -301,7 +302,7 @@ public class TestDFSUtil {
   }
 
   public void checkNameServiceId(Configuration conf, String addr,
-      String expectedNameServiceId) {
+                                 String expectedNameServiceId) {
     InetSocketAddress s = NetUtils.createSocketAddr(addr);
     String nameserviceId = DFSUtil.getNameServiceIdFromAddress(conf, s,
         DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, DFS_NAMENODE_RPC_ADDRESS_KEY);
@@ -317,15 +318,15 @@ public class TestDFSUtil {
     // If DFS_FEDERATION_NAMESERVICES is not set, verify that
     // default namenode address is returned.
     Map<String, Map<String, InetSocketAddress>> addrMap =
-      DFSUtil.getNNServiceRpcAddresses(conf);
+        DFSUtil.getNNServiceRpcAddresses(conf);
     assertEquals(1, addrMap.size());
-    
+
     Map<String, InetSocketAddress> defaultNsMap = addrMap.get(null);
     assertEquals(1, defaultNsMap.size());
-    
+
     assertEquals(9999, defaultNsMap.get(null).getPort());
   }
-  
+
   /**
    * Test to ensure nameservice specific keys in the configuration are
    * copied to generic keys when the namenode starts.
@@ -334,7 +335,7 @@ public class TestDFSUtil {
   public void testConfModificationFederationOnly() {
     final HdfsConfiguration conf = new HdfsConfiguration();
     String nsId = "ns1";
-    
+
     conf.set(DFS_NAMESERVICES, nsId);
     conf.set(DFS_NAMESERVICE_ID, nsId);
 
@@ -353,7 +354,7 @@ public class TestDFSUtil {
       assertEquals(key, conf.get(key));
     }
   }
-  
+
   /**
    * Test to ensure nameservice specific keys in the configuration are
    * copied to generic keys when the namenode starts.
@@ -363,7 +364,7 @@ public class TestDFSUtil {
     final HdfsConfiguration conf = new HdfsConfiguration();
     String nsId = "ns1";
     String nnId = "nn1";
-    
+
     conf.set(DFS_NAMESERVICES, nsId);
     conf.set(DFS_NAMESERVICE_ID, nsId);
     conf.set(DFS_HA_NAMENODES_KEY_PREFIX + "." + nsId, nnId);
@@ -387,7 +388,7 @@ public class TestDFSUtil {
   /**
    * Ensure that fs.defaultFS is set in the configuration even if neither HA nor
    * Federation is enabled.
-   * 
+   *
    * Regression test for HDFS-3351.
    */
   @Test
@@ -395,7 +396,7 @@ public class TestDFSUtil {
     final HdfsConfiguration conf = new HdfsConfiguration();
     String nsId = null;
     String nnId = null;
-    
+
     conf.set(DFS_NAMENODE_RPC_ADDRESS_KEY, "localhost:1234");
 
     assertFalse("hdfs://localhost:1234".equals(conf.get(FS_DEFAULT_NAME_KEY)));
@@ -419,23 +420,23 @@ public class TestDFSUtil {
     Configuration newConf = new Configuration(conf);
     NameNode.initializeGenericKeys(newConf, "ns2", "nn1");
     assertEquals("global-default", newConf.get(key));
-    
+
     // A namenode in another non-HA nameservice should get global default.
     newConf = new Configuration(conf);
     NameNode.initializeGenericKeys(newConf, "ns2", null);
-    assertEquals("global-default", newConf.get(key));    
-    
+    assertEquals("global-default", newConf.get(key));
+
     // A namenode in the same nameservice should get the ns setting
     newConf = new Configuration(conf);
     NameNode.initializeGenericKeys(newConf, "ns1", "nn2");
-    assertEquals("ns1-override", newConf.get(key));    
+    assertEquals("ns1-override", newConf.get(key));
 
     // The nn with the nn-specific setting should get its own override
     newConf = new Configuration(conf);
     NameNode.initializeGenericKeys(newConf, "ns1", "nn1");
-    assertEquals("nn1-override", newConf.get(key));    
+    assertEquals("nn1-override", newConf.get(key));
   }
-  
+
   /**
    * Tests for empty configuration, an exception is thrown from
    * {@link DFSUtil#getNNServiceRpcAddresses(Configuration)}
@@ -456,7 +457,7 @@ public class TestDFSUtil {
 
     try {
       Map<String, Map<String, InetSocketAddress>> map =
-        DFSUtil.getBackupNodeAddresses(conf);
+          DFSUtil.getBackupNodeAddresses(conf);
       fail("Expected IOException is not thrown, result was: " +
           DFSUtil.addressMapToString(map));
     } catch (IOException expected) {
@@ -465,7 +466,7 @@ public class TestDFSUtil {
 
     try {
       Map<String, Map<String, InetSocketAddress>> map =
-        DFSUtil.getSecondaryNameNodeAddresses(conf);
+          DFSUtil.getSecondaryNameNodeAddresses(conf);
       fail("Expected IOException is not thrown, result was: " +
           DFSUtil.addressMapToString(map));
     } catch (IOException expected) {
@@ -502,11 +503,11 @@ public class TestDFSUtil {
   @Test
   public void testGetInfoServer() throws IOException, URISyntaxException {
     HdfsConfiguration conf = new HdfsConfiguration();
-    
+
     URI httpsport = DFSUtil.getInfoServer(null, conf, "https");
     assertEquals(new URI("https", null, "0.0.0.0",
         DFS_NAMENODE_HTTPS_PORT_DEFAULT, null, null, null), httpsport);
-    
+
     URI httpport = DFSUtil.getInfoServer(null, conf, "http");
     assertEquals(new URI("http", null, "0.0.0.0",
         DFS_NAMENODE_HTTP_PORT_DEFAULT, null, null, null), httpport);
@@ -517,17 +518,17 @@ public class TestDFSUtil {
         URI.create("http://localhost:" + DFS_NAMENODE_HTTP_PORT_DEFAULT),
         httpAddress);
   }
-  
+
   @Test
   public void testHANameNodesWithFederation() throws URISyntaxException {
     HdfsConfiguration conf = new HdfsConfiguration();
-    
+
     final String NS1_NN1_HOST = "ns1-nn1.example.com:8020";
     final String NS1_NN2_HOST = "ns1-nn2.example.com:8020";
     final String NS2_NN1_HOST = "ns2-nn1.example.com:8020";
     final String NS2_NN2_HOST = "ns2-nn2.example.com:8020";
     conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "hdfs://ns1");
-    
+
     // Two nameservices, each with two NNs.
     conf.set(DFS_NAMESERVICES, "ns1,ns2");
     conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, "ns1"),
@@ -535,35 +536,35 @@ public class TestDFSUtil {
     conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, "ns2"),
         "ns2-nn1,ns2-nn2");
     conf.set(DFSUtil.addKeySuffixes(
-          DFS_NAMENODE_RPC_ADDRESS_KEY, "ns1", "ns1-nn1"),
+            DFS_NAMENODE_RPC_ADDRESS_KEY, "ns1", "ns1-nn1"),
         NS1_NN1_HOST);
     conf.set(DFSUtil.addKeySuffixes(
-        DFS_NAMENODE_RPC_ADDRESS_KEY, "ns1", "ns1-nn2"),
+            DFS_NAMENODE_RPC_ADDRESS_KEY, "ns1", "ns1-nn2"),
         NS1_NN2_HOST);
     conf.set(DFSUtil.addKeySuffixes(
-        DFS_NAMENODE_RPC_ADDRESS_KEY, "ns2", "ns2-nn1"),
+            DFS_NAMENODE_RPC_ADDRESS_KEY, "ns2", "ns2-nn1"),
         NS2_NN1_HOST);
     conf.set(DFSUtil.addKeySuffixes(
-        DFS_NAMENODE_RPC_ADDRESS_KEY, "ns2", "ns2-nn2"),
+            DFS_NAMENODE_RPC_ADDRESS_KEY, "ns2", "ns2-nn2"),
         NS2_NN2_HOST);
-    
+
     Map<String, Map<String, InetSocketAddress>> map =
         DFSUtilClient.getHaNnRpcAddresses(conf);
 
     assertTrue(HAUtil.isHAEnabled(conf, "ns1"));
     assertTrue(HAUtil.isHAEnabled(conf, "ns2"));
     assertFalse(HAUtil.isHAEnabled(conf, "ns3"));
-    
+
     assertEquals(NS1_NN1_HOST, map.get("ns1").get("ns1-nn1").toString());
     assertEquals(NS1_NN2_HOST, map.get("ns1").get("ns1-nn2").toString());
     assertEquals(NS2_NN1_HOST, map.get("ns2").get("ns2-nn1").toString());
     assertEquals(NS2_NN2_HOST, map.get("ns2").get("ns2-nn2").toString());
-    
-    assertEquals(NS1_NN1_HOST, 
+
+    assertEquals(NS1_NN1_HOST,
         DFSUtil.getNamenodeServiceAddr(conf, "ns1", "ns1-nn1"));
-    assertEquals(NS1_NN2_HOST, 
+    assertEquals(NS1_NN2_HOST,
         DFSUtil.getNamenodeServiceAddr(conf, "ns1", "ns1-nn2"));
-    assertEquals(NS2_NN1_HOST, 
+    assertEquals(NS2_NN1_HOST,
         DFSUtil.getNamenodeServiceAddr(conf, "ns2", "ns2-nn1"));
 
     // No nameservice was given and we can't determine which service addr
@@ -587,15 +588,15 @@ public class TestDFSUtil {
   @Test
   public void getNameNodeServiceAddr() throws IOException {
     HdfsConfiguration conf = new HdfsConfiguration();
-    
+
     // One nameservice with two NNs
     final String NS1_NN1_HOST = "ns1-nn1.example.com:8020";
     final String NS1_NN1_HOST_SVC = "ns1-nn2.example.com:9821";
     final String NS1_NN2_HOST = "ns1-nn1.example.com:8020";
     final String NS1_NN2_HOST_SVC = "ns1-nn2.example.com:9821";
-   
+
     conf.set(DFS_NAMESERVICES, "ns1");
-    conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, "ns1"),"nn1,nn2"); 
+    conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, "ns1"), "nn1,nn2");
 
     conf.set(DFSUtil.addKeySuffixes(
         DFS_NAMENODE_RPC_ADDRESS_KEY, "ns1", "nn1"), NS1_NN1_HOST);
@@ -609,7 +610,7 @@ public class TestDFSUtil {
     // A nameservice is specified explicitly
     assertEquals(NS1_NN1_HOST, DFSUtil.getNamenodeServiceAddr(conf, "ns1", "nn1"));
     assertEquals(null, DFSUtil.getNamenodeServiceAddr(conf, "invalid", "nn1"));
-    
+
     // The service addrs are used when they are defined
     conf.set(DFSUtil.addKeySuffixes(
         DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, "ns1", "nn1"), NS1_NN1_HOST_SVC);
@@ -641,7 +642,7 @@ public class TestDFSUtil {
     HdfsConfiguration conf = new HdfsConfiguration();
 
     conf.set(DFS_NAMESERVICES, "ns1");
-    conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, "ns1"),"nn1,nn2");
+    conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, "ns1"), "nn1,nn2");
     conf.set(DFSUtil.addKeySuffixes(
         DFS_NAMENODE_HTTP_ADDRESS_KEY, "ns1", "nn1"), nnaddr1);
     conf.set(DFSUtil.addKeySuffixes(
@@ -661,7 +662,7 @@ public class TestDFSUtil {
   }
 
   private static Collection<URI> getInternalNameServiceUris(Configuration conf,
-      String... keys) {
+                                                            String... keys) {
     final Collection<String> ids = DFSUtil.getInternalNameServices(conf);
     return DFSUtil.getNameServiceUris(conf, ids, keys);
   }
@@ -671,14 +672,15 @@ public class TestDFSUtil {
    * settings
    * @throws Exception
    */
+  @SuppressWarnings("LocalFinalVariableName")
   @Test
   public void testGetNNUris() throws Exception {
     HdfsConfiguration conf = new HdfsConfiguration();
 
-    final String NS2_NN_ADDR    = "ns2-nn.example.com:8020";
-    final String NN1_ADDR       = "nn.example.com:8020";
-    final String NN1_SRVC_ADDR  = "nn.example.com:9821";
-    final String NN2_ADDR       = "nn2.example.com:8020";
+    final String NS2_NN_ADDR = "ns2-nn.example.com:8020";
+    final String NN1_ADDR = "nn.example.com:8020";
+    final String NN1_SRVC_ADDR = "nn.example.com:9821";
+    final String NN2_ADDR = "nn2.example.com:8020";
 
     conf.set(DFS_NAMESERVICES, "ns1");
     conf.set(DFSUtil.addKeySuffixes(
@@ -687,11 +689,11 @@ public class TestDFSUtil {
     conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "hdfs://" + NN1_ADDR);
 
     Collection<URI> uris = DFSUtil.getInternalNsRpcUris(conf);
-    assertEquals("Incorrect number of URIs returned", 2, uris.size());
-    assertTrue("Missing URI for name service ns1",
-        uris.contains(new URI("hdfs://" + NS1_NN1_ADDR)));
-    assertTrue("Missing URI for service address",
-        uris.contains(new URI("hdfs://" + NN2_ADDR)));
+    assertEquals(2, uris.size(), "Incorrect number of URIs returned");
+    assertTrue(uris.contains(new URI("hdfs://" + NS1_NN1_ADDR)),
+        "Missing URI for name service ns1");
+    assertTrue(uris.contains(new URI("hdfs://" + NN2_ADDR)),
+        "Missing URI for service address");
 
     conf = new HdfsConfiguration();
     conf.set(DFS_NAMESERVICES, "ns1,ns2");
@@ -724,15 +726,13 @@ public class TestDFSUtil {
         + "IPFailoverProxyProvider");
 
     uris = DFSUtil.getInternalNsRpcUris(conf);
-    assertEquals("Incorrect number of URIs returned", 3, uris.size());
-    assertTrue("Missing URI for RPC address",
-        uris.contains(new URI("hdfs://" + NN1_ADDR)));
-    assertTrue("Missing URI for name service ns2",
-        uris.contains(new URI(HdfsConstants.HDFS_URI_SCHEME + "://" +
-            NS1_NN_ADDR)));
-    assertTrue("Missing URI for name service ns2",
-        uris.contains(new URI(HdfsConstants.HDFS_URI_SCHEME + "://" +
-            NS2_NN_ADDR)));
+    assertEquals(3, uris.size(), "Incorrect number of URIs returned");
+    assertTrue(uris.contains(new URI("hdfs://" + NN1_ADDR)),
+        "Missing URI for RPC address");
+    assertTrue(uris.contains(new URI(HdfsConstants.HDFS_URI_SCHEME + "://" +
+        NS1_NN_ADDR)), "Missing URI for name service ns2");
+    assertTrue(uris.contains(new URI(HdfsConstants.HDFS_URI_SCHEME + "://" +
+        NS2_NN_ADDR)), "Missing URI for name service ns2");
 
     /**
      * Second, test ns1 with {@link ConfiguredFailoverProxyProvider} which does
@@ -743,57 +743,57 @@ public class TestDFSUtil {
         + "ConfiguredFailoverProxyProvider");
 
     uris = DFSUtil.getInternalNsRpcUris(conf);
-    assertEquals("Incorrect number of URIs returned", 3, uris.size());
-    assertTrue("Missing URI for name service ns1",
-        uris.contains(new URI("hdfs://ns1")));
-    assertTrue("Missing URI for name service ns2",
-        uris.contains(new URI("hdfs://" + NS2_NN_ADDR)));
-    assertTrue("Missing URI for RPC address",
-        uris.contains(new URI("hdfs://" + NN1_ADDR)));
+    assertEquals(3, uris.size(), "Incorrect number of URIs returned");
+    assertTrue(uris.contains(new URI("hdfs://ns1")), "" +
+        "Missing URI for name service ns1");
+    assertTrue(uris.contains(new URI("hdfs://" + NS2_NN_ADDR)),
+        "Missing URI for name service ns2");
+    assertTrue(uris.contains(new URI("hdfs://" + NN1_ADDR)),
+        "Missing URI for RPC address");
 
     // Make sure that non-HDFS URIs in fs.defaultFS don't get included.
     conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY,
         "viewfs://vfs-name.example.com");
 
     uris = DFSUtil.getInternalNsRpcUris(conf);
-    assertEquals("Incorrect number of URIs returned", 3, uris.size());
-    assertTrue("Missing URI for name service ns1",
-        uris.contains(new URI("hdfs://ns1")));
-    assertTrue("Missing URI for name service ns2",
-        uris.contains(new URI("hdfs://" + NS2_NN_ADDR)));
-    assertTrue("Missing URI for RPC address",
-        uris.contains(new URI("hdfs://" + NN1_ADDR)));
+    assertEquals(3, uris.size(), "Incorrect number of URIs returned");
+    assertTrue(uris.contains(new URI("hdfs://ns1")),
+        "Missing URI for name service ns1");
+    assertTrue(uris.contains(new URI("hdfs://" + NS2_NN_ADDR)),
+        "Missing URI for name service ns2");
+    assertTrue(uris.contains(new URI("hdfs://" + NN1_ADDR)),
+        "Missing URI for RPC address");
 
     // Make sure that an HA URI being the default URI doesn't result in multiple
     // entries being returned.
     conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "hdfs://ns1");
-    
+
     uris = DFSUtil.getInternalNsRpcUris(conf);
-    assertEquals("Incorrect number of URIs returned", 3, uris.size());
-    assertTrue("Missing URI for name service ns1",
-        uris.contains(new URI("hdfs://ns1")));
-    assertTrue("Missing URI for name service ns2",
-        uris.contains(new URI("hdfs://" + NS2_NN_ADDR)));
-    assertTrue("Missing URI for RPC address",
-        uris.contains(new URI("hdfs://" + NN1_ADDR)));
+    assertEquals(3, uris.size(), "Incorrect number of URIs returned");
+    assertTrue(uris.contains(new URI("hdfs://ns1")),
+        "Missing URI for name service ns1");
+    assertTrue(uris.contains(new URI("hdfs://" + NS2_NN_ADDR)),
+        "Missing URI for name service ns2");
+    assertTrue(uris.contains(new URI("hdfs://" + NN1_ADDR)),
+        "Missing URI for RPC address");
 
     // Check that the default URI is returned if there's nothing else to return.
     conf = new HdfsConfiguration();
     conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "hdfs://" + NN1_ADDR);
 
     uris = DFSUtil.getInternalNsRpcUris(conf);
-    assertEquals("Incorrect number of URIs returned", 1, uris.size());
-    assertTrue("Missing URI for RPC address (defaultFS)",
-        uris.contains(new URI("hdfs://" + NN1_ADDR)));
+    assertEquals(1, uris.size(), "Incorrect number of URIs returned");
+    assertTrue(uris.contains(new URI("hdfs://" + NN1_ADDR)),
+        "Missing URI for RPC address (defaultFS)");
 
     // Check that the RPC address is the only address returned when the RPC
     // and the default FS is given.
     conf.set(DFS_NAMENODE_RPC_ADDRESS_KEY, NN2_ADDR);
 
     uris = DFSUtil.getInternalNsRpcUris(conf);
-    assertEquals("Incorrect number of URIs returned", 1, uris.size());
-    assertTrue("Missing URI for RPC address",
-        uris.contains(new URI("hdfs://" + NN2_ADDR)));
+    assertEquals(1, uris.size(), "Incorrect number of URIs returned");
+    assertTrue(uris.contains(new URI("hdfs://" + NN2_ADDR)),
+        "Missing URI for RPC address");
 
     // Make sure that when a service RPC address is used that is distinct from
     // the client RPC address, and that client RPC address is also used as the
@@ -802,20 +802,20 @@ public class TestDFSUtil {
     conf.set(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, NN1_ADDR);
 
     uris = DFSUtil.getInternalNsRpcUris(conf);
-    assertEquals("Incorrect number of URIs returned", 1, uris.size());
-    assertTrue("Missing URI for service ns1",
-        uris.contains(new URI("hdfs://" + NN1_ADDR)));
+    assertEquals(1, uris.size(), "Incorrect number of URIs returned");
+    assertTrue(uris.contains(new URI("hdfs://" + NN1_ADDR)),
+        "Missing URI for service ns1");
 
     // Check that when the default FS and service address are given, but
     // the RPC address isn't, that only the service address is returned.
     conf = new HdfsConfiguration();
     conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "hdfs://" + NN1_ADDR);
     conf.set(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, NN1_SRVC_ADDR);
-    
+
     uris = DFSUtil.getInternalNsRpcUris(conf);
-    assertEquals("Incorrect number of URIs returned", 1, uris.size());
-    assertTrue("Missing URI for service address",
-        uris.contains(new URI("hdfs://" + NN1_SRVC_ADDR)));
+    assertEquals(1, uris.size(), "Incorrect number of URIs returned");
+    assertTrue(uris.contains(new URI("hdfs://" + NN1_SRVC_ADDR)),
+        "Missing URI for service address");
   }
 
   @Test
@@ -843,12 +843,13 @@ public class TestDFSUtil {
 
     Collection<URI> uris = DFSUtil.getInternalNsRpcUris(conf);
 
-    assertEquals("Incorrect number of URIs returned", 1, uris.size());
-    assertTrue("Missing URI for name service ns1",
-        uris.contains(new URI("hdfs://ns1")));
+    assertEquals(1, uris.size(), "Incorrect number of URIs returned");
+    assertTrue(uris.contains(new URI("hdfs://ns1")),
+        "Missing URI for name service ns1");
   }
 
-  @Test (timeout=15000)
+  @Test
+  @Timeout(value = 15)
   public void testLocalhostReverseLookup() {
     // 127.0.0.1 -> localhost reverse resolution does not happen on Windows.
     assumeNotWindows();
@@ -860,67 +861,71 @@ public class TestDFSUtil {
     Collection<URI> uris = getInternalNameServiceUris(conf);
     assertEquals(1, uris.size());
     for (URI uri : uris) {
-      assertThat(uri.getHost(), not("127.0.0.1"));
+      assertThat(uri.getHost()).isNotEqualTo("127.0.0.1");
     }
   }
 
-  @Test (timeout=15000)
+  @Test
+  @Timeout(value = 15)
   public void testIsValidName() {
     String validPaths[] = new String[]{"/", "/bar/"};
     for (String path : validPaths) {
-      assertTrue("Should have been accepted '" + path + "'", DFSUtil.isValidName(path));
+      assertTrue(DFSUtil.isValidName(path), "Should have been accepted '" + path + "'");
     }
 
     String invalidPaths[] =
         new String[]{"/foo/../bar", "/foo/./bar", "/foo//bar", "/foo/:/bar", "/foo:bar"};
     for (String path : invalidPaths) {
-      assertFalse("Should have been rejected '" + path + "'", DFSUtil.isValidName(path));
+      assertFalse(DFSUtil.isValidName(path), "Should have been rejected '" + path + "'");
     }
 
     String windowsPath = "/C:/foo/bar";
     if (Shell.WINDOWS) {
-      assertTrue("Should have been accepted '" + windowsPath + "' in windows os.",
-          DFSUtil.isValidName(windowsPath));
+      assertTrue(DFSUtil.isValidName(windowsPath), "Should have been accepted '" +
+          windowsPath + "' in windows os.");
     } else {
-      assertFalse("Should have been rejected '" + windowsPath + "' in unix os.",
-          DFSUtil.isValidName(windowsPath));
+      assertFalse(DFSUtil.isValidName(windowsPath), "Should have been rejected '" +
+          windowsPath + "' in unix os.");
     }
   }
-  
-  @Test(timeout=5000)
+
+  @Test
+  @Timeout(value = 5)
   public void testGetSpnegoKeytabKey() {
     HdfsConfiguration conf = new HdfsConfiguration();
     String defaultKey = "default.spengo.key";
     conf.unset(DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY);
-    assertEquals("Test spnego key in config is null", defaultKey,
-        DFSUtil.getSpnegoKeytabKey(conf, defaultKey));
+    assertEquals(defaultKey, DFSUtil.getSpnegoKeytabKey(conf, defaultKey),
+        "Test spnego key in config is null");
 
     conf.set(DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY, "");
-    assertEquals("Test spnego key is empty", defaultKey,
-        DFSUtil.getSpnegoKeytabKey(conf, defaultKey));
+    assertEquals(defaultKey, DFSUtil.getSpnegoKeytabKey(conf, defaultKey),
+        "Test spnego key is empty");
 
     String spengoKey = "spengo.key";
     conf.set(DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY,
         spengoKey);
-    assertEquals("Test spnego key is NOT null",
+    assertEquals(
         DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY,
-        DFSUtil.getSpnegoKeytabKey(conf, defaultKey));
+        DFSUtil.getSpnegoKeytabKey(conf, defaultKey), "Test spnego key is NOT null");
   }
 
-  @Test(timeout=10000)
+  @Test
+  @Timeout(value = 10)
   public void testDurationToString() throws Exception {
     assertEquals("000:00:00:00.000", DFSUtil.durationToString(0));
     assertEquals("001:01:01:01.000",
-        DFSUtil.durationToString(((24*60*60)+(60*60)+(60)+1)*1000));
+        DFSUtil.durationToString(((24 * 60 * 60) + (60 * 60) + (60) + 1) * 1000));
     assertEquals("000:23:59:59.999",
-        DFSUtil.durationToString(((23*60*60)+(59*60)+(59))*1000+999));
+        DFSUtil.durationToString(((23 * 60 * 60) + (59 * 60) + (59)) * 1000 + 999));
     assertEquals("-001:01:01:01.000",
-        DFSUtil.durationToString(-((24*60*60)+(60*60)+(60)+1)*1000));
+        DFSUtil.durationToString(-((24 * 60 * 60) + (60 * 60) + (60) + 1) * 1000));
     assertEquals("-000:23:59:59.574",
-        DFSUtil.durationToString(-(((23*60*60)+(59*60)+(59))*1000+574)));
+        DFSUtil.durationToString(-(((23 * 60 * 60) + (59 * 60) + (59)) * 1000 + 574)));
   }
 
-  @Test(timeout=5000)
+  @Test
+  @Timeout(value = 5)
   public void testRelativeTimeConversion() throws Exception {
     try {
       DFSUtil.parseRelativeTime("1");
@@ -937,14 +942,14 @@ public class TestDFSUtil {
     } catch (IOException e) {
       assertExceptionContains("is not a number", e);
     }
-    assertEquals(61*1000, DFSUtil.parseRelativeTime("61s"));
-    assertEquals(61*60*1000, DFSUtil.parseRelativeTime("61m"));
+    assertEquals(61 * 1000, DFSUtil.parseRelativeTime("61s"));
+    assertEquals(61 * 60 * 1000, DFSUtil.parseRelativeTime("61m"));
     assertEquals(0, DFSUtil.parseRelativeTime("0s"));
-    assertEquals(25*60*60*1000, DFSUtil.parseRelativeTime("25h"));
-    assertEquals(4*24*60*60*1000l, DFSUtil.parseRelativeTime("4d"));
-    assertEquals(999*24*60*60*1000l, DFSUtil.parseRelativeTime("999d"));
+    assertEquals(25 * 60 * 60 * 1000, DFSUtil.parseRelativeTime("25h"));
+    assertEquals(4 * 24 * 60 * 60 * 1000L, DFSUtil.parseRelativeTime("4d"));
+    assertEquals(999 * 24 * 60 * 60 * 1000L, DFSUtil.parseRelativeTime("999d"));
   }
-  
+
   @Test
   public void testAssertAllResultsEqual() {
     checkAllResults(new Long[]{}, true);
@@ -953,7 +958,7 @@ public class TestDFSUtil {
     checkAllResults(new Long[]{1l, 1l, 1l}, true);
     checkAllResults(new Long[]{new Long(1), new Long(1)}, true);
     checkAllResults(new Long[]{null, null, null}, true);
-    
+
     checkAllResults(new Long[]{1l, 2l}, false);
     checkAllResults(new Long[]{2l, 1l}, false);
     checkAllResults(new Long[]{1l, 2l, 1l}, false);
@@ -963,7 +968,7 @@ public class TestDFSUtil {
     checkAllResults(new Long[]{null, 1l}, false);
     checkAllResults(new Long[]{1l, null, 1l}, false);
   }
-  
+
   private static void checkAllResults(Long[] toCheck, boolean shouldSucceed) {
     if (shouldSucceed) {
       DFSUtil.assertAllResultsEqual(Arrays.asList(toCheck));
@@ -985,7 +990,7 @@ public class TestDFSUtil {
     Configuration conf = new Configuration();
     final Path jksPath = new Path(testDir.toString(), "test.jks");
     final String ourUrl =
-    JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri();
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri();
 
     File file = new File(testDir, "test.jks");
     file.delete();
@@ -1031,15 +1036,15 @@ public class TestDFSUtil {
         DFS_SERVER_HTTPS_TRUSTSTORE_PASSWORD_KEY).getCredential());
 
     // use WebAppUtils as would be used by loadSslConfiguration
-    Assert.assertEquals("keypass",
+    assertEquals("keypass",
         DFSUtil.getPassword(conf, DFS_SERVER_HTTPS_KEYPASSWORD_KEY));
-    Assert.assertEquals("storepass",
+    assertEquals("storepass",
         DFSUtil.getPassword(conf, DFS_SERVER_HTTPS_KEYSTORE_PASSWORD_KEY));
-    Assert.assertEquals("trustpass",
+    assertEquals("trustpass",
         DFSUtil.getPassword(conf, DFS_SERVER_HTTPS_TRUSTSTORE_PASSWORD_KEY));
 
     // let's make sure that a password that doesn't exist returns null
-    Assert.assertEquals(null, DFSUtil.getPassword(conf,"invalid-alias"));
+    assertEquals(null, DFSUtil.getPassword(conf, "invalid-alias"));
   }
 
   @Test
@@ -1051,9 +1056,9 @@ public class TestDFSUtil {
     final String NN1_ADDRESS = "localhost:9000";
     final String NN2_ADDRESS = "localhost:9001";
     conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY, "nn1"),
-            NN1_ADDRESS);
+        NN1_ADDRESS);
     conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY, "nn2"),
-            NN2_ADDRESS);
+        NN2_ADDRESS);
 
     {
       Collection<String> internal = DFSUtil.getInternalNameServices(conf);
@@ -1064,7 +1069,7 @@ public class TestDFSUtil {
     }
 
     Map<String, Map<String, InetSocketAddress>> nnMap = DFSUtil
-            .getNNServiceRpcAddressesForCluster(conf);
+        .getNNServiceRpcAddressesForCluster(conf);
     assertEquals(1, nnMap.size());
     assertTrue(nnMap.containsKey("nn1"));
 
@@ -1080,20 +1085,21 @@ public class TestDFSUtil {
   public void testEncryptionProbe() throws Throwable {
     Configuration conf = new Configuration(false);
     conf.unset(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH);
-    assertFalse("encryption enabled on no provider key",
-        DFSUtilClient.isHDFSEncryptionEnabled(conf));
+    assertFalse(
+        DFSUtilClient.isHDFSEncryptionEnabled(conf),
+        "encryption enabled on no provider key");
     conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
         "");
-    assertFalse("encryption enabled on empty provider key",
-        DFSUtilClient.isHDFSEncryptionEnabled(conf));
+    assertFalse(DFSUtilClient.isHDFSEncryptionEnabled(conf),
+        "encryption enabled on empty provider key");
     conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
         "\n\t\n");
-    assertFalse("encryption enabled on whitespace provider key",
-        DFSUtilClient.isHDFSEncryptionEnabled(conf));
+    assertFalse(DFSUtilClient.isHDFSEncryptionEnabled(conf),
+        "encryption enabled on whitespace provider key");
     conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
         "http://hadoop.apache.org");
-    assertTrue("encryption disabled on valid provider key",
-        DFSUtilClient.isHDFSEncryptionEnabled(conf));
+    assertTrue(DFSUtilClient.isHDFSEncryptionEnabled(conf),
+        "encryption disabled on valid provider key");
 
   }
 
@@ -1101,8 +1107,8 @@ public class TestDFSUtil {
   public void testFileIdPath() throws Throwable {
     // /.reserved/.inodes/
     String prefix = Path.SEPARATOR + HdfsConstants.DOT_RESERVED_STRING +
-                    Path.SEPARATOR + HdfsConstants.DOT_INODES_STRING +
-                    Path.SEPARATOR;
+        Path.SEPARATOR + HdfsConstants.DOT_INODES_STRING +
+        Path.SEPARATOR;
     Random r = new Random();
     for (int i = 0; i < 100; ++i) {
       long inode = r.nextLong() & Long.MAX_VALUE;
@@ -1116,12 +1122,12 @@ public class TestDFSUtil {
     Configuration conf = new HdfsConfiguration();
     conf.set(DFSConfigKeys.DFS_NAMESERVICES, "ns1, ns2");
     String expectedErrorMessage = "Incorrect configuration: namenode address "
-            + DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY + ".[ns1, ns2]"
-            + " or "
-            + DFS_NAMENODE_RPC_ADDRESS_KEY + ".[ns1, ns2]"
-            + " is not configured.";
+        + DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY + ".[ns1, ns2]"
+        + " or "
+        + DFS_NAMENODE_RPC_ADDRESS_KEY + ".[ns1, ns2]"
+        + " is not configured.";
     LambdaTestUtils.intercept(IOException.class, expectedErrorMessage,
-        ()->DFSUtil.getNNServiceRpcAddressesForCluster(conf));
+        () -> DFSUtil.getNNServiceRpcAddressesForCluster(conf));
   }
 
   @Test
@@ -1198,7 +1204,7 @@ public class TestDFSUtil {
         if (isLazy) {
           // Lazy resolved. There is no need to change host->ip in advance.
           assertTrue(inetSocketAddress.isUnresolved());
-        }else {
+        } else {
           // Need resolve all host->ip.
           assertFalse(inetSocketAddress.isUnresolved());
         }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDisableConnCache.java

@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -25,7 +25,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.impl.BlockReaderTestUtil;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 /**
  * This class tests disabling client connection caching in a single node

+ 59 - 53
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java

@@ -23,12 +23,12 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_FILE_CLOSE_NUM_C
 import static org.apache.hadoop.hdfs.client.HdfsAdmin.TRASH_PERMISSION;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CONTEXT;
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyBoolean;
 import static org.mockito.ArgumentMatchers.anyString;
@@ -132,8 +132,8 @@ import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.concurrent.HadoopExecutors;
 import org.apache.hadoop.util.functional.RemoteIterators;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.mockito.InOrder;
 import org.mockito.Mockito;
 import org.slf4j.Logger;
@@ -234,13 +234,13 @@ public class TestDistributedFileSystem {
       types.add(OpenFilesIterator.OpenFilesType.ALL_OPEN_FILES);
       RemoteIterator<OpenFileEntry> listOpenFiles =
           fileSys.listOpenFiles(EnumSet.copyOf(types));
-      assertTrue("Two files should be open", listOpenFiles.hasNext());
+      assertTrue(listOpenFiles.hasNext(), "Two files should be open");
       int countOpenFiles = 0;
       while (listOpenFiles.hasNext()) {
         listOpenFiles.next();
         ++countOpenFiles;
       }
-      assertEquals("Mismatch of open files count", 2, countOpenFiles);
+      assertEquals(2, countOpenFiles, "Mismatch of open files count");
 
       // create another file, close it, and read it, so
       // the client gets a socket in its SocketCache
@@ -452,7 +452,7 @@ public class TestDistributedFileSystem {
         // success
         threw = true;
       }
-      assertTrue("Failed to throw IOE when seeking past end", threw);
+      assertTrue(threw, "Failed to throw IOE when seeking past end");
       input.close();
       threw = false;
       try {
@@ -461,7 +461,7 @@ public class TestDistributedFileSystem {
         //success
         threw = true;
       }
-      assertTrue("Failed to throw IOE when seeking after close", threw);
+      assertTrue(threw, "Failed to throw IOE when seeking after close");
       fileSys.close();
     }
     finally {
@@ -570,7 +570,7 @@ public class TestDistributedFileSystem {
         // Check to see if opening a non-existent file triggers a FNF
         FileSystem fs = cluster.getFileSystem();
         Path dir = new Path("/wrwelkj");
-        assertFalse("File should not exist for test.", fs.exists(dir));
+        assertFalse(fs.exists(dir), "File should not exist for test.");
 
         try {
           FSDataInputStream in = fs.open(dir);
@@ -578,8 +578,8 @@ public class TestDistributedFileSystem {
             in.close();
             fs.close();
           } finally {
-            assertTrue("Did not get a FileNotFoundException for non-existing" +
-                " file.", false);
+            assertTrue(false,
+                "Did not get a FileNotFoundException for non-existing" + " file.");
           }
         } catch (FileNotFoundException fnf) {
           // This is the proper exception to catch; move on.
@@ -629,11 +629,11 @@ public class TestDistributedFileSystem {
         fs.create(new Path("/tmp/nonEmptyDir/emptyFile")).close();
         try {
           fs.delete(new Path("/tmp/nonEmptyDir"), false);
-          Assert.fail("Expecting PathIsNotEmptyDirectoryException");
+          fail("Expecting PathIsNotEmptyDirectoryException");
         } catch (PathIsNotEmptyDirectoryException ex) {
           // This is the proper exception to catch; move on.
         }
-        Assert.assertTrue(fs.exists(new Path("/test/nonEmptyDir")));
+        assertTrue(fs.exists(new Path("/test/nonEmptyDir")));
         fs.delete(new Path("/tmp/nonEmptyDir"), true);
       }
 
@@ -1057,7 +1057,8 @@ public class TestDistributedFileSystem {
   }
 
   @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
-  @Test (timeout = 180000)
+  @Test
+  @Timeout(value = 180)
   public void testConcurrentStatistics()
       throws IOException, InterruptedException {
     FileSystem.getStatistics(HdfsConstants.HDFS_URI_SCHEME,
@@ -1106,8 +1107,7 @@ public class TestDistributedFileSystem {
       // wait until all threads are done
       allDone.await();
 
-     assertNull("Child failed with exception " + childError.get(),
-          childError.get());
+      assertNull(childError.get(), "Child failed with exception " + childError.get());
 
       checkStatistics(fs, 0, numThreads, 0);
       // check the single operation count stat
@@ -1119,8 +1119,8 @@ public class TestDistributedFileSystem {
            opCountIter.hasNext();) {
         final LongStatistic opCount = opCountIter.next();
         if (OpType.MKDIRS.getSymbol().equals(opCount.getName())) {
-          assertEquals("Unexpected op count from iterator!",
-              numThreads + oldMkdirOpCount, opCount.getValue());
+          assertEquals(numThreads + oldMkdirOpCount, opCount.getValue(),
+              "Unexpected op count from iterator!");
         }
         LOG.info(opCount.getName() + "\t" + opCount.getValue());
       }
@@ -1238,8 +1238,8 @@ public class TestDistributedFileSystem {
   }
 
   public static void checkOpStatistics(OpType op, long count) {
-    assertEquals("Op " + op.getSymbol() + " has unexpected count!",
-        count, getOpStatistics(op));
+    assertEquals(count, getOpStatistics(op), "Op " + op.getSymbol() +
+        " has unexpected count!");
   }
 
   public static long getOpStatistics(OpType op) {
@@ -1270,8 +1270,8 @@ public class TestDistributedFileSystem {
           "/test/TestNonExistingFile"));
       fail("Expecting FileNotFoundException");
     } catch (FileNotFoundException e) {
-      assertTrue("Not throwing the intended exception message", e.getMessage()
-          .contains("File does not exist: /test/TestNonExistingFile"));
+      assertTrue(e.getMessage().contains("File does not exist: /test/TestNonExistingFile"),
+          "Not throwing the intended exception message");
     }
 
     try {
@@ -1280,8 +1280,8 @@ public class TestDistributedFileSystem {
       hdfs.getFileChecksum(path);
       fail("Expecting FileNotFoundException");
     } catch (FileNotFoundException e) {
-      assertTrue("Not throwing the intended exception message", e.getMessage()
-          .contains("Path is not a file: /test/TestExistingDir"));
+      assertTrue(e.getMessage().contains("Path is not a file: /test/TestExistingDir"),
+          "Not throwing the intended exception message");
     }
 
     //webhdfs
@@ -1417,7 +1417,8 @@ public class TestDistributedFileSystem {
     }
   }
 
-  @Test(timeout=120000)
+  @Test
+  @Timeout(value = 120)
   public void testLocatedFileStatusStorageIdsTypes() throws Exception {
     final Configuration conf = getTestConfiguration();
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
@@ -1434,10 +1435,10 @@ public class TestDistributedFileSystem {
       DFSTestUtil.waitForReplication(fs, testFile, (short) repl, 30000);
       // Get the listing
       RemoteIterator<LocatedFileStatus> it = fs.listLocatedStatus(testFile);
-      assertTrue("Expected file to be present", it.hasNext());
+      assertTrue(it.hasNext(), "Expected file to be present");
       LocatedFileStatus stat = it.next();
       BlockLocation[] locs = stat.getBlockLocations();
-      assertEquals("Unexpected number of locations", numBlocks, locs.length);
+      assertEquals(numBlocks, locs.length, "Unexpected number of locations");
 
       Set<String> dnStorageIds = new HashSet<>();
       for (DataNode d : cluster.getDataNodes()) {
@@ -1454,15 +1455,14 @@ public class TestDistributedFileSystem {
         // Run it through a set to deduplicate, since there should be no dupes
         Set<String> storageIds = new HashSet<>();
         Collections.addAll(storageIds, ids);
-        assertEquals("Unexpected num storage ids", repl, storageIds.size());
+        assertEquals(repl, storageIds.size(), "Unexpected num storage ids");
         // Make sure these are all valid storage IDs
-        assertTrue("Unknown storage IDs found!", dnStorageIds.containsAll
-            (storageIds));
+        assertTrue(dnStorageIds.containsAll(storageIds), "Unknown storage IDs found!");
         // Check storage types are the default, since we didn't set any
         StorageType[] types = loc.getStorageTypes();
-        assertEquals("Unexpected num storage types", repl, types.length);
+        assertEquals(repl, types.length, "Unexpected num storage types");
         for (StorageType t: types) {
-          assertEquals("Unexpected storage type", StorageType.DEFAULT, t);
+          assertEquals(StorageType.DEFAULT, t, "Unexpected storage type");
         }
       }
     } finally {
@@ -1528,7 +1528,8 @@ public class TestDistributedFileSystem {
     }
   }
 
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testFileCloseStatus() throws IOException {
     Configuration conf = getTestConfiguration();
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
@@ -1540,9 +1541,9 @@ public class TestDistributedFileSystem {
       // write to file
       output.writeBytes("Some test data");
       output.flush();
-      assertFalse("File status should be open", fs.isFileClosed(file));
+      assertFalse(fs.isFileClosed(file), "File status should be open");
       output.close();
-      assertTrue("File status should be closed", fs.isFileClosed(file));
+      assertTrue(fs.isFileClosed(file), "File status should be closed");
     } finally {
       cluster.shutdown();
     }
@@ -1587,7 +1588,8 @@ public class TestDistributedFileSystem {
     }
   }
 
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testListFiles() throws IOException {
     Configuration conf = getTestConfiguration();
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
@@ -1679,7 +1681,8 @@ public class TestDistributedFileSystem {
     }
   }
 
-  @Test(timeout=10000)
+  @Test
+  @Timeout(value = 10)
   public void testDFSClientPeerReadTimeout() throws IOException {
     final int timeout = 1000;
     final Configuration conf = getTestConfiguration();
@@ -1697,7 +1700,7 @@ public class TestDistributedFileSystem {
       long start = Time.now();
       try {
         peer.getInputStream().read();
-        Assert.fail("read should timeout");
+        fail("read should timeout");
       } catch (SocketTimeoutException ste) {
         long delta = Time.now() - start;
         if (delta < timeout*0.9) {
@@ -1714,7 +1717,8 @@ public class TestDistributedFileSystem {
     }
   }
 
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testGetServerDefaults() throws IOException {
     Configuration conf = getTestConfiguration();
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
@@ -1728,7 +1732,8 @@ public class TestDistributedFileSystem {
     }
   }
 
-  @Test(timeout=10000)
+  @Test
+  @Timeout(value = 10)
   public void testDFSClientPeerWriteTimeout() throws IOException {
     final int timeout = 1000;
     final Configuration conf = getTestConfiguration();
@@ -1748,7 +1753,7 @@ public class TestDistributedFileSystem {
         byte[] buf = new byte[10 * 1024 * 1024];
         peer.getOutputStream().write(buf);
         long delta = Time.now() - start;
-        Assert.fail("write finish in " + delta + " ms" + "but should timedout");
+        fail("write finish in " + delta + " ms" + "but should timedout");
       } catch (SocketTimeoutException ste) {
         long delta = Time.now() - start;
 
@@ -1766,7 +1771,8 @@ public class TestDistributedFileSystem {
     }
   }
 
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testTotalDfsUsed() throws Exception {
     Configuration conf = getTestConfiguration();
     MiniDFSCluster cluster = null;
@@ -1906,15 +1912,15 @@ public class TestDistributedFileSystem {
       } catch (FileNotFoundException e) {
         // As expected.
       }
-      assertFalse("parent directory should not be created",
-          fs.exists(new Path("/parent")));
+      assertFalse(fs.exists(new Path("/parent")),
+          "parent directory should not be created");
 
       try (FSDataOutputStream out = fs.createFile(nonParentFile).recursive()
         .build()) {
         out.write(1);
       }
-      assertTrue("parent directory has not been created",
-          fs.exists(new Path("/parent")));
+      assertTrue(fs.exists(new Path("/parent")),
+          "parent directory has not been created");
     }
   }
 
@@ -2108,7 +2114,7 @@ public class TestDistributedFileSystem {
       //test enable a policy that doesn't exist
       try {
         fs.enableErasureCodingPolicy("notExistECName");
-        Assert.fail("enable the policy that doesn't exist should fail");
+        fail("enable the policy that doesn't exist should fail");
       } catch (Exception e) {
         GenericTestUtils.assertExceptionContains("does not exist", e);
         // pass
@@ -2117,7 +2123,7 @@ public class TestDistributedFileSystem {
       //test disable a policy that doesn't exist
       try {
         fs.disableErasureCodingPolicy("notExistECName");
-        Assert.fail("disable the policy that doesn't exist should fail");
+        fail("disable the policy that doesn't exist should fail");
       } catch (Exception e) {
         GenericTestUtils.assertExceptionContains("does not exist", e);
         // pass
@@ -2179,7 +2185,7 @@ public class TestDistributedFileSystem {
           .getBlockLocations(file1.toUri().getPath(), 0, Long.MAX_VALUE);
       int numSSD = Collections.frequency(
           Arrays.asList(locations[0].getStorageTypes()), StorageType.SSD);
-      assertEquals("Number of SSD should be 1 but was : " + numSSD, 1, numSSD);
+      assertEquals(1, numSSD, "Number of SSD should be 1 but was : " + numSSD);
     }
   }
 

+ 17 - 17
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystemWithECFile.java

@@ -28,20 +28,19 @@ import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.io.IOUtils;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.List;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 /**
  * Testing correctness of FileSystem.getFileBlockLocations and
@@ -66,10 +65,7 @@ public class TestDistributedFileSystemWithECFile {
     return StripedFileTestUtil.getDefaultECPolicy();
   }
 
-  @Rule
-  public final Timeout globalTimeout = new Timeout(60000 * 3);
-
-  @Before
+  @BeforeEach
   public void setup() throws IOException {
     ecPolicy = getEcPolicy();
     cellSize = ecPolicy.getCellSize();
@@ -92,7 +88,7 @@ public class TestDistributedFileSystemWithECFile {
         ecPolicy.getName());
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws IOException {
     if (cluster != null) {
       cluster.shutdown();
@@ -108,7 +104,8 @@ public class TestDistributedFileSystemWithECFile {
     StripedFileTestUtil.verifyLength(fs, src, size);
   }
 
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testListECFilesSmallerThanOneCell() throws Exception {
     createFile("/ec/smallcell", 1);
     final List<LocatedFileStatus> retVal = new ArrayList<>();
@@ -142,7 +139,8 @@ public class TestDistributedFileSystemWithECFile {
     assertTrue(blockLocation.getHosts().length == 1 + parityBlocks);
   }
 
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testListECFilesSmallerThanOneStripe() throws Exception {
     int dataBlocksNum = dataBlocks;
     createFile("/ec/smallstripe", cellSize * dataBlocksNum);
@@ -173,7 +171,8 @@ public class TestDistributedFileSystemWithECFile {
     assertTrue(blockLocation.getLength() == dataBlocksNum * cellSize);
   }
 
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testListECFilesMoreThanOneBlockGroup() throws Exception {
     createFile("/ec/group", blockGroupSize + 123);
     RemoteIterator<LocatedFileStatus> iter =
@@ -207,7 +206,8 @@ public class TestDistributedFileSystemWithECFile {
     assertTrue(lastBlock.getLength() == lastBlockSize);
   }
 
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testReplayEditLogsForReplicatedFile() throws Exception {
     cluster.shutdown();
 

+ 77 - 57
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptedTransfer.java

@@ -17,10 +17,10 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.Mockito.times;
 
 import java.io.IOException;
@@ -33,6 +33,9 @@ import java.util.List;
 import java.util.concurrent.TimeoutException;
 
 import java.util.function.Supplier;
+
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -53,19 +56,10 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.Assert;
-import org.junit.rules.Timeout;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
+import org.junit.jupiter.api.AfterEach;
 import org.mockito.Mockito;
 import org.slf4j.event.Level;
 
-@RunWith(Parameterized.class)
 public class TestEncryptedTransfer {
   {
     GenericTestUtils.setLogLevel(
@@ -73,11 +67,7 @@ public class TestEncryptedTransfer {
     GenericTestUtils.setLogLevel(
         LoggerFactory.getLogger(DataTransferSaslUtil.class), Level.DEBUG);
   }
-
-  @Rule
-  public Timeout timeout = new Timeout(300000);
   
-  @Parameters
   public static Collection<Object[]> data() {
     Collection<Object[]> params = new ArrayList<Object[]>();
     params.add(new Object[]{null});
@@ -114,16 +104,16 @@ public class TestEncryptedTransfer {
   }
   
   String resolverClazz;
-  public TestEncryptedTransfer(String resolverClazz){
-    this.resolverClazz = resolverClazz;
+  public void initTestEncryptedTransfer(String pResolverClazz) throws IOException {
+    this.resolverClazz = pResolverClazz;
+    setup();
   }
 
-  @Before
   public void setup() throws IOException {
     conf = new Configuration();
   }
 
-  @After
+  @AfterEach
   public void teardown() throws IOException {
     if (fs != null) {
       fs.close();
@@ -206,29 +196,39 @@ public class TestEncryptedTransfer {
     }
   }
 
-  @Test
-  public void testEncryptedReadDefaultAlgorithmCipherSuite()
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testEncryptedReadDefaultAlgorithmCipherSuite(String pResolverClazz)
       throws IOException {
+    initTestEncryptedTransfer(pResolverClazz);
     testEncryptedRead("", "", false, false);
   }
 
-  @Test
-  public void testEncryptedReadWithRC4() throws IOException {
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testEncryptedReadWithRC4(String pResolverClazz) throws IOException {
+    initTestEncryptedTransfer(pResolverClazz);
     testEncryptedRead("rc4", "", false, false);
   }
 
-  @Test
-  public void testEncryptedReadWithAES() throws IOException {
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testEncryptedReadWithAES(String pResolverClazz) throws IOException {
+    initTestEncryptedTransfer(pResolverClazz);
     testEncryptedRead("", "AES/CTR/NoPadding", true, false);
   }
 
-  @Test
-  public void testEncryptedReadAfterNameNodeRestart() throws IOException {
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testEncryptedReadAfterNameNodeRestart(String pResolverClazz) throws IOException {
+    initTestEncryptedTransfer(pResolverClazz);
     testEncryptedRead("", "", false, true);
   }
 
-  @Test
-  public void testClientThatDoesNotSupportEncryption() throws IOException {
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testClientThatDoesNotSupportEncryption(String pResolverClazz) throws IOException {
+    initTestEncryptedTransfer(pResolverClazz);
     // Set short retry timeouts so this test runs faster
     conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 10);
 
@@ -260,8 +260,10 @@ public class TestEncryptedTransfer {
     }
   }
 
-  @Test
-  public void testLongLivedReadClientAfterRestart() throws IOException {
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testLongLivedReadClientAfterRestart(String pResolverClazz) throws IOException {
+    initTestEncryptedTransfer(pResolverClazz);
     FileChecksum checksum = writeUnencryptedAndThenRestartEncryptedCluster();
 
     assertEquals(PLAIN_TEXT, DFSTestUtil.readFile(fs, TEST_PATH));
@@ -276,8 +278,10 @@ public class TestEncryptedTransfer {
     assertEquals(checksum, fs.getFileChecksum(TEST_PATH));
   }
 
-  @Test
-  public void testLongLivedWriteClientAfterRestart() throws IOException {
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testLongLivedWriteClientAfterRestart(String pResolverClazz) throws IOException {
+    initTestEncryptedTransfer(pResolverClazz);
     setEncryptionConfigKeys();
     cluster = new MiniDFSCluster.Builder(conf).build();
 
@@ -295,9 +299,11 @@ public class TestEncryptedTransfer {
     writeTestDataToFile(fs);
     assertEquals(PLAIN_TEXT + PLAIN_TEXT, DFSTestUtil.readFile(fs, TEST_PATH));
   }
-  
-  @Test
-  public void testLongLivedClient() throws IOException, InterruptedException {
+
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testLongLivedClient(String pResolverClazz) throws IOException, InterruptedException {
+    initTestEncryptedTransfer(pResolverClazz);
     FileChecksum checksum = writeUnencryptedAndThenRestartEncryptedCluster();
 
     BlockTokenSecretManager btsm = cluster.getNamesystem().getBlockManager()
@@ -321,9 +327,11 @@ public class TestEncryptedTransfer {
     assertEquals(checksum, fs.getFileChecksum(TEST_PATH));
   }
 
-  @Test
-  public void testFileChecksumWithInvalidEncryptionKey()
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testFileChecksumWithInvalidEncryptionKey(String pResolverClazz)
       throws IOException, InterruptedException, TimeoutException {
+    initTestEncryptedTransfer(pResolverClazz);
     if (resolverClazz != null) {
       // TestTrustedChannelResolver does not use encryption keys.
       return;
@@ -365,16 +373,18 @@ public class TestEncryptedTransfer {
     LOG.info("The encryption key is invalid on all nodes now.");
     fs.getFileChecksum(TEST_PATH);
     // verify that InvalidEncryptionKeyException is handled properly
-    Assert.assertTrue(client.getEncryptionKey() == null);
+    assertTrue(client.getEncryptionKey() == null);
     Mockito.verify(spyClient, times(1)).clearDataEncryptionKey();
     // Retry the operation after clearing the encryption key
     FileChecksum verifyChecksum = fs.getFileChecksum(TEST_PATH);
-    Assert.assertEquals(checksum, verifyChecksum);
+    assertEquals(checksum, verifyChecksum);
   }
 
-  @Test
-  public void testLongLivedClientPipelineRecovery()
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testLongLivedClientPipelineRecovery(String pResolverClazz)
       throws IOException, InterruptedException, TimeoutException {
+    initTestEncryptedTransfer(pResolverClazz);
     if (resolverClazz != null) {
       // TestTrustedChannelResolver does not use encryption keys.
       return;
@@ -428,25 +438,31 @@ public class TestEncryptedTransfer {
       // write data to induce pipeline recovery
       out.write(PLAIN_TEXT.getBytes());
       out.hflush();
-      assertFalse("The first datanode in the pipeline was not replaced.",
-          Arrays.asList(dfstream.getPipeline()).contains(targets[0]));
+      assertFalse(Arrays.asList(dfstream.getPipeline()).contains(targets[0]),
+          "The first datanode in the pipeline was not replaced.");
     }
     // verify that InvalidEncryptionKeyException is handled properly
     Mockito.verify(spyClient, times(1)).clearDataEncryptionKey();
   }
 
-  @Test
-  public void testEncryptedWriteWithOneDn() throws IOException {
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testEncryptedWriteWithOneDn(String pResolverClazz) throws IOException {
+    initTestEncryptedTransfer(pResolverClazz);
     testEncryptedWrite(1);
   }
 
-  @Test
-  public void testEncryptedWriteWithTwoDns() throws IOException {
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testEncryptedWriteWithTwoDns(String pResolverClazz) throws IOException {
+    initTestEncryptedTransfer(pResolverClazz);
     testEncryptedWrite(2);
   }
 
-  @Test
-  public void testEncryptedWriteWithMultipleDns() throws IOException {
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testEncryptedWriteWithMultipleDns(String pResolverClazz) throws IOException {
+    initTestEncryptedTransfer(pResolverClazz);
     testEncryptedWrite(10);
   }
 
@@ -479,8 +495,10 @@ public class TestEncryptedTransfer {
     }
   }
 
-  @Test
-  public void testEncryptedAppend() throws IOException {
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testEncryptedAppend(String pResolverClazz) throws IOException {
+    initTestEncryptedTransfer(pResolverClazz);
     setEncryptionConfigKeys();
 
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
@@ -494,8 +512,10 @@ public class TestEncryptedTransfer {
     assertEquals(PLAIN_TEXT + PLAIN_TEXT, DFSTestUtil.readFile(fs, TEST_PATH));
   }
 
-  @Test
-  public void testEncryptedAppendRequiringBlockTransfer() throws IOException {
+  @MethodSource("data")
+  @ParameterizedTest
+  public void testEncryptedAppendRequiringBlockTransfer(String pResolverClazz) throws IOException {
+    initTestEncryptedTransfer(pResolverClazz);
     setEncryptionConfigKeys();
 
     // start up 4 DNs

+ 16 - 13
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesWithHA.java

@@ -28,15 +28,17 @@ import org.apache.hadoop.hdfs.client.CreateEncryptionZoneFlag;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
 import org.apache.hadoop.hdfs.client.HdfsAdmin;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import java.io.File;
 import java.io.IOException;
 import java.util.EnumSet;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
 /**
  * Tests interaction of encryption zones with HA failover.
  */
@@ -55,7 +57,7 @@ public class TestEncryptionZonesWithHA {
   protected static final EnumSet< CreateEncryptionZoneFlag > NO_TRASH =
       EnumSet.of(CreateEncryptionZoneFlag.NO_TRASH);
 
-  @Before
+  @BeforeEach
   public void setupCluster() throws Exception {
     conf = new Configuration();
     conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
@@ -87,7 +89,7 @@ public class TestEncryptionZonesWithHA {
     fs.getClient().setKeyProvider(nn0Provider);
   }
 
-  @After
+  @AfterEach
   public void shutdownCluster() throws IOException {
     if (cluster != null) {
       cluster.shutdown();
@@ -98,7 +100,8 @@ public class TestEncryptionZonesWithHA {
   /**
    * Test that encryption zones are properly tracked by the standby.
    */
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testEncryptionZonesTrackedOnStandby() throws Exception {
     final int len = 8196;
     final Path dir = new Path("/enc");
@@ -115,12 +118,12 @@ public class TestEncryptionZonesWithHA {
     cluster.shutdownNameNode(0);
     cluster.transitionToActive(1);
 
-    Assert.assertEquals("Got unexpected ez path", dir.toString(),
-        dfsAdmin1.getEncryptionZoneForPath(dir).getPath().toString());
-    Assert.assertEquals("Got unexpected ez path", dir.toString(),
-        dfsAdmin1.getEncryptionZoneForPath(dirChild).getPath().toString());
-    Assert.assertEquals("File contents after failover were changed",
-        contents, DFSTestUtil.readFile(fs, dirFile));
+    assertEquals(dir.toString(), dfsAdmin1.getEncryptionZoneForPath(dir).getPath().toString(),
+        "Got unexpected ez path");
+    assertEquals(dir.toString(), dfsAdmin1.getEncryptionZoneForPath(dirChild).getPath().toString(),
+        "Got unexpected ez path");
+    assertEquals(contents, DFSTestUtil.readFile(fs, dirFile),
+        "File contents after failover were changed");
   }
 
 }

+ 14 - 15
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodeBenchmarkThroughput.java

@@ -23,27 +23,26 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 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.Rule;
-import org.junit.rules.Timeout;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 
 import java.io.IOException;
+import org.junit.jupiter.api.Timeout;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
 
 /**
  * To test {@link org.apache.hadoop.hdfs.ErasureCodeBenchmarkThroughput}.
  */
+@Timeout(300)
 public class TestErasureCodeBenchmarkThroughput {
   private static MiniDFSCluster cluster;
   private static Configuration conf;
   private static FileSystem fs;
 
-  @Rule
-  public Timeout globalTimeout = new Timeout(300000);
-
-  @BeforeClass
+  @BeforeAll
   public static void setup() throws IOException {
     conf = new HdfsConfiguration();
     int numDN = ErasureCodeBenchmarkThroughput.getEcPolicy().getNumDataUnits() +
@@ -55,7 +54,7 @@ public class TestErasureCodeBenchmarkThroughput {
         ErasureCodeBenchmarkThroughput.getEcPolicy().getName());
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() {
     if (cluster != null) {
       cluster.shutdown(true);
@@ -63,9 +62,9 @@ public class TestErasureCodeBenchmarkThroughput {
   }
 
   private static void runBenchmark(String[] args) throws Exception {
-    Assert.assertNotNull(conf);
-    Assert.assertNotNull(fs);
-    Assert.assertEquals(0, ToolRunner.run(conf,
+    assertNotNull(conf);
+    assertNotNull(fs);
+    assertEquals(0, ToolRunner.run(conf,
         new ErasureCodeBenchmarkThroughput(fs), args));
   }
 
@@ -80,7 +79,7 @@ public class TestErasureCodeBenchmarkThroughput {
             ErasureCodeBenchmarkThroughput.getFilePath(dataSize, isEc));
       }
     });
-    Assert.assertEquals(numFile, statuses.length);
+    assertEquals(numFile, statuses.length);
   }
 
   @Test

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingAddConfig.java

@@ -17,10 +17,10 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.IOException;
 
@@ -28,7 +28,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.io.erasurecode.ECSchema;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 /**
  * Test that ensures addition of user defined EC policies is allowed only when

+ 13 - 10
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingExerciseAPIs.java

@@ -33,9 +33,9 @@ import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Lists;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -58,7 +58,12 @@ import static org.apache.hadoop.fs.permission.FsAction.ALL;
 import static org.apache.hadoop.fs.permission.FsAction.NONE;
 import static org.apache.hadoop.fs.permission.FsAction.READ_EXECUTE;
 import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.aclEntry;
-import static org.junit.Assert.*;
+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.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 /**
  * Test after enable Erasure Coding on cluster, exercise Java API make sure they
@@ -82,7 +87,7 @@ public class TestErasureCodingExerciseAPIs {
       LoggerFactory.getLogger(TestErasureCodingExerciseAPIs.class);
 
 
-  @Before
+  @BeforeEach
   public void setupCluster() throws IOException {
     ecPolicy = getEcPolicy();
     conf = new HdfsConfiguration();
@@ -252,8 +257,7 @@ public class TestErasureCodingExerciseAPIs {
     AclStatus as = fs.getAclStatus(p);
 
     for (AclEntry entry : aclSpec) {
-      assertTrue(String.format("as: %s, entry: %s", as, entry),
-          as.getEntries().contains(entry));
+      assertTrue(as.getEntries().contains(entry), String.format("as: %s, entry: %s", as, entry));
     }
     List<AclEntry> maclSpec = Lists.newArrayList(
         aclEntry(ACCESS, USER, "bar", READ_EXECUTE),
@@ -262,8 +266,7 @@ public class TestErasureCodingExerciseAPIs {
 
     as = fs.getAclStatus(p);
     for (AclEntry entry : maclSpec) {
-      assertTrue(String.format("as: %s, entry: %s", as, entry),
-          as.getEntries().contains(entry));
+      assertTrue(as.getEntries().contains(entry), String.format("as: %s, entry: %s", as, entry));
     }
 
     fs.removeAclEntries(p, maclSpec);
@@ -539,7 +542,7 @@ public class TestErasureCodingExerciseAPIs {
     }
   }
 
-  @After
+  @AfterEach
   public void shutdownCluster() {
     if (cluster != null) {
       cluster.shutdown();

+ 9 - 12
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingMultipleRacks.java

@@ -27,11 +27,9 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFaultTolerant;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.event.Level;
@@ -40,12 +38,14 @@ import java.util.HashMap;
 import java.util.Map;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import org.junit.jupiter.api.Timeout;
 
 /**
  * Test erasure coding block placement with skewed # nodes per rack.
  */
+@Timeout(300)
 public class TestErasureCodingMultipleRacks {
   public static final Logger LOG =
       LoggerFactory.getLogger(TestErasureCodingMultipleRacks.class);
@@ -58,9 +58,6 @@ public class TestErasureCodingMultipleRacks {
     GenericTestUtils.setLogLevel(NetworkTopology.LOG, Level.DEBUG);
   }
 
-  @Rule
-  public Timeout globalTimeout = new Timeout(300000);
-
   public ErasureCodingPolicy getPolicy() {
     return StripedFileTestUtil.getDefaultECPolicy();
   }
@@ -70,7 +67,7 @@ public class TestErasureCodingMultipleRacks {
   private Configuration conf;
   private DistributedFileSystem dfs;
 
-  @Before
+  @BeforeEach
   public void setup() {
     ecPolicy = getPolicy();
     conf = new HdfsConfiguration();
@@ -98,7 +95,7 @@ public class TestErasureCodingMultipleRacks {
     dfs.setErasureCodingPolicy(new Path("/"), ecPolicy.getName());
   }
 
-  @After
+  @AfterEach
   public void teardown() throws Exception {
     if (cluster != null) {
       cluster.shutdown();

+ 84 - 96
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java

@@ -41,12 +41,9 @@ import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 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.Timeout;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -58,8 +55,15 @@ import java.util.List;
 import java.util.Map;
 
 import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
-import static org.junit.Assert.*;
-
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+import org.junit.jupiter.api.Timeout;
+
+@Timeout(60)
 public class TestErasureCodingPolicies {
   private Configuration conf;
   private MiniDFSCluster cluster;
@@ -72,10 +76,7 @@ public class TestErasureCodingPolicies {
     return StripedFileTestUtil.getDefaultECPolicy();
   }
 
-  @Rule
-  public Timeout timeout = new Timeout(60 * 1000);
-
-  @Before
+  @BeforeEach
   public void setupCluster() throws IOException {
     ecPolicy = getEcPolicy();
     conf = new HdfsConfiguration();
@@ -89,7 +90,7 @@ public class TestErasureCodingPolicies {
     DFSTestUtil.enableAllECPolicies(fs);
   }
 
-  @After
+  @AfterEach
   public void shutdownCluster() throws IOException {
     if (cluster != null) {
       cluster.shutdown();
@@ -164,7 +165,7 @@ public class TestErasureCodingPolicies {
     final Path subdir = new Path("/ec/sub");
     fs.mkdir(subdir, FsPermission.getDirDefault());
     ContentSummary contentSummary = fs.getContentSummary(subdir);
-    assertEquals(ecPolicy.getName(),contentSummary.getErasureCodingPolicy());
+    assertEquals(ecPolicy.getName(), contentSummary.getErasureCodingPolicy());
   }
 
   @Test
@@ -227,13 +228,11 @@ public class TestErasureCodingPolicies {
 
     // Already set directory-level policies should still be in effect
     Path disabledPolicy = new Path(dir1, "afterDisabled");
-    Assert.assertEquals("Dir does not have policy set",
-        ecPolicy,
-        fs.getErasureCodingPolicy(dir1));
+    assertEquals(ecPolicy, fs.getErasureCodingPolicy(dir1),
+        "Dir does not have policy set");
     fs.create(disabledPolicy).close();
-    Assert.assertEquals("File did not inherit dir's policy",
-        ecPolicy,
-        fs.getErasureCodingPolicy(disabledPolicy));
+    assertEquals(ecPolicy, fs.getErasureCodingPolicy(disabledPolicy),
+        "File did not inherit dir's policy");
 
     // Also check loading disabled EC policies from fsimage
     fs.setSafeMode(SafeModeAction.ENTER);
@@ -241,12 +240,10 @@ public class TestErasureCodingPolicies {
     fs.setSafeMode(SafeModeAction.LEAVE);
     cluster.restartNameNodes();
 
-    Assert.assertEquals("Dir does not have policy set",
-        ecPolicy,
-        fs.getErasureCodingPolicy(dir1));
-    Assert.assertEquals("File does not have policy set",
-        ecPolicy,
-        fs.getErasureCodingPolicy(disabledPolicy));
+    assertEquals(ecPolicy, fs.getErasureCodingPolicy(dir1),
+        "Dir does not have policy set");
+    assertEquals(ecPolicy, fs.getErasureCodingPolicy(disabledPolicy),
+        "File does not have policy set");
   }
 
   @Test
@@ -325,19 +322,19 @@ public class TestErasureCodingPolicies {
     final Path reserveDir = new Path("/.reserved");
     // verify the EC policy is null, not an exception
     ErasureCodingPolicy policy = fs.getErasureCodingPolicy(reserveDir);
-    assertNull("Got unexpected erasure coding policy", policy);
+    assertNull(policy, "Got unexpected erasure coding policy");
 
     // root EC policy before being set is null, verify the reserved raw dir
     // is treated as root
     final Path root = new Path("/");
     final Path rawRoot = new Path("/.reserved/raw");
     final Path rawRootSlash = new Path("/.reserved/raw/");
-    assertNull("Got unexpected erasure coding policy",
-        fs.getErasureCodingPolicy(root));
-    assertNull("Got unexpected erasure coding policy",
-        fs.getErasureCodingPolicy(rawRoot));
-    assertNull("Got unexpected erasure coding policy",
-        fs.getErasureCodingPolicy(rawRootSlash));
+    assertNull(fs.getErasureCodingPolicy(root),
+        "Got unexpected erasure coding policy");
+    assertNull(fs.getErasureCodingPolicy(rawRoot),
+        "Got unexpected erasure coding policy");
+    assertNull(fs.getErasureCodingPolicy(rawRootSlash),
+        "Got unexpected erasure coding policy");
 
     // verify the EC policy correctness under the reserved raw dir
     final Path ecDir = new Path("/ec");
@@ -345,21 +342,18 @@ public class TestErasureCodingPolicies {
     fs.setErasureCodingPolicy(ecDir, ecPolicy.getName());
 
     ErasureCodingPolicy policyBase = fs.getErasureCodingPolicy(ecDir);
-    assertEquals("Got unexpected erasure coding policy", ecPolicy,
-        policyBase);
+    assertEquals(ecPolicy, policyBase, "Got unexpected erasure coding policy");
 
     final Path rawRootEc = new Path("/.reserved/raw/ec");
     ErasureCodingPolicy policyMap = fs.getErasureCodingPolicy(rawRootEc);
-    assertEquals("Got unexpected erasure coding policy", ecPolicy,
-        policyMap);
+    assertEquals(ecPolicy, policyMap, "Got unexpected erasure coding policy");
   }
 
   @Test
   public void testGetErasureCodingPolicy() throws Exception {
     List<ErasureCodingPolicy> sysECPolicies =
         SystemErasureCodingPolicies.getPolicies();
-    assertTrue("System ecPolicies should exist",
-        sysECPolicies.size() > 0);
+    assertTrue(sysECPolicies.size() > 0, "System ecPolicies should exist");
 
     ErasureCodingPolicy usingECPolicy = sysECPolicies.get(0);
     String src = "/ec2";
@@ -380,8 +374,8 @@ public class TestErasureCodingPolicies {
     HdfsFileStatus hdfsFileStatus = fs.getClient().getFileInfo(src);
     ErasureCodingPolicy actualPolicy = hdfsFileStatus.getErasureCodingPolicy();
     assertNotNull(actualPolicy);
-    assertEquals("Actually used ecPolicy should be equal with target ecPolicy",
-        usingECPolicy, actualPolicy);
+    assertEquals(usingECPolicy, actualPolicy,
+        "Actually used ecPolicy should be equal with target ecPolicy");
   }
 
   @Test
@@ -434,7 +428,7 @@ public class TestErasureCodingPolicies {
         sysPolicies.remove(ecpi.getPolicy());
       }
     }
-    assertTrue("All system policies should be enabled", sysPolicies.isEmpty());
+    assertTrue(sysPolicies.isEmpty(), "All system policies should be enabled");
 
     // Query after add a new policy
     ECSchema toAddSchema = new ECSchema("rs", 5, 2);
@@ -443,9 +437,8 @@ public class TestErasureCodingPolicies {
     ErasureCodingPolicy[] policyArray = new ErasureCodingPolicy[]{newPolicy};
     fs.addErasureCodingPolicies(policyArray);
     allECPolicies = fs.getAllErasureCodingPolicies();
-    assertEquals("Should return new added policy",
-        SystemErasureCodingPolicies.getPolicies().size() + 1,
-        allECPolicies.size());
+    assertEquals(SystemErasureCodingPolicies.getPolicies().size() + 1,
+        allECPolicies.size(), "Should return new added policy");
 
   }
 
@@ -515,13 +508,11 @@ public class TestErasureCodingPolicies {
     userfs.mkdirs(ecdir);
     final String ecPolicyName = ecPolicy.getName();
     useradmin.setErasureCodingPolicy(ecdir, ecPolicyName);
-    assertEquals("Policy not present on dir",
-        ecPolicyName,
-        useradmin.getErasureCodingPolicy(ecdir).getName());
+    assertEquals(ecPolicyName, useradmin.getErasureCodingPolicy(ecdir).getName(),
+        "Policy not present on dir");
     userfs.create(ecfile).close();
-    assertEquals("Policy not present on file",
-        ecPolicyName,
-        useradmin.getErasureCodingPolicy(ecfile).getName());
+    assertEquals(ecPolicyName, useradmin.getErasureCodingPolicy(ecfile).getName(),
+        "Policy not present on file");
 
     // Unset and re-set
     useradmin.unsetErasureCodingPolicy(ecdir);
@@ -631,7 +622,7 @@ public class TestErasureCodingPolicies {
     final String illegalPolicyName = "RS-DEFAULT-1-2-64k";
     try {
       fs.createFile(filePath1).ecPolicyName(illegalPolicyName).build().close();
-      Assert.fail("illegal erasure coding policy should not be found");
+      fail("illegal erasure coding policy should not be found");
     } catch (Exception e) {
       GenericTestUtils.assertExceptionContains("Policy '" + illegalPolicyName
           + "' does not match any enabled erasure coding policies", e);
@@ -691,7 +682,7 @@ public class TestErasureCodingPolicies {
           .ecPolicyName(ecPolicyName)
           .replicate()
           .build().close();
-      Assert.fail("shouldReplicate and ecPolicyName are exclusive " +
+      fail("shouldReplicate and ecPolicyName are exclusive " +
           "parameters. Set both is not allowed.");
     }catch (Exception e){
       GenericTestUtils.assertExceptionContains("SHOULD_REPLICATE flag and " +
@@ -704,7 +695,7 @@ public class TestErasureCodingPolicies {
           EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE,
               CreateFlag.SHOULD_REPLICATE), false, (short) 1, 1024, null, 1024,
           null, null, ecPolicyName);
-      Assert.fail("SHOULD_REPLICATE flag and ecPolicyName are exclusive " +
+      fail("SHOULD_REPLICATE flag and ecPolicyName are exclusive " +
           "parameters. Set both is not allowed.");
     }catch (Exception e){
       GenericTestUtils.assertExceptionContains("SHOULD_REPLICATE flag and " +
@@ -723,8 +714,8 @@ public class TestErasureCodingPolicies {
   public void testGetAllErasureCodingCodecs() throws Exception {
     Map<String, String> allECCodecs = fs
         .getAllErasureCodingCodecs();
-    assertTrue("At least 3 system codecs should be enabled",
-        allECCodecs.size() >= 3);
+    assertTrue(allECCodecs.size() >= 3,
+        "At least 3 system codecs should be enabled");
     System.out.println("Erasure Coding Codecs: Codec [Coder List]");
     for (String codec : allECCodecs.keySet()) {
       String coders = allECCodecs.get(codec);
@@ -771,7 +762,7 @@ public class TestErasureCodingPolicies {
     for (int cellSize: cellSizes) {
       try {
         new ErasureCodingPolicy(toAddSchema, cellSize);
-        Assert.fail("Invalid cell size should be detected.");
+        fail("Invalid cell size should be detected.");
       } catch (Exception e){
         GenericTestUtils.assertExceptionContains("cellSize must be", e);
       }
@@ -860,67 +851,64 @@ public class TestErasureCodingPolicies {
     // 1. At first, child directory will inherit parent's EC policy
     fs.mkdirs(replicaDir);
     fs.createFile(replicaFile).build().close();
-    HdfsFileStatus fileStatus = (HdfsFileStatus)fs.getFileStatus(replicaFile);
-    assertEquals("File should inherit EC policy.", ecPolicy, fileStatus
-        .getErasureCodingPolicy());
-    assertEquals("File should be a EC file.", true, fileStatus
-        .isErasureCoded());
-    assertEquals("File should have the same EC policy as its ancestor.",
-        ecPolicy, fs.getErasureCodingPolicy(replicaFile));
+    HdfsFileStatus fileStatus = (HdfsFileStatus) fs.getFileStatus(replicaFile);
+    assertEquals(ecPolicy, fileStatus
+        .getErasureCodingPolicy(), "File should inherit EC policy.");
+    assertEquals(true, fileStatus
+        .isErasureCoded(), "File should be a EC file.");
+    assertEquals(ecPolicy, fs.getErasureCodingPolicy(replicaFile),
+        "File should have the same EC policy as its ancestor.");
     fs.delete(replicaFile, false);
 
     // 2. Set replication policy on child directory, then get back the policy
     fs.setErasureCodingPolicy(replicaDir, replicaPolicy.getName());
     ErasureCodingPolicy temp = fs.getErasureCodingPolicy(replicaDir);
-    assertEquals("Directory should hide replication EC policy.",
-        null, temp);
+    assertEquals(null, temp, "Directory should hide replication EC policy.");
 
     // 3. New file will be replication file. Please be noted that replication
     //    policy only set on directory, not on file
     fs.createFile(replicaFile).build().close();
-    assertEquals("Replication file should have default replication factor.",
-        fs.getDefaultReplication(),
-        fs.getFileStatus(replicaFile).getReplication());
+    assertEquals(fs.getDefaultReplication(), fs.getFileStatus(replicaFile).getReplication(),
+        "Replication file should have default replication factor.");
     fs.setReplication(replicaFile, (short) 2);
-    assertEquals("File should have replication factor as expected.",
-        2, fs.getFileStatus(replicaFile).getReplication());
-    fileStatus = (HdfsFileStatus)fs.getFileStatus(replicaFile);
-    assertEquals("File should not have EC policy.", null, fileStatus
-        .getErasureCodingPolicy());
-    assertEquals("File should not be a EC file.", false,
-        fileStatus.isErasureCoded());
+    assertEquals(2, fs.getFileStatus(replicaFile).getReplication(),
+        "File should have replication factor as expected.");
+    fileStatus = (HdfsFileStatus) fs.getFileStatus(replicaFile);
+    assertEquals(null, fileStatus
+        .getErasureCodingPolicy(), "File should not have EC policy.");
+    assertEquals(false, fileStatus.isErasureCoded(),
+        "File should not be a EC file.");
     ErasureCodingPolicy ecPolicyOnFile = fs.getErasureCodingPolicy(replicaFile);
-    assertEquals("File should not have EC policy.", null, ecPolicyOnFile);
+    assertEquals(null, ecPolicyOnFile, "File should not have EC policy.");
     fs.delete(replicaFile, false);
 
     // 4. New directory under replication directory, is also replication
     // directory
     fs.mkdirs(subReplicaDir);
-    assertEquals("Directory should inherit hiding replication EC policy.",
-        null, fs.getErasureCodingPolicy(subReplicaDir));
+    assertEquals(null, fs.getErasureCodingPolicy(subReplicaDir),
+        "Directory should inherit hiding replication EC policy.");
     fs.createFile(subReplicaFile).build().close();
-    assertEquals("File should have default replication factor.",
-        fs.getDefaultReplication(),
-        fs.getFileStatus(subReplicaFile).getReplication());
-    fileStatus = (HdfsFileStatus)fs.getFileStatus(subReplicaFile);
-    assertEquals("File should not have EC policy.", null,
-        fileStatus.getErasureCodingPolicy());
-    assertEquals("File should not be a EC file.", false,
-        fileStatus.isErasureCoded());
-    assertEquals("File should not have EC policy.", null,
-        fs.getErasureCodingPolicy(subReplicaFile));
+    assertEquals(fs.getDefaultReplication(), fs.getFileStatus(subReplicaFile).getReplication(),
+        "File should have default replication factor.");
+    fileStatus = (HdfsFileStatus) fs.getFileStatus(subReplicaFile);
+    assertEquals(null, fileStatus.getErasureCodingPolicy(),
+        "File should not have EC policy.");
+    assertEquals(false, fileStatus.isErasureCoded(),
+        "File should not be a EC file.");
+    assertEquals(null, fs.getErasureCodingPolicy(subReplicaFile),
+        "File should not have EC policy.");
     fs.delete(subReplicaFile, false);
 
     // 5. Unset replication policy on directory, new file will be EC file
     fs.unsetErasureCodingPolicy(replicaDir);
     fs.createFile(subReplicaFile).build().close();
-    fileStatus = (HdfsFileStatus)fs.getFileStatus(subReplicaFile);
-    assertEquals("File should inherit EC policy.", ecPolicy,
-        fileStatus.getErasureCodingPolicy());
-    assertEquals("File should be a EC file.", true,
-        fileStatus.isErasureCoded());
-    assertEquals("File should have the same EC policy as its ancestor",
-        ecPolicy, fs.getErasureCodingPolicy(subReplicaFile));
+    fileStatus = (HdfsFileStatus) fs.getFileStatus(subReplicaFile);
+    assertEquals(ecPolicy, fileStatus.getErasureCodingPolicy(),
+        "File should inherit EC policy.");
+    assertEquals(true, fileStatus.isErasureCoded(),
+        "File should be a EC file.");
+    assertEquals(ecPolicy, fs.getErasureCodingPolicy(subReplicaFile),
+        "File should have the same EC policy as its ancestor");
     fs.delete(subReplicaFile, false);
   }
 

+ 50 - 61
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicyWithSnapshot.java

@@ -17,8 +17,8 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
 
 import java.io.IOException;
 
@@ -31,12 +31,12 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.util.ToolRunner;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.Rule;
-import org.junit.rules.Timeout;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
+@Timeout(120)
 public class TestErasureCodingPolicyWithSnapshot {
   private MiniDFSCluster cluster;
   private DistributedFileSystem fs;
@@ -50,10 +50,7 @@ public class TestErasureCodingPolicyWithSnapshot {
     return StripedFileTestUtil.getDefaultECPolicy();
   }
 
-  @Rule
-  public Timeout globalTimeout = new Timeout(120000);
-
-  @Before
+  @BeforeEach
   public void setupCluster() throws IOException {
     ecPolicy = getEcPolicy();
     groupSize = (short) (ecPolicy.getNumDataUnits()
@@ -65,7 +62,7 @@ public class TestErasureCodingPolicyWithSnapshot {
     fs.enableErasureCodingPolicy(ecPolicy.getName());
   }
 
-  @After
+  @AfterEach
   public void shutdownCluster() throws IOException {
     if (cluster != null) {
       cluster.shutdown();
@@ -91,8 +88,8 @@ public class TestErasureCodingPolicyWithSnapshot {
     String contents = DFSTestUtil.readFile(fs, ecFile);
     final Path snap1 = fs.createSnapshot(ecDirParent, "snap1");
     final Path snap1ECDir = new Path(snap1, ecDir.getName());
-    assertEquals("Got unexpected erasure coding policy", ecPolicy,
-        fs.getErasureCodingPolicy(snap1ECDir));
+    assertEquals(ecPolicy, fs.getErasureCodingPolicy(snap1ECDir),
+        "Got unexpected erasure coding policy");
 
     // Now delete the dir which has erasure coding policy. Re-create the dir again, and
     // take another snapshot
@@ -100,8 +97,8 @@ public class TestErasureCodingPolicyWithSnapshot {
     fs.mkdir(ecDir, FsPermission.getDirDefault());
     final Path snap2 = fs.createSnapshot(ecDirParent, "snap2");
     final Path snap2ECDir = new Path(snap2, ecDir.getName());
-    assertNull("Expected null erasure coding policy",
-        fs.getErasureCodingPolicy(snap2ECDir));
+    assertNull(fs.getErasureCodingPolicy(snap2ECDir),
+        "Expected null erasure coding policy");
 
     // Make dir again with system default ec policy
     fs.setErasureCodingPolicy(ecDir, ecPolicy.getName());
@@ -109,31 +106,30 @@ public class TestErasureCodingPolicyWithSnapshot {
     final Path snap3ECDir = new Path(snap3, ecDir.getName());
     // Check that snap3's ECPolicy has the correct settings
     ErasureCodingPolicy ezSnap3 = fs.getErasureCodingPolicy(snap3ECDir);
-    assertEquals("Got unexpected erasure coding policy", ecPolicy,
-        ezSnap3);
+    assertEquals(ecPolicy, ezSnap3, "Got unexpected erasure coding policy");
 
     // Check that older snapshots still have the old ECPolicy settings
-    assertEquals("Got unexpected erasure coding policy", ecPolicy,
-        fs.getErasureCodingPolicy(snap1ECDir));
-    assertNull("Expected null erasure coding policy",
-        fs.getErasureCodingPolicy(snap2ECDir));
+    assertEquals(ecPolicy, fs.getErasureCodingPolicy(snap1ECDir),
+        "Got unexpected erasure coding policy");
+    assertNull(fs.getErasureCodingPolicy(snap2ECDir),
+        "Expected null erasure coding policy");
 
     // Verify contents of the snapshotted file
     final Path snapshottedECFile = new Path(snap1.toString() + "/"
         + ecDir.getName() + "/" + ecFile.getName());
-    assertEquals("Contents of snapshotted file have changed unexpectedly",
-        contents, DFSTestUtil.readFile(fs, snapshottedECFile));
+    assertEquals(contents, DFSTestUtil.readFile(fs, snapshottedECFile),
+        "Contents of snapshotted file have changed unexpectedly");
 
     // Now delete the snapshots out of order and verify the EC policy
     // correctness
     fs.deleteSnapshot(ecDirParent, snap2.getName());
-    assertEquals("Got unexpected erasure coding policy", ecPolicy,
-        fs.getErasureCodingPolicy(snap1ECDir));
-    assertEquals("Got unexpected erasure coding policy", ecPolicy,
-        fs.getErasureCodingPolicy(snap3ECDir));
+    assertEquals(ecPolicy, fs.getErasureCodingPolicy(snap1ECDir),
+        "Got unexpected erasure coding policy");
+    assertEquals(ecPolicy, fs.getErasureCodingPolicy(snap3ECDir),
+        "Got unexpected erasure coding policy");
     fs.deleteSnapshot(ecDirParent, snap1.getName());
-    assertEquals("Got unexpected erasure coding policy", ecPolicy,
-        fs.getErasureCodingPolicy(snap3ECDir));
+    assertEquals(ecPolicy, fs.getErasureCodingPolicy(snap3ECDir),
+        "Got unexpected erasure coding policy");
   }
 
   /**
@@ -147,8 +143,8 @@ public class TestErasureCodingPolicyWithSnapshot {
 
     fs.setErasureCodingPolicy(ecDir, ecPolicy.getName());
     final Path snap1 = fs.createSnapshot(ecDir, "snap1");
-    assertEquals("Got unexpected erasure coding policy", ecPolicy,
-        fs.getErasureCodingPolicy(snap1));
+    assertEquals(ecPolicy, fs.getErasureCodingPolicy(snap1),
+        "Got unexpected erasure coding policy");
   }
 
   /**
@@ -164,8 +160,7 @@ public class TestErasureCodingPolicyWithSnapshot {
     fs.setErasureCodingPolicy(ecDir, ecPolicy.getName());
     final Path snap1 = fs.createSnapshot(ecDir, "snap1");
     ErasureCodingPolicy ecSnap = fs.getErasureCodingPolicy(snap1);
-    assertEquals("Got unexpected erasure coding policy", ecPolicy,
-        ecSnap);
+    assertEquals(ecPolicy, ecSnap, "Got unexpected erasure coding policy");
 
     // save namespace, restart namenode, and check ec policy correctness.
     fs.setSafeMode(SafeModeAction.ENTER);
@@ -174,10 +169,8 @@ public class TestErasureCodingPolicyWithSnapshot {
     cluster.restartNameNode(true);
 
     ErasureCodingPolicy ecSnap1 = fs.getErasureCodingPolicy(snap1);
-    assertEquals("Got unexpected erasure coding policy", ecPolicy,
-        ecSnap1);
-    assertEquals("Got unexpected ecSchema", ecSnap.getSchema(),
-        ecSnap1.getSchema());
+    assertEquals(ecPolicy, ecSnap1, "Got unexpected erasure coding policy");
+    assertEquals(ecSnap.getSchema(), ecSnap1.getSchema(), "Got unexpected ecSchema");
   }
 
   /**
@@ -202,15 +195,16 @@ public class TestErasureCodingPolicyWithSnapshot {
     String[] argv = new String[] { "-cp", "-px", snap1.toUri().toString(),
         snap1Copy.toUri().toString() };
     int ret = ToolRunner.run(new FsShell(conf), argv);
-    assertEquals("cp -px is not working on a snapshot", SUCCESS, ret);
+    assertEquals(SUCCESS, ret, "cp -px is not working on a snapshot");
 
-    assertNull("Got unexpected erasure coding policy",
-        fs.getErasureCodingPolicy(snap1CopyECDir));
-    assertEquals("Got unexpected erasure coding policy", ecPolicy,
-        fs.getErasureCodingPolicy(snap1));
+    assertNull(fs.getErasureCodingPolicy(snap1CopyECDir),
+        "Got unexpected erasure coding policy");
+    assertEquals(ecPolicy, fs.getErasureCodingPolicy(snap1),
+        "Got unexpected erasure coding policy");
   }
 
-  @Test (timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testFileStatusAcrossNNRestart() throws IOException {
     final int len = 1024;
     final Path normalFile = new Path("/", "normalFile");
@@ -247,13 +241,12 @@ public class TestErasureCodingPolicyWithSnapshot {
 
     // verify the EC policy correctness
     ErasureCodingPolicy ecSnap = fs.getErasureCodingPolicy(snap);
-    assertEquals("Got unexpected erasure coding policy", ecPolicy,
-        ecSnap);
+    assertEquals(ecPolicy, ecSnap, "Got unexpected erasure coding policy");
 
     // verify the EC policy is null, not an exception
     final Path ecDotSnapshotDir = new Path(ecDir, ".snapshot");
     ErasureCodingPolicy ecSnap1 = fs.getErasureCodingPolicy(ecDotSnapshotDir);
-    assertNull("Got unexpected erasure coding policy", ecSnap1);
+    assertNull(ecSnap1, "Got unexpected erasure coding policy");
   }
 
   /**
@@ -268,22 +261,20 @@ public class TestErasureCodingPolicyWithSnapshot {
     fs.allowSnapshot(ecDir);
 
     final Path snap1 = fs.createSnapshot(ecDir, "snap1");
-    assertNull("Expected null erasure coding policy",
-        fs.getErasureCodingPolicy(snap1));
+    assertNull(fs.getErasureCodingPolicy(snap1), "Expected null erasure coding policy");
 
     // Set erasure coding policy
     final ErasureCodingPolicy ec63Policy = SystemErasureCodingPolicies
         .getByID(SystemErasureCodingPolicies.RS_6_3_POLICY_ID);
     fs.setErasureCodingPolicy(ecDir, ec63Policy.getName());
     final Path snap2 = fs.createSnapshot(ecDir, "snap2");
-    assertEquals("Got unexpected erasure coding policy", ec63Policy,
-        fs.getErasureCodingPolicy(snap2));
+    assertEquals(ec63Policy, fs.getErasureCodingPolicy(snap2),
+        "Got unexpected erasure coding policy");
 
     // Verify the EC policy correctness after the unset operation
     fs.unsetErasureCodingPolicy(ecDir);
     final Path snap3 = fs.createSnapshot(ecDir, "snap3");
-    assertNull("Expected null erasure coding policy",
-        fs.getErasureCodingPolicy(snap3));
+    assertNull(fs.getErasureCodingPolicy(snap3), "Expected null erasure coding policy");
 
     // Change the erasure coding policy and take another snapshot
     final ErasureCodingPolicy ec32Policy = SystemErasureCodingPolicies
@@ -291,15 +282,13 @@ public class TestErasureCodingPolicyWithSnapshot {
     fs.enableErasureCodingPolicy(ec32Policy.getName());
     fs.setErasureCodingPolicy(ecDir, ec32Policy.getName());
     final Path snap4 = fs.createSnapshot(ecDir, "snap4");
-    assertEquals("Got unexpected erasure coding policy", ec32Policy,
-        fs.getErasureCodingPolicy(snap4));
+    assertEquals(ec32Policy, fs.getErasureCodingPolicy(snap4),
+        "Got unexpected erasure coding policy");
 
     // Check that older snapshot still have the old ECPolicy settings
-    assertNull("Expected null erasure coding policy",
-        fs.getErasureCodingPolicy(snap1));
-    assertEquals("Got unexpected erasure coding policy", ec63Policy,
-        fs.getErasureCodingPolicy(snap2));
-    assertNull("Expected null erasure coding policy",
-        fs.getErasureCodingPolicy(snap3));
+    assertNull(fs.getErasureCodingPolicy(snap1), "Expected null erasure coding policy");
+    assertEquals(ec63Policy, fs.getErasureCodingPolicy(snap2),
+        "Got unexpected erasure coding policy");
+    assertNull(fs.getErasureCodingPolicy(snap3), "Expected null erasure coding policy");
   }
 }

+ 8 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestExtendedAcls.java

@@ -27,9 +27,9 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Lists;
-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.IOException;
 import java.security.PrivilegedExceptionAction;
@@ -49,9 +49,9 @@ import static org.apache.hadoop.fs.permission.AclEntryScope.DEFAULT;
 import static org.apache.hadoop.fs.permission.FsAction.ALL;
 import static org.apache.hadoop.fs.permission.AclEntryType.GROUP;
 import static org.apache.hadoop.fs.permission.AclEntryType.OTHER;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 /**
  * A class for testing the behavior of HDFS directory and file ACL.
@@ -65,7 +65,7 @@ public class TestExtendedAcls {
 
   private static DistributedFileSystem hdfs;
 
-  @BeforeClass
+  @BeforeAll
   public static void setup() throws IOException {
     conf = new Configuration();
     conf.setBoolean(DFS_NAMENODE_ACLS_ENABLED_KEY, true);
@@ -76,7 +76,7 @@ public class TestExtendedAcls {
     hdfs = cluster.getFileSystem();
   }
 
-  @AfterClass
+  @AfterAll
   public static void shutdown() throws IOException {
     if (cluster != null) {
       cluster.shutdown();

+ 27 - 23
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestExternalBlockReader.java

@@ -28,8 +28,7 @@ import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.NetUtils;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.io.IOException;
 import java.nio.BufferOverflowException;
@@ -38,6 +37,11 @@ import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.UUID;
 
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
 public class TestExternalBlockReader {
   private static final Logger LOG =
           LoggerFactory.getLogger(TestExternalBlockReader.class);
@@ -63,7 +67,7 @@ public class TestExternalBlockReader {
       IOUtils.readFully(stream, buf, 0, TEST_LENGTH);
       byte expected[] = DFSTestUtil.
           calculateFileContentsFromSeed(SEED, TEST_LENGTH);
-      Assert.assertArrayEquals(expected, buf);
+      assertArrayEquals(expected, buf);
       stream.close();
     } finally {
       dfs.close();
@@ -293,36 +297,36 @@ public class TestExternalBlockReader {
       byte expected[] = DFSTestUtil.
           calculateFileContentsFromSeed(SEED, TEST_LENGTH);
       ReadStatistics stats = stream.getReadStatistics();
-      Assert.assertEquals(1024, stats.getTotalShortCircuitBytesRead());
-      Assert.assertEquals(2047, stats.getTotalLocalBytesRead());
-      Assert.assertEquals(2047, stats.getTotalBytesRead());
-      Assert.assertArrayEquals(expected, buf);
+      assertEquals(1024, stats.getTotalShortCircuitBytesRead());
+      assertEquals(2047, stats.getTotalLocalBytesRead());
+      assertEquals(2047, stats.getTotalBytesRead());
+      assertArrayEquals(expected, buf);
       stream.close();
       ExtendedBlock block = DFSTestUtil.getFirstBlock(dfs, new Path("/a"));
-      Assert.assertNotNull(block);
+      assertNotNull(block);
       LinkedList<SyntheticReplicaAccessor> accessorList = accessors.get(uuid);
-      Assert.assertNotNull(accessorList);
-      Assert.assertEquals(3, accessorList.size());
+      assertNotNull(accessorList);
+      assertEquals(3, accessorList.size());
       SyntheticReplicaAccessor accessor = accessorList.get(0);
-      Assert.assertTrue(accessor.builder.allowShortCircuit);
-      Assert.assertEquals(block.getBlockPoolId(),
+      assertTrue(accessor.builder.allowShortCircuit);
+      assertEquals(block.getBlockPoolId(),
           accessor.builder.blockPoolId);
-      Assert.assertEquals(block.getBlockId(),
+      assertEquals(block.getBlockId(),
           accessor.builder.blockId);
-      Assert.assertEquals(dfs.getClient().clientName,
+      assertEquals(dfs.getClient().clientName,
           accessor.builder.clientName);
-      Assert.assertEquals("/a", accessor.builder.fileName);
-      Assert.assertEquals(block.getGenerationStamp(),
+      assertEquals("/a", accessor.builder.fileName);
+      assertEquals(block.getGenerationStamp(),
           accessor.getGenerationStamp());
-      Assert.assertTrue(accessor.builder.verifyChecksum);
-      Assert.assertEquals(1024L, accessor.builder.visibleLength);
-      Assert.assertEquals(24L, accessor.totalRead);
-      Assert.assertEquals("", accessor.getError());
-      Assert.assertEquals(1, accessor.numCloses);
+      assertTrue(accessor.builder.verifyChecksum);
+      assertEquals(1024L, accessor.builder.visibleLength);
+      assertEquals(24L, accessor.totalRead);
+      assertEquals("", accessor.getError());
+      assertEquals(1, accessor.numCloses);
       byte[] tempBuf = new byte[5];
-      Assert.assertEquals(-1, accessor.read(TEST_LENGTH,
+      assertEquals(-1, accessor.read(TEST_LENGTH,
             tempBuf, 0, 0));
-      Assert.assertEquals(-1, accessor.read(TEST_LENGTH,
+      assertEquals(-1, accessor.read(TEST_LENGTH,
             tempBuf, 0, tempBuf.length));
       accessors.remove(uuid);
     } finally {

+ 7 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSInputChecker.java

@@ -17,10 +17,10 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.EOFException;
 import java.io.File;
@@ -40,7 +40,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.PathUtils;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 /**
  * This class tests if FSInputChecker works correctly.
@@ -71,9 +71,8 @@ public class TestFSInputChecker {
   private void checkAndEraseData(byte[] actual, int from, byte[] expected, 
       String message) throws Exception {
     for (int idx = 0; idx < actual.length; idx++) {
-      assertEquals(message+" byte "+(from+idx)+" differs. expected "+
-                        expected[from+idx]+" actual "+actual[idx],
-                        actual[idx], expected[from+idx]);
+      assertEquals(actual[idx], expected[from + idx], message + " byte " + (from + idx)
+          + " differs. expected " + expected[from + idx] + " actual " + actual[idx]);
       actual[idx] = 0;
     }
   }

+ 11 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFetchImage.java

@@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.io.File;
 import java.io.IOException;
@@ -41,10 +41,11 @@ import org.apache.hadoop.hdfs.util.MD5FileUtils;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.PathUtils;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 public class TestFetchImage {
   
@@ -58,17 +59,17 @@ public class TestFetchImage {
   private NameNode nn1 = null;
   private Configuration conf = null;
 
-  @BeforeClass
+  @BeforeAll
   public static void setupImageDir() {
     FETCHED_IMAGE_FILE.mkdirs();
   }
 
-  @AfterClass
+  @AfterAll
   public static void cleanup() {
     FileUtil.fullyDelete(FETCHED_IMAGE_FILE);
   }
 
-  @Before
+  @BeforeEach
   public void setupCluster() throws IOException, URISyntaxException {
     conf = new Configuration();
     conf.setInt(DFS_HEARTBEAT_INTERVAL_KEY, 1);
@@ -89,7 +90,8 @@ public class TestFetchImage {
    * Download a few fsimages using `hdfs dfsadmin -fetchImage ...' and verify
    * the results.
    */
-  @Test(timeout=30000)
+  @Test
+  @Timeout(value = 30)
   public void testFetchImageHA() throws Exception {
     final Path parent = new Path(
         PathUtils.getTestPath(getClass()),

+ 40 - 35
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java

@@ -17,9 +17,10 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.File;
 import java.io.FileNotFoundException;
@@ -58,8 +59,8 @@ import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Time;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 /**
  * This class tests the building blocks that are needed to
@@ -137,8 +138,8 @@ public class TestFileAppend{
 
       // Get a handle to the datanode
       DataNode[] dn = cluster.listDataNodes();
-      assertTrue("There should be only one datanode but found " + dn.length,
-                  dn.length == 1);
+      assertTrue(
+                 dn.length == 1, "There should be only one datanode but found " + dn.length);
 
       LocatedBlocks locations = client.getNamenode().getBlockLocations(
                                   file1.toString(), 0, Long.MAX_VALUE);
@@ -161,8 +162,8 @@ public class TestFileAppend{
       for (int i = 0; i < blocks.size(); i++) {
         ExtendedBlock b = blocks.get(i).getBlock();
         System.out.println("breakHardlinksIfNeeded detaching block " + b);
-        assertTrue("breakHardlinksIfNeeded(" + b + ") should have returned true",
-            FsDatasetTestUtil.breakHardlinksIfNeeded(fsd, b));
+        assertTrue(FsDatasetTestUtil.breakHardlinksIfNeeded(fsd, b),
+            "breakHardlinksIfNeeded(" + b + ") should have returned true");
       }
 
       // Since the blocks were already detached earlier, these calls should
@@ -171,8 +172,8 @@ public class TestFileAppend{
         ExtendedBlock b = blocks.get(i).getBlock();
         System.out.println("breakHardlinksIfNeeded re-attempting to " +
                 "detach block " + b);
-        assertTrue("breakHardlinksIfNeeded(" + b + ") should have returned false",
-            FsDatasetTestUtil.breakHardlinksIfNeeded(fsd, b));
+        assertTrue(FsDatasetTestUtil.breakHardlinksIfNeeded(fsd, b),
+            "breakHardlinksIfNeeded(" + b + ") should have returned false");
       }
     } finally {
       client.close();
@@ -290,20 +291,22 @@ public class TestFileAppend{
    * 
    * @throws FileNotFoundException as the result
    */
-  @Test(expected = FileNotFoundException.class)
+  @Test
   public void testFileNotFound() throws IOException {
-    Configuration conf = new HdfsConfiguration();
-    File builderBaseDir = new File(GenericTestUtils.getRandomizedTempPath());
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf, builderBaseDir)
-        .build();
-    FileSystem fs = cluster.getFileSystem();
-    try {
-      Path file1 = new Path("/nonexistingfile.dat");
-      fs.append(file1);
-    } finally {
-      fs.close();
-      cluster.shutdown();
-    }
+    assertThrows(FileNotFoundException.class, () -> {
+      Configuration conf = new HdfsConfiguration();
+      File builderBaseDir = new File(GenericTestUtils.getRandomizedTempPath());
+      MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf, builderBaseDir)
+          .build();
+      FileSystem fs = cluster.getFileSystem();
+      try {
+        Path file1 = new Path("/nonexistingfile.dat");
+        fs.append(file1);
+      } finally {
+        fs.close();
+        cluster.shutdown();
+      }
+    });
   }
 
   /** Test two consecutive appends on a file with a full block. */
@@ -335,10 +338,10 @@ public class TestFileAppend{
       
       //2nd append should get AlreadyBeingCreatedException
       fs1.append(p);
-      Assert.fail();
+      fail();
     } catch(RemoteException re) {
       AppendTestUtil.LOG.info("Got an exception:", re);
-      Assert.assertEquals(AlreadyBeingCreatedException.class.getName(),
+      assertEquals(AlreadyBeingCreatedException.class.getName(),
           re.getClassName());
     } finally {
       fs2.close();
@@ -376,10 +379,10 @@ public class TestFileAppend{
 
       // 2nd append should get AlreadyBeingCreatedException
       fs1.append(p);
-      Assert.fail();
+      fail();
     } catch(RemoteException re) {
       AppendTestUtil.LOG.info("Got an exception:", re);
-      Assert.assertEquals(AlreadyBeingCreatedException.class.getName(),
+      assertEquals(AlreadyBeingCreatedException.class.getName(),
           re.getClassName());
     } finally {
       fs2.close();
@@ -428,13 +431,13 @@ public class TestFileAppend{
         fileLen += appendLen;
       }
 
-      Assert.assertEquals(fileLen, fs.getFileStatus(p).getLen());
+      assertEquals(fileLen, fs.getFileStatus(p).getLen());
       final byte[] actual = new byte[fileLen];
       final FSDataInputStream in = fs.open(p);
       in.readFully(actual);
       in.close();
       for(int i = 0; i < fileLen; i++) {
-        Assert.assertEquals(data[i], actual[i]);
+        assertEquals(data[i], actual[i]);
       }
     } finally {
       fs.close();
@@ -662,7 +665,8 @@ public class TestFileAppend{
     }
   }
   
-  @Test(timeout = 10000)
+  @Test
+  @Timeout(value = 10)
   public void testAppendCorruptedBlock() throws Exception {
     Configuration conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1024);
@@ -675,7 +679,7 @@ public class TestFileAppend{
       Path fileName = new Path("/appendCorruptBlock");
       DFSTestUtil.createFile(fs, fileName, 512, (short) 1, 0);
       DFSTestUtil.waitReplication(fs, fileName, (short) 1);
-      Assert.assertTrue("File not created", fs.exists(fileName));
+      assertTrue(fs.exists(fileName), "File not created");
       ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, fileName);
       cluster.corruptBlockOnDataNodes(block);
       DFSTestUtil.appendFile(fs, fileName, "appendCorruptBlock");
@@ -684,7 +688,8 @@ public class TestFileAppend{
     }
   }
 
-  @Test(timeout = 10000)
+  @Test
+  @Timeout(value = 10)
   public void testConcurrentAppendRead()
       throws IOException, TimeoutException, InterruptedException {
     // Create a finalized replica and append to it
@@ -707,7 +712,7 @@ public class TestFileAppend{
       Path fileName = new Path("/appendCorruptBlock");
       DFSTestUtil.createFile(fs, fileName, initialFileLength, (short) 1, 0);
       DFSTestUtil.waitReplication(fs, fileName, (short) 1);
-      Assert.assertTrue("File not created", fs.exists(fileName));
+      assertTrue(fs.exists(fileName), "File not created");
 
       // Call FsDatasetImpl#append to append the block file,
       // which converts it to a rbw replica.
@@ -738,7 +743,7 @@ public class TestFileAppend{
       // checksum, rather than on-disk checksum. Otherwise it will see a
       // checksum mismatch error.
       final byte[] readBlock = DFSTestUtil.readFileBuffer(fs, fileName);
-      assertEquals("should have read only one byte!", 1, readBlock.length);
+      assertEquals(1, readBlock.length, "should have read only one byte!");
     } finally {
       cluster.shutdown();
     }

+ 11 - 13
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend2.java

@@ -17,10 +17,10 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -42,7 +42,7 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.slf4j.event.Level;
 
 /**
@@ -445,10 +445,9 @@ public class TestFileAppend2 {
             } catch (InterruptedException e) {}
           }
 
-          assertTrue("File " + testfile + " size is " + 
-                     fs.getFileStatus(testfile).getLen() +
-                     " but expected " + (len + sizeToAppend),
-                    fs.getFileStatus(testfile).getLen() == (len + sizeToAppend));
+          assertTrue(fs.getFileStatus(testfile).getLen() == (len + sizeToAppend),
+              "File " + testfile + " size is " + fs.getFileStatus(testfile).getLen() +
+                  " but expected " + (len + sizeToAppend));
 
           AppendTestUtil.checkFullFile(fs, testfile, (int) (len + sizeToAppend),
               fileContents, "Read 2");
@@ -460,9 +459,8 @@ public class TestFileAppend2 {
                                " " + e);
             e.printStackTrace();
           }
-          assertTrue("Workload exception " + id + " testfile " + testfile +
-                     " expected size " + (len + sizeToAppend),
-                     false);
+          assertTrue(false, "Workload exception " + id + " testfile " + testfile +
+              " expected size " + (len + sizeToAppend));
         }
 
         // Add testfile back to the pool of files.
@@ -530,7 +528,7 @@ public class TestFileAppend2 {
     // If any of the worker thread failed in their job, indicate that
     // this test failed.
     //
-    assertTrue("testComplexAppend Worker encountered exceptions.", globalStatus);
+    assertTrue(globalStatus, "testComplexAppend Worker encountered exceptions.");
   }
 
   @Test

+ 15 - 16
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java

@@ -17,9 +17,9 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.IOException;
 import java.util.EnumSet;
@@ -45,10 +45,9 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 import org.slf4j.event.Level;
 
 /** This class implements some of tests posted in HADOOP-2658. */
@@ -69,7 +68,7 @@ public class TestFileAppend3  {
   private static MiniDFSCluster cluster;
   private static DistributedFileSystem fs;
 
-  @BeforeClass
+  @BeforeAll
   public static void setUp() throws java.lang.Exception {
     AppendTestUtil.LOG.info("setUp()");
     conf = new HdfsConfiguration();
@@ -79,7 +78,7 @@ public class TestFileAppend3  {
     fs = cluster.getFileSystem();
   }
    
-  @AfterClass
+  @AfterAll
   public static void tearDown() throws Exception {
     AppendTestUtil.LOG.info("tearDown()");
     if(fs != null) fs.close();
@@ -200,10 +199,10 @@ public class TestFileAppend3  {
     AppendTestUtil.check(fs, p, len1 + len2);
     List<LocatedBlock> blocks = fs.getClient().getLocatedBlocks(
         p.toString(), 0L).getLocatedBlocks();
-    Assert.assertEquals(3, blocks.size());
-    Assert.assertEquals(BLOCK_SIZE, blocks.get(0).getBlockSize());
-    Assert.assertEquals(BLOCK_SIZE / 2, blocks.get(1).getBlockSize());
-    Assert.assertEquals(BLOCK_SIZE / 4, blocks.get(2).getBlockSize());
+    assertEquals(3, blocks.size());
+    assertEquals(BLOCK_SIZE, blocks.get(0).getBlockSize());
+    assertEquals(BLOCK_SIZE / 2, blocks.get(1).getBlockSize());
+    assertEquals(BLOCK_SIZE / 4, blocks.get(2).getBlockSize());
   }
 
   /**
@@ -428,9 +427,9 @@ public class TestFileAppend3  {
     AppendTestUtil.check(fs, p, len1 + len2);
     if (appendToNewBlock) {
       LocatedBlocks blks = fs.dfs.getLocatedBlocks(p.toString(), 0);
-      Assert.assertEquals(2, blks.getLocatedBlocks().size());
-      Assert.assertEquals(len1, blks.getLocatedBlocks().get(0).getBlockSize());
-      Assert.assertEquals(len2, blks.getLocatedBlocks().get(1).getBlockSize());
+      assertEquals(2, blks.getLocatedBlocks().size());
+      assertEquals(len1, blks.getLocatedBlocks().get(0).getBlockSize());
+      assertEquals(len2, blks.getLocatedBlocks().get(1).getBlockSize());
       AppendTestUtil.check(fs, p, 0, len1);
       AppendTestUtil.check(fs, p, len1, len2);
     }

+ 16 - 11
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java

@@ -18,9 +18,9 @@
 package org.apache.hadoop.hdfs;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.ArgumentMatchers.anyLong;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyString;
@@ -47,8 +47,9 @@ import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.slf4j.event.Level;
 
 /* File Append tests for HDFS-200 & HDFS-142, specifically focused on:
@@ -72,7 +73,7 @@ public class TestFileAppend4 {
     GenericTestUtils.setLogLevel(DFSClient.LOG, Level.TRACE);
   }
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     this.conf = new Configuration();
 
@@ -142,7 +143,8 @@ public class TestFileAppend4 {
    * before calling completeFile, and then tries to recover
    * the lease from another thread.
    */
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testRecoverFinalizedBlock() throws Throwable {
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(5).build();
  
@@ -212,7 +214,8 @@ public class TestFileAppend4 {
    * starts writing from that writer, and then has the old lease holder
    * call completeFile
    */
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testCompleteOtherLeaseHoldersFile() throws Throwable {
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(5).build();
  
@@ -290,7 +293,8 @@ public class TestFileAppend4 {
   /**
    * Test the updation of NeededReplications for the Appended Block
    */
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testUpdateNeededReplicationsForAppendedFile() throws Exception {
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
@@ -326,7 +330,8 @@ public class TestFileAppend4 {
    * Test that an append with no locations fails with an exception
    * showing insufficient locations.
    */
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testAppendInsufficientLocations() throws Exception {
     Configuration conf = new Configuration();
 
@@ -379,7 +384,7 @@ public class TestFileAppend4 {
       FSDirectory dir = cluster.getNamesystem().getFSDirectory();
       final INodeFile inode = INodeFile.
           valueOf(dir.getINode("/testAppend"), "/testAppend");
-      assertTrue("File should remain closed", !inode.isUnderConstruction());
+      assertTrue(!inode.isUnderConstruction(), "File should remain closed");
     } finally {
       if (null != fileSystem) {
         fileSystem.close();

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java

@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.io.File;
 import java.io.IOException;
@@ -38,7 +38,7 @@ import org.apache.hadoop.hdfs.util.Holder;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.PathUtils;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 /**
  * Unit test to make sure that Append properly logs the right

+ 209 - 109
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileChecksum.java

@@ -30,12 +30,10 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeFaultInjector;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Timeout;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
@@ -45,6 +43,11 @@ import java.io.IOException;
 import java.util.Random;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY;
+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.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
 
@@ -54,7 +57,6 @@ import static org.mockito.Mockito.mock;
  * layout. For simple, it assumes 6 data blocks in both files and the block size
  * are the same.
  */
-@RunWith(Parameterized.class)
 public class TestFileChecksum {
   private static final Logger LOG = LoggerFactory
       .getLogger(TestFileChecksum.class);
@@ -84,21 +86,17 @@ public class TestFileChecksum {
 
   private static String checksumCombineMode;
 
-  public TestFileChecksum(String mode) {
-    checksumCombineMode = mode;
+  public void initTestFileChecksum(String pMode) throws IOException {
+    checksumCombineMode = pMode;
+    setup(pMode);
   }
 
-  @Parameterized.Parameters
   public static Object[] getParameters() {
     return new Object[] {
         ChecksumCombineMode.MD5MD5CRC.name(),
         ChecksumCombineMode.COMPOSITE_CRC.name()};
   }
 
-  @Rule
-  public ExpectedException exception = ExpectedException.none();
-
-  @Parameterized.BeforeParam
   public static void setup(String mode) throws IOException {
     checksumCombineMode = mode;
     int numDNs = dataBlocks + parityBlocks + 2;
@@ -123,58 +121,79 @@ public class TestFileChecksum {
     GenericTestUtils.setLogLevel(FileChecksumHelper.LOG, Level.DEBUG);
   }
 
-  @Parameterized.AfterParam
-  public static void tearDown() {
+  @AfterEach
+  public void tearDown() {
     if (cluster != null) {
       cluster.shutdown();
       cluster = null;
     }
   }
 
-  @Test(timeout = 90000)
-  public void testStripedFileChecksum1() throws Exception {
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksum1(String pMode) throws Exception {
+    initTestFileChecksum(pMode);
     int length = 0;
     prepareTestFiles(fileSize, new String[] {stripedFile1, stripedFile2});
     testStripedFileChecksum(length, length + 10);
   }
 
-  @Test(timeout = 90000)
-  public void testStripedFileChecksum2() throws Exception {
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksum2(String pMode) throws Exception {
+    initTestFileChecksum(pMode);
     int length = stripSize - 1;
     prepareTestFiles(fileSize, new String[] {stripedFile1, stripedFile2});
     testStripedFileChecksum(length, length - 10);
   }
 
-  @Test(timeout = 90000)
-  public void testStripedFileChecksum3() throws Exception {
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksum3(String pMode) throws Exception {
+    initTestFileChecksum(pMode);
     int length = stripSize;
     prepareTestFiles(fileSize, new String[] {stripedFile1, stripedFile2});
     testStripedFileChecksum(length, length - 10);
   }
 
-  @Test(timeout = 90000)
-  public void testStripedFileChecksum4() throws Exception {
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksum4(String pMode) throws Exception {
+    initTestFileChecksum(pMode);
     int length = stripSize + cellSize * 2;
     prepareTestFiles(fileSize, new String[] {stripedFile1, stripedFile2});
     testStripedFileChecksum(length, length - 10);
   }
 
-  @Test(timeout = 90000)
-  public void testStripedFileChecksum5() throws Exception {
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksum5(String pMode) throws Exception {
+    initTestFileChecksum(pMode);
     int length = blockGroupSize;
     prepareTestFiles(fileSize, new String[] {stripedFile1, stripedFile2});
     testStripedFileChecksum(length, length - 10);
   }
 
-  @Test(timeout = 90000)
-  public void testStripedFileChecksum6() throws Exception {
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksum6(String pMode) throws Exception {
+    initTestFileChecksum(pMode);
     int length = blockGroupSize + blockSize;
     prepareTestFiles(fileSize, new String[] {stripedFile1, stripedFile2});
     testStripedFileChecksum(length, length - 10);
   }
 
-  @Test(timeout = 90000)
-  public void testStripedFileChecksum7() throws Exception {
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksum7(String pMode) throws Exception {
+    initTestFileChecksum(pMode);
     int length = -1; // whole file
     prepareTestFiles(fileSize, new String[] {stripedFile1, stripedFile2});
     testStripedFileChecksum(length, fileSize);
@@ -193,14 +212,17 @@ public class TestFileChecksum {
     LOG.info("stripedFileChecksum2:" + stripedFileChecksum2);
     LOG.info("stripedFileChecksum3:" + stripedFileChecksum3);
 
-    Assert.assertTrue(stripedFileChecksum1.equals(stripedFileChecksum2));
+    assertTrue(stripedFileChecksum1.equals(stripedFileChecksum2));
     if (range1 >=0 && range1 != range2) {
-      Assert.assertFalse(stripedFileChecksum1.equals(stripedFileChecksum3));
+      assertFalse(stripedFileChecksum1.equals(stripedFileChecksum3));
     }
   }
 
-  @Test(timeout = 90000)
-  public void testStripedAndReplicatedFileChecksum() throws Exception {
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedAndReplicatedFileChecksum(String pMode) throws Exception {
+    initTestFileChecksum(pMode);
     prepareTestFiles(fileSize, new String[] {stripedFile1, replicatedFile});
     FileChecksum stripedFileChecksum1 = getFileChecksum(stripedFile1,
         10, false);
@@ -208,9 +230,9 @@ public class TestFileChecksum {
         10, false);
 
     if (checksumCombineMode.equals(ChecksumCombineMode.COMPOSITE_CRC.name())) {
-      Assert.assertEquals(stripedFileChecksum1, replicatedFileChecksum);
+      assertEquals(stripedFileChecksum1, replicatedFileChecksum);
     } else {
-      Assert.assertNotEquals(stripedFileChecksum1, replicatedFileChecksum);
+      assertNotEquals(stripedFileChecksum1, replicatedFileChecksum);
     }
   }
 
@@ -222,8 +244,11 @@ public class TestFileChecksum {
    *    but the last block size in the check length is ((dataBlocks - 1) * blockSize
    *    + (int) (blockSize * 0.6))
    */
-  @Test(timeout = 90000)
-  public void testStripedAndReplicatedFileChecksum2() throws Exception {
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedAndReplicatedFileChecksum2(String pMode) throws Exception {
+    initTestFileChecksum(pMode);
     final int lastBlockSize = (int) (blockSize * 0.5);
     final int fullStripeLength = dataBlocks * blockSize;
     final int testFileSize = fullStripeLength + lastBlockSize;
@@ -232,8 +257,8 @@ public class TestFileChecksum {
     final int specialLength = (dataBlocks - 1) * blockSize
         + (int) (blockSize * 0.6);
 
-    Assert.assertTrue(specialLength % blockSize > lastBlockSize);
-    Assert.assertTrue(specialLength % fullStripeLength > lastBlockSize);
+    assertTrue(specialLength % blockSize > lastBlockSize);
+    assertTrue(specialLength % fullStripeLength > lastBlockSize);
 
     FileChecksum stripedFileChecksum = getFileChecksum(stripedFile1,
         specialLength, false);
@@ -241,14 +266,17 @@ public class TestFileChecksum {
         specialLength, false);
 
     if (checksumCombineMode.equals(ChecksumCombineMode.COMPOSITE_CRC.name())) {
-      Assert.assertEquals(replicatedFileChecksum, stripedFileChecksum);
+      assertEquals(replicatedFileChecksum, stripedFileChecksum);
     } else {
-      Assert.assertNotEquals(replicatedFileChecksum, stripedFileChecksum);
+      assertNotEquals(replicatedFileChecksum, stripedFileChecksum);
     }
   }
 
-  @Test(timeout = 90000)
-  public void testDifferentBlockSizeReplicatedFileChecksum() throws Exception {
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testDifferentBlockSizeReplicatedFileChecksum(String pMode) throws Exception {
+    initTestFileChecksum(pMode);
     byte[] fileData = StripedFileTestUtil.generateBytes(fileSize);
     String replicatedFile1 = "/replicatedFile1";
     String replicatedFile2 = "/replicatedFile2";
@@ -260,14 +288,17 @@ public class TestFileChecksum {
     FileChecksum checksum2 = getFileChecksum(replicatedFile2, -1, false);
 
     if (checksumCombineMode.equals(ChecksumCombineMode.COMPOSITE_CRC.name())) {
-      Assert.assertEquals(checksum1, checksum2);
+      assertEquals(checksum1, checksum2);
     } else {
-      Assert.assertNotEquals(checksum1, checksum2);
+      assertNotEquals(checksum1, checksum2);
     }
   }
 
-  @Test(timeout = 90000)
-  public void testStripedFileChecksumWithMissedDataBlocks1() throws Exception {
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksumWithMissedDataBlocks1(String pMode) throws Exception {
+    initTestFileChecksum(pMode);
     prepareTestFiles(fileSize, new String[] {stripedFile1});
     FileChecksum stripedFileChecksum1 = getFileChecksum(stripedFile1, fileSize,
         false);
@@ -277,12 +308,15 @@ public class TestFileChecksum {
     LOG.info("stripedFileChecksum1:" + stripedFileChecksum1);
     LOG.info("stripedFileChecksumRecon:" + stripedFileChecksumRecon);
 
-    Assert.assertTrue("Checksum mismatches!",
-        stripedFileChecksum1.equals(stripedFileChecksumRecon));
+    assertTrue(stripedFileChecksum1.equals(stripedFileChecksumRecon),
+        "Checksum mismatches!");
   }
 
-  @Test(timeout = 90000)
-  public void testStripedFileChecksumWithMissedDataBlocks2() throws Exception {
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksumWithMissedDataBlocks2(String pMode) throws Exception {
+    initTestFileChecksum(pMode);
     prepareTestFiles(fileSize, new String[] {stripedFile1, stripedFile2});
     FileChecksum stripedFileChecksum1 = getFileChecksum(stripedFile1, -1,
         false);
@@ -295,12 +329,12 @@ public class TestFileChecksum {
     LOG.info("stripedFileChecksum2:" + stripedFileChecksum1);
     LOG.info("stripedFileChecksum2Recon:" + stripedFileChecksum2Recon);
 
-    Assert.assertTrue("Checksum mismatches!",
-        stripedFileChecksum1.equals(stripedFileChecksum2));
-    Assert.assertTrue("Checksum mismatches!",
-        stripedFileChecksum1.equals(stripedFileChecksum2Recon));
-    Assert.assertTrue("Checksum mismatches!",
-        stripedFileChecksum2.equals(stripedFileChecksum2Recon));
+    assertTrue(stripedFileChecksum1.equals(stripedFileChecksum2),
+        "Checksum mismatches!");
+    assertTrue(stripedFileChecksum1.equals(stripedFileChecksum2Recon),
+        "Checksum mismatches!");
+    assertTrue(stripedFileChecksum2.equals(stripedFileChecksum2Recon),
+        "Checksum mismatches!");
   }
 
   private void testStripedFileChecksumWithMissedDataBlocksRangeQuery(
@@ -316,17 +350,20 @@ public class TestFileChecksum {
     LOG.info("stripedFileChecksum1:" + stripedFileChecksum1);
     LOG.info("stripedFileChecksumRecon:" + stripedFileChecksumRecon);
 
-    Assert.assertTrue("Checksum mismatches!",
-        stripedFileChecksum1.equals(stripedFileChecksumRecon));
+    assertTrue(stripedFileChecksum1.equals(stripedFileChecksumRecon),
+        "Checksum mismatches!");
   }
 
   /**
    * Test to verify that the checksum can be computed for a small file less than
    * bytesPerCRC size.
    */
-  @Test(timeout = 90000)
-  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery1()
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery1(String pMode)
       throws Exception {
+    initTestFileChecksum(pMode);
     testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1, 1);
   }
 
@@ -334,9 +371,12 @@ public class TestFileChecksum {
    * Test to verify that the checksum can be computed for a small file less than
    * bytesPerCRC size.
    */
-  @Test(timeout = 90000)
-  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery2()
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery2(String pMode)
       throws Exception {
+    initTestFileChecksum(pMode);
     testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1, 10);
   }
 
@@ -345,9 +385,12 @@ public class TestFileChecksum {
    * length of file range for checksum calculation. 512 is the value of
    * bytesPerCRC.
    */
-  @Test(timeout = 90000)
-  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery3()
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery3(String pMode)
       throws Exception {
+    initTestFileChecksum(pMode);
     testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1,
         bytesPerCRC);
   }
@@ -356,9 +399,12 @@ public class TestFileChecksum {
    * Test to verify that the checksum can be computed by giving 'cellsize'
    * length of file range for checksum calculation.
    */
-  @Test(timeout = 90000)
-  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery4()
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery4(String pMode)
       throws Exception {
+    initTestFileChecksum(pMode);
     testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1,
         cellSize);
   }
@@ -367,9 +413,12 @@ public class TestFileChecksum {
    * Test to verify that the checksum can be computed by giving less than
    * cellsize length of file range for checksum calculation.
    */
-  @Test(timeout = 90000)
-  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery5()
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery5(String pMode)
       throws Exception {
+    initTestFileChecksum(pMode);
     testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1,
         cellSize - 1);
   }
@@ -378,9 +427,12 @@ public class TestFileChecksum {
    * Test to verify that the checksum can be computed by giving greater than
    * cellsize length of file range for checksum calculation.
    */
-  @Test(timeout = 90000)
-  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery6()
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery6(String pMode)
       throws Exception {
+    initTestFileChecksum(pMode);
     testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1,
         cellSize + 1);
   }
@@ -389,9 +441,12 @@ public class TestFileChecksum {
    * Test to verify that the checksum can be computed by giving two times
    * cellsize length of file range for checksum calculation.
    */
-  @Test(timeout = 90000)
-  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery7()
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery7(String pMode)
       throws Exception {
+    initTestFileChecksum(pMode);
     testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1,
         cellSize * 2);
   }
@@ -400,9 +455,12 @@ public class TestFileChecksum {
    * Test to verify that the checksum can be computed by giving stripSize
    * length of file range for checksum calculation.
    */
-  @Test(timeout = 90000)
-  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery8()
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery8(String pMode)
       throws Exception {
+    initTestFileChecksum(pMode);
     testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1,
         stripSize);
   }
@@ -411,9 +469,12 @@ public class TestFileChecksum {
    * Test to verify that the checksum can be computed by giving less than
    * stripSize length of file range for checksum calculation.
    */
-  @Test(timeout = 90000)
-  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery9()
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery9(String pMode)
       throws Exception {
+    initTestFileChecksum(pMode);
     testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1,
         stripSize - 1);
   }
@@ -422,9 +483,12 @@ public class TestFileChecksum {
    * Test to verify that the checksum can be computed by giving greater than
    * stripSize length of file range for checksum calculation.
    */
-  @Test(timeout = 90000)
-  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery10()
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery10(String pMode)
       throws Exception {
+    initTestFileChecksum(pMode);
     testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1,
         stripSize + 1);
   }
@@ -433,9 +497,12 @@ public class TestFileChecksum {
    * Test to verify that the checksum can be computed by giving less than
    * blockGroupSize length of file range for checksum calculation.
    */
-  @Test(timeout = 90000)
-  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery11()
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery11(String pMode)
       throws Exception {
+    initTestFileChecksum(pMode);
     testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1,
         blockGroupSize - 1);
   }
@@ -444,9 +511,12 @@ public class TestFileChecksum {
    * Test to verify that the checksum can be computed by giving greaterthan
    * blockGroupSize length of file range for checksum calculation.
    */
-  @Test(timeout = 90000)
-  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery12()
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery12(String pMode)
       throws Exception {
+    initTestFileChecksum(pMode);
     testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1,
         blockGroupSize + 1);
   }
@@ -455,9 +525,12 @@ public class TestFileChecksum {
    * Test to verify that the checksum can be computed by giving greater than
    * blockGroupSize length of file range for checksum calculation.
    */
-  @Test(timeout = 90000)
-  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery13()
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery13(String pMode)
       throws Exception {
+    initTestFileChecksum(pMode);
     testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1,
         blockGroupSize * numBlockGroups / 2);
   }
@@ -466,9 +539,12 @@ public class TestFileChecksum {
    * Test to verify that the checksum can be computed by giving lessthan
    * fileSize length of file range for checksum calculation.
    */
-  @Test(timeout = 90000)
-  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery14()
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery14(String pMode)
       throws Exception {
+    initTestFileChecksum(pMode);
     testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1,
         fileSize - 1);
   }
@@ -477,9 +553,12 @@ public class TestFileChecksum {
    * Test to verify that the checksum can be computed for a length greater than
    * file size.
    */
-  @Test(timeout = 90000)
-  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery15()
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery15(String pMode)
       throws Exception {
+    initTestFileChecksum(pMode);
     testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1,
         fileSize * 2);
   }
@@ -488,9 +567,12 @@ public class TestFileChecksum {
    * Test to verify that the checksum can be computed for a small file less than
    * bytesPerCRC size.
    */
-  @Test(timeout = 90000)
-  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery16()
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery16(String pMode)
       throws Exception {
+    initTestFileChecksum(pMode);
     int fileLength = 100;
     String stripedFile3 = ecDir + "/stripedFileChecksum3";
     prepareTestFiles(fileLength, new String[] {stripedFile3});
@@ -502,9 +584,12 @@ public class TestFileChecksum {
    * Test to verify that the checksum can be computed for a small file less than
    * bytesPerCRC size.
    */
-  @Test(timeout = 90000)
-  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery17()
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery17(String pMode)
       throws Exception {
+    initTestFileChecksum(pMode);
     int fileLength = 100;
     String stripedFile3 = ecDir + "/stripedFileChecksum3";
     prepareTestFiles(fileLength, new String[] {stripedFile3});
@@ -515,9 +600,12 @@ public class TestFileChecksum {
    * Test to verify that the checksum can be computed for a small file less than
    * bytesPerCRC size.
    */
-  @Test(timeout = 90000)
-  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery18()
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery18(String pMode)
       throws Exception {
+    initTestFileChecksum(pMode);
     int fileLength = 100;
     String stripedFile3 = ecDir + "/stripedFileChecksum3";
     prepareTestFiles(fileLength, new String[] {stripedFile3});
@@ -528,9 +616,12 @@ public class TestFileChecksum {
    * Test to verify that the checksum can be computed with greater than file
    * length.
    */
-  @Test(timeout = 90000)
-  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery19()
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery19(String pMode)
       throws Exception {
+    initTestFileChecksum(pMode);
     int fileLength = 100;
     String stripedFile3 = ecDir + "/stripedFileChecksum3";
     prepareTestFiles(fileLength, new String[] {stripedFile3});
@@ -542,9 +633,12 @@ public class TestFileChecksum {
    * Test to verify that the checksum can be computed for small file with less
    * than file length.
    */
-  @Test(timeout = 90000)
-  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery20()
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksumWithMissedDataBlocksRangeQuery20(String pMode)
       throws Exception {
+    initTestFileChecksum(pMode);
     int fileLength = bytesPerCRC;
     String stripedFile3 = ecDir + "/stripedFileChecksum3";
     prepareTestFiles(fileLength, new String[] {stripedFile3});
@@ -552,9 +646,12 @@ public class TestFileChecksum {
         bytesPerCRC - 1);
   }
 
-  @Test(timeout = 90000)
-  public void testStripedFileChecksumWithReconstructFail()
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testStripedFileChecksumWithReconstructFail(String pMode)
       throws Exception {
+    initTestFileChecksum(pMode);
     String stripedFile4 = ecDir + "/stripedFileChecksum4";
     prepareTestFiles(fileSize, new String[] {stripedFile4});
 
@@ -576,15 +673,17 @@ public class TestFileChecksum {
       // getting result.
       FileChecksum fileChecksum1 = getFileChecksum(stripedFile4, -1, true);
 
-      Assert.assertEquals("checksum should be same", fileChecksum,
-          fileChecksum1);
+      assertEquals(fileChecksum, fileChecksum1, "checksum should be same");
     } finally {
       DataNodeFaultInjector.set(oldInjector);
     }
   }
 
-  @Test(timeout = 90000)
-  public void testMixedBytesPerChecksum() throws Exception {
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  @Timeout(value = 90)
+  public void testMixedBytesPerChecksum(String pMode) throws Exception {
+    initTestFileChecksum(pMode);
     int fileLength = bytesPerCRC * 3;
     byte[] fileData = StripedFileTestUtil.generateBytes(fileLength);
     String replicatedFile1 = "/replicatedFile1";
@@ -610,10 +709,11 @@ public class TestFileChecksum {
       DFSTestUtil.writeFile(fs, new Path(replicatedFile2), fileData);
       FileChecksum checksum1 = getFileChecksum(replicatedFile1, -1, false);
       FileChecksum checksum2 = getFileChecksum(replicatedFile2, -1, false);
-      Assert.assertEquals(checksum1, checksum2);
+      assertEquals(checksum1, checksum2);
     } else {
-      exception.expect(IOException.class);
-      FileChecksum checksum = getFileChecksum(replicatedFile1, -1, false);
+      assertThrows(IOException.class, () -> {
+        FileChecksum checksum = getFileChecksum(replicatedFile1, -1, false);
+      });
     }
   }
 

+ 30 - 25
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileConcurrentReader.java

@@ -16,10 +16,10 @@
  * limitations under the License.
  */
 package org.apache.hadoop.hdfs;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.IOException;
 import java.util.Arrays;
@@ -38,10 +38,11 @@ import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.StringUtils;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.event.Level;
@@ -77,13 +78,13 @@ public class TestFileConcurrentReader {
   private FileSystem fileSystem;
 
 
-  @Before
+  @BeforeEach
   public void setUp() throws IOException {
     conf = new Configuration();
     init(conf);
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     if (cluster != null) {
       cluster.shutdown();
@@ -124,9 +125,8 @@ public class TestFileConcurrentReader {
     IOUtils.readFully(inputStream, buffer, 0, numBytes);
     inputStream.close();
 
-    assertTrue(
-      "unable to validate bytes",
-      validateSequentialBytes(buffer, 0, numBytes)
+    assertTrue(validateSequentialBytes(buffer, 0, numBytes),
+        "unable to validate bytes"
     );
   }
 
@@ -153,7 +153,8 @@ public class TestFileConcurrentReader {
   /**
    * Test that that writes to an incomplete block are available to a reader
    */
-  @Test (timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testUnfinishedBlockRead()
     throws IOException {
     // create a new file in the root, write data, do no close
@@ -176,7 +177,8 @@ public class TestFileConcurrentReader {
    * would result in too small a buffer to do the buffer-copy needed
    * for partial chunks.
    */
-  @Test (timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testUnfinishedBlockPacketBufferOverrun() throws IOException {
     // check that / exists
     Path path = new Path("/");
@@ -202,7 +204,8 @@ public class TestFileConcurrentReader {
   // use a small block size and a large write so that DN is busy creating
   // new blocks.  This makes it almost 100% sure we can reproduce
   // case of client getting a DN that hasn't yet created the blocks
-  @Test (timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testImmediateReadOfNewFile()
     throws IOException {
     final int blockSize = 64 * 1024;
@@ -279,37 +282,41 @@ public class TestFileConcurrentReader {
 
   // for some reason, using tranferTo evokes the race condition more often
   // so test separately
-  @Test (timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testUnfinishedBlockCRCErrorTransferTo() throws IOException {
     runTestUnfinishedBlockCRCError(true, SyncType.SYNC, DEFAULT_WRITE_SIZE);
   }
 
-  @Test (timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testUnfinishedBlockCRCErrorTransferToVerySmallWrite()
     throws IOException {
     runTestUnfinishedBlockCRCError(true, SyncType.SYNC, SMALL_WRITE_SIZE);
   }
 
   // fails due to issue w/append, disable 
-  @Ignore
+  @Disabled
   public void _testUnfinishedBlockCRCErrorTransferToAppend()
     throws IOException {
     runTestUnfinishedBlockCRCError(true, SyncType.APPEND, DEFAULT_WRITE_SIZE);
   }
 
-  @Test (timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testUnfinishedBlockCRCErrorNormalTransfer() throws IOException {
     runTestUnfinishedBlockCRCError(false, SyncType.SYNC, DEFAULT_WRITE_SIZE);
   }
 
-  @Test (timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testUnfinishedBlockCRCErrorNormalTransferVerySmallWrite()
     throws IOException {
     runTestUnfinishedBlockCRCError(false, SyncType.SYNC, SMALL_WRITE_SIZE);
   }
 
   // fails due to issue w/append, disable 
-  @Ignore
+  @Disabled
   public void _testUnfinishedBlockCRCErrorNormalTransferAppend()
     throws IOException {
     runTestUnfinishedBlockCRCError(false, SyncType.APPEND, DEFAULT_WRITE_SIZE);
@@ -407,9 +414,7 @@ public class TestFileConcurrentReader {
       writer.join();
       tailer.join();
 
-      assertFalse(
-        "error occurred, see log above", error.get()
-      );
+      assertFalse(error.get(), "error occurred, see log above");
     } catch (InterruptedException e) {
       LOG.info("interrupted waiting for writer or tailer to complete");
 

+ 12 - 12
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java

@@ -22,9 +22,9 @@ import java.util.function.Supplier;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
@@ -56,7 +56,7 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.util.RwLockMode;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.PathUtils;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.event.Level;
 
@@ -87,7 +87,7 @@ public class TestFileCorruption {
       DataNode dn = cluster.getDataNodes().get(2);
       Map<DatanodeStorage, BlockListAsLongs> blockReports =
           dn.getFSDataset().getBlockReports(bpid);
-      assertTrue("Blocks do not exist on data-dir", !blockReports.isEmpty());
+      assertTrue(!blockReports.isEmpty(), "Blocks do not exist on data-dir");
       for (BlockListAsLongs report : blockReports.values()) {
         for (BlockReportReplica brr : report) {
           LOG.info("Deliberately removing block {}", brr.getBlockName());
@@ -95,8 +95,8 @@ public class TestFileCorruption {
               new ExtendedBlock(bpid, brr)).deleteData();
         }
       }
-      assertTrue("Corrupted replicas not handled properly.",
-                 util.checkFiles(fs, "/srcdat"));
+      assertTrue(
+                util.checkFiles(fs, "/srcdat"), "Corrupted replicas not handled properly.");
       util.cleanup(fs, "/srcdat");
     } finally {
       if (cluster != null) { cluster.shutdown(); }
@@ -147,8 +147,8 @@ public class TestFileCorruption {
       // get the block
       final String bpid = cluster.getNamesystem().getBlockPoolId();
       ExtendedBlock blk = getFirstBlock(cluster.getDataNodes().get(0), bpid);
-      assertFalse("Data directory does not contain any blocks or there was an "
-          + "IO error", blk==null);
+      assertFalse(blk==null, "Data directory does not contain any blocks or there was an "
+          + "IO error");
 
       // start a third datanode
       cluster.startDataNodes(conf, 1, true, null, null);
@@ -198,14 +198,14 @@ public class TestFileCorruption {
       final String bpid = cluster.getNamesystem().getBlockPoolId();
       File storageDir = cluster.getInstanceStorageDir(0, 0);
       File dataDir = MiniDFSCluster.getFinalizedDir(storageDir, bpid);
-      assertTrue("Data directory does not exist", dataDir.exists());
+      assertTrue(dataDir.exists(), "Data directory does not exist");
       ExtendedBlock blk = getFirstBlock(cluster.getDataNodes().get(0), bpid);
       if (blk == null) {
         blk = getFirstBlock(cluster.getDataNodes().get(0), bpid);
       }
-      assertFalse("Data directory does not contain any blocks or there was an" +
+      assertFalse(blk == null, "Data directory does not contain any blocks or there was an" +
           " " +
-          "IO error", blk == null);
+          "IO error");
       ArrayList<DataNode> datanodes = cluster.getDataNodes();
       assertEquals(datanodes.size(), 3);
       FSNamesystem ns = cluster.getNamesystem();

+ 63 - 61
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java

@@ -34,10 +34,11 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
 import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
 import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.junit.Assume.assumeTrue;
 import static org.mockito.Mockito.doReturn;
 
@@ -90,8 +91,8 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.ReflectionUtils;
 import org.apache.hadoop.util.Time;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.mockito.Mockito;
 import org.slf4j.event.Level;
 
@@ -114,7 +115,7 @@ public class TestFileCreation {
   static final int numBlocks = 2;
   static final int fileSize = numBlocks * blockSize + 1;
   boolean simulatedStorage = false;
-  
+
   private static final String[] NON_CANONICAL_PATHS = new String[] {
     "//foo",
     "///foo2",
@@ -353,8 +354,8 @@ public class TestFileCreation {
       Path path = new Path("/");
       System.out.println("Path : \"" + path.toString() + "\"");
       System.out.println(fs.getFileStatus(path).isDirectory()); 
-      assertTrue("/ should be a directory", 
-                 fs.getFileStatus(path).isDirectory());
+      assertTrue(
+                 fs.getFileStatus(path).isDirectory(), "/ should be a directory");
 
       //
       // Create a directory inside /, then try to overwrite it
@@ -366,7 +367,7 @@ public class TestFileCreation {
       try {
         fs.create(dir1, true); // Create path, overwrite=true
         fs.close();
-        assertTrue("Did not prevent directory from being overwritten.", false);
+        assertTrue(false, "Did not prevent directory from being overwritten.");
       } catch (FileAlreadyExistsException e) {
         // expected
       }
@@ -382,8 +383,8 @@ public class TestFileCreation {
       FSDataOutputStream stm = createFile(fs, file1, 1);
 
       // verify that file exists in FS namespace
-      assertTrue(file1 + " should be a file", 
-                 fs.getFileStatus(file1).isFile());
+      assertTrue(
+                 fs.getFileStatus(file1).isFile(), file1 + " should be a file");
       System.out.println("Path : \"" + file1 + "\"");
 
       // write to file
@@ -393,14 +394,14 @@ public class TestFileCreation {
 
       // verify that file size has changed to the full size
       long len = fs.getFileStatus(file1).getLen();
-      assertTrue(file1 + " should be of size " + fileSize +
-                 " but found to be of size " + len, 
-                  len == fileSize);
+      assertTrue(
+                  len == fileSize, file1 + " should be of size " + fileSize +
+                 " but found to be of size " + len);
       
       // verify the disk space the file occupied
       long diskSpace = dfs.getContentSummary(file1.getParent()).getLength();
-      assertEquals(file1 + " should take " + fileSize + " bytes disk space " +
-          "but found to take " + diskSpace + " bytes", fileSize, diskSpace);
+      assertEquals(fileSize, diskSpace, file1 + " should take " + fileSize + " bytes disk space " +
+          "but found to take " + diskSpace + " bytes");
       
       // Check storage usage 
       // can't check capacities for real storage since the OS file system may be changing under us.
@@ -464,12 +465,12 @@ public class TestFileCreation {
       fs = cluster.getFileSystem();
       localfs = FileSystem.getLocal(conf);
 
-      assertTrue(file1 + " still exists inspite of deletOnExit set.",
-                 !fs.exists(file1));
-      assertTrue(file2 + " still exists inspite of deletOnExit set.",
-                 !fs.exists(file2));
-      assertTrue(file3 + " still exists inspite of deletOnExit set.",
-                 !localfs.exists(file3));
+      assertTrue(
+                !fs.exists(file1), file1 + " still exists inspite of deletOnExit set.");
+      assertTrue(
+                !fs.exists(file2), file2 + " still exists inspite of deletOnExit set.");
+      assertTrue(
+                !localfs.exists(file3), file3 + " still exists inspite of deletOnExit set.");
       System.out.println("DeleteOnExit successful.");
 
     } finally {
@@ -564,8 +565,8 @@ public class TestFileCreation {
       FSDataOutputStream stm = createFile(fs, file1, 1);
 
       // verify that file exists in FS namespace
-      assertTrue(file1 + " should be a file", 
-                 fs.getFileStatus(file1).isFile());
+      assertTrue(
+                 fs.getFileStatus(file1).isFile(), file1 + " should be a file");
       System.out.println("Path : \"" + file1 + "\"");
 
       // kill the datanode
@@ -601,8 +602,8 @@ public class TestFileCreation {
       LocatedBlocks locations = client.getNamenode().getBlockLocations(
                                   file1.toString(), 0, Long.MAX_VALUE);
       System.out.println("locations = " + locations.locatedBlockCount());
-      assertTrue("Error blocks were not cleaned up",
-                 locations.locatedBlockCount() == 0);
+      assertTrue(
+                locations.locatedBlockCount() == 0, "Error blocks were not cleaned up");
     } finally {
       cluster.shutdown();
       client.close();
@@ -737,14 +738,14 @@ public class TestFileCreation {
       HdfsDataOutputStream stm = create(fs, file1, 1);
       System.out.println("testFileCreationNamenodeRestart: "
                          + "Created file " + file1);
-      assertEquals(file1 + " should be replicated to 1 datanode.", 1,
-          stm.getCurrentBlockReplication());
+      assertEquals(1, stm.getCurrentBlockReplication(),
+          file1 + " should be replicated to 1 datanode.");
 
       // write two full blocks.
       writeFile(stm, numBlocks * blockSize);
       stm.hflush();
-      assertEquals(file1 + " should still be replicated to 1 datanode.", 1,
-          stm.getCurrentBlockReplication());
+      assertEquals(1, stm.getCurrentBlockReplication(),
+          file1 + " should still be replicated to 1 datanode.");
 
       // rename file wile keeping it open.
       Path fileRenamed = new Path("/filestatusRenamed.dat");
@@ -834,15 +835,15 @@ public class TestFileCreation {
       LocatedBlocks locations = client.getNamenode().getBlockLocations(
                                   file1.toString(), 0, Long.MAX_VALUE);
       System.out.println("locations = " + locations.locatedBlockCount());
-      assertTrue("Error blocks were not cleaned up for file " + file1,
-                 locations.locatedBlockCount() == 3);
+      assertTrue(locations.locatedBlockCount() == 3,
+          "Error blocks were not cleaned up for file " + file1);
 
       // verify filestatus2.dat
       locations = client.getNamenode().getBlockLocations(
                                   file2.toString(), 0, Long.MAX_VALUE);
       System.out.println("locations = " + locations.locatedBlockCount());
-      assertTrue("Error blocks were not cleaned up for file " + file2,
-                 locations.locatedBlockCount() == 1);
+      assertTrue(locations.locatedBlockCount() == 1,
+          "Error blocks were not cleaned up for file " + file2);
     } finally {
       IOUtils.closeStream(fs);
       cluster.shutdown();
@@ -879,8 +880,8 @@ public class TestFileCreation {
       dfsclient.close();
 
       // reopen file system and verify that file exists.
-      assertTrue(file1 + " does not exist.", 
-          AppendTestUtil.createHdfsWithDifferentUsername(conf).exists(file1));
+      assertTrue(AppendTestUtil.createHdfsWithDifferentUsername(conf).exists(file1),
+          file1 + " does not exist.");
     } finally {
       cluster.shutdown();
     }
@@ -922,19 +923,19 @@ public class TestFileCreation {
     // Create a file when parent dir exists as file, should fail
     expectedException = createNonRecursive(fs, new Path(path, "Create"), 1, createFlag);
 
-    assertTrue("Create a file when parent directory exists as a file"
-        + " should throw ParentNotDirectoryException ",
-        expectedException != null
-            && expectedException instanceof ParentNotDirectoryException);
+    assertTrue(expectedException != null
+            && expectedException instanceof ParentNotDirectoryException,
+        "Create a file when parent directory exists as a file"
+            + " should throw ParentNotDirectoryException ");
     fs.delete(path, true);
     // Create a file in a non-exist directory, should fail
     final Path path2 = new Path(nonExistDir + "/testCreateNonRecursive");
     expectedException =  createNonRecursive(fs, path2, 1, createFlag);
 
-    assertTrue("Create a file in a non-exist dir using"
-        + " createNonRecursive() should throw FileNotFoundException ",
-        expectedException != null
-            && expectedException instanceof FileNotFoundException);
+    assertTrue(expectedException != null
+            && expectedException instanceof FileNotFoundException,
+        "Create a file in a non-exist dir using"
+            + " createNonRecursive() should throw FileNotFoundException ");
 
     EnumSet<CreateFlag> overwriteFlag =
       EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE);
@@ -944,20 +945,20 @@ public class TestFileCreation {
     // Overwrite a file when parent dir exists as file, should fail
     expectedException = createNonRecursive(fs, new Path(path, "Overwrite"), 1, overwriteFlag);
 
-    assertTrue("Overwrite a file when parent directory exists as a file"
-        + " should throw ParentNotDirectoryException ",
-        expectedException != null
-            && expectedException instanceof ParentNotDirectoryException);
+    assertTrue(expectedException != null
+            && expectedException instanceof ParentNotDirectoryException,
+        "Overwrite a file when parent directory exists as a file"
+            + " should throw ParentNotDirectoryException ");
     fs.delete(path, true);
 
     // Overwrite a file in a non-exist directory, should fail
     final Path path3 = new Path(nonExistDir + "/testOverwriteNonRecursive");
     expectedException = createNonRecursive(fs, path3, 1, overwriteFlag);
 
-    assertTrue("Overwrite a file in a non-exist dir using"
-        + " createNonRecursive() should throw FileNotFoundException ",
-        expectedException != null
-            && expectedException instanceof FileNotFoundException);
+    assertTrue(expectedException != null
+            && expectedException instanceof FileNotFoundException,
+        "Overwrite a file in a non-exist dir using"
+            + " createNonRecursive() should throw FileNotFoundException ");
   }
 
   // Attempts to create and close a file using FileSystem.createNonRecursive(),
@@ -1086,8 +1087,8 @@ public class TestFileCreation {
       out.write("something".getBytes());
       out.hflush();
       int actualRepl = out.getCurrentBlockReplication();
-      assertTrue(f + " should be replicated to " + DATANODE_NUM + " datanodes.",
-                 actualRepl == DATANODE_NUM);
+      assertTrue(actualRepl == DATANODE_NUM, f + " should be replicated to " +
+          DATANODE_NUM + " datanodes.");
 
       // set the soft and hard limit to be 1 second so that the
       // namenode triggers lease recovery
@@ -1110,7 +1111,7 @@ public class TestFileCreation {
         }
       }
       System.out.println("successcount=" + successcount);
-      assertTrue(successcount > 0); 
+      assertTrue(successcount > 0);
     } finally {
       IOUtils.closeStream(dfs);
       cluster.shutdown();
@@ -1186,7 +1187,7 @@ public class TestFileCreation {
       } catch (IOException e) {
         hasException = true;
       }
-      assertTrue("Failed to close file after cluster shutdown", hasException);
+      assertTrue(hasException, "Failed to close file after cluster shutdown");
     } finally {
       System.out.println("testFsCloseAfterClusterShutdown successful");
       if (cluster != null) {
@@ -1321,7 +1322,8 @@ public class TestFileCreation {
    * 2. Restart NN, check the file
    * 3. Save new checkpoint and restart NN, check the file
    */
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testFileCreationWithOverwrite() throws Exception {
     Configuration conf = new Configuration();
     conf.setInt("dfs.blocksize", blockSize);
@@ -1376,7 +1378,7 @@ public class TestFileCreation {
       } finally {
         in.close();
       }
-      Assert.assertArrayEquals(newData, result);
+      assertArrayEquals(newData, result);
       
       // Case 2: Restart NN, check the file
       cluster.restartNameNode();
@@ -1387,7 +1389,7 @@ public class TestFileCreation {
       } finally {
         in.close();
       }
-      Assert.assertArrayEquals(newData, result);
+      assertArrayEquals(newData, result);
       
       // Case 3: Save new checkpoint and restart NN, check the file
       NameNodeAdapter.enterSafeMode(nn, false);
@@ -1401,7 +1403,7 @@ public class TestFileCreation {
       } finally {
         in.close();
       }
-      Assert.assertArrayEquals(newData, result);
+      assertArrayEquals(newData, result);
     } finally {
       if (dfs != null) {
         dfs.close();

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationClient.java

@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -30,7 +30,7 @@ import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.slf4j.event.Level;
 
 /**

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationDelete.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 package org.apache.hadoop.hdfs;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.IOException;
 
@@ -24,7 +24,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.slf4j.event.Level;
 
 public class TestFileCreationDelete {

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationEmpty.java

@@ -16,14 +16,14 @@
  * limitations under the License.
  */
 package org.apache.hadoop.hdfs;
-import static org.junit.Assert.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertFalse;
 
 import java.util.ConcurrentModificationException;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 /**
  * Test empty file creation.

+ 11 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileLengthOnClusterRestart.java

@@ -23,8 +23,12 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 /** Test the fileLength on cluster restarts */
 public class TestFileLengthOnClusterRestart {
@@ -32,7 +36,8 @@ public class TestFileLengthOnClusterRestart {
    * Tests the fileLength when we sync the file and restart the cluster and
    * Datanodes not report to Namenode yet.
    */
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testFileLengthWithHSyncAndClusterRestartWithOutDNsRegister()
       throws Exception {
     final Configuration conf = new HdfsConfiguration();
@@ -55,7 +60,7 @@ public class TestFileLengthOnClusterRestart {
       in = (HdfsDataInputStream) dfs.open(path, 1024);
       // Verify the length when we just restart NN. DNs will register
       // immediately.
-      Assert.assertEquals(fileLength, in.getVisibleLength());
+      assertEquals(fileLength, in.getVisibleLength());
       cluster.shutdownDataNodes();
       cluster.restartNameNode(false);
       // This is just for ensuring NN started.
@@ -63,9 +68,9 @@ public class TestFileLengthOnClusterRestart {
 
       try {
         in = (HdfsDataInputStream) dfs.open(path);
-        Assert.fail("Expected IOException");
+        fail("Expected IOException");
       } catch (IOException e) {
-        Assert.assertTrue(e.getLocalizedMessage().indexOf(
+        assertTrue(e.getLocalizedMessage().indexOf(
             "Name node is in safe mode") >= 0);
       }
     } finally {

+ 41 - 41
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java

@@ -17,10 +17,10 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -38,9 +38,9 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 import org.slf4j.event.Level;
 
 /**
@@ -63,7 +63,7 @@ public class TestFileStatus {
   private static DFSClient dfsClient;
   private static Path file1;
   
-  @BeforeClass
+  @BeforeAll
   public static void testSetUp() throws Exception {
     conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_LIST_LIMIT, 2);
@@ -76,7 +76,7 @@ public class TestFileStatus {
         seed);
   }
   
-  @AfterClass
+  @AfterAll
   public static void testTearDown() throws Exception {
     if (fs != null) {
       fs.close();
@@ -96,13 +96,13 @@ public class TestFileStatus {
   public void testGetFileInfo() throws IOException {
     // Check that / exists
     Path path = new Path("/");
-    assertTrue("/ should be a directory", 
-               fs.getFileStatus(path).isDirectory());
+    assertTrue(
+               fs.getFileStatus(path).isDirectory(), "/ should be a directory");
     ContractTestUtils.assertNotErasureCoded(fs, path);
 
     // Make sure getFileInfo returns null for files which do not exist
     HdfsFileStatus fileInfo = dfsClient.getFileInfo("/noSuchFile");
-    assertEquals("Non-existant file should result in null", null, fileInfo);
+    assertEquals(null, fileInfo, "Non-existant file should result in null");
     
     Path path1 = new Path("/name1");
     Path path2 = new Path("/name1/name2");
@@ -119,8 +119,8 @@ public class TestFileStatus {
       dfsClient.getFileInfo("non-absolute");
       fail("getFileInfo for a non-absolute path did not throw IOException");
     } catch (RemoteException re) {
-      assertTrue("Wrong exception for invalid file name: "+re,
-          re.toString().contains("Absolute path required"));
+      assertTrue(re.toString().contains("Absolute path required"),
+          "Wrong exception for invalid file name: " + re);
     }
   }
 
@@ -131,7 +131,7 @@ public class TestFileStatus {
     checkFile(fs, file1, 1);
     // test getFileStatus on a file
     FileStatus status = fs.getFileStatus(file1);
-    assertFalse(file1 + " should be a file", status.isDirectory());
+    assertFalse(status.isDirectory(), file1 + " should be a file");
     assertEquals(blockSize, status.getBlockSize());
     assertEquals(1, status.getReplication());
     assertEquals(fileSize, status.getLen());
@@ -139,9 +139,9 @@ public class TestFileStatus {
     assertEquals(file1.makeQualified(fs.getUri(),
         fs.getWorkingDirectory()).toString(), 
         status.getPath().toString());
-    assertTrue(file1 + " should have erasure coding unset in " +
-            "FileStatus#toString(): " + status,
-        status.toString().contains("isErasureCoded=false"));
+    assertTrue(status.toString().contains("isErasureCoded=false"),
+        file1 + " should have erasure coding unset in " +
+            "FileStatus#toString(): " + status);
   }
 
   /** Test the FileStatus obtained calling listStatus on a file */
@@ -150,7 +150,7 @@ public class TestFileStatus {
     FileStatus[] stats = fs.listStatus(file1);
     assertEquals(1, stats.length);
     FileStatus status = stats[0];
-    assertFalse(file1 + " should be a file", status.isDirectory());
+    assertFalse(status.isDirectory(), file1 + " should be a file");
     assertEquals(blockSize, status.getBlockSize());
     assertEquals(1, status.getReplication());
     assertEquals(fileSize, status.getLen());
@@ -162,7 +162,7 @@ public class TestFileStatus {
     RemoteIterator<FileStatus> itor = fc.listStatus(file1);
     status = itor.next();
     assertEquals(stats[0], status);
-    assertFalse(file1 + " should be a file", status.isDirectory());
+    assertFalse(status.isDirectory(), file1 + " should be a file");
   }
 
   /** Test getting a FileStatus object using a non-existant path */
@@ -186,8 +186,8 @@ public class TestFileStatus {
       fs.getFileStatus(dir);
       fail("getFileStatus of non-existent path should fail");
     } catch (FileNotFoundException fe) {
-      assertTrue("Exception doesn't indicate non-existant path", 
-          fe.getMessage().startsWith("File does not exist"));
+      assertTrue(fe.getMessage().startsWith("File does not exist"),
+          "Exception doesn't indicate non-existant path");
     }
   }
 
@@ -196,13 +196,13 @@ public class TestFileStatus {
   public void testGetFileStatusOnDir() throws Exception {
     // Create the directory
     Path dir = new Path("/test/mkdirs");
-    assertTrue("mkdir failed", fs.mkdirs(dir));
-    assertTrue("mkdir failed", fs.exists(dir));
+    assertTrue(fs.mkdirs(dir), "mkdir failed");
+    assertTrue(fs.exists(dir), "mkdir failed");
     
     // test getFileStatus on an empty directory
     FileStatus status = fs.getFileStatus(dir);
-    assertTrue(dir + " should be a directory", status.isDirectory());
-    assertTrue(dir + " should be zero size ", status.getLen() == 0);
+    assertTrue(status.isDirectory(), dir + " should be a directory");
+    assertTrue(status.getLen() == 0, dir + " should be zero size ");
     ContractTestUtils.assertNotErasureCoded(fs, dir);
     assertEquals(dir.makeQualified(fs.getUri(),
         fs.getWorkingDirectory()).toString(), 
@@ -210,15 +210,15 @@ public class TestFileStatus {
     
     // test listStatus on an empty directory
     FileStatus[] stats = fs.listStatus(dir);
-    assertEquals(dir + " should be empty", 0, stats.length);
-    assertEquals(dir + " should be zero size ",
-        0, fs.getContentSummary(dir).getLength());
+    assertEquals(0, stats.length, dir + " should be empty");
+    assertEquals(0, fs.getContentSummary(dir).getLength(),
+        dir + " should be zero size ");
     
     RemoteIterator<FileStatus> itor = fc.listStatus(dir);
-    assertFalse(dir + " should be empty", itor.hasNext());
+    assertFalse(itor.hasNext(), dir + " should be empty");
 
     itor = fs.listStatusIterator(dir);
-    assertFalse(dir + " should be empty", itor.hasNext());
+    assertFalse(itor.hasNext(), dir + " should be empty");
 
     // create another file that is smaller than a block.
     Path file2 = new Path(dir, "filestatus2.dat");
@@ -242,25 +242,25 @@ public class TestFileStatus {
 
     // Verify that the size of the directory increased by the size 
     // of the two files
-    final int expected = blockSize/2;  
-    assertEquals(dir + " size should be " + expected, 
-        expected, fs.getContentSummary(dir).getLength());
+    final int expected = blockSize/2;
+    assertEquals(expected, fs.getContentSummary(dir).getLength(),
+        dir + " size should be " + expected);
 
     // Test listStatus on a non-empty directory
     stats = fs.listStatus(dir);
-    assertEquals(dir + " should have two entries", 2, stats.length);
+    assertEquals(2, stats.length, dir + " should have two entries");
     assertEquals(file2.toString(), stats[0].getPath().toString());
     assertEquals(file3.toString(), stats[1].getPath().toString());
 
     itor = fc.listStatus(dir);
     assertEquals(file2.toString(), itor.next().getPath().toString());
     assertEquals(file3.toString(), itor.next().getPath().toString());
-    assertFalse("Unexpected addtional file", itor.hasNext());
+    assertFalse(itor.hasNext(), "Unexpected addtional file");
 
     itor = fs.listStatusIterator(dir);
     assertEquals(file2.toString(), itor.next().getPath().toString());
     assertEquals(file3.toString(), itor.next().getPath().toString());
-    assertFalse("Unexpected addtional file", itor.hasNext());
+    assertFalse(itor.hasNext(), "Unexpected addtional file");
 
 
     // Test iterative listing. Now dir has 2 entries, create one more.
@@ -268,7 +268,7 @@ public class TestFileStatus {
     fs.mkdirs(dir3);
     dir3 = fs.makeQualified(dir3);
     stats = fs.listStatus(dir);
-    assertEquals(dir + " should have three entries", 3, stats.length);
+    assertEquals(3, stats.length, dir + " should have three entries");
     assertEquals(dir3.toString(), stats[0].getPath().toString());
     assertEquals(file2.toString(), stats[1].getPath().toString());
     assertEquals(file3.toString(), stats[2].getPath().toString());
@@ -277,13 +277,13 @@ public class TestFileStatus {
     assertEquals(dir3.toString(), itor.next().getPath().toString());
     assertEquals(file2.toString(), itor.next().getPath().toString());
     assertEquals(file3.toString(), itor.next().getPath().toString());
-    assertFalse("Unexpected addtional file", itor.hasNext());
+    assertFalse(itor.hasNext(), "Unexpected addtional file");
 
     itor = fs.listStatusIterator(dir);
     assertEquals(dir3.toString(), itor.next().getPath().toString());
     assertEquals(file2.toString(), itor.next().getPath().toString());
     assertEquals(file3.toString(), itor.next().getPath().toString());
-    assertFalse("Unexpected addtional file", itor.hasNext());
+    assertFalse(itor.hasNext(), "Unexpected addtional file");
 
     // Now dir has 3 entries, create two more
     Path dir4 = fs.makeQualified(new Path(dir, "dir4"));
@@ -293,7 +293,7 @@ public class TestFileStatus {
     fs.mkdirs(dir5);
     dir5 = fs.makeQualified(dir5);
     stats = fs.listStatus(dir);
-    assertEquals(dir + " should have five entries", 5, stats.length);
+    assertEquals(5, stats.length, dir + " should have five entries");
     assertEquals(dir3.toString(), stats[0].getPath().toString());
     assertEquals(dir4.toString(), stats[1].getPath().toString());
     assertEquals(dir5.toString(), stats[2].getPath().toString());

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusSerialization.java

@@ -36,8 +36,8 @@ import org.apache.hadoop.io.DataOutputBuffer;
 
 import org.apache.hadoop.thirdparty.protobuf.ByteString;
 
-import org.junit.Test;
-import static org.junit.Assert.*;
+import org.junit.jupiter.api.Test;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 /**
  * Verify compatible FileStatus/HdfsFileStatus serialization.

+ 12 - 16
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithDefaultECPolicy.java

@@ -17,9 +17,9 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.IOException;
 
@@ -28,24 +28,21 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 /**
  * This test ensures the statuses of EC files with the default policy.
  */
+@Timeout(300)
 public class TestFileStatusWithDefaultECPolicy {
   private MiniDFSCluster cluster;
   private DistributedFileSystem fs;
   private DFSClient client;
 
-  @Rule
-  public Timeout globalTimeout = new Timeout(300000);
-
-  @Before
+  @BeforeEach
   public void before() throws IOException {
     HdfsConfiguration conf = new HdfsConfiguration();
     cluster =
@@ -56,7 +53,7 @@ public class TestFileStatusWithDefaultECPolicy {
     fs.enableErasureCodingPolicy(getEcPolicy().getName());
   }
 
-  @After
+  @AfterEach
   public void after() {
     if (cluster != null) {
       cluster.shutdown();
@@ -100,8 +97,7 @@ public class TestFileStatusWithDefaultECPolicy {
     assertTrue(ecPolicy1.equals(ecPolicy3));
     ContractTestUtils.assertErasureCoded(fs, file);
     FileStatus status = fs.getFileStatus(file);
-    assertTrue(file + " should have erasure coding set in " +
-            "FileStatus#toString(): " + status,
-        status.toString().contains("isErasureCoded=true"));
+    assertTrue(status.toString().contains("isErasureCoded=true"),
+        file + " should have erasure coding set in " + "FileStatus#toString(): " + status);
   }
 }

Деякі файли не було показано, через те що забагато файлів було змінено