فهرست منبع

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

Co-authored-by: Chris Nauroth <cnauroth@apache.org>
Co-authored-by: Shilun Fan <slfan1989@apache.org>
Reviewed-by: Chris Nauroth <cnauroth@apache.org>
Reviewed-by: Shilun Fan <slfan1989@apache.org>
Signed-off-by: Shilun Fan <slfan1989@apache.org>
zhtttylz 2 هفته پیش
والد
کامیت
71aa0e4778
37فایلهای تغییر یافته به همراه744 افزوده شده و 694 حذف شده
  1. 77 70
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
  2. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsCreateMkdir.java
  3. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsPermission.java
  4. 33 30
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsSetUMask.java
  5. 59 55
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestGlobPaths.java
  6. 42 36
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestHDFSFileContextMainOperations.java
  7. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestHdfsNativeCodeLoader.java
  8. 10 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestResolveHdfsSymlink.java
  9. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSWebHdfsFileContextMainOperations.java
  10. 5 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfsDisable.java
  11. 17 18
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUnbuffer.java
  12. 12 12
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUrlStreamHandler.java
  13. 9 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestWebHdfsFileContextMainOperations.java
  14. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/loadGenerator/TestLoadGenerator.java
  15. 11 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/permission/TestStickyBit.java
  16. 7 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/shell/TestHdfsTextCommand.java
  17. 10 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestNNStartupWhenViewFSOverloadSchemeEnabled.java
  18. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFSOverloadSchemeWithMountTableConfigInHDFS.java
  19. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemAtHdfsRoot.java
  20. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemClose.java
  21. 31 31
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java
  22. 66 71
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkFallback.java
  23. 15 18
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkMergeSlash.java
  24. 14 15
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java
  25. 18 13
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemOverloadSchemeHdfsFileSystemContract.java
  26. 105 87
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemOverloadSchemeWithHdfsScheme.java
  27. 11 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithAcls.java
  28. 15 13
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithTruncate.java
  29. 11 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithXAttrs.java
  30. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsAtHdfsRoot.java
  31. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsDefaultValue.java
  32. 11 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsFileStatusHdfs.java
  33. 10 10
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsHdfs.java
  34. 61 60
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsLinkFallback.java
  35. 11 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsWithAcls.java
  36. 11 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsWithXAttrs.java
  37. 14 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestViewDistributedFileSystemWithMountLinks.java

+ 77 - 70
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java

@@ -21,6 +21,13 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CACHEREPORT_INTERVAL_MSEC
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS;
+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.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.junit.jupiter.api.Assumptions.assumeTrue;
 
 import java.io.File;
 import java.io.FileInputStream;
@@ -65,11 +72,10 @@ import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Assume;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import org.apache.hadoop.util.Preconditions;
 import java.util.function.Supplier;
@@ -85,7 +91,7 @@ public class TestEnhancedByteBufferAccess {
 
   static private CacheManipulator prevCacheManipulator;
 
-  @BeforeClass
+  @BeforeAll
   public static void init() {
     sockDir = new TemporarySocketDirectory();
     DomainSocket.disableBindPathValidation();
@@ -99,7 +105,7 @@ public class TestEnhancedByteBufferAccess {
     });
   }
 
