Selaa lähdekoodia

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

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

Co-authored-by: Chris Nauroth <cnauroth@apache.org>
Co-authored-by: Shilun Fan <slfan1989@apache.org>
Reviewed-by: Chris Nauroth <cnauroth@apache.org>
Reviewed-by: Shilun Fan <slfan1989@apache.org>
Signed-off-by: Shilun Fan <slfan1989@apache.org>
zhtttylz 18 tuntia sitten
vanhempi
commit
b68dc8a83b
41 muutettua tiedostoa jossa 892 lisäystä ja 868 poistoa
  1. 10 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPipelines.java
  2. 38 31
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
  3. 77 82
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java
  4. 24 24
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuotaAllowOwner.java
  5. 24 18
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRead.java
  6. 13 17
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDNFailure.java
  7. 20 23
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
  8. 8 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java
  9. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadWhileWriting.java
  10. 66 50
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReconstructStripedFile.java
  11. 7 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReconstructStripedFileWithValidator.java
  12. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRenameWhileOpen.java
  13. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplaceDatanodeFailureReplication.java
  14. 13 13
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplaceDatanodeOnFailure.java
  15. 16 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java
  16. 7 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRestartDFS.java
  17. 84 73
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java
  18. 64 59
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgradeDowngrade.java
  19. 20 21
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgradeRollback.java
  20. 49 51
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
  21. 10 13
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java
  22. 14 17
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSecureEncryptionZoneWithKMS.java
  23. 78 74
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSeekBug.java
  24. 13 13
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSetTimes.java
  25. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSetrepDecreasing.java
  26. 11 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSetrepIncreasing.java
  27. 21 21
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSlowDatanodeReport.java
  28. 6 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSmallBlock.java
  29. 14 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSnapshotCommands.java
  30. 14 12
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestStateAlignmentContextWithHA.java
  31. 7 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestStoragePolicyPermissionSettings.java
  32. 10 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestStripedFileAppend.java
  33. 9 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestTrashWithEncryptionZones.java
  34. 33 38
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestTrashWithSecureEncryptionZones.java
  35. 46 53
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestUnsetAndChangeDirectoryEcPolicy.java
  36. 13 16
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestViewDistributedFileSystem.java
  37. 4 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteBlockGetsBlockLengthHint.java
  38. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteConfigurationToDFS.java
  39. 9 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteRead.java
  40. 10 14
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java
  41. 10 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java

+ 10 - 11
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPipelines.java

@@ -17,8 +17,8 @@
  */
 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.IOException;
 import java.util.List;
@@ -35,9 +35,9 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.Replica;
 import org.apache.hadoop.test.GenericTestUtils;
-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.event.Level;
 
 public class TestPipelines {
@@ -57,13 +57,13 @@ public class TestPipelines {
     setConfiguration();
   }
 
-  @Before
+  @BeforeEach
   public void startUpCluster() throws IOException {
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPL_FACTOR).build();
     fs = cluster.getFileSystem();
   }
 
