diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/ITestProvidedImplementation.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/ITestProvidedImplementation.java index fa8c1c172d278..e2076485f8fa7 100644 --- a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/ITestProvidedImplementation.java +++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/ITestProvidedImplementation.java @@ -90,11 +90,12 @@ import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NodeBase; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TestName; +import org.apache.hadoop.test.TestName; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.RegisterExtension; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -107,14 +108,20 @@ import static org.apache.hadoop.hdfs.server.common.Util.fileAsURI; import static org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap.fileNameFromBlockPoolID; import static org.apache.hadoop.net.NodeBase.PATH_SEPARATOR_STR; -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.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; /** * Integration tests for the Provided implementation. */ public class ITestProvidedImplementation { - @Rule public TestName name = new TestName(); + @RegisterExtension + private TestName name = new TestName(); + public static final Logger LOG = LoggerFactory.getLogger(ITestProvidedImplementation.class); @@ -136,7 +143,7 @@ public class ITestProvidedImplementation { private Configuration conf; private MiniDFSCluster cluster; - @Before + @BeforeEach public void setSeed() throws Exception { if (fBASE.exists() && !FileUtil.fullyDelete(fBASE)) { throw new IOException("Could not fully delete " + fBASE); @@ -196,7 +203,7 @@ public void setSeed() throws Exception { } } - @After + @AfterEach public void shutdown() throws Exception { try { if (cluster != null) { @@ -312,7 +319,8 @@ private static List getProvidedNamenodeDirs(String baseDir, return nnDirs; } - @Test(timeout=20000) + @Test + @Timeout(value = 20) public void testLoadImage() throws Exception { final long seed = r.nextLong(); LOG.info("providedPath: " + providedPath); @@ -338,7 +346,8 @@ public void testLoadImage() throws Exception { } } - @Test(timeout=30000) + @Test + @Timeout(value = 30) public void testProvidedReporting() throws Exception { conf.setClass(ImageWriter.Options.UGI_CLASS, SingleUGIResolver.class, UGIResolver.class); @@ -417,7 +426,8 @@ public void testProvidedReporting() throws Exception { } } - @Test(timeout=500000) + @Test + @Timeout(value = 500) public void testDefaultReplication() throws Exception { int targetReplication = 2; conf.setInt(FixedBlockMultiReplicaResolver.REPLICATION, targetReplication); @@ -529,7 +539,8 @@ private BlockLocation[] createFile(Path path, short replication, return fs.getFileBlockLocations(path, 0, fileLen); } - @Test(timeout=30000) + @Test + @Timeout(value = 30) public void testClusterWithEmptyImage() throws IOException { // start a cluster with 2 datanodes without any provided storage startCluster(nnDirPath, 2, null, @@ -567,8 +578,8 @@ private DatanodeInfo[] getAndCheckBlockLocations(DFSClient client, private void checkUniqueness(DatanodeInfo[] locations) { Set set = new HashSet<>(); for (DatanodeInfo info: locations) { - assertFalse("All locations should be unique", - set.contains(info.getDatanodeUuid())); + assertFalse(set.contains(info.getDatanodeUuid()), + "All locations should be unique"); set.add(info.getDatanodeUuid()); } } @@ -577,7 +588,8 @@ private void checkUniqueness(DatanodeInfo[] locations) { * Tests setting replication of provided files. * @throws Exception */ - @Test(timeout=50000) + @Test + @Timeout(value = 50) public void testSetReplicationForProvidedFiles() throws Exception { createImage(new FSTreeWalk(providedPath, conf), nnDirPath, FixedBlockResolver.class); @@ -618,7 +630,8 @@ private void setAndUnsetReplication(String filename) throws Exception { defaultReplication); } - @Test(timeout=30000) + @Test + @Timeout(value = 30) public void testProvidedDatanodeFailures() throws Exception { createImage(new FSTreeWalk(providedPath, conf), nnDirPath, FixedBlockResolver.class); @@ -689,7 +702,8 @@ public void testProvidedDatanodeFailures() throws Exception { } } - @Test(timeout=300000) + @Test + @Timeout(value = 300) public void testTransientDeadDatanodes() throws Exception { createImage(new FSTreeWalk(providedPath, conf), nnDirPath, FixedBlockResolver.class); @@ -727,7 +741,8 @@ private DatanodeStorageInfo getProvidedDatanodeStorageInfo() { return providedStorageMap.getProvidedStorageInfo(); } - @Test(timeout=30000) + @Test + @Timeout(value = 30) public void testNamenodeRestart() throws Exception { createImage(new FSTreeWalk(providedPath, conf), nnDirPath, FixedBlockResolver.class); @@ -768,7 +783,8 @@ private void verifyFileLocation(int fileIndex, int replication) } } - @Test(timeout=30000) + @Test + @Timeout(value = 30) public void testSetClusterID() throws Exception { String clusterID = "PROVIDED-CLUSTER"; createImage(new FSTreeWalk(providedPath, conf), nnDirPath, @@ -783,7 +799,8 @@ public void testSetClusterID() throws Exception { assertEquals(clusterID, nn.getNamesystem().getClusterId()); } - @Test(timeout=30000) + @Test + @Timeout(value = 30) public void testNumberOfProvidedLocations() throws Exception { // set default replication to 4 conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 4); @@ -814,7 +831,8 @@ public void testNumberOfProvidedLocations() throws Exception { } } - @Test(timeout=30000) + @Test + @Timeout(value = 30) public void testNumberOfProvidedLocationsManyBlocks() throws Exception { // increase number of blocks per file to at least 10 blocks per file conf.setLong(FixedBlockResolver.BLOCKSIZE, baseFileLen/10); diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSTreeWalk.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSTreeWalk.java index 19e52cf43a9ee..88d66db02d370 100644 --- a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSTreeWalk.java +++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSTreeWalk.java @@ -23,14 +23,12 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.AclStatus; import org.apache.hadoop.hdfs.DFSConfigKeys; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.HashMap; import java.util.Map; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.*; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -84,37 +82,37 @@ public void testImportAcl() throws Exception { * Verify ACL enabled TreeWalk iterator throws an error if the external file * system does not support ACLs. */ - @Test(expected = UnsupportedOperationException.class) + @Test public void testACLNotSupported() throws Exception { - Configuration conf = new Configuration(); - conf.setBoolean(DFSConfigKeys.DFS_PROVIDED_ACLS_IMPORT_ENABLED, true); - - FileSystem fs = mock(FileSystem.class); - when(fs.getAclStatus(any())).thenThrow(new UnsupportedOperationException()); - Path root = mock(Path.class); - when(root.getFileSystem(conf)).thenReturn(fs); - FileStatus rootFileStatus = new FileStatus(0, true, 0, 0, 1, root); - when(fs.getFileStatus(root)).thenReturn(rootFileStatus); - - FSTreeWalk fsTreeWalk = new FSTreeWalk(root, conf); - TreeWalk.TreeIterator iter = fsTreeWalk.iterator(); - fail("Unexpected successful creation of iter: " + iter); + assertThrows(UnsupportedOperationException.class, () -> { + Configuration conf = new Configuration(); + conf.setBoolean(DFSConfigKeys.DFS_PROVIDED_ACLS_IMPORT_ENABLED, true); + FileSystem fs = mock(FileSystem.class); + when(fs.getAclStatus(any())).thenThrow(new UnsupportedOperationException()); + Path root = mock(Path.class); + when(root.getFileSystem(conf)).thenReturn(fs); + FileStatus rootFileStatus = new FileStatus(0, true, 0, 0, 1, root); + when(fs.getFileStatus(root)).thenReturn(rootFileStatus); + FSTreeWalk fsTreeWalk = new FSTreeWalk(root, conf); + TreeWalk.TreeIterator iter = fsTreeWalk.iterator(); + fail("Unexpected successful creation of iter: " + iter); + }); } /** * Verify creation of INode for ACL enabled TreePath throws an error. */ - @Test(expected = UnsupportedOperationException.class) + @Test public void testToINodeACLNotSupported() throws Exception { - BlockResolver blockResolver = new FixedBlockResolver(); - Path root = new Path("/"); - FileStatus rootFileStatus = new FileStatus(0, false, 0, 0, 1, root); - - AclStatus acls = mock(AclStatus.class); - TreePath treePath = new TreePath(rootFileStatus, 1, null, null, acls); - - UGIResolver ugiResolver = mock(UGIResolver.class); - when(ugiResolver.getPermissionsProto(null, acls)).thenReturn(1L); - treePath.toINode(ugiResolver, blockResolver, null); + assertThrows(UnsupportedOperationException.class, () -> { + BlockResolver blockResolver = new FixedBlockResolver(); + Path root = new Path("/"); + FileStatus rootFileStatus = new FileStatus(0, false, 0, 0, 1, root); + AclStatus acls = mock(AclStatus.class); + TreePath treePath = new TreePath(rootFileStatus, 1, null, null, acls); + UGIResolver ugiResolver = mock(UGIResolver.class); + when(ugiResolver.getPermissionsProto(null, acls)).thenReturn(1L); + treePath.toINode(ugiResolver, blockResolver, null); + }); } } diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFixedBlockResolver.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFixedBlockResolver.java index 8b52ffd8fa2e7..c6a16d1c63292 100644 --- a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFixedBlockResolver.java +++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFixedBlockResolver.java @@ -25,22 +25,26 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TestName; -import static org.junit.Assert.*; +import org.apache.hadoop.test.TestName; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Validate fixed-size block partitioning. */ public class TestFixedBlockResolver { - @Rule public TestName name = new TestName(); + @RegisterExtension + private TestName name = new TestName(); private final FixedBlockResolver blockId = new FixedBlockResolver(); - @Before + @BeforeEach public void setup() { Configuration conf = new Configuration(false); conf.setLong(FixedBlockResolver.BLOCKSIZE, 512L * (1L << 20)); diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRandomTreeWalk.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRandomTreeWalk.java index b8e6ac9ac6789..25051bbddbb97 100644 --- a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRandomTreeWalk.java +++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRandomTreeWalk.java @@ -25,12 +25,13 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.test.TestName; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TestName; -import static org.junit.Assert.*; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Validate randomly generated hierarchies, including fork() support in @@ -38,11 +39,12 @@ */ public class TestRandomTreeWalk { - @Rule public TestName name = new TestName(); + @RegisterExtension + private TestName name = new TestName(); private Random r = new Random(); - @Before + @BeforeEach public void setSeed() { long seed = r.nextLong(); r.setSeed(seed); diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSingleUGIResolver.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSingleUGIResolver.java index c242c30faaff1..419d913dec582 100644 --- a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSingleUGIResolver.java +++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSingleUGIResolver.java @@ -31,18 +31,20 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.security.UserGroupInformation; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TestName; -import static org.junit.Assert.*; +import org.apache.hadoop.test.TestName; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; /** * Validate resolver assigning all paths to a single owner/group. */ public class TestSingleUGIResolver { - @Rule public TestName name = new TestName(); + @RegisterExtension + private TestName name = new TestName(); private static final int TESTUID = 10101; private static final int TESTGID = 10102; @@ -51,7 +53,7 @@ public class TestSingleUGIResolver { private SingleUGIResolver ugi = new SingleUGIResolver(); - @Before + @BeforeEach public void setup() { Configuration conf = new Configuration(false); conf.setInt(SingleUGIResolver.UID, TESTUID); @@ -125,31 +127,37 @@ public void testAclResolution() { match(perm, p1); } - @Test(expected=IllegalArgumentException.class) + @Test public void testInvalidUid() { - Configuration conf = ugi.getConf(); - conf.setInt(SingleUGIResolver.UID, (1 << 24) + 1); - ugi.setConf(conf); - ugi.resolve(file(TESTUSER, TESTGROUP, new FsPermission((short)0777))); + assertThrows(IllegalArgumentException.class, () -> { + Configuration conf = ugi.getConf(); + conf.setInt(SingleUGIResolver.UID, (1 << 24) + 1); + ugi.setConf(conf); + ugi.resolve(file(TESTUSER, TESTGROUP, new FsPermission((short) 0777))); + }); } - @Test(expected=IllegalArgumentException.class) + @Test public void testInvalidGid() { - Configuration conf = ugi.getConf(); - conf.setInt(SingleUGIResolver.GID, (1 << 24) + 1); - ugi.setConf(conf); - ugi.resolve(file(TESTUSER, TESTGROUP, new FsPermission((short)0777))); + assertThrows(IllegalArgumentException.class, () -> { + Configuration conf = ugi.getConf(); + conf.setInt(SingleUGIResolver.GID, (1 << 24) + 1); + ugi.setConf(conf); + ugi.resolve(file(TESTUSER, TESTGROUP, new FsPermission((short) 0777))); + }); } - @Test(expected=IllegalStateException.class) + @Test public void testDuplicateIds() { - Configuration conf = new Configuration(false); - conf.setInt(SingleUGIResolver.UID, 4344); - conf.setInt(SingleUGIResolver.GID, 4344); - conf.set(SingleUGIResolver.USER, TESTUSER); - conf.set(SingleUGIResolver.GROUP, TESTGROUP); - ugi.setConf(conf); - ugi.ugiMap(); + assertThrows(IllegalStateException.class, () -> { + Configuration conf = new Configuration(false); + conf.setInt(SingleUGIResolver.UID, 4344); + conf.setInt(SingleUGIResolver.GID, 4344); + conf.set(SingleUGIResolver.USER, TESTUSER); + conf.set(SingleUGIResolver.GROUP, TESTGROUP); + ugi.setConf(conf); + ugi.ugiMap(); + }); } static void match(long encoded, FsPermission p) { @@ -176,5 +184,4 @@ static FileStatus file(String user, String group, FsPermission perm) { group, /* String group, */ p); /* Path path */ } - }