-  @AfterClass
+  @AfterAll
   public static void teardown() {
     // Restore the original CacheManipulator
     NativeIO.POSIX.setCacheManipulator(prevCacheManipulator);
@@ -116,8 +122,8 @@ public class TestEnhancedByteBufferAccess {
       (int) NativeIO.POSIX.getCacheManipulator().getOperatingSystemPageSize();
   
   public static HdfsConfiguration initZeroCopyTest() {
-    Assume.assumeTrue(NativeIO.isAvailable());
-    Assume.assumeTrue(SystemUtils.IS_OS_UNIX);
+    assumeTrue(NativeIO.isAvailable());
+    assumeTrue(SystemUtils.IS_OS_UNIX);
     HdfsConfiguration conf = new HdfsConfiguration();
     conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
@@ -152,10 +158,10 @@ public class TestEnhancedByteBufferAccess {
       try {
         DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1);
       } catch (InterruptedException e) {
-        Assert.fail("unexpected InterruptedException during " +
+        fail("unexpected InterruptedException during " +
             "waitReplication: " + e);
       } catch (TimeoutException e) {
-        Assert.fail("unexpected TimeoutException during " +
+        fail("unexpected TimeoutException during " +
             "waitReplication: " + e);
       }
       fsIn = fs.open(TEST_PATH);
@@ -165,13 +171,13 @@ public class TestEnhancedByteBufferAccess {
       fsIn = fs.open(TEST_PATH);
       ByteBuffer result = fsIn.read(null, BLOCK_SIZE,
           EnumSet.of(ReadOption.SKIP_CHECKSUMS));
-      Assert.assertEquals(BLOCK_SIZE, result.remaining());
+      assertEquals(BLOCK_SIZE, result.remaining());
       HdfsDataInputStream dfsIn = (HdfsDataInputStream)fsIn;
-      Assert.assertEquals(BLOCK_SIZE,
+      assertEquals(BLOCK_SIZE,
           dfsIn.getReadStatistics().getTotalBytesRead());
-      Assert.assertEquals(BLOCK_SIZE,
+      assertEquals(BLOCK_SIZE,
           dfsIn.getReadStatistics().getTotalZeroCopyBytesRead());
-      Assert.assertArrayEquals(Arrays.copyOfRange(original, 0, BLOCK_SIZE),
+      assertArrayEquals(Arrays.copyOfRange(original, 0, BLOCK_SIZE),
           byteBufferToArray(result));
       fsIn.releaseBuffer(result);
     } finally {
@@ -198,10 +204,10 @@ public class TestEnhancedByteBufferAccess {
       try {
         DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1);
       } catch (InterruptedException e) {
-        Assert.fail("unexpected InterruptedException during " +
+        fail("unexpected InterruptedException during " +
             "waitReplication: " + e);
       } catch (TimeoutException e) {
-        Assert.fail("unexpected TimeoutException during " +
+        fail("unexpected TimeoutException during " +
             "waitReplication: " + e);
       }
       fsIn = fs.open(TEST_PATH);
@@ -214,20 +220,20 @@ public class TestEnhancedByteBufferAccess {
       HdfsDataInputStream dfsIn = (HdfsDataInputStream)fsIn;
       ByteBuffer result =
         dfsIn.read(null, 2 * BLOCK_SIZE, EnumSet.of(ReadOption.SKIP_CHECKSUMS));
-      Assert.assertEquals(BLOCK_SIZE, result.remaining());
-      Assert.assertEquals(BLOCK_SIZE,
+      assertEquals(BLOCK_SIZE, result.remaining());
+      assertEquals(BLOCK_SIZE,
           dfsIn.getReadStatistics().getTotalBytesRead());
-      Assert.assertEquals(BLOCK_SIZE,
+      assertEquals(BLOCK_SIZE,
           dfsIn.getReadStatistics().getTotalZeroCopyBytesRead());
-      Assert.assertArrayEquals(Arrays.copyOfRange(original, 0, BLOCK_SIZE),
+      assertArrayEquals(Arrays.copyOfRange(original, 0, BLOCK_SIZE),
           byteBufferToArray(result));
       dfsIn.releaseBuffer(result);
       
       // Try to read (1 + ${BLOCK_SIZE}), but only get ${BLOCK_SIZE} because of the block size.
       result = 
           dfsIn.read(null, 1 + BLOCK_SIZE, EnumSet.of(ReadOption.SKIP_CHECKSUMS));
-      Assert.assertEquals(BLOCK_SIZE, result.remaining());
-      Assert.assertArrayEquals(Arrays.copyOfRange(original, BLOCK_SIZE, 2 * BLOCK_SIZE),
+      assertEquals(BLOCK_SIZE, result.remaining());
+      assertArrayEquals(Arrays.copyOfRange(original, BLOCK_SIZE, 2 * BLOCK_SIZE),
           byteBufferToArray(result));
       dfsIn.releaseBuffer(result);
     } finally {
@@ -255,10 +261,10 @@ public class TestEnhancedByteBufferAccess {
       try {
         DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1);
       } catch (InterruptedException e) {
-        Assert.fail("unexpected InterruptedException during " +
+        fail("unexpected InterruptedException during " +
             "waitReplication: " + e);
       } catch (TimeoutException e) {
-        Assert.fail("unexpected TimeoutException during " +
+        fail("unexpected TimeoutException during " +
             "waitReplication: " + e);
       }
       fsIn = fs.open(TEST_PATH);
@@ -270,17 +276,17 @@ public class TestEnhancedByteBufferAccess {
       ByteBuffer result;
       try {
         result = dfsIn.read(null, BLOCK_SIZE + 1, EnumSet.noneOf(ReadOption.class));
-        Assert.fail("expected UnsupportedOperationException");
+        fail("expected UnsupportedOperationException");
       } catch (UnsupportedOperationException e) {
         // expected
       }
       result = dfsIn.read(null, BLOCK_SIZE, EnumSet.of(ReadOption.SKIP_CHECKSUMS));
-      Assert.assertEquals(BLOCK_SIZE, result.remaining());
-      Assert.assertEquals(BLOCK_SIZE,
+      assertEquals(BLOCK_SIZE, result.remaining());
+      assertEquals(BLOCK_SIZE,
           dfsIn.getReadStatistics().getTotalBytesRead());
-      Assert.assertEquals(BLOCK_SIZE,
+      assertEquals(BLOCK_SIZE,
           dfsIn.getReadStatistics().getTotalZeroCopyBytesRead());
-      Assert.assertArrayEquals(Arrays.copyOfRange(original, 0, BLOCK_SIZE),
+      assertArrayEquals(Arrays.copyOfRange(original, 0, BLOCK_SIZE),
           byteBufferToArray(result));
     } finally {
       if (fsIn != null) fsIn.close();
@@ -311,16 +317,16 @@ public class TestEnhancedByteBufferAccess {
         LinkedMap evictable,
         LinkedMap evictableMmapped) {
       if (expectedNumOutstandingMmaps >= 0) {
-        Assert.assertEquals(expectedNumOutstandingMmaps, numOutstandingMmaps);
+        assertEquals(expectedNumOutstandingMmaps, numOutstandingMmaps);
       }
       if (expectedNumReplicas >= 0) {
-        Assert.assertEquals(expectedNumReplicas, replicas.size());
+        assertEquals(expectedNumReplicas, replicas.size());
       }
       if (expectedNumEvictable >= 0) {
-        Assert.assertEquals(expectedNumEvictable, evictable.size());
+        assertEquals(expectedNumEvictable, evictable.size());
       }
       if (expectedNumMmapedEvictable >= 0) {
-        Assert.assertEquals(expectedNumMmapedEvictable, evictableMmapped.size());
+        assertEquals(expectedNumMmapedEvictable, evictableMmapped.size());
       }
     }
   }
@@ -346,10 +352,10 @@ public class TestEnhancedByteBufferAccess {
     try {
       DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1);
     } catch (InterruptedException e) {
-      Assert.fail("unexpected InterruptedException during " +
+      fail("unexpected InterruptedException during " +
           "waitReplication: " + e);
     } catch (TimeoutException e) {
-      Assert.fail("unexpected TimeoutException during " +
+      fail("unexpected TimeoutException during " +
           "waitReplication: " + e);
     }
     fsIn = fs.open(TEST_PATH);
@@ -378,10 +384,10 @@ public class TestEnhancedByteBufferAccess {
           LinkedMap evictableMmapped) {
         ShortCircuitReplica replica = replicas.get(
             new ExtendedBlockId(firstBlock.getBlockId(), firstBlock.getBlockPoolId()));
-        Assert.assertNotNull(replica);
-        Assert.assertTrue(replica.hasMmap());
+        assertNotNull(replica);
+        assertTrue(replica.hasMmap());
         // The replica should not yet be evictable, since we have it open.
-        Assert.assertNull(replica.getEvictableTimeNs());
+        assertNull(replica.getEvictableTimeNs());
       }
     });
 
@@ -449,10 +455,10 @@ public class TestEnhancedByteBufferAccess {
       try {
         DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1);
       } catch (InterruptedException e) {
-        Assert.fail("unexpected InterruptedException during " +
+        fail("unexpected InterruptedException during " +
             "waitReplication: " + e);
       } catch (TimeoutException e) {
-        Assert.fail("unexpected TimeoutException during " +
+        fail("unexpected TimeoutException during " +
             "waitReplication: " + e);
       }
       fsIn = fs.open(TEST_PATH);
@@ -493,22 +499,22 @@ public class TestEnhancedByteBufferAccess {
             stream instanceof ByteBufferReadable);
 
     ByteBuffer result = ByteBufferUtil.fallbackRead(stream, bufferPool, 10);
-    Assert.assertEquals(10, result.remaining());
-    Assert.assertArrayEquals(Arrays.copyOfRange(original, 0, 10),
+    assertEquals(10, result.remaining());
+    assertArrayEquals(Arrays.copyOfRange(original, 0, 10),
         byteBufferToArray(result));
 
     result = ByteBufferUtil.fallbackRead(stream, bufferPool, 5000);
-    Assert.assertEquals(5000, result.remaining());
-    Assert.assertArrayEquals(Arrays.copyOfRange(original, 10, 5010),
+    assertEquals(5000, result.remaining());
+    assertArrayEquals(Arrays.copyOfRange(original, 10, 5010),
         byteBufferToArray(result));
 
     result = ByteBufferUtil.fallbackRead(stream, bufferPool, 9999999);
-    Assert.assertEquals(11375, result.remaining());
-    Assert.assertArrayEquals(Arrays.copyOfRange(original, 5010, 16385),
+    assertEquals(11375, result.remaining());
+    assertArrayEquals(Arrays.copyOfRange(original, 5010, 16385),
         byteBufferToArray(result));
 
     result = ByteBufferUtil.fallbackRead(stream, bufferPool, 10);
-    Assert.assertNull(result);
+    assertNull(result);
   }
 
   /**
@@ -533,10 +539,10 @@ public class TestEnhancedByteBufferAccess {
       try {
         DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1);
       } catch (InterruptedException e) {
-        Assert.fail("unexpected InterruptedException during " +
+        fail("unexpected InterruptedException during " +
             "waitReplication: " + e);
       } catch (TimeoutException e) {
-        Assert.fail("unexpected TimeoutException during " +
+        fail("unexpected TimeoutException during " +
             "waitReplication: " + e);
       }
       fsIn = fs.open(TEST_PATH);
@@ -584,7 +590,8 @@ public class TestEnhancedByteBufferAccess {
    * Test that we can zero-copy read cached data even without disabling
    * checksums.
    */
-  @Test(timeout=120000)
+  @Test
+  @Timeout(value = 120)
   public void testZeroCopyReadOfCachedData() throws Exception {
     BlockReaderTestUtil.enableShortCircuitShmTracing();
     BlockReaderTestUtil.enableBlockReaderFactoryTracing();
@@ -618,7 +625,7 @@ public class TestEnhancedByteBufferAccess {
     try {
       result = fsIn.read(null, TEST_FILE_LENGTH / 2,
           EnumSet.noneOf(ReadOption.class));
-      Assert.fail("expected UnsupportedOperationException");
+      fail("expected UnsupportedOperationException");
     } catch (UnsupportedOperationException e) {
       // expected
     }
@@ -637,9 +644,9 @@ public class TestEnhancedByteBufferAccess {
       result = fsIn.read(null, TEST_FILE_LENGTH,
           EnumSet.noneOf(ReadOption.class));
     } catch (UnsupportedOperationException e) {
-      Assert.fail("expected to be able to read cached file via zero-copy");
+      fail("expected to be able to read cached file via zero-copy");
     }
-    Assert.assertArrayEquals(Arrays.copyOfRange(original, 0,
+    assertArrayEquals(Arrays.copyOfRange(original, 0,
         BLOCK_SIZE), byteBufferToArray(result));
     // Test that files opened after the cache operation has finished
     // still get the benefits of zero-copy (regression test for HDFS-6086)
@@ -648,9 +655,9 @@ public class TestEnhancedByteBufferAccess {
       result2 = fsIn2.read(null, TEST_FILE_LENGTH,
           EnumSet.noneOf(ReadOption.class));
     } catch (UnsupportedOperationException e) {
-      Assert.fail("expected to be able to read cached file via zero-copy");
+      fail("expected to be able to read cached file via zero-copy");
     }
-    Assert.assertArrayEquals(Arrays.copyOfRange(original, 0,
+    assertArrayEquals(Arrays.copyOfRange(original, 0,
         BLOCK_SIZE), byteBufferToArray(result2));
     fsIn2.releaseBuffer(result2);
     fsIn2.close();
@@ -688,10 +695,10 @@ public class TestEnhancedByteBufferAccess {
               Map<ExtendedBlockId, InvalidToken> failedLoads,
               LinkedMap evictable,
               LinkedMap evictableMmapped) {
-            Assert.assertEquals(expectedOutstandingMmaps, numOutstandingMmaps);
+            assertEquals(expectedOutstandingMmaps, numOutstandingMmaps);
             ShortCircuitReplica replica =
                 replicas.get(ExtendedBlockId.fromExtendedBlock(block));
-            Assert.assertNotNull(replica);
+            assertNotNull(replica);
             Slot slot = replica.getSlot();
             if ((expectedIsAnchorable != slot.isAnchorable()) ||
                 (expectedIsAnchored != slot.isAnchored())) {
@@ -734,7 +741,7 @@ public class TestEnhancedByteBufferAccess {
       fsIn = fs.open(TEST_PATH);
       try {
         fsIn.read(null, 1, EnumSet.of(ReadOption.SKIP_CHECKSUMS));
-        Assert.fail("expected zero-copy read to fail when client mmaps " +
+        fail("expected zero-copy read to fail when client mmaps " +
             "were disabled.");
       } catch (UnsupportedOperationException e) {
       }
@@ -764,7 +771,7 @@ public class TestEnhancedByteBufferAccess {
       // Test EOF behavior
       IOUtils.skipFully(fsIn, TEST_FILE_LENGTH - 1);
       buf = fsIn.read(null, 1, EnumSet.of(ReadOption.SKIP_CHECKSUMS));
-      Assert.assertEquals(null, buf);
+      assertEquals(null, buf);
     } finally {
       if (fsIn != null) fsIn.close();
       if (fs != null) fs.close();
@@ -774,7 +781,7 @@ public class TestEnhancedByteBufferAccess {
   
   @Test
   public void test2GBMmapLimit() throws Exception {
-    Assume.assumeTrue(BlockReaderTestUtil.shouldTestLargeFiles());
+    assumeTrue(BlockReaderTestUtil.shouldTestLargeFiles());
     HdfsConfiguration conf = initZeroCopyTest();
     final long TEST_FILE_LENGTH = 2469605888L;
     conf.set(DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY, "NULL");
@@ -795,20 +802,20 @@ public class TestEnhancedByteBufferAccess {
       
       fsIn = fs.open(TEST_PATH);
       buf1 = fsIn.read(null, 1, EnumSet.of(ReadOption.SKIP_CHECKSUMS));
-      Assert.assertEquals(1, buf1.remaining());
+      assertEquals(1, buf1.remaining());
       fsIn.releaseBuffer(buf1);
       buf1 = null;
       fsIn.seek(2147483640L);
       buf1 = fsIn.read(null, 1024, EnumSet.of(ReadOption.SKIP_CHECKSUMS));
-      Assert.assertEquals(7, buf1.remaining());
-      Assert.assertEquals(Integer.MAX_VALUE, buf1.limit());
+      assertEquals(7, buf1.remaining());
+      assertEquals(Integer.MAX_VALUE, buf1.limit());
       fsIn.releaseBuffer(buf1);
       buf1 = null;
-      Assert.assertEquals(2147483647L, fsIn.getPos());
+      assertEquals(2147483647L, fsIn.getPos());
       try {
         buf1 = fsIn.read(null, 1024,
             EnumSet.of(ReadOption.SKIP_CHECKSUMS));
-        Assert.fail("expected UnsupportedOperationException");
+        fail("expected UnsupportedOperationException");
       } catch (UnsupportedOperationException e) {
         // expected; can't read past 2GB boundary.
       }
@@ -825,13 +832,13 @@ public class TestEnhancedByteBufferAccess {
       fsIn2 = fs.open(TEST_PATH2);
       fsIn2.seek(2147483640L);
       buf2 = fsIn2.read(null, 1024, EnumSet.of(ReadOption.SKIP_CHECKSUMS));
-      Assert.assertEquals(8, buf2.remaining());
-      Assert.assertEquals(2147483648L, fsIn2.getPos());
+      assertEquals(8, buf2.remaining());
+      assertEquals(2147483648L, fsIn2.getPos());
       fsIn2.releaseBuffer(buf2);
       buf2 = null;
       buf2 = fsIn2.read(null, 1024, EnumSet.of(ReadOption.SKIP_CHECKSUMS));
-      Assert.assertEquals(1024, buf2.remaining());
-      Assert.assertEquals(2147484672L, fsIn2.getPos());
+      assertEquals(1024, buf2.remaining());
+      assertEquals(2147484672L, fsIn2.getPos());
       fsIn2.releaseBuffer(buf2);
       buf2 = null;
     } finally {

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

@@ -27,10 +27,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
 
 public class TestFcHdfsCreateMkdir extends
                     FileContextCreateMkdirBaseTest {
@@ -44,7 +44,7 @@ public class TestFcHdfsCreateMkdir extends
   }
 
 
-  @BeforeClass
+  @BeforeAll
   public static void clusterSetupAtBegining()
                                     throws IOException, LoginException, URISyntaxException  {
     Configuration conf = new HdfsConfiguration();
@@ -56,7 +56,7 @@ public class TestFcHdfsCreateMkdir extends
   }
 
       
-  @AfterClass
+  @AfterAll
   public static void ClusterShutdownAtEnd() throws Exception {
     if (cluster != null) {
       cluster.shutdown();
@@ -64,13 +64,13 @@ public class TestFcHdfsCreateMkdir extends
   }
   
   @Override
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     super.setUp();
   }
   
   @Override
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     super.tearDown();
   }

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

@@ -27,10 +27,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
 
 public class TestFcHdfsPermission extends FileContextPermissionBase {
   
@@ -51,7 +51,7 @@ public class TestFcHdfsPermission extends FileContextPermissionBase {
     return fc;
   }
   
-  @BeforeClass
+  @BeforeAll
   public static void clusterSetupAtBegining()
                                     throws IOException, LoginException, URISyntaxException  {
     Configuration conf = new HdfsConfiguration();
@@ -63,7 +63,7 @@ public class TestFcHdfsPermission extends FileContextPermissionBase {
   }
 
       
-  @AfterClass
+  @AfterAll
   public static void ClusterShutdownAtEnd() throws Exception {
     if (cluster != null) {
       cluster.shutdown();
@@ -71,13 +71,13 @@ public class TestFcHdfsPermission extends FileContextPermissionBase {
   }
   
   @Override
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     super.setUp();
   }
   
   @Override
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     super.tearDown();
   }

+ 33 - 30
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsSetUMask.java

@@ -30,12 +30,15 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.security.UserGroupInformation;
 import static org.apache.hadoop.fs.FileContextTestHelper.*;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-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.AfterEach;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 
 public class TestFcHdfsSetUMask {
   
@@ -78,7 +81,7 @@ public class TestFcHdfsSetUMask {
   private static final FsPermission WIDE_OPEN_TEST_UMASK = FsPermission
       .createImmutable((short) (0777 ^ 0777));
   
-  @BeforeClass
+  @BeforeAll
   public static void clusterSetupAtBegining()
         throws IOException, LoginException, URISyntaxException  {
     Configuration conf = new HdfsConfiguration();
@@ -91,20 +94,20 @@ public class TestFcHdfsSetUMask {
     fc.mkdir(defaultWorkingDirectory, FileContext.DEFAULT_PERM, true);
   }
 
-  @AfterClass
+  @AfterAll
   public static void ClusterShutdownAtEnd() throws Exception {
     if (cluster != null) {
       cluster.shutdown();
     }
   }
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     fc.setUMask(WIDE_OPEN_TEST_UMASK);
     fc.mkdir(fileContextTestHelper.getTestRootPath(fc), FileContext.DEFAULT_PERM, true);
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     fc.delete(fileContextTestHelper.getTestRootPath(fc), true);
   }
@@ -165,7 +168,7 @@ public class TestFcHdfsSetUMask {
 
   @Test
   public void testCreateRecursiveWithNonExistingDirClear() throws IOException {
-    // directory permission inherited from parent so this must match the @Before
+    // directory permission inherited from parent so this must match the @BeforeEach
     // set of umask
     testCreateRecursiveWithNonExistingDir(BLANK_TEST_UMASK,
         WIDE_OPEN_PERMISSIONS, BLANK_PERMISSIONS);
@@ -173,7 +176,7 @@ public class TestFcHdfsSetUMask {
 
   @Test
   public void testCreateRecursiveWithNonExistingDirOpen() throws IOException {
-    // directory permission inherited from parent so this must match the @Before
+    // directory permission inherited from parent so this must match the @BeforeEach
     // set of umask
     testCreateRecursiveWithNonExistingDir(WIDE_OPEN_TEST_UMASK,
         WIDE_OPEN_PERMISSIONS, WIDE_OPEN_FILE_PERMISSIONS);
@@ -181,7 +184,7 @@ public class TestFcHdfsSetUMask {
 
   @Test
   public void testCreateRecursiveWithNonExistingDirMiddle() throws IOException {
-    // directory permission inherited from parent so this must match the @Before
+    // directory permission inherited from parent so this must match the @BeforeEach
     // set of umask
     testCreateRecursiveWithNonExistingDir(USER_GROUP_OPEN_TEST_UMASK, 
         WIDE_OPEN_PERMISSIONS, USER_GROUP_OPEN_FILE_PERMISSIONS);
@@ -193,9 +196,9 @@ public class TestFcHdfsSetUMask {
     Path f = fileContextTestHelper.getTestRootPath(fc, "aDir");
     fc.setUMask(umask);
     fc.mkdir(f, FileContext.DEFAULT_PERM, true);
-    Assert.assertTrue(isDir(fc, f));
-    Assert.assertEquals("permissions on directory are wrong",  
-        expectedPerms, fc.getFileStatus(f).getPermission());
+    assertTrue(isDir(fc, f));
+    assertEquals(expectedPerms, fc.getFileStatus(f).getPermission(),
+        "permissions on directory are wrong");
   }
   
   public void testMkdirRecursiveWithNonExistingDir(FsPermission umask,
@@ -204,12 +207,12 @@ public class TestFcHdfsSetUMask {
     Path f = fileContextTestHelper.getTestRootPath(fc, "NonExistant2/aDir");
     fc.setUMask(umask);
     fc.mkdir(f, FileContext.DEFAULT_PERM, true);
-    Assert.assertTrue(isDir(fc, f));
-    Assert.assertEquals("permissions on directory are wrong",  
-        expectedPerms, fc.getFileStatus(f).getPermission());
+    assertTrue(isDir(fc, f));
+    assertEquals(expectedPerms, fc.getFileStatus(f).getPermission(),
+        "permissions on directory are wrong");
     Path fParent = fileContextTestHelper.getTestRootPath(fc, "NonExistant2");
-    Assert.assertEquals("permissions on parent directory are wrong",  
-        expectedParentPerms, fc.getFileStatus(fParent).getPermission());
+    assertEquals(expectedParentPerms, fc.getFileStatus(fParent).getPermission(),
+        "permissions on parent directory are wrong");
   }
 
 
@@ -218,9 +221,9 @@ public class TestFcHdfsSetUMask {
     Path f = fileContextTestHelper.getTestRootPath(fc,"foo");
     fc.setUMask(umask);
     createFile(fc, f);
-    Assert.assertTrue(isFile(fc, f));
-    Assert.assertEquals("permissions on file are wrong",  
-        expectedPerms , fc.getFileStatus(f).getPermission());
+    assertTrue(isFile(fc, f));
+    assertEquals(expectedPerms, fc.getFileStatus(f).getPermission(),
+        "permissions on file are wrong");
   }
   
   
@@ -229,14 +232,14 @@ public class TestFcHdfsSetUMask {
       throws IOException {
     Path f = fileContextTestHelper.getTestRootPath(fc,"NonExisting/foo");
     Path fParent = fileContextTestHelper.getTestRootPath(fc, "NonExisting");
-    Assert.assertFalse(exists(fc, fParent));
+    assertFalse(exists(fc, fParent));
     fc.setUMask(umask);
     createFile(fc, f);
-    Assert.assertTrue(isFile(fc, f));
-    Assert.assertEquals("permissions on file are wrong",  
-        expectedFilePerms, fc.getFileStatus(f).getPermission());
-    Assert.assertEquals("permissions on parent directory are wrong",  
-        expectedDirPerms, fc.getFileStatus(fParent).getPermission());
+    assertTrue(isFile(fc, f));
+    assertEquals(expectedFilePerms, fc.getFileStatus(f).getPermission(),
+        "permissions on file are wrong");
+    assertEquals(expectedDirPerms, fc.getFileStatus(fParent).getPermission(),
+        "permissions on parent directory are wrong");
   }
  
 }

+ 59 - 55
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestGlobPaths.java

@@ -18,7 +18,11 @@
 package org.apache.hadoop.fs;
 
 import static org.apache.hadoop.test.PlatformAssumptions.assumeNotWindows;
-import static org.junit.Assert.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.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;
@@ -34,7 +38,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.junit.*;
+import org.junit.jupiter.api.*;
 
 public class TestGlobPaths {
 
@@ -65,7 +69,7 @@ public class TestGlobPaths {
   static private String USER_DIR;
   private final Path[] path = new Path[NUM_OF_PATHS];
 
-  @BeforeClass
+  @BeforeAll
   public static void setUp() throws Exception {
     final Configuration conf = new HdfsConfiguration();
     dfsCluster = new MiniDFSCluster.Builder(conf).build();
@@ -81,7 +85,7 @@ public class TestGlobPaths {
     USER_DIR = fs.getHomeDirectory().toUri().getPath().toString();
   }
   
-  @AfterClass
+  @AfterAll
   public static void tearDown() throws Exception {
     if(dfsCluster!=null) {
       dfsCluster.shutdown();
@@ -102,8 +106,8 @@ public class TestGlobPaths {
     fs.createNewFile(fNormal);
     fs.createNewFile(fWithCR);
     statuses = fs.globStatus(new Path(d1, "f1*"));
-    assertEquals("Expected both normal and CR-carrying files in result: ",
-        2, statuses.length);
+    assertEquals(2, statuses.length,
+        "Expected both normal and CR-carrying files in result: ");
     cleanupDFS();
   }
 
@@ -892,14 +896,14 @@ public class TestGlobPaths {
       // Test simple glob
       FileStatus[] statuses = wrap.globStatus(new Path(USER_DIR + "/alpha/*"),
           new AcceptAllPathFilter());
-      Assert.assertEquals(1, statuses.length);
-      Assert.assertEquals(USER_DIR + "/alpha/beta", statuses[0].getPath()
+      assertEquals(1, statuses.length);
+      assertEquals(USER_DIR + "/alpha/beta", statuses[0].getPath()
           .toUri().getPath());
       // Test glob through symlink
       statuses = wrap.globStatus(new Path(USER_DIR + "/alphaLink/*"),
           new AcceptAllPathFilter());
-      Assert.assertEquals(1, statuses.length);
-      Assert.assertEquals(USER_DIR + "/alphaLink/beta", statuses[0].getPath()
+      assertEquals(1, statuses.length);
+      assertEquals(USER_DIR + "/alphaLink/beta", statuses[0].getPath()
           .toUri().getPath());
       // If the terminal path component in a globbed path is a symlink,
       // we don't dereference that link.
@@ -907,20 +911,20 @@ public class TestGlobPaths {
           + "/alphaLink/betaLink"), false);
       statuses = wrap.globStatus(new Path(USER_DIR + "/alpha/betaLi*"),
           new AcceptAllPathFilter());
-      Assert.assertEquals(1, statuses.length);
-      Assert.assertEquals(USER_DIR + "/alpha/betaLink", statuses[0].getPath()
+      assertEquals(1, statuses.length);
+      assertEquals(USER_DIR + "/alpha/betaLink", statuses[0].getPath()
           .toUri().getPath());
       // todo: test symlink-to-symlink-to-dir, etc.
     }
   }
 
-  @Ignore
+  @Disabled
   @Test
   public void testGlobWithSymlinksOnFS() throws Exception {
     testOnFileSystem(new TestGlobWithSymlinks(false));
   }
 
-  @Ignore
+  @Disabled
   @Test
   public void testGlobWithSymlinksOnFC() throws Exception {
     testOnFileContext(new TestGlobWithSymlinks(true));
@@ -951,20 +955,20 @@ public class TestGlobPaths {
       // Test glob through symlink to a symlink to a directory
       FileStatus statuses[] = wrap.globStatus(new Path(USER_DIR
           + "/alphaLinkLink"), new AcceptAllPathFilter());
-      Assert.assertEquals(1, statuses.length);
-      Assert.assertEquals(USER_DIR + "/alphaLinkLink", statuses[0].getPath()
+      assertEquals(1, statuses.length);
+      assertEquals(USER_DIR + "/alphaLinkLink", statuses[0].getPath()
           .toUri().getPath());
       statuses = wrap.globStatus(new Path(USER_DIR + "/alphaLinkLink/*"),
           new AcceptAllPathFilter());
-      Assert.assertEquals(1, statuses.length);
-      Assert.assertEquals(USER_DIR + "/alphaLinkLink/beta", statuses[0]
+      assertEquals(1, statuses.length);
+      assertEquals(USER_DIR + "/alphaLinkLink/beta", statuses[0]
           .getPath().toUri().getPath());
       // Test glob of dangling symlink (theta does not actually exist)
       wrap.createSymlink(new Path(USER_DIR + "theta"), new Path(USER_DIR
           + "/alpha/kappa"), false);
       statuses = wrap.globStatus(new Path(USER_DIR + "/alpha/kappa/kappa"),
           new AcceptAllPathFilter());
-      Assert.assertNull(statuses);
+      assertNull(statuses);
       // Test glob of symlinks
       wrap.createFile(USER_DIR + "/alpha/beta/gamma");
       wrap.createSymlink(new Path(USER_DIR + "gamma"), new Path(USER_DIR
@@ -975,12 +979,12 @@ public class TestGlobPaths {
           USER_DIR + "/alpha/beta/gammaLinkLinkLink"), false);
       statuses = wrap.globStatus(new Path(USER_DIR
           + "/alpha/*/gammaLinkLinkLink"), new AcceptAllPathFilter());
-      Assert.assertEquals(1, statuses.length);
-      Assert.assertEquals(USER_DIR + "/alpha/beta/gammaLinkLinkLink",
+      assertEquals(1, statuses.length);
+      assertEquals(USER_DIR + "/alpha/beta/gammaLinkLinkLink",
           statuses[0].getPath().toUri().getPath());
       statuses = wrap.globStatus(new Path(USER_DIR + "/alpha/beta/*"),
           new AcceptAllPathFilter());
-      Assert.assertEquals(USER_DIR + "/alpha/beta/gamma;" + USER_DIR
+      assertEquals(USER_DIR + "/alpha/beta/gamma;" + USER_DIR
           + "/alpha/beta/gammaLink;" + USER_DIR + "/alpha/beta/gammaLinkLink;"
           + USER_DIR + "/alpha/beta/gammaLinkLinkLink",
           TestPath.mergeStatuses(statuses));
@@ -992,17 +996,17 @@ public class TestGlobPaths {
       statuses = wrap.globStatus(
           new Path(USER_DIR + "/tweedledee/unobtainium"),
           new AcceptAllPathFilter());
-      Assert.assertNull(statuses);
+      assertNull(statuses);
     }
   }
 
-  @Ignore
+  @Disabled
   @Test
   public void testGlobWithSymlinksToSymlinksOnFS() throws Exception {
     testOnFileSystem(new TestGlobWithSymlinksToSymlinks(false));
   }
 
-  @Ignore
+  @Disabled
   @Test
   public void testGlobWithSymlinksToSymlinksOnFC() throws Exception {
     testOnFileContext(new TestGlobWithSymlinksToSymlinks(true));
@@ -1032,31 +1036,31 @@ public class TestGlobPaths {
       // PathFilter
       FileStatus statuses[] = wrap.globStatus(
           new Path(USER_DIR + "/alpha/beta"), new AcceptPathsEndingInZ());
-      Assert.assertNull(statuses);
+      assertNull(statuses);
       statuses = wrap.globStatus(new Path(USER_DIR + "/alphaLinkz/betaz"),
           new AcceptPathsEndingInZ());
-      Assert.assertEquals(1, statuses.length);
-      Assert.assertEquals(USER_DIR + "/alphaLinkz/betaz", statuses[0].getPath()
+      assertEquals(1, statuses.length);
+      assertEquals(USER_DIR + "/alphaLinkz/betaz", statuses[0].getPath()
           .toUri().getPath());
       statuses = wrap.globStatus(new Path(USER_DIR + "/*/*"),
           new AcceptPathsEndingInZ());
-      Assert.assertEquals(USER_DIR + "/alpha/betaz;" + USER_DIR
+      assertEquals(USER_DIR + "/alpha/betaz;" + USER_DIR
           + "/alphaLinkz/betaz", TestPath.mergeStatuses(statuses));
       statuses = wrap.globStatus(new Path(USER_DIR + "/*/*"),
           new AcceptAllPathFilter());
-      Assert.assertEquals(USER_DIR + "/alpha/beta;" + USER_DIR
+      assertEquals(USER_DIR + "/alpha/beta;" + USER_DIR
           + "/alpha/betaz;" + USER_DIR + "/alphaLinkz/beta;" + USER_DIR
           + "/alphaLinkz/betaz", TestPath.mergeStatuses(statuses));
     }
   }
 
-  @Ignore
+  @Disabled
   @Test
   public void testGlobSymlinksWithCustomPathFilterOnFS() throws Exception {
     testOnFileSystem(new TestGlobSymlinksWithCustomPathFilter(false));
   }
 
-  @Ignore
+  @Disabled
   @Test
   public void testGlobSymlinksWithCustomPathFilterOnFC() throws Exception {
     testOnFileContext(new TestGlobSymlinksWithCustomPathFilter(true));
@@ -1078,22 +1082,22 @@ public class TestGlobPaths {
           + "/alphaLink"), false);
       FileStatus statuses[] = wrap.globStatus(
           new Path(USER_DIR + "/alphaLink"), new AcceptAllPathFilter());
-      Assert.assertEquals(1, statuses.length);
+      assertEquals(1, statuses.length);
       Path path = statuses[0].getPath();
-      Assert.assertEquals(USER_DIR + "/alpha", path.toUri().getPath());
-      Assert.assertEquals("hdfs", path.toUri().getScheme());
+      assertEquals(USER_DIR + "/alpha", path.toUri().getPath());
+      assertEquals("hdfs", path.toUri().getScheme());
 
       // FileContext can list a file:/// URI.
       // Since everyone should have the root directory, we list that.
       statuses = fc.util().globStatus(new Path("file:///"),
           new AcceptAllPathFilter());
-      Assert.assertEquals(1, statuses.length);
+      assertEquals(1, statuses.length);
       Path filePath = statuses[0].getPath();
-      Assert.assertEquals("file", filePath.toUri().getScheme());
-      Assert.assertEquals("/", filePath.toUri().getPath());
+      assertEquals("file", filePath.toUri().getScheme());
+      assertEquals("/", filePath.toUri().getPath());
 
       // The FileSystem should have scheme 'hdfs'
-      Assert.assertEquals("hdfs", fs.getScheme());
+      assertEquals("hdfs", fs.getScheme());
     }
   }
 
@@ -1176,13 +1180,13 @@ public class TestGlobPaths {
       try {
         wrap.globStatus(new Path("/no*/*"),
             new AcceptAllPathFilter());
-        Assert.fail("expected to get an AccessControlException when " +
+        fail("expected to get an AccessControlException when " +
             "globbing through a directory we don't have permissions " +
             "to list.");
       } catch (AccessControlException ioe) {
       }
 
-      Assert.assertEquals("/norestrictions/val",
+      assertEquals("/norestrictions/val",
         TestPath.mergeStatuses(wrap.globStatus(
             new Path("/norestrictions/*"),
                 new AcceptAllPathFilter())));
@@ -1209,7 +1213,7 @@ public class TestGlobPaths {
 
     void run() throws Exception {
       String reservedRoot = "/.reserved/.inodes/" + INodeId.ROOT_INODE_ID;
-      Assert.assertEquals(reservedRoot,
+      assertEquals(reservedRoot,
         TestPath.mergeStatuses(wrap.
             globStatus(new Path(reservedRoot), new AcceptAllPathFilter())));
     }
@@ -1240,8 +1244,8 @@ public class TestGlobPaths {
       privWrap.setOwner(new Path("/"), newOwner, null);
       FileStatus[] status = 
           wrap.globStatus(rootPath, new AcceptAllPathFilter());
-      Assert.assertEquals(1, status.length);
-      Assert.assertEquals(newOwner, status[0].getOwner());
+      assertEquals(1, status.length);
+      assertEquals(newOwner, status[0].getOwner());
       privWrap.setOwner(new Path("/"), oldRootStatus.getOwner(), null);
     }
   }
@@ -1273,8 +1277,8 @@ public class TestGlobPaths {
         FileStatus[] statuses =
             wrap.globStatus(new Path("/filed*/alpha"),
                   new AcceptAllPathFilter());
-        Assert.assertEquals(1, statuses.length);
-        Assert.assertEquals("/filed_away/alpha", statuses[0].getPath()
+        assertEquals(1, statuses.length);
+        assertEquals("/filed_away/alpha", statuses[0].getPath()
             .toUri().getPath());
         privWrap.mkdir(new Path("/filed_away/alphabet"),
             new FsPermission((short)0777), true);
@@ -1282,8 +1286,8 @@ public class TestGlobPaths {
             new FsPermission((short)0777), true);
         statuses = wrap.globStatus(new Path("/filed*/alph*/*b*"),
                   new AcceptAllPathFilter());
-        Assert.assertEquals(1, statuses.length);
-        Assert.assertEquals("/filed_away/alphabet/abc", statuses[0].getPath()
+        assertEquals(1, statuses.length);
+        assertEquals("/filed_away/alphabet/abc", statuses[0].getPath()
             .toUri().getPath());
       } finally {
         privWrap.delete(new Path("/filed"), true);
@@ -1308,12 +1312,12 @@ public class TestGlobPaths {
     FileSystem fs = FileSystem.getLocal(conf);
     String localTmp = System.getProperty("java.io.tmpdir");
     Path base = new Path(new Path(localTmp), UUID.randomUUID().toString());
-    Assert.assertTrue(fs.mkdirs(base));
-    Assert.assertTrue(fs.mkdirs(new Path(base, "e")));
-    Assert.assertTrue(fs.mkdirs(new Path(base, "c")));
-    Assert.assertTrue(fs.mkdirs(new Path(base, "a")));
-    Assert.assertTrue(fs.mkdirs(new Path(base, "d")));
-    Assert.assertTrue(fs.mkdirs(new Path(base, "b")));
+    assertTrue(fs.mkdirs(base));
+    assertTrue(fs.mkdirs(new Path(base, "e")));
+    assertTrue(fs.mkdirs(new Path(base, "c")));
+    assertTrue(fs.mkdirs(new Path(base, "a")));
+    assertTrue(fs.mkdirs(new Path(base, "d")));
+    assertTrue(fs.mkdirs(new Path(base, "b")));
     fs.deleteOnExit(base);
     FileStatus[] status = fs.globStatus(new Path(base, "*"));
     ArrayList list = new ArrayList();
@@ -1321,7 +1325,7 @@ public class TestGlobPaths {
         list.add(f.getPath().toString());
     }
     boolean sorted = Ordering.natural().isOrdered(list);
-    Assert.assertTrue(sorted);
+    assertTrue(sorted);
   }
 }
 

+ 42 - 36
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestHDFSFileContextMainOperations.java

@@ -19,7 +19,11 @@
 package org.apache.hadoop.fs;
 
 import static org.apache.hadoop.fs.FileContextTestHelper.exists;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.IOException;
 import java.net.URI;
@@ -36,12 +40,11 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 
 public class TestHDFSFileContextMainOperations extends
     FileContextMainOperationsBaseTest {
@@ -54,7 +57,7 @@ public class TestHDFSFileContextMainOperations extends
     return new FileContextTestHelper("/tmp/TestHDFSFileContextMainOperations");
   }
 
-  @BeforeClass
+  @BeforeAll
   public static void clusterSetupAtBegining() throws IOException,
       LoginException, URISyntaxException {
     cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(2).build();
@@ -80,7 +83,7 @@ public class TestHDFSFileContextMainOperations extends
     fc.mkdir(defaultWorkingDirectory, FileContext.DEFAULT_PERM, true);
   }
       
-  @AfterClass
+  @AfterAll
   public static void ClusterShutdownAtEnd() throws Exception {
     if (cluster != null) {
       cluster.shutdown();
@@ -89,13 +92,13 @@ public class TestHDFSFileContextMainOperations extends
   }
   
   @Override
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     super.setUp();
   }
   
   @Override
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     super.tearDown();
   }
@@ -134,16 +137,15 @@ public class TestHDFSFileContextMainOperations extends
 
     boolean isReady = fc.truncate(file, newLength);
 
-    Assert.assertTrue("Recovery is not expected.", isReady);
+    assertTrue(isReady, "Recovery is not expected.");
 
     FileStatus fileStatus = fc.getFileStatus(file);
-    Assert.assertEquals(fileStatus.getLen(), newLength);
+    assertEquals(fileStatus.getLen(), newLength);
     AppendTestUtil.checkFullFile(fs, file, newLength, data, file.toString());
 
     ContentSummary cs = fs.getContentSummary(dir);
-    Assert.assertEquals("Bad disk space usage", cs.getSpaceConsumed(),
-        newLength * repl);
-    Assert.assertTrue(fs.delete(dir, true));
+    assertEquals(cs.getSpaceConsumed(), newLength * repl, "Bad disk space usage");
+    assertTrue(fs.delete(dir, true));
   }
 
   @Test
@@ -239,20 +241,24 @@ public class TestHDFSFileContextMainOperations extends
     rename(dst1, src1, false, true, Rename.OVERWRITE);
   }
   
-  @Test(expected = RemoteException.class)
+  @Test
   public void testRenameRoot() throws Exception {
-    Path src = getTestRootPath(fc, "test/testRenameRoot/srcdir/src1");
-    Path dst = new Path("/");
-    createFile(src);
-    rename(dst, src, true, true, Rename.OVERWRITE);
+    assertThrows(RemoteException.class, () -> {
+      Path src = getTestRootPath(fc, "test/testRenameRoot/srcdir/src1");
+      Path dst = new Path("/");
+      createFile(src);
+      rename(dst, src, true, true, Rename.OVERWRITE);
+    });
   }
 
-  @Test(expected = RemoteException.class)
+  @Test
   public void testRenameToRoot() throws Exception {
-    Path src = getTestRootPath(fc, "test/testRenameRoot/srcdir/src1");
-    Path dst = new Path("/");
-    createFile(src);
-    rename(src, dst, true, true, Rename.OVERWRITE);
+    assertThrows(RemoteException.class, () -> {
+      Path src = getTestRootPath(fc, "test/testRenameRoot/srcdir/src1");
+      Path dst = new Path("/");
+      createFile(src);
+      rename(src, dst, true, true, Rename.OVERWRITE);
+    });
   }
   
   /**
@@ -280,8 +286,8 @@ public class TestHDFSFileContextMainOperations extends
     fs = cluster.getFileSystem();
     src1 = getTestRootPath(fc, "testEditsLogOldRename/srcdir/src1");
     dst1 = getTestRootPath(fc, "testEditsLogOldRename/dstdir/dst1");
-    Assert.assertFalse(fs.exists(src1));   // ensure src1 is already renamed
-    Assert.assertTrue(fs.exists(dst1));    // ensure rename dst exists
+    assertFalse(fs.exists(src1));   // ensure src1 is already renamed
+    assertTrue(fs.exists(dst1));    // ensure rename dst exists
   }
   
   /**
@@ -309,8 +315,8 @@ public class TestHDFSFileContextMainOperations extends
     fs = cluster.getFileSystem();
     src1 = getTestRootPath(fc, "testEditsLogRename/srcdir/src1");
     dst1 = getTestRootPath(fc, "testEditsLogRename/dstdir/dst1");
-    Assert.assertFalse(fs.exists(src1));   // ensure src1 is already renamed
-    Assert.assertTrue(fs.exists(dst1));    // ensure rename dst exists
+    assertFalse(fs.exists(src1));   // ensure src1 is already renamed
+    assertTrue(fs.exists(dst1));    // ensure rename dst exists
   }
 
   @Test
@@ -323,8 +329,8 @@ public class TestHDFSFileContextMainOperations extends
     };
 
     for (String invalidName: invalidNames) {
-      Assert.assertFalse(invalidName + " is not valid",
-        fc.getDefaultFileSystem().isValidName(invalidName));
+      assertFalse(fc.getDefaultFileSystem().isValidName(invalidName),
+          invalidName + " is not valid");
     }
   }
 
@@ -332,12 +338,12 @@ public class TestHDFSFileContextMainOperations extends
       boolean exception) throws Exception {
     DistributedFileSystem fs = cluster.getFileSystem();
     try {
-      Assert.assertEquals(renameSucceeds, fs.rename(src, dst));
+      assertEquals(renameSucceeds, fs.rename(src, dst));
     } catch (Exception ex) {
-      Assert.assertTrue(exception);
+      assertTrue(exception);
     }
-    Assert.assertEquals(renameSucceeds, !exists(fc, src));
-    Assert.assertEquals(renameSucceeds, exists(fc, dst));
+    assertEquals(renameSucceeds, !exists(fc, src));
+    assertEquals(renameSucceeds, exists(fc, dst));
   }
   
   @Override

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestHdfsNativeCodeLoader.java

@@ -17,13 +17,14 @@
  */
 package org.apache.hadoop.fs;
 
-import org.junit.Test;
-import static org.junit.Assert.*;
+import org.junit.jupiter.api.Test;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.util.NativeCodeLoader;
 
+import static org.junit.jupiter.api.Assertions.fail;
+
 public class TestHdfsNativeCodeLoader {
   static final Logger LOG =
       LoggerFactory.getLogger(TestHdfsNativeCodeLoader.class);

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

@@ -19,7 +19,9 @@
 package org.apache.hadoop.fs;
 
 import java.io.File;
-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.FileNotFoundException;
 import java.io.IOException;
@@ -38,10 +40,9 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
 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;
 
 /**
  * Tests whether FileContext can resolve an hdfs path that has a symlink to
@@ -52,7 +53,7 @@ public class TestResolveHdfsSymlink {
   private static final FileContextTestHelper helper = new FileContextTestHelper();
   private static MiniDFSCluster cluster = null;
 
-  @BeforeClass
+  @BeforeAll
   public static void setUp() throws IOException {
     Configuration conf = new HdfsConfiguration();
     conf.setBoolean(
@@ -62,7 +63,7 @@ public class TestResolveHdfsSymlink {
 
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() {
     if (cluster != null) {
       cluster.shutdown();
@@ -99,11 +100,11 @@ public class TestResolveHdfsSymlink {
 
     Set<AbstractFileSystem> afsList = fcHdfs
         .resolveAbstractFileSystems(alphaHdfsPathViaLink);
-    Assert.assertEquals(2, afsList.size());
+    assertEquals(2, afsList.size());
     for (AbstractFileSystem afs : afsList) {
       if ((!afs.equals(fcHdfs.getDefaultFileSystem()))
           && (!afs.equals(fcLocal.getDefaultFileSystem()))) {
-        Assert.fail("Failed to resolve AFS correctly");
+        fail("Failed to resolve AFS correctly");
       }
     }
   }

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

@@ -24,8 +24,8 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
 import org.apache.hadoop.security.ssl.SSLFactory;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
 
 import javax.security.auth.login.LoginException;
 import java.io.File;
@@ -56,7 +56,7 @@ public class TestSWebHdfsFileContextMainOperations
   protected static final byte[] data = getFileData(numBlocks,
       getDefaultBlockSize());
 
-  @BeforeClass
+  @BeforeAll
   public static void clusterSetupAtBeginning()
       throws IOException, LoginException, URISyntaxException {
 
@@ -103,7 +103,7 @@ public class TestSWebHdfsFileContextMainOperations
     return webhdfsUrl;
   }
 
-  @AfterClass
+  @AfterAll
   public static void ClusterShutdownAtEnd() throws Exception {
     if (cluster != null) {
       cluster.shutdown();

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfsDisable.java

@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.fs;
 
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.IOException;
 
@@ -27,11 +27,13 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 public class TestSymlinkHdfsDisable {
 
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testSymlinkHdfsDisable() throws Exception {
     Configuration conf = new HdfsConfiguration();
     // disable symlink resolution

+ 17 - 18
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUnbuffer.java

@@ -26,19 +26,17 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.PeerCache;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.io.IOUtils;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
+import org.junit.jupiter.api.Test;
 import org.mockito.Mockito;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
 public class TestUnbuffer {
   private static final Logger LOG =
       LoggerFactory.getLogger(TestUnbuffer.class.getName());
 
-  @Rule
-  public ExpectedException exception = ExpectedException.none();
-
   /**
    * Test that calling Unbuffer closes sockets.
    */
@@ -72,19 +70,19 @@ public class TestUnbuffer {
       // Read a byte.  This will trigger the creation of a block reader.
       stream.seek(2);
       int b = stream.read();
-      Assert.assertTrue(-1 != b);
+      assertTrue(-1 != b);
 
       // The Peer cache should start off empty.
       PeerCache cache = dfs.getClient().getClientContext().getPeerCache();
-      Assert.assertEquals(0, cache.size());
+      assertEquals(0, cache.size());
 
       // Unbuffer should clear the block reader and return the socket to the
       // cache.
       stream.unbuffer();
       stream.seek(2);
-      Assert.assertEquals(1, cache.size());
+      assertEquals(1, cache.size());
       int b2 = stream.read();
-      Assert.assertEquals(b, b2);
+      assertEquals(b, b2);
     } finally {
       if (stream != null) {
         IOUtils.cleanupWithLogger(null, stream);
@@ -117,7 +115,7 @@ public class TestUnbuffer {
       for (int i = 0; i < NUM_OPENS; i++) {
         streams[i] = dfs.open(TEST_PATH);
         LOG.info("opening file " + i + "...");
-        Assert.assertTrue(-1 != streams[i].read());
+        assertTrue(-1 != streams[i].read());
         streams[i].unbuffer();
       }
     } finally {
@@ -147,11 +145,12 @@ public class TestUnbuffer {
     BuggyStream bs = Mockito.mock(BuggyStream.class);
     Mockito.when(bs.hasCapability(Mockito.anyString())).thenReturn(true);
 
-    exception.expect(UnsupportedOperationException.class);
-    exception.expectMessage(
-            StreamCapabilitiesPolicy.CAN_UNBUFFER_NOT_IMPLEMENTED_MESSAGE);
-
-    FSDataInputStream fs = new FSDataInputStream(bs);
-    fs.unbuffer();
+    UnsupportedOperationException exception = assertThrows(UnsupportedOperationException.class,
+        () -> {
+          FSDataInputStream fs = new FSDataInputStream(bs);
+          fs.unbuffer();
+        });
+    assertTrue(exception.getMessage().
+        contains(StreamCapabilitiesPolicy.CAN_UNBUFFER_NOT_IMPLEMENTED_MESSAGE));
   }
 }

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

@@ -17,9 +17,9 @@
  */
 package org.apache.hadoop.fs;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
+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 java.io.File;
 import java.io.IOException;
@@ -34,8 +34,8 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.test.PathUtils;
 
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 
 /**
  * Test of the URL stream handler.
@@ -48,7 +48,7 @@ public class TestUrlStreamHandler {
   private static final FsUrlStreamHandlerFactory HANDLER_FACTORY
       = new FsUrlStreamHandlerFactory();
 
-  @BeforeClass
+  @BeforeAll
   public static void setupHandler() {
 
     // Setup our own factory
@@ -166,20 +166,20 @@ public class TestUrlStreamHandler {
 
   @Test
   public void testHttpDefaultHandler() throws Throwable {
-    assertNull("Handler for HTTP is the Hadoop one",
-        HANDLER_FACTORY.createURLStreamHandler("http"));
+    assertNull(HANDLER_FACTORY.createURLStreamHandler("http"),
+        "Handler for HTTP is the Hadoop one");
   }
 
   @Test
   public void testHttpsDefaultHandler() throws Throwable {
-    assertNull("Handler for HTTPS is the Hadoop one",
-        HANDLER_FACTORY.createURLStreamHandler("https"));
+    assertNull(HANDLER_FACTORY.createURLStreamHandler("https"),
+        "Handler for HTTPS is the Hadoop one");
   }
 
   @Test
   public void testUnknownProtocol() throws Throwable {
-    assertNull("Unknown protocols are not handled",
-        HANDLER_FACTORY.createURLStreamHandler("gopher"));
+    assertNull(HANDLER_FACTORY.createURLStreamHandler("gopher"),
+        "Unknown protocols are not handled");
   }
 
 }

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

@@ -24,10 +24,10 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 
 import javax.security.auth.login.LoginException;
 import java.io.IOException;
@@ -38,8 +38,8 @@ import java.util.EnumSet;
 import static org.apache.hadoop.fs.CreateFlag.CREATE;
 import static org.apache.hadoop.fs.FileContextTestHelper.getDefaultBlockSize;
 import static org.apache.hadoop.fs.FileContextTestHelper.getFileData;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 /**
  * Test of FileContext apis on Webhdfs.
@@ -71,7 +71,7 @@ public class TestWebHdfsFileContextMainOperations
     return webhdfsUrl;
   }
 
-  @BeforeClass
+  @BeforeAll
   public static void clusterSetupAtBeginning()
       throws IOException, LoginException, URISyntaxException {
 
@@ -85,7 +85,7 @@ public class TestWebHdfsFileContextMainOperations
     fc.mkdir(defaultWorkingDirectory, FileContext.DEFAULT_PERM, true);
   }
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     URI webhdfsUrlReal = getWebhdfsUrl();
     Path testBuildData = new Path(
@@ -153,7 +153,7 @@ public class TestWebHdfsFileContextMainOperations
     assertArrayEquals(data, bb);
   }
 
-  @AfterClass
+  @AfterAll
   public static void ClusterShutdownAtEnd() throws Exception {
     if (cluster != null) {
       cluster.shutdown();

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/loadGenerator/TestLoadGenerator.java

@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.fs.loadGenerator;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.io.BufferedReader;
 import java.io.File;
@@ -33,7 +33,7 @@ import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 /**
  * This class tests if a balancer schedules tasks correctly.
  */

+ 11 - 11
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/permission/TestStickyBit.java

@@ -22,10 +22,10 @@ import static org.apache.hadoop.fs.permission.AclEntryScope.DEFAULT;
 import static org.apache.hadoop.fs.permission.AclEntryType.USER;
 import static org.apache.hadoop.fs.permission.FsAction.ALL;
 import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.aclEntry;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.IOException;
 import java.util.Arrays;
@@ -45,10 +45,10 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -66,7 +66,7 @@ public class TestStickyBit {
   private static FileSystem hdfsAsUser1;
   private static FileSystem hdfsAsUser2;
 
-  @BeforeClass
+  @BeforeAll
   public static void init() throws Exception {
     conf = new HdfsConfiguration();
     conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
@@ -85,7 +85,7 @@ public class TestStickyBit {
     assertTrue(hdfsAsUser2 instanceof DistributedFileSystem);
   }
 
-  @Before
+  @BeforeEach
   public void setup() throws Exception {
     if (hdfs != null) {
       for (FileStatus stat: hdfs.listStatus(new Path("/"))) {
@@ -94,7 +94,7 @@ public class TestStickyBit {
     }
   }
 
-  @AfterClass
+  @AfterAll
   public static void shutdown() throws Exception {
     IOUtils.cleanupWithLogger(null, hdfs, hdfsAsUser1, hdfsAsUser2);
     if (cluster != null) {

+ 7 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/shell/TestHdfsTextCommand.java

@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.fs.shell;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -34,9 +34,9 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-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;
 
 
 /**
@@ -49,7 +49,7 @@ public class TestHdfsTextCommand {
   private static MiniDFSCluster cluster;
   private static FileSystem fs;
   
-  @Before
+  @BeforeEach
     public void setUp() throws IOException{
     Configuration conf = new HdfsConfiguration();
     cluster = new MiniDFSCluster.Builder(conf).build();
@@ -57,7 +57,7 @@ public class TestHdfsTextCommand {
     fs = cluster.getFileSystem();
   }
 
-  @After
+  @AfterEach
     public void tearDown() throws IOException{
     if(fs != null){
       fs.close();

+ 10 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestNNStartupWhenViewFSOverloadSchemeEnabled.java

@@ -24,9 +24,10 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
-import org.junit.After;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 /**
  * Tests that the NN startup is successful with ViewFSOverloadScheme.
@@ -37,7 +38,7 @@ public class TestNNStartupWhenViewFSOverloadSchemeEnabled {
   private static final String HDFS_SCHEME = "hdfs";
   private static final Configuration CONF = new Configuration();
 
-  @BeforeClass
+  @BeforeAll
   public static void setUp() {
     CONF.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     CONF.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
@@ -57,7 +58,8 @@ public class TestNNStartupWhenViewFSOverloadSchemeEnabled {
    * Tests that the HA mode NameNode startup is successful when
    * ViewFSOverloadScheme configured.
    */
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testHANameNodeAndDataNodeStartup() throws Exception {
     cluster = new MiniDFSCluster.Builder(CONF)
         .nnTopology(MiniDFSNNTopology.simpleHATopology()).numDataNodes(0)
@@ -70,7 +72,8 @@ public class TestNNStartupWhenViewFSOverloadSchemeEnabled {
    * Tests that the NameNode startup is successful when ViewFSOverloadScheme
    * configured.
    */
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testNameNodeAndDataNodeStartup() throws Exception {
     cluster =
         new MiniDFSCluster.Builder(CONF).numDataNodes(0).waitSafeMode(false)
@@ -78,7 +81,7 @@ public class TestNNStartupWhenViewFSOverloadSchemeEnabled {
     cluster.waitActive();
   }
 
-  @After
+  @AfterEach
   public void shutdownCluster() {
     if (cluster != null) {
       cluster.shutdown();

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFSOverloadSchemeWithMountTableConfigInHDFS.java

@@ -25,7 +25,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.junit.Before;
+import org.junit.jupiter.api.BeforeEach;
 
 /**
  * Tests ViewFileSystemOverloadScheme with configured mount links.
@@ -35,7 +35,7 @@ public class TestViewFSOverloadSchemeWithMountTableConfigInHDFS
   private Path oldVersionMountTablePath;
   private Path newVersionMountTablePath;
 
-  @Before
+  @BeforeEach
   @Override
   public void setUp() throws IOException {
     super.setUp();

+ 6 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemAtHdfsRoot.java

@@ -29,9 +29,9 @@ import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
 
 /**
  * Make sure that ViewFileSystem works when the root of an FS is mounted to a
@@ -48,7 +48,7 @@ public class TestViewFileSystemAtHdfsRoot extends ViewFileSystemBaseTest {
     return new FileSystemTestHelper("/tmp/TestViewFileSystemAtHdfsRoot");
   }
   
-  @BeforeClass
+  @BeforeAll
   public static void clusterSetupAtBegining() throws IOException,
       LoginException, URISyntaxException {
     SupportsBlocks = true;
@@ -63,7 +63,7 @@ public class TestViewFileSystemAtHdfsRoot extends ViewFileSystemBaseTest {
     fHdfs = cluster.getFileSystem();
   }
       
-  @AfterClass
+  @AfterAll
   public static void clusterShutdownAtEnd() throws Exception {
     if (cluster != null) {
       cluster.shutdown();
@@ -71,7 +71,7 @@ public class TestViewFileSystemAtHdfsRoot extends ViewFileSystemBaseTest {
   }
 
   @Override
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     fsTarget = fHdfs;
     super.setUp();

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemClose.java

@@ -24,7 +24,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.test.AbstractHadoopTestBase;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 

+ 31 - 31
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java

@@ -58,12 +58,12 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IPC_CLIENT_CONN
 import static org.apache.hadoop.fs.FileSystem.TRASH_PREFIX;
 
 import org.apache.hadoop.test.LambdaTestUtils;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import static org.junit.Assert.*;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import static org.junit.jupiter.api.Assertions.*;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -87,7 +87,7 @@ public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
     return new FileSystemTestHelper("/tmp/TestViewFileSystemHdfs");
   }
 
-  @BeforeClass
+  @BeforeAll
   public static void clusterSetupAtBegining() throws IOException,
       LoginException, URISyntaxException {
 
@@ -131,7 +131,7 @@ public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
   }
 
       
-  @AfterClass
+  @AfterAll
   public static void ClusterShutdownAtEnd() throws Exception {
     if (cluster != null) {
       cluster.shutdown();
@@ -139,7 +139,7 @@ public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
   }
 
   @Override
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     // create the test root on local_fs
     fsTarget = fHdfs;
@@ -149,7 +149,7 @@ public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
   }
 
   @Override
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     super.tearDown();
   }
@@ -214,13 +214,13 @@ public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
 
       //Verify file deletion within EZ
       DFSTestUtil.verifyDelete(shell, fsTarget, encFile, true);
-      assertTrue("ViewFileSystem trash roots should include EZ file trash",
-          (fsView.getTrashRoots(true).size() == 1));
+      assertTrue((fsView.getTrashRoots(true).size() == 1),
+          "ViewFileSystem trash roots should include EZ file trash");
 
       //Verify deletion of EZ
       DFSTestUtil.verifyDelete(shell, fsTarget, zone, true);
-      assertTrue("ViewFileSystem trash roots should include EZ zone trash",
-          (fsView.getTrashRoots(true).size() == 2));
+      assertTrue((fsView.getTrashRoots(true).size() == 2),
+          "ViewFileSystem trash roots should include EZ zone trash");
     } finally {
       DFSTestUtil.deleteKey("test_key", cluster);
     }
@@ -266,15 +266,15 @@ public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
         viewFs.getFileChecksum(mountDataFilePath);
     FileChecksum fileChecksumViaTargetFs =
         fsTarget.getFileChecksum(fsTargetFilePath);
-    assertTrue("File checksum not matching!",
-        fileChecksumViaViewFs.equals(fileChecksumViaTargetFs));
+    assertTrue(fileChecksumViaViewFs.equals(fileChecksumViaTargetFs),
+        "File checksum not matching!");
 
     fileChecksumViaViewFs =
         viewFs.getFileChecksum(mountDataFilePath, fileLength / 2);
     fileChecksumViaTargetFs =
         fsTarget.getFileChecksum(fsTargetFilePath, fileLength / 2);
-    assertTrue("File checksum not matching!",
-        fileChecksumViaViewFs.equals(fileChecksumViaTargetFs));
+    assertTrue(fileChecksumViaViewFs.equals(fileChecksumViaTargetFs),
+        "File checksum not matching!");
   }
 
   //Rename should fail on across different fileSystems
@@ -338,7 +338,7 @@ public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
     // 1. test mkdirs
     final Path testDir = new Path("testdir1/sub1/sub3");
     final Path testDir_tmp = new Path("testdir1/sub1/sub3_temp");
-    assertTrue(testDir + ": Failed to create!", nfly.mkdirs(testDir));
+    assertTrue(nfly.mkdirs(testDir), testDir + ": Failed to create!");
 
     // Test renames
     assertTrue(nfly.rename(testDir, testDir_tmp));
@@ -346,7 +346,7 @@ public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
 
     for (final URI testUri : testUris) {
       final FileSystem fs = FileSystem.get(testUri, testConf);
-      assertTrue(testDir + " should exist!", fs.exists(testDir));
+      assertTrue(fs.exists(testDir), testDir + " should exist!");
     }
 
     // 2. test write
@@ -362,7 +362,7 @@ public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
       final FileSystem fs = FileSystem.get(testUri, testConf);
       final FSDataInputStream fsdis = fs.open(testFile);
       try {
-        assertEquals("Wrong file content", testString, fsdis.readUTF());
+        assertEquals(testString, fsdis.readUTF(), "Wrong file content");
       } finally {
         fsdis.close();
       }
@@ -377,7 +377,7 @@ public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
       FSDataInputStream fsDis = null;
       try {
         fsDis = nfly.open(testFile);
-        assertEquals("Wrong file content", testString, fsDis.readUTF());
+        assertEquals(testString, fsDis.readUTF(), "Wrong file content");
       } finally {
         IOUtils.cleanupWithLogger(LOG, fsDis);
         cluster.restartNameNode(i);
@@ -391,7 +391,7 @@ public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
     FSDataInputStream fsDis = null;
     try {
       fsDis = nfly.open(testFile);
-      assertEquals("Wrong file content", testString, fsDis.readUTF());
+      assertEquals(testString, fsDis.readUTF(), "Wrong file content");
       assertTrue(fs1.exists(testFile));
     } finally {
       IOUtils.cleanupWithLogger(LOG, fsDis);
@@ -406,18 +406,18 @@ public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
       for (final URI testUri : testUris) {
         final FileSystem fs = FileSystem.get(testUri, conf);
         fs.setTimes(testFile, 1L, 1L);
-        assertEquals(testUri + "Set mtime failed!", 1L,
-            fs.getFileStatus(testFile).getModificationTime());
-        assertEquals("nfly file status wrong", expectedMtime,
-            nfly.getFileStatus(testFile).getModificationTime());
+        assertEquals(1L, fs.getFileStatus(testFile).getModificationTime(),
+            testUri + "Set mtime failed!");
+        assertEquals(expectedMtime, nfly.getFileStatus(testFile).getModificationTime(),
+            "nfly file status wrong");
         FSDataInputStream fsDis2 = null;
         try {
           fsDis2 = nfly.open(testFile);
-          assertEquals("Wrong file content", testString, fsDis2.readUTF());
+          assertEquals(testString, fsDis2.readUTF(), "Wrong file content");
           // repair is done, now trying via normal fs
           //
-          assertEquals("Repair most recent failed!", expectedMtime,
-              fs.getFileStatus(testFile).getModificationTime());
+          assertEquals(expectedMtime, fs.getFileStatus(testFile).getModificationTime(),
+              "Repair most recent failed!");
         } finally {
           IOUtils.cleanupWithLogger(LOG, fsDis2);
         }
@@ -489,7 +489,7 @@ public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
     FileSystem otherfs = map.get("user1");
     otherfs.mkdirs(user1Path);
     String owner = otherfs.getFileStatus(user1Path).getOwner();
-    assertEquals("The owner did not match ", owner, userUgi.getShortUserName());
+    assertEquals(owner, userUgi.getShortUserName(), "The owner did not match ");
     otherfs.delete(user1Path, false);
   }
 

+ 66 - 71
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkFallback.java

@@ -18,12 +18,12 @@
 package org.apache.hadoop.fs.viewfs;
 
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.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.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import java.io.File;
 import java.io.IOException;
 import java.net.URI;
@@ -45,11 +45,10 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -79,7 +78,7 @@ public class TestViewFileSystemLinkFallback extends ViewFileSystemBaseTest {
     return new FileSystemTestHelper(TEST_BASE_PATH);
   }
 
-  @BeforeClass
+  @BeforeAll
   public static void clusterSetupAtBeginning() throws IOException,
       LoginException, URISyntaxException {
     SupportsBlocks = true;
@@ -100,7 +99,7 @@ public class TestViewFileSystemLinkFallback extends ViewFileSystemBaseTest {
         Constants.CONFIG_VIEWFS_DEFAULT_MOUNT_TABLE, "/", null, null);
   }
 
-  @AfterClass
+  @AfterAll
   public static void clusterShutdownAtEnd() throws Exception {
     if (cluster != null) {
       cluster.shutdown();
@@ -108,7 +107,7 @@ public class TestViewFileSystemLinkFallback extends ViewFileSystemBaseTest {
   }
 
   @Override
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     fsTarget = fsDefault;
     super.setUp();
@@ -179,10 +178,10 @@ public class TestViewFileSystemLinkFallback extends ViewFileSystemBaseTest {
     FileStatus baseFileRelStat = vfs.getFileStatus(new Path(viewFsUri.toString()
         + testBaseFileRelative.toUri().toString()));
     LOG.info("BaseFileRelStat: " + baseFileRelStat);
-    Assert.assertEquals("Unexpected file length for " + testBaseFile,
-        1, baseFileStat.getLen());
-    Assert.assertEquals("Unexpected file length for " + testBaseFileRelative,
-        baseFileStat.getLen(), baseFileRelStat.getLen());
+    assertEquals(1, baseFileStat.getLen(),
+        "Unexpected file length for " + testBaseFile);
+    assertEquals(baseFileStat.getLen(), baseFileRelStat.getLen(),
+        "Unexpected file length for " + testBaseFileRelative);
     FileStatus level2FileStat = vfs.getFileStatus(new Path(viewFsUri.toString()
         + testLevel2File.toUri().toString()));
     LOG.info("Level2FileStat: " + level2FileStat);
@@ -228,8 +227,8 @@ public class TestViewFileSystemLinkFallback extends ViewFileSystemBaseTest {
     FileStatus baseFileStat = vfs.getFileStatus(
         new Path(viewFsUri.toString() + testBaseFile.toUri().toString()));
     LOG.info("BaseFileStat: " + baseFileStat);
-    Assert.assertEquals("Unexpected file length for " + testBaseFile,
-        0, baseFileStat.getLen());
+    assertEquals(0, baseFileStat.getLen(),
+        "Unexpected file length for " + testBaseFile);
     FileStatus level2FileStat = vfs.getFileStatus(new Path(viewFsUri.toString()
         + testLevel2File.toUri().toString()));
     LOG.info("Level2FileStat: " + level2FileStat);
@@ -240,8 +239,8 @@ public class TestViewFileSystemLinkFallback extends ViewFileSystemBaseTest {
 
     FileStatus level2FileStatAfterWrite = vfs.getFileStatus(
         new Path(viewFsUri.toString() + testLevel2File.toUri().toString()));
-    Assert.assertTrue("Unexpected file length for " + testLevel2File,
-        level2FileStatAfterWrite.getLen() > level2FileStat.getLen());
+    assertTrue(level2FileStatAfterWrite.getLen() > level2FileStat.getLen(),
+        "Unexpected file length for " + testLevel2File);
 
     vfs.close();
   }
@@ -265,8 +264,8 @@ public class TestViewFileSystemLinkFallback extends ViewFileSystemBaseTest {
       FileSystem.get(viewFsUri, conf);
       fail("Shouldn't allow linkMergeSlash to take extra mount points!");
     } catch (IOException e) {
-      assertTrue("Unexpected error: " + e.getMessage(),
-          e.getMessage().contains(expectedErrorMsg));
+      assertTrue(e.getMessage().contains(expectedErrorMsg),
+          "Unexpected error: " + e.getMessage());
     }
   }
 
@@ -299,13 +298,12 @@ public class TestViewFileSystemLinkFallback extends ViewFileSystemBaseTest {
         afterFallback.add(stat.getPath());
       }
       afterFallback.removeAll(beforeFallback);
-      assertTrue("Listing didn't include fallback link",
-          afterFallback.size() == 1);
+      assertTrue(afterFallback.size() == 1,
+          "Listing didn't include fallback link");
       Path[] fallbackArray = new Path[afterFallback.size()];
       afterFallback.toArray(fallbackArray);
       Path expected = new Path(viewFsUri.toString(), "dir1");
-      assertEquals("Path did not match",
-          expected, fallbackArray[0]);
+      assertEquals(expected, fallbackArray[0], "Path did not match");
 
       // Create a directory using the returned fallback path and verify
       Path childDir = new Path(fallbackArray[0], "child");
@@ -349,13 +347,12 @@ public class TestViewFileSystemLinkFallback extends ViewFileSystemBaseTest {
         afterFallback.add(stat.getPath());
       }
       afterFallback.removeAll(beforeFallback);
-      assertEquals("The same directory name in fallback link should be shaded",
-          1, afterFallback.size());
+      assertEquals(1, afterFallback.size(),
+          "The same directory name in fallback link should be shaded");
       Path[] fallbackArray = new Path[afterFallback.size()];
       // Only user1 should be listed as fallback link
       Path expected = new Path(viewFsDefaultClusterUri.toString(), "user1");
-      assertEquals("Path did not match",
-          expected, afterFallback.toArray(fallbackArray)[0]);
+      assertEquals(expected, afterFallback.toArray(fallbackArray)[0], "Path did not match");
 
       // Create a directory using the returned fallback path and verify
       Path childDir = new Path(fallbackArray[0], "child");
@@ -430,8 +427,8 @@ public class TestViewFileSystemLinkFallback extends ViewFileSystemBaseTest {
       }
       //viewfs://default/user1/hive/warehouse
       afterFallback.removeAll(beforeFallback);
-      assertEquals("The same directory name in fallback link should be shaded",
-          1, afterFallback.size());
+      assertEquals(1, afterFallback.size(),
+          "The same directory name in fallback link should be shaded");
     }
   }
 
@@ -502,8 +499,7 @@ public class TestViewFileSystemLinkFallback extends ViewFileSystemBaseTest {
         }
       }
       afterFallback.removeAll(beforeFallback);
-      assertEquals("Just to make sure paths are same.", 0,
-          afterFallback.size());
+      assertEquals(0, afterFallback.size(), "Just to make sure paths are same.");
     }
   }
 
@@ -559,14 +555,14 @@ public class TestViewFileSystemLinkFallback extends ViewFileSystemBaseTest {
           assertEquals(FsPermission.valueOf("-rwxr--rw-"),
               stat.getPermission());
         } else {
-          assertEquals("Path is: " + stat.getPath(),
-              FsPermission.valueOf("-rwxr--r--"), stat.getPermission());
+          assertEquals(FsPermission.valueOf("-rwxr--r--"),
+              stat.getPermission(), "Path is: " + stat.getPath());
         }
       }
       afterFallback.removeAll(beforeFallback);
       assertEquals(1, afterFallback.size());
-      assertEquals("/user2 dir from fallback should be listed.", "user2",
-          afterFallback.iterator().next().getName());
+      assertEquals("user2", afterFallback.iterator().next().getName(),
+          "/user2 dir from fallback should be listed.");
     }
   }
 
@@ -849,45 +845,44 @@ public class TestViewFileSystemLinkFallback extends ViewFileSystemBaseTest {
    * Tests the create of a file on root where the path is matching to an
    * existing file on fallback's file on root.
    */
-  @Test (expected = FileAlreadyExistsException.class)
+  @Test
   public void testCreateFileOnRootWithFallbackWithFileAlreadyExist()
       throws Exception {
-    Configuration conf = new Configuration();
-    Path fallbackTarget = new Path(targetTestRoot, "fallbackDir");
-    Path testFile = new Path(fallbackTarget, "test.file");
-    // pre-creating test file in fallback.
-    fsTarget.create(testFile).close();
-
-    ConfigUtil.addLink(conf, "/user1/hive/",
-        new Path(targetTestRoot.toString()).toUri());
-    ConfigUtil.addLinkFallback(conf, fallbackTarget.toUri());
-
-    try (FileSystem vfs = FileSystem.get(viewFsDefaultClusterUri, conf)) {
-      Path vfsTestFile = new Path("/test.file");
-      assertTrue(fsTarget.exists(testFile));
-      vfs.create(vfsTestFile, false).close();
-    }
+    assertThrows(FileAlreadyExistsException.class, () -> {
+      Configuration conf = new Configuration();
+      Path fallbackTarget = new Path(targetTestRoot, "fallbackDir");
+      Path testFile = new Path(fallbackTarget, "test.file");
+      fsTarget.create(testFile).close();
+      ConfigUtil.addLink(conf, "/user1/hive/",
+          new Path(targetTestRoot.toString()).toUri());
+      ConfigUtil.addLinkFallback(conf, fallbackTarget.toUri());
+      try (FileSystem vfs = FileSystem.get(viewFsDefaultClusterUri, conf)) {
+        Path vfsTestFile = new Path("/test.file");
+        assertTrue(fsTarget.exists(testFile));
+        vfs.create(vfsTestFile, false).close();
+      }
+    });
   }
 
   /**
    * Tests the creating of a file where the path is same as mount link path.
    */
-  @Test(expected= FileAlreadyExistsException.class)
+  @Test
   public void testCreateFileWhereThePathIsSameAsItsMountLinkPath()
       throws Exception {
-    Configuration conf = new Configuration();
-    Path fallbackTarget = new Path(targetTestRoot, "fallbackDir");
-    fsTarget.mkdirs(fallbackTarget);
-
-    ConfigUtil.addLink(conf, "/user1/hive/",
-        new Path(targetTestRoot.toString()).toUri());
-    ConfigUtil.addLinkFallback(conf, fallbackTarget.toUri());
-
-    try (FileSystem vfs = FileSystem.get(viewFsDefaultClusterUri, conf)) {
-      Path vfsTestDir = new Path("/user1/hive");
-      assertFalse(fsTarget.exists(Path.mergePaths(fallbackTarget, vfsTestDir)));
-      vfs.create(vfsTestDir).close();
-    }
+    assertThrows(FileAlreadyExistsException.class, () -> {
+      Configuration conf = new Configuration();
+      Path fallbackTarget = new Path(targetTestRoot, "fallbackDir");
+      fsTarget.mkdirs(fallbackTarget);
+      ConfigUtil.addLink(conf, "/user1/hive/",
+          new Path(targetTestRoot.toString()).toUri());
+      ConfigUtil.addLinkFallback(conf, fallbackTarget.toUri());
+      try (FileSystem vfs = FileSystem.get(viewFsDefaultClusterUri, conf)) {
+        Path vfsTestDir = new Path("/user1/hive");
+        assertFalse(fsTarget.exists(Path.mergePaths(fallbackTarget, vfsTestDir)));
+        vfs.create(vfsTestDir).close();
+      }
+    });
   }
 
   /**
@@ -908,7 +903,7 @@ public class TestViewFileSystemLinkFallback extends ViewFileSystemBaseTest {
       assertFalse(fsTarget.exists(Path.mergePaths(fallbackTarget, vfsTestDir)));
       try {
         vfs.create(vfsTestDir);
-        Assert.fail("Should fail to create file as this is an internal dir.");
+        fail("Should fail to create file as this is an internal dir.");
       } catch (NotInMountpointException e){
         // This tree is part of internal tree. The above exception will be
         // thrown from getDefaultReplication, getDefaultBlockSize APIs which was

+ 15 - 18
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkMergeSlash.java

@@ -34,17 +34,16 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-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 javax.security.auth.login.LoginException;
 
@@ -74,7 +73,7 @@ public class TestViewFileSystemLinkMergeSlash extends ViewFileSystemBaseTest {
     return new FileSystemTestHelper(TEST_TEMP_PATH);
   }
 
-  @BeforeClass
+  @BeforeAll
   public static void clusterSetupAtBeginning() throws IOException,
       LoginException, URISyntaxException {
     SupportsBlocks = true;
@@ -93,7 +92,7 @@ public class TestViewFileSystemLinkMergeSlash extends ViewFileSystemBaseTest {
     fsDefault = FS_HDFS[FS_INDEX_DEFAULT];
   }
 
-  @AfterClass
+  @AfterAll
   public static void clusterShutdownAtEnd() throws Exception {
     if (cluster != null) {
       cluster.shutdown();
@@ -101,7 +100,7 @@ public class TestViewFileSystemLinkMergeSlash extends ViewFileSystemBaseTest {
   }
 
   @Override
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     fsTarget = fsDefault;
     super.setUp();
@@ -191,9 +190,8 @@ public class TestViewFileSystemLinkMergeSlash extends ViewFileSystemBaseTest {
       fail("Shouldn't allow both merge slash link and regular link on same "
           + "mount table.");
     } catch (IOException e) {
-      assertTrue("Unexpected error message: " + e.getMessage(),
-          e.getMessage().contains(expectedErrorMsg1) || e.getMessage()
-              .contains(expectedErrorMsg2));
+      assertTrue(e.getMessage().contains(expectedErrorMsg1) || e.getMessage()
+              .contains(expectedErrorMsg2), "Unexpected error message: " + e.getMessage());
     }
   }
 
@@ -226,9 +224,8 @@ public class TestViewFileSystemLinkMergeSlash extends ViewFileSystemBaseTest {
         LINK_MERGE_SLASH_CLUSTER_1_NAME, "/", null, null);
     FileSystem fs = FileSystem.get(viewFsUri, conf);
     FileSystem[] childFs = fs.getChildFileSystems();
-    Assert.assertEquals("Unexpected number of child filesystems!",
-        1, childFs.length);
-    Assert.assertEquals("Unexpected child filesystem!",
-        DistributedFileSystem.class, childFs[0].getClass());
+    assertEquals(1, childFs.length, "Unexpected number of child filesystems!");
+    assertEquals(DistributedFileSystem.class, childFs[0].getClass(),
+        "Unexpected child filesystem!");
   }
 }

+ 14 - 15
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java

@@ -36,16 +36,17 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.apache.hadoop.fs.viewfs.RegexMountPoint.INTERCEPTOR_INTERNAL_SEP;
-import static org.junit.Assert.assertSame;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 /**
  * Test linkRegex node type for view file system.
@@ -73,7 +74,7 @@ public class TestViewFileSystemLinkRegex extends ViewFileSystemBaseTest {
     return new FileSystemTestHelper(TEST_BASE_PATH);
   }
 
-  @BeforeClass
+  @BeforeAll
   public static void clusterSetupAtBeginning() throws IOException {
     SupportsBlocks = true;
     clusterConfig = ViewFileSystemTestSetup.createConfig();
@@ -91,7 +92,7 @@ public class TestViewFileSystemLinkRegex extends ViewFileSystemBaseTest {
     fsDefault = FS_HDFS[FS_INDEX_DEFAULT];
   }
 
-  @AfterClass
+  @AfterAll
   public static void clusterShutdownAtEnd() throws Exception {
     if (cluster != null) {
       cluster.shutdown();
@@ -99,7 +100,7 @@ public class TestViewFileSystemLinkRegex extends ViewFileSystemBaseTest {
   }
 
   @Override
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     fsTarget = fsDefault;
     super.setUp();
@@ -157,7 +158,7 @@ public class TestViewFileSystemLinkRegex extends ViewFileSystemBaseTest {
   private void createDirWithChildren(
       FileSystem fileSystem, Path dir, List<Path> childrenFiles)
       throws IOException {
-    Assert.assertTrue(fileSystem.mkdirs(dir));
+    assertTrue(fileSystem.mkdirs(dir));
     int index = 0;
     for (Path childFile : childrenFiles) {
       createFile(fileSystem, childFile, index, true);
@@ -224,12 +225,10 @@ public class TestViewFileSystemLinkRegex extends ViewFileSystemBaseTest {
     URI viewFsUri = new URI(
         FsConstants.VIEWFS_SCHEME, CLUSTER_NAME, "/", null, null);
     try (FileSystem vfs = FileSystem.get(viewFsUri, config)) {
-      Assert.assertEquals(expectedResolveResult.toString(),
+      assertEquals(expectedResolveResult.toString(),
           vfs.resolvePath(dirPathBeforeMountPoint).toString());
-      Assert.assertTrue(
-          vfs.getFileStatus(dirPathBeforeMountPoint).isDirectory());
-      Assert.assertEquals(
-          childrenFilesCnt, vfs.listStatus(dirPathBeforeMountPoint).length);
+      assertTrue(vfs.getFileStatus(dirPathBeforeMountPoint).isDirectory());
+      assertEquals(childrenFilesCnt, vfs.listStatus(dirPathBeforeMountPoint).length);
 
       // Test Inner cache, the resolved result's filesystem should be the same.
       ViewFileSystem viewFileSystem = (ViewFileSystem) vfs;

+ 18 - 13
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemOverloadSchemeHdfsFileSystemContract.java

@@ -19,7 +19,8 @@ package org.apache.hadoop.fs.viewfs;
 
 import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_IGNORE_PORT_IN_MOUNT_TABLE_NAME;
 import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_IGNORE_PORT_IN_MOUNT_TABLE_NAME_DEFAULT;
-import static org.junit.Assume.assumeTrue;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assumptions.assumeTrue;
 
 import java.io.File;
 import java.io.IOException;
@@ -39,11 +40,12 @@ import org.apache.hadoop.hdfs.TestHDFSFileSystemContract;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 /**
  * Tests ViewFileSystemOverloadScheme with file system contract tests.
@@ -55,7 +57,7 @@ public class TestViewFileSystemOverloadSchemeHdfsFileSystemContract
   private static String defaultWorkingDirectory;
   private static Configuration conf = new HdfsConfiguration();
 
-  @BeforeClass
+  @BeforeAll
   public static void init() throws IOException {
     final File basedir = GenericTestUtils.getRandomizedTestDir();
     conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY,
@@ -67,7 +69,7 @@ public class TestViewFileSystemOverloadSchemeHdfsFileSystemContract
         "/user/" + UserGroupInformation.getCurrentUser().getShortUserName();
   }
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     conf.set(String.format("fs.%s.impl", "hdfs"),
         ViewFileSystemOverloadScheme.class.getName());
@@ -89,7 +91,7 @@ public class TestViewFileSystemOverloadSchemeHdfsFileSystemContract
     fs = FileSystem.get(conf);
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDownAfter() throws Exception {
     if (cluster != null) {
       cluster.shutdown();
@@ -103,15 +105,18 @@ public class TestViewFileSystemOverloadSchemeHdfsFileSystemContract
   }
 
   @Override
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testAppend() throws IOException {
     AppendTestUtil.testAppend(fs, new Path("/append/f"));
   }
 
   @Override
-  @Test(expected = AccessControlException.class)
+  @Test
   public void testRenameRootDirForbidden() throws Exception {
-    super.testRenameRootDirForbidden();
+    assertThrows(AccessControlException.class, () -> {
+      super.testRenameRootDirForbidden();
+    });
   }
 
   @Override
@@ -129,7 +134,7 @@ public class TestViewFileSystemOverloadSchemeHdfsFileSystemContract
   }
 
   @Override
-  @Ignore // This test same as above in this case.
+  @Disabled // This test same as above in this case.
   public void testLSRootDir() throws Throwable {
   }
 }

+ 105 - 87
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemOverloadSchemeWithHdfsScheme.java

@@ -39,16 +39,20 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.test.LambdaTestUtils;
 import org.apache.hadoop.test.PathUtils;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_IGNORE_PORT_IN_MOUNT_TABLE_NAME;
 import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_IGNORE_PORT_IN_MOUNT_TABLE_NAME_DEFAULT;
-import static org.junit.Assert.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 
 /**
@@ -67,7 +71,7 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
   private static final String HDFS_USER_FOLDER = "/HDFSUser";
   private static final String LOCAL_FOLDER = "/local";
 
-  @BeforeClass
+  @BeforeAll
   public static void init() throws IOException {
     cluster =
         new MiniDFSCluster.Builder(new Configuration()).numDataNodes(2).build();
@@ -77,7 +81,7 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
   /**
    * Sets up the configurations and starts the MiniDFSCluster.
    */
-  @Before
+  @BeforeEach
   public void setUp() throws IOException {
     Configuration config = getNewConf();
     config.setInt(
@@ -91,10 +95,10 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
         URI.create(config.get(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY));
     localTargetDir = new File(TEST_ROOT_DIR, "/root/");
     localTargetDir.mkdirs();
-    Assert.assertEquals(HDFS_SCHEME, defaultFSURI.getScheme()); // hdfs scheme.
+    assertEquals(HDFS_SCHEME, defaultFSURI.getScheme()); // hdfs scheme.
   }
 
-  @After
+  @AfterEach
   public void cleanUp() throws IOException {
     if (cluster != null) {
       FileSystem fs = new DistributedFileSystem();
@@ -102,7 +106,7 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
       try {
         FileStatus[] statuses = fs.listStatus(new Path("/"));
         for (FileStatus st : statuses) {
-          Assert.assertTrue(fs.delete(st.getPath(), true));
+          assertTrue(fs.delete(st.getPath(), true));
         }
       } finally {
         fs.close();
@@ -111,7 +115,7 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
     }
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() throws IOException {
     if (cluster != null) {
       FileSystem.closeAll();
@@ -136,7 +140,8 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
    * create file /HDFSUser/testfile should create in hdfs
    * create file /local/test should create directory in local fs
    */
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testMountLinkWithLocalAndHDFS() throws Exception {
     final Path hdfsTargetPath = new Path(defaultFSURI + HDFS_USER_FOLDER);
     final Path localTragetPath = new Path(localTargetDir.toURI());
@@ -154,7 +159,7 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
 
     try (FileSystem fs
         =  FileSystem.get(conf)) {
-      Assert.assertEquals(2, fs.getChildFileSystems().length);
+      assertEquals(2, fs.getChildFileSystems().length);
       fs.createNewFile(hdfsFile); // /HDFSUser/testfile
       fs.mkdirs(localDir); // /local/test
     }
@@ -162,20 +167,20 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
     // Initialize HDFS and test files exist in ls or not
     try (DistributedFileSystem dfs = new DistributedFileSystem()) {
       dfs.initialize(defaultFSURI, conf);
-      Assert.assertTrue(dfs.exists(
+      assertTrue(dfs.exists(
           new Path(Path.getPathWithoutSchemeAndAuthority(hdfsTargetPath),
               hdfsFile.getName()))); // should be in hdfs.
-      Assert.assertFalse(dfs.exists(
+      assertFalse(dfs.exists(
           new Path(Path.getPathWithoutSchemeAndAuthority(localTragetPath),
               localDir.getName()))); // should not be in local fs.
     }
 
     try (RawLocalFileSystem lfs = new RawLocalFileSystem()) {
       lfs.initialize(localTragetPath.toUri(), conf);
-      Assert.assertFalse(lfs.exists(
+      assertFalse(lfs.exists(
           new Path(Path.getPathWithoutSchemeAndAuthority(hdfsTargetPath),
               hdfsFile.getName()))); // should not be in hdfs.
-      Assert.assertTrue(lfs.exists(
+      assertTrue(lfs.exists(
           new Path(Path.getPathWithoutSchemeAndAuthority(localTragetPath),
               localDir.getName()))); // should be in local fs.
     }
@@ -186,7 +191,8 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
    * hdfs://localhost:xxx/HDFSUser --> nonexistent://NonExistent/User/
    * It should fail to add non existent fs link.
    */
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testMountLinkWithNonExistentLink() throws Exception {
     testMountLinkWithNonExistentLink(true);
   }
@@ -210,7 +216,7 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
       });
     } else {
       try (FileSystem fs = FileSystem.get(conf)) {
-        Assert.assertEquals("hdfs", fs.getScheme());
+        assertEquals("hdfs", fs.getScheme());
       }
     }
   }
@@ -221,7 +227,8 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
    * hdfs://localhost:xxx/local --> file://TEST_ROOT_DIR/root/
    * ListStatus on / should list the mount links.
    */
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testListStatusOnRootShouldListAllMountLinks() throws Exception {
     final Path hdfsTargetPath = new Path(defaultFSURI + HDFS_USER_FOLDER);
     addMountLinks(defaultFSURI.getAuthority(),
@@ -241,15 +248,13 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
     try (FileSystem fs = FileSystem.get(conf)) {
       fs.mkdirs(hdfsTargetPath);
       FileStatus[] ls = fs.listStatus(new Path("/"));
-      Assert.assertEquals(2, ls.length);
+      assertEquals(2, ls.length);
       String lsPath1 =
           Path.getPathWithoutSchemeAndAuthority(ls[0].getPath()).toString();
       String lsPath2 =
           Path.getPathWithoutSchemeAndAuthority(ls[1].getPath()).toString();
-      Assert.assertTrue(
-          HDFS_USER_FOLDER.equals(lsPath1) || LOCAL_FOLDER.equals(lsPath1));
-      Assert.assertTrue(
-          HDFS_USER_FOLDER.equals(lsPath2) || LOCAL_FOLDER.equals(lsPath2));
+      assertTrue(HDFS_USER_FOLDER.equals(lsPath1) || LOCAL_FOLDER.equals(lsPath1));
+      assertTrue(HDFS_USER_FOLDER.equals(lsPath2) || LOCAL_FOLDER.equals(lsPath2));
     }
   }
 
@@ -259,19 +264,21 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
    * hdfs://localhost:xxx/local --> file://TEST_ROOT_DIR/root/
    * ListStatus non mount directory should fail.
    */
-  @Test(expected = IOException.class, timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testListStatusOnNonMountedPath() throws Exception {
-    final Path hdfsTargetPath = new Path(defaultFSURI + HDFS_USER_FOLDER);
-    addMountLinks(defaultFSURI.getAuthority(),
-        new String[] {HDFS_USER_FOLDER, LOCAL_FOLDER },
-        new String[] {hdfsTargetPath.toUri().toString(),
-            localTargetDir.toURI().toString() },
-        conf);
-
-    try (FileSystem fs = FileSystem.get(conf)) {
-      fs.listStatus(new Path("/nonMount"));
-      Assert.fail("It should fail as no mount link with /nonMount");
-    }
+    assertThrows(IOException.class, () -> {
+      final Path hdfsTargetPath = new Path(defaultFSURI + HDFS_USER_FOLDER);
+      addMountLinks(defaultFSURI.getAuthority(),
+          new String[]{HDFS_USER_FOLDER, LOCAL_FOLDER},
+          new String[]{hdfsTargetPath.toUri().toString(),
+              localTargetDir.toURI().toString()},
+          conf);
+      try (FileSystem fs = FileSystem.get(conf)) {
+        fs.listStatus(new Path("/nonMount"));
+        fail("It should fail as no mount link with /nonMount");
+      }
+    });
   }
 
   /**
@@ -335,7 +342,8 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
    * Creating file or directory at non root level should succeed with fallback
    * links.
    */
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testWithLinkFallBack() throws Exception {
     final Path hdfsTargetPath = new Path(defaultFSURI + HDFS_USER_FOLDER);
     addMountLinks(defaultFSURI.getAuthority(),
@@ -349,10 +357,8 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
     try (FileSystem fs = FileSystem.get(conf)) {
       fs.createNewFile(new Path("/nonMount/myfile"));
       FileStatus[] ls = fs.listStatus(new Path("/nonMount"));
-      Assert.assertEquals(1, ls.length);
-      Assert.assertEquals(
-          Path.getPathWithoutSchemeAndAuthority(ls[0].getPath()).getName(),
-          "myfile");
+      assertEquals(1, ls.length);
+      assertEquals(Path.getPathWithoutSchemeAndAuthority(ls[0].getPath()).getName(), "myfile");
     }
   }
 
@@ -363,7 +369,8 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
    *
    * It cannot find any mount link. ViewFS expects a mount point from root.
    */
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testCreateOnRoot() throws Exception {
     testCreateOnRoot(false);
   }
@@ -376,7 +383,7 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
             localTargetDir.toURI().toString()}, conf);
     try (FileSystem fs = FileSystem.get(conf)) {
       if (fallbackExist) {
-        Assert.assertTrue(fs.createNewFile(new Path("/newFileOnRoot")));
+        assertTrue(fs.createNewFile(new Path("/newFileOnRoot")));
       } else {
         LambdaTestUtils.intercept(NotInMountpointException.class, () -> {
           fs.createNewFile(new Path("/newFileOnRoot"));
@@ -397,33 +404,36 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
    * Unset fs.viewfs.overload.scheme.target.hdfs.impl property.
    * So, OverloadScheme target fs initialization will fail.
    */
-  @Test(expected = IOException.class, timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testInvalidOverloadSchemeTargetFS() throws Exception {
-    final Path hdfsTargetPath = new Path(defaultFSURI + HDFS_USER_FOLDER);
-    String mountTableIfSet = conf.get(Constants.CONFIG_VIEWFS_MOUNTTABLE_PATH);
-    conf = new Configuration();
-    if (mountTableIfSet != null) {
-      conf.set(Constants.CONFIG_VIEWFS_MOUNTTABLE_PATH, mountTableIfSet);
-    }
-    addMountLinks(defaultFSURI.getHost(),
-        new String[] {HDFS_USER_FOLDER, LOCAL_FOLDER,
-            Constants.CONFIG_VIEWFS_LINK_FALLBACK },
-        new String[] {hdfsTargetPath.toUri().toString(),
-            localTargetDir.toURI().toString(),
-            hdfsTargetPath.toUri().toString() },
-        conf);
-    conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY,
-        defaultFSURI.toString());
-    conf.set(String.format(FS_IMPL_PATTERN_KEY, HDFS_SCHEME),
-        ViewFileSystemOverloadScheme.class.getName());
-    conf.unset(String.format(
-        FsConstants.FS_VIEWFS_OVERLOAD_SCHEME_TARGET_FS_IMPL_PATTERN,
-        HDFS_SCHEME));
+    assertThrows(IOException.class, () -> {
+      final Path hdfsTargetPath = new Path(defaultFSURI + HDFS_USER_FOLDER);
+      String mountTableIfSet = conf.get(Constants.CONFIG_VIEWFS_MOUNTTABLE_PATH);
+      conf = new Configuration();
+      if (mountTableIfSet != null) {
+        conf.set(Constants.CONFIG_VIEWFS_MOUNTTABLE_PATH, mountTableIfSet);
+      }
+      addMountLinks(defaultFSURI.getHost(),
+          new String[]{HDFS_USER_FOLDER, LOCAL_FOLDER,
+              Constants.CONFIG_VIEWFS_LINK_FALLBACK},
+          new String[]{hdfsTargetPath.toUri().toString(),
+              localTargetDir.toURI().toString(),
+              hdfsTargetPath.toUri().toString()},
+          conf);
+      conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY,
+          defaultFSURI.toString());
+      conf.set(String.format(FS_IMPL_PATTERN_KEY, HDFS_SCHEME),
+          ViewFileSystemOverloadScheme.class.getName());
+      conf.unset(String.format(
+          FsConstants.FS_VIEWFS_OVERLOAD_SCHEME_TARGET_FS_IMPL_PATTERN,
+          HDFS_SCHEME));
+      try (FileSystem fs = FileSystem.get(conf)) {
+        fs.createNewFile(new Path("/onRootWhenFallBack"));
+        fail("OverloadScheme target fs should be valid.");
+      }
+    });
 
-    try (FileSystem fs = FileSystem.get(conf)) {
-      fs.createNewFile(new Path("/onRootWhenFallBack"));
-      Assert.fail("OverloadScheme target fs should be valid.");
-    }
   }
 
   /**
@@ -433,7 +443,8 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
    *
    * It should be able to create file using ViewFileSystemOverloadScheme.
    */
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testViewFsOverloadSchemeWhenInnerCacheDisabled()
       throws Exception {
     final Path hdfsTargetPath = new Path(defaultFSURI + HDFS_USER_FOLDER);
@@ -446,7 +457,7 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
     try (FileSystem fs = FileSystem.get(conf)) {
       Path testFile = new Path(HDFS_USER_FOLDER + "/testFile");
       fs.createNewFile(testFile);
-      Assert.assertTrue(fs.exists(testFile));
+      assertTrue(fs.exists(testFile));
     }
   }
 
@@ -458,7 +469,8 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
    * 1. With cache, only one hdfs child file system instance should be there.
    * 2. Without cache, there should 2 hdfs instances.
    */
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testViewFsOverloadSchemeWithInnerCache()
       throws Exception {
     final Path hdfsTargetPath = new Path(defaultFSURI + HDFS_USER_FOLDER);
@@ -470,13 +482,13 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
 
     // 1. Only 1 hdfs child file system should be there with cache.
     try (FileSystem vfs = FileSystem.get(conf)) {
-      Assert.assertEquals(1, vfs.getChildFileSystems().length);
+      assertEquals(1, vfs.getChildFileSystems().length);
     }
 
     // 2. Two hdfs file systems should be there if no cache.
     conf.setBoolean(Constants.CONFIG_VIEWFS_ENABLE_INNER_CACHE, false);
     try (FileSystem vfs = FileSystem.get(conf)) {
-      Assert.assertEquals(isFallBackExist(conf) ? 3 : 2,
+      assertEquals(isFallBackExist(conf) ? 3 : 2,
           vfs.getChildFileSystems().length);
     }
   }
@@ -496,7 +508,8 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
    * When InnerCache disabled, all matching ViewFileSystemOverloadScheme
    * initialized scheme file systems would not use FileSystem cache.
    */
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testViewFsOverloadSchemeWithNoInnerCacheAndHdfsTargets()
       throws Exception {
     final Path hdfsTargetPath = new Path(defaultFSURI + HDFS_USER_FOLDER);
@@ -509,7 +522,7 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
     conf.setBoolean(Constants.CONFIG_VIEWFS_ENABLE_INNER_CACHE, false);
     // Two hdfs file systems should be there if no cache.
     try (FileSystem vfs = FileSystem.get(conf)) {
-      Assert.assertEquals(isFallBackExist(conf) ? 3 : 2,
+      assertEquals(isFallBackExist(conf) ? 3 : 2,
           vfs.getChildFileSystems().length);
     }
   }
@@ -523,7 +536,8 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
    * initialized scheme file systems should continue to take advantage of
    * FileSystem cache.
    */
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testViewFsOverloadSchemeWithNoInnerCacheAndLocalSchemeTargets()
       throws Exception {
     final Path localTragetPath = new Path(localTargetDir.toURI());
@@ -537,7 +551,7 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
     // cache should work.
     conf.setBoolean(Constants.CONFIG_VIEWFS_ENABLE_INNER_CACHE, false);
     try (FileSystem vfs = FileSystem.get(conf)) {
-      Assert.assertEquals(isFallBackExist(conf) ? 2 : 1,
+      assertEquals(isFallBackExist(conf) ? 2 : 1,
           vfs.getChildFileSystems().length);
     }
   }
@@ -545,7 +559,8 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
   /**
    * Tests the rename with nfly mount link.
    */
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testNflyRename() throws Exception {
     final Path hdfsTargetPath1 = new Path(defaultFSURI + HDFS_USER_FOLDER);
     final Path hdfsTargetPath2 = new Path(defaultFSURI + HDFS_USER_FOLDER + 1);
@@ -561,7 +576,7 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
 
     final Path testDir = new Path("/nflyroot/testdir1/sub1/sub3");
     final Path testDirTmp = new Path("/nflyroot/testdir1/sub1/sub3_temp");
-    assertTrue(testDir + ": Failed to create!", nfly.mkdirs(testDir));
+    assertTrue(nfly.mkdirs(testDir), testDir + ": Failed to create!");
 
     // Test renames
     assertTrue(nfly.rename(testDir, testDirTmp));
@@ -570,14 +585,15 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
     final URI[] testUris = new URI[] {uri1, uri2 };
     for (final URI testUri : testUris) {
       final FileSystem fs = FileSystem.get(testUri, conf);
-      assertTrue(testDir + " should exist!", fs.exists(testDir));
+      assertTrue(fs.exists(testDir), testDir + " should exist!");
     }
   }
 
   /**
    * Tests the write and read contents with nfly mount link.
    */
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testNflyWriteRead() throws Exception {
     final Path hdfsTargetPath1 = new Path(defaultFSURI + HDFS_USER_FOLDER);
     final Path hdfsTargetPath2 = new Path(defaultFSURI + HDFS_USER_FOLDER + 1);
@@ -604,7 +620,8 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
    * target file. 3. Tests the read works with repairOnRead flag. 4. Tests that
    * previously deleted file fully recovered and exists.
    */
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testNflyRepair() throws Exception {
     final NflyFSystem.NflyKey repairKey = NflyFSystem.NflyKey.repairOnRead;
     final Path hdfsTargetPath1 = new Path(defaultFSURI + HDFS_USER_FOLDER);
@@ -643,7 +660,8 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
    * Tests that the fs initialization should ignore the port number when it's
    * extracting the mount table name from uri.
    */
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testMountTableNameShouldIgnorePortFromURI() throws Exception {
     final Path hdfsTargetPath = new Path(defaultFSURI + HDFS_USER_FOLDER);
     conf = new Configuration(getConf());
@@ -688,7 +706,7 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
   private void readString(final FileSystem nfly, final Path testFile,
       final String testString, final URI testUri) throws IOException {
     try (FSDataInputStream fsDis = nfly.open(testFile)) {
-      assertEquals("Wrong file content", testString, fsDis.readUTF());
+      assertEquals(testString, fsDis.readUTF(), "Wrong file content");
     }
   }
 

+ 11 - 11
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithAcls.java

@@ -28,11 +28,11 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.util.Lists;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 
 import java.io.IOException;
 import java.util.List;
@@ -42,8 +42,8 @@ import static org.apache.hadoop.fs.permission.AclEntryScope.DEFAULT;
 import static org.apache.hadoop.fs.permission.AclEntryType.*;
 import static org.apache.hadoop.fs.permission.FsAction.*;
 import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.aclEntry;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 /**
  * Verify ACL through ViewFileSystem functionality.
@@ -61,7 +61,7 @@ public class TestViewFileSystemWithAcls {
   private FileSystemTestHelper fileSystemTestHelper =
       new FileSystemTestHelper("/tmp/TestViewFileSystemWithAcls");
 
-  @BeforeClass
+  @BeforeAll
   public static void clusterSetupAtBeginning() throws IOException {
     clusterConf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
     cluster = new MiniDFSCluster.Builder(clusterConf)
@@ -74,14 +74,14 @@ public class TestViewFileSystemWithAcls {
     fHdfs2 = cluster.getFileSystem(1);
   }
 
-  @AfterClass
+  @AfterAll
   public static void ClusterShutdownAtEnd() throws Exception {
     if (cluster != null) {
       cluster.shutdown();
     }
   }
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     fsTarget = fHdfs;
     fsTarget2 = fHdfs2;
@@ -105,7 +105,7 @@ public class TestViewFileSystemWithAcls {
     ConfigUtil.addLink(fsViewConf, mountOnNn2.toString(), targetTestRoot2.toUri());
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     fsTarget.delete(fileSystemTestHelper.getTestRootPath(fsTarget), true);
     fsTarget2.delete(fileSystemTestHelper.getTestRootPath(fsTarget2), true);

+ 15 - 13
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithTruncate.java

@@ -31,13 +31,14 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
-import static org.junit.Assert.*;
+import static org.junit.jupiter.api.Assertions.*;
 
 /**
  * Verify truncate through ViewFileSystem functionality.
@@ -54,7 +55,7 @@ public class TestViewFileSystemWithTruncate {
   private FileSystemTestHelper fileSystemTestHelper =
       new FileSystemTestHelper("/tmp/TestViewFileSystemWithXAttrs");
 
-  @BeforeClass
+  @BeforeAll
   public static void clusterSetupAtBeginning() throws IOException {
     cluster = new MiniDFSCluster.Builder(clusterConf)
         .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(2))
@@ -64,14 +65,14 @@ public class TestViewFileSystemWithTruncate {
     fHdfs = cluster.getFileSystem(0);
   }
 
-  @AfterClass
+  @AfterAll
   public static void clusterShutdownAtEnd() throws Exception {
     if (cluster != null) {
       cluster.shutdown();
     }
   }
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     fsTarget = fHdfs;
     targetTestRoot = fileSystemTestHelper.getAbsoluteTestRootPath(fsTarget);
@@ -90,12 +91,13 @@ public class TestViewFileSystemWithTruncate {
         .addLink(fsViewConf, mountOnNn1.toString(), targetTestRoot.toUri());
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     fsTarget.delete(fileSystemTestHelper.getTestRootPath(fsTarget), true);
   }
 
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testTruncateWithViewFileSystem()
       throws Exception {
     Path filePath = new Path(mountOnNn1 + "/ttest");
@@ -104,8 +106,8 @@ public class TestViewFileSystemWithTruncate {
     out.writeBytes("drtatedasfdasfgdfas");
     out.close();
     int newLength = 10;
-    assertTrue("ViewFS supports truncate",
-        fsView.hasPathCapability(filePath, CommonPathCapabilities.FS_TRUNCATE));
+    assertTrue(fsView.hasPathCapability(filePath, CommonPathCapabilities.FS_TRUNCATE),
+        "ViewFS supports truncate");
     boolean isReady = fsView.truncate(filePath, newLength);
     if (!isReady) {
       GenericTestUtils.waitFor(new Supplier<Boolean>() {

+ 11 - 11
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithXAttrs.java

@@ -24,16 +24,16 @@ import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 
 import java.io.IOException;
 
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 /**
  * Verify XAttrs through ViewFileSystem functionality.
@@ -57,7 +57,7 @@ public class TestViewFileSystemWithXAttrs {
   protected static final String name2 = "user.a2";
   protected static final byte[] value2 = {0x37, 0x38, 0x39};
 
-  @BeforeClass
+  @BeforeAll
   public static void clusterSetupAtBeginning() throws IOException {
     cluster = new MiniDFSCluster.Builder(clusterConf)
         .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(2))
@@ -69,14 +69,14 @@ public class TestViewFileSystemWithXAttrs {
     fHdfs2 = cluster.getFileSystem(1);
   }
 
-  @AfterClass
+  @AfterAll
   public static void ClusterShutdownAtEnd() throws Exception {
     if (cluster != null) {
       cluster.shutdown();
     }
   }
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     fsTarget = fHdfs;
     fsTarget2 = fHdfs2;
@@ -102,7 +102,7 @@ public class TestViewFileSystemWithXAttrs {
         targetTestRoot2.toUri());
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     fsTarget.delete(fileSystemTestHelper.getTestRootPath(fsTarget), true);
     fsTarget2.delete(fileSystemTestHelper.getTestRootPath(fsTarget2), true);

+ 6 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsAtHdfsRoot.java

@@ -30,9 +30,9 @@ import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
 
 /**
  * Make sure that ViewFs works when the root of an FS is mounted to a ViewFs
@@ -49,7 +49,7 @@ public class TestViewFsAtHdfsRoot extends ViewFsBaseTest {
     return new FileContextTestHelper("/tmp/TestViewFsAtHdfsRoot");
   }
 
-  @BeforeClass
+  @BeforeAll
   public static void clusterSetupAtBegining() throws IOException,
       LoginException, URISyntaxException {
     SupportsBlocks = true;
@@ -62,7 +62,7 @@ public class TestViewFsAtHdfsRoot extends ViewFsBaseTest {
   }
 
       
-  @AfterClass
+  @AfterAll
   public static void ClusterShutdownAtEnd() throws Exception {
     if (cluster != null) {
       cluster.shutdown();
@@ -70,7 +70,7 @@ public class TestViewFsAtHdfsRoot extends ViewFsBaseTest {
   }
 
   @Override
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     // create the test root on local_fs
     fcTarget = fc;

+ 8 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsDefaultValue.java

@@ -28,9 +28,9 @@ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_WRIT
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.IOException;
 import java.net.URI;
@@ -49,9 +49,9 @@ import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 
 /**
  * Tests for viewfs implementation of default fs level values.
@@ -74,7 +74,7 @@ public class TestViewFsDefaultValue {
   // Use NotInMountpoint path to trigger the exception
   private static Path notInMountpointPath;
 
-  @BeforeClass
+  @BeforeAll
   public static void clusterSetupAtBegining() throws IOException,
       LoginException, URISyntaxException {
 
@@ -218,7 +218,7 @@ public class TestViewFsDefaultValue {
     assertTrue(qu.getSpaceConsumed() > 0);
   }
 
-  @AfterClass
+  @AfterAll
   public static void cleanup() throws IOException {
     fHdfs.delete(new Path(testFileName), true);
     fHdfs.delete(notInMountpointPath, true);

+ 11 - 11
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsFileStatusHdfs.java

@@ -23,8 +23,8 @@ package org.apache.hadoop.fs.viewfs;
  * Since viewfs has overlayed ViewFsFileStatus, we ran into
  * serialization problems. THis test is test the fix.
  */
-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.IOException;
 import java.net.URI;
@@ -43,9 +43,9 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 
 public class TestViewFsFileStatusHdfs {
   
@@ -59,7 +59,7 @@ public class TestViewFsFileStatusHdfs {
   private static FileSystem fHdfs;
   private static FileSystem vfs;
   
-  @BeforeClass
+  @BeforeAll
   public static void clusterSetupAtBegining() throws IOException,
       LoginException, URISyntaxException {
     cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(2).build();
@@ -109,14 +109,14 @@ public class TestViewFsFileStatusHdfs {
     FileChecksum otherHdfsFileCheckSum = fHdfs.getFileChecksum(
       new Path(someFile+"other"));
     // Checksums of the same file (got through HDFS and ViewFS should be same)
-    assertEquals("HDFS and ViewFS checksums were not the same", viewFSCheckSum,
-      hdfsCheckSum);
+    assertEquals(viewFSCheckSum, hdfsCheckSum,
+        "HDFS and ViewFS checksums were not the same");
     // Checksum of different files should be different.
-    assertFalse("Some other HDFS file which should not have had the same " +
-      "checksum as viewFS did!", viewFSCheckSum.equals(otherHdfsFileCheckSum));
+    assertFalse(viewFSCheckSum.equals(otherHdfsFileCheckSum),
+        "Some other HDFS file which should not have had the same checksum as viewFS did!");
   }
 
-  @AfterClass
+  @AfterAll
   public static void cleanup() throws IOException {
     fHdfs.delete(new Path(testfilename), true);
     fHdfs.delete(new Path(someFile), true);

+ 10 - 10
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsHdfs.java

@@ -36,13 +36,13 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
 
 public class TestViewFsHdfs extends ViewFsBaseTest {
 
@@ -56,7 +56,7 @@ public class TestViewFsHdfs extends ViewFsBaseTest {
   }
 
 
-  @BeforeClass
+  @BeforeAll
   public static void clusterSetupAtBegining() throws IOException,
       LoginException, URISyntaxException {
     SupportsBlocks = true;
@@ -72,7 +72,7 @@ public class TestViewFsHdfs extends ViewFsBaseTest {
   }
 
       
-  @AfterClass
+  @AfterAll
   public static void ClusterShutdownAtEnd() throws Exception {
     if (cluster != null) {
       cluster.shutdown();
@@ -80,7 +80,7 @@ public class TestViewFsHdfs extends ViewFsBaseTest {
   }
 
   @Override
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     // create the test root on local_fs
     fcTarget = fc;
@@ -160,7 +160,7 @@ public class TestViewFsHdfs extends ViewFsBaseTest {
     FileContext otherfs = map.get("user1");
     otherfs.mkdir(user1Path, FileContext.DEFAULT_PERM, false);
     String owner = otherfs.getFileStatus(user1Path).getOwner();
-    assertEquals("The owner did not match ", owner, userUgi.getShortUserName());
+    assertEquals(owner, userUgi.getShortUserName(), "The owner did not match ");
     otherfs.delete(user1Path, false);
   }
  

+ 61 - 60
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsLinkFallback.java

@@ -18,10 +18,12 @@
 package org.apache.hadoop.fs.viewfs;
 
 import static org.apache.hadoop.fs.CreateFlag.CREATE;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -48,11 +50,10 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.test.LambdaTestUtils;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 
 /**
  * Test for viewfs with LinkFallback mount table entries.
@@ -64,7 +65,7 @@ public class TestViewFsLinkFallback {
   private static URI viewFsDefaultClusterUri;
   private Path targetTestRoot;
 
-  @BeforeClass
+  @BeforeAll
   public static void clusterSetupAtBeginning()
       throws IOException, URISyntaxException {
     int nameSpacesCount = 3;
@@ -90,14 +91,14 @@ public class TestViewFsLinkFallback {
 
   }
 
-  @AfterClass
+  @AfterAll
   public static void clusterShutdownAtEnd() throws Exception {
     if (cluster != null) {
       cluster.shutdown();
     }
   }
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     fsTarget = fsDefault;
     initializeTargetTestRoot();
@@ -317,7 +318,7 @@ public class TestViewFsLinkFallback {
       // attempt to create in fallback.
       vfs.mkdir(nextLevelToInternalDir, FsPermission.getDirDefault(),
           false);
-      Assert.fail("It should throw IOE when fallback fs not available.");
+      fail("It should throw IOE when fallback fs not available.");
     } catch (IOException e) {
       cluster.restartNameNodes();
       // should succeed when fallback fs is back to normal.
@@ -413,69 +414,69 @@ public class TestViewFsLinkFallback {
    * Tests the create of a file on root where the path is matching to an
    * existing file on fallback's file on root.
    */
-  @Test (expected = FileAlreadyExistsException.class)
+  @Test
   public void testCreateFileOnRootWithFallbackWithFileAlreadyExist()
       throws Exception {
-    Configuration conf = new Configuration();
-    Path fallbackTarget = new Path(targetTestRoot, "fallbackDir");
-    Path testFile = new Path(fallbackTarget, "test.file");
-    // pre-creating test file in fallback.
-    fsTarget.create(testFile).close();
-
-    ConfigUtil.addLink(conf, "/user1/hive/",
-        new Path(targetTestRoot.toString()).toUri());
-    ConfigUtil.addLinkFallback(conf, fallbackTarget.toUri());
-
-    AbstractFileSystem vfs =
-        AbstractFileSystem.get(viewFsDefaultClusterUri, conf);
-    Path vfsTestFile = new Path("/test.file");
-    assertTrue(fsTarget.exists(testFile));
-    vfs.create(vfsTestFile, EnumSet.of(CREATE),
-        Options.CreateOpts.perms(FsPermission.getDefault())).close();
+    assertThrows(FileAlreadyExistsException.class, () -> {
+      Configuration conf = new Configuration();
+      Path fallbackTarget = new Path(targetTestRoot, "fallbackDir");
+      Path testFile = new Path(fallbackTarget, "test.file");
+      fsTarget.create(testFile).close();
+      ConfigUtil.addLink(conf, "/user1/hive/",
+          new Path(targetTestRoot.toString()).toUri());
+      ConfigUtil.addLinkFallback(conf, fallbackTarget.toUri());
+      AbstractFileSystem vfs =
+          AbstractFileSystem.get(viewFsDefaultClusterUri, conf);
+      Path vfsTestFile = new Path("/test.file");
+      assertTrue(fsTarget.exists(testFile));
+      vfs.create(vfsTestFile, EnumSet.of(CREATE),
+          Options.CreateOpts.perms(FsPermission.getDefault())).close();
+    });
   }
 
   /**
    * Tests the creating of a file where the path is same as mount link path.
    */
-  @Test(expected= FileAlreadyExistsException.class)
+  @Test
   public void testCreateFileWhereThePathIsSameAsItsMountLinkPath()
       throws Exception {
-    Configuration conf = new Configuration();
-    Path fallbackTarget = new Path(targetTestRoot, "fallbackDir");
-    fsTarget.mkdirs(fallbackTarget);
-
-    ConfigUtil.addLink(conf, "/user1/hive/",
-        new Path(targetTestRoot.toString()).toUri());
-    ConfigUtil.addLinkFallback(conf, fallbackTarget.toUri());
-
-    AbstractFileSystem vfs =
-        AbstractFileSystem.get(viewFsDefaultClusterUri, conf);
-    Path vfsTestDir = new Path("/user1/hive");
-    assertFalse(fsTarget.exists(Path.mergePaths(fallbackTarget, vfsTestDir)));
-    vfs.create(vfsTestDir, EnumSet.of(CREATE),
-        Options.CreateOpts.perms(FsPermission.getDefault())).close();
+    assertThrows(FileAlreadyExistsException.class, () -> {
+      Configuration conf = new Configuration();
+      Path fallbackTarget = new Path(targetTestRoot, "fallbackDir");
+      fsTarget.mkdirs(fallbackTarget);
+      ConfigUtil.addLink(conf, "/user1/hive/",
+          new Path(targetTestRoot.toString()).toUri());
+      ConfigUtil.addLinkFallback(conf, fallbackTarget.toUri());
+      AbstractFileSystem vfs =
+          AbstractFileSystem.get(viewFsDefaultClusterUri, conf);
+      Path vfsTestDir = new Path("/user1/hive");
+      assertFalse(fsTarget.exists(Path.mergePaths(fallbackTarget, vfsTestDir)));
+      vfs.create(vfsTestDir, EnumSet.of(CREATE),
+          Options.CreateOpts.perms(FsPermission.getDefault())).close();
+    });
   }
 
   /**
    * Tests the create of a file where the path is same as one of of the internal
    * dir path should fail.
    */
-  @Test(expected = FileAlreadyExistsException.class)
+  @Test
   public void testCreateFileSameAsInternalDirPath()
       throws Exception {
-    Configuration conf = new Configuration();
-    Path fallbackTarget = new Path(targetTestRoot, "fallbackDir");
-    fsTarget.mkdirs(fallbackTarget);
-    ConfigUtil.addLink(conf, "/user1/hive/",
-        new Path(targetTestRoot.toString()).toUri());
-    ConfigUtil.addLinkFallback(conf, fallbackTarget.toUri());
-
-    AbstractFileSystem vfs =
-        AbstractFileSystem.get(viewFsDefaultClusterUri, conf);
-    Path vfsTestDir = new Path("/user1");
-    assertFalse(fsTarget.exists(Path.mergePaths(fallbackTarget, vfsTestDir)));
-    vfs.create(vfsTestDir, EnumSet.of(CREATE),
-        Options.CreateOpts.perms(FsPermission.getDefault())).close();
+    assertThrows(FileAlreadyExistsException.class, () -> {
+      Configuration conf = new Configuration();
+      Path fallbackTarget = new Path(targetTestRoot, "fallbackDir");
+      fsTarget.mkdirs(fallbackTarget);
+      ConfigUtil.addLink(conf, "/user1/hive/",
+          new Path(targetTestRoot.toString()).toUri());
+      ConfigUtil.addLinkFallback(conf, fallbackTarget.toUri());
+      AbstractFileSystem vfs =
+          AbstractFileSystem.get(viewFsDefaultClusterUri, conf);
+      Path vfsTestDir = new Path("/user1");
+      assertFalse(fsTarget.exists(Path.mergePaths(fallbackTarget, vfsTestDir)));
+      vfs.create(vfsTestDir, EnumSet.of(CREATE),
+          Options.CreateOpts.perms(FsPermission.getDefault())).close();
+    });
   }
 
   /**
@@ -592,7 +593,7 @@ public class TestViewFsLinkFallback {
     fs.rename(src, dst, Options.Rename.OVERWRITE);
     LambdaTestUtils
         .intercept(FileNotFoundException.class, () -> fs.getFileStatus(src));
-    Assert.assertNotNull(fs.getFileStatus(dst));
+    assertNotNull(fs.getFileStatus(dst));
   }
 
 }

+ 11 - 11
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsWithAcls.java

@@ -29,11 +29,11 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.util.Lists;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 import java.util.List;
 
 import java.io.IOException;
@@ -44,8 +44,8 @@ import static org.apache.hadoop.fs.permission.AclEntryType.*;
 import static org.apache.hadoop.fs.permission.FsAction.*;
 import static org.apache.hadoop.fs.permission.FsAction.NONE;
 import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.aclEntry;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 /**
  * Verify ACL through ViewFs functionality.
@@ -61,7 +61,7 @@ public class TestViewFsWithAcls {
   private FileContextTestHelper fileContextTestHelper =
       new FileContextTestHelper("/tmp/TestViewFsWithAcls");
 
-  @BeforeClass
+  @BeforeAll
   public static void clusterSetupAtBeginning() throws IOException {
     clusterConf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
     cluster = new MiniDFSCluster.Builder(clusterConf)
@@ -74,14 +74,14 @@ public class TestViewFsWithAcls {
     fc2 = FileContext.getFileContext(cluster.getURI(1), clusterConf);
   }
 
-  @AfterClass
+  @AfterAll
   public static void ClusterShutdownAtEnd() throws Exception {
     if (cluster != null) {
       cluster.shutdown();
     }
   }
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     fcTarget = fc;
     fcTarget2 = fc2;
@@ -105,7 +105,7 @@ public class TestViewFsWithAcls {
     ConfigUtil.addLink(fsViewConf, mountOnNn2.toString(), targetTestRoot2.toUri());
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     fcTarget.delete(fileContextTestHelper.getTestRootPath(fcTarget), true);
     fcTarget2.delete(fileContextTestHelper.getTestRootPath(fcTarget2), true);

+ 11 - 11
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsWithXAttrs.java

@@ -25,16 +25,16 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 
 import java.io.IOException;
 
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 /**
  * Verify XAttrs through ViewFs functionality.
@@ -56,7 +56,7 @@ public class TestViewFsWithXAttrs {
   protected static final String name2 = "user.a2";
   protected static final byte[] value2 = {0x37, 0x38, 0x39};
 
-  @BeforeClass
+  @BeforeAll
   public static void clusterSetupAtBeginning() throws IOException {
     cluster = new MiniDFSCluster.Builder(clusterConf)
         .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(2))
@@ -68,14 +68,14 @@ public class TestViewFsWithXAttrs {
     fc2 = FileContext.getFileContext(cluster.getURI(1), clusterConf);
   }
 
-  @AfterClass
+  @AfterAll
   public static void ClusterShutdownAtEnd() throws Exception {
     if (cluster != null) {
       cluster.shutdown();
     }
   }
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     fcTarget = fc;
     fcTarget2 = fc2;
@@ -99,7 +99,7 @@ public class TestViewFsWithXAttrs {
     ConfigUtil.addLink(fsViewConf, mountOnNn2.toString(), targetTestRoot2.toUri());
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     fcTarget.delete(fileContextTestHelper.getTestRootPath(fcTarget), true);
     fcTarget2.delete(fileContextTestHelper.getTestRootPath(fcTarget2), true);

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

@@ -25,17 +25,22 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.viewfs.ConfigUtil;
 import org.apache.hadoop.fs.viewfs.TestViewFileSystemOverloadSchemeWithHdfsScheme;
 import org.apache.hadoop.fs.viewfs.ViewFsTestSetup;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+import org.junit.jupiter.api.BeforeEach;
 
 import java.io.IOException;
 import java.net.URI;
 
 import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_IGNORE_PORT_IN_MOUNT_TABLE_NAME;
 import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_IGNORE_PORT_IN_MOUNT_TABLE_NAME_DEFAULT;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class TestViewDistributedFileSystemWithMountLinks extends
     TestViewFileSystemOverloadSchemeWithHdfsScheme {
+
+  @BeforeEach
   @Override
   public void setUp() throws IOException {
     super.setUp();
@@ -55,12 +60,14 @@ public class TestViewDistributedFileSystemWithMountLinks extends
     setConf(conf);
   }
 
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testCreateOnRoot() throws Exception {
     testCreateOnRoot(true);
   }
 
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testMountLinkWithNonExistentLink() throws Exception {
     testMountLinkWithNonExistentLink(false);
   }
@@ -144,14 +151,14 @@ public class TestViewDistributedFileSystemWithMountLinks extends
       dst = new Path("/InternalDirDoesNotExistInFallback/file");
       fs.create(src).close();
       // If fallback does not have same structure as internal, rename will fail.
-      Assert.assertFalse(fs.rename(src, dst));
+      assertFalse(fs.rename(src, dst));
     }
   }
 
   private void verifyRename(FileSystem fs, Path src, Path dst)
       throws IOException {
     fs.rename(src, dst);
-    Assert.assertFalse(fs.exists(src));
-    Assert.assertTrue(fs.exists(dst));
+    assertFalse(fs.exists(src));
+    assertTrue(fs.exists(dst));
   }
 }