-  @After
+  @AfterEach
   public void shutDownCluster() throws IOException {
     if (fs != null) {
       fs.close();
@@ -108,10 +108,9 @@ public class TestPipelines {
       Replica r =
           cluster.getFsDatasetTestUtils(dn).fetchReplica(lb.get(0).getBlock());
 
-      assertTrue("Replica on DN " + dn + " shouldn't be null", r != null);
-      assertEquals("Should be RBW replica on " + dn
-          + " after sequence of calls append()/write()/hflush()",
-          HdfsServerConstants.ReplicaState.RBW, r.getState());
+      assertTrue(r != null, "Replica on DN " + dn + " shouldn't be null");
+      assertEquals(HdfsServerConstants.ReplicaState.RBW, r.getState(),
+          "Should be RBW replica on " + dn + " after sequence of calls append()/write()/hflush()");
     }
     ofs.close();
   }

+ 38 - 31
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java

@@ -18,9 +18,9 @@
 package org.apache.hadoop.hdfs;
 
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY;
-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.EOFException;
 import java.io.IOException;
@@ -53,10 +53,11 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
@@ -82,11 +83,13 @@ public class TestPread {
       LoggerFactory.getLogger(TestPread.class.getName());
   private final GenericTestUtils.LogCapturer dfsClientLog =
       GenericTestUtils.LogCapturer.captureLogs(DFSClient.LOG);
-  @BeforeClass
+
+  @BeforeAll
   public static void setLogLevel() {
     GenericTestUtils.setLogLevel(DFSClient.LOG, org.apache.log4j.Level.WARN);
   }
-  @Before
+
+  @BeforeEach
   public void setup() {
     simulatedStorage = false;
     isHedgedRead = false;
@@ -107,10 +110,10 @@ public class TestPread {
       // should throw an exception
       res = e;
     }
-    assertTrue("Error reading beyond file boundary.", res != null);
+    assertTrue(res != null, "Error reading beyond file boundary.");
     in.close();
     if (!fileSys.delete(name, true))
-      assertTrue("Cannot delete file", false);
+      assertTrue(false, "Cannot delete file");
     
     // now create the real file
     DFSTestUtil.createFile(fileSys, name, fileSize, fileSize,
@@ -119,9 +122,9 @@ public class TestPread {
   
   private void checkAndEraseData(byte[] actual, int from, byte[] expected, String message) {
     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;
     }
   }
@@ -140,17 +143,17 @@ public class TestPread {
     while (nread < length) {
       int nbytes =
           stm.read(position + nread, buffer, offset + nread, length - nread);
-      assertTrue("Error in pread", nbytes > 0);
+      assertTrue(nbytes > 0, "Error in pread");
       nread += nbytes;
     }
 
     if (dfstm != null) {
       if (isHedgedRead) {
-        assertTrue("Expected read statistic to be incremented", length <= dfstm
-            .getReadStatistics().getTotalBytesRead() - totalRead);
+        assertTrue(length <= dfstm.getReadStatistics().getTotalBytesRead() - totalRead,
+            "Expected read statistic to be incremented");
       } else {
-        assertEquals("Expected read statistic to be incremented", length, dfstm
-            .getReadStatistics().getTotalBytesRead() - totalRead);
+        assertEquals(length, dfstm.getReadStatistics().getTotalBytesRead() - totalRead,
+            "Expected read statistic to be incremented");
       }
     }
   }
@@ -221,7 +224,7 @@ public class TestPread {
       // should throw an exception
       res = e;
     }
-    assertTrue("Error reading beyond file boundary.", res != null);
+    assertTrue(res != null, "Error reading beyond file boundary.");
     
     stm.close();
   }
@@ -553,9 +556,9 @@ public class TestPread {
       });
       try {
         future.get(4, TimeUnit.SECONDS);
-        Assert.fail();
+        Assertions.fail();
       } catch (ExecutionException ee) {
-        assertTrue(ee.toString(), ee.getCause() instanceof EOFException);
+        assertTrue(ee.getCause() instanceof EOFException, ee.toString());
       } finally {
         future.cancel(true);
         executor.shutdown();
@@ -570,7 +573,8 @@ public class TestPread {
    * retrying on a different datanode or by refreshing data nodes and retrying each data node one
    * more time.
    */
-  @Test(timeout=120000)
+  @Test
+  @Timeout(value = 120)
   public void testGetFromOneDataNodeExceptionLogging() throws IOException {
     // With maxBlockAcquireFailures = 0, we would try on each datanode only once and if
     // we fail on all three datanodes, we fail the read request.
@@ -647,7 +651,8 @@ public class TestPread {
   /**
    * Test the case where we always hit IOExceptions, causing the read request to fail.
    */
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testFetchFromDataNodeExceptionLoggingFailedRequest()
       throws IOException {
     testFetchFromDataNodeExceptionLoggingFailedRequest(0);
@@ -723,7 +728,8 @@ public class TestPread {
     }
   }
 
-  @Test(timeout=30000)
+  @Test
+  @Timeout(value = 30)
   public void testHedgedReadFromAllDNFailed() throws IOException {
     Configuration conf = new Configuration();
     int numHedgedReadPoolThreads = 5;
@@ -768,7 +774,7 @@ public class TestPread {
       byte[] buffer = new byte[64 * 1024];
       input = dfsClient.open(filename);
       input.read(0, buffer, 0, 1024);
-      Assert.fail("Reading the block should have thrown BlockMissingException");
+      Assertions.fail("Reading the block should have thrown BlockMissingException");
     } catch (BlockMissingException e) {
       // The result of 9 is due to 2 blocks by 4 iterations plus one because
       // hedgedReadOpsLoopNumForTesting is incremented at start of the loop.
@@ -808,7 +814,8 @@ public class TestPread {
    * 7. Consider next calls to getBlockLocations() always returns DN3 as last
    * location.<br>
    */
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testPreadHedgedFailureWithChangedBlockLocations()
       throws Exception {
     isHedgedRead = true;
@@ -929,10 +936,10 @@ public class TestPread {
       byte[] buf = new byte[1024];
       int n = din.read(0, buf, 0, data.length());
       assertEquals(data.length(), n);
-      assertEquals("Data should be read", data, new String(buf, 0, n));
-      assertTrue("Read should complete with maximum " + maxFailures
-              + " failures, but completed with " + din.failures,
-          din.failures <= maxFailures);
+      assertEquals(data, new String(buf, 0, n), "Data should be read");
+      assertTrue(din.failures <= maxFailures,
+          "Read should complete with maximum " + maxFailures
+              + " failures, but completed with " + din.failures);
       DFSClient.LOG.info("Read completed");
     }
   }

+ 77 - 82
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java

@@ -18,14 +18,11 @@
 package org.apache.hadoop.hdfs;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
-import static org.hamcrest.CoreMatchers.allOf;
-import static org.hamcrest.CoreMatchers.containsString;
-import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-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.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.assertj.core.api.Assertions.assertThat;
 
 import java.io.ByteArrayOutputStream;
 import java.io.FileNotFoundException;
@@ -61,18 +58,18 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.Lists;
 import org.apache.hadoop.util.ToolRunner;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
-import org.junit.rules.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.event.Level;
 import org.slf4j.LoggerFactory;
 
 /** A class for testing quota-related commands */
+@Timeout(120)
 public class TestQuota {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestQuota.class);
@@ -88,10 +85,7 @@ public class TestQuota {
   /* set a smaller block size so that we can test with smaller space quotas */
   private static final int DEFAULT_BLOCK_SIZE = 512;
 
-  @Rule
-  public final Timeout testTestout = new Timeout(120000);
-
-  @BeforeClass
+  @BeforeAll
   public static void setUpClass() throws Exception {
     conf = new HdfsConfiguration();
     conf.set(
@@ -133,7 +127,7 @@ public class TestQuota {
     ERR_STREAM.reset();
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDownClass() {
     try {
       System.out.flush();
@@ -289,7 +283,7 @@ public class TestQuota {
     try {
       fout.write(new byte[fileLen]);
       fout.close();
-      Assert.fail();
+      Assertions.fail();
     } catch (QuotaExceededException e) {
       IOUtils.closeStream(fout);
     }
@@ -397,8 +391,8 @@ public class TestQuota {
     ugi.doAs(new PrivilegedExceptionAction<Object>() {
       @Override
       public Object run() throws Exception {
-        assertEquals("Not running as new user", username,
-            UserGroupInformation.getCurrentUser().getShortUserName());
+        assertEquals(username, UserGroupInformation.getCurrentUser().getShortUserName(),
+            "Not running as new user");
         DFSAdmin userAdmin = new DFSAdmin(conf);
 
         args2[1] = "100";
@@ -981,20 +975,20 @@ public class TestQuota {
 
     DFSTestUtil.createFile(dfs, wow, size, repl, 0);
     DFSTestUtil.createFile(dfs, wow2, size, repl, 0);
-    assertTrue("Without storage policy, typeConsumed should be 0.",
-        dfs.getQuotaUsage(bs1).getTypeConsumed(StorageType.SSD) == 0);
-    assertTrue("With storage policy, typeConsumed should not be 0.",
-        dfs.getQuotaUsage(bs2).getTypeConsumed(StorageType.SSD) != 0);
+    assertTrue(dfs.getQuotaUsage(bs1).getTypeConsumed(StorageType.SSD) == 0,
+        "Without storage policy, typeConsumed should be 0.");
+    assertTrue(dfs.getQuotaUsage(bs2).getTypeConsumed(StorageType.SSD) != 0,
+        "With storage policy, typeConsumed should not be 0.");
     // wow3 without storage policy , rename will not change typeConsumed
     dfs.rename(wow3, bs1);
-    assertTrue("Rename src without storagePolicy, dst typeConsumed should not be changed.",
-        dfs.getQuotaUsage(bs2).getTypeConsumed(StorageType.SSD) == 0);
+    assertTrue(dfs.getQuotaUsage(bs2).getTypeConsumed(StorageType.SSD) == 0,
+        "Rename src without storagePolicy, dst typeConsumed should not be changed.");
 
     long srcTypeQuota = dfs.getQuotaUsage(bs2).getTypeQuota(StorageType.SSD);
     dfs.rename(bs2, bs1);
     long dstTypeQuota = dfs.getQuotaUsage(bs1).getTypeConsumed(StorageType.SSD);
-    assertTrue("Rename with storage policy, typeConsumed should not be 0.",
-        dstTypeQuota != srcTypeQuota);
+    assertTrue(dstTypeQuota != srcTypeQuota,
+        "Rename with storage policy, typeConsumed should not be 0.");
   }
 
   private static void checkContentSummary(final ContentSummary expected,
@@ -1013,8 +1007,7 @@ public class TestQuota {
     assertTrue(dfs.mkdirs(parent));
 
     final FileSystem fs = cluster.getFileSystem();
-    assertTrue("Not a HDFS: "+fs.getUri(),
-                fs instanceof DistributedFileSystem);
+    assertTrue(fs instanceof DistributedFileSystem, "Not a HDFS: " + fs.getUri());
     final DistributedFileSystem dfs = (DistributedFileSystem)fs;
 
     // create test directory
@@ -1025,25 +1018,25 @@ public class TestQuota {
     dfs.setQuota(testFolder, Long.MAX_VALUE - 1, 10);
     ContentSummary c = dfs.getContentSummary(testFolder);
     compareQuotaUsage(c, dfs, testFolder);
-    assertTrue("Quota not set properly", c.getQuota() == Long.MAX_VALUE - 1);
+    assertTrue(c.getQuota() == Long.MAX_VALUE - 1, "Quota not set properly");
 
     // setting diskspace quota to Long.MAX_VALUE - 1 should work
     dfs.setQuota(testFolder, 10, Long.MAX_VALUE - 1);
     c = dfs.getContentSummary(testFolder);
     compareQuotaUsage(c, dfs, testFolder);
-    assertTrue("Quota not set properly", c.getSpaceQuota() == Long.MAX_VALUE - 1);
+    assertTrue(c.getSpaceQuota() == Long.MAX_VALUE - 1, "Quota not set properly");
 
     // setting namespace quota to Long.MAX_VALUE should not work + no error
     dfs.setQuota(testFolder, Long.MAX_VALUE, 10);
     c = dfs.getContentSummary(testFolder);
     compareQuotaUsage(c, dfs, testFolder);
-    assertTrue("Quota should not have changed", c.getQuota() == 10);
+    assertTrue(c.getQuota() == 10, "Quota should not have changed");
 
     // setting diskspace quota to Long.MAX_VALUE should not work + no error
     dfs.setQuota(testFolder, 10, Long.MAX_VALUE);
     c = dfs.getContentSummary(testFolder);
     compareQuotaUsage(c, dfs, testFolder);
-    assertTrue("Quota should not have changed", c.getSpaceQuota() == 10);
+    assertTrue(c.getSpaceQuota() == 10, "Quota should not have changed");
 
     // setting namespace quota to Long.MAX_VALUE + 1 should not work + error
     try {
@@ -1096,8 +1089,7 @@ public class TestQuota {
     c = dfs.getContentSummary(dir);
     compareQuotaUsage(c, dfs, dir);
     checkContentSummary(c, webhdfs.getContentSummary(dir));
-    assertEquals("Quota is half consumed", QUOTA_SIZE / 2,
-                 c.getSpaceConsumed());
+    assertEquals(QUOTA_SIZE / 2, c.getSpaceConsumed(), "Quota is half consumed");
 
     // We can not create the 2nd file because even though the total spaced
     // used by two files (2 * 3 * 512/2) would fit within the quota (3 * 512)
@@ -1110,7 +1102,7 @@ public class TestQuota {
     } catch (QuotaExceededException e) {
       exceededQuota = true;
     }
-    assertTrue("Quota not exceeded", exceededQuota);
+    assertTrue(exceededQuota, "Quota not exceeded");
  }
 
  /**
@@ -1148,8 +1140,8 @@ public class TestQuota {
       long nsQuota = FSImageTestUtil.getNSQuota(dfsCluster.getNameNode()
           .getNamesystem());
       assertTrue(
-          "Default namespace quota expected as long max. But the value is :"
-              + nsQuota, nsQuota == Long.MAX_VALUE);
+          nsQuota == Long.MAX_VALUE,
+          "Default namespace quota expected as long max. But the value is :" + nsQuota);
       
       Path dir = new Path(parent, "test");
       boolean exceededQuota = false;
@@ -1184,10 +1176,10 @@ public class TestQuota {
       c = fs.getContentSummary(dir);
       compareQuotaUsage(c, fs, dir);
       checkContentSummary(c, webHDFS.getContentSummary(dir));
-      assertEquals("Invalid space consumed", 59 * FILE_SIZE * 3,
-          c.getSpaceConsumed());
-      assertEquals("Invalid space consumed", QUOTA_SIZE - (59 * FILE_SIZE * 3),
-          3 * (fs.getDefaultBlockSize(dir) - FILE_SIZE));
+      assertEquals(59 * FILE_SIZE * 3, c.getSpaceConsumed(),
+          "Invalid space consumed");
+      assertEquals(QUOTA_SIZE - (59 * FILE_SIZE * 3), 3 * (fs.getDefaultBlockSize(dir) - FILE_SIZE),
+          "Invalid space consumed");
 
       // Now check that trying to create another file violates the quota
       try {
@@ -1197,7 +1189,7 @@ public class TestQuota {
       } catch (QuotaExceededException e) {
         exceededQuota = true;
       }
-      assertTrue("Quota not exceeded", exceededQuota);
+      assertTrue(exceededQuota, "Quota not exceeded");
       assertEquals(2, dfsCluster.getNamesystem().getFSDirectory().getYieldCount());
     } finally {
       dfsCluster.shutdown();
@@ -1259,7 +1251,8 @@ public class TestQuota {
   /**
    * Test to set space quote using negative number.
    */
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testSetSpaceQuotaNegativeNumber() throws Exception {
 
     final DFSAdmin dfsAdmin = new DFSAdmin(conf);
@@ -1279,19 +1272,20 @@ public class TestQuota {
     assertEquals(-1, ret);
     scanIntoList(ERR_STREAM, outs);
     assertEquals(
+        2, outs.size(),
         "It should be two lines of error messages,"
-        + " the 1st one is about Illegal option,"
-        + " the 2nd one is about SetSpaceQuota usage.",
-        2, outs.size());
-    assertThat(outs.get(0),
-        is(allOf(containsString("setSpaceQuota"),
-            containsString("Illegal option"))));
+            + " the 1st one is about Illegal option,"
+            + " the 2nd one is about SetSpaceQuota usage.");
+    assertThat(outs.get(0))
+        .contains("setSpaceQuota")
+        .contains("Illegal option");
   }
 
   /**
    * Test to set and clear space quote, regular usage.
    */
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testSetAndClearSpaceQuotaRegular() throws Exception {
 
     final Path dir = new Path(
@@ -1317,7 +1311,8 @@ public class TestQuota {
   /**
    * Test to all the commands by passing the fully qualified path.
    */
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testQuotaCommandsWithURI() throws Exception {
     DFSAdmin dfsAdmin = new DFSAdmin(conf);
     final Path dir = new Path("/" + this.getClass().getSimpleName(),
@@ -1355,15 +1350,15 @@ public class TestQuota {
     final QuotaUsage quotaUsage = dfs.getQuotaUsage(dir);
     assertEquals(spaceQuota, quotaUsage.getSpaceQuota());
     scanIntoList(OUT_STREAM, outs);
-    assertTrue(
-        "There should be no output if it runs successfully.",
-        outs.isEmpty());
+    assertTrue(outs.isEmpty(),
+        "There should be no output if it runs successfully.");
   }
 
   /**
    * Test to set and clear space quote by storage type.
    */
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testSetAndClearSpaceQuotaByStorageType() throws Exception {
 
     final Path dir = new Path(
@@ -1412,14 +1407,15 @@ public class TestQuota {
         quotaUsage.getTypeQuota(StorageType.DISK));
     scanIntoList(OUT_STREAM, outs);
     assertTrue(
-        "There should be no output if it runs successfully.",
-        outs.isEmpty());
+        outs.isEmpty(),
+        "There should be no output if it runs successfully.");
   }
 
   /**
    * Test to set and clear space quote when directory doesn't exist.
    */
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testSetAndClearSpaceQuotaDirectoryNotExist() throws Exception {
     final Path dir = new Path(
         PathUtils.getTestDir(getClass()).getPath(),
@@ -1453,20 +1449,18 @@ public class TestQuota {
     final int ret = ToolRunner.run(dfsAdmin, args);
     assertEquals(cmdRet, ret);
     scanIntoList(ERR_STREAM, outs);
-    assertEquals(
-        "It should be one line error message like: clrSpaceQuota:"
-            + " Directory does not exist: <full path of XXX directory>",
-        1, outs.size());
-    assertThat(outs.get(0),
-        is(allOf(containsString(cmdName),
-            containsString("does not exist"),
-            containsString(dir.toString()))));
+    assertEquals(1, outs.size(), "It should be one line error message like: clrSpaceQuota:"
+        + " Directory does not exist: <full path of XXX directory>");
+    assertThat(outs.get(0)).contains(cmdName)
+        .contains("does not exist")
+        .contains(dir.toString());
   }
 
   /**
    * Test to set and clear space quote when path is a file.
    */
-  @Test (timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testSetAndClearSpaceQuotaPathIsFile() throws Exception {
 
     final Path parent = new Path(
@@ -1505,19 +1499,20 @@ public class TestQuota {
     assertEquals(cmdRet, ret);
     scanIntoList(ERR_STREAM, outs);
     assertEquals(
+        1, outs.size(),
         "It should be one line error message like: clrSpaceQuota:"
-            + " <full path of XXX file> is not a directory",
-        1, outs.size());
-    assertThat(outs.get(0),
-        is(allOf(containsString(cmdName),
-            containsString(file.toString()),
-            containsString("Is not a directory"))));
+            + " <full path of XXX file> is not a directory");
+    assertThat(outs.get(0))
+        .contains(cmdName)
+        .contains(file.toString())
+        .contains("Is not a directory");
   }
 
   /**
    * Test to set and clear space quote when user has no access right.
    */
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testSetAndClearSpaceQuotaNoAccess() throws Exception {
 
     final Path dir = new Path(
@@ -1709,10 +1704,10 @@ public class TestQuota {
     });
     assertEquals(cmdRet, ret);
     scanIntoList(ERR_STREAM, outs);
-    assertThat(outs.get(0),
-        is(allOf(containsString(cmdName),
-            containsString("Access denied for user whoever"),
-            containsString("Superuser privilege is required"))));
+    assertThat(outs.get(0))
+        .contains(cmdName)
+        .contains("Access denied for user whoever")
+        .contains("Superuser privilege is required");
   }
 
   private static void scanIntoList(

+ 24 - 24
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuotaAllowOwner.java

@@ -23,22 +23,22 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class TestQuotaAllowOwner {
   private static Configuration conf;
   private static MiniDFSCluster cluster;
   private static DistributedFileSystem dfs;
 
-  @BeforeClass
+  @BeforeAll
   public static void setUpClass() throws Exception {
     conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 512);
@@ -47,7 +47,7 @@ public class TestQuotaAllowOwner {
     restartCluster();
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDownClass() {
     if (cluster != null) {
       cluster.shutdown();
@@ -96,8 +96,8 @@ public class TestQuotaAllowOwner {
     UserGroupInformation ugi = UserGroupInformation.createUserForTesting(
         userName,  new String[]{groupName});
     ugi.doAs((PrivilegedExceptionAction<Object>) () -> {
-      assertEquals("Not running as new user", userName,
-          UserGroupInformation.getCurrentUser().getShortUserName());
+      assertEquals(userName, UserGroupInformation.getCurrentUser().getShortUserName(),
+          "Not running as new user");
       DFSAdmin userAdmin = new DFSAdmin(conf);
 
       String[] args2 = new String[]{"-setQuota", "5", subDir};
@@ -106,18 +106,18 @@ public class TestQuotaAllowOwner {
       TestQuota.runCommand(userAdmin, args2, false);
 
       ContentSummary c = dfs.getContentSummary(new Path(subDir));
-      assertEquals("Not same with setting quota",
-          5, c.getQuota());
-      assertEquals("Not same with setting space quota",
-          64, c.getSpaceQuota());
+      assertEquals(5, c.getQuota(),
+          "Not same with setting quota");
+      assertEquals(64, c.getSpaceQuota(),
+          "Not same with setting space quota");
       args2 = new String[]{"-clrQuota", subDir};
       TestQuota.runCommand(userAdmin, args2, false);
       args2 = new String[]{"-clrSpaceQuota", subDir};
       TestQuota.runCommand(userAdmin, args2, false);
       c = dfs.getContentSummary(new Path(subDir));
-      assertEquals("Not clean quota", -1, c.getQuota());
-      assertEquals("Not clean space quota",
-          -1, c.getSpaceQuota());
+      assertEquals(-1, c.getQuota(), "Not clean quota");
+      assertEquals(-1, c.getSpaceQuota(),
+          "Not clean space quota");
       return null;
     });
   }
@@ -138,8 +138,8 @@ public class TestQuotaAllowOwner {
     UserGroupInformation ugi2 = UserGroupInformation.createUserForTesting(
         userName,  new String[]{groupName});
     ugi2.doAs((PrivilegedExceptionAction<Object>) () -> {
-      assertEquals("Not running as new user", userName,
-          UserGroupInformation.getCurrentUser().getShortUserName());
+      assertEquals(userName, UserGroupInformation.getCurrentUser().getShortUserName(),
+          "Not running as new user");
       DFSAdmin userAdmin = new DFSAdmin(conf);
 
       String[] args2 = new String[]{"-setQuota", "5", subDir};
@@ -167,8 +167,8 @@ public class TestQuotaAllowOwner {
     UserGroupInformation ugi = UserGroupInformation.createUserForTesting(
         userOther, new String[]{groupOther});
     ugi.doAs((PrivilegedExceptionAction<Object>) () -> {
-      assertEquals("Not running as new user", userOther,
-          UserGroupInformation.getCurrentUser().getShortUserName());
+      assertEquals(userOther, UserGroupInformation.getCurrentUser().getShortUserName(),
+          "Not running as new user");
       DFSAdmin userAdmin = new DFSAdmin(conf);
 
       String[] args2 = new String[]{"-setQuota", "5", subDir.toString()};
@@ -196,8 +196,8 @@ public class TestQuotaAllowOwner {
     UserGroupInformation ugi = UserGroupInformation.createUserForTesting(
         userOther, new String[]{groupName});
     ugi.doAs((PrivilegedExceptionAction<Object>) () -> {
-      assertEquals("Not running as new user", userOther,
-          UserGroupInformation.getCurrentUser().getShortUserName());
+      assertEquals(userOther, UserGroupInformation.getCurrentUser().getShortUserName(),
+          "Not running as new user");
       DFSAdmin userAdmin = new DFSAdmin(conf);
 
       String[] args2 = new String[]{"-setQuota", "5", subDir.toString()};
@@ -228,8 +228,8 @@ public class TestQuotaAllowOwner {
       UserGroupInformation ugi = UserGroupInformation.createUserForTesting(
           userName, new String[]{groupName});
       ugi.doAs((PrivilegedExceptionAction<Object>) () -> {
-        assertEquals("Not running as new user", userName,
-            UserGroupInformation.getCurrentUser().getShortUserName());
+        assertEquals(userName, UserGroupInformation.getCurrentUser().getShortUserName(),
+            "Not running as new user");
 
         DFSAdmin userAdmin = new DFSAdmin(conf);
 

+ 24 - 18
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRead.java

@@ -35,15 +35,16 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.log4j.Level;
-import org.junit.Assert;
+import org.junit.jupiter.api.Assertions;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSTestUtil.ShortCircuitTestContext;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
@@ -51,9 +52,8 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
 
 public class TestRead {
   static final private int BLOCK_SIZE = 512;
@@ -64,7 +64,7 @@ public class TestRead {
   private final GenericTestUtils.LogCapturer dfsClientLog =
       GenericTestUtils.LogCapturer.captureLogs(DFSClient.LOG);
 
-  @BeforeClass
+  @BeforeAll
   public static void setLogLevel() {
     GenericTestUtils.setLogLevel(DFSClient.LOG, Level.WARN);
   }
@@ -76,19 +76,20 @@ public class TestRead {
     FSDataInputStream fis = fs.open(path);
     ByteBuffer empty = ByteBuffer.allocate(0);
     // A read into an empty bytebuffer at the beginning of the file gives 0.
-    Assert.assertEquals(0, fis.read(empty));
+    Assertions.assertEquals(0, fis.read(empty));
     fis.seek(fileLength);
     // A read into an empty bytebuffer at the end of the file gives -1.
-    Assert.assertEquals(-1, fis.read(empty));
+    Assertions.assertEquals(-1, fis.read(empty));
     if (fileLength > BLOCK_SIZE) {
       fis.seek(fileLength - BLOCK_SIZE + 1);
       ByteBuffer dbb = ByteBuffer.allocateDirect(BLOCK_SIZE);
-      Assert.assertEquals(BLOCK_SIZE - 1, fis.read(dbb));
+      Assertions.assertEquals(BLOCK_SIZE - 1, fis.read(dbb));
     }
     fis.close();
   }
 
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testEOFWithBlockReaderLocal() throws Exception {
     ShortCircuitTestContext testContext = 
         new ShortCircuitTestContext("testEOFWithBlockReaderLocal");
@@ -106,7 +107,8 @@ public class TestRead {
     }
   }
 
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testEOFWithRemoteBlockReader() throws Exception {
     final Configuration conf = new Configuration();
     conf.setLong(HdfsClientConfigKeys.DFS_CLIENT_CACHE_READAHEAD, BLOCK_SIZE);
@@ -123,7 +125,8 @@ public class TestRead {
    * If deadlock happen, the test will time out.
    * @throws Exception
    */
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testReadReservedPath() throws Exception {
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).
@@ -131,7 +134,7 @@ public class TestRead {
     try {
       FileSystem fs = cluster.getFileSystem();
       fs.open(new Path("/.reserved/.inodes/file"));
-      Assert.fail("Open a non existing file should fail.");
+      Assertions.fail("Open a non existing file should fail.");
     } catch (FileNotFoundException e) {
       // Expected
     } finally {
@@ -139,7 +142,8 @@ public class TestRead {
     }
   }
 
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testInterruptReader() throws Exception {
     final Configuration conf = new HdfsConfiguration();
     conf.set(DFSConfigKeys.DFS_DATANODE_FSDATASET_FACTORY_KEY,
@@ -174,7 +178,7 @@ public class TestRead {
       reader.interrupt();
       reader.join();
 
-      Assert.assertTrue(readInterrupted.get());
+      Assertions.assertTrue(readInterrupted.get());
     } finally {
       cluster.shutdown();
     }
@@ -220,7 +224,8 @@ public class TestRead {
    * Test logging in readBuffer() when the number of IOExceptions can be recovered by retrying on
    * a different datanode or by refreshing data nodes and retrying each data node one more time.
    */
-  @Test(timeout=120000)
+  @Test
+  @Timeout(value = 120)
   public void testReadBufferIOExceptionLogging() throws IOException {
     testReadBufferIOExceptionLogging(0, 0);
     testReadBufferIOExceptionLogging(1, 0);
@@ -283,7 +288,8 @@ public class TestRead {
   /**
    * Test the case where we always hit IOExceptions, causing the read request to fail.
    */
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testReadBufferIOExceptionLoggingFailedRequest() throws IOException {
     testReadBufferIOExceptionLoggingFailedRequest(0);
     testReadBufferIOExceptionLoggingFailedRequest(1);

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

@@ -17,13 +17,10 @@
  */
 package org.apache.hadoop.hdfs;
 
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.AfterAll;
+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;
 
@@ -41,7 +38,7 @@ import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.tearDownC
 /**
  * Test online recovery with failed DNs. This test is parameterized.
  */
-@RunWith(Parameterized.class)
+@Timeout(300)
 public class TestReadStripedFileWithDNFailure {
   static final Logger LOG =
       LoggerFactory.getLogger(TestReadStripedFileWithDNFailure.class);
@@ -49,21 +46,16 @@ public class TestReadStripedFileWithDNFailure {
   private static MiniDFSCluster cluster;
   private static DistributedFileSystem dfs;
 
-  @Rule
-  public Timeout globalTimeout = new Timeout(300000);
-
-  @BeforeClass
   public static void setup() throws IOException {
     cluster = initializeCluster();
     dfs = cluster.getFileSystem();
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() throws IOException {
     tearDownCluster(cluster);
   }
 
-  @Parameterized.Parameters
   public static Collection<Object[]> getParameters() {
     ArrayList<Object[]> params = new ArrayList<>();
     for (int fileLength : FILE_LENGTHS) {
@@ -77,7 +69,8 @@ public class TestReadStripedFileWithDNFailure {
   private int fileLength;
   private int dnFailureNum;
 
-  public TestReadStripedFileWithDNFailure(int fileLength, int dnFailureNum) {
+  public void initTestReadStripedFileWithDNFailure(int pFileLength, int pDnFailureNum)
+      throws IOException {
     this.fileLength = fileLength;
     this.dnFailureNum = dnFailureNum;
   }
@@ -86,9 +79,12 @@ public class TestReadStripedFileWithDNFailure {
    * Shutdown tolerable number of Datanode before reading.
    * Verify the decoding works correctly.
    */
-  @Test
-  public void testReadWithDNFailure() throws Exception {
+  @MethodSource("getParameters")
+  @ParameterizedTest
+  public void testReadWithDNFailure(int pFileLength, int pDnFailureNum)
+      throws Exception {
     try {
+      initTestReadStripedFileWithDNFailure(pFileLength, pDnFailureNum);
       // setup a new cluster with no dead datanode
       setup();
       ReadStripedFileWithDecodingHelper.testReadWithDNFailure(cluster,

+ 20 - 23
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java

@@ -33,12 +33,11 @@ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 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.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -56,6 +55,7 @@ import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.findFirst
 import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.initializeCluster;
 import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.tearDownCluster;
 
+@Timeout(300)
 public class TestReadStripedFileWithDecoding {
   private static final Logger LOG =
       LoggerFactory.getLogger(TestReadStripedFileWithDecoding.class);
@@ -63,16 +63,13 @@ public class TestReadStripedFileWithDecoding {
   private MiniDFSCluster cluster;
   private DistributedFileSystem dfs;
 
-  @Rule
-  public Timeout globalTimeout = new Timeout(300000);
-
-  @Before
+  @BeforeEach
   public void setup() throws IOException {
     cluster = initializeCluster();
     dfs = cluster.getFileSystem();
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws IOException {
     tearDownCluster(cluster);
   }
@@ -92,7 +89,7 @@ public class TestReadStripedFileWithDecoding {
     // corrupt the first data block
     int dnIndex = ReadStripedFileWithDecodingHelper.findFirstDataNode(
         cluster, dfs, file, CELL_SIZE * NUM_DATA_UNITS);
-    Assert.assertNotEquals(-1, dnIndex);
+    Assertions.assertNotEquals(-1, dnIndex);
     LocatedStripedBlock slb = (LocatedStripedBlock) dfs.getClient()
         .getLocatedBlocks(file.toString(), 0, CELL_SIZE * NUM_DATA_UNITS)
         .get(0);
@@ -101,7 +98,7 @@ public class TestReadStripedFileWithDecoding {
     // Find the first block file.
     File storageDir = cluster.getInstanceStorageDir(dnIndex, 0);
     File blkFile = MiniDFSCluster.getBlockFile(storageDir, blks[0].getBlock());
-    Assert.assertTrue("Block file does not exist", blkFile.exists());
+    Assertions.assertTrue(blkFile.exists(), "Block file does not exist");
     // corrupt the block file
     LOG.info("Deliberately corrupting file " + blkFile.getName());
     try (FileOutputStream out = new FileOutputStream(blkFile)) {
@@ -124,7 +121,7 @@ public class TestReadStripedFileWithDecoding {
       final BlockManager bm = ns.getBlockManager();
       BlockInfo blockInfo = (ns.getFSDirectory().getINode4Write(file.toString())
           .asFile().getBlocks())[0];
-      Assert.assertEquals(1, bm.getCorruptReplicas(blockInfo).size());
+      Assertions.assertEquals(1, bm.getCorruptReplicas(blockInfo).size());
     } finally {
       for (DataNode dn : cluster.getDataNodes()) {
         DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, false);
@@ -141,7 +138,7 @@ public class TestReadStripedFileWithDecoding {
 
     int dnIndex = findFirstDataNode(cluster, dfs, file,
         CELL_SIZE * NUM_DATA_UNITS);
-    Assert.assertNotEquals(-1, dnIndex);
+    Assertions.assertNotEquals(-1, dnIndex);
     LocatedStripedBlock slb = (LocatedStripedBlock) dfs.getClient()
         .getLocatedBlocks(file.toString(), 0, CELL_SIZE * NUM_DATA_UNITS)
         .get(0);
@@ -164,7 +161,7 @@ public class TestReadStripedFileWithDecoding {
       final BlockManager bm = fsn.getBlockManager();
       DatanodeDescriptor dnd =
           NameNodeAdapter.getDatanode(fsn, dn.getDatanodeId());
-      Assert.assertTrue(bm.containsInvalidateBlock(
+      Assertions.assertTrue(bm.containsInvalidateBlock(
           blks[0].getLocations()[0], b) || dnd.containsInvalidateBlock(b));
     } finally {
       DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, false);
@@ -201,8 +198,8 @@ public class TestReadStripedFileWithDecoding {
         CELL_SIZE * NUM_DATA_UNITS);
     int dnIndex2 = findDataNodeAtIndex(cluster, dfs, file,
         CELL_SIZE * NUM_DATA_UNITS, 2);
-    Assert.assertNotEquals(-1, dnIndex);
-    Assert.assertNotEquals(-1, dnIndex2);
+    Assertions.assertNotEquals(-1, dnIndex);
+    Assertions.assertNotEquals(-1, dnIndex2);
 
     LocatedStripedBlock slb = (LocatedStripedBlock) dfs.getClient()
         .getLocatedBlocks(file.toString(), 0, CELL_SIZE * NUM_DATA_UNITS)
@@ -216,7 +213,7 @@ public class TestReadStripedFileWithDecoding {
     // Find the first block file.
     File storageDir = cluster.getInstanceStorageDir(dnIndex, 0);
     File blkFile = MiniDFSCluster.getBlockFile(storageDir, blks[0].getBlock());
-    Assert.assertTrue("Block file does not exist", blkFile.exists());
+    Assertions.assertTrue(blkFile.exists(), "Block file does not exist");
     // Corrupt the block file.
     LOG.info("Deliberately corrupting file " + blkFile.getName());
     try (FileOutputStream out = new FileOutputStream(blkFile)) {
@@ -227,7 +224,7 @@ public class TestReadStripedFileWithDecoding {
     // Find the second block file.
     File storageDir2 = cluster.getInstanceStorageDir(dnIndex2, 0);
     File blkFile2 = MiniDFSCluster.getBlockFile(storageDir2, blks[1].getBlock());
-    Assert.assertTrue("Block file does not exist", blkFile2.exists());
+    Assertions.assertTrue(blkFile2.exists(), "Block file does not exist");
     // Corrupt the second block file.
     LOG.info("Deliberately corrupting file " + blkFile2.getName());
     try (FileOutputStream out = new FileOutputStream(blkFile2)) {
@@ -257,7 +254,7 @@ public class TestReadStripedFileWithDecoding {
         return bm.getCorruptReplicas(blockInfo).size() == 2;
       }, 250, 60000);
       // Double check.
-      Assert.assertEquals(2, bm.getCorruptReplicas(blockInfo).size());
+      Assertions.assertEquals(2, bm.getCorruptReplicas(blockInfo).size());
 
       DatanodeDescriptor dnd =
           NameNodeAdapter.getDatanode(ns, cluster.getDataNodes().get(dnIndex).getDatanodeId());
@@ -276,7 +273,7 @@ public class TestReadStripedFileWithDecoding {
         return bm.containsInvalidateBlock(
             blks[0].getLocations()[0], b) || dnd.containsInvalidateBlock(b);
       }, 250, 60000);
-      Assert.assertTrue(bm.containsInvalidateBlock(
+      Assertions.assertTrue(bm.containsInvalidateBlock(
           blks[0].getLocations()[0], b) || dnd.containsInvalidateBlock(b));
 
       GenericTestUtils.waitFor(() -> {
@@ -284,7 +281,7 @@ public class TestReadStripedFileWithDecoding {
             blks[1].getLocations()[0], b2) || dnd2.containsInvalidateBlock(b2);
       }, 250, 60000);
 
-      Assert.assertTrue(bm.containsInvalidateBlock(
+      Assertions.assertTrue(bm.containsInvalidateBlock(
           blks[1].getLocations()[0], b2) || dnd2.containsInvalidateBlock(b2));
 
     } finally {

+ 8 - 11
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.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.
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs;
 
+import org.junit.jupiter.api.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -26,10 +27,8 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.Rule;
-import org.junit.rules.Timeout;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
 
 import java.io.IOException;
 
@@ -37,6 +36,7 @@ import java.io.IOException;
  * Test reading a striped file when some of its blocks are missing (not included
  * in the block locations returned by the NameNode).
  */
+@Timeout(300)
 public class TestReadStripedFileWithMissingBlocks {
   public static final Logger LOG = LoggerFactory
       .getLogger(TestReadStripedFileWithMissingBlocks.class);
@@ -57,9 +57,6 @@ public class TestReadStripedFileWithMissingBlocks {
   private final int numDNs = dataBlocks + parityBlocks + 2;
   private final int fileLength = blockSize * dataBlocks + 123;
 
-  @Rule
-  public Timeout globalTimeout = new Timeout(300000);
-
   public void setup() throws IOException {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
@@ -124,7 +121,7 @@ public class TestReadStripedFileWithMissingBlocks {
 
     // make sure there are missing block locations
     BlockLocation[] newLocs = fs.getFileBlockLocations(srcPath, 0, cellSize);
-    Assert.assertTrue(
+    Assertions.assertTrue(
         newLocs[0].getNames().length < locs[0].getNames().length);
 
     byte[] smallBuf = new byte[1024];

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

@@ -32,8 +32,8 @@ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
 import org.slf4j.event.Level;
 
 /** Test reading from hdfs while a file is being written. */
@@ -149,11 +149,11 @@ public class TestReadWhileWriting {
     final HdfsDataInputStream in = (HdfsDataInputStream)fs.open(p);
 
     //Check visible length
-    Assert.assertTrue(in.getVisibleLength() >= expectedsize);
+    Assertions.assertTrue(in.getVisibleLength() >= expectedsize);
 
     //Able to read?
-    for(int i = 0; i < expectedsize; i++) {
-      Assert.assertEquals((byte)i, (byte)in.read());  
+    for (int i = 0; i < expectedsize; i++) {
+      Assertions.assertEquals((byte) i, (byte) in.read());
     }
 
     in.close();

+ 66 - 50
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReconstructStripedFile.java

@@ -17,8 +17,8 @@
  */
 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 static org.junit.Assume.assumeTrue;
 
 import java.io.File;
@@ -41,8 +41,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.datanode.erasurecode.ErasureCodingTestHelper;
 import org.apache.hadoop.io.ElasticByteBufferPool;
-import org.junit.Rule;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.io.TempDir;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -68,10 +67,11 @@ import org.apache.hadoop.io.erasurecode.ErasureCodeNative;
 import org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawErasureCoderFactory;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.LambdaTestUtils;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.slf4j.event.Level;
 
 public class TestReconstructStripedFile {
@@ -98,8 +98,8 @@ public class TestReconstructStripedFile {
     Any
   }
 
-  @Rule
-  public TemporaryFolder baseDir = new TemporaryFolder();
+  @TempDir
+  private java.nio.file.Path baseDir;
 
   private Configuration conf;
   private MiniDFSCluster cluster;
@@ -129,7 +129,7 @@ public class TestReconstructStripedFile {
     return cluster;
   }
 
-  @Before
+  @BeforeEach
   public void setup() throws IOException {
     ecPolicy = getEcPolicy();
     dataBlkNum = ecPolicy.getNumDataUnits();
@@ -155,7 +155,7 @@ public class TestReconstructStripedFile {
         getPendingTimeout());
     conf.setBoolean(DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_VALIDATION_KEY,
         isValidationEnabled());
-    cluster = new MiniDFSCluster.Builder(conf, baseDir.getRoot()).numDataNodes(dnNum)
+    cluster = new MiniDFSCluster.Builder(conf, baseDir.toFile()).numDataNodes(dnNum)
         .build();
     cluster.waitActive();
 
@@ -169,7 +169,7 @@ public class TestReconstructStripedFile {
     }
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
     if (cluster != null) {
       cluster.shutdown();
@@ -177,84 +177,96 @@ public class TestReconstructStripedFile {
     }
   }
 
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testRecoverOneParityBlock() throws Exception {
     int fileLen = (dataBlkNum + 1) * blockSize + blockSize / 10;
     assertFileBlocksReconstruction("/testRecoverOneParityBlock", fileLen,
         ReconstructionType.ParityOnly, 1);
   }
 
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testRecoverOneParityBlock1() throws Exception {
     int fileLen = cellSize + cellSize / 10;
     assertFileBlocksReconstruction("/testRecoverOneParityBlock1", fileLen,
         ReconstructionType.ParityOnly, 1);
   }
 
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testRecoverOneParityBlock2() throws Exception {
     int fileLen = 1;
     assertFileBlocksReconstruction("/testRecoverOneParityBlock2", fileLen,
         ReconstructionType.ParityOnly, 1);
   }
 
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testRecoverOneParityBlock3() throws Exception {
     int fileLen = (dataBlkNum - 1) * blockSize + blockSize / 10;
     assertFileBlocksReconstruction("/testRecoverOneParityBlock3", fileLen,
         ReconstructionType.ParityOnly, 1);
   }
 
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testRecoverAllParityBlocks() throws Exception {
     int fileLen = dataBlkNum * blockSize + blockSize / 10;
     assertFileBlocksReconstruction("/testRecoverAllParityBlocks", fileLen,
         ReconstructionType.ParityOnly, parityBlkNum);
   }
 
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testRecoverAllDataBlocks() throws Exception {
     int fileLen = (dataBlkNum + parityBlkNum) * blockSize + blockSize / 10;
     assertFileBlocksReconstruction("/testRecoverAllDataBlocks", fileLen,
         ReconstructionType.DataOnly, parityBlkNum);
   }
 
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testRecoverAllDataBlocks1() throws Exception {
     int fileLen = parityBlkNum * blockSize + blockSize / 10;
     assertFileBlocksReconstruction("/testRecoverAllDataBlocks1", fileLen,
         ReconstructionType.DataOnly, parityBlkNum);
   }
 
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testRecoverOneDataBlock() throws Exception {
     int fileLen = (dataBlkNum + 1) * blockSize + blockSize / 10;
     assertFileBlocksReconstruction("/testRecoverOneDataBlock", fileLen,
         ReconstructionType.DataOnly, 1);
   }
 
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testRecoverOneDataBlock1() throws Exception {
     int fileLen = cellSize + cellSize/10;
     assertFileBlocksReconstruction("/testRecoverOneDataBlock1", fileLen,
         ReconstructionType.DataOnly, 1);
   }
 
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testRecoverOneDataBlock2() throws Exception {
     int fileLen = 1;
     assertFileBlocksReconstruction("/testRecoverOneDataBlock2", fileLen,
         ReconstructionType.DataOnly, 1);
   }
 
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testRecoverAnyBlocks() throws Exception {
     int fileLen = parityBlkNum * blockSize + blockSize / 10;
     assertFileBlocksReconstruction("/testRecoverAnyBlocks", fileLen,
         ReconstructionType.Any, random.nextInt(parityBlkNum) + 1);
   }
 
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testRecoverAnyBlocks1() throws Exception {
     int fileLen = (dataBlkNum + parityBlkNum) * blockSize + blockSize / 10;
     assertFileBlocksReconstruction("/testRecoverAnyBlocks1", fileLen,
@@ -332,9 +344,9 @@ public class TestReconstructStripedFile {
   void assertFileBlocksReconstruction(String fileName, int fileLen,
       ReconstructionType type, int toRecoverBlockNum) throws Exception {
     if (toRecoverBlockNum < 1 || toRecoverBlockNum > parityBlkNum) {
-      Assert.fail("toRecoverBlockNum should be between 1 ~ " + parityBlkNum);
+      Assertions.fail("toRecoverBlockNum should be between 1 ~ " + parityBlkNum);
     }
-    assertTrue("File length must be positive.", fileLen > 0);
+    assertTrue(fileLen > 0, "File length must be positive.");
 
     Path file = new Path(fileName);
 
@@ -380,8 +392,8 @@ public class TestReconstructStripedFile {
       metadatas[i] = cluster.getBlockMetadataFile(deadDnIndices[i], blocks[i]);
       // the block replica on the datanode should be the same as expected
       assertEquals(replicaLengths[i],
-          StripedBlockUtil.getInternalBlockLength(
-          lastBlock.getBlockSize(), cellSize, dataBlkNum, indices[dead[i]]));
+          StripedBlockUtil.getInternalBlockLength(lastBlock.getBlockSize(), cellSize, dataBlkNum,
+              indices[dead[i]]));
       assertTrue(metadatas[i].getName().
           endsWith(blocks[i].getGenerationStamp() + ".meta"));
       LOG.info("replica " + i + " locates in file: " + replicas[i]);
@@ -427,7 +439,7 @@ public class TestReconstructStripedFile {
       byte[] replicaContentAfterReconstruction =
           DFSTestUtil.readFileAsBytes(replicaAfterReconstruction);
 
-      Assert.assertArrayEquals(replicaContents[i], replicaContentAfterReconstruction);
+      Assertions.assertArrayEquals(replicaContents[i], replicaContentAfterReconstruction);
     }
   }
 
@@ -446,7 +458,7 @@ public class TestReconstructStripedFile {
         }
       }
       if (result[i] == -1) {
-        Assert.fail("Failed to reconstruct striped block: "
+        Assertions.fail("Failed to reconstruct striped block: "
             + blocks[i].getBlockId());
       }
     }
@@ -483,7 +495,8 @@ public class TestReconstructStripedFile {
   }
 
   // HDFS-12044
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testNNSendsErasureCodingTasks() throws Exception {
     testNNSendsErasureCodingTasks(1);
     testNNSendsErasureCodingTasks(2);
@@ -530,9 +543,9 @@ public class TestReconstructStripedFile {
     // Make sure that all pending reconstruction tasks can be processed.
     while (ns.getPendingReconstructionBlocks() > 0) {
       long timeoutPending = ns.getNumTimedOutPendingReconstructions();
-      assertEquals(String
+      assertEquals(0, timeoutPending, String
           .format("Found %d timeout pending reconstruction tasks",
-              timeoutPending), 0, timeoutPending);
+              timeoutPending));
       Thread.sleep(1000);
     }
 
@@ -544,7 +557,8 @@ public class TestReconstructStripedFile {
     );
   }
 
-  @Test(timeout = 180000)
+  @Test
+  @Timeout(value = 180)
   public void testErasureCodingWorkerXmitsWeight() throws Exception {
     testErasureCodingWorkerXmitsWeight(0.5f,
         (int) (ecPolicy.getNumDataUnits() * 0.5f));
@@ -614,7 +628,8 @@ public class TestReconstructStripedFile {
    * Or the NPE will be thrown, which will stop reading the remaining data, and
    * the reconstruction task will fail.
    */
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testTimeoutReadBlockInReconstruction() throws Exception {
     assumeTrue("Ignore case where num parity units <= 1",
         ecPolicy.getNumParityUnits() > 1);
@@ -634,7 +649,7 @@ public class TestReconstructStripedFile {
 
     LocatedBlocks locatedBlocks =
         StripedFileTestUtil.getLocatedBlocks(file, fs);
-    Assert.assertEquals(1, locatedBlocks.getLocatedBlocks().size());
+    Assertions.assertEquals(1, locatedBlocks.getLocatedBlocks().size());
     // The file only has one block group
     LocatedBlock lblock = locatedBlocks.get(0);
     DatanodeInfo[] datanodeinfos = lblock.getLocations();
@@ -646,10 +661,10 @@ public class TestReconstructStripedFile {
         DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_TIMEOUT_MILLIS_KEY,
         DFSConfigKeys.
             DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_TIMEOUT_MILLIS_DEFAULT);
-    Assert.assertTrue(
+    Assertions.assertTrue(
+        stripedReadTimeoutInMills > 2000,
         DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_TIMEOUT_MILLIS_KEY
-            + " must be greater than 2000",
-        stripedReadTimeoutInMills > 2000);
+            + " must be greater than 2000");
 
     DataNodeFaultInjector oldInjector = DataNodeFaultInjector.get();
     DataNodeFaultInjector timeoutInjector = new DataNodeFaultInjector() {
@@ -669,7 +684,7 @@ public class TestReconstructStripedFile {
                 stripedReadTimeoutInMills * 3
             );
           } catch (TimeoutException e) {
-            Assert.fail("Can't reconstruct the file's first part.");
+            Assertions.fail("Can't reconstruct the file's first part.");
           } catch (InterruptedException e) {
           }
         }
@@ -702,7 +717,8 @@ public class TestReconstructStripedFile {
    * This UT is used to ensure that we should close block reader
    * before freeing the buffer.
    */
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testAbnormallyCloseDoesNotWriteBufferAgain() throws Exception {
     assumeTrue("Ignore case where num parity units <= 1",
         ecPolicy.getNumParityUnits() > 1);
@@ -718,7 +734,7 @@ public class TestReconstructStripedFile {
 
     LocatedBlocks locatedBlocks =
         StripedFileTestUtil.getLocatedBlocks(file, fs);
-    Assert.assertEquals(1, locatedBlocks.getLocatedBlocks().size());
+    Assertions.assertEquals(1, locatedBlocks.getLocatedBlocks().size());
     // The file only has one block group
     LocatedBlock lblock = locatedBlocks.get(0);
     DatanodeInfo[] datanodeinfos = lblock.getLocations();
@@ -730,10 +746,10 @@ public class TestReconstructStripedFile {
         DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_TIMEOUT_MILLIS_KEY,
         DFSConfigKeys.
             DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_TIMEOUT_MILLIS_DEFAULT);
-    Assert.assertTrue(
+    Assertions.assertTrue(
+        stripedReadTimeoutInMills > 2000,
         DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_TIMEOUT_MILLIS_KEY
-            + " must be greater than 2000",
-        stripedReadTimeoutInMills > 2000);
+            + " must be greater than 2000");
 
     ElasticByteBufferPool bufferPool =
         (ElasticByteBufferPool) ErasureCodingTestHelper.getBufferPool();
@@ -762,7 +778,7 @@ public class TestReconstructStripedFile {
                 stripedReadTimeoutInMills * 3
             );
           } catch (TimeoutException e) {
-            Assert.fail("Can't reconstruct the file's first part.");
+            Assertions.fail("Can't reconstruct the file's first part.");
           } catch (InterruptedException e) {
           }
         }
@@ -777,7 +793,7 @@ public class TestReconstructStripedFile {
                 stripedReadTimeoutInMills * 3
             );
           } catch (TimeoutException e) {
-            Assert.fail("Can't reconstruct the file's remaining part.");
+            Assertions.fail("Can't reconstruct the file's remaining part.");
           } catch (InterruptedException e) {
           }
         }
@@ -803,7 +819,7 @@ public class TestReconstructStripedFile {
                 stripedReadTimeoutInMills * 3
             );
           } catch (TimeoutException e) {
-            Assert.fail("Can't finish the file's reconstruction.");
+            Assertions.fail("Can't finish the file's reconstruction.");
           } catch (InterruptedException e) {
           }
         }
@@ -831,7 +847,7 @@ public class TestReconstructStripedFile {
     while (bufferPool.size(direct) != 0) {
       // iterate all ByteBuffers in ElasticByteBufferPool
       ByteBuffer byteBuffer =  bufferPool.getBuffer(direct, 0);
-      Assert.assertEquals(0, byteBuffer.position());
+      Assertions.assertEquals(0, byteBuffer.position());
     }
   }
 

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

@@ -20,8 +20,9 @@ package org.apache.hadoop.hdfs;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeFaultInjector;
 import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -51,7 +52,8 @@ public class TestReconstructStripedFileWithValidator
    * On the other hand, when validation disabled, the first reconstruction task
    * will succeed and then lead to data corruption.
    */
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testValidatorWithBadDecoding()
       throws Exception {
     MiniDFSCluster cluster = getCluster();
@@ -59,7 +61,7 @@ public class TestReconstructStripedFileWithValidator
     cluster.getDataNodes().stream()
         .map(DataNode::getMetrics)
         .map(DataNodeMetrics::getECInvalidReconstructionTasks)
-        .forEach(n -> Assert.assertEquals(0, (long) n));
+        .forEach(n -> Assertions.assertEquals(0, (long) n));
 
     DataNodeFaultInjector oldInjector = DataNodeFaultInjector.get();
     DataNodeFaultInjector badDecodingInjector = new DataNodeFaultInjector() {
@@ -93,7 +95,7 @@ public class TestReconstructStripedFileWithValidator
           .map(DataNode::getMetrics)
           .mapToLong(DataNodeMetrics::getECInvalidReconstructionTasks)
           .sum();
-      Assert.assertEquals(1, sum);
+      Assertions.assertEquals(1, sum);
     } finally {
       DataNodeFaultInjector.set(oldInjector);
     }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRenameWhileOpen.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 static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.spy;
 
@@ -27,7 +27,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.mockito.Mockito;
 import org.slf4j.event.Level;
 

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

@@ -31,8 +31,8 @@ import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
 import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure.Policy;
 import org.apache.hadoop.io.IOUtils;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
 
 /**
  * Verify the behaviours of HdfsClientConfigKeys.BlockWrite.
@@ -127,7 +127,7 @@ public class TestReplaceDatanodeFailureReplication {
       for (SlowWriter s : slowwriters) {
         try {
           s.out.getCurrentBlockReplication();
-          Assert.fail(
+          Assertions.fail(
               "Must throw exception as failed to add a new datanode for write "
                   + "pipeline, minimum failure replication");
         } catch (IOException e) {
@@ -198,7 +198,7 @@ public class TestReplaceDatanodeFailureReplication {
       cluster.waitFirstBRCompleted(0, 10000);
       // check replication and interrupt.
       for (SlowWriter s : slowwriters) {
-        Assert.assertEquals(failRF, s.out.getCurrentBlockReplication());
+        Assertions.assertEquals(failRF, s.out.getCurrentBlockReplication());
         s.interruptRunning();
       }
 
@@ -228,7 +228,7 @@ public class TestReplaceDatanodeFailureReplication {
         for (int j = 0, x;; j++) {
           x = in.read();
           if ((x) != -1) {
-            Assert.assertEquals(j, x);
+            Assertions.assertEquals(j, x);
           } else {
             return;
           }

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

@@ -37,8 +37,8 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
 import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure.Policy;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
 import org.slf4j.event.Level;
 
 /**
@@ -79,7 +79,7 @@ public class TestReplaceDatanodeOnFailure {
     for(short replication = 1; replication <= infos.length; replication++) {
       for(int nExistings = 0; nExistings < datanodes.length; nExistings++) {
         final DatanodeInfo[] existings = datanodes[nExistings];
-        Assert.assertEquals(nExistings, existings.length);
+        Assertions.assertEquals(nExistings, existings.length);
 
         for(int i = 0; i < isAppend.length; i++) {
           for(int j = 0; j < isHflushed.length; j++) {
@@ -100,7 +100,7 @@ public class TestReplaceDatanodeOnFailure {
             final boolean computed = p.satisfy(
                 replication, existings, isAppend[i], isHflushed[j]);
             try {
-              Assert.assertEquals(expected, computed);
+              Assertions.assertEquals(expected, computed);
             } catch(AssertionError e) {
               final String s = "replication=" + replication
                            + "\nnExistings =" + nExistings
@@ -185,7 +185,7 @@ public class TestReplaceDatanodeOnFailure {
         try {
           in = fs.open(slowwriters[i].filepath);
           for(int j = 0, x; (x = in.read()) != -1; j++) {
-            Assert.assertEquals(j, x);
+            Assertions.assertEquals(j, x);
           }
         }
         finally {
@@ -270,7 +270,7 @@ public class TestReplaceDatanodeOnFailure {
     }
 
     void checkReplication() throws IOException {
-      Assert.assertEquals(REPLICATION, out.getCurrentBlockReplication());
+      Assertions.assertEquals(REPLICATION, out.getCurrentBlockReplication());
     }        
   }
 
@@ -290,8 +290,8 @@ public class TestReplaceDatanodeOnFailure {
         LOG.info("create an empty file " + f);
         fs.create(f, REPLICATION).close();
         final FileStatus status = fs.getFileStatus(f);
-        Assert.assertEquals(REPLICATION, status.getReplication());
-        Assert.assertEquals(0L, status.getLen());
+        Assertions.assertEquals(REPLICATION, status.getReplication());
+        Assertions.assertEquals(0L, status.getLen());
       }
       
       
@@ -303,8 +303,8 @@ public class TestReplaceDatanodeOnFailure {
         out.close();
 
         final FileStatus status = fs.getFileStatus(f);
-        Assert.assertEquals(REPLICATION, status.getReplication());
-        Assert.assertEquals(bytes.length, status.getLen());
+        Assertions.assertEquals(REPLICATION, status.getReplication());
+        Assertions.assertEquals(bytes.length, status.getLen());
       }
 
       {
@@ -314,7 +314,7 @@ public class TestReplaceDatanodeOnFailure {
           out.write(bytes);
           out.close();
 
-          Assert.fail();
+          Assertions.fail();
         } catch(IOException ioe) {
           LOG.info("This exception is expected", ioe);
         }
@@ -346,8 +346,8 @@ public class TestReplaceDatanodeOnFailure {
         out.close();
 
         final FileStatus status = fs.getFileStatus(f);
-        Assert.assertEquals(REPLICATION, status.getReplication());
-        Assert.assertEquals(bytes.length, status.getLen());
+        Assertions.assertEquals(REPLICATION, status.getReplication());
+        Assertions.assertEquals(bytes.length, status.getLen());
       }
 
       {

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

@@ -19,9 +19,9 @@ package org.apache.hadoop.hdfs;
 
 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.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 static org.mockito.ArgumentMatchers.any;
 
 import java.util.function.Supplier;
@@ -66,7 +66,8 @@ import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils.DelayAnswer;
 import org.apache.hadoop.util.Time;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.mockito.Mockito;
 
 /**
@@ -245,7 +246,7 @@ public class TestReplication {
             cluster.corruptBlockOnDataNodesByDeletingBlockFile(block) :
               cluster.corruptBlockOnDataNodes(block);       
 
-    assertEquals("Corrupted too few blocks", replFactor, blockFilesCorrupted); 
+    assertEquals(replFactor, blockFilesCorrupted, "Corrupted too few blocks");
 
     // Increase replication factor, this should invoke transfer request
     // Receiving datanode fails on checksum and reports it to namenode
@@ -269,7 +270,8 @@ public class TestReplication {
     cluster.shutdown();
   }
 
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testBadBlockReportOnTransferCorruptFile() throws Exception {
     Configuration conf = new HdfsConfiguration();
     conf.set(DFSConfigKeys.DFS_DATANODE_FSDATASET_FACTORY_KEY,
@@ -310,7 +312,7 @@ public class TestReplication {
       replicaCount = dfsClient.getNamenode()
           .getBlockLocations(file1.toString(), 0, Long.MAX_VALUE).get(0)
           .getLocations().length;
-      assertEquals("replication should not success", 1, replicaCount);
+      assertEquals(1, replicaCount, "replication should not success");
     } finally {
       cluster.shutdown();
     }
@@ -351,7 +353,7 @@ public class TestReplication {
     DFSClient client = new DFSClient(addr, conf);
     
     DatanodeInfo[] info = client.datanodeReport(DatanodeReportType.LIVE);
-    assertEquals("Number of Datanodes ", numDatanodes, info.length);
+    assertEquals(numDatanodes, info.length, "Number of Datanodes ");
     FileSystem fileSys = cluster.getFileSystem();
     try {
       Path file1 = new Path("/smallblocktest.dat");
@@ -611,7 +613,8 @@ public class TestReplication {
    * Simulate rbw blocks by creating dummy copies, then a DN restart to detect
    * those corrupted blocks asap.
    */
-  @Test(timeout=30000)
+  @Test
+  @Timeout(value = 30)
   public void testReplicationWhenBlockCorruption() throws Exception {
     MiniDFSCluster cluster = null;
     try {
@@ -657,7 +660,8 @@ public class TestReplication {
    * the NameNode doesn't consider the block under-replicated too
    * aggressively. It is a regression test for HDFS-1172.
    */
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testNoExtraReplicationWhenBlockReceivedIsLate()
       throws Exception {
     LOG.info("Test block replication when blockReceived is late" );
@@ -736,7 +740,8 @@ public class TestReplication {
    * in the middle of that file are properly re-replicated if they
    * become corrupt.
    */
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testReplicationWhileUnderConstruction()
       throws Exception {
     LOG.info("Test block replication in under construction" );

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

@@ -18,14 +18,14 @@
 
 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 org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 /**
  * A JUnit test for checking if restarting DFS preserves integrity.
@@ -70,8 +70,8 @@ public class TestRestartDFS {
       // Here we restart the MiniDFScluster without formatting namenode
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4).format(false).build(); 
       FileSystem fs = cluster.getFileSystem();
-      assertTrue("Filesystem corrupted after restart.",
-                 files.checkFiles(fs, dir));
+      assertTrue(files.checkFiles(fs, dir),
+          "Filesystem corrupted after restart.");
 
       final FileStatus newrootstatus = fs.getFileStatus(rootpath);
       assertEquals(rootmtime, newrootstatus.getModificationTime());
@@ -94,8 +94,8 @@ public class TestRestartDFS {
       // the image written in parallel to both places did not get corrupted
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4).format(false).build();
       FileSystem fs = cluster.getFileSystem();
-      assertTrue("Filesystem corrupted after restart.",
-                 files.checkFiles(fs, dir));
+      assertTrue(files.checkFiles(fs, dir),
+          "Filesystem corrupted after restart.");
 
       final FileStatus newrootstatus = fs.getFileStatus(rootpath);
       assertEquals(rootmtime, newrootstatus.getModificationTime());

+ 84 - 73
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java

@@ -36,8 +36,7 @@ import javax.management.openmbean.CompositeDataSupport;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
-import org.junit.Rule;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.io.TempDir;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -61,13 +60,14 @@ import org.apache.hadoop.hdfs.server.namenode.TestFileTruncate;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import static org.apache.hadoop.hdfs.server.namenode.ImageServlet.RECENT_IMAGE_CHECK_ENABLED;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertNull;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
 
 /**
  * This class tests rolling upgrade.
@@ -81,12 +81,12 @@ public class TestRollingUpgrade {
     if (success) {
       assertEquals(0, dfsadmin.run(args));
     } else {
-      Assert.assertTrue(dfsadmin.run(args) != 0);
+      Assertions.assertTrue(dfsadmin.run(args) != 0);
     }
   }
 
-  @Rule
-  public TemporaryFolder baseDir = new TemporaryFolder();
+  @TempDir
+  private java.nio.file.Path baseDir;
 
   /**
    * Test DFSAdmin Upgrade Command.
@@ -95,7 +95,7 @@ public class TestRollingUpgrade {
   public void testDFSAdminRollingUpgradeCommands() throws Exception {
     // start a cluster
     final Configuration conf = new HdfsConfiguration();
-    try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf, baseDir.getRoot())
+    try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf, baseDir.toFile())
         .numDataNodes(0)
         .build()) {
       cluster.waitActive();
@@ -140,9 +140,9 @@ public class TestRollingUpgrade {
 
         // All directories created before upgrade, when upgrade in progress and
         // after upgrade finalize exists
-        Assert.assertTrue(dfs.exists(foo));
-        Assert.assertTrue(dfs.exists(bar));
-        Assert.assertTrue(dfs.exists(baz));
+        Assertions.assertTrue(dfs.exists(foo));
+        Assertions.assertTrue(dfs.exists(bar));
+        Assertions.assertTrue(dfs.exists(baz));
 
         dfs.setSafeMode(SafeModeAction.ENTER);
         dfs.saveNamespace();
@@ -153,9 +153,9 @@ public class TestRollingUpgrade {
       cluster.restartNameNode();
       {
         final DistributedFileSystem dfs = cluster.getFileSystem();
-        Assert.assertTrue(dfs.exists(foo));
-        Assert.assertTrue(dfs.exists(bar));
-        Assert.assertTrue(dfs.exists(baz));
+        Assertions.assertTrue(dfs.exists(foo));
+        Assertions.assertTrue(dfs.exists(bar));
+        Assertions.assertTrue(dfs.exists(baz));
       }
     }
   }
@@ -169,7 +169,8 @@ public class TestRollingUpgrade {
     return conf;
   }
 
-  @Test (timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testRollingUpgradeWithQJM() throws Exception {
     String nnDirPrefix = MiniDFSCluster.getBaseDirectory() + "/nn/";
     final File nn1Dir = new File(nnDirPrefix + "image1");
@@ -179,14 +180,14 @@ public class TestRollingUpgrade {
     LOG.info("nn2Dir=" + nn2Dir);
 
     final Configuration conf = new HdfsConfiguration();
-    try (MiniJournalCluster mjc = new MiniJournalCluster.Builder(conf, baseDir.getRoot())
+    try (MiniJournalCluster mjc = new MiniJournalCluster.Builder(conf, baseDir.toFile())
         .build()) {
       mjc.waitActive();
       setConf(conf, nn1Dir, mjc);
 
       {
         // Start the cluster once to generate the dfs dirs
-        final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf, baseDir.getRoot())
+        final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf, baseDir.toFile())
             .numDataNodes(0)
             .manageNameDfsDirs(false)
             .checkExitOnShutdown(false)
@@ -206,7 +207,7 @@ public class TestRollingUpgrade {
             new Path(nn2Dir.getAbsolutePath()), false, conf);
 
         // Start the cluster again
-        final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf, baseDir.getRoot())
+        final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf, baseDir.toFile())
             .numDataNodes(0)
             .format(false)
             .manageNameDfsDirs(false)
@@ -245,9 +246,9 @@ public class TestRollingUpgrade {
         final DistributedFileSystem dfs2 = cluster2.getFileSystem();
 
         // Check that cluster2 sees the edits made on cluster1
-        Assert.assertTrue(dfs2.exists(foo));
-        Assert.assertTrue(dfs2.exists(bar));
-        Assert.assertFalse(dfs2.exists(baz));
+        Assertions.assertTrue(dfs2.exists(foo));
+        Assertions.assertTrue(dfs2.exists(bar));
+        Assertions.assertFalse(dfs2.exists(baz));
 
         //query rolling upgrade in cluster2
         assertEquals(info1, dfs2.rollingUpgrade(RollingUpgradeAction.QUERY));
@@ -257,9 +258,9 @@ public class TestRollingUpgrade {
         LOG.info("RESTART cluster 2");
         cluster2.restartNameNode();
         assertEquals(info1, dfs2.rollingUpgrade(RollingUpgradeAction.QUERY));
-        Assert.assertTrue(dfs2.exists(foo));
-        Assert.assertTrue(dfs2.exists(bar));
-        Assert.assertTrue(dfs2.exists(baz));
+        Assertions.assertTrue(dfs2.exists(foo));
+        Assertions.assertTrue(dfs2.exists(bar));
+        Assertions.assertTrue(dfs2.exists(baz));
 
         //restart cluster with -upgrade should fail.
         try {
@@ -271,21 +272,21 @@ public class TestRollingUpgrade {
         LOG.info("RESTART cluster 2 again");
         cluster2.restartNameNode();
         assertEquals(info1, dfs2.rollingUpgrade(RollingUpgradeAction.QUERY));
-        Assert.assertTrue(dfs2.exists(foo));
-        Assert.assertTrue(dfs2.exists(bar));
-        Assert.assertTrue(dfs2.exists(baz));
+        Assertions.assertTrue(dfs2.exists(foo));
+        Assertions.assertTrue(dfs2.exists(bar));
+        Assertions.assertTrue(dfs2.exists(baz));
 
         //finalize rolling upgrade
         final RollingUpgradeInfo finalize = dfs2.rollingUpgrade(
             RollingUpgradeAction.FINALIZE);
-        Assert.assertTrue(finalize.isFinalized());
+        Assertions.assertTrue(finalize.isFinalized());
 
         LOG.info("RESTART cluster 2 with regular startup option");
         cluster2.getNameNodeInfos()[0].setStartOpt(StartupOption.REGULAR);
         cluster2.restartNameNode();
-        Assert.assertTrue(dfs2.exists(foo));
-        Assert.assertTrue(dfs2.exists(bar));
-        Assert.assertTrue(dfs2.exists(baz));
+        Assertions.assertTrue(dfs2.exists(foo));
+        Assertions.assertTrue(dfs2.exists(bar));
+        Assertions.assertTrue(dfs2.exists(baz));
       } finally {
         if (cluster2 != null) cluster2.shutdown();
       }
@@ -318,7 +319,7 @@ public class TestRollingUpgrade {
   public void testRollback() throws Exception {
     // start a cluster
     final Configuration conf = new HdfsConfiguration();
-    try (MiniDFSCluster cluster  = new MiniDFSCluster.Builder(conf, baseDir.getRoot())
+    try (MiniDFSCluster cluster  = new MiniDFSCluster.Builder(conf, baseDir.toFile())
         .numDataNodes(1)
         .build()) {
       cluster.waitActive();
@@ -386,8 +387,8 @@ public class TestRollingUpgrade {
     dfs.setSafeMode(SafeModeAction.LEAVE);
 
     dfs.mkdirs(bar);
-    Assert.assertTrue(dfs.exists(foo));
-    Assert.assertTrue(dfs.exists(bar));
+    Assertions.assertTrue(dfs.exists(foo));
+    Assertions.assertTrue(dfs.exists(bar));
 
     //truncate a file
     final int newLength = ThreadLocalRandom.current().nextInt(data.length - 1)
@@ -405,8 +406,8 @@ public class TestRollingUpgrade {
     cluster.restartDataNode(dnprop, true);
 
     final DistributedFileSystem dfs = cluster.getFileSystem();
-    Assert.assertTrue(dfs.exists(foo));
-    Assert.assertFalse(dfs.exists(bar));
+    Assertions.assertTrue(dfs.exists(foo));
+    Assertions.assertFalse(dfs.exists(bar));
     AppendTestUtil.checkFullFile(dfs, file, data.length, data);
   }
 
@@ -414,7 +415,7 @@ public class TestRollingUpgrade {
   public void testDFSAdminDatanodeUpgradeControlCommands() throws Exception {
     // start a cluster
     final Configuration conf = new HdfsConfiguration();
-    try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf, baseDir.getRoot())
+    try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf, baseDir.toFile())
         .numDataNodes(1)
         .build()) {
       cluster.waitActive();
@@ -433,29 +434,33 @@ public class TestRollingUpgrade {
       // the datanode should be down.
       GenericTestUtils.waitForThreadTermination(
           "Async datanode shutdown thread", 100, 10000);
-      Assert.assertFalse("DataNode should exit", dn.isDatanodeUp());
+      Assertions.assertFalse(dn.isDatanodeUp(), "DataNode should exit");
 
       // ping should fail.
       assertEquals(-1, dfsadmin.run(args1));
     }
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testFinalize() throws Exception {
     testFinalize(2);
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testFinalizeWithMultipleNN() throws Exception {
     testFinalize(3);
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testFinalizeWithDeltaCheck() throws Exception {
     testFinalize(2, true);
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testFinalizeWithMultipleNNDeltaCheck() throws Exception {
     testFinalize(3, true);
   }
@@ -472,7 +477,7 @@ public class TestRollingUpgrade {
     final Path bar = new Path("/bar");
 
     try {
-      cluster = new MiniQJMHACluster.Builder(conf, baseDir.getRoot())
+      cluster = new MiniQJMHACluster.Builder(conf, baseDir.toFile())
           .setNumNameNodes(nnCount).build();
       MiniDFSCluster dfsCluster = cluster.getDfsCluster();
       dfsCluster.waitActive();
@@ -497,20 +502,20 @@ public class TestRollingUpgrade {
       // start rolling upgrade
       RollingUpgradeInfo info = dfs
           .rollingUpgrade(RollingUpgradeAction.PREPARE);
-      Assert.assertTrue(info.isStarted());
+      Assertions.assertTrue(info.isStarted());
       dfs.mkdirs(bar);
 
       queryForPreparation(dfs);
 
       // The NN should have a copy of the fsimage in case of rollbacks.
-      Assert.assertTrue(fsimage.hasRollbackFSImage());
+      Assertions.assertTrue(fsimage.hasRollbackFSImage());
 
       info = dfs.rollingUpgrade(RollingUpgradeAction.FINALIZE);
-      Assert.assertTrue(info.isFinalized());
-      Assert.assertTrue(dfs.exists(foo));
+      Assertions.assertTrue(info.isFinalized());
+      Assertions.assertTrue(dfs.exists(foo));
 
       // Once finalized, there should be no more fsimage for rollbacks.
-      Assert.assertFalse(fsimage.hasRollbackFSImage());
+      Assertions.assertFalse(fsimage.hasRollbackFSImage());
 
       // Should have no problem in restart and replaying edits that include
       // the FINALIZE op.
@@ -522,19 +527,21 @@ public class TestRollingUpgrade {
     }
   }
 
-  @Test (timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testQuery() throws Exception {
     testQuery(2);
   }
 
-  @Test (timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testQueryWithMultipleNN() throws Exception {
     testQuery(3);
   }
 
   private void testQuery(int nnCount) throws Exception{
     final Configuration conf = new HdfsConfiguration();
-    try (MiniQJMHACluster cluster = new MiniQJMHACluster.Builder(conf, baseDir.getRoot())
+    try (MiniQJMHACluster cluster = new MiniQJMHACluster.Builder(conf, baseDir.toFile())
         .setNumNameNodes(nnCount)
         .build()) {
       MiniDFSCluster dfsCluster = cluster.getDfsCluster();
@@ -551,10 +558,10 @@ public class TestRollingUpgrade {
       // start rolling upgrade
       RollingUpgradeInfo info = dfs
           .rollingUpgrade(RollingUpgradeAction.PREPARE);
-      Assert.assertTrue(info.isStarted());
+      Assertions.assertTrue(info.isStarted());
 
       info = dfs.rollingUpgrade(RollingUpgradeAction.QUERY);
-      Assert.assertFalse(info.createdRollbackImages());
+      Assertions.assertFalse(info.createdRollbackImages());
 
       // restart other NNs
       for (int i = 1; i < nnCount; i++) {
@@ -564,15 +571,16 @@ public class TestRollingUpgrade {
       queryForPreparation(dfs);
 
       // The NN should have a copy of the fsimage in case of rollbacks.
-      Assert.assertTrue(dfsCluster.getNamesystem(0).getFSImage()
+      Assertions.assertTrue(dfsCluster.getNamesystem(0).getFSImage()
               .hasRollbackFSImage());
     }
   }
 
-  @Test (timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testQueryAfterRestart() throws IOException, InterruptedException {
     final Configuration conf = new HdfsConfiguration();
-    try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf, baseDir.getRoot())
+    try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf, baseDir.toFile())
         .numDataNodes(0)
         .build()) {
       cluster.waitActive();
@@ -591,17 +599,20 @@ public class TestRollingUpgrade {
     }
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testCheckpoint() throws IOException, InterruptedException {
     testCheckpoint(2);
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testCheckpointWithMultipleNN() throws IOException, InterruptedException {
     testCheckpoint(3);
   }
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testRollBackImage() throws Exception {
     final Configuration conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY, 10);
@@ -610,7 +621,7 @@ public class TestRollingUpgrade {
     MiniQJMHACluster cluster = null;
     CheckpointFaultInjector old = CheckpointFaultInjector.getInstance();
     try {
-      cluster = new MiniQJMHACluster.Builder(conf, baseDir.getRoot()).setNumNameNodes(2).build();
+      cluster = new MiniQJMHACluster.Builder(conf, baseDir.toFile()).setNumNameNodes(2).build();
       MiniDFSCluster dfsCluster = cluster.getDfsCluster();
       dfsCluster.waitActive();
       dfsCluster.transitionToActive(0);
@@ -634,11 +645,11 @@ public class TestRollingUpgrade {
       ruEdit.await();
       RollingUpgradeInfo info = dfs
           .rollingUpgrade(RollingUpgradeAction.PREPARE);
-      Assert.assertTrue(info.isStarted());
+      Assertions.assertTrue(info.isStarted());
       FSImage fsimage = dfsCluster.getNamesystem(0).getFSImage();
       queryForPreparation(dfs);
       // The NN should have a copy of the fsimage in case of rollbacks.
-      Assert.assertTrue(fsimage.hasRollbackFSImage());
+      Assertions.assertTrue(fsimage.hasRollbackFSImage());
     } finally {
       CheckpointFaultInjector.set(old);
       if (cluster != null) {
@@ -654,7 +665,7 @@ public class TestRollingUpgrade {
 
     final Path foo = new Path("/foo");
 
-    try (MiniQJMHACluster cluster = new MiniQJMHACluster.Builder(conf, baseDir.getRoot())
+    try (MiniQJMHACluster cluster = new MiniQJMHACluster.Builder(conf, baseDir.toFile())
         .setNumNameNodes(nnCount)
         .build()) {
       MiniDFSCluster dfsCluster = cluster.getDfsCluster();
@@ -666,13 +677,13 @@ public class TestRollingUpgrade {
       // start rolling upgrade
       RollingUpgradeInfo info = dfs
           .rollingUpgrade(RollingUpgradeAction.PREPARE);
-      Assert.assertTrue(info.isStarted());
+      Assertions.assertTrue(info.isStarted());
 
       queryForPreparation(dfs);
 
       dfs.mkdirs(foo);
       long txid = dfs.rollEdits();
-      Assert.assertTrue(txid > 0);
+      Assertions.assertTrue(txid > 0);
 
       for(int i=1; i< nnCount; i++) {
         verifyNNCheckpoint(dfsCluster, txid, i);
@@ -695,7 +706,7 @@ public class TestRollingUpgrade {
       }
       Thread.sleep(1000);
     }
-    Assert.fail("new checkpoint does not exist");
+    Assertions.fail("new checkpoint does not exist");
   }
 
   static void queryForPreparation(DistributedFileSystem dfs) throws IOException,
@@ -711,7 +722,7 @@ public class TestRollingUpgrade {
     }
 
     if (retries >= 10) {
-      Assert.fail("Query return false");
+      Assertions.fail("Query return false");
     }
   }
 
@@ -737,12 +748,12 @@ public class TestRollingUpgrade {
       // RU start should trigger rollback image in standbycheckpointer
       nn1.getRpcServer().rollingUpgrade(HdfsConstants.RollingUpgradeAction.PREPARE);
       HATestUtil.waitForStandbyToCatchUp(nn1, nn2);
-      Assert.assertTrue(nn2.getNamesystem().isNeedRollbackFsImage());
+      Assertions.assertTrue(nn2.getNamesystem().isNeedRollbackFsImage());
 
       // RU finalize should reset rollback image flag in standbycheckpointer
       nn1.getRpcServer().rollingUpgrade(HdfsConstants.RollingUpgradeAction.FINALIZE);
       HATestUtil.waitForStandbyToCatchUp(nn1, nn2);
-      Assert.assertFalse(nn2.getNamesystem().isNeedRollbackFsImage());
+      Assertions.assertFalse(nn2.getNamesystem().isNeedRollbackFsImage());
     } finally {
       cluster.shutdown();
     }
@@ -760,7 +771,7 @@ public class TestRollingUpgrade {
 
     try {
       Configuration conf = new HdfsConfiguration();
-      cluster = new MiniDFSCluster.Builder(conf, baseDir.getRoot()).build();
+      cluster = new MiniDFSCluster.Builder(conf, baseDir.toFile()).build();
       cluster.waitActive();
 
       conf.set(DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY,

+ 64 - 59
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgradeDowngrade.java

@@ -30,8 +30,9 @@ import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 public class TestRollingUpgradeDowngrade {
 
@@ -39,78 +40,82 @@ public class TestRollingUpgradeDowngrade {
    * Downgrade option is already obsolete. It should throw exception.
    * @throws Exception
    */
-  @Test(timeout = 300000, expected = IllegalArgumentException.class)
+  @Test
+  @Timeout(value = 300)
   public void testDowngrade() throws Exception {
-    final Configuration conf = new HdfsConfiguration();
-    MiniQJMHACluster cluster = null;
-    final Path foo = new Path("/foo");
-    final Path bar = new Path("/bar");
+    Assertions.assertThrows(IllegalArgumentException.class, () -> {
+      final Configuration conf = new HdfsConfiguration();
+      MiniQJMHACluster cluster = null;
+      final Path foo = new Path("/foo");
+      final Path bar = new Path("/bar");
+      try {
+        cluster = new MiniQJMHACluster.Builder(conf).build();
+        MiniDFSCluster dfsCluster = cluster.getDfsCluster();
+        dfsCluster.waitActive();
 
-    try {
-      cluster = new MiniQJMHACluster.Builder(conf).build();
-      MiniDFSCluster dfsCluster = cluster.getDfsCluster();
-      dfsCluster.waitActive();
+        // let NN1 tail editlog every 1s
+        dfsCluster.getConfiguration(1).setInt(
+            DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+        dfsCluster.restartNameNode(1);
 
-      // let NN1 tail editlog every 1s
-      dfsCluster.getConfiguration(1).setInt(
-          DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
-      dfsCluster.restartNameNode(1);
+        dfsCluster.transitionToActive(0);
+        DistributedFileSystem dfs = dfsCluster.getFileSystem(0);
+        dfs.mkdirs(foo);
 
-      dfsCluster.transitionToActive(0);
-      DistributedFileSystem dfs = dfsCluster.getFileSystem(0);
-      dfs.mkdirs(foo);
+        // start rolling upgrade
+        RollingUpgradeInfo info = dfs
+            .rollingUpgrade(RollingUpgradeAction.PREPARE);
+        Assertions.assertTrue(info.isStarted());
+        dfs.mkdirs(bar);
 
-      // start rolling upgrade
-      RollingUpgradeInfo info = dfs
-          .rollingUpgrade(RollingUpgradeAction.PREPARE);
-      Assert.assertTrue(info.isStarted());
-      dfs.mkdirs(bar);
+        TestRollingUpgrade.queryForPreparation(dfs);
+        dfs.close();
 
-      TestRollingUpgrade.queryForPreparation(dfs);
-      dfs.close();
+        dfsCluster.restartNameNode(0, true, "-rollingUpgrade", "downgrade");
+        // Once downgraded, there should be no more fsimage for rollbacks.
+        Assertions.assertFalse(dfsCluster.getNamesystem(0).getFSImage()
+            .hasRollbackFSImage());
+        // shutdown NN1
+        dfsCluster.shutdownNameNode(1);
+        dfsCluster.transitionToActive(0);
 
-      dfsCluster.restartNameNode(0, true, "-rollingUpgrade", "downgrade");
-      // Once downgraded, there should be no more fsimage for rollbacks.
-      Assert.assertFalse(dfsCluster.getNamesystem(0).getFSImage()
-          .hasRollbackFSImage());
-      // shutdown NN1
-      dfsCluster.shutdownNameNode(1);
-      dfsCluster.transitionToActive(0);
-
-      dfs = dfsCluster.getFileSystem(0);
-      Assert.assertTrue(dfs.exists(foo));
-      Assert.assertTrue(dfs.exists(bar));
-    } finally {
-      if (cluster != null) {
-        cluster.shutdown();
+        dfs = dfsCluster.getFileSystem(0);
+        Assertions.assertTrue(dfs.exists(foo));
+        Assertions.assertTrue(dfs.exists(bar));
+      } finally {
+        if (cluster != null) {
+          cluster.shutdown();
+        }
       }
-    }
+    });
   }
 
   /**
    * Ensure that restart namenode with downgrade option should throw exception
    * because it has been obsolete.
    */
-  @Test(expected = IllegalArgumentException.class)
+  @Test
   public void testRejectNewFsImage() throws IOException {
-    final Configuration conf = new Configuration();
-    MiniDFSCluster cluster = null;
-    try {
-      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
-      cluster.waitActive();
-      DistributedFileSystem fs = cluster.getFileSystem();
-      fs.setSafeMode(SafeModeAction.ENTER);
-      fs.saveNamespace();
-      fs.setSafeMode(SafeModeAction.LEAVE);
-      NNStorage storage = spy(cluster.getNameNode().getFSImage().getStorage());
-      int futureVersion = NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION - 1;
-      doReturn(futureVersion).when(storage).getServiceLayoutVersion();
-      storage.writeAll();
-      cluster.restartNameNode(0, true, "-rollingUpgrade", "downgrade");
-    } finally {
-      if (cluster != null) {
-        cluster.shutdown();
+    Assertions.assertThrows(IllegalArgumentException.class, () -> {
+      final Configuration conf = new Configuration();
+      MiniDFSCluster cluster = null;
+      try {
+        cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
+        cluster.waitActive();
+        DistributedFileSystem fs = cluster.getFileSystem();
+        fs.setSafeMode(SafeModeAction.ENTER);
+        fs.saveNamespace();
+        fs.setSafeMode(SafeModeAction.LEAVE);
+        NNStorage storage = spy(cluster.getNameNode().getFSImage().getStorage());
+        int futureVersion = NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION - 1;
+        doReturn(futureVersion).when(storage).getServiceLayoutVersion();
+        storage.writeAll();
+        cluster.restartNameNode(0, true, "-rollingUpgrade", "downgrade");
+      } finally {
+        if (cluster != null) {
+          cluster.shutdown();
+        }
       }
-    }
+    });
   }
 }

+ 20 - 21
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgradeRollback.java

@@ -31,8 +31,8 @@ import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
 
 /**
  * This class tests rollback for rolling upgrade.
@@ -56,35 +56,35 @@ public class TestRollingUpgradeRollback {
     List<File> finalizedEdits = storage.getFiles(
         NNStorage.NameNodeDirType.EDITS,
         NNStorage.getFinalizedEditsFileName(1, imageTxId));
-    Assert.assertTrue(fileExists(finalizedEdits));
+    Assertions.assertTrue(fileExists(finalizedEdits));
     List<File> inprogressEdits = storage.getFiles(
         NNStorage.NameNodeDirType.EDITS,
         NNStorage.getInProgressEditsFileName(imageTxId + 1));
     // For rollback case we will have an inprogress file for future transactions
-    Assert.assertTrue(fileExists(inprogressEdits));
+    Assertions.assertTrue(fileExists(inprogressEdits));
     if (trashEndTxId > 0) {
       List<File> trashedEdits = storage.getFiles(
           NNStorage.NameNodeDirType.EDITS,
           NNStorage.getFinalizedEditsFileName(imageTxId + 1, trashEndTxId)
               + ".trash");
-      Assert.assertTrue(fileExists(trashedEdits));
+      Assertions.assertTrue(fileExists(trashedEdits));
     }
     String imageFileName = trashEndTxId > 0 ? NNStorage
         .getImageFileName(imageTxId) : NNStorage
         .getRollbackImageFileName(imageTxId);
     List<File> imageFiles = storage.getFiles(
         NNStorage.NameNodeDirType.IMAGE, imageFileName);
-    Assert.assertTrue(fileExists(imageFiles));
+    Assertions.assertTrue(fileExists(imageFiles));
   }
 
   private void checkJNStorage(File dir, long discardStartTxId,
       long discardEndTxId) {
     File finalizedEdits = new File(dir, NNStorage.getFinalizedEditsFileName(1,
         discardStartTxId - 1));
-    Assert.assertTrue(finalizedEdits.exists());
+    Assertions.assertTrue(finalizedEdits.exists());
     File trashEdits = new File(dir, NNStorage.getFinalizedEditsFileName(
         discardStartTxId, discardEndTxId) + ".trash");
-    Assert.assertTrue(trashEdits.exists());
+    Assertions.assertTrue(trashEdits.exists());
   }
 
   @Test
@@ -103,8 +103,8 @@ public class TestRollingUpgradeRollback {
 
       // start rolling upgrade
       dfs.setSafeMode(SafeModeAction.ENTER);
-      Assert.assertEquals(0,
-          dfsadmin.run(new String[] { "-rollingUpgrade", "prepare" }));
+      Assertions.assertEquals(0,
+          dfsadmin.run(new String[]{"-rollingUpgrade", "prepare"}));
       dfs.setSafeMode(SafeModeAction.LEAVE);
       // create new directory
       dfs.mkdirs(bar);
@@ -125,10 +125,10 @@ public class TestRollingUpgradeRollback {
       // make sure /foo is still there, but /bar is not
       INode fooNode = nn.getNamesystem().getFSDirectory()
           .getINode4Write(foo.toString());
-      Assert.assertNotNull(fooNode);
+      Assertions.assertNotNull(fooNode);
       INode barNode = nn.getNamesystem().getFSDirectory()
           .getINode4Write(bar.toString());
-      Assert.assertNull(barNode);
+      Assertions.assertNull(barNode);
 
       // check the details of NNStorage
       NNStorage storage = nn.getNamesystem().getFSImage().getStorage();
@@ -165,8 +165,7 @@ public class TestRollingUpgradeRollback {
 
       // start rolling upgrade
       dfs.setSafeMode(SafeModeAction.ENTER);
-      Assert.assertEquals(0,
-          dfsadmin.run(new String[] { "-rollingUpgrade", "prepare" }));
+      Assertions.assertEquals(0, dfsadmin.run(new String[]{"-rollingUpgrade", "prepare"}));
       dfs.setSafeMode(SafeModeAction.LEAVE);
       // create new directory
       dfs.mkdirs(bar);
@@ -176,8 +175,8 @@ public class TestRollingUpgradeRollback {
       cluster.restartNameNode("-rollingUpgrade", "rollback");
       // make sure /foo is still there, but /bar is not
       dfs = cluster.getFileSystem();
-      Assert.assertTrue(dfs.exists(foo));
-      Assert.assertFalse(dfs.exists(bar));
+      Assertions.assertTrue(dfs.exists(foo));
+      Assertions.assertFalse(dfs.exists(bar));
 
       // check storage in JNs
       for (int i = 0; i < NUM_JOURNAL_NODES; i++) {
@@ -223,7 +222,7 @@ public class TestRollingUpgradeRollback {
 
       // start rolling upgrade
       RollingUpgradeInfo info = dfs.rollingUpgrade(RollingUpgradeAction.PREPARE);
-      Assert.assertTrue(info.isStarted());
+      Assertions.assertTrue(info.isStarted());
 
       // create new directory
       dfs.mkdirs(bar);
@@ -234,9 +233,9 @@ public class TestRollingUpgradeRollback {
 
       // If the query returns true, both active and the standby NN should have
       // rollback fsimage ready.
-      Assert.assertTrue(dfsCluster.getNameNode(0).getFSImage()
+      Assertions.assertTrue(dfsCluster.getNameNode(0).getFSImage()
           .hasRollbackFSImage());
-      Assert.assertTrue(dfsCluster.getNameNode(1).getFSImage()
+      Assertions.assertTrue(dfsCluster.getNameNode(1).getFSImage()
           .hasRollbackFSImage());
       
       // rollback NN0
@@ -248,8 +247,8 @@ public class TestRollingUpgradeRollback {
 
       // make sure /foo is still there, but /bar is not
       dfs = dfsCluster.getFileSystem(0);
-      Assert.assertTrue(dfs.exists(foo));
-      Assert.assertFalse(dfs.exists(bar));
+      Assertions.assertTrue(dfs.exists(foo));
+      Assertions.assertFalse(dfs.exists(bar));
 
       // check the details of NNStorage
       NNStorage storage = dfsCluster.getNamesystem(0).getFSImage()

+ 49 - 51
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java

@@ -20,10 +20,10 @@ package org.apache.hadoop.hdfs;
 
 import static org.apache.hadoop.test.MetricsAsserts.getLongCounter;
 import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
-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.security.PrivilegedExceptionAction;
@@ -56,9 +56,10 @@ import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.StringUtils;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import java.util.function.Supplier;
 
@@ -76,7 +77,7 @@ public class TestSafeMode {
   DistributedFileSystem dfs;
   private static final String NN_METRICS = "NameNodeActivity";
 
-  @Before
+  @BeforeEach
   public void startUp() throws IOException {
     conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
@@ -88,7 +89,7 @@ public class TestSafeMode {
     dfs = (DistributedFileSystem)fs;
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws IOException {
     if (fs != null) {
       fs.close();
@@ -133,8 +134,8 @@ public class TestSafeMode {
     cluster.waitActive();
     dfs = cluster.getFileSystem();
     
-    assertTrue("No datanode is started. Should be in SafeMode", 
-               dfs.setSafeMode(SafeModeAction.GET));
+    assertTrue(dfs.setSafeMode(SafeModeAction.GET),
+        "No datanode is started. Should be in SafeMode");
     
     // manually set safemode.
     dfs.setSafeMode(SafeModeAction.ENTER);
@@ -148,17 +149,16 @@ public class TestSafeMode {
       Thread.sleep(2000);
     } catch (InterruptedException ignored) {}
 
-    assertTrue("should still be in SafeMode",
-        dfs.setSafeMode(SafeModeAction.GET));
-    assertFalse("should not be in SafeMode", 
-        dfs.setSafeMode(SafeModeAction.LEAVE));
+    assertTrue(dfs.setSafeMode(SafeModeAction.GET), "should still be in SafeMode");
+    assertFalse(dfs.setSafeMode(SafeModeAction.LEAVE), "should not be in SafeMode");
   }
 
   /**
    * Test that, if there are no blocks in the filesystem,
    * the NameNode doesn't enter the "safemode extension" period.
    */
-  @Test(timeout=45000)
+  @Test
+  @Timeout(value = 45)
   public void testNoExtensionIfNoBlocks() throws IOException {
     cluster.getConfiguration(0).setInt(
         DFSConfigKeys.DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, 60000);
@@ -173,7 +173,8 @@ public class TestSafeMode {
    * Test that the NN initializes its under-replicated blocks queue
    * before it is ready to exit safemode (HDFS-1476)
    */
-  @Test(timeout=45000)
+  @Test
+  @Timeout(value = 45)
   public void testInitializeReplQueuesEarly() throws Exception {
     LOG.info("Starting testInitializeReplQueuesEarly");
     // Spray the blocks around the cluster when we add DNs instead of
@@ -202,13 +203,13 @@ public class TestSafeMode {
     
     String status = nn.getNamesystem().getSafemode();
     assertEquals("Safe mode is ON. The reported blocks 0 needs additional "
-        + "14 blocks to reach the threshold 0.9990 of total blocks 15."
-        + NEWLINE + "The minimum number of live datanodes is not required. "
-        + "Safe mode will be turned off automatically once the thresholds have "
-        + "been reached.", status);
-    assertFalse("Mis-replicated block queues should not be initialized " +
-        "until threshold is crossed",
-        NameNodeAdapter.safeModeInitializedReplQueues(nn));
+            + "14 blocks to reach the threshold 0.9990 of total blocks 15." + NEWLINE
+            + "The minimum number of live datanodes is not required. "
+            + "Safe mode will be turned off automatically once the thresholds have "
+            + "been reached.",
+        status);
+    assertFalse(NameNodeAdapter.safeModeInitializedReplQueues(nn),
+        "Mis-replicated block queues should not be initialized " + "until threshold is crossed");
     
     LOG.info("Restarting one DataNode");
     cluster.restartDataNode(dnprops.remove(0));
@@ -224,8 +225,8 @@ public class TestSafeMode {
     }, 10, 10000);
 
     final long safe = NameNodeAdapter.getSafeModeSafeBlocks(nn);
-    assertTrue("Expected first block report to make some blocks safe.", safe > 0);
-    assertTrue("Did not expect first block report to make all blocks safe.", safe < 15);
+    assertTrue(safe > 0, "Expected first block report to make some blocks safe.");
+    assertTrue(safe < 15, "Did not expect first block report to make all blocks safe.");
 
     assertTrue(NameNodeAdapter.safeModeInitializedReplQueues(nn));
 
@@ -310,8 +311,7 @@ public class TestSafeMode {
   public void testSafeModeExceptionText() throws Exception {
     final Path file1 = new Path("/file1");
     DFSTestUtil.createFile(fs, file1, 1024, (short)1, 0);
-    assertTrue("Could not enter SM",
-        dfs.setSafeMode(SafeModeAction.ENTER));
+    assertTrue(dfs.setSafeMode(SafeModeAction.ENTER), "Could not enter SM");
     try {
       FSRun fsRun = new FSRun() {
         @Override
@@ -341,8 +341,7 @@ public class TestSafeMode {
 
     assertFalse(dfs.setSafeMode(SafeModeAction.GET));
     DFSTestUtil.createFile(fs, file1, 1024, (short)1, 0);
-    assertTrue("Could not enter SM", 
-        dfs.setSafeMode(SafeModeAction.ENTER));
+    assertTrue(dfs.setSafeMode(SafeModeAction.ENTER), "Could not enter SM");
 
     runFsFun("Set quota while in SM", new FSRun() { 
       @Override
@@ -491,8 +490,8 @@ public class TestSafeMode {
       // expected
     }
 
-    assertFalse("Could not leave SM",
-        dfs.setSafeMode(SafeModeAction.LEAVE));
+    assertFalse(dfs.setSafeMode(SafeModeAction.LEAVE),
+        "Could not leave SM");
   }
 
   /**
@@ -510,10 +509,10 @@ public class TestSafeMode {
     fs = cluster.getFileSystem();
 
     String tipMsg = cluster.getNamesystem().getSafemode();
-    assertTrue("Safemode tip message doesn't look right: " + tipMsg,
-      tipMsg.contains("The number of live datanodes 0 needs an additional " +
-                      "1 live datanodes to reach the minimum number 1." +
-                      NEWLINE + "Safe mode will be turned off automatically"));
+    assertTrue(tipMsg.contains("The number of live datanodes 0 needs an additional " +
+            "1 live datanodes to reach the minimum number 1." +
+            NEWLINE + "Safe mode will be turned off automatically"),
+        "Safemode tip message doesn't look right: " + tipMsg);
 
     // Start a datanode
     cluster.startDataNodes(conf, 1, true, null, null);
@@ -537,11 +536,11 @@ public class TestSafeMode {
 
     // Enter safemode.
     dfs.setSafeMode(SafeModeAction.ENTER);
-    assertTrue("State was expected to be in safemode.", dfs.isInSafeMode());
+    assertTrue(dfs.isInSafeMode(), "State was expected to be in safemode.");
 
     // Exit safemode.
     dfs.setSafeMode(SafeModeAction.LEAVE);
-    assertFalse("State was expected to be out of safemode.", dfs.isInSafeMode());
+    assertFalse(dfs.isInSafeMode(), "State was expected to be out of safemode.");
   }
   
   @Test
@@ -557,18 +556,17 @@ public class TestSafeMode {
       DFSTestUtil.createFile(fs, file1, 1000, (short)1, 0);
       DFSTestUtil.createFile(fs, file2, 2000, (short)1, 0);
       checkGetBlockLocationsWorks(fs, file1);
-      
+
       NameNode namenode = cluster.getNameNode();
 
       // manually set safemode.
       dfs.setSafeMode(SafeModeAction.ENTER);
-      assertTrue("should still be in SafeMode", namenode.isInSafeMode());
+      assertTrue(namenode.isInSafeMode(), "should still be in SafeMode");
       // getBlock locations should still work since block locations exists
       checkGetBlockLocationsWorks(fs, file1);
       dfs.setSafeMode(SafeModeAction.LEAVE);
-      assertFalse("should not be in SafeMode", namenode.isInSafeMode());
-      
-      
+      assertFalse(namenode.isInSafeMode(), "should not be in SafeMode");
+
       // Now 2nd part of the tests where there aren't block locations
       cluster.shutdownDataNodes();
       cluster.shutdownNameNode(0);
@@ -580,23 +578,23 @@ public class TestSafeMode {
       System.out.println("Restarted cluster with just the NameNode");
       
       namenode = cluster.getNameNode();
-      
-      assertTrue("No datanode is started. Should be in SafeMode", 
-                 namenode.isInSafeMode());
+
+      assertTrue(namenode.isInSafeMode(),
+          "No datanode is started. Should be in SafeMode");
       FileStatus stat = fs.getFileStatus(file1);
       try {
         fs.getFileBlockLocations(stat, 0, 1000);
-        assertTrue("Should have got safemode exception", false);
+        assertTrue(false, "Should have got safemode exception");
       } catch (SafeModeException e) {
         // as expected 
       } catch (RemoteException re) {
         if (!re.getClassName().equals(SafeModeException.class.getName()))
-          assertTrue("Should have got safemode exception", false);   
+            assertTrue(false, "Should have got safemode exception");
       }
 
 
       dfs.setSafeMode(SafeModeAction.LEAVE);
-      assertFalse("Should not be in safemode", namenode.isInSafeMode());
+      assertFalse(namenode.isInSafeMode(), "Should not be in safemode");
       checkGetBlockLocationsWorks(fs, file1);
 
     } finally {
@@ -610,9 +608,9 @@ public class TestSafeMode {
     try {  
       fs.getFileBlockLocations(stat, 0, 1000);
     } catch (SafeModeException e) {
-      assertTrue("Should have not got safemode exception", false);
+      assertTrue(false, "Should have not got safemode exception");
     } catch (RemoteException re) {
-      assertTrue("Should have not got remote exception", false);
+      assertTrue(false, "Should have not got remote exception");
     }    
   }
 }

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

@@ -27,19 +27,19 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.util.Lists;
-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.util.List;
 
-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;
 
+@Timeout(300)
 public class TestSafeModeWithStripedFile {
 
   private ErasureCodingPolicy ecPolicy;
@@ -52,14 +52,11 @@ public class TestSafeModeWithStripedFile {
   private MiniDFSCluster cluster;
   private Configuration conf;
 
-  @Rule
-  public Timeout globalTimeout = new Timeout(300000);
-
   public ErasureCodingPolicy getEcPolicy() {
     return StripedFileTestUtil.getDefaultECPolicy();
   }
 
-  @Before
+  @BeforeEach
   public void setup() throws IOException {
     ecPolicy = getEcPolicy();
     dataBlocks = (short) ecPolicy.getNumDataUnits();
@@ -78,7 +75,7 @@ public class TestSafeModeWithStripedFile {
     cluster.waitActive();
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws IOException {
     if (cluster != null) {
       cluster.shutdown();

+ 14 - 17
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSecureEncryptionZoneWithKMS.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 static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SASL_KEY;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.KMS_CLIENT_ENC_KEY_CACHE_LOW_WATERMARK;
@@ -64,13 +64,12 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
+import org.junit.jupiter.api.AfterEach;
+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;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -79,6 +78,7 @@ import org.slf4j.LoggerFactory;
  * Kerby-based MiniKDC, MiniKMS and MiniDFSCluster. This provides additional
  * unit test coverage on Secure(Kerberos) KMS + HDFS.
  */
+@Timeout(120)
 public class TestSecureEncryptionZoneWithKMS {
   public static final Logger LOG = LoggerFactory.getLogger(
       TestSecureEncryptionZoneWithKMS.class);
@@ -129,10 +129,7 @@ public class TestSecureEncryptionZoneWithKMS {
     return file;
   }
 
-  @Rule
-  public Timeout timeout = new Timeout(120000);
-
-  @BeforeClass
+  @BeforeAll
   public static void init() throws Exception {
     baseDir = getTestDir();
     FileUtil.fullyDelete(baseDir);
@@ -146,8 +143,8 @@ public class TestSecureEncryptionZoneWithKMS {
     SecurityUtil.setAuthenticationMethod(AuthenticationMethod.KERBEROS,
         baseConf);
     UserGroupInformation.setConfiguration(baseConf);
-    assertTrue("Expected configuration to enable security",
-        UserGroupInformation.isSecurityEnabled());
+    assertTrue(UserGroupInformation.isSecurityEnabled(),
+        "Expected configuration to enable security");
 
     File keytabFile = new File(baseDir, "test.keytab");
     keytab = keytabFile.getAbsolutePath();
@@ -231,7 +228,7 @@ public class TestSecureEncryptionZoneWithKMS {
     miniKMS.start();
   }
 
-  @AfterClass
+  @AfterAll
   public static void destroy() throws Exception {
     if (kdc != null) {
       kdc.stop();
@@ -243,7 +240,7 @@ public class TestSecureEncryptionZoneWithKMS {
     KeyStoreTestUtil.cleanupSSLConfig(keystoresDir, sslConfDir);
   }
 
-  @Before
+  @BeforeEach
   public void setup() throws Exception {
     // Start MiniDFS Cluster
     baseConf
@@ -271,7 +268,7 @@ public class TestSecureEncryptionZoneWithKMS {
     }
   }
 
-  @After
+  @AfterEach
   public void shutdown() throws IOException {
     IOUtils.cleanupWithLogger(null, fs);
     if (cluster != null) {

+ 78 - 74
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSeekBug.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.
@@ -17,8 +17,8 @@
  */
 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.IOException;
 import java.util.Random;
@@ -30,31 +30,31 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Assertions;
 
 /**
  * This class tests the presence of seek bug as described
- * in HADOOP-508 
+ * in HADOOP-508
  */
 public class TestSeekBug {
   static final long seed = 0xDEADBEEFL;
   static final int ONEMB = 1 << 20;
-  
+
   private void checkAndEraseData(byte[] actual, int from, byte[] expected, String message) {
     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;
     }
   }
-  
+
   private void seekReadFile(FileSystem fileSys, Path name) throws IOException {
     FSDataInputStream stm = fileSys.open(name, 4096);
     byte[] expected = new byte[ONEMB];
     Random rand = new Random(seed);
     rand.nextBytes(expected);
-    
+
     // First read 128 bytes to set count in BufferedInputStream
     byte[] actual = new byte[128];
     stm.read(actual, 0, actual.length);
@@ -76,14 +76,14 @@ public class TestSeekBug {
    */
   private void smallReadSeek(FileSystem fileSys, Path name) throws IOException {
     if (fileSys instanceof ChecksumFileSystem) {
-      fileSys = ((ChecksumFileSystem)fileSys).getRawFileSystem();
+      fileSys = ((ChecksumFileSystem) fileSys).getRawFileSystem();
     }
     // Make the buffer size small to trigger code for HADOOP-922
     FSDataInputStream stmRaw = fileSys.open(name, 1);
     byte[] expected = new byte[ONEMB];
     Random rand = new Random(seed);
     rand.nextBytes(expected);
-    
+
     // Issue a simple read first.
     byte[] actual = new byte[128];
     stmRaw.seek(100000);
@@ -105,13 +105,13 @@ public class TestSeekBug {
     // all done
     stmRaw.close();
   }
-  
+
   private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
     assertTrue(fileSys.exists(name));
     fileSys.delete(name, true);
     assertTrue(!fileSys.exists(name));
   }
-  
+
   /**
    * Test if the seek bug exists in FSDataInputStream in DFS.
    */
@@ -134,68 +134,72 @@ public class TestSeekBug {
     }
   }
 
- /**
-  * Test (expected to throw IOE) for negative
-  * <code>FSDataInpuStream#seek</code> argument
-  */
-  @Test (expected=IOException.class)
+  /**
+   * Test (expected to throw IOE) for negative.
+   * <code>FSDataInpuStream#seek</code> argument
+   */
+  @Test
   public void testNegativeSeek() throws IOException {
-    Configuration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
-    FileSystem fs = cluster.getFileSystem();
-    try {
-      Path seekFile = new Path("seekboundaries.dat");
-      DFSTestUtil.createFile(
-        fs,
-        seekFile,
-        ONEMB,
-        ONEMB,
-        fs.getDefaultBlockSize(seekFile),
-        fs.getDefaultReplication(seekFile),
-        seed);
-      FSDataInputStream stream = fs.open(seekFile);
-      // Perform "safe seek" (expected to pass)
-      stream.seek(65536);
-      assertEquals(65536, stream.getPos());
-      // expect IOE for this call
-      stream.seek(-73);
-    } finally {
-      fs.close();
-      cluster.shutdown();
-    }
+    Assertions.assertThrows(IOException.class, () -> {
+      Configuration conf = new HdfsConfiguration();
+      MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
+      FileSystem fs = cluster.getFileSystem();
+      try {
+        Path seekFile = new Path("seekboundaries.dat");
+        DFSTestUtil.createFile(
+            fs,
+            seekFile,
+            ONEMB,
+            ONEMB,
+            fs.getDefaultBlockSize(seekFile),
+            fs.getDefaultReplication(seekFile),
+            seed);
+        FSDataInputStream stream = fs.open(seekFile);
+        // Perform "safe seek" (expected to pass)
+        stream.seek(65536);
+        assertEquals(65536, stream.getPos());
+        // expect IOE for this call
+        stream.seek(-73);
+      } finally {
+        fs.close();
+        cluster.shutdown();
+      }
+    });
   }
 
- /**
-  * Test (expected to throw IOE) for <code>FSDataInpuStream#seek</code>
-  * when the position argument is larger than the file size.
-  */
-  @Test (expected=IOException.class)
+  /**
+   * Test (expected to throw IOE) for <code>FSDataInpuStream#seek</code>
+   * when the position argument is larger than the file size.
+   */
+  @Test
   public void testSeekPastFileSize() throws IOException {
-    Configuration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
-    FileSystem fs = cluster.getFileSystem();
-    try {
-      Path seekFile = new Path("seekboundaries.dat");
-      DFSTestUtil.createFile(
-        fs,
-        seekFile,
-        ONEMB,
-        ONEMB,
-        fs.getDefaultBlockSize(seekFile),
-        fs.getDefaultReplication(seekFile),
-        seed);
-      FSDataInputStream stream = fs.open(seekFile);
-      // Perform "safe seek" (expected to pass)
-      stream.seek(65536);
-      assertEquals(65536, stream.getPos());
-      // expect IOE for this call
-      stream.seek(ONEMB + ONEMB + ONEMB);
-    } finally {
-      fs.close();
-      cluster.shutdown();
-    }
+    Assertions.assertThrows(IOException.class, () -> {
+      Configuration conf = new HdfsConfiguration();
+      MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
+      FileSystem fs = cluster.getFileSystem();
+      try {
+        Path seekFile = new Path("seekboundaries.dat");
+        DFSTestUtil.createFile(
+            fs,
+            seekFile,
+            ONEMB,
+            ONEMB,
+            fs.getDefaultBlockSize(seekFile),
+            fs.getDefaultReplication(seekFile),
+            seed);
+        FSDataInputStream stream = fs.open(seekFile);
+        // Perform "safe seek" (expected to pass)
+        stream.seek(65536);
+        assertEquals(65536, stream.getPos());
+        // expect IOE for this call
+        stream.seek(ONEMB + ONEMB + ONEMB);
+      } finally {
+        fs.close();
+        cluster.shutdown();
+      }
+    });
   }
- 
+
   /**
    * Tests if the seek bug exists in FSDataInputStream in LocalFS.
    */

+ 13 - 13
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSetTimes.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.FileNotFoundException;
 import java.io.IOException;
@@ -40,8 +40,9 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapterMockitoUtil;
 import org.apache.hadoop.test.MockitoUtil;
 import org.apache.hadoop.util.Time;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 /**
  * This class tests the access time on files.
@@ -106,7 +107,7 @@ public class TestSetTimes {
                                                    cluster.getNameNodePort());
     DFSClient client = new DFSClient(addr, conf);
     DatanodeInfo[] info = client.datanodeReport(DatanodeReportType.LIVE);
-    assertEquals("Number of Datanodes ", numDatanodes, info.length);
+    assertEquals(numDatanodes, info.length, "Number of Datanodes ");
     FileSystem fileSys = cluster.getFileSystem();
     int replicas = 1;
     assertTrue(fileSys instanceof DistributedFileSystem);
@@ -183,10 +184,8 @@ public class TestSetTimes {
       fileSys.setTimes(dir1, mtime4, atime4);
       // check new modification time on file
       stat = fileSys.getFileStatus(dir1);
-      assertTrue("Not matching the modification times", mtime4 == stat
-          .getModificationTime());
-      assertTrue("Not matching the access times", atime4 == stat
-          .getAccessTime());
+      assertTrue(mtime4 == stat.getModificationTime(), "Not matching the modification times");
+      assertTrue(atime4 == stat.getAccessTime(), "Not matching the access times");
 
       Path nonExistingDir = new Path(dir1, "/nonExistingDir/");
       try {
@@ -246,7 +245,7 @@ public class TestSetTimes {
                                                      cluster.getNameNodePort());
     DFSClient client = new DFSClient(addr, conf);
     DatanodeInfo[] info = client.datanodeReport(DatanodeReportType.LIVE);
-    assertEquals("Number of Datanodes ", numDatanodes, info.length);
+    assertEquals(numDatanodes, info.length, "Number of Datanodes ");
     FileSystem fileSys = cluster.getFileSystem();
     assertTrue(fileSys instanceof DistributedFileSystem);
 
@@ -290,7 +289,8 @@ public class TestSetTimes {
    * write lock is not taken by getBlockLocations.
    * Regression test for HDFS-3981.
    */
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testGetBlockLocationsOnlyUsesReadLock() throws IOException {
     Configuration conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, 100*1000);
@@ -339,7 +339,7 @@ public class TestSetTimes {
       DFSTestUtil.createFile(cluster.getFileSystem(), p, 0, (short)1, 0L);
 
       fs.setTimes(p, -1L, 123456L);
-      Assert.assertEquals(123456L, fs.getFileStatus(p).getAccessTime());
+      Assertions.assertEquals(123456L, fs.getFileStatus(p).getAccessTime());
     } finally {
       if (cluster != null) {
         cluster.shutdown();

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

@@ -19,10 +19,12 @@ package org.apache.hadoop.hdfs;
 
 import java.io.IOException;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 public class TestSetrepDecreasing {
-  @Test(timeout=120000)
+  @Test
+  @Timeout(value = 120)
   public void testSetrepDecreasing() throws IOException {
     TestSetrepIncreasing.setrep(5, 3, false);
   }

+ 11 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSetrepIncreasing.java

@@ -17,8 +17,8 @@
  */
 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.ByteArrayOutputStream;
 import java.io.IOException;
@@ -32,7 +32,8 @@ import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 public class TestSetrepIncreasing {
   static void setrep(int fromREP, int toREP, boolean simulatedStorage) throws IOException {
@@ -45,7 +46,7 @@ public class TestSetrepIncreasing {
     conf.set(DFSConfigKeys.DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_KEY, Integer.toString(2));
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(10).build();
     FileSystem fs = cluster.getFileSystem();
-    assertTrue("Not a HDFS: "+fs.getUri(), fs instanceof DistributedFileSystem);
+    assertTrue(fs instanceof DistributedFileSystem, "Not a HDFS: " + fs.getUri());
 
     try {
       Path root = TestDFSShell.mkdir(fs, 
@@ -60,7 +61,7 @@ public class TestSetrepIncreasing {
         try {
           assertEquals(0, shell.run(args));
         } catch (Exception e) {
-          assertTrue("-setrep " + e, false);
+          assertTrue(false, "-setrep " + e);
         }
       }
 
@@ -78,11 +79,14 @@ public class TestSetrepIncreasing {
     }
   }
 
-  @Test(timeout=120000)
+  @Test
+  @Timeout(value = 120)
   public void testSetrepIncreasing() throws IOException {
     setrep(3, 7, false);
   }
-  @Test(timeout=120000)
+
+  @Test
+  @Timeout(value = 120)
   public void testSetrepIncreasingSimulatedStorage() throws IOException {
     setrep(3, 7, true);
   }

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

@@ -23,10 +23,10 @@ import java.util.Arrays;
 import java.util.List;
 
 import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -51,7 +51,7 @@ public class TestSlowDatanodeReport {
 
   private MiniDFSCluster cluster;
 
-  @Before
+  @BeforeEach
   public void testSetup() throws Exception {
     Configuration conf = new Configuration();
 
@@ -64,7 +64,7 @@ public class TestSlowDatanodeReport {
     cluster.waitActive();
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     cluster.shutdown();
   }
@@ -78,7 +78,7 @@ public class TestSlowDatanodeReport {
         ImmutableMap.of(slowNode.getDatanodeHostname() + ":" + slowNode.getIpcPort(),
             outlierMetrics));
     DistributedFileSystem distributedFileSystem = cluster.getFileSystem();
-    Assert.assertEquals(3, distributedFileSystem.getDataNodeStats().length);
+    Assertions.assertEquals(3, distributedFileSystem.getDataNodeStats().length);
     GenericTestUtils.waitFor(() -> {
       try {
         DatanodeInfo[] slowNodeInfo = distributedFileSystem.getSlowDatanodeStats();
@@ -90,13 +90,13 @@ public class TestSlowDatanodeReport {
       }
     }, 2000, 180000, "Slow nodes could not be detected");
     LOG.info("Slow peer report: {}", cluster.getNameNode().getSlowPeersReport());
-    Assert.assertTrue(cluster.getNameNode().getSlowPeersReport().length() > 0);
-    Assert.assertTrue(
+    Assertions.assertTrue(cluster.getNameNode().getSlowPeersReport().length() > 0);
+    Assertions.assertTrue(
         cluster.getNameNode().getSlowPeersReport().contains(slowNode.getDatanodeHostname()));
-    Assert.assertTrue(cluster.getNameNode().getSlowPeersReport().contains("15.5"));
-    Assert.assertTrue(cluster.getNameNode().getSlowPeersReport().contains("1.245"));
-    Assert.assertTrue(cluster.getNameNode().getSlowPeersReport().contains("2.69375"));
-    Assert.assertTrue(cluster.getNameNode().getSlowPeersReport().contains("4.5667"));
+    Assertions.assertTrue(cluster.getNameNode().getSlowPeersReport().contains("15.5"));
+    Assertions.assertTrue(cluster.getNameNode().getSlowPeersReport().contains("1.245"));
+    Assertions.assertTrue(cluster.getNameNode().getSlowPeersReport().contains("2.69375"));
+    Assertions.assertTrue(cluster.getNameNode().getSlowPeersReport().contains("4.5667"));
   }
 
   @Test
@@ -111,7 +111,7 @@ public class TestSlowDatanodeReport {
         dataNodes.get(2).getDatanodeHostname() + ":" + dataNodes.get(2).getIpcPort(),
         outlierMetrics2));
     DistributedFileSystem distributedFileSystem = cluster.getFileSystem();
-    Assert.assertEquals(3, distributedFileSystem.getDataNodeStats().length);
+    Assertions.assertEquals(3, distributedFileSystem.getDataNodeStats().length);
     GenericTestUtils.waitFor(() -> {
       try {
         DatanodeInfo[] slowNodeInfo = distributedFileSystem.getSlowDatanodeStats();
@@ -123,15 +123,15 @@ public class TestSlowDatanodeReport {
       }
     }, 2000, 200000, "Slow nodes could not be detected");
     LOG.info("Slow peer report: {}", cluster.getNameNode().getSlowPeersReport());
-    Assert.assertTrue(cluster.getNameNode().getSlowPeersReport().length() > 0);
-    Assert.assertTrue(cluster.getNameNode().getSlowPeersReport()
+    Assertions.assertTrue(cluster.getNameNode().getSlowPeersReport().length() > 0);
+    Assertions.assertTrue(cluster.getNameNode().getSlowPeersReport()
         .contains(dataNodes.get(1).getDatanodeHostname()));
-    Assert.assertTrue(cluster.getNameNode().getSlowPeersReport()
+    Assertions.assertTrue(cluster.getNameNode().getSlowPeersReport()
         .contains(dataNodes.get(2).getDatanodeHostname()));
-    Assert.assertTrue(cluster.getNameNode().getSlowPeersReport().contains("14.5"));
-    Assert.assertTrue(cluster.getNameNode().getSlowPeersReport().contains("18.7"));
-    Assert.assertTrue(cluster.getNameNode().getSlowPeersReport().contains("23.568204"));
-    Assert.assertTrue(cluster.getNameNode().getSlowPeersReport().contains("22.4945"));
+    Assertions.assertTrue(cluster.getNameNode().getSlowPeersReport().contains("14.5"));
+    Assertions.assertTrue(cluster.getNameNode().getSlowPeersReport().contains("18.7"));
+    Assertions.assertTrue(cluster.getNameNode().getSlowPeersReport().contains("23.568204"));
+    Assertions.assertTrue(cluster.getNameNode().getSlowPeersReport().contains("22.4945"));
   }
 
 }

+ 6 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSmallBlock.java

@@ -17,8 +17,8 @@
  */
 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.IOException;
 import java.util.Random;
@@ -30,7 +30,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 /**
  * This class tests the creation of files with block-size
@@ -44,9 +44,8 @@ public class TestSmallBlock {
   
   private void checkAndEraseData(byte[] actual, int from, byte[] expected, String message) {
     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;
     }
   }
@@ -55,7 +54,7 @@ public class TestSmallBlock {
       throws IOException {
     BlockLocation[] locations = fileSys.getFileBlockLocations(
         fileSys.getFileStatus(name), 0, fileSize);
-    assertEquals("Number of blocks", fileSize, locations.length);
+    assertEquals(fileSize, locations.length, "Number of blocks");
     FSDataInputStream stm = fileSys.open(name);
     byte[] expected = new byte[fileSize];
     if (simulatedStorage) {

+ 14 - 11
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSnapshotCommands.java

@@ -26,11 +26,12 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.tools.snapshot.SnapshotDiff;
 
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 /**
  * This class includes end-to-end tests for snapshot related FsShell and
@@ -42,7 +43,7 @@ public class TestSnapshotCommands {
   private static MiniDFSCluster cluster;
   private static DistributedFileSystem fs;
   
-  @BeforeClass
+  @BeforeAll
   public static void clusterSetUp() throws IOException {
     conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_MAX_LIMIT, 3);
@@ -51,7 +52,7 @@ public class TestSnapshotCommands {
     fs = cluster.getFileSystem();
   }
 
-  @AfterClass
+  @AfterAll
   public static void clusterShutdown() throws IOException{
     if(fs != null){
       fs.close();
@@ -61,7 +62,7 @@ public class TestSnapshotCommands {
     }
   }
 
-  @Before
+  @BeforeEach
   public void setUp() throws IOException {
     fs.mkdirs(new Path("/sub1"));
     fs.mkdirs(new Path("/Fully/QPath"));
@@ -70,7 +71,7 @@ public class TestSnapshotCommands {
     fs.mkdirs(new Path("/sub1/sub1sub2"));
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws IOException {
     if (fs.exists(new Path("/sub1"))) {
       if (fs.exists(new Path("/sub1/.snapshot"))) {
@@ -193,7 +194,8 @@ public class TestSnapshotCommands {
     DFSTestUtil.FsShellRun("-rmr /sub1", conf);
   }
 
-  @Test (timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testSnapshotCommandsWithURI()throws Exception {
     Configuration config = new HdfsConfiguration();
     //fs.defaultFS should not be used, when path is fully qualified.
@@ -224,7 +226,8 @@ public class TestSnapshotCommands {
     fs.delete(new Path("/Fully/QPath"), true);
   }
 
-  @Test (timeout=120000)
+  @Test
+  @Timeout(value = 120)
   public void testSnapshotDiff()throws Exception {
     Configuration config = new HdfsConfiguration();
     Path snapDirPath = new Path(fs.getUri().toString() + "/snap_dir");

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

@@ -18,8 +18,8 @@
 
 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 org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -30,11 +30,12 @@ import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAProxyFactory;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
 import org.apache.hadoop.hdfs.server.namenode.ha.ObserverReadProxyProvider;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -87,7 +88,7 @@ public class TestStateAlignmentContextWithHA {
     }
   }
 
-  @BeforeClass
+  @BeforeAll
   public static void startUpCluster() throws IOException {
     // Set short retry timeouts so this test runs faster
     CONF.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 10);
@@ -100,20 +101,20 @@ public class TestStateAlignmentContextWithHA {
     cluster = qjmhaCluster.getDfsCluster();
   }
 
-  @Before
+  @BeforeEach
   public void before() throws IOException, URISyntaxException {
     dfs = HATestUtil.configureObserverReadFs(
         cluster, CONF, ORPPwithAlignmentContexts.class, true);
   }
 
-  @AfterClass
+  @AfterAll
   public static void shutDownCluster() throws IOException {
     if (qjmhaCluster != null) {
       qjmhaCluster.shutdown();
     }
   }
 
-  @After
+  @AfterEach
   public void after() throws IOException {
     killWorkers();
     cluster.transitionToStandby(1);
@@ -211,7 +212,8 @@ public class TestStateAlignmentContextWithHA {
     assertEquals(clientStateFO, writeStateFO);
   }
 
-  @Test(timeout=300000)
+  @Test
+  @Timeout(value = 300)
   public void testMultiClientStatesWithRandomFailovers() throws Exception {
     // First run, half the load, with one failover.
     runClientsWithFailover(1, NUMCLIENTS/2, NUMFILES/2);

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

@@ -17,8 +17,8 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
 
 import java.io.IOException;
 
@@ -33,9 +33,9 @@ import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.LambdaTestUtils;
 import org.apache.hadoop.test.ReflectionUtils;
-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;
 
 public class TestStoragePolicyPermissionSettings {
 
@@ -50,7 +50,7 @@ public class TestStoragePolicyPermissionSettings {
   private static UserGroupInformation nonAdmin;
   private static UserGroupInformation admin;
 
-  @BeforeClass
+  @BeforeAll
   public static void clusterSetUp() throws IOException {
     conf = new HdfsConfiguration();
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPL).build();
@@ -64,7 +64,7 @@ public class TestStoragePolicyPermissionSettings {
         new String[]{"supergroup"});
   }
 
-  @AfterClass
+  @AfterAll
   public static void clusterShutdown() throws IOException {
     if (fs != null) {
       fs.close();

+ 10 - 11
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestStripedFileAppend.java

@@ -24,9 +24,9 @@ import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.event.Level;
@@ -38,10 +38,10 @@ import java.util.EnumSet;
 import java.util.List;
 import java.util.Random;
 
-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;
 
 /**
  * Tests append on erasure coded file.
@@ -68,7 +68,7 @@ public class TestStripedFileAppend {
   private Path dir = new Path("/TestFileAppendStriped");
   private HdfsConfiguration conf = new HdfsConfiguration();
 
-  @Before
+  @BeforeEach
   public void setup() throws IOException {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DN).build();
@@ -78,7 +78,7 @@ public class TestStripedFileAppend {
     dfs.setErasureCodingPolicy(dir, null);
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws IOException {
     if (cluster != null) {
       cluster.shutdown();
@@ -143,7 +143,6 @@ public class TestStripedFileAppend {
 
     RemoteIterator<OpenFileEntry> listOpenFiles = dfs
         .listOpenFiles(EnumSet.copyOf(types), file.toString());
-    assertFalse("No file should be open after append failure",
-        listOpenFiles.hasNext());
+    assertFalse(listOpenFiles.hasNext(), "No file should be open after append failure");
   }
 }

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

@@ -18,7 +18,7 @@
 package org.apache.hadoop.hdfs;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
-import static org.junit.Assert.*;
+import static org.junit.jupiter.api.Assertions.*;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.JavaKeyStoreProvider;
@@ -36,9 +36,9 @@ import org.apache.hadoop.hdfs.server.namenode.EncryptionZoneManager;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.ToolRunner;
-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.LoggerFactory;
 import org.slf4j.event.Level;
 
@@ -78,7 +78,7 @@ public class TestTrashWithEncryptionZones {
         new Path(testRootDir.toString(), "test.jks").toUri();
   }
 
-  @Before
+  @BeforeEach
   public void setup() throws Exception {
     conf = new HdfsConfiguration();
     fsHelper = new FileSystemTestHelper();
@@ -114,7 +114,7 @@ public class TestTrashWithEncryptionZones {
         .getProvider());
   }
 
-  @After
+  @AfterEach
   public void teardown() {
     if (cluster != null) {
       cluster.shutdown();
@@ -178,9 +178,9 @@ public class TestTrashWithEncryptionZones {
         // Delete /zones/zone1 should not succeed as current user is not admin
         String[] argv = new String[]{"-rm", "-r", zone1.toString()};
         int res = ToolRunner.run(shell, argv);
-        assertEquals("Non-admin could delete an encryption zone with multiple" +
-            " users : " + zone1, 1, res);
-        return null;
+            assertEquals(1, res,
+                "Non-admin could delete an encryption zone with multiple" + " users : " + zone1);
+            return null;
       }
     });
 

+ 33 - 38
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestTrashWithSecureEncryptionZones.java

@@ -18,9 +18,9 @@
 package org.apache.hadoop.hdfs;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
-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 static org.apache.hadoop.fs.CommonConfigurationKeys
     .IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SASL_KEY;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic
@@ -51,11 +51,11 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
 import org.apache.hadoop.util.Lists;
 import org.apache.hadoop.util.ToolRunner;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.FixMethodOrder;
-import org.junit.Test;
-import org.junit.runners.MethodSorters;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.MethodOrderer;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestMethodOrder;
 
 import java.io.File;
 import java.io.FileWriter;
@@ -70,7 +70,7 @@ import java.util.concurrent.atomic.AtomicInteger;
  * This class tests Trash functionality in Encryption Zones with Kerberos
  * enabled.
  */
-@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+@TestMethodOrder(MethodOrderer.MethodName.class)
 public class TestTrashWithSecureEncryptionZones {
   private static HdfsConfiguration baseConf;
   private static File baseDir;
@@ -118,7 +118,7 @@ public class TestTrashWithSecureEncryptionZones {
     return file;
   }
 
-  @BeforeClass
+  @BeforeAll
   public static void init() throws Exception {
     baseDir = getTestDir();
     FileUtil.fullyDelete(baseDir);
@@ -132,8 +132,8 @@ public class TestTrashWithSecureEncryptionZones {
     SecurityUtil.setAuthenticationMethod(UserGroupInformation
         .AuthenticationMethod.KERBEROS, baseConf);
     UserGroupInformation.setConfiguration(baseConf);
-    assertTrue("Expected configuration to enable security",
-        UserGroupInformation.isSecurityEnabled());
+    assertTrue(UserGroupInformation.isSecurityEnabled(),
+        "Expected configuration to enable security");
 
     File keytabFile = new File(baseDir, "test.keytab");
     keytab = keytabFile.getAbsolutePath();
@@ -227,7 +227,7 @@ public class TestTrashWithSecureEncryptionZones {
     shell = new FsShell(clientConf);
   }
 
-  @AfterClass
+  @AfterAll
   public static void destroy() {
     IOUtils.cleanupWithLogger(null, fs);
     if (cluster != null) {
@@ -267,16 +267,16 @@ public class TestTrashWithSecureEncryptionZones {
         "/" + CURRENT);
     String trashPath = trashDir.toString() + encFile1.toString();
     Path deletedFile = verifyTrashLocationWithShellDelete(encFile1);
-    assertEquals("Deleted file not at the expected trash location: " +
-        trashPath, trashPath, deletedFile.toUri().getPath());
+    assertEquals(trashPath, deletedFile.toUri().getPath(),
+        "Deleted file not at the expected trash location: " + trashPath);
 
     //Verify Trash checkpoint outside the encryption zone when the whole
     // encryption zone is deleted and moved
     trashPath = fs.getHomeDirectory().toUri().getPath() + "/" + fs
         .TRASH_PREFIX + "/" + CURRENT + zone2;
     Path deletedDir = verifyTrashLocationWithShellDelete(zone2);
-    assertEquals("Deleted zone not at the expected trash location: " +
-        trashPath, trashPath, deletedDir.toUri().getPath());
+    assertEquals(trashPath, deletedDir.toUri().getPath(),
+        "Deleted zone not at the expected trash location: " + trashPath);
   }
 
   @Test
@@ -344,16 +344,14 @@ public class TestTrashWithSecureEncryptionZones {
     //Delete empty directory with -r option
     String[] argv1 = new String[]{"-rm", "-r", zone1.toString()};
     int res = ToolRunner.run(shell, argv1);
-    assertEquals("rm failed", 0, res);
-    assertTrue("Empty directory not deleted even with -r : " + trashDir1, fs
-        .exists(trashDir1));
+    assertEquals(0, res, "rm failed");
+    assertTrue(fs.exists(trashDir1), "Empty directory not deleted even with -r : " + trashDir1);
 
     //Delete empty directory without -r option
     String[] argv2 = new String[]{"-rm", zone2.toString()};
     res = ToolRunner.run(shell, argv2);
-    assertEquals("rm on empty directory did not fail", 1, res);
-    assertTrue("Empty directory deleted without -r : " + trashDir2, !fs.exists(
-        trashDir2));
+    assertEquals(1, res, "rm on empty directory did not fail");
+    assertTrue(!fs.exists(trashDir2), "Empty directory deleted without -r : " + trashDir2);
   }
 
   @Test
@@ -371,12 +369,12 @@ public class TestTrashWithSecureEncryptionZones {
 
     String[] argv = new String[]{"-rm", "-r", encFile1.toString()};
     int res = ToolRunner.run(shell, argv);
-    assertEquals("rm failed", 0, res);
+    assertEquals(0, res, "rm failed");
 
     String[] argvDeleteTrash = new String[]{"-rm", "-r", trashFile.toString()};
     int resDeleteTrash = ToolRunner.run(shell, argvDeleteTrash);
-    assertEquals("rm failed", 0, resDeleteTrash);
-    assertFalse("File deleted from Trash : " + trashFile, fs.exists(trashFile));
+    assertEquals(0, resDeleteTrash, "rm failed");
+    assertFalse(fs.exists(trashFile), "File deleted from Trash : " + trashFile);
   }
 
   @Test
@@ -393,15 +391,14 @@ public class TestTrashWithSecureEncryptionZones {
         encFile1);
     String[] argv = new String[]{"-rm", "-r", encFile1.toString()};
     int res = ToolRunner.run(shell, argv);
-    assertEquals("rm failed", 0, res);
+    assertEquals(0, res, "rm failed");
 
-    assertTrue("File not in trash : " + trashFile, fs.exists(trashFile));
+    assertTrue(fs.exists(trashFile), "File not in trash : " + trashFile);
     cluster.restartNameNode(0);
     cluster.waitActive();
     fs = cluster.getFileSystem();
 
-    assertTrue("On Namenode restart, file deleted from trash : " +
-        trashFile, fs.exists(trashFile));
+    assertTrue(fs.exists(trashFile), "On Namenode restart, file deleted from trash : " + trashFile);
   }
 
   private Path verifyTrashLocationWithShellDelete(Path path)
@@ -410,8 +407,7 @@ public class TestTrashWithSecureEncryptionZones {
     final Path trashFile = new Path(shell.getCurrentTrashDir(path) + "/" +
         path);
     File deletedFile = new File(String.valueOf(trashFile));
-    assertFalse("File already present in Trash before delete", deletedFile
-        .exists());
+    assertFalse(deletedFile.exists(), "File already present in Trash before delete");
 
     DFSTestUtil.verifyDelete(shell, fs, path, trashFile, true);
     return trashFile;
@@ -420,23 +416,22 @@ public class TestTrashWithSecureEncryptionZones {
   private void verifyTrashExpunge(List<Path> trashFiles) throws Exception {
     String[] argv = new String[]{"-expunge"};
     int res = ToolRunner.run(shell, argv);
-    assertEquals("expunge failed", 0, res);
+    assertEquals(0, res, "expunge failed");
 
     for (Path trashFile : trashFiles) {
-      assertFalse("File exists in trash after expunge : " + trashFile, fs
-          .exists(trashFile));
+      assertFalse(fs.exists(trashFile), "File exists in trash after expunge : " + trashFile);
     }
   }
 
   private void verifyDeleteWithSkipTrash(Path path) throws Exception {
-    assertTrue(path + " file does not exist", fs.exists(path));
+    assertTrue(fs.exists(path), path + " file does not exist");
 
     final Path trashFile = new Path(shell.getCurrentTrashDir(path) + "/" +
         path);
 
     String[] argv = new String[]{"-rm", "-r", "-skipTrash", path.toString()};
     int res = ToolRunner.run(shell, argv);
-    assertEquals("rm failed", 0, res);
-    assertFalse("File in trash even with -skipTrash", fs.exists(trashFile));
+    assertEquals(0, res, "rm failed");
+    assertFalse(fs.exists(trashFile), "File in trash even with -skipTrash");
   }
 }

+ 46 - 53
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestUnsetAndChangeDirectoryEcPolicy.java

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs;
 
+import org.junit.jupiter.api.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -27,22 +28,21 @@ import org.apache.hadoop.hdfs.protocol.NoECPolicySetException;
 import org.apache.hadoop.io.erasurecode.CodecUtil;
 import org.apache.hadoop.io.erasurecode.ErasureCodeNative;
 import org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawErasureCoderFactory;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
-import org.junit.Assert;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Assertions;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
 
 import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.fail;
 
 /**
  * Test unset and change directory's erasure coding policy.
  */
+@Timeout(300)
 public class TestUnsetAndChangeDirectoryEcPolicy {
 
   public static final Logger LOG =
@@ -59,10 +59,7 @@ public class TestUnsetAndChangeDirectoryEcPolicy {
   private final int blockSize = stripsPerBlock * cellSize;
   private final int blockGroupSize =  dataBlocks * blockSize;
 
-  @Rule
-  public Timeout globalTimeout = new Timeout(300000);
-
-  @Before
+  @BeforeEach
   public void setup() throws IOException {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
@@ -78,7 +75,7 @@ public class TestUnsetAndChangeDirectoryEcPolicy {
     DFSTestUtil.enableAllECPolicies(fs);
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
     if (cluster != null) {
       cluster.shutdown();
@@ -114,18 +111,16 @@ public class TestUnsetAndChangeDirectoryEcPolicy {
     // ec_file should has EC policy
     ErasureCodingPolicy tempEcPolicy =
         fs.getErasureCodingPolicy(ecFilePath);
-    Assert.assertTrue("Erasure coding policy mismatch!",
-        tempEcPolicy.getName().equals(ecPolicy.getName()));
+    Assertions.assertTrue(tempEcPolicy.getName().equals(ecPolicy.getName()),
+        "Erasure coding policy mismatch!");
 
     // rep_file should not have EC policy
     tempEcPolicy = fs.getErasureCodingPolicy(replicateFilePath);
-    Assert.assertNull("Replicate file should not have erasure coding policy!",
-        tempEcPolicy);
+    Assertions.assertNull(tempEcPolicy, "Replicate file should not have erasure coding policy!");
 
     // Directory should not return erasure coding policy
     tempEcPolicy = fs.getErasureCodingPolicy(dirPath);
-    Assert.assertNull("Directory should no have erasure coding policy set!",
-        tempEcPolicy);
+    Assertions.assertNull(tempEcPolicy, "Directory should no have erasure coding policy set!");
 
     fs.delete(dirPath, true);
   }
@@ -159,19 +154,20 @@ public class TestUnsetAndChangeDirectoryEcPolicy {
     // ec_6_3_file should has RS-6-3 EC policy
     ErasureCodingPolicy tempEcPolicy =
         fs.getErasureCodingPolicy(ec63FilePath);
-    Assert.assertTrue("Erasure coding policy mismatch!",
-        tempEcPolicy.getName().equals(ecPolicy.getName()));
+    Assertions.assertTrue(tempEcPolicy.getName().equals(ecPolicy.getName()),
+        "Erasure coding policy mismatch!");
 
     // ec_3_2_file should have RS-3-2 policy
     tempEcPolicy = fs.getErasureCodingPolicy(ec32FilePath);
-    Assert.assertTrue("Erasure coding policy mismatch!",
-        tempEcPolicy.getName().equals(ec32Policy.getName()));
+    Assertions.assertTrue(tempEcPolicy.getName().equals(ec32Policy.getName()),
+        "Erasure coding policy mismatch!");
 
     // Child directory should have RS-3-2 policy
     tempEcPolicy = fs.getErasureCodingPolicy(childDir);
-    Assert.assertTrue(
-        "Directory should have erasure coding policy set!",
-        tempEcPolicy.getName().equals(ec32Policy.getName()));
+    Assertions.assertTrue(
+
+        tempEcPolicy.getName().equals(ec32Policy.getName()),
+        "Directory should have erasure coding policy set!");
 
     // Unset EC policy on child directory
     fs.unsetErasureCodingPolicy(childDir);
@@ -179,14 +175,15 @@ public class TestUnsetAndChangeDirectoryEcPolicy {
 
     // ec_6_3_file_2 should have RS-6-3 policy
     tempEcPolicy = fs.getErasureCodingPolicy(ec63FilePath2);
-    Assert.assertTrue("Erasure coding policy mismatch!",
-        tempEcPolicy.getName().equals(ecPolicy.getName()));
+    Assertions.assertTrue(tempEcPolicy.getName().equals(ecPolicy.getName()),
+        "Erasure coding policy mismatch!");
 
     // Child directory should have RS-6-3 policy now
     tempEcPolicy = fs.getErasureCodingPolicy(childDir);
-    Assert.assertTrue(
-        "Directory should have erasure coding policy set!",
-        tempEcPolicy.getName().equals(ecPolicy.getName()));
+    Assertions.assertTrue(
+
+        tempEcPolicy.getName().equals(ecPolicy.getName()),
+        "Directory should have erasure coding policy set!");
 
     fs.delete(parentDir, true);
   }
@@ -218,18 +215,16 @@ public class TestUnsetAndChangeDirectoryEcPolicy {
     // ec_file should has EC policy set
     ErasureCodingPolicy tempEcPolicy =
         fs.getErasureCodingPolicy(ecFilePath);
-    Assert.assertTrue("Erasure coding policy mismatch!",
-        tempEcPolicy.getName().equals(ecPolicy.getName()));
+    Assertions.assertTrue(tempEcPolicy.getName().equals(ecPolicy.getName()),
+        "Erasure coding policy mismatch!");
 
     // rep_file should not have EC policy set
     tempEcPolicy = fs.getErasureCodingPolicy(replicateFilePath);
-    Assert.assertNull("Replicate file should not have erasure coding policy!",
-        tempEcPolicy);
+    Assertions.assertNull(tempEcPolicy, "Replicate file should not have erasure coding policy!");
 
     // Directory should not return erasure coding policy
     tempEcPolicy = fs.getErasureCodingPolicy(rootPath);
-    Assert.assertNull("Directory should not have erasure coding policy set!",
-        tempEcPolicy);
+    Assertions.assertNull(tempEcPolicy, "Directory should not have erasure coding policy set!");
 
     fs.delete(rootPath, true);
   }
@@ -263,19 +258,20 @@ public class TestUnsetAndChangeDirectoryEcPolicy {
     // ec_6_3_file should has RS-6-3 ec policy set
     ErasureCodingPolicy tempEcPolicy =
         fs.getErasureCodingPolicy(ec63FilePath);
-    Assert.assertTrue("Erasure coding policy mismatch!",
-        tempEcPolicy.getName().equals(ecPolicy.getName()));
+    Assertions.assertTrue(tempEcPolicy.getName().equals(ecPolicy.getName()),
+        "Erasure coding policy mismatch!");
 
     // ec_3_2_file should have RS-3-2 policy
     tempEcPolicy = fs.getErasureCodingPolicy(ec32FilePath);
-    Assert.assertTrue("Erasure coding policy mismatch!",
-        tempEcPolicy.getName().equals(ec32Policy.getName()));
+    Assertions.assertTrue(tempEcPolicy.getName().equals(ec32Policy.getName()),
+        "Erasure coding policy mismatch!");
 
     // Root directory should have RS-3-2 policy
     tempEcPolicy = fs.getErasureCodingPolicy(rootPath);
-    Assert.assertTrue(
-        "Directory should have erasure coding policy!",
-        tempEcPolicy.getName().equals(ec32Policy.getName()));
+    Assertions.assertTrue(
+
+        tempEcPolicy.getName().equals(ec32Policy.getName()),
+        "Directory should have erasure coding policy!");
 
     fs.delete(rootPath, true);
   }
@@ -302,21 +298,18 @@ public class TestUnsetAndChangeDirectoryEcPolicy {
     // ec_file should has EC policy set
     ErasureCodingPolicy tempEcPolicy =
         fs.getErasureCodingPolicy(ecFilePath);
-    Assert.assertTrue("Erasure coding policy mismatch!",
-        tempEcPolicy.getName().equals(ecPolicy.getName()));
+    Assertions.assertTrue(tempEcPolicy.getName().equals(ecPolicy.getName()),
+        "Erasure coding policy mismatch!");
 
     // rep_file should not have EC policy set
     tempEcPolicy = fs.getErasureCodingPolicy(replicateFilePath);
-    Assert.assertNull("Replicate file should not have erasure coding policy!",
-        tempEcPolicy);
+    Assertions.assertNull(tempEcPolicy, "Replicate file should not have erasure coding policy!");
     tempEcPolicy = fs.getErasureCodingPolicy(replicateFilePath2);
-    Assert.assertNull("Replicate file should not have erasure coding policy!",
-        tempEcPolicy);
+    Assertions.assertNull(tempEcPolicy, "Replicate file should not have erasure coding policy!");
 
     // Directory should not return erasure coding policy
     tempEcPolicy = fs.getErasureCodingPolicy(ecDirPath);
-    Assert.assertNull("Directory should not have erasure coding policy set!",
-        tempEcPolicy);
+    Assertions.assertNull(tempEcPolicy, "Directory should not have erasure coding policy set!");
 
     fs.delete(ecDirPath, true);
   }
@@ -382,10 +375,10 @@ public class TestUnsetAndChangeDirectoryEcPolicy {
   @Test
   public void testUnsetEcPolicyInEditLog() throws IOException {
     fs.getClient().setErasureCodingPolicy("/", ecPolicy.getName());
-    Assert.assertEquals(ecPolicy, fs.getErasureCodingPolicy(new Path("/")));
+    Assertions.assertEquals(ecPolicy, fs.getErasureCodingPolicy(new Path("/")));
     fs.getClient().unsetErasureCodingPolicy("/");
 
     cluster.restartNameNode(true);
-    Assert.assertNull(fs.getErasureCodingPolicy(new Path("/")));
+    Assertions.assertNull(fs.getErasureCodingPolicy(new Path("/")));
   }
 }

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

@@ -32,18 +32,17 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.test.Whitebox;
 import org.apache.hadoop.util.functional.ConsumerRaisingIOE;
 
-import org.assertj.core.api.Assertions;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
 
 import java.io.IOException;
 import java.net.URI;
 
 import static org.apache.hadoop.fs.CommonPathCapabilities.LEASE_RECOVERABLE;
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class TestViewDistributedFileSystem extends TestDistributedFileSystem{
   @Override
@@ -122,8 +121,8 @@ public class TestViewDistributedFileSystem extends TestDistributedFileSystem{
         final Path renameDir = new Path("/testRename");
         fileSystem.mkdirs(testDir);
         fileSystem.rename(testDir, renameDir, Options.Rename.TO_TRASH);
-        Assert.assertTrue(fileSystem.exists(renameDir));
-        Assert.assertFalse(fileSystem.exists(testDir));
+        Assertions.assertTrue(fileSystem.exists(renameDir));
+        Assertions.assertFalse(fileSystem.exists(testDir));
       }
     } finally {
       if (cluster != null) {
@@ -191,8 +190,7 @@ public class TestViewDistributedFileSystem extends TestDistributedFileSystem{
         // Set Quota via ViewDFS
         fileSystem.setQuota(testDir, 10L, 10L);
         // Check quota through actual DFS
-        assertEquals(10,
-            defaultFs.getQuotaUsage(target).getSpaceQuota());
+        assertEquals(10, defaultFs.getQuotaUsage(target).getSpaceQuota());
       }
     } finally {
       if (cluster != null) {
@@ -208,8 +206,8 @@ public class TestViewDistributedFileSystem extends TestDistributedFileSystem{
         ViewDistributedFileSystem fileSystem = (ViewDistributedFileSystem) FileSystem.get(
             cluster.getConfiguration(0))) {
       final Path testFile = new Path("/test");
-      assertTrue("ViewDfs supports truncate",
-          fileSystem.hasPathCapability(testFile, CommonPathCapabilities.FS_TRUNCATE));
+      assertTrue(fileSystem.hasPathCapability(testFile, CommonPathCapabilities.FS_TRUNCATE),
+          "ViewDfs supports truncate");
       final boolean isLeaseRecoverable = fileSystem.hasPathCapability(testFile, LEASE_RECOVERABLE);
       assertThat(isLeaseRecoverable).describedAs("path capabilities %s=%s in %s",
           LEASE_RECOVERABLE, fileSystem.hasPathCapability(testFile, LEASE_RECOVERABLE),
@@ -241,7 +239,7 @@ public class TestViewDistributedFileSystem extends TestDistributedFileSystem{
   }
 
   private SafeMode verifyAndGetSafeModeInstance(FileSystem fs) {
-    Assertions.assertThat(fs)
+    assertThat(fs)
         .describedAs("File system %s must be an instance of %s", fs, SafeMode.class.getClass())
         .isInstanceOf(SafeMode.class);
     return (SafeMode) fs;
@@ -278,15 +276,14 @@ public class TestViewDistributedFileSystem extends TestDistributedFileSystem{
 
   private void assertSafeModeStatus(SafeMode fsWithSafeMode, SafeModeAction action,
       boolean expectedStatus, String message) throws IOException {
-    Assertions.assertThat(fsWithSafeMode.setSafeMode(action)).describedAs(message)
+    assertThat(fsWithSafeMode.setSafeMode(action)).describedAs(message)
         .isEqualTo(expectedStatus);
   }
 
   private void assertSafeModeStatus(ViewDistributedFileSystem fileSystem,
       HdfsConstants.SafeModeAction action, boolean expectedStatus, String message)
       throws IOException {
-    Assertions.assertThat(fileSystem.setSafeMode(action)).describedAs(message)
-        .isEqualTo(expectedStatus);
+    assertThat(fileSystem.setSafeMode(action)).describedAs(message).isEqualTo(expectedStatus);
   }
 
   private Configuration getViewFsConfiguration() {

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

@@ -26,11 +26,10 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.*;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Test;
-
-import static org.hamcrest.core.Is.is;
-import static org.junit.Assert.*;
+import org.junit.jupiter.api.Test;
 
+import static org.junit.jupiter.api.Assertions.*;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /**
  * Test to verify that the DFSClient passes the expected block length to
@@ -101,7 +100,7 @@ public class TestWriteBlockGetsBlockLengthHint {
     public synchronized ReplicaHandler createRbw(StorageType storageType,
         String storageId, ExtendedBlock b, boolean allowLazyPersist)
         throws IOException {
-      assertThat(b.getLocalBlock().getNumBytes(), is(EXPECTED_BLOCK_LENGTH));
+      assertThat(b.getLocalBlock().getNumBytes()).isEqualTo(EXPECTED_BLOCK_LENGTH);
       return super.createRbw(storageType, storageId, b, allowLazyPersist);
     }
   }

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

@@ -23,7 +23,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 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;
 
 /**
  * Regression test for HDFS-1542, a deadlock between the main thread
@@ -31,7 +32,8 @@ import org.junit.Test;
  * Configuration.writeXML holds a lock on itself while writing to DFS.
  */
 public class TestWriteConfigurationToDFS {
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testWriteConf() throws Exception {
     Configuration conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 4096);

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

@@ -32,10 +32,10 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 public class TestWriteRead {
 
@@ -67,7 +67,7 @@ public class TestWriteRead {
   private static final Logger LOG =
       LoggerFactory.getLogger(TestWriteRead.class);
 
-  @Before
+  @BeforeEach
   public void initJunitModeTest() throws Exception {
     LOG.info("initJunitModeTest");
 
@@ -85,7 +85,7 @@ public class TestWriteRead {
     mfs.mkdirs(rootdir);
   }
 
-  @After
+  @AfterEach
   public void shutdown() {
     if (cluster != null) {
       cluster.shutdown();
@@ -113,7 +113,7 @@ public class TestWriteRead {
     // need to run long enough to fail: takes 25 to 35 seec on Mac
     int stat = testWriteAndRead(fname, WR_NTIMES, WR_CHUNK_SIZE, rdBeginPos);
     LOG.info("Summary status from test1: status= " + stat);
-    Assert.assertEquals(0, stat);
+    Assertions.assertEquals(0, stat);
   }
 
   /** Junit Test position read while writing. */
@@ -123,7 +123,7 @@ public class TestWriteRead {
     positionReadOption = true;   // position read
     long rdBeginPos = 0;
     int stat = testWriteAndRead(fname, WR_NTIMES, WR_CHUNK_SIZE, rdBeginPos);
-    Assert.assertEquals(0, stat);
+    Assertions.assertEquals(0, stat);
   }
 
   /** Junit Test position read of the current block being written. */
@@ -135,7 +135,7 @@ public class TestWriteRead {
     long rdBeginPos = blockSize+1;
     int numTimes=5;
     int stat = testWriteAndRead(fname, numTimes, wrChunkSize, rdBeginPos);
-    Assert.assertEquals(0, stat);
+    Assertions.assertEquals(0, stat);
   }
    
   // equivalent of TestWriteRead1

+ 10 - 14
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs;
 
+import org.junit.jupiter.api.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.event.Level;
@@ -32,18 +33,17 @@ import org.apache.hadoop.hdfs.web.WebHdfsConstants;
 import org.apache.hadoop.hdfs.web.WebHdfsTestUtil;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.After;
-import org.junit.Assert;
-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.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.Random;
 
+@Timeout(300)
 public class TestWriteReadStripedFile {
   public static final Logger LOG =
       LoggerFactory.getLogger(TestWriteReadStripedFile.class);
@@ -69,10 +69,7 @@ public class TestWriteReadStripedFile {
     GenericTestUtils.setLogLevel(BlockPlacementPolicy.LOG, Level.TRACE);
   }
 
-  @Rule
-  public Timeout globalTimeout = new Timeout(300000);
-
-  @Before
+  @BeforeEach
   public void setup() throws IOException {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
@@ -83,7 +80,7 @@ public class TestWriteReadStripedFile {
         ecPolicy.getName());
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws IOException {
     if (cluster != null) {
       cluster.shutdown();
@@ -322,10 +319,9 @@ public class TestWriteReadStripedFile {
     }
     try {
       fs.concat(target, srcs);
-      Assert.fail("non-ec file shouldn't concat with ec file");
+      Assertions.fail("non-ec file shouldn't concat with ec file");
     } catch (RemoteException e){
-      Assert.assertTrue(e.getMessage()
-          .contains("have different erasure coding policy"));
+      Assertions.assertTrue(e.getMessage().contains("have different erasure coding policy"));
     }
   }
 }

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

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs;
 
+import org.junit.jupiter.api.Disabled;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -25,9 +26,9 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Assert;
-import org.junit.Ignore;
-import org.junit.Test;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.slf4j.event.Level;
 
 import java.io.IOException;
@@ -71,8 +72,9 @@ public class TestWriteStripedFileWithFailure {
 
   // Test writing file with some Datanodes failure
   // TODO: enable this test after HDFS-8704 and HDFS-9040
-  @Ignore
-  @Test(timeout = 300000)
+  @Disabled
+  @Test
+  @Timeout(value = 300)
   public void testWriteStripedFileWithDNFailure() throws IOException {
     for (int fileLength : fileLengths) {
       for (int dataDelNum = 1; dataDelNum <= parityBlocks; dataDelNum++) {
@@ -125,10 +127,10 @@ public class TestWriteStripedFileWithFailure {
 
     int[] dataDNFailureIndices = StripedFileTestUtil.randomArray(0, dataBlocks,
         dataDNFailureNum);
-    Assert.assertNotNull(dataDNFailureIndices);
+    Assertions.assertNotNull(dataDNFailureIndices);
     int[] parityDNFailureIndices = StripedFileTestUtil.randomArray(dataBlocks,
         dataBlocks + parityBlocks, parityDNFailureNum);
-    Assert.assertNotNull(parityDNFailureIndices);
+    Assertions.assertNotNull(parityDNFailureIndices);
 
     int[] failedDataNodes = new int[dataDNFailureNum + parityDNFailureNum];
     System.arraycopy(dataDNFailureIndices, 0, failedDataNodes,
@@ -150,7 +152,7 @@ public class TestWriteStripedFileWithFailure {
 
     // make sure the expected number of Datanode have been killed
     int dnFailureNum = dataDNFailureNum + parityDNFailureNum;
-    Assert.assertEquals(cluster.getDataNodes().size(), numDNs - dnFailureNum);
+    Assertions.assertEquals(cluster.getDataNodes().size(), numDNs - dnFailureNum);
 
     byte[] smallBuf = new byte[1024];
     byte[] largeBuf = new byte[fileLength + 100];