From d11184b9b66f4af507a975caf10d382ba17e0a42 Mon Sep 17 00:00:00 2001 From: zhtttylz Date: Tue, 15 Apr 2025 00:33:09 +0800 Subject: [PATCH] HDFS-12431. [JDK17] Upgrade JUnit from 4 to 5 in hadoop-hdfs Part2. --- .../fs/TestEnhancedByteBufferAccess.java | 147 +++++++------- .../hadoop/fs/TestFcHdfsCreateMkdir.java | 16 +- .../hadoop/fs/TestFcHdfsPermission.java | 16 +- .../apache/hadoop/fs/TestFcHdfsSetUMask.java | 63 +++--- .../org/apache/hadoop/fs/TestGlobPaths.java | 114 ++++++----- .../fs/TestHDFSFileContextMainOperations.java | 78 +++---- .../hadoop/fs/TestHdfsNativeCodeLoader.java | 5 +- .../hadoop/fs/TestResolveHdfsSymlink.java | 19 +- ...TestSWebHdfsFileContextMainOperations.java | 8 +- .../hadoop/fs/TestSymlinkHdfsDisable.java | 8 +- .../org/apache/hadoop/fs/TestUnbuffer.java | 35 ++-- .../hadoop/fs/TestUrlStreamHandler.java | 24 +-- .../TestWebHdfsFileContextMainOperations.java | 18 +- .../fs/loadGenerator/TestLoadGenerator.java | 4 +- .../hadoop/fs/permission/TestStickyBit.java | 22 +- .../hadoop/fs/shell/TestHdfsTextCommand.java | 14 +- ...tartupWhenViewFSOverloadSchemeEnabled.java | 17 +- ...rloadSchemeWithMountTableConfigInHDFS.java | 4 +- .../viewfs/TestViewFileSystemAtHdfsRoot.java | 12 +- .../fs/viewfs/TestViewFileSystemClose.java | 2 +- .../fs/viewfs/TestViewFileSystemHdfs.java | 62 +++--- .../TestViewFileSystemLinkFallback.java | 137 ++++++------- .../TestViewFileSystemLinkMergeSlash.java | 33 ++- .../viewfs/TestViewFileSystemLinkRegex.java | 29 ++- ...mOverloadSchemeHdfsFileSystemContract.java | 31 +-- ...ileSystemOverloadSchemeWithHdfsScheme.java | 192 ++++++++++-------- .../fs/viewfs/TestViewFileSystemWithAcls.java | 22 +- .../TestViewFileSystemWithTruncate.java | 28 +-- .../viewfs/TestViewFileSystemWithXAttrs.java | 22 +- .../fs/viewfs/TestViewFsAtHdfsRoot.java | 12 +- .../fs/viewfs/TestViewFsDefaultValue.java | 16 +- .../fs/viewfs/TestViewFsFileStatusHdfs.java | 22 +- .../hadoop/fs/viewfs/TestViewFsHdfs.java | 20 +- .../fs/viewfs/TestViewFsLinkFallback.java | 121 +++++------ .../hadoop/fs/viewfs/TestViewFsWithAcls.java | 22 +- .../fs/viewfs/TestViewFsWithXAttrs.java | 22 +- ...ewDistributedFileSystemWithMountLinks.java | 21 +- 37 files changed, 744 insertions(+), 694 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java index d918ba0822f1f..e34c45d87f518 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java +++ b/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_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.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 void mlock(String identifier, }); } - @AfterClass + @AfterAll public static void teardown() { // Restore the original CacheManipulator NativeIO.POSIX.setCacheManipulator(prevCacheManipulator); @@ -116,8 +122,8 @@ private static byte[] byteBufferToArray(ByteBuffer buf) { (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 void testZeroCopyReads() throws Exception { 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 void testZeroCopyReads() throws Exception { 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 void testShortZeroCopyReads() throws Exception { 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 void testShortZeroCopyReads() throws Exception { 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 void testZeroCopyReadsNoFallback() throws Exception { 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 void testZeroCopyReadsNoFallback() throws Exception { 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 void visit(int numOutstandingMmaps, 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 void testZeroCopyMmapCache() throws Exception { 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 void visit(int numOutstandingMmaps, 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 void testHdfsFallbackReads() throws Exception { 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 @@ private static void testFallbackImpl(InputStream stream, 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 void testFallbackRead() throws Exception { 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 void testIndirectFallbackReads() throws Exception { * 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 void testZeroCopyReadOfCachedData() throws Exception { 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 void testZeroCopyReadOfCachedData() throws Exception { 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 void testZeroCopyReadOfCachedData() throws Exception { 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 void visit(int numOutstandingMmaps, Map 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 void testClientMmapDisable() throws Exception { 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 void testClientMmapDisable() throws Exception { // 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 void testClientMmapDisable() throws Exception { @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 void test2GBMmapLimit() throws Exception { 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 void test2GBMmapLimit() throws Exception { 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 { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsCreateMkdir.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsCreateMkdir.java index 0d3e6ff8379ff..32fb9673a6182 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsCreateMkdir.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsCreateMkdir.java @@ -27,10 +27,10 @@ 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 @@ protected FileContextTestHelper createFileContextHelper() { } - @BeforeClass + @BeforeAll public static void clusterSetupAtBegining() throws IOException, LoginException, URISyntaxException { Configuration conf = new HdfsConfiguration(); @@ -56,7 +56,7 @@ public static void clusterSetupAtBegining() } - @AfterClass + @AfterAll public static void ClusterShutdownAtEnd() throws Exception { if (cluster != null) { cluster.shutdown(); @@ -64,13 +64,13 @@ public static void ClusterShutdownAtEnd() throws Exception { } @Override - @Before + @BeforeEach public void setUp() throws Exception { super.setUp(); } @Override - @After + @AfterEach public void tearDown() throws Exception { super.tearDown(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsPermission.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsPermission.java index 10ae1ef4b1e5e..50a1ffc87d46d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsPermission.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsPermission.java @@ -27,10 +27,10 @@ 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 @@ protected FileContext getFileContext() { return fc; } - @BeforeClass + @BeforeAll public static void clusterSetupAtBegining() throws IOException, LoginException, URISyntaxException { Configuration conf = new HdfsConfiguration(); @@ -63,7 +63,7 @@ public static void clusterSetupAtBegining() } - @AfterClass + @AfterAll public static void ClusterShutdownAtEnd() throws Exception { if (cluster != null) { cluster.shutdown(); @@ -71,13 +71,13 @@ public static void ClusterShutdownAtEnd() throws Exception { } @Override - @Before + @BeforeEach public void setUp() throws Exception { super.setUp(); } @Override - @After + @AfterEach public void tearDown() throws Exception { super.tearDown(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsSetUMask.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsSetUMask.java index eef22356778bf..1f0c1dd5ded90 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsSetUMask.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsSetUMask.java @@ -30,12 +30,15 @@ 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 static void clusterSetupAtBegining() 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 void testCreateRecursiveWithExistingDirMiddle() throws IOException { @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 void testCreateRecursiveWithNonExistingDirClear() throws IOException { @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 void testCreateRecursiveWithNonExistingDirOpen() throws IOException { @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 void testMkdirWithExistingDir(FsPermission umask, 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 void testMkdirRecursiveWithNonExistingDir(FsPermission umask, 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 void testCreateRecursiveWithExistingDir(FsPermission umask, 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 void testCreateRecursiveWithNonExistingDir(FsPermission umask, 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"); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestGlobPaths.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestGlobPaths.java index f8adf017332d1..08205b6da7971 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestGlobPaths.java +++ b/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.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 boolean accept(Path path) { 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 static void setUp() throws Exception { 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 void testCRInPathGlob() throws IOException { 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 @@ void run() throws Exception { // 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 @@ void run() throws Exception { + "/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 @@ void run() throws Exception { // 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 @@ void run() throws Exception { 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 @@ void run() throws Exception { 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 @@ void run() throws Exception { // 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 @@ void run() throws Exception { + "/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 @@ void run() throws Exception { 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 @@ private class TestReservedHdfsPaths extends FSTestWrapperGlobTest { 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 @@ void run() throws Exception { 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 @@ void run() throws Exception { 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 @@ void run() throws Exception { 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 void testLocalFilesystem() throws Exception { 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 void testLocalFilesystem() throws Exception { list.add(f.getPath().toString()); } boolean sorted = Ordering.natural().isOrdered(list); - Assert.assertTrue(sorted); + assertTrue(sorted); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestHDFSFileContextMainOperations.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestHDFSFileContextMainOperations.java index 8c37351f41f36..3b55b7d7368b2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestHDFSFileContextMainOperations.java +++ b/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.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 @@ protected FileContextTestHelper createFileContextHelper() { 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 @@ private static void restartCluster() throws IOException, LoginException { 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 static void ClusterShutdownAtEnd() throws Exception { } @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 void testTruncate() throws Exception { 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 void testRenameWithQuota() throws Exception { 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 void testEditsLogOldRename() throws Exception { 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 void testEditsLogRename() throws Exception { 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 void testIsValidNameInvalidNames() { }; 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 @@ private void oldRename(Path src, Path dst, boolean renameSucceeds, 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 diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestHdfsNativeCodeLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestHdfsNativeCodeLoader.java index 4ecca5e55a1db..82883ce39f19d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestHdfsNativeCodeLoader.java +++ b/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); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestResolveHdfsSymlink.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestResolveHdfsSymlink.java index aefb4835b14d4..bce41dbec3e93 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestResolveHdfsSymlink.java +++ b/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.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 static void setUp() throws IOException { } - @AfterClass + @AfterAll public static void tearDown() { if (cluster != null) { cluster.shutdown(); @@ -99,11 +100,11 @@ public void testFcResolveAfs() throws IOException, InterruptedException { Set 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"); } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSWebHdfsFileContextMainOperations.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSWebHdfsFileContextMainOperations.java index 061858c3d2863..5bdb5a4eb0c1b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSWebHdfsFileContextMainOperations.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSWebHdfsFileContextMainOperations.java @@ -24,8 +24,8 @@ 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 URI getWebhdfsUrl() { return webhdfsUrl; } - @AfterClass + @AfterAll public static void ClusterShutdownAtEnd() throws Exception { if (cluster != null) { cluster.shutdown(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfsDisable.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfsDisable.java index e5a513394a9bf..bfdfb688aca8e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfsDisable.java +++ b/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.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 diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUnbuffer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUnbuffer.java index feb77f8457c9c..b81274dca3b63 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUnbuffer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUnbuffer.java @@ -26,19 +26,17 @@ 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 void testUnbufferClosesSockets() throws Exception { // 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 void testOpenManyFilesViaTcp() throws Exception { 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 @@ abstract class BuggyStream 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)); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUrlStreamHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUrlStreamHandler.java index 5a04f67846b15..dc2aea9757171 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUrlStreamHandler.java +++ b/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.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 void testFileUrls() throws IOException, URISyntaxException { @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"); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestWebHdfsFileContextMainOperations.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestWebHdfsFileContextMainOperations.java index 7544835c7a3dd..24d2cb6c0344e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestWebHdfsFileContextMainOperations.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestWebHdfsFileContextMainOperations.java @@ -24,10 +24,10 @@ 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 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 URI getWebhdfsUrl() { return webhdfsUrl; } - @BeforeClass + @BeforeAll public static void clusterSetupAtBeginning() throws IOException, LoginException, URISyntaxException { @@ -85,7 +85,7 @@ public static void clusterSetupAtBeginning() 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 void testSetVerifyChecksum() throws IOException { assertArrayEquals(data, bb); } - @AfterClass + @AfterAll public static void ClusterShutdownAtEnd() throws Exception { if (cluster != null) { cluster.shutdown(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/loadGenerator/TestLoadGenerator.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/loadGenerator/TestLoadGenerator.java index 6c7bac31ea860..af901d8e494f6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/loadGenerator/TestLoadGenerator.java +++ b/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.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. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/permission/TestStickyBit.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/permission/TestStickyBit.java index 96d15e59a52ab..8127968b87687 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/permission/TestStickyBit.java +++ b/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.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.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 @@ private static void initCluster(boolean format) throws Exception { 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 void setup() throws Exception { } } - @AfterClass + @AfterAll public static void shutdown() throws Exception { IOUtils.cleanupWithLogger(null, hdfs, hdfsAsUser1, hdfsAsUser2); if (cluster != null) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/shell/TestHdfsTextCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/shell/TestHdfsTextCommand.java index 57e7cf415f65e..ea67358cbb47a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/shell/TestHdfsTextCommand.java +++ b/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.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 void setUp() throws IOException{ fs = cluster.getFileSystem(); } - @After + @AfterEach public void tearDown() throws IOException{ if(fs != null){ fs.close(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestNNStartupWhenViewFSOverloadSchemeEnabled.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestNNStartupWhenViewFSOverloadSchemeEnabled.java index bbdbd5a0b0d40..16b665161c18c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestNNStartupWhenViewFSOverloadSchemeEnabled.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestNNStartupWhenViewFSOverloadSchemeEnabled.java @@ -24,9 +24,10 @@ 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 static void setUp() { * 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 void testHANameNodeAndDataNodeStartup() throws Exception { * 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 void testNameNodeAndDataNodeStartup() throws Exception { cluster.waitActive(); } - @After + @AfterEach public void shutdownCluster() { if (cluster != null) { cluster.shutdown(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFSOverloadSchemeWithMountTableConfigInHDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFSOverloadSchemeWithMountTableConfigInHDFS.java index 5e2f42b77a3a7..f0221bf79f383 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFSOverloadSchemeWithMountTableConfigInHDFS.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFSOverloadSchemeWithMountTableConfigInHDFS.java @@ -25,7 +25,7 @@ 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(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemAtHdfsRoot.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemAtHdfsRoot.java index 9b2953c61cd00..f72a930342da1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemAtHdfsRoot.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemAtHdfsRoot.java @@ -29,9 +29,9 @@ 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 @@ protected FileSystemTestHelper createFileSystemHelper() { return new FileSystemTestHelper("/tmp/TestViewFileSystemAtHdfsRoot"); } - @BeforeClass + @BeforeAll public static void clusterSetupAtBegining() throws IOException, LoginException, URISyntaxException { SupportsBlocks = true; @@ -63,7 +63,7 @@ public static void clusterSetupAtBegining() throws IOException, fHdfs = cluster.getFileSystem(); } - @AfterClass + @AfterAll public static void clusterShutdownAtEnd() throws Exception { if (cluster != null) { cluster.shutdown(); @@ -71,7 +71,7 @@ public static void clusterShutdownAtEnd() throws Exception { } @Override - @Before + @BeforeEach public void setUp() throws Exception { fsTarget = fHdfs; super.setUp(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemClose.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemClose.java index 7301e0c0acfa3..3fd44d7850d24 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemClose.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemClose.java @@ -24,7 +24,7 @@ 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; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java index afa6ccaf2dad4..6b7a50c55e0ec 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java +++ b/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.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 @@ protected FileSystemTestHelper createFileSystemHelper() { return new FileSystemTestHelper("/tmp/TestViewFileSystemHdfs"); } - @BeforeClass + @BeforeAll public static void clusterSetupAtBegining() throws IOException, LoginException, URISyntaxException { @@ -131,7 +131,7 @@ public static void clusterSetupAtBegining() throws IOException, } - @AfterClass + @AfterAll public static void ClusterShutdownAtEnd() throws Exception { if (cluster != null) { cluster.shutdown(); @@ -139,7 +139,7 @@ public static void ClusterShutdownAtEnd() throws Exception { } @Override - @Before + @BeforeEach public void setUp() throws Exception { // create the test root on local_fs fsTarget = fHdfs; @@ -149,7 +149,7 @@ public void setUp() throws Exception { } @Override - @After + @AfterEach public void tearDown() throws Exception { super.tearDown(); } @@ -214,13 +214,13 @@ public void testTrashRootsAfterEncryptionZoneDeletion() throws Exception { //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 void testFileChecksum() throws IOException { 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 @@ private void testNflyRepair(NflyFSystem.NflyKey repairKey) // 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 @@ private void testNflyRepair(NflyFSystem.NflyKey repairKey) 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 @@ private void testNflyRepair(NflyFSystem.NflyKey repairKey) 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 @@ private void testNflyRepair(NflyFSystem.NflyKey repairKey) 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 @@ private void testNflyRepair(NflyFSystem.NflyKey repairKey) 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 @@ private void testNflyRepair(NflyFSystem.NflyKey repairKey) 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 Object run() throws IOException { 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); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkFallback.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkFallback.java index e7317608147be..84f26f8c4c336 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkFallback.java +++ b/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.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 @@ protected FileSystemTestHelper createFileSystemHelper() { return new FileSystemTestHelper(TEST_BASE_PATH); } - @BeforeClass + @BeforeAll public static void clusterSetupAtBeginning() throws IOException, LoginException, URISyntaxException { SupportsBlocks = true; @@ -100,7 +99,7 @@ public static void clusterSetupAtBeginning() throws IOException, 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 static void clusterShutdownAtEnd() throws Exception { } @Override - @Before + @BeforeEach public void setUp() throws Exception { fsTarget = fsDefault; super.setUp(); @@ -179,10 +178,10 @@ public void testConfLinkFallback() throws Exception { 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 void testConfLinkFallbackWithRegularLinks() throws Exception { 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 void testConfLinkFallbackWithRegularLinks() throws Exception { 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 void testConfLinkFallbackWithMountPoint() throws Exception { 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 void testListingWithFallbackLink() throws Exception { 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 void testListingWithFallbackLinkWithSameMountDirectories() 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 void testListingWithFallbackLinkWithSameMountDirectoryTree() } //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 void testLSOnLinkParentWithFallbackLinkWithSameMountDirectoryTree() } } 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 void testLSOnRootWithFallbackLinkWithSameMountDirectories() 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 void testCreateFileOnRootWithFallbackEnabled() throws Exception { * 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 void testCreateFileSameAsInternalDirPath() throws Exception { 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 diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkMergeSlash.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkMergeSlash.java index 606743f582dfc..a2d9c46215ce8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkMergeSlash.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkMergeSlash.java @@ -34,17 +34,16 @@ 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 @@ protected FileSystemTestHelper createFileSystemHelper() { return new FileSystemTestHelper(TEST_TEMP_PATH); } - @BeforeClass + @BeforeAll public static void clusterSetupAtBeginning() throws IOException, LoginException, URISyntaxException { SupportsBlocks = true; @@ -93,7 +92,7 @@ public static void clusterSetupAtBeginning() throws IOException, fsDefault = FS_HDFS[FS_INDEX_DEFAULT]; } - @AfterClass + @AfterAll public static void clusterShutdownAtEnd() throws Exception { if (cluster != null) { cluster.shutdown(); @@ -101,7 +100,7 @@ public static void clusterShutdownAtEnd() throws Exception { } @Override - @Before + @BeforeEach public void setUp() throws Exception { fsTarget = fsDefault; super.setUp(); @@ -191,9 +190,8 @@ public void testConfLinkMergeSlashWithRegularLinks() throws Exception { 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 void testChildFileSystems() throws Exception { 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!"); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java index d3afa47f7554b..85c417102c6be 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java @@ -36,16 +36,17 @@ 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 @@ protected FileSystemTestHelper createFileSystemHelper() { return new FileSystemTestHelper(TEST_BASE_PATH); } - @BeforeClass + @BeforeAll public static void clusterSetupAtBeginning() throws IOException { SupportsBlocks = true; clusterConfig = ViewFileSystemTestSetup.createConfig(); @@ -91,7 +92,7 @@ public static void clusterSetupAtBeginning() throws IOException { fsDefault = FS_HDFS[FS_INDEX_DEFAULT]; } - @AfterClass + @AfterAll public static void clusterShutdownAtEnd() throws Exception { if (cluster != null) { cluster.shutdown(); @@ -99,7 +100,7 @@ public static void clusterShutdownAtEnd() throws Exception { } @Override - @Before + @BeforeEach public void setUp() throws Exception { fsTarget = fsDefault; super.setUp(); @@ -157,7 +158,7 @@ public String linkInterceptorSettings( private void createDirWithChildren( FileSystem fileSystem, Path dir, List 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 @@ private void testRegexMountpoint( 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; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemOverloadSchemeHdfsFileSystemContract.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemOverloadSchemeHdfsFileSystemContract.java index 3d0e705e6a98c..38cab5c93c45c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemOverloadSchemeHdfsFileSystemContract.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemOverloadSchemeHdfsFileSystemContract.java @@ -19,7 +19,8 @@ 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.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 static void init() throws IOException { "/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 void setUp() throws Exception { fs = FileSystem.get(conf); } - @AfterClass + @AfterAll public static void tearDownAfter() throws Exception { if (cluster != null) { cluster.shutdown(); @@ -103,15 +105,18 @@ protected String getDefaultWorkingDirectory() { } @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 void testListStatusRootDir() throws Throwable { } @Override - @Ignore // This test same as above in this case. + @Disabled // This test same as above in this case. public void testLSRootDir() throws Throwable { } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemOverloadSchemeWithHdfsScheme.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemOverloadSchemeWithHdfsScheme.java index 76eee6d727c06..07d171228033d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemOverloadSchemeWithHdfsScheme.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemOverloadSchemeWithHdfsScheme.java @@ -39,16 +39,20 @@ 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 static void init() throws IOException { /** * 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 void setUp() throws IOException { 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 void cleanUp() throws IOException { 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 void cleanUp() throws IOException { } } - @AfterClass + @AfterAll public static void tearDown() throws IOException { if (cluster != null) { FileSystem.closeAll(); @@ -136,7 +140,8 @@ void addMountLinks(String mountTable, String[] sources, String[] targets, * 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 void testMountLinkWithLocalAndHDFS() throws Exception { 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 void testMountLinkWithLocalAndHDFS() throws Exception { // 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 void testMountLinkWithLocalAndHDFS() throws Exception { * 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 void testMountLinkWithNonExistentLink(boolean expectFsInitFailure) }); } else { try (FileSystem fs = FileSystem.get(conf)) { - Assert.assertEquals("hdfs", fs.getScheme()); + assertEquals("hdfs", fs.getScheme()); } } } @@ -221,7 +227,8 @@ public void testMountLinkWithNonExistentLink(boolean expectFsInitFailure) * 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 void testListStatusOnRootShouldListAllMountLinks() throws Exception { 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 void testListStatusOnRootShouldListAllMountLinks() throws Exception { * 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 void testAccessViewFsPathWithoutAuthority() throws Exception { * 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 void testWithLinkFallBack() throws Exception { 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 void testWithLinkFallBack() throws Exception { * * 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 void testCreateOnRoot(boolean fallbackExist) throws Exception { 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 void testCreateOnRoot(boolean fallbackExist) throws Exception { * 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 void testInvalidOverloadSchemeTargetFS() throws Exception { * * 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 void testViewFsOverloadSchemeWhenInnerCacheDisabled() 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 void testViewFsOverloadSchemeWhenInnerCacheDisabled() * 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 void testViewFsOverloadSchemeWithInnerCache() // 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 @@ private boolean isFallBackExist(Configuration config) { * 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 void testViewFsOverloadSchemeWithNoInnerCacheAndHdfsTargets() 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 void testViewFsOverloadSchemeWithNoInnerCacheAndHdfsTargets() * 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 void testViewFsOverloadSchemeWithNoInnerCacheAndLocalSchemeTargets() // 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 void testViewFsOverloadSchemeWithNoInnerCacheAndLocalSchemeTargets() /** * 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 void testNflyRename() throws Exception { 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 void testNflyRename() throws Exception { 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 void testNflyWriteRead() throws Exception { * 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 void testNflyRepair() throws Exception { * 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 @@ private void writeString(final FileSystem nfly, final String testString, 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"); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithAcls.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithAcls.java index 10b6f17ad2843..dcde967c8287f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithAcls.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithAcls.java @@ -28,11 +28,11 @@ 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.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 static void clusterSetupAtBeginning() throws IOException { 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 @@ private void setupMountPoints() { 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); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithTruncate.java index 45138b78f73f1..d0aee8553c1b7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithTruncate.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithTruncate.java @@ -31,13 +31,14 @@ 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 static void clusterSetupAtBeginning() throws IOException { 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 @@ private void setupMountPoints() { .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 void testTruncateWithViewFileSystem() 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() { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithXAttrs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithXAttrs.java index b487188c4e1fa..8de34db4f6694 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithXAttrs.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithXAttrs.java @@ -24,16 +24,16 @@ 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 static void clusterSetupAtBeginning() throws IOException { 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 @@ private void setupMountPoints() { targetTestRoot2.toUri()); } - @After + @AfterEach public void tearDown() throws Exception { fsTarget.delete(fileSystemTestHelper.getTestRootPath(fsTarget), true); fsTarget2.delete(fileSystemTestHelper.getTestRootPath(fsTarget2), true); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsAtHdfsRoot.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsAtHdfsRoot.java index 886646518838b..d53844f24c1ef 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsAtHdfsRoot.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsAtHdfsRoot.java @@ -30,9 +30,9 @@ 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 @@ protected FileContextTestHelper createFileContextHelper() { return new FileContextTestHelper("/tmp/TestViewFsAtHdfsRoot"); } - @BeforeClass + @BeforeAll public static void clusterSetupAtBegining() throws IOException, LoginException, URISyntaxException { SupportsBlocks = true; @@ -62,7 +62,7 @@ public static void clusterSetupAtBegining() throws IOException, } - @AfterClass + @AfterAll public static void ClusterShutdownAtEnd() throws Exception { if (cluster != null) { cluster.shutdown(); @@ -70,7 +70,7 @@ public static void ClusterShutdownAtEnd() throws Exception { } @Override - @Before + @BeforeEach public void setUp() throws Exception { // create the test root on local_fs fcTarget = fc; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsDefaultValue.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsDefaultValue.java index a49735c2e86cd..e1881eea7e0f6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsDefaultValue.java +++ b/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_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.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 void testGetQuotaUsageWithQuotaDefined() throws IOException { assertTrue(qu.getSpaceConsumed() > 0); } - @AfterClass + @AfterAll public static void cleanup() throws IOException { fHdfs.delete(new Path(testFileName), true); fHdfs.delete(notInMountpointPath, true); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsFileStatusHdfs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsFileStatusHdfs.java index e3b4fe25fc332..3bdeb0c47b1ef 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsFileStatusHdfs.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsFileStatusHdfs.java @@ -23,8 +23,8 @@ * 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.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 void testGetFileChecksum() throws IOException, URISyntaxException { 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); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsHdfs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsHdfs.java index 540883dd2e208..f0eeab9f44ca2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsHdfs.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsHdfs.java @@ -36,13 +36,13 @@ 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 @@ protected FileContextTestHelper createFileContextHelper() { } - @BeforeClass + @BeforeAll public static void clusterSetupAtBegining() throws IOException, LoginException, URISyntaxException { SupportsBlocks = true; @@ -72,7 +72,7 @@ public static void clusterSetupAtBegining() throws IOException, } - @AfterClass + @AfterAll public static void ClusterShutdownAtEnd() throws Exception { if (cluster != null) { cluster.shutdown(); @@ -80,7 +80,7 @@ public static void ClusterShutdownAtEnd() throws Exception { } @Override - @Before + @BeforeEach public void setUp() throws Exception { // create the test root on local_fs fcTarget = fc; @@ -160,7 +160,7 @@ public Object run() throws IOException { 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); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsLinkFallback.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsLinkFallback.java index ba4a80ca11549..28cb96ec33f32 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsLinkFallback.java +++ b/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.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 static void clusterSetupAtBeginning() } - @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 void testMkdirShouldFailWhenFallbackFSNotAvailable() // 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 void testCreateFileOnRootWithFallbackEnabled() * 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 @@ private void verifyRename(AbstractFileSystem fs, Path src, Path dst) fs.rename(src, dst, Options.Rename.OVERWRITE); LambdaTestUtils .intercept(FileNotFoundException.class, () -> fs.getFileStatus(src)); - Assert.assertNotNull(fs.getFileStatus(dst)); + assertNotNull(fs.getFileStatus(dst)); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsWithAcls.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsWithAcls.java index 1243add66a136..920cfcc7cc14b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsWithAcls.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsWithAcls.java @@ -29,11 +29,11 @@ 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.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 static void clusterSetupAtBeginning() throws IOException { 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 @@ private void setupMountPoints() { 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); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsWithXAttrs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsWithXAttrs.java index 9a4223a86fe20..90bfedbe991ad 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsWithXAttrs.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsWithXAttrs.java @@ -25,16 +25,16 @@ 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 static void clusterSetupAtBeginning() throws IOException { 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 @@ private void setupMountPoints() { 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); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestViewDistributedFileSystemWithMountLinks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestViewDistributedFileSystemWithMountLinks.java index 1e66252d5171a..6e89e44d3ce8f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestViewDistributedFileSystemWithMountLinks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestViewDistributedFileSystemWithMountLinks.java @@ -25,17 +25,22 @@ 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 void setUp() throws IOException { 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 void testRenameWhenDstOnInternalDirWithFallback() throws Exception { 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)); } }