浏览代码

HADOOP-19435. [JDK17] Upgrade JUnit from 4 to 5 in hadoop-fs2img. (#7579)

* HADOOP-19435. [JDK17] Upgrade JUnit from 4 to 5 in hadoop-fs2img.

Co-authored-by: Chris Nauroth <cnauroth@apache.org>
Reviewed-by: Chris Nauroth <cnauroth@apache.org>
Signed-off-by: Shilun Fan <slfan1989@apache.org>
slfan1989 3 周之前
父节点
当前提交
af38897f90

+ 40 - 22
hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/ITestProvidedImplementation.java

@@ -90,11 +90,12 @@ import org.apache.hadoop.hdfs.util.RwLockMode;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NodeBase;
 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.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
@@ -107,14 +108,20 @@ import static org.apache.hadoop.hdfs.MiniDFSCluster.HDFS_MINIDFS_BASEDIR;
 import static org.apache.hadoop.hdfs.server.common.Util.fileAsURI;
 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.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap.fileNameFromBlockPoolID;
 import static org.apache.hadoop.net.NodeBase.PATH_SEPARATOR_STR;
 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.
  * Integration tests for the Provided implementation.
  */
  */
 public class ITestProvidedImplementation {
 public class ITestProvidedImplementation {
 
 
-  @Rule public TestName name = new TestName();
+  @RegisterExtension
+  private TestName name = new TestName();
+
   public static final Logger LOG =
   public static final Logger LOG =
       LoggerFactory.getLogger(ITestProvidedImplementation.class);
       LoggerFactory.getLogger(ITestProvidedImplementation.class);
 
 
@@ -136,7 +143,7 @@ public class ITestProvidedImplementation {
   private Configuration conf;
   private Configuration conf;
   private MiniDFSCluster cluster;
   private MiniDFSCluster cluster;
 
 
-  @Before
+  @BeforeEach
   public void setSeed() throws Exception {
   public void setSeed() throws Exception {
     if (fBASE.exists() && !FileUtil.fullyDelete(fBASE)) {
     if (fBASE.exists() && !FileUtil.fullyDelete(fBASE)) {
       throw new IOException("Could not fully delete " + fBASE);
       throw new IOException("Could not fully delete " + fBASE);
@@ -196,7 +203,7 @@ public class ITestProvidedImplementation {
     }
     }
   }
   }
 
 
-  @After
+  @AfterEach
   public void shutdown() throws Exception {
   public void shutdown() throws Exception {
     try {
     try {
       if (cluster != null) {
       if (cluster != null) {
@@ -312,7 +319,8 @@ public class ITestProvidedImplementation {
     return nnDirs;
     return nnDirs;
   }
   }
 
 
-  @Test(timeout=20000)
+  @Test
+  @Timeout(value = 20)
   public void testLoadImage() throws Exception {
   public void testLoadImage() throws Exception {
     final long seed = r.nextLong();
     final long seed = r.nextLong();
     LOG.info("providedPath: " + providedPath);
     LOG.info("providedPath: " + providedPath);
@@ -338,7 +346,8 @@ public class ITestProvidedImplementation {
     }
     }
   }
   }
 
 
-  @Test(timeout=30000)
+  @Test
+  @Timeout(value = 30)
   public void testProvidedReporting() throws Exception {
   public void testProvidedReporting() throws Exception {
     conf.setClass(ImageWriter.Options.UGI_CLASS,
     conf.setClass(ImageWriter.Options.UGI_CLASS,
         SingleUGIResolver.class, UGIResolver.class);
         SingleUGIResolver.class, UGIResolver.class);
@@ -417,7 +426,8 @@ public class ITestProvidedImplementation {
     }
     }
   }
   }
 
 
-  @Test(timeout=500000)
+  @Test
+  @Timeout(value = 500)
   public void testDefaultReplication() throws Exception {
   public void testDefaultReplication() throws Exception {
     int targetReplication = 2;
     int targetReplication = 2;
     conf.setInt(FixedBlockMultiReplicaResolver.REPLICATION, targetReplication);
     conf.setInt(FixedBlockMultiReplicaResolver.REPLICATION, targetReplication);
@@ -529,7 +539,8 @@ public class ITestProvidedImplementation {
     return fs.getFileBlockLocations(path, 0, fileLen);
     return fs.getFileBlockLocations(path, 0, fileLen);
   }
   }
 
 
-  @Test(timeout=30000)
+  @Test
+  @Timeout(value = 30)
   public void testClusterWithEmptyImage() throws IOException {
   public void testClusterWithEmptyImage() throws IOException {
     // start a cluster with 2 datanodes without any provided storage
     // start a cluster with 2 datanodes without any provided storage
     startCluster(nnDirPath, 2, null,
     startCluster(nnDirPath, 2, null,
@@ -567,8 +578,8 @@ public class ITestProvidedImplementation {
   private void checkUniqueness(DatanodeInfo[] locations) {
   private void checkUniqueness(DatanodeInfo[] locations) {
     Set<String> set = new HashSet<>();
     Set<String> set = new HashSet<>();
     for (DatanodeInfo info: locations) {
     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());
       set.add(info.getDatanodeUuid());
     }
     }
   }
   }
@@ -577,7 +588,8 @@ public class ITestProvidedImplementation {
    * Tests setting replication of provided files.
    * Tests setting replication of provided files.
    * @throws Exception
    * @throws Exception
    */
    */
-  @Test(timeout=50000)
+  @Test
+  @Timeout(value = 50)
   public void testSetReplicationForProvidedFiles() throws Exception {
   public void testSetReplicationForProvidedFiles() throws Exception {
     createImage(new FSTreeWalk(providedPath, conf), nnDirPath,
     createImage(new FSTreeWalk(providedPath, conf), nnDirPath,
         FixedBlockResolver.class);
         FixedBlockResolver.class);
@@ -618,7 +630,8 @@ public class ITestProvidedImplementation {
         defaultReplication);
         defaultReplication);
   }
   }
 
 
-  @Test(timeout=30000)
+  @Test
+  @Timeout(value = 30)
   public void testProvidedDatanodeFailures() throws Exception {
   public void testProvidedDatanodeFailures() throws Exception {
     createImage(new FSTreeWalk(providedPath, conf), nnDirPath,
     createImage(new FSTreeWalk(providedPath, conf), nnDirPath,
             FixedBlockResolver.class);
             FixedBlockResolver.class);
@@ -689,7 +702,8 @@ public class ITestProvidedImplementation {
     }
     }
   }
   }
 
 
-  @Test(timeout=300000)
+  @Test
+  @Timeout(value = 300)
   public void testTransientDeadDatanodes() throws Exception {
   public void testTransientDeadDatanodes() throws Exception {
     createImage(new FSTreeWalk(providedPath, conf), nnDirPath,
     createImage(new FSTreeWalk(providedPath, conf), nnDirPath,
             FixedBlockResolver.class);
             FixedBlockResolver.class);
@@ -727,7 +741,8 @@ public class ITestProvidedImplementation {
     return providedStorageMap.getProvidedStorageInfo();
     return providedStorageMap.getProvidedStorageInfo();
   }
   }
 
 
-  @Test(timeout=30000)
+  @Test
+  @Timeout(value = 30)
   public void testNamenodeRestart() throws Exception {
   public void testNamenodeRestart() throws Exception {
     createImage(new FSTreeWalk(providedPath, conf), nnDirPath,
     createImage(new FSTreeWalk(providedPath, conf), nnDirPath,
         FixedBlockResolver.class);
         FixedBlockResolver.class);
@@ -768,7 +783,8 @@ public class ITestProvidedImplementation {
     }
     }
   }
   }
 
 
-  @Test(timeout=30000)
+  @Test
+  @Timeout(value = 30)
   public void testSetClusterID() throws Exception {
   public void testSetClusterID() throws Exception {
     String clusterID = "PROVIDED-CLUSTER";
     String clusterID = "PROVIDED-CLUSTER";
     createImage(new FSTreeWalk(providedPath, conf), nnDirPath,
     createImage(new FSTreeWalk(providedPath, conf), nnDirPath,
@@ -783,7 +799,8 @@ public class ITestProvidedImplementation {
     assertEquals(clusterID, nn.getNamesystem().getClusterId());
     assertEquals(clusterID, nn.getNamesystem().getClusterId());
   }
   }
 
 
-  @Test(timeout=30000)
+  @Test
+  @Timeout(value = 30)
   public void testNumberOfProvidedLocations() throws Exception {
   public void testNumberOfProvidedLocations() throws Exception {
     // set default replication to 4
     // set default replication to 4
     conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 4);
     conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 4);
@@ -814,7 +831,8 @@ public class ITestProvidedImplementation {
     }
     }
   }
   }
 
 
-  @Test(timeout=30000)
+  @Test
+  @Timeout(value = 30)
   public void testNumberOfProvidedLocationsManyBlocks() throws Exception {
   public void testNumberOfProvidedLocationsManyBlocks() throws Exception {
     // increase number of blocks per file to at least 10 blocks per file
     // increase number of blocks per file to at least 10 blocks per file
     conf.setLong(FixedBlockResolver.BLOCKSIZE, baseFileLen/10);
     conf.setLong(FixedBlockResolver.BLOCKSIZE, baseFileLen/10);

+ 27 - 29
hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSTreeWalk.java

@@ -23,14 +23,12 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.Map;
 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.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.when;
@@ -84,37 +82,37 @@ public class TestFSTreeWalk {
    * Verify ACL enabled TreeWalk iterator throws an error if the external file
    * Verify ACL enabled TreeWalk iterator throws an error if the external file
    * system does not support ACLs.
    * system does not support ACLs.
    */
    */
-  @Test(expected = UnsupportedOperationException.class)
+  @Test
   public void testACLNotSupported() throws Exception {
   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.
    * Verify creation of INode for ACL enabled TreePath throws an error.
    */
    */
-  @Test(expected = UnsupportedOperationException.class)
+  @Test
   public void testToINodeACLNotSupported() throws Exception {
   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);
+    });
   }
   }
 }
 }

+ 11 - 7
hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFixedBlockResolver.java

@@ -25,22 +25,26 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 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.
  * Validate fixed-size block partitioning.
  */
  */
 public class TestFixedBlockResolver {
 public class TestFixedBlockResolver {
 
 
-  @Rule public TestName name = new TestName();
+  @RegisterExtension
+  private TestName name = new TestName();
 
 
   private final FixedBlockResolver blockId = new FixedBlockResolver();
   private final FixedBlockResolver blockId = new FixedBlockResolver();
 
 
-  @Before
+  @BeforeEach
   public void setup() {
   public void setup() {
     Configuration conf = new Configuration(false);
     Configuration conf = new Configuration(false);
     conf.setLong(FixedBlockResolver.BLOCKSIZE, 512L * (1L << 20));
     conf.setLong(FixedBlockResolver.BLOCKSIZE, 512L * (1L << 20));

+ 9 - 7
hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRandomTreeWalk.java

@@ -25,12 +25,13 @@ import java.util.Set;
 
 
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 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
  * Validate randomly generated hierarchies, including fork() support in
@@ -38,11 +39,12 @@ import static org.junit.Assert.*;
  */
  */
 public class TestRandomTreeWalk {
 public class TestRandomTreeWalk {
 
 
-  @Rule public TestName name = new TestName();
+  @RegisterExtension
+  private TestName name = new TestName();
 
 
   private Random r = new Random();
   private Random r = new Random();
 
 
-  @Before
+  @BeforeEach
   public void setSeed() {
   public void setSeed() {
     long seed = r.nextLong();
     long seed = r.nextLong();
     r.setSeed(seed);
     r.setSeed(seed);

+ 33 - 26
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.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.UserGroupInformation;
 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.
  * Validate resolver assigning all paths to a single owner/group.
  */
  */
 public class TestSingleUGIResolver {
 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 TESTUID = 10101;
   private static final int TESTGID = 10102;
   private static final int TESTGID = 10102;
@@ -51,7 +53,7 @@ public class TestSingleUGIResolver {
 
 
   private SingleUGIResolver ugi = new SingleUGIResolver();
   private SingleUGIResolver ugi = new SingleUGIResolver();
 
 
-  @Before
+  @BeforeEach
   public void setup() {
   public void setup() {
     Configuration conf = new Configuration(false);
     Configuration conf = new Configuration(false);
     conf.setInt(SingleUGIResolver.UID, TESTUID);
     conf.setInt(SingleUGIResolver.UID, TESTUID);
@@ -125,31 +127,37 @@ public class TestSingleUGIResolver {
     match(perm, p1);
     match(perm, p1);
   }
   }
 
 
-  @Test(expected=IllegalArgumentException.class)
+  @Test
   public void testInvalidUid() {
   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() {
   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() {
   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) {
   static void match(long encoded, FsPermission p) {
@@ -176,5 +184,4 @@ public class TestSingleUGIResolver {
           group,                   /* String group,            */
           group,                   /* String group,            */
           p);                      /* Path path                */
           p);                      /* Path path                */
   }
   }
-
 }
 }