Procházet zdrojové kódy

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

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

Co-authored-by: Shilun Fan <slfan1989@apache.org>
Reviewed-by: Shilun Fan <slfan1989@apache.org>
Signed-off-by: Shilun Fan <slfan1989@apache.org>
zhtttylz před 1 měsícem
rodič
revize
93b0453f72
43 změnil soubory, kde provedl 1092 přidání a 1003 odebrání
  1. 9 10
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AppendTestUtil.java
  2. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/FileAppendTest4.java
  3. 10 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/ReadStripedFileWithDecodingHelper.java
  4. 24 22
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
  5. 9 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAbandonBlock.java
  6. 159 150
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAclsEndToEnd.java
  7. 11 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAppendDifferentChecksum.java
  8. 17 12
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAppendSnapshotTruncate.java
  9. 7 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestApplyingStoragePolicy.java
  10. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBalancerBandwidth.java
  11. 43 44
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBatchedListDirectories.java
  12. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockMissingException.java
  13. 88 82
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
  14. 12 12
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlocksScheduledCounter.java
  15. 9 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestByteBufferPread.java
  16. 36 33
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
  17. 16 14
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java
  18. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClose.java
  19. 8 10
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestConnCache.java
  20. 17 13
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestCrcCorruption.java
  21. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSAddressConfig.java
  22. 14 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientExcludedNodes.java
  23. 32 30
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
  24. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientSocketSize.java
  25. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSFinalize.java
  26. 185 176
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java
  27. 16 19
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStreamKerberized.java
  28. 22 17
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInputStream.java
  29. 59 52
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInputStreamBlockLocations.java
  30. 13 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSMkdirs.java
  31. 31 27
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
  32. 43 42
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java
  33. 8 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataStream.java
  34. 18 13
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferKeepalive.java
  35. 7 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
  36. 13 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeConfig.java
  37. 10 12
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeDeath.java
  38. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeLayoutUpgrade.java
  39. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeReport.java
  40. 12 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeStartupFixesLegacyStorageIDs.java
  41. 15 16
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDeadNodeDetection.java
  42. 78 62
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommissionWithStriped.java
  43. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDeprecatedKeys.java

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

@@ -17,9 +17,9 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.IOException;
 import java.io.OutputStream;
@@ -97,9 +97,9 @@ public class AppendTestUtil {
     }
     
     LOG.info("partition=" + Arrays.toString(p));
-    assertTrue("i=0", p[0] > 0 && p[0] < n);
+    assertTrue(p[0] > 0 && p[0] < n, "i=0");
     for(int i = 1; i < p.length; i++) {
-      assertTrue("i=" + i, p[i] > p[i - 1] && p[i] < n);
+      assertTrue(p[i] > p[i - 1] && p[i] < n, "i=" + i);
     }
     return p;
   }
@@ -217,8 +217,7 @@ public class AppendTestUtil {
       boolean checkFileStatus) throws IOException {
     if (checkFileStatus) {
       final FileStatus status = fs.getFileStatus(name);
-      assertEquals("len=" + len + " but status.getLen()=" + status.getLen(),
-          len, status.getLen());
+      assertEquals(len, status.getLen(), "len=" + len + " but status.getLen()=" + status.getLen());
     }
 
     FSDataInputStream stm = fs.open(name);
@@ -231,9 +230,9 @@ public class AppendTestUtil {
   private static void checkData(final byte[] actual, int from,
                                 final 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],
-                   expected[from+idx], actual[idx]);
+      assertEquals(expected[from + idx], actual[idx],
+          message + " byte " + (from + idx) + " differs. expected " +
+              expected[from + idx] + " actual " + actual[idx]);
       actual[idx] = 0;
     }
   }

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

@@ -26,9 +26,9 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
-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;
 
 /** This is a comprehensive append test that tries
  * all combinations of file length and number of appended bytes
@@ -59,7 +59,7 @@ public class FileAppendTest4 {
     conf.setInt(HdfsClientConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, PACKET_SIZE);
   }
   
-  @BeforeClass
+  @BeforeAll
   public static void startUp () throws IOException {
     conf = new HdfsConfiguration();
     init(conf);
@@ -67,7 +67,7 @@ public class FileAppendTest4 {
     fs = cluster.getFileSystem();
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() {
     if (cluster != null) {
       cluster.shutdown();

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

@@ -30,7 +30,6 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.event.Level;
@@ -40,6 +39,9 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
 
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
 /**
  * Utility class for testing online recovery of striped files.
  */
@@ -216,11 +218,11 @@ abstract public class ReadStripedFileWithDecodingHelper {
         + ", parityBlkDelNum = " + parityBlkDelNum
         + ", deleteBlockFile? " + deleteBlockFile);
     int recoverBlkNum = dataBlkDelNum + parityBlkDelNum;
-    Assert.assertTrue("dataBlkDelNum and parityBlkDelNum should be positive",
-        dataBlkDelNum >= 0 && parityBlkDelNum >= 0);
-    Assert.assertTrue("The sum of dataBlkDelNum and parityBlkDelNum " +
-        "should be between 1 ~ " + NUM_PARITY_UNITS, recoverBlkNum <=
-        NUM_PARITY_UNITS);
+    assertTrue(dataBlkDelNum >= 0 && parityBlkDelNum >= 0,
+        "dataBlkDelNum and parityBlkDelNum should be positive");
+    assertTrue(recoverBlkNum <=
+        NUM_PARITY_UNITS, "The sum of dataBlkDelNum and parityBlkDelNum " +
+        "should be between 1 ~ " + NUM_PARITY_UNITS);
 
     // write a file with the length of writeLen
     Path srcPath = new Path(src);
@@ -248,10 +250,10 @@ abstract public class ReadStripedFileWithDecodingHelper {
 
     int[] delDataBlkIndices = StripedFileTestUtil.randomArray(0, NUM_DATA_UNITS,
         dataBlkDelNum);
-    Assert.assertNotNull(delDataBlkIndices);
+    assertNotNull(delDataBlkIndices);
     int[] delParityBlkIndices = StripedFileTestUtil.randomArray(NUM_DATA_UNITS,
         NUM_DATA_UNITS + NUM_PARITY_UNITS, parityBlkDelNum);
-    Assert.assertNotNull(delParityBlkIndices);
+    assertNotNull(delParityBlkIndices);
 
     int[] delBlkIndices = new int[recoverBlkNum];
     System.arraycopy(delDataBlkIndices, 0,

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

@@ -37,7 +37,6 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.erasurecode.CodecUtil;
 import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
-import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -55,7 +54,11 @@ import java.util.Set;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 public class StripedFileTestUtil {
   public static final Logger LOG =
@@ -77,7 +80,7 @@ public class StripedFileTestUtil {
   static void verifyLength(FileSystem fs, Path srcPath, int fileLength)
       throws IOException {
     FileStatus status = fs.getFileStatus(srcPath);
-    assertEquals("File length should be the same", fileLength, status.getLen());
+    assertEquals(fileLength, status.getLen(), "File length should be the same");
   }
 
   static void verifyPread(DistributedFileSystem fs, Path srcPath,
@@ -109,9 +112,8 @@ public class StripedFileTestUtil {
           offset += target;
         }
         for (int i = 0; i < fileLength - startOffset; i++) {
-          assertEquals("Byte at " + (startOffset + i) + " is different, "
-              + "the startOffset is " + startOffset, expected[startOffset + i],
-              result[i]);
+          assertEquals(expected[startOffset + i], result[i], "Byte at " + (startOffset + i) +
+              " is different, " + "the startOffset is " + startOffset);
         }
       }
     }
@@ -127,8 +129,8 @@ public class StripedFileTestUtil {
         System.arraycopy(buf, 0, result, readLen, ret);
         readLen += ret;
       }
-      assertEquals("The length of file should be the same to write size", fileLength, readLen);
-      Assert.assertArrayEquals(expected, result);
+      assertEquals(fileLength, readLen, "The length of file should be the same to write size");
+      assertArrayEquals(expected, result);
     }
   }
 
@@ -144,8 +146,8 @@ public class StripedFileTestUtil {
         result.put(buf);
         buf.clear();
       }
-      assertEquals("The length of file should be the same to write size", fileLength, readLen);
-      Assert.assertArrayEquals(expected, result.array());
+      assertEquals(fileLength, readLen, "The length of file should be the same to write size");
+      assertArrayEquals(expected, result.array());
     }
   }
 
@@ -185,14 +187,14 @@ public class StripedFileTestUtil {
       if (!(in.getWrappedStream() instanceof WebHdfsInputStream)) {
         try {
           in.seek(-1);
-          Assert.fail("Should be failed if seek to negative offset");
+          fail("Should be failed if seek to negative offset");
         } catch (EOFException e) {
           // expected
         }
 
         try {
           in.seek(fileLength + 1);
-          Assert.fail("Should be failed if seek after EOF");
+          fail("Should be failed if seek after EOF");
         } catch (EOFException e) {
           // expected
         }
@@ -206,8 +208,8 @@ public class StripedFileTestUtil {
     byte[] buf = new byte[writeBytes - pos];
     IOUtils.readFully(fsdis, buf, 0, buf.length);
     for (int i = 0; i < buf.length; i++) {
-      assertEquals("Byte at " + i + " should be the same",
-          StripedFileTestUtil.getByte(pos + i), buf[i]);
+      assertEquals(StripedFileTestUtil.getByte(pos + i),
+          buf[i], "Byte at " + i + " should be the same");
     }
   }
 
@@ -225,7 +227,7 @@ public class StripedFileTestUtil {
       final DatanodeInfo[] datanodes = streamer.getNodes();
       if (datanodes != null) {
         assertEquals(1, datanodes.length);
-        Assert.assertNotNull(datanodes[0]);
+        assertNotNull(datanodes[0]);
         return datanodes[0];
       }
       try {
@@ -377,13 +379,13 @@ public class StripedFileTestUtil {
     final int parityBlkNum = ecPolicy.getNumParityUnits();
     int index = 0;
     for (LocatedBlock firstBlock : lbs.getLocatedBlocks()) {
-      Assert.assertTrue(firstBlock instanceof LocatedStripedBlock);
+      assertTrue(firstBlock instanceof LocatedStripedBlock);
 
       final long gs = firstBlock.getBlock().getGenerationStamp();
       final long oldGS = oldGSList != null ? oldGSList.get(index++) : -1L;
       final String s = "gs=" + gs + ", oldGS=" + oldGS;
       LOG.info(s);
-      Assert.assertTrue(s, gs >= oldGS);
+      assertTrue(gs >= oldGS, s);
 
       LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(
           (LocatedStripedBlock) firstBlock, cellSize,
@@ -456,7 +458,7 @@ public class StripedFileTestUtil {
         for (int posInBlk = 0; posInBlk < actual.length; posInBlk++) {
           final long posInFile = StripedBlockUtil.offsetInBlkToOffsetInBG(
               cellSize, dataBlkNum, posInBlk, i) + groupPosInFile;
-          Assert.assertTrue(posInFile < length);
+          assertTrue(posInFile < length);
           final byte expected = getByte(posInFile);
 
           if (killed) {
@@ -466,7 +468,7 @@ public class StripedFileTestUtil {
               String s = "expected=" + expected + " but actual=" + actual[posInBlk]
                   + ", posInFile=" + posInFile + ", posInBlk=" + posInBlk
                   + ". group=" + group + ", i=" + i;
-              Assert.fail(s);
+              fail(s);
             }
           }
         }
@@ -507,12 +509,12 @@ public class StripedFileTestUtil {
     try {
       encoder.encode(dataBytes, expectedParityBytes);
     } catch (IOException e) {
-      Assert.fail("Unexpected IOException: " + e.getMessage());
+      fail("Unexpected IOException: " + e.getMessage());
     }
     for (int i = 0; i < parityBytes.length; i++) {
       if (checkSet.contains(i + dataBytes.length)){
-        Assert.assertArrayEquals("i=" + i, expectedParityBytes[i],
-            parityBytes[i]);
+        assertArrayEquals(expectedParityBytes[i],
+            parityBytes[i], "i=" + i);
       }
     }
   }

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

@@ -17,7 +17,8 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.IOException;
 
@@ -30,10 +31,9 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 /**
  * Test abandoning blocks, which clients do on pipeline creation failure.
@@ -48,14 +48,14 @@ public class TestAbandonBlock {
   private MiniDFSCluster cluster;
   private DistributedFileSystem fs;
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(2).build();
     fs = cluster.getFileSystem();
     cluster.waitActive();
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     if (fs != null) {
       fs.close();
@@ -100,8 +100,8 @@ public class TestAbandonBlock {
     cluster.restartNameNode();
     blocks = dfsclient.getNamenode().getBlockLocations(src, 0,
         Integer.MAX_VALUE);
-    Assert.assertEquals("Blocks " + b + " has not been abandoned.",
-        orginalNumBlocks, blocks.locatedBlockCount() + 1);
+    assertEquals(orginalNumBlocks, blocks.locatedBlockCount() + 1, "Blocks " +
+        b + " has not been abandoned.");
   }
 
   @Test

+ 159 - 150
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAclsEndToEnd.java

@@ -43,12 +43,11 @@ import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.ProxyUsers;
-import org.junit.Assert;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 
 /**
  * This class tests the ACLs system through the full code path.  It overlaps
@@ -89,7 +88,7 @@ public class TestAclsEndToEnd {
   private MiniDFSCluster cluster;
   private DistributedFileSystem fs;
 
-  @BeforeClass
+  @BeforeAll
   public static void captureUser() throws IOException {
     realUgi = UserGroupInformation.getCurrentUser();
     realUser = System.getProperty("user.name");
@@ -174,7 +173,7 @@ public class TestAclsEndToEnd {
 
       kmsDir = new File(fsHelper.getTestRootDir()).getAbsoluteFile();
 
-      Assert.assertTrue(kmsDir.mkdirs());
+      assertTrue(kmsDir.mkdirs());
     }
 
     writeConf(kmsDir, conf);
@@ -412,14 +411,14 @@ public class TestAclsEndToEnd {
       setup(conf);
 
       // Create a test key
-      assertTrue("Exception during creation of key " + KEY1 + " by "
-          + keyadminUgi.getUserName(), createKey(keyadminUgi, KEY1, conf));
+      assertTrue(createKey(keyadminUgi, KEY1, conf),
+          "Exception during creation of key " + KEY1 + " by " + keyadminUgi.getUserName());
 
       // Fail to create a test key
-      assertFalse("Allowed creation of key " + KEY2 + " by "
-          + hdfsUgi.getUserName(), createKey(hdfsUgi, KEY2, conf));
-      assertFalse("Allowed creation of key " + KEY2 + " by "
-          + userUgi.getUserName(), createKey(userUgi, KEY2, conf));
+      assertFalse(createKey(hdfsUgi, KEY2, conf),
+          "Allowed creation of key " + KEY2 + " by " + hdfsUgi.getUserName());
+      assertFalse(createKey(userUgi, KEY2, conf),
+          "Allowed creation of key " + KEY2 + " by " + userUgi.getUserName());
 
       // Create a directory and chown it to the normal user.
       fs.mkdirs(ZONE1);
@@ -427,50 +426,57 @@ public class TestAclsEndToEnd {
           userUgi.getPrimaryGroupName());
 
       // Create an EZ
-      assertTrue("Exception during creation of EZ " + ZONE1 + " by "
-          + hdfsUgi.getUserName() + " using key " + KEY1,
-            createEncryptionZone(hdfsUgi, KEY1, ZONE1));
+      assertTrue(createEncryptionZone(hdfsUgi, KEY1, ZONE1),
+          "Exception during creation of EZ " + ZONE1 + " by "
+          + hdfsUgi.getUserName() + " using key " + KEY1);
 
       // Fail to create an EZ
-      assertFalse("Allowed creation of EZ " + ZONE2 + " by "
-          + keyadminUgi.getUserName() + " using key " + KEY1,
-            createEncryptionZone(keyadminUgi, KEY1, ZONE2));
-      assertFalse("Allowed creation of EZ " + ZONE2 + " by "
-          + userUgi.getUserName() + " using key " + KEY1,
-            createEncryptionZone(userUgi, KEY1, ZONE2));
+      assertFalse(createEncryptionZone(keyadminUgi, KEY1, ZONE2),
+          "Allowed creation of EZ " + ZONE2 + " by "
+          + keyadminUgi.getUserName() + " using key " + KEY1);
+      assertFalse(createEncryptionZone(userUgi, KEY1, ZONE2),
+          "Allowed creation of EZ " + ZONE2 + " by "
+          + userUgi.getUserName() + " using key " + KEY1);
 
       // Create a file in the zone
-      assertTrue("Exception during creation of file " + FILE1 + " by "
-          + userUgi.getUserName(), createFile(userUgi, FILE1, TEXT));
+      assertTrue(createFile(userUgi, FILE1, TEXT),
+          "Exception during creation of file " + FILE1 + " by "
+              + userUgi.getUserName());
 
       // Fail to create a file in the zone
-      assertFalse("Allowed creation of file " + FILE1A + " by "
-          + hdfsUgi.getUserName(), createFile(hdfsUgi, FILE1A, TEXT));
-      assertFalse("Allowed creation of file " + FILE1A + " by "
-          + keyadminUgi.getUserName(), createFile(keyadminUgi, FILE1A, TEXT));
+      assertFalse(createFile(hdfsUgi, FILE1A, TEXT),
+          "Allowed creation of file " + FILE1A + " by "
+              + hdfsUgi.getUserName());
+      assertFalse(createFile(keyadminUgi, FILE1A, TEXT),
+          "Allowed creation of file " + FILE1A + " by "
+              + keyadminUgi.getUserName());
 
       // Read a file in the zone
-      assertTrue("Exception while reading file " + FILE1 + " by "
-          + userUgi.getUserName(), compareFile(userUgi, FILE1, TEXT));
+      assertTrue(compareFile(userUgi, FILE1, TEXT),
+          "Exception while reading file " + FILE1 + " by "
+              + userUgi.getUserName());
 
       // Fail to read a file in the zone
-      assertFalse("Allowed reading of file " + FILE1 + " by "
-          + hdfsUgi.getUserName(), compareFile(hdfsUgi, FILE1, TEXT));
-      assertFalse("Allowed reading of file " + FILE1 + " by "
-          + keyadminUgi.getUserName(), compareFile(keyadminUgi, FILE1, TEXT));
+      assertFalse(compareFile(hdfsUgi, FILE1, TEXT),
+          "Allowed reading of file " + FILE1 + " by "
+              + hdfsUgi.getUserName());
+      assertFalse(compareFile(keyadminUgi, FILE1, TEXT),
+          "Allowed reading of file " + FILE1 + " by "
+              + keyadminUgi.getUserName());
 
       // Remove the zone
       fs.delete(ZONE1, true);
 
       // Fail to remove the key
-      assertFalse("Allowed deletion of file " + FILE1 + " by "
-          + hdfsUgi.getUserName(), deleteKey(hdfsUgi, KEY1));
-      assertFalse("Allowed deletion of file " + FILE1 + " by "
-          + userUgi.getUserName(), deleteKey(userUgi, KEY1));
+      assertFalse(deleteKey(hdfsUgi, KEY1), "Allowed deletion of file " + FILE1 + " by "
+          + hdfsUgi.getUserName());
+      assertFalse(deleteKey(userUgi, KEY1), "Allowed deletion of file " + FILE1 + " by "
+          + userUgi.getUserName());
 
       // Remove
-      assertTrue("Exception during deletion of file " + FILE1 + " by "
-          + keyadminUgi.getUserName(), deleteKey(keyadminUgi, KEY1));
+      assertTrue(deleteKey(keyadminUgi, KEY1),
+          "Exception during deletion of file " + FILE1 + " by "
+              + keyadminUgi.getUserName());
     } finally {
       fs.delete(ZONE1, true);
       fs.delete(ZONE2, true);
@@ -495,8 +501,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf);
 
-      assertTrue("Exception during key creation with correct config"
-          + " using whitelist key ACLs", createKey(realUgi, KEY1, conf));
+      assertTrue(createKey(realUgi, KEY1, conf),
+          "Exception during key creation with correct config using whitelist key ACLs");
     } finally {
       teardown();
     }
@@ -512,8 +518,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf);
 
-      assertTrue("Exception during key creation with correct config"
-          + " using default key ACLs", createKey(realUgi, KEY2, conf));
+      assertTrue(createKey(realUgi, KEY2, conf),
+          "Exception during key creation with correct config using default key ACLs");
     } finally {
       teardown();
     }
@@ -531,8 +537,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf);
 
-      assertFalse("Allowed key creation with blacklist for CREATE",
-          createKey(realUgi, KEY3, conf));
+      assertFalse(createKey(realUgi, KEY3, conf),
+          "Allowed key creation with blacklist for CREATE");
     } finally {
       teardown();
     }
@@ -547,8 +553,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf);
 
-      assertFalse("Allowed key creation without CREATE KMS ACL",
-          createKey(realUgi, KEY3, conf));
+      assertFalse(createKey(realUgi, KEY3, conf),
+          "Allowed key creation without CREATE KMS ACL");
     } finally {
       teardown();
     }
@@ -562,8 +568,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf);
 
-      assertFalse("Allowed key creation without MANAGMENT key ACL",
-          createKey(realUgi, KEY3, conf));
+      assertFalse(createKey(realUgi, KEY3, conf),
+          "Allowed key creation without MANAGMENT key ACL");
     } finally {
       teardown();
     }
@@ -581,8 +587,9 @@ public class TestAclsEndToEnd {
     try {
       setup(conf);
 
-      assertFalse("Allowed key creation when default key ACL should have been"
-          + " overridden by key ACL", createKey(realUgi, KEY3, conf));
+      assertFalse(createKey(realUgi, KEY3, conf),
+          "Allowed key creation when default key ACL should have been"
+              + " overridden by key ACL");
     } finally {
       teardown();
     }
@@ -596,8 +603,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf);
 
-      assertTrue("Exception during key creation with default KMS ACLs",
-          createKey(realUgi, KEY3, conf));
+      assertTrue(createKey(realUgi, KEY3, conf),
+          "Exception during key creation with default KMS ACLs");
     } finally {
       teardown();
     }
@@ -620,8 +627,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf);
 
-      assertTrue("Exception during key creation",
-          createKey(realUgi, KEY1, conf));
+      assertTrue(createKey(realUgi, KEY1, conf),
+          "Exception during key creation");
     } finally {
       teardown();
     }
@@ -647,8 +654,9 @@ public class TestAclsEndToEnd {
 
       fs.mkdirs(ZONE1);
 
-      assertTrue("Exception during zone creation with correct config using"
-          + " whitelist key ACLs", createEncryptionZone(realUgi, KEY1, ZONE1));
+      assertTrue(createEncryptionZone(realUgi, KEY1, ZONE1),
+          "Exception during zone creation with correct config using"
+              + " whitelist key ACLs");
     } finally {
       fs.delete(ZONE1, true);
       teardown();
@@ -671,8 +679,9 @@ public class TestAclsEndToEnd {
 
       fs.mkdirs(ZONE2);
 
-      assertTrue("Exception during zone creation with correct config using"
-          + " default key ACLs", createEncryptionZone(realUgi, KEY1, ZONE2));
+      assertTrue(createEncryptionZone(realUgi, KEY1, ZONE2),
+          "Exception during zone creation with correct config using"
+              + " default key ACLs");
     } finally {
       fs.delete(ZONE2, true);
       teardown();
@@ -697,9 +706,9 @@ public class TestAclsEndToEnd {
 
       fs.mkdirs(ZONE3);
 
-      assertFalse("Allowed creation of zone when default key ACLs should have"
-          + " been overridden by key ACL",
-            createEncryptionZone(realUgi, KEY1, ZONE3));
+      assertFalse(createEncryptionZone(realUgi, KEY1, ZONE3),
+          "Allowed creation of zone when default key ACLs should have"
+              + " been overridden by key ACL");
     } finally {
       fs.delete(ZONE3, true);
       teardown();
@@ -724,8 +733,8 @@ public class TestAclsEndToEnd {
 
       fs.mkdirs(ZONE3);
 
-      assertFalse("Allowed zone creation of zone with blacklisted GET_METADATA",
-          createEncryptionZone(realUgi, KEY1, ZONE3));
+      assertFalse(createEncryptionZone(realUgi, KEY1, ZONE3),
+          "Allowed zone creation of zone with blacklisted GET_METADATA");
     } finally {
       fs.delete(ZONE3, true);
       teardown();
@@ -750,8 +759,8 @@ public class TestAclsEndToEnd {
 
       fs.mkdirs(ZONE3);
 
-      assertFalse("Allowed zone creation of zone with blacklisted GENERATE_EEK",
-          createEncryptionZone(realUgi, KEY1, ZONE3));
+      assertFalse(createEncryptionZone(realUgi, KEY1, ZONE3),
+          "Allowed zone creation of zone with blacklisted GENERATE_EEK");
     } finally {
       fs.delete(ZONE3, true);
       teardown();
@@ -771,8 +780,8 @@ public class TestAclsEndToEnd {
 
       fs.mkdirs(ZONE3);
 
-      assertTrue("Exception during zone creation with default KMS ACLs",
-          createEncryptionZone(realUgi, KEY1, ZONE3));
+      assertTrue(createEncryptionZone(realUgi, KEY1, ZONE3),
+          "Exception during zone creation with default KMS ACLs");
     } finally {
       fs.delete(ZONE3, true);
       teardown();
@@ -794,8 +803,8 @@ public class TestAclsEndToEnd {
 
       fs.mkdirs(ZONE4);
 
-      assertFalse("Allowed zone creation without GET_METADATA KMS ACL",
-          createEncryptionZone(realUgi, KEY1, ZONE4));
+      assertFalse(createEncryptionZone(realUgi, KEY1, ZONE4),
+          "Allowed zone creation without GET_METADATA KMS ACL");
     } finally {
       fs.delete(ZONE4, true);
       teardown();
@@ -817,8 +826,8 @@ public class TestAclsEndToEnd {
 
       fs.mkdirs(ZONE4);
 
-      assertFalse("Allowed zone creation without GENERATE_EEK KMS ACL",
-          createEncryptionZone(realUgi, KEY1, ZONE4));
+      assertFalse(createEncryptionZone(realUgi, KEY1, ZONE4),
+          "Allowed zone creation without GENERATE_EEK KMS ACL");
     } finally {
       fs.delete(ZONE4, true);
       teardown();
@@ -839,8 +848,8 @@ public class TestAclsEndToEnd {
 
       fs.mkdirs(ZONE4);
 
-      assertFalse("Allowed zone creation without READ ACL",
-          createEncryptionZone(realUgi, KEY1, ZONE4));
+      assertFalse(createEncryptionZone(realUgi, KEY1, ZONE4),
+          "Allowed zone creation without READ ACL");
     } finally {
       fs.delete(ZONE4, true);
       teardown();
@@ -861,8 +870,8 @@ public class TestAclsEndToEnd {
 
       fs.mkdirs(ZONE4);
 
-      assertFalse("Allowed zone creation without GENERATE_EEK ACL",
-          createEncryptionZone(realUgi, KEY1, ZONE4));
+      assertFalse(createEncryptionZone(realUgi, KEY1, ZONE4),
+          "Allowed zone creation without GENERATE_EEK ACL");
     } finally {
       fs.delete(ZONE4, true);
       teardown();
@@ -896,20 +905,20 @@ public class TestAclsEndToEnd {
     try {
       setup(conf);
 
-      assertTrue("Exception during key creation",
-          createKey(realUgi, KEY1, conf));
+      assertTrue(createKey(realUgi, KEY1, conf),
+          "Exception during key creation");
       fs.mkdirs(ZONE1);
-      assertTrue("Exception during zone creation",
-          createEncryptionZone(realUgi, KEY1, ZONE1));
+      assertTrue(createEncryptionZone(realUgi, KEY1, ZONE1),
+          "Exception during zone creation");
       fs.mkdirs(ZONE2);
-      assertTrue("Exception during zone creation",
-          createEncryptionZone(realUgi, KEY1, ZONE2));
+      assertTrue(createEncryptionZone(realUgi, KEY1, ZONE2),
+          "Exception during zone creation");
       fs.mkdirs(ZONE3);
-      assertTrue("Exception during zone creation",
-          createEncryptionZone(realUgi, KEY1, ZONE3));
+      assertTrue(createEncryptionZone(realUgi, KEY1, ZONE3),
+          "Exception during zone creation");
       fs.mkdirs(ZONE4);
-      assertTrue("Exception during zone creation",
-          createEncryptionZone(realUgi, KEY1, ZONE4));
+      assertTrue(createEncryptionZone(realUgi, KEY1, ZONE4),
+          "Exception during zone creation");
     } catch (Throwable ex) {
       fs.delete(ZONE1, true);
       fs.delete(ZONE2, true);
@@ -941,8 +950,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf, false, false);
 
-      assertTrue("Exception during file creation with correct config"
-          + " using whitelist ACL", createFile(realUgi, FILE1, TEXT));
+      assertTrue(createFile(realUgi, FILE1, TEXT),
+          "Exception during file creation with correct config" + " using whitelist ACL");
     } finally {
       fs.delete(ZONE1, true);
       teardown();
@@ -963,8 +972,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf, false, false);
 
-      assertTrue("Exception during file creation with correct config"
-          + " using whitelist ACL", createFile(realUgi, FILE2, TEXT));
+      assertTrue(createFile(realUgi, FILE2, TEXT),
+          "Exception during file creation with correct config using whitelist ACL");
     } finally {
       fs.delete(ZONE2, true);
       teardown();
@@ -987,8 +996,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf, false, false);
 
-      assertFalse("Allowed file creation when default key ACLs should have been"
-          + " overridden by key ACL", createFile(realUgi, FILE3, TEXT));
+      assertFalse(createFile(realUgi, FILE3, TEXT),
+          "Allowed file creation when default key ACLs should have been overridden by key ACL");
     } catch (Exception ex) {
       fs.delete(ZONE3, true);
 
@@ -1014,8 +1023,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf, false, false);
 
-      assertFalse("Allowed file creation with blacklist for GENERATE_EEK",
-          createFile(realUgi, FILE3, TEXT));
+      assertFalse(createFile(realUgi, FILE3, TEXT),
+          "Allowed file creation with blacklist for GENERATE_EEK");
     } catch (Exception ex) {
       fs.delete(ZONE3, true);
 
@@ -1041,8 +1050,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf, false, false);
 
-      assertFalse("Allowed file creation with blacklist for DECRYPT_EEK",
-          createFile(realUgi, FILE3, TEXT));
+      assertFalse(createFile(realUgi, FILE3, TEXT),
+          "Allowed file creation with blacklist for DECRYPT_EEK");
     } catch (Exception ex) {
       fs.delete(ZONE3, true);
 
@@ -1062,8 +1071,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf, false, false);
 
-      assertTrue("Exception during file creation with default KMS ACLs",
-          createFile(realUgi, FILE3, TEXT));
+      assertTrue(createFile(realUgi, FILE3, TEXT),
+          "Exception during file creation with default KMS ACLs");
     } catch (Exception ex) {
       fs.delete(ZONE3, true);
 
@@ -1086,8 +1095,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf, false, false);
 
-      assertFalse("Allowed file creation without GENERATE_EEK KMS ACL",
-          createFile(realUgi, FILE4, TEXT));
+      assertFalse(createFile(realUgi, FILE4, TEXT),
+          "Allowed file creation without GENERATE_EEK KMS ACL");
     } catch (Exception ex) {
       fs.delete(ZONE3, true);
 
@@ -1110,8 +1119,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf, false, false);
 
-      assertFalse("Allowed file creation without DECRYPT_EEK KMS ACL",
-          createFile(realUgi, FILE3, TEXT));
+      assertFalse(createFile(realUgi, FILE3, TEXT),
+          "Allowed file creation without DECRYPT_EEK KMS ACL");
     } catch (Exception ex) {
       fs.delete(ZONE3, true);
 
@@ -1133,8 +1142,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf, false, false);
 
-      assertFalse("Allowed file creation without GENERATE_EEK key ACL",
-          createFile(realUgi, FILE3, TEXT));
+      assertFalse(createFile(realUgi, FILE3, TEXT),
+          "Allowed file creation without GENERATE_EEK key ACL");
     } catch (Exception ex) {
       fs.delete(ZONE3, true);
 
@@ -1156,8 +1165,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf, false, false);
 
-      assertFalse("Allowed file creation without DECRYPT_EEK key ACL",
-          createFile(realUgi, FILE3, TEXT));
+      assertFalse(createFile(realUgi, FILE3, TEXT),
+          "Allowed file creation without DECRYPT_EEK key ACL");
     } catch (Exception ex) {
       fs.delete(ZONE3, true);
 
@@ -1198,13 +1207,13 @@ public class TestAclsEndToEnd {
     try {
       setup(conf);
 
-      assertTrue("Exception during key creation",
-          createKey(realUgi, KEY1, conf));
+      assertTrue(createKey(realUgi, KEY1, conf),
+          "Exception during key creation");
       fs.mkdirs(ZONE1);
-      assertTrue("Exception during zone creation",
-          createEncryptionZone(realUgi, KEY1, ZONE1));
-      assertTrue("Exception during file creation",
-              createFile(realUgi, FILE1, TEXT));
+      assertTrue(createEncryptionZone(realUgi, KEY1, ZONE1),
+          "Exception during zone creation");
+      assertTrue(createFile(realUgi, FILE1, TEXT),
+          "Exception during file creation");
     } catch (Throwable ex) {
       fs.delete(ZONE1, true);
 
@@ -1229,8 +1238,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf, false, false);
 
-      assertTrue("Exception while reading file with correct config with"
-          + " whitelist ACLs", compareFile(realUgi, FILE1, TEXT));
+      assertTrue(compareFile(realUgi, FILE1, TEXT),
+          "Exception while reading file with correct config with whitelist ACLs");
     } catch (Throwable ex) {
       fs.delete(ZONE1, true);
 
@@ -1250,8 +1259,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf, false, false);
 
-      assertTrue("Exception while reading file with correct config"
-          + " with default ACLs", compareFile(realUgi, FILE1, TEXT));
+      assertTrue(compareFile(realUgi, FILE1, TEXT),
+          "Exception while reading file with correct config with default ACLs");
     } catch (Throwable ex) {
       fs.delete(ZONE1, true);
 
@@ -1273,8 +1282,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf, false, false);
 
-      assertFalse("Allowed file read when default key ACLs should have been"
-          + " overridden by key ACL", compareFile(realUgi, FILE1, TEXT));
+      assertFalse(compareFile(realUgi, FILE1, TEXT),
+          "Allowed file read when default key ACLs should have been overridden by key ACL");
     } catch (Throwable ex) {
       fs.delete(ZONE1, true);
 
@@ -1296,8 +1305,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf, false, false);
 
-      assertFalse("Allowed file read with blacklist for DECRYPT_EEK",
-          compareFile(realUgi, FILE1, TEXT));
+      assertFalse(compareFile(realUgi, FILE1, TEXT),
+          "Allowed file read with blacklist for DECRYPT_EEK");
     } catch (Throwable ex) {
       fs.delete(ZONE1, true);
 
@@ -1315,8 +1324,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf, false, false);
 
-      assertTrue("Exception while reading file with default KMS ACLs",
-          compareFile(realUgi, FILE1, TEXT));
+      assertTrue(compareFile(realUgi, FILE1, TEXT),
+          "Exception while reading file with default KMS ACLs");
     } catch (Throwable ex) {
       fs.delete(ZONE1, true);
 
@@ -1335,8 +1344,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf, false, false);
 
-      assertFalse("Allowed file read without DECRYPT_EEK KMS ACL",
-          compareFile(realUgi, FILE1, TEXT));
+      assertFalse(compareFile(realUgi, FILE1, TEXT),
+          "Allowed file read without DECRYPT_EEK KMS ACL");
     } catch (Throwable ex) {
       fs.delete(ZONE1, true);
 
@@ -1351,8 +1360,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf, false, false);
 
-      assertFalse("Allowed file read without DECRYPT_EEK key ACL",
-          compareFile(realUgi, FILE1, TEXT));
+      assertFalse(compareFile(realUgi, FILE1, TEXT),
+          "Allowed file read without DECRYPT_EEK key ACL");
     } catch (Throwable ex) {
       fs.delete(ZONE1, true);
 
@@ -1379,12 +1388,12 @@ public class TestAclsEndToEnd {
     try {
       setup(conf);
 
-      assertTrue("Exception during key creation",
-          createKey(realUgi, KEY1, conf));
-      assertTrue("Exception during key creation",
-          createKey(realUgi, KEY2, conf));
-      assertTrue("Exception during key creation",
-          createKey(realUgi, KEY3, conf));
+      assertTrue(createKey(realUgi, KEY1, conf),
+          "Exception during key creation");
+      assertTrue(createKey(realUgi, KEY2, conf),
+          "Exception during key creation");
+      assertTrue(createKey(realUgi, KEY3, conf),
+          "Exception during key creation");
     } finally {
       teardown();
     }
@@ -1405,8 +1414,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf, false);
 
-      assertTrue("Exception during key deletion with correct config"
-          + " using whitelist key ACLs", deleteKey(realUgi, KEY1));
+      assertTrue(deleteKey(realUgi, KEY1), "Exception during key deletion with correct config"
+          + " using whitelist key ACLs");
     } finally {
       teardown();
     }
@@ -1422,8 +1431,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf, false);
 
-      assertTrue("Exception during key deletion with correct config"
-          + " using default key ACLs", deleteKey(realUgi, KEY2));
+      assertTrue(deleteKey(realUgi, KEY2), "Exception during key deletion with correct config"
+          + " using default key ACLs");
     } finally {
       teardown();
     }
@@ -1441,8 +1450,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf, false);
 
-      assertFalse("Allowed key deletion with blacklist for DELETE",
-          deleteKey(realUgi, KEY3));
+      assertFalse(deleteKey(realUgi, KEY3),
+          "Allowed key deletion with blacklist for DELETE");
     } finally {
       teardown();
     }
@@ -1457,8 +1466,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf, false);
 
-      assertFalse("Allowed key deletion without DELETE KMS ACL",
-          deleteKey(realUgi, KEY3));
+      assertFalse(deleteKey(realUgi, KEY3),
+          "Allowed key deletion without DELETE KMS ACL");
     } finally {
       teardown();
     }
@@ -1473,8 +1482,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf, false);
 
-      assertFalse("Allowed key deletion without MANAGMENT key ACL",
-          deleteKey(realUgi, KEY3));
+      assertFalse(deleteKey(realUgi, KEY3),
+          "Allowed key deletion without MANAGMENT key ACL");
     } finally {
       teardown();
     }
@@ -1492,8 +1501,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf, false);
 
-      assertFalse("Allowed key deletion when default key ACL should have been"
-          + " overridden by key ACL", deleteKey(realUgi, KEY3));
+      assertFalse(deleteKey(realUgi, KEY3),
+          "Allowed key deletion when default key ACL should have been overridden by key ACL");
     } finally {
       teardown();
     }
@@ -1507,8 +1516,8 @@ public class TestAclsEndToEnd {
     try {
       setup(conf, false);
 
-      assertTrue("Exception during key deletion with default KMS ACLs",
-          deleteKey(realUgi, KEY3));
+      assertTrue(deleteKey(realUgi, KEY3),
+          "Exception during key deletion with default KMS ACLs");
     } finally {
       teardown();
     }
@@ -1596,8 +1605,8 @@ public class TestAclsEndToEnd {
         FSDataInputStream din =  cluster.getFileSystem().open(file);
         BufferedReader in = new BufferedReader(new InputStreamReader(din));
 
-        assertEquals("The text read does not match the text written",
-            text, in.readLine());
+        assertEquals(text, in.readLine(),
+            "The text read does not match the text written");
       }
     });
   }

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

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs;
 
 import java.io.IOException;
 import java.util.Random;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -26,10 +27,11 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.Time;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 /**
  * Test cases for trying to append to a file with a different
@@ -44,7 +46,7 @@ public class TestAppendDifferentChecksum {
   private static FileSystem fs; 
   
 
-  @BeforeClass
+  @BeforeAll
   public static void setupCluster() throws IOException {
     Configuration conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 4096);
@@ -55,7 +57,7 @@ public class TestAppendDifferentChecksum {
     fs = cluster.getFileSystem();
   }
   
-  @AfterClass
+  @AfterAll
   public static void teardown() throws IOException {
     if (cluster != null) {
       cluster.shutdown();
@@ -68,7 +70,7 @@ public class TestAppendDifferentChecksum {
    * difficulties in doing so.
    */
   @Test
-  @Ignore("this is not implemented! See HDFS-2130")
+  @Disabled("this is not implemented! See HDFS-2130")
   public void testSwitchChunkSize() throws IOException {
     FileSystem fsWithSmallChunk = createFsWithChecksum("CRC32", 512);
     FileSystem fsWithBigChunk = createFsWithChecksum("CRC32", 1024);
@@ -100,7 +102,8 @@ public class TestAppendDifferentChecksum {
    * CRC32 and with CRC32C, crossing several block boundaries.
    * Then, checks that all of the data can be read back correct.
    */
-  @Test(timeout=RANDOM_TEST_RUNTIME*2)
+  @Test
+  @Timeout(value = RANDOM_TEST_RUNTIME * 2, unit = TimeUnit.MILLISECONDS)
   public void testAlgoSwitchRandomized() throws IOException {
     FileSystem fsWithCrc32 = createFsWithChecksum("CRC32", 512);
     FileSystem fsWithCrc32C = createFsWithChecksum("CRC32C", 512);

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

@@ -34,6 +34,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.commons.io.FileUtils;
+import org.junit.jupiter.api.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -45,14 +46,17 @@ import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite.ReplaceData
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.TestFileTruncate;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.util.Preconditions;
 import org.slf4j.event.Level;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
 /**
  * Test randomly mixing append, snapshot and truncate operations.
  * Use local file system to simulate the each operation and verify
@@ -78,7 +82,7 @@ public class TestAppendSnapshotTruncate {
   static MiniDFSCluster cluster;
   static DistributedFileSystem dfs;
 
-  @BeforeClass
+  @BeforeAll
   public static void startUp() throws IOException {
     conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_KEY, BLOCK_SIZE);
@@ -95,7 +99,7 @@ public class TestAppendSnapshotTruncate {
     dfs = cluster.getFileSystem();
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() throws IOException {
     if(dfs != null) {
       dfs.close();
@@ -107,7 +111,8 @@ public class TestAppendSnapshotTruncate {
 
 
   /** Test randomly mixing append, snapshot and truncate operations. */
-  @Test(timeout=TEST_TIMEOUT_SECOND*1000)
+  @Test
+  @Timeout(TEST_TIMEOUT_SECOND)
   public void testAST() throws Exception {
     final String dirPathString = "/dir";
     final Path dir = new Path(dirPathString);
@@ -167,7 +172,7 @@ public class TestAppendSnapshotTruncate {
       {
         //copy all local files to a sub dir to simulate snapshot. 
         final File subDir = new File(localDir, snapshot);
-        Assert.assertFalse(subDir.exists());
+        assertFalse(subDir.exists());
         subDir.mkdir();
 
         for(File f : localDir.listFiles(FILE_ONLY)) {
@@ -185,12 +190,12 @@ public class TestAppendSnapshotTruncate {
           .append(snapshot);
 
       final File subDir = new File(localDir, snapshot);
-      Assert.assertTrue(subDir.exists());
+      assertTrue(subDir.exists());
       
       final File[] localFiles = subDir.listFiles(FILE_ONLY);
       final Path p = snapshotPaths.get(snapshot);
       final FileStatus[] statuses = dfs.listStatus(p);
-      Assert.assertEquals(localFiles.length, statuses.length);
+      assertEquals(localFiles.length, statuses.length);
       b.append(p).append(" vs ").append(subDir).append(", ")
        .append(statuses.length).append(" entries");
       
@@ -374,8 +379,8 @@ public class TestAppendSnapshotTruncate {
 
     static int checkLength(Path file, File localFile) throws IOException {
       final long length = dfs.getFileStatus(file).getLen();
-      Assert.assertEquals(localFile.length(), length);
-      Assert.assertTrue(length <= Integer.MAX_VALUE);
+      assertEquals(localFile.length(), length);
+      assertTrue(length <= Integer.MAX_VALUE);
       return (int)length;
     }
     

+ 7 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestApplyingStoragePolicy.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.FileNotFoundException;
 import java.io.IOException;
@@ -27,9 +27,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
-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;
 
 public class TestApplyingStoragePolicy {
   private static final short REPL = 1;
@@ -39,7 +39,7 @@ public class TestApplyingStoragePolicy {
   private static MiniDFSCluster cluster;
   private static DistributedFileSystem fs;
 
-  @Before
+  @BeforeEach
   public void clusterSetUp() throws IOException {
     conf = new HdfsConfiguration();
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPL).build();
@@ -47,7 +47,7 @@ public class TestApplyingStoragePolicy {
     fs = cluster.getFileSystem();
   }
 
-  @After
+  @AfterEach
   public void clusterShutdown() throws IOException{
     if(fs != null) {
       fs.close();

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBalancerBandwidth.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.PrintStream;
@@ -35,7 +35,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.ToolRunner;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 /**
  * This test ensures that the balancer bandwidth is dynamically adjusted
@@ -132,11 +132,11 @@ public class TestBalancerBandwidth {
     try {
       System.setOut(outStream);
       int exitCode = admin.run(args);
-      assertEquals("DFSAdmin should return 0", 0, exitCode);
+      assertEquals(0, exitCode, "DFSAdmin should return 0");
       String bandwidthOutMsg = "Balancer bandwidth is " + expectedBandwidth
           + " bytes per second.";
       String strOut = new String(outContent.toByteArray(), UTF8);
-      assertTrue("Wrong balancer bandwidth!", strOut.contains(bandwidthOutMsg));
+      assertTrue(strOut.contains(bandwidthOutMsg), "Wrong balancer bandwidth!");
     } finally {
       System.setOut(initialStdOut);
     }

+ 43 - 44
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBatchedListDirectories.java

@@ -32,12 +32,9 @@ import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Lists;
-import org.hamcrest.core.StringContains;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -46,9 +43,10 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 /**
  * Tests for the batched listing API.
@@ -59,9 +57,6 @@ public class TestBatchedListDirectories {
   private static Configuration conf;
   private static DistributedFileSystem dfs;
 
-  @Rule
-  public ExpectedException thrown = ExpectedException.none();
-
   private static final List<Path> SUBDIR_PATHS = Lists.newArrayList();
   private static final List<Path> FILE_PATHS = Lists.newArrayList();
   private static final int FIRST_LEVEL_DIRS = 2;
@@ -85,15 +80,15 @@ public class TestBatchedListDirectories {
   private static void assertSubDirEquals(int i, int j, Path p) {
     assertTrue(p.toString().startsWith("hdfs://"));
     Path expected = getSubDirName(i, j);
-    assertEquals("Unexpected subdir name",
-        expected.toString(), p.toUri().getPath());
+    assertEquals(expected.toString(), p.toUri().getPath(),
+        "Unexpected subdir name");
   }
 
   private static void assertFileEquals(int i, int j, int k, Path p) {
     assertTrue(p.toString().startsWith("hdfs://"));
     Path expected = getFileName(i, j, k);
-    assertEquals("Unexpected file name",
-        expected.toString(), p.toUri().getPath());
+    assertEquals(expected.toString(), p.toUri().getPath(),
+        "Unexpected file name");
   }
 
   private static void loadData() throws Exception {
@@ -119,7 +114,7 @@ public class TestBatchedListDirectories {
     dfs.setPermission(INACCESSIBLE_DIR_PATH, new FsPermission(0000));
   }
 
-  @BeforeClass
+  @BeforeAll
   public static void beforeClass() throws Exception {
     conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_LIST_LIMIT, 7);
@@ -132,7 +127,7 @@ public class TestBatchedListDirectories {
     loadData();
   }
 
-  @AfterClass
+  @AfterAll
   public static void afterClass() {
     if (cluster != null) {
       cluster.shutdown();
@@ -167,9 +162,10 @@ public class TestBatchedListDirectories {
 
   @Test
   public void testEmptyPath() throws Exception {
-    thrown.expect(FileNotFoundException.class);
-    List<Path> paths = Lists.newArrayList();
-    getStatuses(paths);
+    assertThrows(FileNotFoundException.class, () -> {
+      List<Path> paths = Lists.newArrayList();
+      getStatuses(paths);
+    });
   }
 
   @Test
@@ -192,10 +188,11 @@ public class TestBatchedListDirectories {
 
   @Test
   public void listDoesNotExist() throws Exception {
-    thrown.expect(FileNotFoundException.class);
-    List<Path> paths = Lists.newArrayList();
-    paths.add(new Path("/does/not/exist"));
-    getStatuses(paths);
+    assertThrows(FileNotFoundException.class, () -> {
+      List<Path> paths = Lists.newArrayList();
+      paths.add(new Path("/does/not/exist"));
+      getStatuses(paths);
+    });
   }
 
   @Test
@@ -233,8 +230,8 @@ public class TestBatchedListDirectories {
     dfs.setWorkingDirectory(new Path("/dir0"));
     List<Path> paths = Lists.newArrayList(new Path("."));
     List<FileStatus> statuses = getStatuses(paths);
-    assertEquals("Wrong number of items",
-        SECOND_LEVEL_DIRS, statuses.size());
+    assertEquals(SECOND_LEVEL_DIRS, statuses.size(),
+        "Wrong number of items");
     for (int i = 0; i < SECOND_LEVEL_DIRS; i++) {
       FileStatus stat = statuses.get(i);
       assertSubDirEquals(0, i, stat.getPath());
@@ -246,8 +243,8 @@ public class TestBatchedListDirectories {
     dfs.setWorkingDirectory(new Path("/dir0"));
     List<Path> paths = Lists.newArrayList(new Path("subdir0"));
     List<FileStatus> statuses = getStatuses(paths);
-    assertEquals("Wrong number of items",
-        FILES_PER_DIR, statuses.size());
+    assertEquals(FILES_PER_DIR, statuses.size(),
+        "Wrong number of items");
     for (int i = 0; i < FILES_PER_DIR; i++) {
       FileStatus stat = statuses.get(i);
       assertFileEquals(0, 0, i, stat.getPath());
@@ -256,9 +253,9 @@ public class TestBatchedListDirectories {
 
   @Test
   public void testDFSHasCapability() throws Throwable {
-    assertTrue("FS does not declare PathCapability support",
-        dfs.hasPathCapability(new Path("/"),
-            CommonPathCapabilities.FS_EXPERIMENTAL_BATCH_LISTING));
+    assertTrue(dfs.hasPathCapability(new Path("/"),
+            CommonPathCapabilities.FS_EXPERIMENTAL_BATCH_LISTING),
+        "FS does not declare PathCapability support");
   }
 
   private void listFilesInternal(int numFiles) throws Exception {
@@ -328,12 +325,12 @@ public class TestBatchedListDirectories {
 
   @Test
   public void listTooManyDirectories() throws Exception {
-    thrown.expect(RemoteException.class);
-    thrown.expectMessage(
-        StringContains.containsString("Too many source paths"));
-    List<Path> paths = Lists.newArrayList(FILE_PATHS);
-    paths.add(SUBDIR_PATHS.get(0));
-    getStatuses(paths);
+    RemoteException ex = assertThrows(RemoteException.class, () -> {
+      List<Path> paths = Lists.newArrayList(FILE_PATHS);
+      paths.add(SUBDIR_PATHS.get(0));
+      getStatuses(paths);
+    });
+    assertTrue(ex.getMessage().contains("Too many source paths"));
   }
 
   @Test
@@ -405,15 +402,17 @@ public class TestBatchedListDirectories {
 
   @Test
   public void listInaccessibleDir() throws Exception {
-    thrown.expect(AccessControlException.class);
-    List<Path> paths = Lists.newArrayList(INACCESSIBLE_DIR_PATH);
-    listAsNormalUser(paths);
+    assertThrows(AccessControlException.class, () -> {
+      List<Path> paths = Lists.newArrayList(INACCESSIBLE_DIR_PATH);
+      listAsNormalUser(paths);
+    });
   }
 
   @Test
   public void listInaccessibleFile() throws Exception {
-    thrown.expect(AccessControlException.class);
-    List<Path> paths = Lists.newArrayList(INACCESSIBLE_FILE_PATH);
-    listAsNormalUser(paths);
+    assertThrows(AccessControlException.class, () -> {
+      List<Path> paths = Lists.newArrayList(INACCESSIBLE_FILE_PATH);
+      listAsNormalUser(paths);
+    });
   }
 }

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockMissingException.java

@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.IOException;
 
@@ -31,7 +31,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 public class TestBlockMissingException {
   final static Logger LOG =
@@ -116,6 +116,6 @@ public class TestBlockMissingException {
       gotException = true;
     }
     stm.close();
-    assertTrue("Expected BlockMissingException ", gotException);
+    assertTrue(gotException, "Expected BlockMissingException ");
   }
 }

+ 88 - 82
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java

@@ -47,9 +47,14 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.Lists;
 import org.apache.hadoop.util.Sets;
-import org.junit.Assert;
-import static org.junit.Assert.fail;
-import org.junit.Test;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 /** Test {@link BlockStoragePolicy} */
 public class TestBlockStoragePolicy {
@@ -87,7 +92,8 @@ public class TestBlockStoragePolicy {
   static final byte PROVIDED  = HdfsConstants.PROVIDED_STORAGE_POLICY_ID;
   static final byte ALLNVDIMM = HdfsConstants.ALLNVDIMM_STORAGE_POLICY_ID;
 
-  @Test (timeout=300000)
+  @Test
+  @Timeout(value = 300)
   public void testConfigKeyEnabled() throws IOException {
     Configuration conf = new HdfsConfiguration();
     conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY, true);
@@ -107,19 +113,22 @@ public class TestBlockStoragePolicy {
    * dfs.storage.policy.enabled is set to false.
    * @throws IOException
    */
-  @Test (timeout=300000, expected=IOException.class)
+  @Test
+  @Timeout(value = 300)
   public void testConfigKeyDisabled() throws IOException {
-    Configuration conf = new HdfsConfiguration();
-    conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY, false);
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-        .numDataNodes(1).build();
-    try {
-      cluster.waitActive();
-      cluster.getFileSystem().setStoragePolicy(new Path("/"),
-          HdfsConstants.COLD_STORAGE_POLICY_NAME);
-    } finally {
-      cluster.shutdown();
-    }
+    assertThrows(IOException.class, () -> {
+      Configuration cfg = new HdfsConfiguration();
+      cfg.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY, false);
+      MiniDFSCluster cluster = new MiniDFSCluster.Builder(cfg)
+          .numDataNodes(1).build();
+      try {
+        cluster.waitActive();
+        cluster.getFileSystem().setStoragePolicy(new Path("/"),
+            HdfsConstants.COLD_STORAGE_POLICY_NAME);
+      } finally {
+        cluster.shutdown();
+      }
+    });
   }
 
   @Test
@@ -160,10 +169,10 @@ public class TestBlockStoragePolicy {
       final BlockStoragePolicy policy = POLICY_SUITE.getPolicy(i); 
       if (policy != null) {
         final String s = policy.toString();
-        Assert.assertEquals(expectedPolicyStrings.get(i), s);
+        assertEquals(expectedPolicyStrings.get(i), s);
       }
     }
-    Assert.assertEquals(POLICY_SUITE.getPolicy(HOT), POLICY_SUITE.getDefaultPolicy());
+    assertEquals(POLICY_SUITE.getPolicy(HOT), POLICY_SUITE.getDefaultPolicy());
     
     // check Cold policy
     final BlockStoragePolicy cold = POLICY_SUITE.getPolicy(COLD);
@@ -260,11 +269,11 @@ public class TestBlockStoragePolicy {
 
   static void assertStorageType(List<StorageType> computed, short replication,
       StorageType... answers) {
-    Assert.assertEquals(replication, computed.size());
+    assertEquals(replication, computed.size());
     final StorageType last = answers[answers.length - 1];
     for(int i = 0; i < computed.size(); i++) {
       final StorageType expected = i < answers.length? answers[i]: last;
-      Assert.assertEquals(expected, computed.get(i));
+      assertEquals(expected, computed.get(i));
     }
   }
 
@@ -272,27 +281,26 @@ public class TestBlockStoragePolicy {
       StorageType noneExpected, StorageType archiveExpected,
       StorageType diskExpected, StorageType ssdExpected,
       StorageType disk_archiveExpected, StorageType nvdimmExpected) {
-    Assert.assertEquals(noneExpected, policy.getCreationFallback(none));
-    Assert.assertEquals(archiveExpected, policy.getCreationFallback(archive));
-    Assert.assertEquals(diskExpected, policy.getCreationFallback(disk));
-    Assert.assertEquals(ssdExpected, policy.getCreationFallback(ssd));
-    Assert.assertEquals(nvdimmExpected, policy.getCreationFallback(nvdimm));
-    Assert.assertEquals(disk_archiveExpected,
+    assertEquals(noneExpected, policy.getCreationFallback(none));
+    assertEquals(archiveExpected, policy.getCreationFallback(archive));
+    assertEquals(diskExpected, policy.getCreationFallback(disk));
+    assertEquals(ssdExpected, policy.getCreationFallback(ssd));
+    assertEquals(nvdimmExpected, policy.getCreationFallback(nvdimm));
+    assertEquals(disk_archiveExpected,
         policy.getCreationFallback(disk_archive));
-    Assert.assertEquals(null, policy.getCreationFallback(all));
+    assertEquals(null, policy.getCreationFallback(all));
   }
 
   static void assertReplicationFallback(BlockStoragePolicy policy,
       StorageType noneExpected, StorageType archiveExpected,
       StorageType diskExpected, StorageType ssdExpected,
       StorageType nvdimmExpected) {
-    Assert.assertEquals(noneExpected, policy.getReplicationFallback(none));
-    Assert
-        .assertEquals(archiveExpected, policy.getReplicationFallback(archive));
-    Assert.assertEquals(diskExpected, policy.getReplicationFallback(disk));
-    Assert.assertEquals(ssdExpected, policy.getReplicationFallback(ssd));
-    Assert.assertEquals(nvdimmExpected, policy.getReplicationFallback(nvdimm));
-    Assert.assertEquals(null, policy.getReplicationFallback(all));
+    assertEquals(noneExpected, policy.getReplicationFallback(none));
+    assertEquals(archiveExpected, policy.getReplicationFallback(archive));
+    assertEquals(diskExpected, policy.getReplicationFallback(disk));
+    assertEquals(ssdExpected, policy.getReplicationFallback(ssd));
+    assertEquals(nvdimmExpected, policy.getReplicationFallback(nvdimm));
+    assertEquals(null, policy.getReplicationFallback(all));
   }
 
   private static interface CheckChooseStorageTypes {
@@ -879,7 +887,7 @@ public class TestBlockStoragePolicy {
   static void assertStorageTypes(StorageType[] computed, StorageType... expected) {
     Arrays.sort(expected);
     Arrays.sort(computed);
-    Assert.assertArrayEquals(expected, computed);
+    assertArrayEquals(expected, computed);
   }
 
   @Test
@@ -924,9 +932,9 @@ public class TestBlockStoragePolicy {
   }
 
   private void checkDirectoryListing(HdfsFileStatus[] stats, byte... policies) {
-    Assert.assertEquals(stats.length, policies.length);
+    assertEquals(stats.length, policies.length);
     for (int i = 0; i < stats.length; i++) {
-      Assert.assertEquals(stats[i].getStoragePolicy(), policies[i]);
+      assertEquals(stats[i].getStoragePolicy(), policies[i]);
     }
   }
 
@@ -949,7 +957,7 @@ public class TestBlockStoragePolicy {
       final String invalidPolicyName = "INVALID-POLICY";
       try {
         fs.setStoragePolicy(fooFile, invalidPolicyName);
-        Assert.fail("Should throw a HadoopIllegalArgumentException");
+        fail("Should throw a HadoopIllegalArgumentException");
       } catch (RemoteException e) {
         GenericTestUtils.assertExceptionContains(invalidPolicyName, e);
       }
@@ -967,14 +975,14 @@ public class TestBlockStoragePolicy {
       final Path invalidPath = new Path("/invalidPath");
       try {
         fs.setStoragePolicy(invalidPath, HdfsConstants.WARM_STORAGE_POLICY_NAME);
-        Assert.fail("Should throw a FileNotFoundException");
+        fail("Should throw a FileNotFoundException");
       } catch (FileNotFoundException e) {
         GenericTestUtils.assertExceptionContains(invalidPath.toString(), e);
       }
 
       try {
         fs.getStoragePolicy(invalidPath);
-        Assert.fail("Should throw a FileNotFoundException");
+        fail("Should throw a FileNotFoundException");
       } catch (FileNotFoundException e) {
         GenericTestUtils.assertExceptionContains(invalidPath.toString(), e);
       }
@@ -982,15 +990,15 @@ public class TestBlockStoragePolicy {
       fs.setStoragePolicy(fooFile, HdfsConstants.COLD_STORAGE_POLICY_NAME);
       fs.setStoragePolicy(barDir, HdfsConstants.WARM_STORAGE_POLICY_NAME);
       fs.setStoragePolicy(barFile2, HdfsConstants.HOT_STORAGE_POLICY_NAME);
-      Assert.assertEquals("File storage policy should be COLD",
-          HdfsConstants.COLD_STORAGE_POLICY_NAME,
-          fs.getStoragePolicy(fooFile).getName());
-      Assert.assertEquals("File storage policy should be WARM",
-          HdfsConstants.WARM_STORAGE_POLICY_NAME,
-          fs.getStoragePolicy(barDir).getName());
-      Assert.assertEquals("File storage policy should be HOT",
-          HdfsConstants.HOT_STORAGE_POLICY_NAME,
-          fs.getStoragePolicy(barFile2).getName());
+      assertEquals(HdfsConstants.COLD_STORAGE_POLICY_NAME,
+          fs.getStoragePolicy(fooFile).getName(),
+          "File storage policy should be COLD");
+      assertEquals(HdfsConstants.WARM_STORAGE_POLICY_NAME,
+          fs.getStoragePolicy(barDir).getName(),
+          "File storage policy should be WARM");
+      assertEquals(HdfsConstants.HOT_STORAGE_POLICY_NAME,
+          fs.getStoragePolicy(barFile2).getName(),
+          "File storage policy should be HOT");
 
       dirList = fs.getClient().listPaths(dir.toString(),
           HdfsFileStatus.EMPTY_NAME).getPartialListing();
@@ -1040,8 +1048,8 @@ public class TestBlockStoragePolicy {
           HdfsConstants.COLD_STORAGE_POLICY_NAME);
       String policyName = client.getStoragePolicy("/testGetStoragePolicy/foo")
           .getName();
-      Assert.assertEquals("File storage policy should be COLD",
-          HdfsConstants.COLD_STORAGE_POLICY_NAME, policyName);
+      assertEquals(HdfsConstants.COLD_STORAGE_POLICY_NAME, policyName,
+          "File storage policy should be COLD");
     } finally {
       cluster.shutdown();
     }
@@ -1140,14 +1148,14 @@ public class TestBlockStoragePolicy {
     List<StorageType> typeList = Lists.newArrayList();
     Collections.addAll(typeList, types);
     LocatedBlocks lbs = status.getLocatedBlocks();
-    Assert.assertEquals(blockNum, lbs.getLocatedBlocks().size());
+    assertEquals(blockNum, lbs.getLocatedBlocks().size());
     for (LocatedBlock lb : lbs.getLocatedBlocks()) {
-      Assert.assertEquals(replicaNum, lb.getStorageTypes().length);
+      assertEquals(replicaNum, lb.getStorageTypes().length);
       for (StorageType type : lb.getStorageTypes()) {
-        Assert.assertTrue(typeList.remove(type));
+        assertTrue(typeList.remove(type));
       }
     }
-    Assert.assertTrue(typeList.isEmpty());
+    assertTrue(typeList.isEmpty());
   }
 
   private void testChangeFileRep(String policyName, byte policyId,
@@ -1286,12 +1294,12 @@ public class TestBlockStoragePolicy {
               dataNodes[0], Collections.<DatanodeStorageInfo>emptyList(), false,
               new HashSet<Node>(), 0, policy1, null);
       System.out.println(Arrays.asList(targets));
-      Assert.assertEquals(3, targets.length);
+      assertEquals(3, targets.length);
       targets = replicator.chooseTarget("/foo", 3,
               dataNodes[0], Collections.<DatanodeStorageInfo>emptyList(), false,
               new HashSet<Node>(), 0, policy2, null);
       System.out.println(Arrays.asList(targets));
-      Assert.assertEquals(3, targets.length);
+      assertEquals(3, targets.length);
     } finally {
       if (namenode != null) {
         namenode.stop();
@@ -1339,9 +1347,9 @@ public class TestBlockStoragePolicy {
               dataNodes[0], Collections.<DatanodeStorageInfo>emptyList(), false,
               new HashSet<Node>(), 0, policy, null);
       System.out.println(policy.getName() + ": " + Arrays.asList(targets));
-      Assert.assertEquals(2, targets.length);
-      Assert.assertEquals(StorageType.SSD, targets[0].getStorageType());
-      Assert.assertEquals(StorageType.DISK, targets[1].getStorageType());
+      assertEquals(2, targets.length);
+      assertEquals(StorageType.SSD, targets[0].getStorageType());
+      assertEquals(StorageType.DISK, targets[1].getStorageType());
     } finally {
       if (namenode != null) {
         namenode.stop();
@@ -1400,7 +1408,7 @@ public class TestBlockStoragePolicy {
       DatanodeStorageInfo[] targets = replicator.chooseTarget("/foo", 1,
               null, chsenDs, true,
               new HashSet<Node>(), 0, policy, null);
-      Assert.assertEquals(3, targets.length);
+      assertEquals(3, targets.length);
     } finally {
       if (namenode != null) {
         namenode.stop();
@@ -1432,16 +1440,14 @@ public class TestBlockStoragePolicy {
       fs.setStoragePolicy(dir, "HOT");
       HdfsFileStatus status = fs.getClient().getFileInfo(file);
       // 5. get file policy, it should be parent policy.
-      Assert
-          .assertTrue("File storage policy should be HOT",
-              status.getStoragePolicy() == HOT);
+      assertTrue(status.getStoragePolicy() == HOT,
+          "File storage policy should be HOT");
       // 6. restart NameNode for reloading edits logs.
       cluster.restartNameNode(true);
       // 7. get file policy, it should be parent policy.
       status = fs.getClient().getFileInfo(file);
-      Assert
-          .assertTrue("File storage policy should be HOT",
-              status.getStoragePolicy() == HOT);
+      assertTrue(status.getStoragePolicy() == HOT,
+          "File storage policy should be HOT");
 
     } finally {
       cluster.shutdown();
@@ -1479,8 +1485,8 @@ public class TestBlockStoragePolicy {
       }
 
       // Ensure that we got the same set of policies in both cases.
-      Assert.assertTrue(Sets.difference(policyNamesSet1, policyNamesSet2).isEmpty());
-      Assert.assertTrue(Sets.difference(policyNamesSet2, policyNamesSet1).isEmpty());
+      assertTrue(Sets.difference(policyNamesSet1, policyNamesSet2).isEmpty());
+      assertTrue(Sets.difference(policyNamesSet2, policyNamesSet1).isEmpty());
     } finally {
       cluster.shutdown();
     }
@@ -1499,21 +1505,21 @@ public class TestBlockStoragePolicy {
 
     {
       final Iterator<StorageType> i = map.keySet().iterator();
-      Assert.assertEquals(StorageType.RAM_DISK, i.next());
-      Assert.assertEquals(StorageType.SSD, i.next());
-      Assert.assertEquals(StorageType.DISK, i.next());
-      Assert.assertEquals(StorageType.ARCHIVE, i.next());
-      Assert.assertEquals(StorageType.NVDIMM, i.next());
+      assertEquals(StorageType.RAM_DISK, i.next());
+      assertEquals(StorageType.SSD, i.next());
+      assertEquals(StorageType.DISK, i.next());
+      assertEquals(StorageType.ARCHIVE, i.next());
+      assertEquals(StorageType.NVDIMM, i.next());
     }
 
     {
       final Iterator<Map.Entry<StorageType, Integer>> i
           = map.entrySet().iterator();
-      Assert.assertEquals(StorageType.RAM_DISK, i.next().getKey());
-      Assert.assertEquals(StorageType.SSD, i.next().getKey());
-      Assert.assertEquals(StorageType.DISK, i.next().getKey());
-      Assert.assertEquals(StorageType.ARCHIVE, i.next().getKey());
-      Assert.assertEquals(StorageType.NVDIMM, i.next().getKey());
+      assertEquals(StorageType.RAM_DISK, i.next().getKey());
+      assertEquals(StorageType.SSD, i.next().getKey());
+      assertEquals(StorageType.DISK, i.next().getKey());
+      assertEquals(StorageType.ARCHIVE, i.next().getKey());
+      assertEquals(StorageType.NVDIMM, i.next().getKey());
     }
   }
 
@@ -1671,7 +1677,7 @@ public class TestBlockStoragePolicy {
   public void testCreateDefaultPoliciesFromConf() {
     BlockStoragePolicySuite suite =
         BlockStoragePolicySuite.createDefaultSuite();
-    Assert.assertEquals(HdfsConstants.StoragePolicy.HOT.value(),
+    assertEquals(HdfsConstants.StoragePolicy.HOT.value(),
         suite.getDefaultPolicy().getId());
 
     Configuration newConf = new Configuration();
@@ -1679,7 +1685,7 @@ public class TestBlockStoragePolicy {
         HdfsConstants.StoragePolicy.ONE_SSD);
     BlockStoragePolicySuite suiteConf =
         BlockStoragePolicySuite.createDefaultSuite(newConf);
-    Assert.assertEquals(HdfsConstants.StoragePolicy.ONE_SSD.value(),
+    assertEquals(HdfsConstants.StoragePolicy.ONE_SSD.value(),
         suiteConf.getDefaultPolicy().getId());
   }
 
@@ -1698,7 +1704,7 @@ public class TestBlockStoragePolicy {
       DFSTestUtil.createFile(newfs, fooFile, 0, REPLICATION, 0L);
 
       String policy = newfs.getStoragePolicy(fooFile).getName();
-      Assert.assertEquals(HdfsConstants.StoragePolicy.WARM.name(), policy);
+      assertEquals(HdfsConstants.StoragePolicy.WARM.name(), policy);
     } finally {
       cluster.shutdown();
     }

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

@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -36,8 +36,8 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.util.RwLockMode;
-import org.junit.After;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
 
 /**
  * This class tests DatanodeDescriptor.getBlocksScheduled() at the
@@ -48,7 +48,7 @@ public class TestBlocksScheduledCounter {
   MiniDFSCluster cluster = null;
   FileSystem fs = null;
 
-  @After
+  @AfterEach
   public void tearDown() throws IOException {
     if (fs != null) {
       fs.close();
@@ -105,8 +105,8 @@ public class TestBlocksScheduledCounter {
     ArrayList<DatanodeDescriptor> dnList = new ArrayList<DatanodeDescriptor>();
     datanodeManager.fetchDatanodes(dnList, dnList, false);
     for (DatanodeDescriptor descriptor : dnList) {
-      assertEquals("Blocks scheduled should be 0 for " + descriptor.getName(),
-          0, descriptor.getBlocksScheduled());
+      assertEquals(0, descriptor.getBlocksScheduled(),
+          "Blocks scheduled should be 0 for " + descriptor.getName());
     }
 
     cluster.getDataNodes().get(0).shutdown();
@@ -121,21 +121,21 @@ public class TestBlocksScheduledCounter {
 
     DatanodeDescriptor abandonedDn = datanodeManager.getDatanode(cluster
         .getDataNodes().get(0).getDatanodeId());
-    assertEquals("for the abandoned dn scheduled counts should be 0", 0,
-        abandonedDn.getBlocksScheduled());
+    assertEquals(0, abandonedDn.getBlocksScheduled(),
+        "for the abandoned dn scheduled counts should be 0");
 
     for (DatanodeDescriptor descriptor : dnList) {
       if (descriptor.equals(abandonedDn)) {
         continue;
       }
-      assertEquals("Blocks scheduled should be 1 for " + descriptor.getName(),
-          1, descriptor.getBlocksScheduled());
+      assertEquals(1, descriptor.getBlocksScheduled(),
+          "Blocks scheduled should be 1 for " + descriptor.getName());
     }
     // close the file and the counter should go to zero.
     out.close();
     for (DatanodeDescriptor descriptor : dnList) {
-      assertEquals("Blocks scheduled should be 0 for " + descriptor.getName(),
-          0, descriptor.getBlocksScheduled());
+      assertEquals(0, descriptor.getBlocksScheduled(),
+          "Blocks scheduled should be 0 for " + descriptor.getName());
     }
   }
 

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

@@ -28,14 +28,14 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 
-import static org.junit.Assert.assertArrayEquals;
-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.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 /**
  * This class tests the DFS positional read functionality on a single node
@@ -55,7 +55,7 @@ public class TestByteBufferPread {
   private static final int BLOCK_SIZE = 4096;
   private static final int FILE_SIZE = 12 * BLOCK_SIZE;
 
-  @BeforeClass
+  @BeforeAll
   public static void setup() throws IOException {
     // Setup the cluster with a small block size so we can create small files
     // that span multiple blocks
@@ -278,7 +278,7 @@ public class TestByteBufferPread {
     }
   }
 
-  @AfterClass
+  @AfterAll
   public static void shutdown() throws IOException {
     try {
       fs.delete(testFile, false);

+ 36 - 33
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java

@@ -17,9 +17,11 @@
  */
 package org.apache.hadoop.hdfs;
 
-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.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -55,8 +57,8 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 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.Test;
+import org.junit.jupiter.api.Timeout;
 import org.mockito.Mockito;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -85,9 +87,9 @@ public class TestClientProtocolForPipelineRecovery {
       // test getNewStampAndToken on a finalized block
       try {
         namenode.updateBlockForPipeline(firstBlock, "");
-        Assert.fail("Can not get a new GS from a finalized block");
+        fail("Can not get a new GS from a finalized block");
       } catch (IOException e) {
-        Assert.assertTrue(e.getMessage().contains(
+        assertTrue(e.getMessage().contains(
             "not " + BlockUCState.UNDER_CONSTRUCTION));
       }
       
@@ -97,9 +99,9 @@ public class TestClientProtocolForPipelineRecovery {
         ExtendedBlock newBlock = new ExtendedBlock(firstBlock.getBlockPoolId(),
             newBlockId, 0, firstBlock.getGenerationStamp());
         namenode.updateBlockForPipeline(newBlock, "");
-        Assert.fail("Cannot get a new GS from a non-existent block");
+        fail("Cannot get a new GS from a non-existent block");
       } catch (IOException e) {
-        Assert.assertTrue(e.getMessage().contains("does not exist"));
+        assertTrue(e.getMessage().contains("does not exist"));
       }
 
       
@@ -123,17 +125,17 @@ public class TestClientProtocolForPipelineRecovery {
         DFSClient dfs = ((DistributedFileSystem)fileSys).dfs;
         try {
           namenode.updateBlockForPipeline(firstBlock, "test" + dfs.clientName);
-          Assert.fail("Cannot get a new GS for a non lease holder");
+          fail("Cannot get a new GS for a non lease holder");
         } catch (LeaseExpiredException e) {
-          Assert.assertTrue(e.getMessage().startsWith("Lease mismatch"));
+          assertTrue(e.getMessage().startsWith("Lease mismatch"));
         }
 
         // test null lease holder
         try {
           namenode.updateBlockForPipeline(firstBlock, null);
-          Assert.fail("Cannot get a new GS for a null lease holder");
+          fail("Cannot get a new GS for a null lease holder");
         } catch (LeaseExpiredException e) {
-          Assert.assertTrue(e.getMessage().startsWith("Lease mismatch"));
+          assertTrue(e.getMessage().startsWith("Lease mismatch"));
         }
 
         // test getNewStampAndToken on a rbw block
@@ -178,7 +180,7 @@ public class TestClientProtocolForPipelineRecovery {
         // Test will fail with BlockMissingException if NN does not update the
         // replica state based on the latest report.
       } catch (org.apache.hadoop.hdfs.BlockMissingException bme) {
-        Assert.fail("Block is missing because the file was closed with"
+        fail("Block is missing because the file was closed with"
             + " corrupt replicas.");
       }
     } finally {
@@ -240,7 +242,7 @@ public class TestClientProtocolForPipelineRecovery {
           contains = true;
         }
       }
-      Assert.assertTrue(contains);
+      assertTrue(contains);
     } finally {
       DataNodeFaultInjector.set(oldDnInjector);
       if (cluster != null) {
@@ -323,7 +325,7 @@ public class TestClientProtocolForPipelineRecovery {
       final String dnAddr = dn.getDatanodeId().getIpcAddr(false);
       // issue shutdown to the datanode.
       final String[] args1 = {"-shutdownDatanode", dnAddr, "upgrade" };
-      Assert.assertEquals(0, dfsadmin.run(args1));
+      assertEquals(0, dfsadmin.run(args1));
       // Wait long enough to receive an OOB ack before closing the file.
       GenericTestUtils.waitForThreadTermination(
           "Async datanode shutdown thread", 100, 10000);
@@ -359,23 +361,23 @@ public class TestClientProtocolForPipelineRecovery {
       // get nodes in the pipeline
       DFSOutputStream dfsOut = (DFSOutputStream)out.getWrappedStream();
       DatanodeInfo[] nodes = dfsOut.getPipeline();
-      Assert.assertEquals(2, nodes.length);
+      assertEquals(2, nodes.length);
       String dnAddr = nodes[1].getIpcAddr(false);
 
       // evict the writer from the second datanode and wait until
       // the pipeline is rebuilt.
       DFSAdmin dfsadmin = new DFSAdmin(conf);
       final String[] args1 = {"-evictWriters", dnAddr };
-      Assert.assertEquals(0, dfsadmin.run(args1));
+      assertEquals(0, dfsadmin.run(args1));
       out.write(0x31);
       out.hflush();
 
       // get the new pipline and check the node is not in there.
       nodes = dfsOut.getPipeline();
       try {
-        Assert.assertTrue(nodes.length > 0 );
+        assertTrue(nodes.length > 0);
         for (int i = 0; i < nodes.length; i++) {
-          Assert.assertFalse(dnAddr.equals(nodes[i].getIpcAddr(false)));
+          assertFalse(dnAddr.equals(nodes[i].getIpcAddr(false)));
         }
       } finally {
         out.close();
@@ -411,7 +413,7 @@ public class TestClientProtocolForPipelineRecovery {
       final String dnAddr1 = dn.getDatanodeId().getIpcAddr(false);
       // issue shutdown to the datanode.
       final String[] args1 = {"-shutdownDatanode", dnAddr1, "upgrade" };
-      Assert.assertEquals(0, dfsadmin.run(args1));
+      assertEquals(0, dfsadmin.run(args1));
       GenericTestUtils.waitForThreadTermination(
           "Async datanode shutdown thread", 100, 10000);
       // This should succeed without restarting the node. The restart will
@@ -428,7 +430,7 @@ public class TestClientProtocolForPipelineRecovery {
       final String dnAddr2 = dn.getDatanodeId().getIpcAddr(false);
       // issue shutdown to the datanode.
       final String[] args2 = {"-shutdownDatanode", dnAddr2, "upgrade" };
-      Assert.assertEquals(0, dfsadmin.run(args2));
+      assertEquals(0, dfsadmin.run(args2));
       GenericTestUtils.waitForThreadTermination(
           "Async datanode shutdown thread", 100, 10000);
       try {
@@ -450,7 +452,8 @@ public class TestClientProtocolForPipelineRecovery {
    *  (in a row for the same packet, including the heartbeat packet)
    *  (See{@link DataStreamer#getPipelineRecoveryCount})
    */
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testPipelineRecoveryOnDatanodeUpgrade() throws Exception {
     Configuration conf = new HdfsConfiguration();
     MiniDFSCluster cluster = null;
@@ -481,8 +484,8 @@ public class TestClientProtocolForPipelineRecovery {
           return out.getBlock().getGenerationStamp() > oldGs;
         }
       }, 100, 10000);
-      Assert.assertEquals("The pipeline recovery count shouldn't increase",
-          0, out.getStreamer().getPipelineRecoveryCount());
+      assertEquals(0, out.getStreamer().getPipelineRecoveryCount(),
+          "The pipeline recovery count shouldn't increase");
       out.write(1);
       out.close();
       // Ensure that subsequent closes are idempotent and do not throw errors
@@ -540,7 +543,7 @@ public class TestClientProtocolForPipelineRecovery {
       Thread.sleep(1000);
       DatanodeInfo[] pipeline = out.getPipeline();
       for (DatanodeInfo node : pipeline) {
-        assertFalse("Write should be going on", failed.get());
+        assertFalse(failed.get(), "Write should be going on");
         ArrayList<DataNode> dataNodes = cluster.getDataNodes();
         int indexToShutdown = 0;
         for (int i = 0; i < dataNodes.size(); i++) {
@@ -565,15 +568,15 @@ public class TestClientProtocolForPipelineRecovery {
             return out.getBlock().getGenerationStamp() > oldGs;
           }
         }, 100, 10000);
-        Assert.assertEquals("The pipeline recovery count shouldn't increase", 0,
-            out.getStreamer().getPipelineRecoveryCount());
+        assertEquals(0, out.getStreamer().getPipelineRecoveryCount(),
+            "The pipeline recovery count shouldn't increase");
       }
-      assertFalse("Write should be going on", failed.get());
+      assertFalse(failed.get(), "Write should be going on");
       running.set(false);
       t.join();
       out.write("testagain".getBytes());
-      assertTrue("There should be atleast 2 nodes in pipeline still", out
-          .getPipeline().length >= 2);
+      assertTrue(out.getPipeline().length >= 2,
+          "There should be atleast 2 nodes in pipeline still");
       out.close();
     } finally {
       DFSClientFaultInjector.set(old);
@@ -724,7 +727,7 @@ public class TestClientProtocolForPipelineRecovery {
         o.hflush();
       }
 
-      assertTrue("Expected a failure in the pipeline", failed.get());
+      assertTrue(failed.get(), "Expected a failure in the pipeline");
       DatanodeInfo[] newNodes = dfsO.getStreamer().getNodes();
       o.close();
       // Trigger block report to NN
@@ -940,7 +943,7 @@ public class TestClientProtocolForPipelineRecovery {
         count++;
         o.hflush();
       }
-      Assert.assertNotEquals(lastDn, dfsO.getStreamer().getNodes()[2].getXferAddr(false));
+      assertNotEquals(lastDn, dfsO.getStreamer().getNodes()[2].getXferAddr(false));
     } finally {
       DataNodeFaultInjector.set(old);
       cluster.shutdown();

+ 16 - 14
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java

@@ -41,10 +41,12 @@ import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck;
 import org.apache.hadoop.hdfs.tools.DFSck;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.util.ToolRunner;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 /**
  * Class is used to test client reporting corrupted block replica to name node.
@@ -67,7 +69,7 @@ public class TestClientReportBadBlock {
 
   Random rand = new Random();
 
-  @Before
+  @BeforeEach
   public void startUpCluster() throws IOException {
     // disable block scanner
     conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1); 
@@ -80,7 +82,7 @@ public class TestClientReportBadBlock {
     buffersize = conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096);
   }
 
-  @After
+  @AfterEach
   public void shutDownCluster() throws IOException {
     if (dfs != null) {
       dfs.close();
@@ -211,7 +213,7 @@ public class TestClientReportBadBlock {
     // Locate the file blocks by asking name node
     final LocatedBlocks locatedblocks = dfs.dfs.getNamenode()
         .getBlockLocations(filePath.toString(), 0L, BLOCK_SIZE);
-    Assert.assertEquals(repl, locatedblocks.get(0).getLocations().length);
+    assertEquals(repl, locatedblocks.get(0).getLocations().length);
     // The file only has one block
     LocatedBlock lblock = locatedblocks.get(0);
     DatanodeInfo[] datanodeinfos = lblock.getLocations();
@@ -236,7 +238,7 @@ public class TestClientReportBadBlock {
     final LocatedBlocks locatedBlocks = dfs.dfs.getNamenode()
         .getBlockLocations(filePath.toUri().getPath(), 0, Long.MAX_VALUE);
     final LocatedBlock firstLocatedBlock = locatedBlocks.get(0);
-    Assert.assertEquals(isCorrupted, firstLocatedBlock.isCorrupt());
+    assertEquals(isCorrupted, firstLocatedBlock.isCorrupt());
   }
 
   /**
@@ -250,7 +252,7 @@ public class TestClientReportBadBlock {
         filePath.toUri().getPath(), 0, Long.MAX_VALUE);
     // we expect only the first block of the file is used for this test
     LocatedBlock firstLocatedBlock = lBlocks.get(0);
-    Assert.assertEquals(expectedReplicas,
+    assertEquals(expectedReplicas,
         firstLocatedBlock.getLocations().length);
   }
 
@@ -300,23 +302,23 @@ public class TestClientReportBadBlock {
     // Make sure filesystem is in healthy state
     String outStr = runFsck(conf, 0, true, "/");
     LOG.info(outStr);
-    Assert.assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS));
+    assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS));
     if (!expected.equals("")) {
-      Assert.assertTrue(outStr.contains(expected));
+      assertTrue(outStr.contains(expected));
     }
   }
 
   private static void verifyFsckBlockCorrupted() throws Exception {
     String outStr = runFsck(conf, 1, true, "/");
     LOG.info(outStr);
-    Assert.assertTrue(outStr.contains(NamenodeFsck.CORRUPT_STATUS));
+    assertTrue(outStr.contains(NamenodeFsck.CORRUPT_STATUS));
   }
   
   private static void testFsckListCorruptFilesBlocks(Path filePath, int errorCode) throws Exception{
     String outStr = runFsck(conf, errorCode, true, filePath.toString(), "-list-corruptfileblocks");
     LOG.info("fsck -list-corruptfileblocks out: " + outStr);
     if (errorCode != 0) {
-      Assert.assertTrue(outStr.contains("CORRUPT blocks"));
+      assertTrue(outStr.contains("CORRUPT blocks"));
     }
   }
 
@@ -326,7 +328,7 @@ public class TestClientReportBadBlock {
     PrintStream out = new PrintStream(bStream, true);
     int errCode = ToolRunner.run(new DFSck(conf, out), path);
     if (checkErrorCode)
-      Assert.assertEquals(expectedErrCode, errCode);
+      assertEquals(expectedErrCode, errCode);
     return bStream.toString();
   }
 }

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

@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.IOException;
 import java.io.OutputStream;
@@ -26,7 +26,7 @@ import java.nio.channels.ClosedChannelException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 public class TestClose {
 

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

@@ -17,7 +17,8 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
@@ -27,8 +28,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.impl.BlockReaderTestUtil;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 /**
  * This class tests the client connection caching in a single node
@@ -52,7 +52,7 @@ public class TestConnCache {
                      int length,
                      byte[] authenticData)
       throws IOException {
-    Assert.assertTrue("Test buffer too small", buffer.length >= offset + length);
+    assertTrue(buffer.length >= offset + length, "Test buffer too small");
 
     if (pos >= 0)
       in.seek(pos);
@@ -62,7 +62,7 @@ public class TestConnCache {
 
     while (length > 0) {
       int cnt = in.read(buffer, offset, length);
-      Assert.assertTrue("Error in read", cnt > 0);
+      assertTrue(cnt > 0, "Error in read");
       offset += cnt;
       length -= cnt;
     }
@@ -71,9 +71,8 @@ public class TestConnCache {
     for (int i = 0; i < length; ++i) {
       byte actual = buffer[i];
       byte expect = authenticData[(int)pos + i];
-      assertEquals("Read data mismatch at file offset " + (pos + i) +
-                   ". Expects " + expect + "; got " + actual,
-                   actual, expect);
+      assertEquals(actual, expect, "Read data mismatch at file offset " + (pos + i) +
+              ". Expects " + expect + "; got " + actual);
     }
   }
 
@@ -116,7 +115,6 @@ public class TestConnCache {
 
     in.close();
     client.close();
-    Assert.assertEquals(1,
-        ClientContext.getFromConf(configuration).getPeerCache().size());
+    assertEquals(1, ClientContext.getFromConf(configuration).getPeerCache().size());
   }
 }

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

@@ -18,9 +18,9 @@
 
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.IOException;
 import java.util.List;
@@ -36,8 +36,9 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.io.IOUtils;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.mockito.Mockito;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -75,7 +76,7 @@ public class TestCrcCorruption {
 
   private DFSClientFaultInjector faultInjector;
 
-  @Before
+  @BeforeEach
   public void setUp() throws IOException {
     faultInjector = Mockito.mock(DFSClientFaultInjector.class);
     DFSClientFaultInjector.set(faultInjector);
@@ -86,7 +87,8 @@ public class TestCrcCorruption {
    * create/write. To recover from corruption while writing, at
    * least two replicas are needed.
    */
-  @Test(timeout=50000)
+  @Test
+  @Timeout(value = 50)
   public void testCorruptionDuringWrt() throws Exception {
     Configuration conf = new HdfsConfiguration();
     // Set short retry timeouts so this test runs faster
@@ -174,7 +176,7 @@ public class TestCrcCorruption {
       final String bpid = cluster.getNamesystem().getBlockPoolId();
       List<ReplicaInfo> replicas =
           dn.getFSDataset().getFinalizedBlocks(bpid);
-      assertTrue("Replicas do not exist", !replicas.isEmpty());
+      assertTrue(!replicas.isEmpty(), "Replicas do not exist");
 
       for (int idx = 0; idx < replicas.size(); idx++) {
         ReplicaInfo replica = replicas.get(idx);
@@ -196,8 +198,8 @@ public class TestCrcCorruption {
       // Only one replica is possibly corrupted. The other replica should still
       // be good. Verify.
       //
-      assertTrue("Corrupted replicas not handled properly.",
-                 util.checkFiles(fs, "/srcdat"));
+      assertTrue(util.checkFiles(fs, "/srcdat"),
+          "Corrupted replicas not handled properly.");
       LOG.info("All File still have a valid replica");
 
       //
@@ -250,7 +252,8 @@ public class TestCrcCorruption {
    * there's no infinite loop, but rather it eventually
    * reports the exception to the client.
    */
-  @Test(timeout=300000) // 5 min timeout
+  @Test
+  @Timeout(value = 300)
   public void testEntirelyCorruptFileOneNode() throws Exception {
     doTestEntirelyCorruptFile(1);
   }
@@ -263,7 +266,8 @@ public class TestCrcCorruption {
    * times out, this suggests that the client is retrying
    * indefinitely.
    */
-  @Test(timeout=300000) // 5 min timeout
+  @Test
+  @Timeout(value = 300)
   public void testEntirelyCorruptFileThreeNodes() throws Exception {
     doTestEntirelyCorruptFile(3);
   }
@@ -287,7 +291,7 @@ public class TestCrcCorruption {
 
       ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, file);
       int blockFilesCorrupted = cluster.corruptBlockOnDataNodes(block);
-      assertEquals("All replicas not corrupted", replFactor, blockFilesCorrupted);
+      assertEquals(replFactor, blockFilesCorrupted, "All replicas not corrupted");
 
       try {
         IOUtils.copyBytes(fs.open(file), new IOUtils.NullOutputStream(), conf,

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

@@ -27,8 +27,8 @@ package org.apache.hadoop.hdfs;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -37,7 +37,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 
 public class TestDFSAddressConfig {
@@ -67,7 +67,7 @@ public class TestDFSAddressConfig {
      *------------------------------------------------------------------------*/
     for (int i = 0; i < dns.size(); i++) {
       DataNodeProperties dnp = cluster.stopDataNode(i);
-      assertNotNull("Should have been able to stop simulated datanode", dnp);
+      assertNotNull(dnp, "Should have been able to stop simulated datanode");
     }
 
     conf.unset(DFS_DATANODE_ADDRESS_KEY);
@@ -92,7 +92,7 @@ public class TestDFSAddressConfig {
      *------------------------------------------------------------------------*/
     for (int i = 0; i < dns.size(); i++) {
       DataNodeProperties dnp = cluster.stopDataNode(i);
-      assertNotNull("Should have been able to stop simulated datanode", dnp);
+      assertNotNull(dnp, "Should have been able to stop simulated datanode");
     }
 
     conf.set(DFS_DATANODE_ADDRESS_KEY, "0.0.0.0:0");

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

@@ -17,7 +17,8 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.IOException;
 import java.io.OutputStream;
@@ -29,10 +30,10 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.util.ThreadUtil;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 
 /**
@@ -44,13 +45,13 @@ public class TestDFSClientExcludedNodes {
   private MiniDFSCluster cluster;
   private Configuration conf;
 
-  @Before
+  @BeforeEach
   public void setUp() {
     cluster = null;
     conf = new HdfsConfiguration();
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
     if (cluster != null) {
       cluster.shutdown();
@@ -58,7 +59,8 @@ public class TestDFSClientExcludedNodes {
     }
   }
 
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testExcludedNodes() throws IOException {
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     FileSystem fs = cluster.getFileSystem();
@@ -83,7 +85,8 @@ public class TestDFSClientExcludedNodes {
     }
   }
 
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testExcludedNodesForgiveness() throws IOException {
     // Forgive nodes in under 2.5s for this test case.
     conf.setLong(
@@ -122,8 +125,8 @@ public class TestDFSClientExcludedNodes {
 
     // Bring back the older DNs, since they are gonna be forgiven only
     // afterwards of this previous block write.
-    Assert.assertEquals(true, cluster.restartDataNode(one, true));
-    Assert.assertEquals(true, cluster.restartDataNode(two, true));
+    assertEquals(true, cluster.restartDataNode(one, true));
+    assertEquals(true, cluster.restartDataNode(two, true));
     cluster.waitActive();
 
     // Sleep for 5s, to let the excluded nodes be expired

+ 32 - 30
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java

@@ -18,10 +18,10 @@
 package org.apache.hadoop.hdfs;
 
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyBoolean;
 import static org.mockito.ArgumentMatchers.anyLong;
@@ -90,9 +90,9 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.mockito.Mockito;
 import org.mockito.internal.stubbing.answers.ThrowsException;
 import org.mockito.invocation.InvocationOnMock;
@@ -160,7 +160,7 @@ public class TestDFSClientRetries {
     }
   }
   
-  @Before
+  @BeforeEach
   public void setupConf(){
     conf = new HdfsConfiguration();
   }
@@ -285,8 +285,8 @@ public class TestDFSClientRetries {
     try {
       os.close();
     } catch (Exception e) {
-      assertTrue("Retries are not being stopped correctly: " + e.getMessage(),
-           e.getMessage().equals(exceptionMsg));
+      assertTrue(e.getMessage().equals(exceptionMsg),
+          "Retries are not being stopped correctly: " + e.getMessage());
     }
   }
 
@@ -632,7 +632,7 @@ public class TestDFSClientRetries {
     timestamp = Time.now();
     pass = busyTest(xcievers, threads, fileLen, timeWin, retries);
     timestamp2 = Time.now();
-    assertTrue("Something wrong! Test 2 got Exception with maxmum retries!", pass);
+    assertTrue(pass, "Something wrong! Test 2 got Exception with maxmum retries!");
     LOG.info("Test 2 succeeded! Time spent: "  + (timestamp2-timestamp)/1000.0 + " sec.");
     
     //
@@ -657,7 +657,7 @@ public class TestDFSClientRetries {
     timestamp = Time.now();
     pass = busyTest(xcievers, threads, fileLen, timeWin, retries);
     timestamp2 = Time.now();
-    assertTrue("Something wrong! Test 4 got Exception with maxmum retries!", pass);
+    assertTrue(pass, "Something wrong! Test 4 got Exception with maxmum retries!");
     LOG.info("Test 4 succeeded! Time spent: "  + (timestamp2-timestamp)/1000.0 + " sec.");
   }
 
@@ -694,8 +694,8 @@ public class TestDFSClientRetries {
                                          blockSize);
       
       // verify that file exists in FS namespace
-      assertTrue(file1 + " should be a file", 
-                  fs.getFileStatus(file1).isFile());
+      assertTrue(fs.getFileStatus(file1).isFile(),
+          file1 + " should be a file");
       System.out.println("Path : \"" + file1 + "\"");
       LOG.info("Path : \"" + file1 + "\"");
 
@@ -706,10 +706,10 @@ public class TestDFSClientRetries {
 
       // verify that file size has changed to the full size
       long len = fs.getFileStatus(file1).getLen();
-      
-      assertTrue(file1 + " should be of size " + fileLen +
-                 " but found to be of size " + len, 
-                  len == fileLen);
+
+      assertTrue(len == fileLen, file1 +
+          " should be of size " + fileLen +
+          " but found to be of size " + len);
       
       // read back and check data integrigy
       byte[] read_buf = new byte[fileLen];
@@ -809,11 +809,11 @@ public class TestDFSClientRetries {
         in.close();
         fs.close();
 
-        assertTrue("hashed keys are not the same size",
-                   hash_sha.length == expected_sha.length);
+        assertTrue(hash_sha.length == expected_sha.length,
+            "hashed keys are not the same size");
 
-        assertTrue("hashed keys are not equal",
-                   Arrays.equals(hash_sha, expected_sha));
+        assertTrue(Arrays.equals(hash_sha, expected_sha),
+            "hashed keys are not equal");
         
         counter.inc(); // count this thread as successful
         
@@ -928,8 +928,8 @@ public class TestDFSClientRetries {
 
       ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, path);
       int blockFilesCorrupted = cluster.corruptBlockOnDataNodes(block);
-      assertEquals("All replicas not corrupted", REPL_FACTOR,
-          blockFilesCorrupted);
+      assertEquals(REPL_FACTOR, blockFilesCorrupted,
+          "All replicas not corrupted");
 
       InetSocketAddress nnAddr =
         new InetSocketAddress("localhost", cluster.getNameNodePort());
@@ -952,7 +952,8 @@ public class TestDFSClientRetries {
   }
 
   /** Test client retry with namenode restarting. */
-  @Test(timeout=300000)
+  @Test
+  @Timeout(value = 300)
   public void testNamenodeRestart() throws Exception {
     namenodeRestartTest(new Configuration(), false);
   }
@@ -1107,13 +1108,13 @@ public class TestDFSClientRetries {
         final FSDataInputStream in = fs.open(file4);
         int count = 0;
         for(int r; (r = in.read()) != -1; count++) {
-          Assert.assertEquals(String.format("count=%d", count),
-              bytes[count % bytes.length], (byte)r);
+          assertEquals(bytes[count % bytes.length],
+              (byte) r, String.format("count=%d", count));
         }
         if (!isWebHDFS) {
-          Assert.assertEquals(5 * bytes.length, count);
+          assertEquals(5 * bytes.length, count);
         } else {
-          Assert.assertEquals(2 * bytes.length, count);
+          assertEquals(2 * bytes.length, count);
         }
         in.close();
       }
@@ -1274,7 +1275,8 @@ public class TestDFSClientRetries {
     }
   }
 
-  @Test(timeout=120000)
+  @Test
+  @Timeout(value = 120)
   public void testLeaseRenewAndDFSOutputStreamDeadLock() throws Exception {
     CountDownLatch testLatch = new CountDownLatch(1);
     DFSClientFaultInjector.set(new DFSClientFaultInjector() {

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

@@ -21,7 +21,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder;
 import org.apache.hadoop.test.GenericTestUtils;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -31,7 +31,7 @@ import java.io.IOException;
 import java.net.Socket;
 
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_SOCKET_SEND_BUFFER_SIZE_KEY;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class TestDFSClientSocketSize {
   private static final Logger LOG = LoggerFactory.getLogger(
@@ -49,8 +49,8 @@ public class TestDFSClientSocketSize {
     final int sendBufferSize = getSendBufferSize(new Configuration());
     LOG.info("If not specified, the auto tuned send buffer size is: {}",
         sendBufferSize);
-    assertTrue("Send buffer size should be non-negative value which is " +
-        "determined by system (kernel).", sendBufferSize > 0);
+    assertTrue(sendBufferSize > 0, "Send buffer size should be non-negative value which is " +
+        "determined by system (kernel).");
   }
 
   /**
@@ -69,8 +69,8 @@ public class TestDFSClientSocketSize {
 
     LOG.info("Large buf size is {}, small is {}",
         sendBufferSize1, sendBufferSize2);
-    assertTrue("Larger specified send buffer should have effect",
-        sendBufferSize1 > sendBufferSize2);
+    assertTrue(sendBufferSize1 > sendBufferSize2,
+        "Larger specified send buffer should have effect");
   }
 
   /**
@@ -83,8 +83,8 @@ public class TestDFSClientSocketSize {
     conf.setInt(DFS_CLIENT_SOCKET_SEND_BUFFER_SIZE_KEY, 0);
     final int sendBufferSize = getSendBufferSize(conf);
     LOG.info("The auto tuned send buffer size is: {}", sendBufferSize);
-    assertTrue("Send buffer size should be non-negative value which is " +
-        "determined by system (kernel).", sendBufferSize > 0);
+    assertTrue(sendBufferSize > 0, "Send buffer size should be non-negative value which is " +
+        "determined by system (kernel).");
   }
 
   private int getSendBufferSize(Configuration conf) throws IOException {

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

@@ -18,8 +18,8 @@
 package org.apache.hadoop.hdfs;
 
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.DATA_NODE;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
 
 import java.io.File;
 import java.util.Collections;
@@ -33,8 +33,8 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
 import org.apache.hadoop.hdfs.server.datanode.BlockPoolSliceStorage;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
-import org.junit.After;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
 
 /**
  * This test ensures the appropriate response from the system when 
@@ -184,7 +184,7 @@ public class TestDFSFinalize {
     } // end numDir loop
   }
  
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     LOG.info("Shutting down MiniDFSCluster");
     if (cluster != null) {

+ 185 - 176
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java

@@ -35,8 +35,8 @@ import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
 import org.apache.hadoop.util.ExitUtil;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import java.io.IOException;
 import java.io.OutputStream;
@@ -46,6 +46,11 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
 public class TestDFSInotifyEventInputStream {
 
   private static final int BLOCK_SIZE = 1024;
@@ -60,8 +65,8 @@ public class TestDFSInotifyEventInputStream {
   }
 
   private static long checkTxid(EventBatch batch, long prevTxid){
-    Assert.assertTrue("Previous txid " + prevTxid + " was not less than " +
-        "new txid " + batch.getTxid(), prevTxid < batch.getTxid());
+    assertTrue(prevTxid < batch.getTxid(), "Previous txid " + prevTxid + " was not less than " +
+        "new txid " + batch.getTxid());
     return batch.getTxid();
   }
 
@@ -73,15 +78,16 @@ public class TestDFSInotifyEventInputStream {
    */
   @Test
   public void testOpcodeCount() {
-    Assert.assertEquals(54, FSEditLogOpCodes.values().length);
+    assertEquals(54, FSEditLogOpCodes.values().length);
   }
 
 
   /**
    * Tests all FsEditLogOps that are converted to inotify events.
    */
-  @Test(timeout = 120000)
-  @SuppressWarnings("deprecation")
+  @Test
+  @Timeout(value = 120)
+  @SuppressWarnings({"deprecation", "checkstyle:MethodLength"})
   public void testBasic() throws IOException, URISyntaxException,
       InterruptedException, MissingEventsException {
     Configuration conf = new HdfsConfiguration();
@@ -146,294 +152,294 @@ public class TestDFSInotifyEventInputStream {
 
       // RenameOp
       batch = waitForNextEvents(eis);
-      Assert.assertEquals(1, batch.getEvents().length);
+      assertEquals(1, batch.getEvents().length);
       long txid = batch.getTxid();
-      Assert.assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.RENAME);
+      assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.RENAME);
       Event.RenameEvent re = (Event.RenameEvent) batch.getEvents()[0];
-      Assert.assertEquals("/file4", re.getDstPath());
-      Assert.assertEquals("/file", re.getSrcPath());
-      Assert.assertTrue(re.getTimestamp() > 0);
+      assertEquals("/file4", re.getDstPath());
+      assertEquals("/file", re.getSrcPath());
+      assertTrue(re.getTimestamp() > 0);
       LOG.info(re.toString());
-      Assert.assertTrue(re.toString().startsWith("RenameEvent [srcPath="));
+      assertTrue(re.toString().startsWith("RenameEvent [srcPath="));
 
       long eventsBehind = eis.getTxidsBehindEstimate();
 
       // RenameOldOp
       batch = waitForNextEvents(eis);
-      Assert.assertEquals(1, batch.getEvents().length);
+      assertEquals(1, batch.getEvents().length);
       txid = checkTxid(batch, txid);
-      Assert.assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.RENAME);
+      assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.RENAME);
       Event.RenameEvent re2 = (Event.RenameEvent) batch.getEvents()[0];
-      Assert.assertTrue(re2.getDstPath().equals("/file2"));
-      Assert.assertTrue(re2.getSrcPath().equals("/file4"));
-      Assert.assertTrue(re2.getTimestamp() > 0);
+      assertTrue(re2.getDstPath().equals("/file2"));
+      assertTrue(re2.getSrcPath().equals("/file4"));
+      assertTrue(re2.getTimestamp() > 0);
       LOG.info(re2.toString());
 
       // AddOp with overwrite
       batch = waitForNextEvents(eis);
-      Assert.assertEquals(1, batch.getEvents().length);
+      assertEquals(1, batch.getEvents().length);
       txid = checkTxid(batch, txid);
-      Assert.assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.CREATE);
+      assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.CREATE);
       Event.CreateEvent ce = (Event.CreateEvent) batch.getEvents()[0];
-      Assert.assertTrue(ce.getiNodeType() == Event.CreateEvent.INodeType.FILE);
-      Assert.assertTrue(ce.getPath().equals("/file2"));
-      Assert.assertTrue(ce.getCtime() > 0);
-      Assert.assertTrue(ce.getReplication() > 0);
-      Assert.assertTrue(ce.getSymlinkTarget() == null);
-      Assert.assertTrue(ce.getOverwrite());
-      Assert.assertEquals(BLOCK_SIZE, ce.getDefaultBlockSize());
-      Assert.assertTrue(ce.isErasureCoded().isPresent());
-      Assert.assertFalse(ce.isErasureCoded().get());
+      assertTrue(ce.getiNodeType() == Event.CreateEvent.INodeType.FILE);
+      assertTrue(ce.getPath().equals("/file2"));
+      assertTrue(ce.getCtime() > 0);
+      assertTrue(ce.getReplication() > 0);
+      assertTrue(ce.getSymlinkTarget() == null);
+      assertTrue(ce.getOverwrite());
+      assertEquals(BLOCK_SIZE, ce.getDefaultBlockSize());
+      assertTrue(ce.isErasureCoded().isPresent());
+      assertFalse(ce.isErasureCoded().get());
       LOG.info(ce.toString());
-      Assert.assertTrue(ce.toString().startsWith("CreateEvent [INodeType="));
+      assertTrue(ce.toString().startsWith("CreateEvent [INodeType="));
 
       // CloseOp
       batch = waitForNextEvents(eis);
-      Assert.assertEquals(1, batch.getEvents().length);
+      assertEquals(1, batch.getEvents().length);
       txid = checkTxid(batch, txid);
-      Assert.assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.CLOSE);
+      assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.CLOSE);
       Event.CloseEvent ce2 = (Event.CloseEvent) batch.getEvents()[0];
-      Assert.assertTrue(ce2.getPath().equals("/file2"));
-      Assert.assertTrue(ce2.getFileSize() > 0);
-      Assert.assertTrue(ce2.getTimestamp() > 0);
+      assertTrue(ce2.getPath().equals("/file2"));
+      assertTrue(ce2.getFileSize() > 0);
+      assertTrue(ce2.getTimestamp() > 0);
       LOG.info(ce2.toString());
-      Assert.assertTrue(ce2.toString().startsWith("CloseEvent [path="));
+      assertTrue(ce2.toString().startsWith("CloseEvent [path="));
 
       // AppendOp
       batch = waitForNextEvents(eis);
-      Assert.assertEquals(1, batch.getEvents().length);
+      assertEquals(1, batch.getEvents().length);
       txid = checkTxid(batch, txid);
-      Assert.assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.APPEND);
+      assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.APPEND);
       Event.AppendEvent append2 = (Event.AppendEvent)batch.getEvents()[0];
-      Assert.assertEquals("/file2", append2.getPath());
-      Assert.assertFalse(append2.toNewBlock());
+      assertEquals("/file2", append2.getPath());
+      assertFalse(append2.toNewBlock());
       LOG.info(append2.toString());
-      Assert.assertTrue(append2.toString().startsWith("AppendEvent [path="));
+      assertTrue(append2.toString().startsWith("AppendEvent [path="));
 
       // CloseOp
       batch = waitForNextEvents(eis);
-      Assert.assertEquals(1, batch.getEvents().length);
+      assertEquals(1, batch.getEvents().length);
       txid = checkTxid(batch, txid);
-      Assert.assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.CLOSE);
-      Assert.assertTrue(((Event.CloseEvent) batch.getEvents()[0]).getPath().equals("/file2"));
+      assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.CLOSE);
+      assertTrue(((Event.CloseEvent) batch.getEvents()[0]).getPath().equals("/file2"));
 
       // TimesOp
       batch = waitForNextEvents(eis);
-      Assert.assertEquals(1, batch.getEvents().length);
+      assertEquals(1, batch.getEvents().length);
       txid = checkTxid(batch, txid);
-      Assert.assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.METADATA);
+      assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.METADATA);
       Event.MetadataUpdateEvent mue = (Event.MetadataUpdateEvent) batch.getEvents()[0];
-      Assert.assertTrue(mue.getPath().equals("/file2"));
-      Assert.assertTrue(mue.getMetadataType() ==
+      assertTrue(mue.getPath().equals("/file2"));
+      assertTrue(mue.getMetadataType() ==
           Event.MetadataUpdateEvent.MetadataType.TIMES);
       LOG.info(mue.toString());
-      Assert.assertTrue(mue.toString().startsWith("MetadataUpdateEvent [path="));
+      assertTrue(mue.toString().startsWith("MetadataUpdateEvent [path="));
 
       // SetReplicationOp
       batch = waitForNextEvents(eis);
-      Assert.assertEquals(1, batch.getEvents().length);
+      assertEquals(1, batch.getEvents().length);
       txid = checkTxid(batch, txid);
-      Assert.assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.METADATA);
+      assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.METADATA);
       Event.MetadataUpdateEvent mue2 = (Event.MetadataUpdateEvent) batch.getEvents()[0];
-      Assert.assertTrue(mue2.getPath().equals("/file2"));
-      Assert.assertTrue(mue2.getMetadataType() ==
+      assertTrue(mue2.getPath().equals("/file2"));
+      assertTrue(mue2.getMetadataType() ==
           Event.MetadataUpdateEvent.MetadataType.REPLICATION);
-      Assert.assertTrue(mue2.getReplication() == 1);
+      assertTrue(mue2.getReplication() == 1);
       LOG.info(mue2.toString());
 
       // ConcatDeleteOp
       batch = waitForNextEvents(eis);
-      Assert.assertEquals(3, batch.getEvents().length);
+      assertEquals(3, batch.getEvents().length);
       txid = checkTxid(batch, txid);
-      Assert.assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.APPEND);
-      Assert.assertTrue(((Event.AppendEvent) batch.getEvents()[0]).getPath().equals("/file2"));
-      Assert.assertTrue(batch.getEvents()[1].getEventType() == Event.EventType.UNLINK);
+      assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.APPEND);
+      assertTrue(((Event.AppendEvent) batch.getEvents()[0]).getPath().equals("/file2"));
+      assertTrue(batch.getEvents()[1].getEventType() == Event.EventType.UNLINK);
       Event.UnlinkEvent ue2 = (Event.UnlinkEvent) batch.getEvents()[1];
-      Assert.assertTrue(ue2.getPath().equals("/file3"));
-      Assert.assertTrue(ue2.getTimestamp() > 0);
+      assertTrue(ue2.getPath().equals("/file3"));
+      assertTrue(ue2.getTimestamp() > 0);
       LOG.info(ue2.toString());
-      Assert.assertTrue(ue2.toString().startsWith("UnlinkEvent [path="));
-      Assert.assertTrue(batch.getEvents()[2].getEventType() == Event.EventType.CLOSE);
+      assertTrue(ue2.toString().startsWith("UnlinkEvent [path="));
+      assertTrue(batch.getEvents()[2].getEventType() == Event.EventType.CLOSE);
       Event.CloseEvent ce3 = (Event.CloseEvent) batch.getEvents()[2];
-      Assert.assertTrue(ce3.getPath().equals("/file2"));
-      Assert.assertTrue(ce3.getTimestamp() > 0);
+      assertTrue(ce3.getPath().equals("/file2"));
+      assertTrue(ce3.getTimestamp() > 0);
 
       // DeleteOp
       batch = waitForNextEvents(eis);
-      Assert.assertEquals(1, batch.getEvents().length);
+      assertEquals(1, batch.getEvents().length);
       txid = checkTxid(batch, txid);
-      Assert.assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.UNLINK);
+      assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.UNLINK);
       Event.UnlinkEvent ue = (Event.UnlinkEvent) batch.getEvents()[0];
-      Assert.assertTrue(ue.getPath().equals("/file2"));
-      Assert.assertTrue(ue.getTimestamp() > 0);
+      assertTrue(ue.getPath().equals("/file2"));
+      assertTrue(ue.getTimestamp() > 0);
       LOG.info(ue.toString());
 
       // MkdirOp
       batch = waitForNextEvents(eis);
-      Assert.assertEquals(1, batch.getEvents().length);
+      assertEquals(1, batch.getEvents().length);
       txid = checkTxid(batch, txid);
-      Assert.assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.CREATE);
+      assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.CREATE);
       Event.CreateEvent ce4 = (Event.CreateEvent) batch.getEvents()[0];
-      Assert.assertTrue(ce4.getiNodeType() ==
+      assertTrue(ce4.getiNodeType() ==
           Event.CreateEvent.INodeType.DIRECTORY);
-      Assert.assertTrue(ce4.getPath().equals("/dir"));
-      Assert.assertTrue(ce4.getCtime() > 0);
-      Assert.assertTrue(ce4.getReplication() == 0);
-      Assert.assertTrue(ce4.getSymlinkTarget() == null);
+      assertTrue(ce4.getPath().equals("/dir"));
+      assertTrue(ce4.getCtime() > 0);
+      assertTrue(ce4.getReplication() == 0);
+      assertTrue(ce4.getSymlinkTarget() == null);
       LOG.info(ce4.toString());
 
       // SetPermissionsOp
       batch = waitForNextEvents(eis);
-      Assert.assertEquals(1, batch.getEvents().length);
+      assertEquals(1, batch.getEvents().length);
       txid = checkTxid(batch, txid);
-      Assert.assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.METADATA);
+      assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.METADATA);
       Event.MetadataUpdateEvent mue3 = (Event.MetadataUpdateEvent) batch.getEvents()[0];
-      Assert.assertTrue(mue3.getPath().equals("/dir"));
-      Assert.assertTrue(mue3.getMetadataType() ==
+      assertTrue(mue3.getPath().equals("/dir"));
+      assertTrue(mue3.getMetadataType() ==
           Event.MetadataUpdateEvent.MetadataType.PERMS);
-      Assert.assertTrue(mue3.getPerms().toString().contains("rw-rw-rw-"));
+      assertTrue(mue3.getPerms().toString().contains("rw-rw-rw-"));
       LOG.info(mue3.toString());
 
       // SetOwnerOp
       batch = waitForNextEvents(eis);
-      Assert.assertEquals(1, batch.getEvents().length);
+      assertEquals(1, batch.getEvents().length);
       txid = checkTxid(batch, txid);
-      Assert.assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.METADATA);
+      assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.METADATA);
       Event.MetadataUpdateEvent mue4 = (Event.MetadataUpdateEvent) batch.getEvents()[0];
-      Assert.assertTrue(mue4.getPath().equals("/dir"));
-      Assert.assertTrue(mue4.getMetadataType() ==
+      assertTrue(mue4.getPath().equals("/dir"));
+      assertTrue(mue4.getMetadataType() ==
           Event.MetadataUpdateEvent.MetadataType.OWNER);
-      Assert.assertTrue(mue4.getOwnerName().equals("username"));
-      Assert.assertTrue(mue4.getGroupName().equals("groupname"));
+      assertTrue(mue4.getOwnerName().equals("username"));
+      assertTrue(mue4.getGroupName().equals("groupname"));
       LOG.info(mue4.toString());
 
       // SymlinkOp
       batch = waitForNextEvents(eis);
-      Assert.assertEquals(1, batch.getEvents().length);
+      assertEquals(1, batch.getEvents().length);
       txid = checkTxid(batch, txid);
-      Assert.assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.CREATE);
+      assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.CREATE);
       Event.CreateEvent ce5 = (Event.CreateEvent) batch.getEvents()[0];
-      Assert.assertTrue(ce5.getiNodeType() ==
+      assertTrue(ce5.getiNodeType() ==
           Event.CreateEvent.INodeType.SYMLINK);
-      Assert.assertTrue(ce5.getPath().equals("/dir2"));
-      Assert.assertTrue(ce5.getCtime() > 0);
-      Assert.assertTrue(ce5.getReplication() == 0);
-      Assert.assertTrue(ce5.getSymlinkTarget().equals("/dir"));
+      assertTrue(ce5.getPath().equals("/dir2"));
+      assertTrue(ce5.getCtime() > 0);
+      assertTrue(ce5.getReplication() == 0);
+      assertTrue(ce5.getSymlinkTarget().equals("/dir"));
       LOG.info(ce5.toString());
 
       // SetXAttrOp
       batch = waitForNextEvents(eis);
-      Assert.assertEquals(1, batch.getEvents().length);
+      assertEquals(1, batch.getEvents().length);
       txid = checkTxid(batch, txid);
-      Assert.assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.METADATA);
+      assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.METADATA);
       Event.MetadataUpdateEvent mue5 = (Event.MetadataUpdateEvent) batch.getEvents()[0];
-      Assert.assertTrue(mue5.getPath().equals("/file5"));
-      Assert.assertTrue(mue5.getMetadataType() ==
+      assertTrue(mue5.getPath().equals("/file5"));
+      assertTrue(mue5.getMetadataType() ==
           Event.MetadataUpdateEvent.MetadataType.XATTRS);
-      Assert.assertTrue(mue5.getxAttrs().size() == 1);
-      Assert.assertTrue(mue5.getxAttrs().get(0).getName().contains("field"));
-      Assert.assertTrue(!mue5.isxAttrsRemoved());
+      assertTrue(mue5.getxAttrs().size() == 1);
+      assertTrue(mue5.getxAttrs().get(0).getName().contains("field"));
+      assertTrue(!mue5.isxAttrsRemoved());
       LOG.info(mue5.toString());
 
       // RemoveXAttrOp
       batch = waitForNextEvents(eis);
-      Assert.assertEquals(1, batch.getEvents().length);
+      assertEquals(1, batch.getEvents().length);
       txid = checkTxid(batch, txid);
-      Assert.assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.METADATA);
+      assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.METADATA);
       Event.MetadataUpdateEvent mue6 = (Event.MetadataUpdateEvent) batch.getEvents()[0];
-      Assert.assertTrue(mue6.getPath().equals("/file5"));
-      Assert.assertTrue(mue6.getMetadataType() ==
+      assertTrue(mue6.getPath().equals("/file5"));
+      assertTrue(mue6.getMetadataType() ==
           Event.MetadataUpdateEvent.MetadataType.XATTRS);
-      Assert.assertTrue(mue6.getxAttrs().size() == 1);
-      Assert.assertTrue(mue6.getxAttrs().get(0).getName().contains("field"));
-      Assert.assertTrue(mue6.isxAttrsRemoved());
+      assertTrue(mue6.getxAttrs().size() == 1);
+      assertTrue(mue6.getxAttrs().get(0).getName().contains("field"));
+      assertTrue(mue6.isxAttrsRemoved());
       LOG.info(mue6.toString());
 
       // SetAclOp (1)
       batch = waitForNextEvents(eis);
-      Assert.assertEquals(1, batch.getEvents().length);
+      assertEquals(1, batch.getEvents().length);
       txid = checkTxid(batch, txid);
-      Assert.assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.METADATA);
+      assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.METADATA);
       Event.MetadataUpdateEvent mue7 = (Event.MetadataUpdateEvent) batch.getEvents()[0];
-      Assert.assertTrue(mue7.getPath().equals("/file5"));
-      Assert.assertTrue(mue7.getMetadataType() ==
+      assertTrue(mue7.getPath().equals("/file5"));
+      assertTrue(mue7.getMetadataType() ==
           Event.MetadataUpdateEvent.MetadataType.ACLS);
-      Assert.assertTrue(mue7.getAcls().contains(
+      assertTrue(mue7.getAcls().contains(
           AclEntry.parseAclEntry("user::rwx", true)));
       LOG.info(mue7.toString());
 
       // SetAclOp (2)
       batch = waitForNextEvents(eis);
-      Assert.assertEquals(1, batch.getEvents().length);
+      assertEquals(1, batch.getEvents().length);
       txid = checkTxid(batch, txid);
-      Assert.assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.METADATA);
+      assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.METADATA);
       Event.MetadataUpdateEvent mue8 = (Event.MetadataUpdateEvent) batch.getEvents()[0];
-      Assert.assertTrue(mue8.getPath().equals("/file5"));
-      Assert.assertTrue(mue8.getMetadataType() ==
+      assertTrue(mue8.getPath().equals("/file5"));
+      assertTrue(mue8.getMetadataType() ==
           Event.MetadataUpdateEvent.MetadataType.ACLS);
-      Assert.assertTrue(mue8.getAcls() == null);
+      assertTrue(mue8.getAcls() == null);
       LOG.info(mue8.toString());
 
       // RenameOp (2)
       batch = waitForNextEvents(eis);
-      Assert.assertEquals(1, batch.getEvents().length);
+      assertEquals(1, batch.getEvents().length);
       txid = checkTxid(batch, txid);
-      Assert.assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.RENAME);
+      assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.RENAME);
       Event.RenameEvent re3 = (Event.RenameEvent) batch.getEvents()[0];
-      Assert.assertTrue(re3.getDstPath().equals("/dir/file5"));
-      Assert.assertTrue(re3.getSrcPath().equals("/file5"));
-      Assert.assertTrue(re3.getTimestamp() > 0);
+      assertTrue(re3.getDstPath().equals("/dir/file5"));
+      assertTrue(re3.getSrcPath().equals("/file5"));
+      assertTrue(re3.getTimestamp() > 0);
       LOG.info(re3.toString());
 
       // TruncateOp
       batch = waitForNextEvents(eis);
-      Assert.assertEquals(1, batch.getEvents().length);
+      assertEquals(1, batch.getEvents().length);
       txid = checkTxid(batch, txid);
-      Assert
-          .assertTrue(batch.getEvents()[0].getEventType() ==
+      assertTrue(batch.getEvents()[0].getEventType() ==
           Event.EventType.TRUNCATE);
       Event.TruncateEvent et = ((Event.TruncateEvent) batch.getEvents()[0]);
-      Assert.assertTrue(et.getPath().equals("/truncate_file"));
-      Assert.assertTrue(et.getFileSize() == BLOCK_SIZE);
-      Assert.assertTrue(et.getTimestamp() > 0);
+      assertTrue(et.getPath().equals("/truncate_file"));
+      assertTrue(et.getFileSize() == BLOCK_SIZE);
+      assertTrue(et.getTimestamp() > 0);
       LOG.info(et.toString());
-      Assert.assertTrue(et.toString().startsWith("TruncateEvent [path="));
+      assertTrue(et.toString().startsWith("TruncateEvent [path="));
 
       // CreateEvent without overwrite
       batch = waitForNextEvents(eis);
-      Assert.assertEquals(1, batch.getEvents().length);
+      assertEquals(1, batch.getEvents().length);
       txid = checkTxid(batch, txid);
-      Assert.assertTrue(batch.getEvents()[0].getEventType()
+      assertTrue(batch.getEvents()[0].getEventType()
               == Event.EventType.CREATE);
       ce = (Event.CreateEvent) batch.getEvents()[0];
-      Assert.assertTrue(ce.getiNodeType() == Event.CreateEvent.INodeType.FILE);
-      Assert.assertTrue(ce.getPath().equals("/file_ec_test1"));
-      Assert.assertTrue(ce.getCtime() > 0);
-      Assert.assertTrue(ce.getReplication() > 0);
-      Assert.assertTrue(ce.getSymlinkTarget() == null);
-      Assert.assertFalse(ce.getOverwrite());
-      Assert.assertEquals(BLOCK_SIZE, ce.getDefaultBlockSize());
-      Assert.assertTrue(ce.isErasureCoded().isPresent());
-      Assert.assertFalse(ce.isErasureCoded().get());
+      assertTrue(ce.getiNodeType() == Event.CreateEvent.INodeType.FILE);
+      assertTrue(ce.getPath().equals("/file_ec_test1"));
+      assertTrue(ce.getCtime() > 0);
+      assertTrue(ce.getReplication() > 0);
+      assertTrue(ce.getSymlinkTarget() == null);
+      assertFalse(ce.getOverwrite());
+      assertEquals(BLOCK_SIZE, ce.getDefaultBlockSize());
+      assertTrue(ce.isErasureCoded().isPresent());
+      assertFalse(ce.isErasureCoded().get());
       LOG.info(ce.toString());
-      Assert.assertTrue(ce.toString().startsWith("CreateEvent [INodeType="));
+      assertTrue(ce.toString().startsWith("CreateEvent [INodeType="));
 
       // Returns null when there are no further events
-      Assert.assertTrue(eis.poll() == null);
+      assertTrue(eis.poll() == null);
 
       // make sure the estimate hasn't changed since the above assertion
       // tells us that we are fully caught up to the current namesystem state
       // and we should not have been behind at all when eventsBehind was set
       // either, since there were few enough events that they should have all
       // been read to the client during the first poll() call
-      Assert.assertTrue(eis.getTxidsBehindEstimate() == eventsBehind);
+      assertTrue(eis.getTxidsBehindEstimate() == eventsBehind);
 
     } finally {
       cluster.shutdown();
     }
   }
 
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testErasureCodedFiles() throws Exception {
     ErasureCodingPolicy ecPolicy = StripedFileTestUtil.getDefaultECPolicy();
     final int dataUnits = ecPolicy.getNumDataUnits();
@@ -470,47 +476,48 @@ public class TestDFSInotifyEventInputStream {
       EventBatch batch = null;
 
       batch = waitForNextEvents(eis);
-      Assert.assertEquals(1, batch.getEvents().length);
+      assertEquals(1, batch.getEvents().length);
       long txid = batch.getTxid();
       long eventsBehind = eis.getTxidsBehindEstimate();
-      Assert.assertTrue(batch.getEvents()[0].getEventType()
+      assertTrue(batch.getEvents()[0].getEventType()
               == Event.EventType.CREATE);
       Event.CreateEvent ce = (Event.CreateEvent) batch.getEvents()[0];
-      Assert.assertTrue(ce.getiNodeType() == Event.CreateEvent.INodeType.FILE);
-      Assert.assertTrue(ce.getPath().equals("/ecdir/file_ec_test2"));
-      Assert.assertTrue(ce.getCtime() > 0);
-      Assert.assertEquals(1, ce.getReplication());
-      Assert.assertTrue(ce.getSymlinkTarget() == null);
-      Assert.assertTrue(ce.getOverwrite());
-      Assert.assertEquals(ecPolicy.getCellSize(), ce.getDefaultBlockSize());
-      Assert.assertTrue(ce.isErasureCoded().isPresent());
-      Assert.assertTrue(ce.isErasureCoded().get());
+      assertTrue(ce.getiNodeType() == Event.CreateEvent.INodeType.FILE);
+      assertTrue(ce.getPath().equals("/ecdir/file_ec_test2"));
+      assertTrue(ce.getCtime() > 0);
+      assertEquals(1, ce.getReplication());
+      assertTrue(ce.getSymlinkTarget() == null);
+      assertTrue(ce.getOverwrite());
+      assertEquals(ecPolicy.getCellSize(), ce.getDefaultBlockSize());
+      assertTrue(ce.isErasureCoded().isPresent());
+      assertTrue(ce.isErasureCoded().get());
       LOG.info(ce.toString());
-      Assert.assertTrue(ce.toString().startsWith("CreateEvent [INodeType="));
+      assertTrue(ce.toString().startsWith("CreateEvent [INodeType="));
 
       batch = waitForNextEvents(eis);
-      Assert.assertEquals(1, batch.getEvents().length);
+      assertEquals(1, batch.getEvents().length);
       txid = checkTxid(batch, txid);
-      Assert.assertTrue(batch.getEvents()[0].getEventType()
+      assertTrue(batch.getEvents()[0].getEventType()
               == Event.EventType.CLOSE);
-      Assert.assertTrue(((Event.CloseEvent) batch.getEvents()[0]).getPath()
+      assertTrue(((Event.CloseEvent) batch.getEvents()[0]).getPath()
               .equals("/ecdir/file_ec_test2"));
 
       // Returns null when there are no further events
-      Assert.assertTrue(eis.poll() == null);
+      assertTrue(eis.poll() == null);
 
       // make sure the estimate hasn't changed since the above assertion
       // tells us that we are fully caught up to the current namesystem state
       // and we should not have been behind at all when eventsBehind was set
       // either, since there were few enough events that they should have all
       // been read to the client during the first poll() call
-      Assert.assertTrue(eis.getTxidsBehindEstimate() == eventsBehind);
+      assertTrue(eis.getTxidsBehindEstimate() == eventsBehind);
     } finally {
       cluster.shutdown();
     }
   }
 
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testNNFailover() throws IOException, URISyntaxException,
       MissingEventsException {
     Configuration conf = new HdfsConfiguration();
@@ -532,18 +539,19 @@ public class TestDFSInotifyEventInputStream {
       // active
       for (int i = 0; i < 10; i++) {
         batch = waitForNextEvents(eis);
-        Assert.assertEquals(1, batch.getEvents().length);
-        Assert.assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.CREATE);
-        Assert.assertTrue(((Event.CreateEvent) batch.getEvents()[0]).getPath().equals("/dir" +
+        assertEquals(1, batch.getEvents().length);
+        assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.CREATE);
+        assertTrue(((Event.CreateEvent) batch.getEvents()[0]).getPath().equals("/dir" +
             i));
       }
-      Assert.assertTrue(eis.poll() == null);
+      assertTrue(eis.poll() == null);
     } finally {
       cluster.shutdown();
     }
   }
 
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testTwoActiveNNs() throws IOException, MissingEventsException {
     Configuration conf = new HdfsConfiguration();
     MiniQJMHACluster cluster = new MiniQJMHACluster.Builder(conf).build();
@@ -571,12 +579,12 @@ public class TestDFSInotifyEventInputStream {
       EventBatch batch = null;
       for (int i = 0; i < 10; i++) {
         batch = waitForNextEvents(eis);
-        Assert.assertEquals(1, batch.getEvents().length);
-        Assert.assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.CREATE);
-        Assert.assertTrue(((Event.CreateEvent) batch.getEvents()[0]).getPath().equals("/dir" +
+        assertEquals(1, batch.getEvents().length);
+        assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.CREATE);
+        assertTrue(((Event.CreateEvent) batch.getEvents()[0]).getPath().equals("/dir" +
             i));
       }
-      Assert.assertTrue(eis.poll() == null);
+      assertTrue(eis.poll() == null);
     } finally {
       try {
         cluster.shutdown();
@@ -587,7 +595,8 @@ public class TestDFSInotifyEventInputStream {
     }
   }
 
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testReadEventsWithTimeout() throws IOException,
       InterruptedException, MissingEventsException {
     Configuration conf = new HdfsConfiguration();
@@ -615,10 +624,10 @@ public class TestDFSInotifyEventInputStream {
       // a very generous wait period -- the edit will definitely have been
       // processed by the time this is up
       EventBatch batch = eis.poll(5, TimeUnit.SECONDS);
-      Assert.assertNotNull(batch);
-      Assert.assertEquals(1, batch.getEvents().length);
-      Assert.assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.CREATE);
-      Assert.assertEquals("/dir", ((Event.CreateEvent) batch.getEvents()[0]).getPath());
+      assertNotNull(batch);
+      assertEquals(1, batch.getEvents().length);
+      assertTrue(batch.getEvents()[0].getEventType() == Event.EventType.CREATE);
+      assertEquals("/dir", ((Event.CreateEvent) batch.getEvents()[0]).getPath());
     } finally {
       cluster.shutdown();
     }

+ 16 - 19
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStreamKerberized.java

@@ -33,11 +33,9 @@ import org.apache.hadoop.security.AuthenticationFilterInitializer;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -64,14 +62,16 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIP
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import org.junit.jupiter.api.Timeout;
 
 /**
  * Class for Kerberized test cases for {@link DFSInotifyEventInputStream}.
  */
+@Timeout(180)
 public class TestDFSInotifyEventInputStreamKerberized {
 
   private static final Logger LOG =
@@ -89,9 +89,6 @@ public class TestDFSInotifyEventInputStreamKerberized {
   private File generalHDFSKeytabFile;
   private File nnKeytabFile;
 
-  @Rule
-  public Timeout timeout = new Timeout(180000);
-
   @Test
   public void testWithKerberizedCluster() throws Exception {
     conf = new HdfsConfiguration(baseConf);
@@ -131,7 +128,7 @@ public class TestDFSInotifyEventInputStreamKerberized {
           while ((batch = eis.poll()) != null) {
             LOG.info("txid: " + batch.getTxid());
           }
-          assertNull("poll should not return anything", eis.poll());
+          assertNull(eis.poll(), "poll should not return anything");
 
           Thread.sleep(6000);
           LOG.info("Slept 6 seconds to make sure the TGT has expired.");
@@ -143,16 +140,16 @@ public class TestDFSInotifyEventInputStreamKerberized {
 
           // verify we can poll after a tgt expiration interval
           batch = eis.poll();
-          assertNotNull("poll should return something", batch);
+          assertNotNull(batch, "poll should return something");
           assertEquals(1, batch.getEvents().length);
-          assertNull("poll should not return anything", eis.poll());
+          assertNull(eis.poll(), "poll should not return anything");
           return null;
         }
       }
     });
   }
 
-  @Before
+  @BeforeEach
   public void initKerberizedCluster() throws Exception {
     baseDir = new File(System.getProperty("test.build.dir", "target/test-dir"),
         TestDFSInotifyEventInputStreamKerberized.class.getSimpleName());
@@ -169,8 +166,8 @@ public class TestDFSInotifyEventInputStreamKerberized {
     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");
 
     final String userName = "hdfs";
     nnKeytabFile = new File(baseDir, userName + ".keytab");
@@ -218,7 +215,7 @@ public class TestDFSInotifyEventInputStreamKerberized {
         KeyStoreTestUtil.getServerSSLConfigFileName());
   }
 
-  @After
+  @AfterEach
   public void shutdownCluster() throws Exception {
     if (cluster != null) {
       cluster.shutdown();

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

@@ -18,11 +18,11 @@
 package org.apache.hadoop.hdfs;
 
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_READ_USE_CACHE_PRIORITY;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assumptions.assumeTrue;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -53,8 +53,8 @@ import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Retry;
 
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.log4j.Level;
-import org.junit.Assume;
-import org.junit.Test;
+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;
@@ -93,7 +93,8 @@ public class TestDFSInputStream {
     }
   }
 
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testSkipWithRemoteBlockReader() throws IOException {
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
@@ -104,7 +105,8 @@ public class TestDFSInputStream {
     }
   }
 
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testSkipWithRemoteBlockReader2() throws IOException {
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
@@ -115,9 +117,10 @@ public class TestDFSInputStream {
     }
   }
 
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testSkipWithLocalBlockReader() throws IOException {
-    Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
+    assumeTrue(DomainSocket.getLoadingFailureReason() == null);
     TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
     DomainSocket.disableBindPathValidation();
     Configuration conf = new Configuration();
@@ -136,7 +139,8 @@ public class TestDFSInputStream {
     }
   }
 
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testSeekToNewSource() throws IOException {
     Configuration conf = new Configuration();
     MiniDFSCluster cluster =
@@ -159,7 +163,8 @@ public class TestDFSInputStream {
     }
   }
 
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testOpenInfo() throws IOException {
     Configuration conf = new Configuration();
     conf.setInt(Retry.TIMES_GET_LAST_BLOCK_LENGTH_KEY, 0);
@@ -227,10 +232,10 @@ public class TestDFSInputStream {
       final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
       cluster.getNameNode().getNamesystem().getBlockManager()
           .getDatanodeManager().fetchDatanodes(live, null, false);
-      assertTrue("DN start should be success and live dn should be 2",
-          live.size() == 2);
-      assertTrue("File size should be " + chunkSize,
-          fs.getFileStatus(file).getLen() == chunkSize);
+      assertTrue(live.size() == 2,
+          "DN start should be success and live dn should be 2");
+      assertTrue(fs.getFileStatus(file).getLen() == chunkSize,
+          "File size should be " + chunkSize);
     } finally {
       cluster.shutdown();
     }

+ 59 - 52
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInputStreamBlockLocations.java

@@ -19,12 +19,12 @@
 package org.apache.hadoop.hdfs;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotSame;
-import static org.junit.Assert.assertSame;
-import static org.junit.Assert.assertTrue;
-
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
@@ -42,17 +42,13 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.util.Time;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 
 /**
  * Test the caches expiration of the block locations.
  */
-@RunWith(Parameterized.class)
 public class TestDFSInputStreamBlockLocations {
   private static final int BLOCK_SIZE = 1024 * 1024;
   private static final String[] RACKS = new String[] {
@@ -73,7 +69,6 @@ public class TestDFSInputStreamBlockLocations {
   private Path filePath;
   private boolean enableBlkExpiration;
 
-  @Parameterized.Parameters(name = "{index}: CacheExpirationConfig(Enable {0})")
   public static Collection<Object[]> getTestParameters() {
     return Arrays.asList(new Object[][] {
         {Boolean.TRUE},
@@ -81,11 +76,12 @@ public class TestDFSInputStreamBlockLocations {
     });
   }
 
-  public TestDFSInputStreamBlockLocations(Boolean enableExpiration) {
-    enableBlkExpiration = enableExpiration;
+  public void initTestDFSInputStreamBlockLocations(Boolean pEnableExpiration)
+      throws IOException {
+    enableBlkExpiration = pEnableExpiration;
+    setup();
   }
 
-  @Before
   public void setup() throws IOException {
     conf = new HdfsConfiguration();
     conf.setBoolean(
@@ -120,7 +116,7 @@ public class TestDFSInputStreamBlockLocations {
     fs = dfsCluster.getFileSystem();
   }
 
-  @After
+  @AfterEach
   public void teardown() throws IOException {
     if (dfsClient != null) {
       dfsClient.close();
@@ -137,8 +133,10 @@ public class TestDFSInputStreamBlockLocations {
     }
   }
 
-  @Test
-  public void testRefreshBlockLocations() throws IOException {
+  @MethodSource("getTestParameters")
+  @ParameterizedTest
+  public void testRefreshBlockLocations(Boolean pEnableExpiration) throws IOException {
+    initTestDFSInputStreamBlockLocations(pEnableExpiration);
     final String fileName = "/test_cache_locations";
     filePath = createFile(fileName);
 
@@ -146,19 +144,19 @@ public class TestDFSInputStreamBlockLocations {
       LocatedBlocks existing = fin.locatedBlocks;
       long lastRefreshedAt = fin.getLastRefreshedBlocksAtForTesting();
 
-      assertFalse("should not have attempted refresh",
-          fin.refreshBlockLocations(null));
-      assertEquals("should not have updated lastRefreshedAt",
-          lastRefreshedAt, fin.getLastRefreshedBlocksAtForTesting());
-      assertSame("should not have modified locatedBlocks",
-          existing, fin.locatedBlocks);
+      assertFalse(fin.refreshBlockLocations(null),
+          "should not have attempted refresh");
+      assertEquals(lastRefreshedAt, fin.getLastRefreshedBlocksAtForTesting(),
+          "should not have updated lastRefreshedAt");
+      assertSame(existing, fin.locatedBlocks,
+          "should not have modified locatedBlocks");
 
       // fake a dead node to force refresh
       // refreshBlockLocations should return true, indicating we attempted a refresh
       // nothing should be changed, because locations have not changed
       fin.addToLocalDeadNodes(dfsClient.datanodeReport(DatanodeReportType.LIVE)[0]);
-      assertTrue("should have attempted refresh",
-          fin.refreshBlockLocations(null));
+      assertTrue(fin.refreshBlockLocations(null),
+          "should have attempted refresh");
       verifyChanged(fin, existing, lastRefreshedAt);
 
       // reset
@@ -174,33 +172,39 @@ public class TestDFSInputStreamBlockLocations {
         mockAddressCache.put(dataNode.getDatanodeUuid(), unresolved);
       }
 
-      assertTrue("should have attempted refresh",
-          fin.refreshBlockLocations(mockAddressCache));
+      assertTrue(fin.refreshBlockLocations(mockAddressCache),
+          "should have attempted refresh");
       verifyChanged(fin, existing, lastRefreshedAt);
     }
   }
 
   private void verifyChanged(DFSInputStream fin, LocatedBlocks existing, long lastRefreshedAt) {
-    assertTrue("lastRefreshedAt should have incremented",
-        fin.getLastRefreshedBlocksAtForTesting() > lastRefreshedAt);
-    assertNotSame("located blocks should have changed",
-        existing, fin.locatedBlocks);
-    assertTrue("deadNodes should be empty",
-        fin.getLocalDeadNodes().isEmpty());
+    assertTrue(fin.getLastRefreshedBlocksAtForTesting() > lastRefreshedAt,
+        "lastRefreshedAt should have incremented");
+    assertNotSame(existing, fin.locatedBlocks,
+        "located blocks should have changed");
+    assertTrue(fin.getLocalDeadNodes().isEmpty(),
+        "deadNodes should be empty");
   }
 
-  @Test
-  public void testDeferredRegistrationStatefulRead() throws IOException {
+  @MethodSource("getTestParameters")
+  @ParameterizedTest
+  public void testDeferredRegistrationStatefulRead(Boolean pEnableExpiration) throws IOException {
+    initTestDFSInputStreamBlockLocations(pEnableExpiration);
     testWithRegistrationMethod(DFSInputStream::read);
   }
 
-  @Test
-  public void testDeferredRegistrationPositionalRead() throws IOException {
+  @MethodSource("getTestParameters")
+  @ParameterizedTest
+  public void testDeferredRegistrationPositionalRead(Boolean pEnableExpiration) throws IOException {
+    initTestDFSInputStreamBlockLocations(pEnableExpiration);
     testWithRegistrationMethod(fin -> fin.readFully(0, new byte[1]));
   }
 
-  @Test
-  public void testDeferredRegistrationGetAllBlocks() throws IOException {
+  @MethodSource("getTestParameters")
+  @ParameterizedTest
+  public void testDeferredRegistrationGetAllBlocks(Boolean pEnableExpiration) throws IOException {
+    initTestDFSInputStreamBlockLocations(pEnableExpiration);
     testWithRegistrationMethod(DFSInputStream::getAllBlocks);
   }
 
@@ -209,8 +213,10 @@ public class TestDFSInputStreamBlockLocations {
    * of retries built into chooseDataNode. This is needed for hedged reads
    * @throws IOException
    */
-  @Test
-  public void testClearIgnoreListChooseDataNode() throws IOException {
+  @MethodSource("getTestParameters")
+  @ParameterizedTest
+  public void testClearIgnoreListChooseDataNode(Boolean pEnableExpiration) throws IOException {
+    initTestDFSInputStreamBlockLocations(pEnableExpiration);
     final String fileName = "/test_cache_locations";
     filePath = createFile(fileName);
 
@@ -218,8 +224,8 @@ public class TestDFSInputStreamBlockLocations {
       LocatedBlocks existing = fin.locatedBlocks;
       LocatedBlock block = existing.getLastLocatedBlock();
       ArrayList<DatanodeInfo> ignoreList = new ArrayList<>(Arrays.asList(block.getLocations()));
-      Assert.assertNotNull(fin.chooseDataNode(block, ignoreList, true));
-      Assert.assertEquals(0, ignoreList.size());
+      assertNotNull(fin.chooseDataNode(block, ignoreList, true));
+      assertEquals(0, ignoreList.size());
     }
   }
 
@@ -235,19 +241,20 @@ public class TestDFSInputStreamBlockLocations {
     DFSInputStream fin = null;
     try {
       fin = dfsClient.open(fileName);
-      assertFalse("should not be tracking input stream on open",
-          dfsClient.getLocatedBlockRefresher().isInputStreamTracked(fin));
+      assertFalse(dfsClient.getLocatedBlockRefresher().isInputStreamTracked(fin),
+          "should not be tracking input stream on open");
 
       // still not registered because it hasn't been an hour by the time we call this
       registrationMethod.accept(fin);
-      assertFalse("should not be tracking input stream after first read",
-          dfsClient.getLocatedBlockRefresher().isInputStreamTracked(fin));
+      assertFalse(dfsClient.getLocatedBlockRefresher().isInputStreamTracked(fin),
+          "should not be tracking input stream after first read");
 
       // artificially make it have been an hour
       fin.setLastRefreshedBlocksAtForTesting(Time.monotonicNow() - (dfsInputLocationsTimeout + 1));
       registrationMethod.accept(fin);
-      assertEquals("SHOULD be tracking input stream on read after interval, only if enabled",
-          enableBlkExpiration, dfsClient.getLocatedBlockRefresher().isInputStreamTracked(fin));
+      assertEquals(enableBlkExpiration,
+          dfsClient.getLocatedBlockRefresher().isInputStreamTracked(fin),
+          "SHOULD be tracking input stream on read after interval, only if enabled");
     } finally {
       if (fin != null) {
         fin.close();

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

@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.*;
-
 import java.io.FileNotFoundException;
 import java.io.IOException;
 
@@ -30,7 +28,11 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.util.Time;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 /**
  * This class tests that the DFS command mkdirs only creates valid
@@ -106,10 +108,10 @@ public class TestDFSMkdirs {
       } catch (IOException e) {
         expectedException = e;
       }
-      assertTrue("Create a directory when parent dir exists as file using"
-          + " mkdir() should throw ParentNotDirectoryException ",
-          expectedException != null
-              && expectedException instanceof ParentNotDirectoryException);
+      assertTrue(expectedException != null
+              && expectedException instanceof ParentNotDirectoryException,
+          "Create a directory when parent dir exists as file using"
+              + " mkdir() should throw ParentNotDirectoryException ");
       // Create a dir in a non-exist directory, should fail
       expectedException = null;
       try {
@@ -118,10 +120,10 @@ public class TestDFSMkdirs {
       } catch (IOException e) {
         expectedException = e;
       }
-      assertTrue("Create a directory in a non-exist parent dir using"
-          + " mkdir() should throw FileNotFoundException ",
-          expectedException != null
-              && expectedException instanceof FileNotFoundException);
+      assertTrue(expectedException != null
+              && expectedException instanceof FileNotFoundException,
+          "Create a directory in a non-exist parent dir using"
+              + " mkdir() should throw FileNotFoundException ");
     } finally {
       dfs.close();
       cluster.shutdown();

+ 31 - 27
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java

@@ -60,15 +60,17 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.test.Whitebox;
 import org.apache.hadoop.util.DataChecksum;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Write.RECOVER_LEASE_ON_CLOSE_EXCEPTION_KEY;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.ArgumentMatchers.anyBoolean;
 import org.mockito.Mockito;
 
@@ -76,7 +78,6 @@ import static org.mockito.ArgumentMatchers.anyString;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 
-import static org.junit.Assert.assertEquals;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.doThrow;
@@ -89,7 +90,7 @@ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_WRIT
 public class TestDFSOutputStream {
   static MiniDFSCluster cluster;
 
-  @BeforeClass
+  @BeforeAll
   public static void setup() throws IOException {
     Configuration conf = new Configuration();
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
@@ -111,7 +112,7 @@ public class TestDFSOutputStream {
     LastExceptionInStreamer ex = (LastExceptionInStreamer) Whitebox
         .getInternalState(streamer, "lastException");
     Throwable thrown = (Throwable) Whitebox.getInternalState(ex, "thrown");
-    Assert.assertNull(thrown);
+    assertNull(thrown);
 
     dos.close();
 
@@ -123,7 +124,7 @@ public class TestDFSOutputStream {
       assertEquals(e, dummy);
     }
     thrown = (Throwable) Whitebox.getInternalState(ex, "thrown");
-    Assert.assertNull(thrown);
+    assertNull(thrown);
     dos.close();
   }
 
@@ -149,10 +150,10 @@ public class TestDFSOutputStream {
     Field field = dos.getClass().getDeclaredField("packetSize");
     field.setAccessible(true);
 
-    Assert.assertTrue((Integer) field.get(dos) + 33 < packetSize);
+    assertTrue((Integer) field.get(dos) + 33 < packetSize);
     // If PKT_MAX_HEADER_LEN is 257, actual packet size come to over 64KB
     // without a fix on HDFS-7308.
-    Assert.assertTrue((Integer) field.get(dos) + 257 < packetSize);
+    assertTrue((Integer) field.get(dos) + 257 < packetSize);
   }
 
   /**
@@ -168,7 +169,8 @@ public class TestDFSOutputStream {
    * @throws IllegalAccessException
    * @throws NoSuchMethodException
    */
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testPreventOverflow() throws IOException, NoSuchFieldException,
       SecurityException, IllegalAccessException, IllegalArgumentException,
       InvocationTargetException, NoSuchMethodException {
@@ -250,21 +252,21 @@ public class TestDFSOutputStream {
       final Field writePacketSizeField = dos.getClass()
           .getDeclaredField("writePacketSize");
       writePacketSizeField.setAccessible(true);
-      Assert.assertEquals(writePacketSizeField.getInt(dos),
+      assertEquals(writePacketSizeField.getInt(dos),
           finalWritePacketSize);
 
       /* get and verify chunksPerPacket */
       final Field chunksPerPacketField = dos.getClass()
           .getDeclaredField("chunksPerPacket");
       chunksPerPacketField.setAccessible(true);
-      Assert.assertEquals(chunksPerPacketField.getInt(dos),
+      assertEquals(chunksPerPacketField.getInt(dos),
           (finalWritePacketSize - packateMaxHeaderLength) / chunkSize);
 
       /* get and verify packetSize */
       final Field packetSizeField = dos.getClass()
           .getDeclaredField("packetSize");
       packetSizeField.setAccessible(true);
-      Assert.assertEquals(packetSizeField.getInt(dos),
+      assertEquals(packetSizeField.getInt(dos),
           chunksPerPacketField.getInt(dos) * chunkSize);
     } finally {
       if (dfsCluster != null) {
@@ -303,10 +305,11 @@ public class TestDFSOutputStream {
     DFSPacket packet = mock(DFSPacket.class);
     dataQueue.add(packet);
     stream.run();
-    Assert.assertTrue(congestedNodes.isEmpty());
+    assertTrue(congestedNodes.isEmpty());
   }
 
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testCongestionAckDelay() {
     DfsClientConf dfsClientConf = mock(DfsClientConf.class);
     DFSClient client = mock(DFSClient.class);
@@ -385,7 +388,7 @@ public class TestDFSOutputStream {
       }
     }).start();
     stream.run();
-    Assert.assertFalse(isDelay.get());
+    assertFalse(isDelay.get());
   }
 
   @Test
@@ -448,10 +451,10 @@ public class TestDFSOutputStream {
     FileSystem fs = cluster.getFileSystem();
     FSDataOutputStream os = fs.create(new Path("/normal-file"));
     // Verify output stream supports hsync() and hflush().
-    assertTrue("DFSOutputStream should support hflush()!",
-        os.hasCapability(StreamCapability.HFLUSH.getValue()));
-    assertTrue("DFSOutputStream should support hsync()!",
-        os.hasCapability(StreamCapability.HSYNC.getValue()));
+    assertTrue(os.hasCapability(StreamCapability.HFLUSH.getValue()),
+        "DFSOutputStream should support hflush()!");
+    assertTrue(os.hasCapability(StreamCapability.HSYNC.getValue()),
+        "DFSOutputStream should support hsync()!");
     byte[] bytes = new byte[1024];
     InputStream is = new ByteArrayInputStream(bytes);
     IOUtils.copyBytes(is, os, bytes.length);
@@ -510,7 +513,8 @@ public class TestDFSOutputStream {
     }
   }
 
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testFirstPacketSizeInNewBlocks() throws IOException {
     final long blockSize = (long) 1024 * 1024;
     MiniDFSCluster dfsCluster = cluster;
@@ -542,14 +546,14 @@ public class TestDFSOutputStream {
         fos.write(buf);
         fos.hflush();
         loop++;
-        Assert.assertEquals(((DFSOutputStream) fos.getWrappedStream()).packetSize,
+        assertEquals(((DFSOutputStream) fos.getWrappedStream()).packetSize,
             packetContentSize);
       }
     }
     fs.delete(new Path("/testfile.dat"), true);
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() {
     if (cluster != null) {
       cluster.shutdown();

+ 43 - 42
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java

@@ -17,10 +17,10 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.DataOutputStream;
 import java.io.FileNotFoundException;
@@ -46,9 +46,10 @@ import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.LambdaTestUtils;
 import org.apache.hadoop.util.Time;
-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;
 
 /** Unit tests for permission */
 public class TestDFSPermission {
@@ -116,13 +117,13 @@ public class TestDFSPermission {
     }
   }
 
-  @Before
+  @BeforeEach
   public void setUp() throws IOException {
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     cluster.waitActive();
   }
   
-  @After
+  @AfterEach
   public void tearDown() throws IOException {
     if (cluster != null) {
       cluster.shutdown();
@@ -186,7 +187,7 @@ public class TestDFSPermission {
     // case 5: test non-existent parent directory
     uMask = DEFAULT_UMASK;
     initFileSystem(uMask);
-    assertFalse("File shouldn't exists", fs.exists(NON_EXISTENT_PATH));
+    assertFalse(fs.exists(NON_EXISTENT_PATH), "File shouldn't exists");
     createAndCheckPermission(op, NON_EXISTENT_PATH, uMask, new FsPermission(
         DEFAULT_PERMISSION), false);
     Path parent = NON_EXISTENT_PATH.getParent();
@@ -302,7 +303,8 @@ public class TestDFSPermission {
         FsPermission.createImmutable((short)0777));
   }
   
-  @Test(timeout=30000)
+  @Test
+  @Timeout(value = 30)
   public void testTrashPermission() throws Exception {
     //  /BSS                  user1:group2 777
     //   /BSS/user1            user1:group2 755
@@ -336,8 +338,8 @@ public class TestDFSPermission {
         fail("User2 should not be allowed to delete user1's dir.");
       } catch (AccessControlException e) {
         e.printStackTrace();
-        assertTrue("Permission denied messages must carry the username",
-            e.getMessage().contains(USER2_NAME));
+        assertTrue(e.getMessage().contains(USER2_NAME),
+            "Permission denied messages must carry the username");
       }
 
       // ensure the /BSS/user1 still exists
@@ -369,8 +371,8 @@ public class TestDFSPermission {
         // expect the exception is caused by permission denied
         assertTrue(e.getCause() instanceof AccessControlException);
         e.printStackTrace();
-        assertTrue("Permission denied messages must carry the username",
-            e.getCause().getMessage().contains(USER2_NAME));
+        assertTrue(e.getCause().getMessage().contains(USER2_NAME),
+            "Permission denied messages must carry the username");
       }
 
       // ensure /BSS/user1 still exists
@@ -552,11 +554,11 @@ public class TestDFSPermission {
       fs.access(p1, FsAction.WRITE);
       fail("The access call should have failed.");
     } catch (AccessControlException e) {
-      assertTrue("Permission denied messages must carry the username",
-              e.getMessage().contains(USER1_NAME));
-      assertTrue("Permission denied messages must carry the path parent",
-              e.getMessage().contains(
-                  p1.getParent().toUri().getPath()));
+      assertTrue(e.getMessage().contains(USER1_NAME),
+          "Permission denied messages must carry the username");
+      assertTrue(e.getMessage().contains(
+              p1.getParent().toUri().getPath()),
+          "Permission denied messages must carry the path parent");
     }
 
     Path badPath = new Path("/bad/bad");
@@ -586,11 +588,11 @@ public class TestDFSPermission {
       fs.access(p2, FsAction.EXECUTE);
       fail("The access call should have failed.");
     } catch (AccessControlException e) {
-      assertTrue("Permission denied messages must carry the username",
-              e.getMessage().contains(USER1_NAME));
-      assertTrue("Permission denied messages must carry the path parent",
-              e.getMessage().contains(
-                  p2.getParent().toUri().getPath()));
+      assertTrue(e.getMessage().contains(USER1_NAME),
+          "Permission denied messages must carry the username");
+      assertTrue(e.getMessage().contains(
+              p2.getParent().toUri().getPath()),
+          "Permission denied messages must carry the path parent");
     }
   }
 
@@ -611,11 +613,11 @@ public class TestDFSPermission {
       fs.access(p3, FsAction.READ_WRITE);
       fail("The access call should have failed.");
     } catch (AccessControlException e) {
-      assertTrue("Permission denied messages must carry the username",
-              e.getMessage().contains(USER1_NAME));
-      assertTrue("Permission denied messages must carry the path parent",
-              e.getMessage().contains(
-                  p3.getParent().toUri().getPath()));
+      assertTrue(e.getMessage().contains(USER1_NAME),
+          "Permission denied messages must carry the username");
+      assertTrue(e.getMessage().contains(
+              p3.getParent().toUri().getPath()),
+          "Permission denied messages must carry the path parent");
     }
   }
 
@@ -648,11 +650,11 @@ public class TestDFSPermission {
       fs.exists(nfpath);
       fail("The exists call should have failed.");
     } catch (AccessControlException e) {
-      assertTrue("Permission denied messages must carry file path",
-          e.getMessage().contains(fpath.getName()));
-      assertTrue("Permission denied messages must specify existing_file is not "
-              + "a directory, when checked on /existing_file/non_existing_name",
-          e.getMessage().contains("is not a directory"));
+      assertTrue(e.getMessage().contains(fpath.getName()),
+          "Permission denied messages must carry file path");
+      assertTrue(e.getMessage().contains("is not a directory"),
+          "Permission denied messages must specify existing_file is not "
+              + "a directory, when checked on /existing_file/non_existing_name");
     }
 
     rootFs.setPermission(p4, new FsPermission("600"));
@@ -660,13 +662,12 @@ public class TestDFSPermission {
       fs.exists(nfpath);
       fail("The exists call should have failed.");
     } catch (AccessControlException e) {
-      assertFalse("Permission denied messages must not carry full file path,"
-              + "since the user does not have permission on /p4: "
-              + e.getMessage(),
-          e.getMessage().contains(fpath.getName()));
-      assertFalse("Permission denied messages must not specify /p4"
-          + " is not a directory: " + e.getMessage(),
-          e.getMessage().contains("is not a directory"));
+      assertFalse(e.getMessage().contains(fpath.getName()),
+          "Permission denied messages must not carry full file path,"
+              + "since the user does not have permission on /p4: " + e.getMessage());
+      assertFalse(e.getMessage().contains("is not a directory"),
+          "Permission denied messages must not specify /p4"
+              + " is not a directory: " + e.getMessage());
     }
   }
 

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

@@ -27,15 +27,16 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 public class TestDataStream {
   static MiniDFSCluster cluster;
   static int PACKET_SIZE = 1024;
 
-  @BeforeClass
+  @BeforeAll
   public static void setup() throws IOException {
     Configuration conf = new Configuration();
     conf.setInt(HdfsClientConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY,
@@ -46,7 +47,8 @@ public class TestDataStream {
     cluster = new MiniDFSCluster.Builder(conf).build();
   }
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testDfsClient() throws IOException, InterruptedException {
     LogCapturer logs = GenericTestUtils.LogCapturer.captureLogs(LoggerFactory
         .getLogger(DataStreamer.class));
@@ -77,7 +79,7 @@ public class TestDataStream {
         "Slow ReadProcessor read fields for block");
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() {
     cluster.shutdown();
   }

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

@@ -23,10 +23,10 @@ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_SOCK
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_KEY;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.InputStream;
 
@@ -40,9 +40,10 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.ReflectionUtils;
-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;
 
@@ -55,7 +56,7 @@ public class TestDataTransferKeepalive {
   private static final int KEEPALIVE_TIMEOUT = 1000;
   private static final int WRITE_TIMEOUT = 3000;
   
-  @Before
+  @BeforeEach
   public void setup() throws Exception {
     conf.setInt(DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_KEY,
         KEEPALIVE_TIMEOUT);
@@ -67,7 +68,7 @@ public class TestDataTransferKeepalive {
     dn = cluster.getDataNodes().get(0);
   }
   
-  @After
+  @AfterEach
   public void teardown() {
     if (cluster != null) {
       cluster.shutdown();
@@ -79,7 +80,8 @@ public class TestDataTransferKeepalive {
    * Regression test for HDFS-3357. Check that the datanode is respecting
    * its configured keepalive timeout.
    */
-  @Test(timeout=30000)
+  @Test
+  @Timeout(value = 30)
   public void testDatanodeRespectsKeepAliveTimeout() throws Exception {
     Configuration clientConf = new Configuration(conf);
     // Set a client socket cache expiry time much longer than 
@@ -124,7 +126,8 @@ public class TestDataTransferKeepalive {
   /**
    * Test that the client respects its keepalive timeout.
    */
-  @Test(timeout=30000)
+  @Test
+  @Timeout(value = 30)
   public void testClientResponsesKeepAliveTimeout() throws Exception {
     Configuration clientConf = new Configuration(conf);
     // Set a client socket cache expiry time much shorter than 
@@ -165,7 +168,8 @@ public class TestDataTransferKeepalive {
    * read bytes off the stream quickly. The datanode should time out sending the
    * chunks and the transceiver should die, even if it has a long keepalive.
    */
-  @Test(timeout=300000)
+  @Test
+  @Timeout(value = 300)
   public void testSlowReader() throws Exception {
     // Set a client socket cache expiry time much longer than 
     // the datanode-side expiration time.
@@ -204,7 +208,8 @@ public class TestDataTransferKeepalive {
     IOUtils.closeStream(stm);
   }
   
-  @Test(timeout=30000)
+  @Test
+  @Timeout(value = 30)
   public void testManyClosedSocketsInCache() throws Exception {
     // Make a small file
     Configuration clientConf = new Configuration(conf);

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

@@ -17,9 +17,9 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -67,7 +67,8 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.StringUtils;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.mockito.Mockito;
 
 /**
@@ -599,7 +600,8 @@ public class TestDataTransferProtocol {
         null, null, new String[0]);
   }
 
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testReleaseVolumeRefIfExceptionThrown()
       throws IOException, InterruptedException {
     Path file = new Path("dataprotocol.dat");

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

@@ -18,9 +18,9 @@
 package org.apache.hadoop.hdfs;
 
 import static org.apache.hadoop.hdfs.server.common.Util.fileAsURI;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.junit.Assume.assumeTrue;
 
 import java.io.File;
@@ -34,9 +34,10 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 /**
  * Tests if a data-node can startup depending on configuration parameters.
@@ -47,7 +48,7 @@ public class TestDatanodeConfig {
 
   private static MiniDFSCluster cluster;
 
-  @BeforeClass
+  @BeforeAll
   public static void setUp() throws Exception {
     clearBaseDir();
     Configuration conf = new HdfsConfiguration();
@@ -59,7 +60,7 @@ public class TestDatanodeConfig {
     cluster.waitActive();
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() throws Exception {
     if(cluster != null)
       cluster.shutdown();
@@ -95,7 +96,7 @@ public class TestDatanodeConfig {
         dn.shutdown();
       }
     }
-    assertNull("Data-node startup should have failed.", dn);
+    assertNull(dn, "Data-node startup should have failed.");
 
     // 2. Test "file:" ecPolicy and no ecPolicy (path-only). Both should work.
     String dnDir1 = fileAsURI(dataDir).toString() + "1";
@@ -106,7 +107,7 @@ public class TestDatanodeConfig {
                 dnDir1 + "," + dnDir2 + "," + dnDir3);
     try {
       cluster.startDataNodes(conf, 1, false, StartupOption.REGULAR, null);
-      assertTrue("Data-node should startup.", cluster.isDataNodeUp());
+      assertTrue(cluster.isDataNodeUp(), "Data-node should startup.");
     } finally {
       if (cluster != null) {
         cluster.shutdownDataNodes();
@@ -124,7 +125,8 @@ public class TestDatanodeConfig {
     }
   }
 
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testMemlockLimit() throws Exception {
     assumeTrue(NativeIO.isAvailable());
     final long memlockLimit =

+ 10 - 12
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeDeath.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;
 
@@ -34,7 +34,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.slf4j.event.Level;
 
 /**
@@ -96,7 +96,7 @@ public class TestDatanodeDeath {
           checkFile(fs, filename, replication, numBlocks, fileSize, myseed);
         } catch (Throwable e) {
           System.out.println("Workload exception " + e);
-          assertTrue(e.toString(), false);
+          assertTrue(false, e.toString());
         }
 
         // increment the stamp to indicate that another file is done.
@@ -148,9 +148,8 @@ public class TestDatanodeDeath {
     int attempt = 0;
 
     long len = fileSys.getFileStatus(name).getLen();
-    assertTrue(name + " should be of size " + filesize +
-               " but found to be of size " + len, 
-               len == filesize);
+    assertTrue(len == filesize, name + " should be of size " + filesize +
+        " but found to be of size " + len);
 
     // wait till all full blocks are confirmed by the datanodes.
     while (!done) {
@@ -198,9 +197,8 @@ public class TestDatanodeDeath {
 
   private static void checkData(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;
     }
   }
@@ -259,7 +257,7 @@ public class TestDatanodeDeath {
             // cluster.startDataNodes(conf, 1, true, null, null);
           } catch (IOException e) {
             System.out.println("TestDatanodeDeath Modify exception " + e);
-            assertTrue("TestDatanodeDeath Modify exception " + e, false);
+            assertTrue(false, "TestDatanodeDeath Modify exception " + e);
             running = false;
           }
         }
@@ -399,7 +397,7 @@ public class TestDatanodeDeath {
     } catch (Throwable e) {
       System.out.println("Simple Workload exception " + e);
       e.printStackTrace();
-      assertTrue(e.toString(), false);
+      assertTrue(false, e.toString());
     } finally {
       fs.close();
       cluster.shutdown();

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeLayoutUpgrade.java

@@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.io.File;
 import java.io.IOException;

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

@@ -19,7 +19,8 @@ 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.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.IOException;
 import java.util.Arrays;
@@ -42,8 +43,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.util.HostsFileWriter;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 /**
  * This test ensures the all types of data node report work correctly.
@@ -167,7 +167,7 @@ public class TestDatanodeReport {
       cluster.corruptBlockOnDataNodesByDeletingBlockFile(b);
       try {
         DFSTestUtil.readFile(fs, p);
-        Assert.fail("Must throw exception as the block doesn't exists on disk");
+        fail("Must throw exception as the block doesn't exists on disk");
       } catch (IOException e) {
         // all bad datanodes
       }

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

@@ -22,16 +22,16 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import java.io.File;
 import java.io.IOException;
 
 import org.apache.hadoop.hdfs.TestDFSUpgradeFromImage.ClusterVerifier;
 
-import static org.hamcrest.core.Is.is;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.assertj.core.api.Assertions.assertThat;
 
 
 /**
@@ -74,12 +74,12 @@ public class TestDatanodeStartupFixesLegacyStorageIDs {
         final String bpid = cluster.getNamesystem().getBlockPoolId();
         StorageReport[] reports =
             cluster.getDataNodes().get(0).getFSDataset().getStorageReports(bpid);
-        assertThat(reports.length, is(1));
+        assertThat(reports.length).isEqualTo(1);
         final String storageID = reports[0].getStorage().getStorageID();
         assertTrue(DatanodeStorage.isValidStorageId(storageID));
 
         if (expectedStorageId != null) {
-          assertThat(storageID, is(expectedStorageId));
+          assertThat(storageID).isEqualTo(expectedStorageId);
         }
       }
     });
@@ -110,7 +110,8 @@ public class TestDatanodeStartupFixesLegacyStorageIDs {
    * Upgrade from 2.2 (no storage IDs per volume) correctly generates
    * GUID-based storage IDs. Test case for HDFS-7575.
    */
-  @Test (timeout=300000)
+  @Test
+  @Timeout(value = 300)
   public void testUpgradeFrom22FixesStorageIDs() throws IOException {
     runLayoutUpgradeTest(GenericTestUtils.getMethodName(), null);
   }
@@ -120,7 +121,8 @@ public class TestDatanodeStartupFixesLegacyStorageIDs {
    * generates new storage IDs.
    * Test case for HDFS-7575.
    */
-  @Test (timeout=300000)
+  @Test
+  @Timeout(value = 300)
   public void testUpgradeFrom22via26FixesStorageIDs() throws IOException {
     runLayoutUpgradeTest(GenericTestUtils.getMethodName(), null);
   }
@@ -130,7 +132,8 @@ public class TestDatanodeStartupFixesLegacyStorageIDs {
    * not regenerate the storage IDs.
    * Test case for HDFS-7575.
    */
-  @Test (timeout=300000)
+  @Test
+  @Timeout(value = 300)
   public void testUpgradeFrom26PreservesStorageIDs() throws IOException {
     // StorageId present in the image testUpgradeFrom26PreservesStorageId.tgz
     runLayoutUpgradeTest(GenericTestUtils.getMethodName(),

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

@@ -26,10 +26,9 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 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.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.mockito.Mockito;
 
 import java.io.IOException;
@@ -44,12 +43,12 @@ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_PROBE_SUSPECT_NODE_INTERVAL_MS_KEY;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_IDLE_SLEEP_MS_KEY;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertSame;
-import static org.junit.Assert.assertNotSame;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 /**
  * Tests for dead node detection in DFSClient.
@@ -59,7 +58,7 @@ public class TestDeadNodeDetection {
   private MiniDFSCluster cluster;
   private Configuration conf;
 
-  @Before
+  @BeforeEach
   public void setUp() {
     cluster = null;
     conf = new HdfsConfiguration();
@@ -77,7 +76,7 @@ public class TestDeadNodeDetection {
     conf.setLong(DFS_CLIENT_DEAD_NODE_DETECTION_IDLE_SLEEP_MS_KEY, 100);
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
     if (cluster != null) {
       cluster.shutdown();
@@ -328,15 +327,15 @@ public class TestDeadNodeDetection {
       }
       waitForSuspectNode(din.getDFSClient());
       cluster.restartDataNode(one, true);
-      Assert.assertEquals(1,
+      assertEquals(1,
           deadNodeDetector.getSuspectNodesProbeQueue().size());
-      Assert.assertEquals(0,
+      assertEquals(0,
           deadNodeDetector.clearAndGetDetectedDeadNodes().size());
       deadNodeDetector.startProbeScheduler();
       Thread.sleep(1000);
-      Assert.assertEquals(0,
+      assertEquals(0,
           deadNodeDetector.getSuspectNodesProbeQueue().size());
-      Assert.assertEquals(0,
+      assertEquals(0,
           deadNodeDetector.clearAndGetDetectedDeadNodes().size());
     } finally {
       in.close();

+ 78 - 62
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommissionWithStriped.java

@@ -17,10 +17,11 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -63,18 +64,21 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.security.token.Token;
 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.TemporaryFolder;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.MethodOrderer;
+import org.junit.jupiter.api.Order;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestMethodOrder;
+import org.junit.jupiter.api.Timeout;
+import org.junit.jupiter.api.io.TempDir;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
  * This class tests the decommissioning of datanode with striped blocks.
  */
+@TestMethodOrder(MethodOrderer.OrderAnnotation.class)
 public class TestDecommissionWithStriped {
   private static final Logger LOG = LoggerFactory
       .getLogger(TestDecommissionWithStriped.class);
@@ -94,9 +98,6 @@ public class TestDecommissionWithStriped {
   private Path excludeFile;
   private LocalFileSystem localFileSys;
 
-  @Rule
-  public TemporaryFolder baseDir = new TemporaryFolder();
-
   private Configuration conf;
   private MiniDFSCluster cluster;
   private DistributedFileSystem dfs;
@@ -118,12 +119,12 @@ public class TestDecommissionWithStriped {
     return new HdfsConfiguration();
   }
 
-  @Before
-  public void setup() throws IOException {
+  @BeforeEach
+  public void setup(@TempDir java.nio.file.Path baseDir) throws IOException {
     conf = createConfiguration();
     // Set up the hosts/exclude files.
     localFileSys = FileSystem.getLocal(conf);
-    localFileSys.setWorkingDirectory(new Path(baseDir.getRoot().getPath()));
+    localFileSys.setWorkingDirectory(new Path(baseDir.toAbsolutePath().toString()));
     Path workingDir = localFileSys.getWorkingDirectory();
     decommissionDir = new Path(workingDir, "work-dir/decommission");
     hostsFile = new Path(decommissionDir, "hosts");
@@ -167,7 +168,7 @@ public class TestDecommissionWithStriped {
         StripedFileTestUtil.getDefaultECPolicy().getName());
   }
 
-  @After
+  @AfterEach
   public void teardown() throws IOException {
     cleanupFile(localFileSys, decommissionDir);
     if (cluster != null) {
@@ -176,45 +177,51 @@ public class TestDecommissionWithStriped {
     }
   }
 
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testFileFullBlockGroup() throws Exception {
     LOG.info("Starting test testFileFullBlockGroup");
     testDecommission(blockSize * dataBlocks, 9, 1, "testFileFullBlockGroup");
   }
 
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testFileMultipleBlockGroups() throws Exception {
     LOG.info("Starting test testFileMultipleBlockGroups");
     int writeBytes = 2 * blockSize * dataBlocks;
     testDecommission(writeBytes, 9, 1, "testFileMultipleBlockGroups");
   }
 
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testFileSmallerThanOneCell() throws Exception {
     LOG.info("Starting test testFileSmallerThanOneCell");
     testDecommission(cellSize - 1, 4, 1, "testFileSmallerThanOneCell");
   }
 
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testFileSmallerThanOneStripe() throws Exception {
     LOG.info("Starting test testFileSmallerThanOneStripe");
     testDecommission(cellSize * 2, 5, 1, "testFileSmallerThanOneStripe");
   }
 
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testDecommissionTwoNodes() throws Exception {
     LOG.info("Starting test testDecommissionTwoNodes");
     testDecommission(blockSize * dataBlocks, 9, 2, "testDecommissionTwoNodes");
   }
 
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testDecommissionWithURBlockForSameBlockGroup() throws Exception {
     LOG.info("Starting test testDecommissionWithURBlocksForSameBlockGroup");
 
     final Path ecFile = new Path(ecDir, "testDecommissionWithCorruptBlocks");
     int writeBytes = cellSize * dataBlocks * 2;
     writeStripedFile(dfs, ecFile, writeBytes);
-    Assert.assertEquals(0, bm.numOfUnderReplicatedBlocks());
+    assertEquals(0, bm.numOfUnderReplicatedBlocks());
 
     final List<DatanodeInfo> decommisionNodes = new ArrayList<DatanodeInfo>();
     LocatedBlock lb = dfs.getClient().getLocatedBlocks(ecFile.toString(), 0)
@@ -259,7 +266,7 @@ public class TestDecommissionWithStriped {
           decommissionNode(0, decommisionNodes, AdminStates.DECOMMISSIONED);
         } catch (Exception e) {
           LOG.error("Exception while decommissioning", e);
-          Assert.fail("Shouldn't throw exception!");
+          fail("Shouldn't throw exception!");
         }
       };
     };
@@ -285,8 +292,8 @@ public class TestDecommissionWithStriped {
         fsn.getNumDecomLiveDataNodes());
 
     // Ensure decommissioned datanode is not automatically shutdown
-    assertEquals("All datanodes must be alive", numDNs,
-        client.datanodeReport(DatanodeReportType.LIVE).length);
+    assertEquals(numDNs, client.datanodeReport(DatanodeReportType.LIVE).length,
+        "All datanodes must be alive");
 
     assertNull(checkFile(dfs, ecFile, 9, decommisionNodes, numDNs));
     StripedFileTestUtil.checkData(dfs, ecFile, writeBytes, decommisionNodes,
@@ -298,7 +305,8 @@ public class TestDecommissionWithStriped {
    * DN decommission shouldn't reconstruction busy DN block.
    * @throws Exception
    */
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testDecommissionWithBusyNode() throws Exception {
     byte busyDNIndex = 1;
     byte decommisionDNIndex = 0;
@@ -306,7 +314,7 @@ public class TestDecommissionWithStriped {
     final Path ecFile = new Path(ecDir, "testDecommissionWithBusyNode");
     int writeBytes = cellSize * dataBlocks;
     writeStripedFile(dfs, ecFile, writeBytes);
-    Assert.assertEquals(0, bm.numOfUnderReplicatedBlocks());
+    assertEquals(0, bm.numOfUnderReplicatedBlocks());
     FileChecksum fileChecksum1 = dfs.getFileChecksum(ecFile, writeBytes);
 
     //2. make once DN busy
@@ -330,9 +338,9 @@ public class TestDecommissionWithStriped {
     //4. wait for decommission block to replicate
     Thread.sleep(3000);
     DatanodeStorageInfo[] newDnStorageInfos = bm.getStorages(firstBlock);
-    Assert.assertEquals("Busy DN shouldn't be reconstructed",
-        dnStorageInfos[busyDNIndex].getStorageID(),
-        newDnStorageInfos[busyDNIndex].getStorageID());
+    assertEquals(dnStorageInfos[busyDNIndex].getStorageID(),
+        newDnStorageInfos[busyDNIndex].getStorageID(),
+        "Busy DN shouldn't be reconstructed");
 
     //5. check decommission DN block index, it should be reconstructed again
     LocatedBlocks lbs = cluster.getNameNodeRpc().getBlockLocations(
@@ -345,12 +353,12 @@ public class TestDecommissionWithStriped {
       }
     }
 
-    Assert.assertEquals("Decommission DN block should be reconstructed", 2,
-        decommissionBlockIndexCount);
+    assertEquals(2, decommissionBlockIndexCount,
+        "Decommission DN block should be reconstructed");
 
     FileChecksum fileChecksum2 = dfs.getFileChecksum(ecFile, writeBytes);
-    Assert.assertTrue("Checksum mismatches!",
-        fileChecksum1.equals(fileChecksum2));
+    assertTrue(fileChecksum1.equals(fileChecksum2),
+        "Checksum mismatches!");
   }
 
   /**
@@ -358,7 +366,8 @@ public class TestDecommissionWithStriped {
    * in some case.
    * @throws Exception
    */
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testDecommission2NodeWithBusyNode() throws Exception {
     byte busyDNIndex = 6;
     byte decommissionDNIndex = 6;
@@ -368,7 +377,7 @@ public class TestDecommissionWithStriped {
     int writeBytes = cellSize * dataBlocks;
     writeStripedFile(dfs, ecFile, writeBytes);
 
-    Assert.assertEquals(0, bm.numOfUnderReplicatedBlocks());
+    assertEquals(0, bm.numOfUnderReplicatedBlocks());
     FileChecksum fileChecksum1 = dfs.getFileChecksum(ecFile, writeBytes);
 
     //2. make once DN busy
@@ -403,13 +412,13 @@ public class TestDecommissionWithStriped {
 
     //7. Busy DN shouldn't be reconstructed
     DatanodeStorageInfo[] newDnStorageInfos = bm.getStorages(firstBlock);
-    Assert.assertEquals("Busy DN shouldn't be reconstructed",
-        dnStorageInfos[busyDNIndex].getStorageID(),
-        newDnStorageInfos[busyDNIndex].getStorageID());
+    assertEquals(dnStorageInfos[busyDNIndex].getStorageID(),
+        newDnStorageInfos[busyDNIndex].getStorageID(),
+        "Busy DN shouldn't be reconstructed");
 
     //8. check the checksum of a file
     FileChecksum fileChecksum2 = dfs.getFileChecksum(ecFile, writeBytes);
-    Assert.assertEquals("Checksum mismatches!", fileChecksum1, fileChecksum2);
+    assertEquals(fileChecksum1, fileChecksum2, "Checksum mismatches!");
 
     //9. check the data is correct
     StripedFileTestUtil.checkData(dfs, ecFile, writeBytes, decommissionNodes,
@@ -428,14 +437,15 @@ public class TestDecommissionWithStriped {
    * Here, this list contains duplicated blocks and does not maintaining any
    * order.
    */
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testFileChecksumAfterDecommission() throws Exception {
     LOG.info("Starting test testFileChecksumAfterDecommission");
 
     final Path ecFile = new Path(ecDir, "testFileChecksumAfterDecommission");
     int writeBytes = cellSize * dataBlocks;
     writeStripedFile(dfs, ecFile, writeBytes);
-    Assert.assertEquals(0, bm.numOfUnderReplicatedBlocks());
+    assertEquals(0, bm.numOfUnderReplicatedBlocks());
     FileChecksum fileChecksum1 = dfs.getFileChecksum(ecFile, writeBytes);
 
     final List<DatanodeInfo> decommisionNodes = new ArrayList<DatanodeInfo>();
@@ -458,22 +468,23 @@ public class TestDecommissionWithStriped {
     LOG.info("fileChecksum1:" + fileChecksum1);
     LOG.info("fileChecksum2:" + fileChecksum2);
 
-    Assert.assertTrue("Checksum mismatches!",
-        fileChecksum1.equals(fileChecksum2));
+    assertTrue(fileChecksum1.equals(fileChecksum2),
+        "Checksum mismatches!");
   }
 
   /**
    * Test decommission when DN marked as busy.
    * @throwsException
    */
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testBusyAfterDecommissionNode() throws Exception {
     int busyDNIndex = 0;
     //1. create EC file.
     final Path ecFile = new Path(ecDir, "testBusyAfterDecommissionNode");
     int writeBytes = cellSize * dataBlocks;
     writeStripedFile(dfs, ecFile, writeBytes);
-    Assert.assertEquals(0, bm.numOfUnderReplicatedBlocks());
+    assertEquals(0, bm.numOfUnderReplicatedBlocks());
     FileChecksum fileChecksum1 = dfs.getFileChecksum(ecFile, writeBytes);
 
     //2. make once DN busy.
@@ -543,8 +554,8 @@ public class TestDecommissionWithStriped {
 
     // Ensure decommissioned datanode is not automatically shutdown
     DFSClient client = getDfsClient(cluster.getNameNode(0), conf);
-    assertEquals("All datanodes must be alive", numDNs,
-        client.datanodeReport(DatanodeReportType.LIVE).length);
+    assertEquals(numDNs, client.datanodeReport(DatanodeReportType.LIVE).length,
+        "All datanodes must be alive");
 
     assertNull(checkFile(dfs, ecFile, storageCount, decommisionNodes, numDNs));
     StripedFileTestUtil.checkData(dfs, ecFile, writeBytes, decommisionNodes,
@@ -590,10 +601,10 @@ public class TestDecommissionWithStriped {
           locToTokenList.get(i);
       DatanodeInfo[] di = lb.getLocations();
       for (int j = 0; j < di.length; j++) {
-        Assert.assertEquals("Block index value mismatches after sorting",
-            (byte) locToIndex.get(di[j]), stripedBlk.getBlockIndices()[j]);
-        Assert.assertEquals("Block token value mismatches after sorting",
-            locToToken.get(di[j]), stripedBlk.getBlockTokens()[j]);
+        assertEquals((byte) locToIndex.get(di[j]), stripedBlk.getBlockIndices()[j],
+            "Block index value mismatches after sorting");
+        assertEquals(locToToken.get(di[j]), stripedBlk.getBlockTokens()[j],
+            "Block token value mismatches after sorting");
       }
     }
   }
@@ -677,7 +688,7 @@ public class TestDecommissionWithStriped {
           break;
         }
       }
-      assertTrue("Datanode: " + dn + " is not LIVE", nodeExists);
+      assertTrue(nodeExists, "Datanode: " + dn + " is not LIVE");
       excludeNodes.add(dn.getName());
       LOG.info("Decommissioning node: " + dn.getName());
     }
@@ -729,8 +740,8 @@ public class TestDecommissionWithStriped {
           throws IOException {
     boolean isNodeDown = decommissionedNodes.size() > 0;
     // need a raw stream
-    assertTrue("Not HDFS:" + fileSys.getUri(),
-        fileSys instanceof DistributedFileSystem);
+    assertTrue(fileSys instanceof DistributedFileSystem,
+        "Not HDFS:" + fileSys.getUri());
     HdfsDataInputStream dis = (HdfsDataInputStream) fileSys.open(name);
     Collection<LocatedBlock> dinfo = dis.getAllBlocks();
     for (LocatedBlock blk : dinfo) { // for each block
@@ -784,7 +795,8 @@ public class TestDecommissionWithStriped {
    * Simulate that There are 2 nodes(dn0,dn1) in decommission. Firstly dn0
    * replicates in success, dn1 replicates in failure. Decommissions go on.
    */
-  @Test (timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testDecommissionWithFailedReplicating() throws Exception {
 
     // Write ec file.
@@ -893,7 +905,9 @@ public class TestDecommissionWithStriped {
     return null;
   }
 
-  @Test (timeout = 120000)
+  @Test
+  @Order(1)
+  @Timeout(value = 120)
   public void testDecommissionWithMissingBlock() throws Exception {
     // Write ec file.
     Path ecFile = new Path(ecDir, "missingOneInternalBLockFile");
@@ -988,7 +1002,7 @@ public class TestDecommissionWithStriped {
             decommissionNode(0, decommisionNodes, AdminStates.DECOMMISSIONED);
           } catch (Exception e) {
             LOG.error("Exception while decommissioning", e);
-            Assert.fail("Shouldn't throw exception!");
+            fail("Shouldn't throw exception!");
           }
         }).start();
     decomStarted.await(5, TimeUnit.SECONDS);
@@ -1007,7 +1021,8 @@ public class TestDecommissionWithStriped {
     cleanupFile(dfs, ecFile);
   }
 
-  @Test (timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testCountNodes() throws Exception{
     // Write ec file.
     Path ecFile = new Path(ecDir, "testCountNodes");
@@ -1088,7 +1103,8 @@ public class TestDecommissionWithStriped {
    * in decommissioned. array[4] is null, array[5-12]{b[5-8],b[0-3]} are
    * in live.
    */
-  @Test (timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testRecoveryWithDecommission() throws Exception {
     final Path ecFile = new Path(ecDir, "testRecoveryWithDecommission");
     int writeBytes = cellSize * dataBlocks;

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDeprecatedKeys.java

@@ -18,11 +18,11 @@
 
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertFalse;
 
 import org.apache.hadoop.conf.Configuration;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 public class TestDeprecatedKeys {