diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml index 203daa32745bf..6607a2a300cdf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml @@ -210,6 +210,26 @@ kotlin-stdlib-jdk8 test + + org.junit.jupiter + junit-jupiter-api + test + + + org.junit.jupiter + junit-jupiter-engine + test + + + org.junit.jupiter + junit-jupiter-params + test + + + org.junit.platform + junit-platform-launcher + test + diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java index 4c1d2d176b56f..41ba89d9517cc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java @@ -81,13 +81,10 @@ import org.json.simple.JSONObject; import org.json.simple.parser.ContainerFactory; import org.json.simple.parser.JSONParser; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.webapp.WebAppContext; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import java.io.File; import java.io.FileOutputStream; @@ -107,13 +104,15 @@ import java.util.regex.Pattern; import java.util.concurrent.atomic.AtomicReference; -import static org.junit.Assert.assertArrayEquals; -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.assertArrayEquals; +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.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; +import static org.junit.jupiter.api.Assumptions.assumeFalse; -@RunWith(value = Parameterized.class) public abstract class BaseTestHttpFSWith extends HFSTestCase { protected abstract Path getProxiedFSTestDir(); @@ -251,12 +250,12 @@ private void testCreate() throws Exception { testCreate(path, true); try { testCreate(path, false); - Assert.fail("the create should have failed because the file exists " + - "and override is FALSE"); + fail("the create should have failed because the file exists " + + "and override is FALSE"); } catch (IOException ex) { System.out.println("#"); } catch (Exception ex) { - Assert.fail(ex.toString()); + fail(ex.toString()); } } @@ -299,7 +298,7 @@ private void testTruncate() throws Exception { final int newLength = blockSize; boolean isReady = fs.truncate(file, newLength); - assertTrue("Recovery is not expected.", isReady); + assertTrue(isReady, "Recovery is not expected."); FileStatus fileStatus = fs.getFileStatus(file); assertEquals(fileStatus.getLen(), newLength); @@ -312,8 +311,8 @@ private void testTruncate() throws Exception { private void assertPathCapabilityForTruncate(Path file) throws Exception { FileSystem fs = this.getHttpFSFileSystem(); - assertTrue("HttpFS/WebHdfs/SWebHdfs support truncate", - fs.hasPathCapability(file, CommonPathCapabilities.FS_TRUNCATE)); + assertTrue(fs.hasPathCapability(file, CommonPathCapabilities.FS_TRUNCATE), + "HttpFS/WebHdfs/SWebHdfs support truncate"); fs.close(); } @@ -371,10 +370,10 @@ private void testDelete() throws Exception { assertFalse(fs.exists(foo)); try { hoopFs.delete(new Path(bar.toUri().getPath()), false); - Assert.fail(); + fail(); } catch (IOException ex) { } catch (Exception ex) { - Assert.fail(); + fail(); } assertTrue(fs.exists(bar)); assertTrue(hoopFs.delete(new Path(bar.toUri().getPath()), true)); @@ -467,10 +466,10 @@ private void testListStatus() throws Exception { // The full path should be the path to the file. See HDFS-12139 FileStatus[] statl = fs.listStatus(path); - Assert.assertEquals(1, statl.length); - Assert.assertEquals(status2.getPath(), statl[0].getPath()); - Assert.assertEquals(statl[0].getPath().getName(), path.getName()); - Assert.assertEquals(stati[0].getPath(), statl[0].getPath()); + assertEquals(1, statl.length); + assertEquals(status2.getPath(), statl[0].getPath()); + assertEquals(statl[0].getPath().getName(), path.getName()); + assertEquals(stati[0].getPath(), statl[0].getPath()); } private void testFileStatusAttr() throws Exception { @@ -483,18 +482,18 @@ private void testFileStatusAttr() throws Exception { // Get the FileSystem instance that's being tested FileSystem fs = this.getHttpFSFileSystem(); // Check FileStatus - assertFalse("Snapshot should be disallowed by default", - fs.getFileStatus(path).isSnapshotEnabled()); + assertFalse(fs.getFileStatus(path).isSnapshotEnabled(), + "Snapshot should be disallowed by default"); // Allow snapshot distributedFs.allowSnapshot(path); // Check FileStatus - assertTrue("Snapshot enabled bit is not set in FileStatus", - fs.getFileStatus(path).isSnapshotEnabled()); + assertTrue(fs.getFileStatus(path).isSnapshotEnabled(), + "Snapshot enabled bit is not set in FileStatus"); // Disallow snapshot distributedFs.disallowSnapshot(path); // Check FileStatus - assertFalse("Snapshot enabled bit is not cleared in FileStatus", - fs.getFileStatus(path).isSnapshotEnabled()); + assertFalse(fs.getFileStatus(path).isSnapshotEnabled(), + "Snapshot enabled bit is not cleared in FileStatus"); // Cleanup fs.delete(path, true); fs.close(); @@ -529,7 +528,7 @@ private void testListStatusBatch() throws Exception { // LocalFileSystem writes checksum files next to the data files, which // show up when listing via LFS. This makes the listings not compare // properly. - Assume.assumeFalse(isLocalFS()); + assumeFalse(isLocalFS()); FileSystem proxyFs = FileSystem.get(getProxiedFSConf()); Configuration conf = new Configuration(); @@ -555,13 +554,13 @@ private void testListStatusBatch() throws Exception { RemoteIterator si = proxyFs.listStatusIterator(dir1); FileStatus statusl = si.next(); FileStatus status = proxyFs.getFileStatus(file1); - Assert.assertEquals(file1.getName(), statusl.getPath().getName()); - Assert.assertEquals(status.getPath(), statusl.getPath()); + assertEquals(file1.getName(), statusl.getPath().getName()); + assertEquals(status.getPath(), statusl.getPath()); si = proxyFs.listStatusIterator(file1); statusl = si.next(); - Assert.assertEquals(file1.getName(), statusl.getPath().getName()); - Assert.assertEquals(status.getPath(), statusl.getPath()); + assertEquals(file1.getName(), statusl.getPath().getName()); + assertEquals(status.getPath(), statusl.getPath()); } private void testWorkingdirectory() throws Exception { @@ -845,7 +844,7 @@ private void testSetXAttr() throws Exception { fs.setXAttr(path, name4, value4); try { fs.setXAttr(path, name5, value1); - Assert.fail("Set xAttr with incorrect name format should fail."); + fail("Set xAttr with incorrect name format should fail."); } catch (IOException e) { } catch (IllegalArgumentException e) { } @@ -910,7 +909,7 @@ private void testGetXAttrs() throws Exception { final String name5 = "a1"; try { value = fs.getXAttr(path, name5); - Assert.fail("Get xAttr with incorrect name format should fail."); + fail("Get xAttr with incorrect name format should fail."); } catch (IOException e) { } catch (IllegalArgumentException e) { } @@ -961,7 +960,7 @@ private void testRemoveXAttr() throws Exception { fs.removeXAttr(path, name4); try { fs.removeXAttr(path, name5); - Assert.fail("Remove xAttr with incorrect name format should fail."); + fail("Remove xAttr with incorrect name format should fail."); } catch (IOException e) { } catch (IllegalArgumentException e) { } @@ -1151,7 +1150,7 @@ private void testEncryption() throws Exception { } private void testErasureCoding() throws Exception { - Assume.assumeFalse("Assume its not a local FS!", isLocalFS()); + assumeFalse(isLocalFS(), "Assume its not a local FS!"); FileSystem proxyFs = FileSystem.get(getProxiedFSConf()); FileSystem httpFS = getHttpFSFileSystem(); Path filePath = new Path(getProxiedFSTestDir(), "foo.txt"); @@ -1169,15 +1168,15 @@ private void testErasureCoding() throws Exception { } private void testStoragePolicy() throws Exception { - Assume.assumeFalse("Assume its not a local FS", isLocalFS()); + assumeFalse(isLocalFS(), "Assume its not a local FS"); FileSystem fs = FileSystem.get(getProxiedFSConf()); fs.mkdirs(getProxiedFSTestDir()); Path path = new Path(getProxiedFSTestDir(), "policy.txt"); FileSystem httpfs = getHttpFSFileSystem(); // test getAllStoragePolicies - Assert.assertArrayEquals( - "Policy array returned from the DFS and HttpFS should be equals", - fs.getAllStoragePolicies().toArray(), httpfs.getAllStoragePolicies().toArray()); + assertArrayEquals(fs.getAllStoragePolicies().toArray(), + httpfs.getAllStoragePolicies().toArray(), + "Policy array returned from the DFS and HttpFS should be equals"); // test get/set/unset policies DFSTestUtil.createFile(fs, path, 0, (short) 1, 0L); @@ -1189,22 +1188,15 @@ private void testStoragePolicy() throws Exception { BlockStoragePolicySpi dfsPolicy = fs.getStoragePolicy(path); // get policy from webhdfs BlockStoragePolicySpi httpFsPolicy = httpfs.getStoragePolicy(path); - Assert - .assertEquals( - "Storage policy returned from the get API should" - + " be same as set policy", - HdfsConstants.COLD_STORAGE_POLICY_NAME.toString(), - httpFsPolicy.getName()); - Assert.assertEquals( - "Storage policy returned from the DFS and HttpFS should be equals", - httpFsPolicy, dfsPolicy); + assertEquals(HdfsConstants.COLD_STORAGE_POLICY_NAME.toString(), + httpFsPolicy.getName(), "Storage policy returned from the get API should" + + " be same as set policy"); + assertEquals(httpFsPolicy, dfsPolicy, + "Storage policy returned from the DFS and HttpFS should be equals"); // unset policy httpfs.unsetStoragePolicy(path); - Assert - .assertEquals( - "After unset storage policy, the get API shoudld" - + " return the default policy", - defaultdfsPolicy, httpfs.getStoragePolicy(path)); + assertEquals(defaultdfsPolicy, httpfs.getStoragePolicy(path), + "After unset storage policy, the get API shoudld return the default policy"); fs.close(); } @@ -1380,7 +1372,6 @@ private void operation(Operation op) throws Exception { } } - @Parameterized.Parameters public static Collection operations() { Object[][] ops = new Object[Operation.values().length][]; for (int i = 0; i < Operation.values().length; i++) { @@ -1393,24 +1384,28 @@ public static Collection operations() { private Operation operation; - public BaseTestHttpFSWith(Operation operation) { - this.operation = operation; + public void initBaseTestHttpFSWith(Operation pOperation) { + this.operation = pOperation; } - @Test + @MethodSource("operations") + @ParameterizedTest @TestDir @TestJetty @TestHdfs - public void testOperation() throws Exception { + public void testOperation(Operation pOperation) throws Exception { + initBaseTestHttpFSWith(pOperation); createHttpFSServer(); operation(operation); } - @Test + @MethodSource("operations") + @ParameterizedTest @TestDir @TestJetty @TestHdfs - public void testOperationDoAs() throws Exception { + public void testOperationDoAs(Operation pOperation) throws Exception { + initBaseTestHttpFSWith(pOperation); createHttpFSServer(); UserGroupInformation ugi = UserGroupInformation.createProxyUser(HadoopUsersConfTestHelper.getHadoopUsers()[0], UserGroupInformation.getCurrentUser()); @@ -1436,16 +1431,15 @@ private void testCreateSnapshot(String snapshotName) throws Exception { } Path snapshotsDir = new Path("/tmp/tmp-snap-test/.snapshot"); FileStatus[] snapshotItems = fs.listStatus(snapshotsDir); - assertTrue("Should have exactly one snapshot.", - snapshotItems.length == 1); + assertTrue(snapshotItems.length == 1, + "Should have exactly one snapshot."); String resultingSnapName = snapshotItems[0].getPath().getName(); if (snapshotName == null) { - assertTrue("Snapshot auto generated name not matching pattern", - Pattern.matches("(s)(\\d{8})(-)(\\d{6})(\\.)(\\d{3})", - resultingSnapName)); + assertTrue(Pattern.matches("(s)(\\d{8})(-)(\\d{6})(\\.)(\\d{3})", + resultingSnapName), "Snapshot auto generated name not matching pattern"); } else { - assertTrue("Snapshot name is not same as passed name.", - snapshotName.equals(resultingSnapName)); + assertTrue(snapshotName.equals(resultingSnapName), + "Snapshot name is not same as passed name."); } cleanSnapshotTests(snapshottablePath, resultingSnapName); } @@ -1495,11 +1489,11 @@ private void testRenameSnapshot() throws Exception { "snap-new-name"); Path snapshotsDir = new Path("/tmp/tmp-snap-test/.snapshot"); FileStatus[] snapshotItems = fs.listStatus(snapshotsDir); - assertTrue("Should have exactly one snapshot.", - snapshotItems.length == 1); + assertTrue(snapshotItems.length == 1, + "Should have exactly one snapshot."); String resultingSnapName = snapshotItems[0].getPath().getName(); - assertTrue("Snapshot name is not same as passed name.", - "snap-new-name".equals(resultingSnapName)); + assertTrue("snap-new-name".equals(resultingSnapName), + "Snapshot name is not same as passed name."); cleanSnapshotTests(snapshottablePath, resultingSnapName); } } @@ -1513,12 +1507,12 @@ private void testDeleteSnapshot() throws Exception { fs.createSnapshot(snapshottablePath, "snap-to-delete"); Path snapshotsDir = new Path("/tmp/tmp-snap-test/.snapshot"); FileStatus[] snapshotItems = fs.listStatus(snapshotsDir); - assertTrue("Should have exactly one snapshot.", - snapshotItems.length == 1); + assertTrue(snapshotItems.length == 1, + "Should have exactly one snapshot."); fs.deleteSnapshot(snapshottablePath, "snap-to-delete"); snapshotItems = fs.listStatus(snapshotsDir); - assertTrue("There should be no snapshot anymore.", - snapshotItems.length == 0); + assertTrue(snapshotItems.length == 0, + "There should be no snapshot anymore."); fs.delete(snapshottablePath, true); } } @@ -1531,8 +1525,8 @@ private void testAllowSnapshot() throws Exception { // Get the FileSystem instance that's being tested FileSystem fs = this.getHttpFSFileSystem(); // Check FileStatus - assertFalse("Snapshot should be disallowed by default", - fs.getFileStatus(path).isSnapshotEnabled()); + assertFalse(fs.getFileStatus(path).isSnapshotEnabled(), + "Snapshot should be disallowed by default"); // Allow snapshot if (fs instanceof HttpFSFileSystem) { HttpFSFileSystem httpFS = (HttpFSFileSystem) fs; @@ -1541,12 +1535,12 @@ private void testAllowSnapshot() throws Exception { WebHdfsFileSystem webHdfsFileSystem = (WebHdfsFileSystem) fs; webHdfsFileSystem.allowSnapshot(path); } else { - Assert.fail(fs.getClass().getSimpleName() + + fail(fs.getClass().getSimpleName() + " doesn't support allowSnapshot"); } // Check FileStatus - assertTrue("allowSnapshot failed", - fs.getFileStatus(path).isSnapshotEnabled()); + assertTrue(fs.getFileStatus(path).isSnapshotEnabled(), + "allowSnapshot failed"); // Cleanup fs.delete(path, true); } @@ -1560,8 +1554,8 @@ private void testDisallowSnapshot() throws Exception { // Get the FileSystem instance that's being tested FileSystem fs = this.getHttpFSFileSystem(); // Check FileStatus - assertTrue("Snapshot should be allowed by DFS", - fs.getFileStatus(path).isSnapshotEnabled()); + assertTrue(fs.getFileStatus(path).isSnapshotEnabled(), + "Snapshot should be allowed by DFS"); // Disallow snapshot if (fs instanceof HttpFSFileSystem) { HttpFSFileSystem httpFS = (HttpFSFileSystem) fs; @@ -1570,12 +1564,12 @@ private void testDisallowSnapshot() throws Exception { WebHdfsFileSystem webHdfsFileSystem = (WebHdfsFileSystem) fs; webHdfsFileSystem.disallowSnapshot(path); } else { - Assert.fail(fs.getClass().getSimpleName() + + fail(fs.getClass().getSimpleName() + " doesn't support disallowSnapshot"); } // Check FileStatus - assertFalse("disallowSnapshot failed", - fs.getFileStatus(path).isSnapshotEnabled()); + assertFalse(fs.getFileStatus(path).isSnapshotEnabled(), + "disallowSnapshot failed"); // Cleanup fs.delete(path, true); } @@ -1589,8 +1583,8 @@ private void testDisallowSnapshotException() throws Exception { // Get the FileSystem instance that's being tested FileSystem fs = this.getHttpFSFileSystem(); // Check FileStatus - assertTrue("Snapshot should be allowed by DFS", - fs.getFileStatus(path).isSnapshotEnabled()); + assertTrue(fs.getFileStatus(path).isSnapshotEnabled(), + "Snapshot should be allowed by DFS"); // Create some snapshots fs.createSnapshot(path, "snap-01"); fs.createSnapshot(path, "snap-02"); @@ -1613,17 +1607,17 @@ private void testDisallowSnapshotException() throws Exception { // Expect SnapshotException } } else { - Assert.fail(fs.getClass().getSimpleName() + + fail(fs.getClass().getSimpleName() + " doesn't support disallowSnapshot"); } if (disallowSuccess) { - Assert.fail("disallowSnapshot doesn't throw SnapshotException when " + fail("disallowSnapshot doesn't throw SnapshotException when " + "disallowing snapshot on a directory with at least one snapshot"); } // Check FileStatus, should still be enabled since // disallow snapshot should fail - assertTrue("disallowSnapshot should not have succeeded", - fs.getFileStatus(path).isSnapshotEnabled()); + assertTrue(fs.getFileStatus(path).isSnapshotEnabled(), + "disallowSnapshot should not have succeeded"); // Cleanup fs.deleteSnapshot(path, "snap-02"); fs.deleteSnapshot(path, "snap-01"); @@ -1639,7 +1633,7 @@ private void testGetSnapshotDiff() throws Exception { // Get the FileSystem instance that's being tested FileSystem fs = this.getHttpFSFileSystem(); // Check FileStatus - Assert.assertTrue(fs.getFileStatus(path).isSnapshotEnabled()); + assertTrue(fs.getFileStatus(path).isSnapshotEnabled()); // Create a file and take a snapshot Path file1 = new Path(path, "file1"); testCreate(file1, false); @@ -1659,13 +1653,13 @@ private void testGetSnapshotDiff() throws Exception { WebHdfsFileSystem webHdfsFileSystem = (WebHdfsFileSystem) fs; diffReport = webHdfsFileSystem.getSnapshotDiffReport(path, "snap1", "snap2"); } else { - Assert.fail(fs.getClass().getSimpleName() + " doesn't support getSnapshotDiff"); + fail(fs.getClass().getSimpleName() + " doesn't support getSnapshotDiff"); } // Verify result with DFS DistributedFileSystem dfs = (DistributedFileSystem) FileSystem.get(path.toUri(), this.getProxiedFSConf()); SnapshotDiffReport dfsDiffReport = dfs.getSnapshotDiffReport(path, "snap1", "snap2"); - Assert.assertEquals(diffReport.toString(), dfsDiffReport.toString()); + assertEquals(diffReport.toString(), dfsDiffReport.toString()); } finally { // Cleanup fs.deleteSnapshot(path, "snap2"); @@ -1686,7 +1680,7 @@ private void testGetSnapshotDiffIllegalParamCase(FileSystem fs, Path path, webHdfsFileSystem.getSnapshotDiffReport(path, oldsnapshotname, snapshotname); } else { - Assert.fail(fs.getClass().getSimpleName() + + fail(fs.getClass().getSimpleName() + " doesn't support getSnapshotDiff"); } } catch (SnapshotException|IllegalArgumentException|RemoteException e) { @@ -1694,12 +1688,12 @@ private void testGetSnapshotDiffIllegalParamCase(FileSystem fs, Path path, // or RemoteException(IllegalArgumentException) if (e instanceof RemoteException) { // Check RemoteException class name, should be IllegalArgumentException - Assert.assertEquals(((RemoteException) e).getClassName() + assertEquals(((RemoteException) e).getClassName() .compareTo(java.lang.IllegalArgumentException.class.getName()), 0); } return; } - Assert.fail("getSnapshotDiff illegal param didn't throw Exception"); + fail("getSnapshotDiff illegal param didn't throw Exception"); } private void testGetSnapshotDiffIllegalParam() throws Exception { @@ -1710,9 +1704,9 @@ private void testGetSnapshotDiffIllegalParam() throws Exception { // Get the FileSystem instance that's being tested FileSystem fs = this.getHttpFSFileSystem(); // Check FileStatus - assertTrue("Snapshot should be allowed by DFS", - fs.getFileStatus(path).isSnapshotEnabled()); - Assert.assertTrue(fs.getFileStatus(path).isSnapshotEnabled()); + assertTrue(fs.getFileStatus(path).isSnapshotEnabled(), + "Snapshot should be allowed by DFS"); + assertTrue(fs.getFileStatus(path).isSnapshotEnabled()); // Get snapshot diff testGetSnapshotDiffIllegalParamCase(fs, path, "", ""); testGetSnapshotDiffIllegalParamCase(fs, path, "snap1", ""); @@ -1734,12 +1728,12 @@ private void verifyGetSnapshottableDirListing( WebHdfsFileSystem webHdfsFileSystem = (WebHdfsFileSystem) fs; sds = webHdfsFileSystem.getSnapshottableDirectoryList(); } else { - Assert.fail(fs.getClass().getSimpleName() + + fail(fs.getClass().getSimpleName() + " doesn't support getSnapshottableDirListing"); } // Verify result with DFS SnapshottableDirectoryStatus[] dfssds = dfs.getSnapshottableDirListing(); - Assert.assertEquals(JsonUtil.toJsonString(sds), + assertEquals(JsonUtil.toJsonString(sds), JsonUtil.toJsonString(dfssds)); } @@ -1751,7 +1745,7 @@ private void testGetSnapshotListing() throws Exception { // Get the FileSystem instance that's being tested FileSystem fs = this.getHttpFSFileSystem(); // Check FileStatus - Assert.assertTrue(fs.getFileStatus(path).isSnapshotEnabled()); + assertTrue(fs.getFileStatus(path).isSnapshotEnabled()); // Create a file and take a snapshot Path file1 = new Path(path, "file1"); testCreate(file1, false); @@ -1769,7 +1763,7 @@ private void testGetSnapshotListing() throws Exception { WebHdfsFileSystem webHdfsFileSystem = (WebHdfsFileSystem) fs; snapshotStatus = webHdfsFileSystem.getSnapshotListing(path); } else { - Assert.fail(fs.getClass().getSimpleName() + + fail(fs.getClass().getSimpleName() + " doesn't support getSnapshotDiff"); } // Verify result with DFS @@ -1777,7 +1771,7 @@ private void testGetSnapshotListing() throws Exception { FileSystem.get(path.toUri(), this.getProxiedFSConf()); SnapshotStatus[] dfsStatus = dfs.getSnapshotListing(path); - Assert.assertEquals(JsonUtil.toJsonString(snapshotStatus), + assertEquals(JsonUtil.toJsonString(snapshotStatus), JsonUtil.toJsonString(dfsStatus)); // Cleanup fs.deleteSnapshot(path, "snap2"); @@ -1797,12 +1791,12 @@ private void testGetSnapshottableDirListing() throws Exception { // Verify response when there is no snapshottable directory verifyGetSnapshottableDirListing(fs, dfs); createSnapshotTestsPreconditions(path1); - Assert.assertTrue(fs.getFileStatus(path1).isSnapshotEnabled()); + assertTrue(fs.getFileStatus(path1).isSnapshotEnabled()); // Verify response when there is one snapshottable directory verifyGetSnapshottableDirListing(fs, dfs); Path path2 = new Path("/tmp/tmp-snap-dirlist-test-2"); createSnapshotTestsPreconditions(path2); - Assert.assertTrue(fs.getFileStatus(path2).isSnapshotEnabled()); + assertTrue(fs.getFileStatus(path2).isSnapshotEnabled()); // Verify response when there are two snapshottable directories verifyGetSnapshottableDirListing(fs, dfs); @@ -1829,7 +1823,7 @@ private void testFileAclsCustomizedUserAndGroupNames() throws Exception { FileSystem httpfs = getHttpFSFileSystem(conf); if (!(httpfs instanceof WebHdfsFileSystem) && !(httpfs instanceof HttpFSFileSystem)) { - Assert.fail(httpfs.getClass().getSimpleName() + + fail(httpfs.getClass().getSimpleName() + " doesn't support custom user and group name pattern. " + "Only WebHdfsFileSystem and HttpFSFileSystem support it."); } @@ -1857,8 +1851,8 @@ private void testFileAclsCustomizedUserAndGroupNames() throws Exception { for (AclEntry aclEntry : httpfsAclStat.getEntries()) { strEntries.add(aclEntry.toStringStable()); } - Assert.assertTrue(strEntries.contains(aclUser)); - Assert.assertTrue(strEntries.contains(aclGroup)); + assertTrue(strEntries.contains(aclUser)); + assertTrue(strEntries.contains(aclGroup)); // Clean up proxyFs.delete(new Path(dir), true); } @@ -1873,12 +1867,11 @@ private void verifyGetServerDefaults(FileSystem fs, DistributedFileSystem dfs) WebHdfsFileSystem webHdfsFileSystem = (WebHdfsFileSystem) fs; sds = webHdfsFileSystem.getServerDefaults(); } else { - Assert.fail( - fs.getClass().getSimpleName() + " doesn't support getServerDefaults"); + fail(fs.getClass().getSimpleName() + " doesn't support getServerDefaults"); } // Verify result with DFS FsServerDefaults dfssds = dfs.getServerDefaults(); - Assert.assertEquals(JsonUtil.toJsonString(sds), + assertEquals(JsonUtil.toJsonString(sds), JsonUtil.toJsonString(dfssds)); } @@ -1916,7 +1909,7 @@ private void verifyAccess(FileSystem fs, DistributedFileSystem dfs) WebHdfsFileSystem webHdfsFileSystem = (WebHdfsFileSystem) fs; webHdfsFileSystem.access(p1, FsAction.READ); } else { - Assert.fail(fs.getClass().getSimpleName() + " doesn't support access"); + fail(fs.getClass().getSimpleName() + " doesn't support access"); } } @@ -1942,7 +1935,7 @@ private void testErasureCodingPolicy() throws Exception { assertEquals(ecPolicy, ecPolicy1); httpFS.unsetErasureCodingPolicy(p1); ecPolicy1 = httpFS.getErasureCodingPolicy(p1); - Assert.assertNull(ecPolicy1); + assertNull(ecPolicy1); } else if (fs instanceof WebHdfsFileSystem) { WebHdfsFileSystem webHdfsFileSystem = (WebHdfsFileSystem) fs; webHdfsFileSystem.setErasureCodingPolicy(p1, ecPolicyName); @@ -1951,9 +1944,9 @@ private void testErasureCodingPolicy() throws Exception { assertEquals(ecPolicy, ecPolicy1); webHdfsFileSystem.unsetErasureCodingPolicy(p1); ecPolicy1 = dfs.getErasureCodingPolicy(p1); - Assert.assertNull(ecPolicy1); + assertNull(ecPolicy1); } else { - Assert.fail(fs.getClass().getSimpleName() + " doesn't support access"); + fail(fs.getClass().getSimpleName() + " doesn't support access"); } } } @@ -1988,7 +1981,7 @@ public void testStoragePolicySatisfier() throws Exception { assertTrue(xAttrs .containsKey(HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY)); } else { - Assert.fail(fs.getClass().getSimpleName() + " doesn't support access"); + fail(fs.getClass().getSimpleName() + " doesn't support access"); } dfs.delete(path1, true); } @@ -2020,7 +2013,7 @@ private void testGetFileBlockLocations() throws Exception { blockLocations = webHdfsFileSystem.getFileBlockLocations(testFile, 0, 1); assertNotNull(blockLocations); } else { - Assert.fail(fs.getClass().getSimpleName() + " doesn't support access"); + fail(fs.getClass().getSimpleName() + " doesn't support access"); } } } @@ -2033,7 +2026,7 @@ private void testGetSnapshotDiffListing() throws Exception { // Get the FileSystem instance that's being tested FileSystem fs = this.getHttpFSFileSystem(); // Check FileStatus - Assert.assertTrue(fs.getFileStatus(path).isSnapshotEnabled()); + assertTrue(fs.getFileStatus(path).isSnapshotEnabled()); // Create a file and take a snapshot Path file1 = new Path(path, "file1"); testCreate(file1, false); @@ -2056,7 +2049,7 @@ private void testGetSnapshotDiffListing() throws Exception { .getSnapshotDiffReportListing(path.toUri().getPath(), "snap1", "snap2", emptyBytes, -1); } else { - Assert.fail(fs.getClass().getSimpleName() + " doesn't support getSnapshotDiff"); + fail(fs.getClass().getSimpleName() + " doesn't support getSnapshotDiff"); } // Verify result with DFS DistributedFileSystem dfs = @@ -2117,7 +2110,7 @@ private void testGetStatus() throws Exception { httpFs.close(); dfs.close(); } else { - Assert.fail(fs.getClass().getSimpleName() + " is not of type DistributedFileSystem."); + fail(fs.getClass().getSimpleName() + " is not of type DistributedFileSystem."); } } @@ -2144,7 +2137,7 @@ private void testGetAllEEPolicies() throws Exception { WebHdfsFileSystem webHdfsFileSystem = (WebHdfsFileSystem) httpFs; diffErasureCodingPolicies = webHdfsFileSystem.getAllErasureCodingPolicies(); } else { - Assert.fail(fs.getClass().getSimpleName() + + fail(fs.getClass().getSimpleName() + " is not of type HttpFSFileSystem or WebHdfsFileSystem"); } @@ -2152,7 +2145,7 @@ private void testGetAllEEPolicies() throws Exception { assertEquals(dfsAllErasureCodingPolicies.size(), diffErasureCodingPolicies.size()); assertTrue(dfsAllErasureCodingPolicies.containsAll(diffErasureCodingPolicies)); } else { - Assert.fail(fs.getClass().getSimpleName() + " is not of type DistributedFileSystem."); + fail(fs.getClass().getSimpleName() + " is not of type DistributedFileSystem."); } } @@ -2194,13 +2187,13 @@ private void testGetECCodecs() throws Exception { Map diffErasureCodingCodecs = diffErasureCodingCodecsRef.get(); //Validate testGetECCodecs are the same as DistributedFileSystem - Assert.assertEquals(dfsErasureCodingCodecs.size(), diffErasureCodingCodecs.size()); + assertEquals(dfsErasureCodingCodecs.size(), diffErasureCodingCodecs.size()); for (Map.Entry entry : dfsErasureCodingCodecs.entrySet()) { String key = entry.getKey(); String value = entry.getValue(); - Assert.assertTrue(diffErasureCodingCodecs.containsKey(key)); - Assert.assertEquals(value, diffErasureCodingCodecs.get(key)); + assertTrue(diffErasureCodingCodecs.containsKey(key)); + assertEquals(value, diffErasureCodingCodecs.get(key)); } } @@ -2232,38 +2225,38 @@ private void testGetTrashRoots() throws Exception { WebHdfsFileSystem webHdfsFileSystem = (WebHdfsFileSystem) httpFs; diffTrashRoots = webHdfsFileSystem.getTrashRoots(true); } else { - Assert.fail(fs.getClass().getSimpleName() + + fail(fs.getClass().getSimpleName() + " is not of type HttpFSFileSystem or WebHdfsFileSystem"); } // Validate getTrashRoots are the same as DistributedFileSystem assertEquals(dfsTrashRoots.size(), diffTrashRoots.size()); } else { - Assert.fail(fs.getClass().getSimpleName() + " is not of type DistributedFileSystem."); + fail(fs.getClass().getSimpleName() + " is not of type DistributedFileSystem."); } } private void assertHttpFsReportListingWithDfsClient(SnapshotDiffReportListing diffReportListing, SnapshotDiffReportListing dfsDiffReportListing) { - Assert.assertEquals(diffReportListing.getCreateList().size(), + assertEquals(diffReportListing.getCreateList().size(), dfsDiffReportListing.getCreateList().size()); - Assert.assertEquals(diffReportListing.getDeleteList().size(), + assertEquals(diffReportListing.getDeleteList().size(), dfsDiffReportListing.getDeleteList().size()); - Assert.assertEquals(diffReportListing.getModifyList().size(), + assertEquals(diffReportListing.getModifyList().size(), dfsDiffReportListing.getModifyList().size()); - Assert.assertEquals(diffReportListing.getIsFromEarlier(), + assertEquals(diffReportListing.getIsFromEarlier(), dfsDiffReportListing.getIsFromEarlier()); - Assert.assertEquals(diffReportListing.getLastIndex(), dfsDiffReportListing.getLastIndex()); - Assert.assertEquals(DFSUtil.bytes2String(diffReportListing.getLastPath()), + assertEquals(diffReportListing.getLastIndex(), dfsDiffReportListing.getLastIndex()); + assertEquals(DFSUtil.bytes2String(diffReportListing.getLastPath()), DFSUtil.bytes2String(dfsDiffReportListing.getLastPath())); int i = 0; for (SnapshotDiffReportListing.DiffReportListingEntry entry : diffReportListing .getCreateList()) { SnapshotDiffReportListing.DiffReportListingEntry dfsDiffEntry = dfsDiffReportListing.getCreateList().get(i); - Assert.assertEquals(entry.getDirId(), dfsDiffEntry.getDirId()); - Assert.assertEquals(entry.getFileId(), dfsDiffEntry.getFileId()); - Assert.assertArrayEquals(DFSUtilClient.byteArray2bytes(entry.getSourcePath()), + assertEquals(entry.getDirId(), dfsDiffEntry.getDirId()); + assertEquals(entry.getFileId(), dfsDiffEntry.getFileId()); + assertArrayEquals(DFSUtilClient.byteArray2bytes(entry.getSourcePath()), DFSUtilClient.byteArray2bytes(dfsDiffEntry.getSourcePath())); i++; } @@ -2272,9 +2265,9 @@ private void assertHttpFsReportListingWithDfsClient(SnapshotDiffReportListing di .getDeleteList()) { SnapshotDiffReportListing.DiffReportListingEntry dfsDiffEntry = dfsDiffReportListing.getDeleteList().get(i); - Assert.assertEquals(entry.getDirId(), dfsDiffEntry.getDirId()); - Assert.assertEquals(entry.getFileId(), dfsDiffEntry.getFileId()); - Assert.assertArrayEquals(DFSUtilClient.byteArray2bytes(entry.getSourcePath()), + assertEquals(entry.getDirId(), dfsDiffEntry.getDirId()); + assertEquals(entry.getFileId(), dfsDiffEntry.getFileId()); + assertArrayEquals(DFSUtilClient.byteArray2bytes(entry.getSourcePath()), DFSUtilClient.byteArray2bytes(dfsDiffEntry.getSourcePath())); i++; } @@ -2283,9 +2276,9 @@ private void assertHttpFsReportListingWithDfsClient(SnapshotDiffReportListing di .getModifyList()) { SnapshotDiffReportListing.DiffReportListingEntry dfsDiffEntry = dfsDiffReportListing.getModifyList().get(i); - Assert.assertEquals(entry.getDirId(), dfsDiffEntry.getDirId()); - Assert.assertEquals(entry.getFileId(), dfsDiffEntry.getFileId()); - Assert.assertArrayEquals(DFSUtilClient.byteArray2bytes(entry.getSourcePath()), + assertEquals(entry.getDirId(), dfsDiffEntry.getDirId()); + assertEquals(entry.getFileId(), dfsDiffEntry.getFileId()); + assertArrayEquals(DFSUtilClient.byteArray2bytes(entry.getSourcePath()), DFSUtilClient.byteArray2bytes(dfsDiffEntry.getSourcePath())); i++; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFWithSWebhdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFWithSWebhdfsFileSystem.java index d53bb50f400e1..42e1231aa7a91 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFWithSWebhdfsFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFWithSWebhdfsFileSystem.java @@ -25,16 +25,13 @@ import org.apache.hadoop.security.ssl.KeyStoreTestUtil; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.TestJettyHelper; -import org.junit.AfterClass; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.AfterAll; import java.io.File; import java.net.URI; import java.net.URL; import java.util.UUID; -@RunWith(value = Parameterized.class) public class TestHttpFSFWithSWebhdfsFileSystem extends TestHttpFSWithHttpFSFileSystem { private static String classpathDir; @@ -69,15 +66,14 @@ public class TestHttpFSFWithSWebhdfsFileSystem "serverP"); } - @AfterClass + @AfterAll public static void cleanUp() throws Exception { new File(classpathDir, "ssl-client.xml").delete(); new File(classpathDir, "ssl-server.xml").delete(); KeyStoreTestUtil.cleanupSSLConfig(keyStoreDir, classpathDir); } - public TestHttpFSFWithSWebhdfsFileSystem(Operation operation) { - super(operation); + public TestHttpFSFWithSWebhdfsFileSystem() { } @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFWithWebhdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFWithWebhdfsFileSystem.java index cf1e4f1ec3ab8..26738246bd398 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFWithWebhdfsFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFWithWebhdfsFileSystem.java @@ -19,15 +19,11 @@ package org.apache.hadoop.fs.http.client; import org.apache.hadoop.hdfs.web.WebHdfsFileSystem; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -@RunWith(value = Parameterized.class) public class TestHttpFSFWithWebhdfsFileSystem extends TestHttpFSWithHttpFSFileSystem { - public TestHttpFSFWithWebhdfsFileSystem(Operation operation) { - super(operation); + public TestHttpFSFWithWebhdfsFileSystem() { } @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystemLocalFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystemLocalFileSystem.java index 955529ef9816d..81e0e322c57d5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystemLocalFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystemLocalFileSystem.java @@ -27,13 +27,11 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.TestDirHelper; -import org.junit.Assert; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; import java.io.File; -@RunWith(value = Parameterized.class) +import static org.junit.jupiter.api.Assertions.assertEquals; + public class TestHttpFSFileSystemLocalFileSystem extends BaseTestHttpFSWith { private static String PATH_PREFIX; @@ -45,8 +43,7 @@ public class TestHttpFSFileSystemLocalFileSystem extends BaseTestHttpFSWith { PATH_PREFIX = file.getAbsolutePath(); } - public TestHttpFSFileSystemLocalFileSystem(Operation operation) { - super(operation); + public TestHttpFSFileSystemLocalFileSystem() { } @Override @@ -86,7 +83,7 @@ protected void testSetPermission() throws Exception { FileStatus status1 = fs.getFileStatus(path); fs.close(); FsPermission permission2 = status1.getPermission(); - Assert.assertEquals(permission2, permission1); + assertEquals(permission2, permission1); // sticky bit not supported on Windows with local file system, so the // subclass skips that part of the test diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSWithHttpFSFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSWithHttpFSFileSystem.java index b211e9a466119..23f37fc4c4264 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSWithHttpFSFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSWithHttpFSFileSystem.java @@ -22,14 +22,10 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.Path; import org.apache.hadoop.test.TestHdfsHelper; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -@RunWith(value = Parameterized.class) public class TestHttpFSWithHttpFSFileSystem extends BaseTestHttpFSWith { - public TestHttpFSWithHttpFSFileSystem(Operation operation) { - super(operation); + public TestHttpFSWithHttpFSFileSystem() { } @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestCheckUploadContentTypeFilter.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestCheckUploadContentTypeFilter.java index 947f928a0e232..daa2b34758bbc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestCheckUploadContentTypeFilter.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestCheckUploadContentTypeFilter.java @@ -24,7 +24,7 @@ import javax.servlet.http.HttpServletResponse; import org.apache.hadoop.fs.http.client.HttpFSFileSystem; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; public class TestCheckUploadContentTypeFilter { diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSAccessControlled.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSAccessControlled.java index 1411cbf78a180..99dfb3e281d59 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSAccessControlled.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSAccessControlled.java @@ -29,8 +29,7 @@ import org.apache.hadoop.test.TestDirHelper; import org.apache.hadoop.test.TestJetty; import org.apache.hadoop.test.TestJettyHelper; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.webapp.WebAppContext; @@ -44,6 +43,9 @@ import java.net.URL; import java.text.MessageFormat; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + /** * This test class ensures that everything works as expected when * support with the access controlled HTTPFS file system. @@ -93,9 +95,9 @@ private void startMiniDFS() throws Exception { */ private void createHttpFSServer() throws Exception { File homeDir = TestDirHelper.getTestDir(); - Assert.assertTrue(new File(homeDir, "conf").mkdir()); - Assert.assertTrue(new File(homeDir, "log").mkdir()); - Assert.assertTrue(new File(homeDir, "temp").mkdir()); + assertTrue(new File(homeDir, "conf").mkdir()); + assertTrue(new File(homeDir, "log").mkdir()); + assertTrue(new File(homeDir, "temp").mkdir()); HttpFSServerWebApp.setHomeDirForCurrentThread(homeDir.getAbsolutePath()); File secretFile = new File(new File(homeDir, "conf"), "secret"); @@ -175,9 +177,9 @@ private void getCmd(String filename, String message, String command, boolean exp conn.connect(); int resp = conn.getResponseCode(); if ( expectOK ) { - Assert.assertEquals( outMsg, HttpURLConnection.HTTP_OK, resp); + assertEquals(HttpURLConnection.HTTP_OK, resp, outMsg); } else { - Assert.assertEquals(outMsg, HttpURLConnection.HTTP_FORBIDDEN, resp); + assertEquals(HttpURLConnection.HTTP_FORBIDDEN, resp, outMsg); } } @@ -207,9 +209,9 @@ private void putCmd(String filename, String message, String command, conn.connect(); int resp = conn.getResponseCode(); if ( expectOK ) { - Assert.assertEquals(outMsg, HttpURLConnection.HTTP_OK, resp); + assertEquals(HttpURLConnection.HTTP_OK, resp, outMsg); } else { - Assert.assertEquals(outMsg, HttpURLConnection.HTTP_FORBIDDEN, resp); + assertEquals(HttpURLConnection.HTTP_FORBIDDEN, resp, outMsg); } } @@ -239,9 +241,9 @@ private void deleteCmd(String filename, String message, String command, conn.connect(); int resp = conn.getResponseCode(); if ( expectOK ) { - Assert.assertEquals(outMsg, HttpURLConnection.HTTP_OK, resp); + assertEquals(HttpURLConnection.HTTP_OK, resp, outMsg); } else { - Assert.assertEquals(outMsg, HttpURLConnection.HTTP_FORBIDDEN, resp); + assertEquals(HttpURLConnection.HTTP_FORBIDDEN, resp, outMsg); } } @@ -271,9 +273,9 @@ private void postCmd(String filename, String message, String command, conn.connect(); int resp = conn.getResponseCode(); if ( expectOK ) { - Assert.assertEquals(outMsg, HttpURLConnection.HTTP_OK, resp); + assertEquals(HttpURLConnection.HTTP_OK, resp, outMsg); } else { - Assert.assertEquals(outMsg, HttpURLConnection.HTTP_FORBIDDEN, resp); + assertEquals(HttpURLConnection.HTTP_FORBIDDEN, resp, outMsg); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java index 70535ae31cb31..d9c1b8813f49f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java @@ -17,9 +17,13 @@ */ package org.apache.hadoop.fs.http.server; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSTestUtil; @@ -45,7 +49,6 @@ import org.apache.hadoop.security.token.delegation.web.KerberosDelegationTokenAuthenticationHandler; import org.apache.hadoop.util.JsonSerialization; import org.json.simple.JSONArray; -import org.junit.Assert; import java.io.BufferedReader; import java.io.File; @@ -106,7 +109,7 @@ import org.apache.hadoop.test.TestJettyHelper; import org.json.simple.JSONObject; import org.json.simple.parser.JSONParser; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.webapp.WebAppContext; @@ -205,9 +208,9 @@ public Set getGroupsSet(String user) throws IOException { private Configuration createHttpFSConf(boolean addDelegationTokenAuthHandler, boolean sslEnabled) throws Exception { File homeDir = TestDirHelper.getTestDir(); - Assert.assertTrue(new File(homeDir, "conf").mkdir()); - Assert.assertTrue(new File(homeDir, "log").mkdir()); - Assert.assertTrue(new File(homeDir, "temp").mkdir()); + assertTrue(new File(homeDir, "conf").mkdir()); + assertTrue(new File(homeDir, "log").mkdir()); + assertTrue(new File(homeDir, "temp").mkdir()); HttpFSServerWebApp.setHomeDirForCurrentThread(homeDir.getAbsolutePath()); File secretFile = new File(new File(homeDir, "conf"), "secret"); @@ -269,7 +272,7 @@ private void writeConf(Configuration conf, String sitename) File homeDir = TestDirHelper.getTestDir(); // HDFS configuration File hadoopConfDir = new File(new File(homeDir, "conf"), "hadoop-conf"); - Assert.assertTrue(hadoopConfDir.exists()); + assertTrue(hadoopConfDir.exists()); File siteFile = new File(hadoopConfDir, sitename); OutputStream os = new FileOutputStream(siteFile); @@ -313,7 +316,7 @@ private void delegationTokenCommonTests(boolean sslEnabled) throws Exception { URL url = new URL(TestJettyHelper.getJettyURL(), "/webhdfs/v1/?op=GETHOMEDIRECTORY"); HttpURLConnection conn = (HttpURLConnection) url.openConnection(); - Assert.assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, + assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, conn.getResponseCode()); String tokenSigned = getSignedTokenString(); @@ -323,7 +326,7 @@ private void delegationTokenCommonTests(boolean sslEnabled) throws Exception { conn = (HttpURLConnection) url.openConnection(); conn.setRequestProperty("Cookie", AuthenticatedURL.AUTH_COOKIE + "=" + tokenSigned); - Assert.assertEquals(HttpURLConnection.HTTP_OK, + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); JSONObject json = (JSONObject)new JSONParser().parse( @@ -336,7 +339,7 @@ private void delegationTokenCommonTests(boolean sslEnabled) throws Exception { Token dToken = new Token(); dToken.decodeFromUrlString(tokenStr); - Assert.assertEquals(sslEnabled ? + assertEquals(sslEnabled ? WebHdfsConstants.SWEBHDFS_TOKEN_KIND : WebHdfsConstants.WEBHDFS_TOKEN_KIND, dToken.getKind()); @@ -344,14 +347,14 @@ private void delegationTokenCommonTests(boolean sslEnabled) throws Exception { url = new URL(TestJettyHelper.getJettyURL(), "/webhdfs/v1/?op=GETHOMEDIRECTORY&delegation=" + tokenStr); conn = (HttpURLConnection) url.openConnection(); - Assert.assertEquals(HttpURLConnection.HTTP_OK, + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); url = new URL(TestJettyHelper.getJettyURL(), "/webhdfs/v1/?op=RENEWDELEGATIONTOKEN&token=" + tokenStr); conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod("PUT"); - Assert.assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, + assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, conn.getResponseCode()); url = new URL(TestJettyHelper.getJettyURL(), @@ -360,27 +363,27 @@ private void delegationTokenCommonTests(boolean sslEnabled) throws Exception { conn.setRequestMethod("PUT"); conn.setRequestProperty("Cookie", AuthenticatedURL.AUTH_COOKIE + "=" + tokenSigned); - Assert.assertEquals(HttpURLConnection.HTTP_OK, + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); url = new URL(TestJettyHelper.getJettyURL(), "/webhdfs/v1/?op=CANCELDELEGATIONTOKEN&token=" + tokenStr); conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod("PUT"); - Assert.assertEquals(HttpURLConnection.HTTP_OK, + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); url = new URL(TestJettyHelper.getJettyURL(), "/webhdfs/v1/?op=GETHOMEDIRECTORY&delegation=" + tokenStr); conn = (HttpURLConnection) url.openConnection(); - Assert.assertEquals(HttpURLConnection.HTTP_FORBIDDEN, + assertEquals(HttpURLConnection.HTTP_FORBIDDEN, conn.getResponseCode()); // getTrash test with delegation url = new URL(TestJettyHelper.getJettyURL(), "/webhdfs/v1/?op=GETTRASHROOT&delegation=" + tokenStr); conn = (HttpURLConnection) url.openConnection(); - Assert.assertEquals(HttpURLConnection.HTTP_FORBIDDEN, + assertEquals(HttpURLConnection.HTTP_FORBIDDEN, conn.getResponseCode()); url = new URL(TestJettyHelper.getJettyURL(), @@ -388,7 +391,7 @@ private void delegationTokenCommonTests(boolean sslEnabled) throws Exception { conn = (HttpURLConnection) url.openConnection(); conn.setRequestProperty("Cookie", AuthenticatedURL.AUTH_COOKIE + "=" + tokenSigned); - Assert.assertEquals(HttpURLConnection.HTTP_OK, + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); } @@ -403,26 +406,26 @@ public void instrumentation() throws Exception { MessageFormat.format("/webhdfs/v1?user.name={0}&op=instrumentation", "nobody")); HttpURLConnection conn = (HttpURLConnection) url.openConnection(); - Assert.assertEquals(conn.getResponseCode(), + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_UNAUTHORIZED); url = new URL(TestJettyHelper.getJettyURL(), MessageFormat.format("/webhdfs/v1?user.name={0}&op=instrumentation", HadoopUsersConfTestHelper.getHadoopUsers()[0])); conn = (HttpURLConnection) url.openConnection(); - Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); BufferedReader reader = new BufferedReader( new InputStreamReader(conn.getInputStream())); String line = reader.readLine(); reader.close(); - Assert.assertTrue(line.contains("\"counters\":{")); + assertTrue(line.contains("\"counters\":{")); url = new URL(TestJettyHelper.getJettyURL(), MessageFormat.format( "/webhdfs/v1/foo?user.name={0}&op=instrumentation", HadoopUsersConfTestHelper.getHadoopUsers()[0])); conn = (HttpURLConnection) url.openConnection(); - Assert.assertEquals(conn.getResponseCode(), + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_BAD_REQUEST); } @@ -439,12 +442,12 @@ public void testHdfsAccess() throws Exception { MessageFormat.format("/webhdfs/v1/?user.name={0}&op=liststatus", user)); HttpURLConnection conn = (HttpURLConnection) url.openConnection(); - Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); BufferedReader reader = new BufferedReader( new InputStreamReader(conn.getInputStream())); reader.readLine(); reader.close(); - Assert.assertEquals(1 + oldOpsListStatus, + assertEquals(1 + oldOpsListStatus, (long) metricsGetter.get("LISTSTATUS").call()); } @@ -462,11 +465,11 @@ public void testMkdirs() throws Exception { HttpURLConnection conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod("PUT"); conn.connect(); - Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); getStatus("/tmp/sub-tmp", "LISTSTATUS"); long opsStat = metricsGetter.get("MKDIRS").call(); - Assert.assertEquals(1 + oldMkdirOpsStat, opsStat); + assertEquals(1 + oldMkdirOpsStat, opsStat); } @Test @@ -486,12 +489,12 @@ public void testGlobFilter() throws Exception { MessageFormat.format( "/webhdfs/v1/tmp?user.name={0}&op=liststatus&filter=f*", user)); HttpURLConnection conn = (HttpURLConnection) url.openConnection(); - Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); BufferedReader reader = new BufferedReader( new InputStreamReader(conn.getInputStream())); reader.readLine(); reader.close(); - Assert.assertEquals(1 + oldOpsListStatus, + assertEquals(1 + oldOpsListStatus, (long) metricsGetter.get("LISTSTATUS").call()); } @@ -539,7 +542,7 @@ private void createWithHttp(String filename, String perms, conn.addRequestProperty("Content-Type", "application/octet-stream"); conn.setRequestMethod("PUT"); conn.connect(); - Assert.assertEquals(HttpURLConnection.HTTP_CREATED, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_CREATED, conn.getResponseCode()); } /** @@ -577,8 +580,8 @@ private void createDirWithHttp(String dirname, String perms, HttpURLConnection conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod("PUT"); conn.connect(); - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); - Assert.assertEquals(1 + oldOpsMkdir, + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(1 + oldOpsMkdir, (long) metricsGetter.get("MKDIRS").call()); } @@ -606,13 +609,13 @@ private String getStatus(String filename, String command) URL url = new URL(TestJettyHelper.getJettyURL(), pathOps); HttpURLConnection conn = (HttpURLConnection) url.openConnection(); conn.connect(); - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); BufferedReader reader = new BufferedReader(new InputStreamReader(conn.getInputStream())); long opsStat = metricsGetter.getOrDefault(command, defaultExitMetricGetter).call(); - Assert.assertEquals(oldOpsStat + 1L, opsStat); + assertEquals(oldOpsStat + 1L, opsStat); return reader.readLine(); } @@ -624,7 +627,7 @@ private String getStatus(String filename, String command) */ private void putCmd(String filename, String command, String params) throws Exception { - Assert.assertEquals(HttpURLConnection.HTTP_OK, + assertEquals(HttpURLConnection.HTTP_OK, putCmdWithReturn(filename, command, params).getResponseCode()); } @@ -772,19 +775,19 @@ public void testPerms() throws Exception { createWithHttp("/perm/none", null); String statusJson = getStatus("/perm/none", "GETFILESTATUS"); - Assert.assertTrue("755".equals(getPerms(statusJson))); + assertTrue("755".equals(getPerms(statusJson))); createWithHttp("/perm/p-777", "777"); statusJson = getStatus("/perm/p-777", "GETFILESTATUS"); - Assert.assertTrue("777".equals(getPerms(statusJson))); + assertTrue("777".equals(getPerms(statusJson))); createWithHttp("/perm/p-654", "654"); statusJson = getStatus("/perm/p-654", "GETFILESTATUS"); - Assert.assertTrue("654".equals(getPerms(statusJson))); + assertTrue("654".equals(getPerms(statusJson))); createWithHttp("/perm/p-321", "321"); statusJson = getStatus("/perm/p-321", "GETFILESTATUS"); - Assert.assertTrue("321".equals(getPerms(statusJson))); + assertTrue("321".equals(getPerms(statusJson))); } /** @@ -810,29 +813,29 @@ public void testXAttrs() throws Exception { createWithHttp(path, null); String statusJson = getStatus(path, "GETXATTRS"); Map xAttrs = getXAttrs(statusJson); - Assert.assertEquals(0, xAttrs.size()); + assertEquals(0, xAttrs.size()); // Set two xattrs putCmd(path, "SETXATTR", setXAttrParam(name1, value1)); putCmd(path, "SETXATTR", setXAttrParam(name2, value2)); statusJson = getStatus(path, "GETXATTRS"); xAttrs = getXAttrs(statusJson); - Assert.assertEquals(2, xAttrs.size()); - Assert.assertArrayEquals(value1, xAttrs.get(name1)); - Assert.assertArrayEquals(value2, xAttrs.get(name2)); + assertEquals(2, xAttrs.size()); + assertArrayEquals(value1, xAttrs.get(name1)); + assertArrayEquals(value2, xAttrs.get(name2)); // Remove one xattr putCmd(path, "REMOVEXATTR", "xattr.name=" + name1); statusJson = getStatus(path, "GETXATTRS"); xAttrs = getXAttrs(statusJson); - Assert.assertEquals(1, xAttrs.size()); - Assert.assertArrayEquals(value2, xAttrs.get(name2)); + assertEquals(1, xAttrs.size()); + assertArrayEquals(value2, xAttrs.get(name2)); // Remove another xattr, then there is no xattr putCmd(path, "REMOVEXATTR", "xattr.name=" + name2); statusJson = getStatus(path, "GETXATTRS"); xAttrs = getXAttrs(statusJson); - Assert.assertEquals(0, xAttrs.size()); + assertEquals(0, xAttrs.size()); } /** Params for setting an xAttr. */ @@ -881,14 +884,14 @@ public void testFileAcls() throws Exception { /* getfilestatus and liststatus don't have 'aclBit' in their reply */ statusJson = getStatus(path, "GETFILESTATUS"); - Assert.assertEquals(-1, statusJson.indexOf("aclBit")); + assertEquals(-1, statusJson.indexOf("aclBit")); statusJson = getStatus(dir, "LISTSTATUS"); - Assert.assertEquals(-1, statusJson.indexOf("aclBit")); + assertEquals(-1, statusJson.indexOf("aclBit")); /* getaclstatus works and returns no entries */ statusJson = getStatus(path, "GETACLSTATUS"); aclEntries = getAclEntries(statusJson); - Assert.assertTrue(aclEntries.size() == 0); + assertTrue(aclEntries.size() == 0); /* * Now set an ACL on the file. (getfile|list)status have aclBit, @@ -896,41 +899,41 @@ public void testFileAcls() throws Exception { */ putCmd(path, "SETACL", aclSpec); statusJson = getStatus(path, "GETFILESTATUS"); - Assert.assertNotEquals(-1, statusJson.indexOf("aclBit")); + assertNotEquals(-1, statusJson.indexOf("aclBit")); statusJson = getStatus(dir, "LISTSTATUS"); - Assert.assertNotEquals(-1, statusJson.indexOf("aclBit")); + assertNotEquals(-1, statusJson.indexOf("aclBit")); statusJson = getStatus(path, "GETACLSTATUS"); aclEntries = getAclEntries(statusJson); - Assert.assertTrue(aclEntries.size() == 2); - Assert.assertTrue(aclEntries.contains(aclUser1)); - Assert.assertTrue(aclEntries.contains(aclGroup1)); + assertTrue(aclEntries.size() == 2); + assertTrue(aclEntries.contains(aclUser1)); + assertTrue(aclEntries.contains(aclGroup1)); /* Modify acl entries to add another user acl */ putCmd(path, "MODIFYACLENTRIES", modAclSpec); statusJson = getStatus(path, "GETACLSTATUS"); aclEntries = getAclEntries(statusJson); - Assert.assertTrue(aclEntries.size() == 3); - Assert.assertTrue(aclEntries.contains(aclUser1)); - Assert.assertTrue(aclEntries.contains(aclUser2)); - Assert.assertTrue(aclEntries.contains(aclGroup1)); + assertTrue(aclEntries.size() == 3); + assertTrue(aclEntries.contains(aclUser1)); + assertTrue(aclEntries.contains(aclUser2)); + assertTrue(aclEntries.contains(aclGroup1)); /* Remove the first user acl entry and verify */ putCmd(path, "REMOVEACLENTRIES", remAclSpec); statusJson = getStatus(path, "GETACLSTATUS"); aclEntries = getAclEntries(statusJson); - Assert.assertTrue(aclEntries.size() == 2); - Assert.assertTrue(aclEntries.contains(aclUser2)); - Assert.assertTrue(aclEntries.contains(aclGroup1)); + assertTrue(aclEntries.size() == 2); + assertTrue(aclEntries.contains(aclUser2)); + assertTrue(aclEntries.contains(aclGroup1)); /* Remove all acls and verify */ putCmd(path, "REMOVEACL", null); statusJson = getStatus(path, "GETACLSTATUS"); aclEntries = getAclEntries(statusJson); - Assert.assertTrue(aclEntries.size() == 0); + assertTrue(aclEntries.size() == 0); statusJson = getStatus(path, "GETFILESTATUS"); - Assert.assertEquals(-1, statusJson.indexOf("aclBit")); + assertEquals(-1, statusJson.indexOf("aclBit")); statusJson = getStatus(dir, "LISTSTATUS"); - Assert.assertEquals(-1, statusJson.indexOf("aclBit")); + assertEquals(-1, statusJson.indexOf("aclBit")); } /** @@ -962,30 +965,30 @@ public void testDirAcls() throws Exception { /* getfilestatus and liststatus don't have 'aclBit' in their reply */ statusJson = getStatus(dir, "GETFILESTATUS"); - Assert.assertEquals(-1, statusJson.indexOf("aclBit")); + assertEquals(-1, statusJson.indexOf("aclBit")); /* No ACLs, either */ statusJson = getStatus(dir, "GETACLSTATUS"); aclEntries = getAclEntries(statusJson); - Assert.assertTrue(aclEntries.size() == 0); + assertTrue(aclEntries.size() == 0); /* Give it a default ACL and verify */ putCmd(dir, "SETACL", defSpec1); statusJson = getStatus(dir, "GETFILESTATUS"); - Assert.assertNotEquals(-1, statusJson.indexOf("aclBit")); + assertNotEquals(-1, statusJson.indexOf("aclBit")); statusJson = getStatus(dir, "GETACLSTATUS"); aclEntries = getAclEntries(statusJson); - Assert.assertTrue(aclEntries.size() == 5); + assertTrue(aclEntries.size() == 5); /* 4 Entries are default:(user|group|mask|other):perm */ - Assert.assertTrue(aclEntries.contains(defUser1)); + assertTrue(aclEntries.contains(defUser1)); /* Remove the default ACL and re-verify */ putCmd(dir, "REMOVEDEFAULTACL", null); statusJson = getStatus(dir, "GETFILESTATUS"); - Assert.assertEquals(-1, statusJson.indexOf("aclBit")); + assertEquals(-1, statusJson.indexOf("aclBit")); statusJson = getStatus(dir, "GETACLSTATUS"); aclEntries = getAclEntries(statusJson); - Assert.assertTrue(aclEntries.size() == 0); + assertTrue(aclEntries.size() == 0); } @Test @@ -1026,8 +1029,8 @@ public void testCustomizedUserAndGroupNames() throws Exception { // Verify ACL String statusJson = getStatus(path, "GETACLSTATUS"); List aclEntries = getAclEntries(statusJson); - Assert.assertTrue(aclEntries.contains(aclUser)); - Assert.assertTrue(aclEntries.contains(aclGroup)); + assertTrue(aclEntries.contains(aclUser)); + assertTrue(aclEntries.contains(aclGroup)); } @Test @@ -1050,11 +1053,11 @@ public void testOpenOffsetLength() throws Exception { "/webhdfs/v1/tmp/foo?user.name={0}&op=open&offset=1&length=2", user)); HttpURLConnection conn = (HttpURLConnection) url.openConnection(); - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); InputStream is = conn.getInputStream(); - Assert.assertEquals(1, is.read()); - Assert.assertEquals(2, is.read()); - Assert.assertEquals(-1, is.read()); + assertEquals(1, is.read()); + assertEquals(2, is.read()); + assertEquals(-1, is.read()); } @Test @@ -1089,8 +1092,8 @@ public void testCreateFileWithUnmaskedPermissions() throws Exception { AclStatus aclStatus = fs.getAclStatus(new Path(notUnmaskedFile)); AclEntry theAcl = findAclWithName(aclStatus, "user2"); - Assert.assertNotNull(theAcl); - Assert.assertEquals(FsAction.NONE, + assertNotNull(theAcl); + assertEquals(FsAction.NONE, aclStatus.getEffectivePermission(theAcl)); // Create another file, this time pass a mask of 777. Now the inherited @@ -1100,8 +1103,8 @@ public void testCreateFileWithUnmaskedPermissions() throws Exception { aclStatus = fs.getAclStatus(new Path(unmaskedFile)); theAcl = findAclWithName(aclStatus, "user2"); - Assert.assertNotNull(theAcl); - Assert.assertEquals(FsAction.READ_WRITE, + assertNotNull(theAcl); + assertEquals(FsAction.READ_WRITE, aclStatus.getEffectivePermission(theAcl)); } @@ -1137,8 +1140,8 @@ public void testMkdirWithUnmaskedPermissions() throws Exception { AclStatus aclStatus = fs.getAclStatus(new Path(notUnmaskedDir)); AclEntry theAcl = findAclWithName(aclStatus, "user2"); - Assert.assertNotNull(theAcl); - Assert.assertEquals(FsAction.NONE, + assertNotNull(theAcl); + assertEquals(FsAction.NONE, aclStatus.getEffectivePermission(theAcl)); // Create another file, this time pass a mask of 777. Now the inherited @@ -1148,8 +1151,8 @@ public void testMkdirWithUnmaskedPermissions() throws Exception { aclStatus = fs.getAclStatus(new Path(unmaskedDir)); theAcl = findAclWithName(aclStatus, "user2"); - Assert.assertNotNull(theAcl); - Assert.assertEquals(FsAction.READ_WRITE, + assertNotNull(theAcl); + assertEquals(FsAction.READ_WRITE, aclStatus.getEffectivePermission(theAcl)); } @@ -1167,7 +1170,7 @@ public void testPutNoOperation() throws Exception { conn.setDoInput(true); conn.setDoOutput(true); conn.setRequestMethod("PUT"); - Assert.assertEquals(conn.getResponseCode(), + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_BAD_REQUEST); } @@ -1183,7 +1186,7 @@ public void testGetTrashRoot() throws Exception { Path expectedPath = new Path(FileSystem.USER_HOME_PREFIX, new Path(user, FileSystem.TRASH_PREFIX)); - Assert.assertEquals(expectedPath.toUri().getPath(), trashPath); + assertEquals(expectedPath.toUri().getPath(), trashPath); byte[] array = new byte[]{0, 1, 2, 3}; FileSystem fs = FileSystem.get(TestHdfsHelper.getHdfsConf()); @@ -1194,7 +1197,7 @@ public void testGetTrashRoot() throws Exception { trashJson = getStatus("/tmp/foo", "GETTRASHROOT"); trashPath = getPath(trashJson); - Assert.assertEquals(expectedPath.toUri().getPath(), trashPath); + assertEquals(expectedPath.toUri().getPath(), trashPath); //TestHdfsHelp has already set up EZ environment final Path ezFile = TestHdfsHelper.ENCRYPTED_FILE; @@ -1202,7 +1205,7 @@ public void testGetTrashRoot() throws Exception { trashJson = getStatus(ezFile.toUri().getPath(), "GETTRASHROOT"); trashPath = getPath(trashJson); expectedPath = new Path(ezPath, new Path(FileSystem.TRASH_PREFIX, user)); - Assert.assertEquals(expectedPath.toUri().getPath(), trashPath); + assertEquals(expectedPath.toUri().getPath(), trashPath); } @Test @@ -1226,7 +1229,7 @@ private HttpURLConnection snapshotTestPreconditions(String httpMethod, conn.setRequestMethod("PUT"); conn.connect(); - Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); //needed to make the given dir snapshottable Path snapshottablePath = new Path("/tmp/tmp-snap-test"); @@ -1259,7 +1262,7 @@ public void testAllowSnapshot() throws Exception { DistributedFileSystem dfs = (DistributedFileSystem) FileSystem.get( path.toUri(), TestHdfsHelper.getHdfsConf()); // FileStatus should have snapshot enabled bit unset by default - Assert.assertFalse(dfs.getFileStatus(path).isSnapshotEnabled()); + assertFalse(dfs.getFileStatus(path).isSnapshotEnabled()); // Send a request with ALLOWSNAPSHOT API String user = HadoopUsersConfTestHelper.getHadoopUsers()[0]; URL url = new URL(TestJettyHelper.getJettyURL(), MessageFormat.format( @@ -1269,9 +1272,9 @@ public void testAllowSnapshot() throws Exception { conn.setRequestMethod("PUT"); conn.connect(); // Should return HTTP_OK - Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); // FileStatus should have snapshot enabled bit set - Assert.assertTrue(dfs.getFileStatus(path).isSnapshotEnabled()); + assertTrue(dfs.getFileStatus(path).isSnapshotEnabled()); // Clean up dfs.delete(path, true); } @@ -1292,7 +1295,7 @@ public void testDisallowSnapshot() throws Exception { // Allow snapshot dfs.allowSnapshot(path); // FileStatus should have snapshot enabled bit set so far - Assert.assertTrue(dfs.getFileStatus(path).isSnapshotEnabled()); + assertTrue(dfs.getFileStatus(path).isSnapshotEnabled()); // Send a request with DISALLOWSNAPSHOT API String user = HadoopUsersConfTestHelper.getHadoopUsers()[0]; URL url = new URL(TestJettyHelper.getJettyURL(), MessageFormat.format( @@ -1302,9 +1305,9 @@ public void testDisallowSnapshot() throws Exception { conn.setRequestMethod("PUT"); conn.connect(); // Should return HTTP_OK - Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); // FileStatus should not have snapshot enabled bit set - Assert.assertFalse(dfs.getFileStatus(path).isSnapshotEnabled()); + assertFalse(dfs.getFileStatus(path).isSnapshotEnabled()); // Clean up dfs.delete(path, true); } @@ -1325,7 +1328,7 @@ public void testDisallowSnapshotException() throws Exception { // Allow snapshot dfs.allowSnapshot(path); // FileStatus should have snapshot enabled bit set so far - Assert.assertTrue(dfs.getFileStatus(path).isSnapshotEnabled()); + assertTrue(dfs.getFileStatus(path).isSnapshotEnabled()); // Create some snapshots dfs.createSnapshot(path, "snap-01"); dfs.createSnapshot(path, "snap-02"); @@ -1338,9 +1341,9 @@ public void testDisallowSnapshotException() throws Exception { conn.setRequestMethod("PUT"); conn.connect(); // Should not return HTTP_OK - Assert.assertNotEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertNotEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); // FileStatus should still have snapshot enabled bit set - Assert.assertTrue(dfs.getFileStatus(path).isSnapshotEnabled()); + assertTrue(dfs.getFileStatus(path).isSnapshotEnabled()); // Clean up dfs.deleteSnapshot(path, "snap-02"); dfs.deleteSnapshot(path, "snap-01"); @@ -1356,17 +1359,17 @@ public void testCreateSnapshot() throws Exception { final HttpURLConnection conn = snapshotTestPreconditions("PUT", "CREATESNAPSHOT", "snapshotname=snap-with-name"); - Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); final BufferedReader reader = new BufferedReader(new InputStreamReader(conn.getInputStream())); String result = reader.readLine(); //Validates if the content format is correct - Assert.assertTrue(result. + assertTrue(result. equals("{\"Path\":\"/tmp/tmp-snap-test/.snapshot/snap-with-name\"}")); //Validates if the snapshot is properly created under .snapshot folder result = getStatus("/tmp/tmp-snap-test/.snapshot", "LISTSTATUS"); - Assert.assertTrue(result.contains("snap-with-name")); + assertTrue(result.contains("snap-with-name")); } @Test @@ -1378,19 +1381,19 @@ public void testCreateSnapshotNoSnapshotName() throws Exception { final HttpURLConnection conn = snapshotTestPreconditions("PUT", "CREATESNAPSHOT", ""); - Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); final BufferedReader reader = new BufferedReader( new InputStreamReader(conn.getInputStream())); String result = reader.readLine(); //Validates if the content format is correct - Assert.assertTrue(Pattern.matches( + assertTrue(Pattern.matches( "(\\{\\\"Path\\\"\\:\\\"/tmp/tmp-snap-test/.snapshot/s)" + "(\\d{8})(-)(\\d{6})(\\.)(\\d{3})(\\\"\\})", result)); //Validates if the snapshot is properly created under .snapshot folder result = getStatus("/tmp/tmp-snap-test/.snapshot", "LISTSTATUS"); - Assert.assertTrue(Pattern.matches("(.+)(\\\"pathSuffix\\\":\\\"s)" + + assertTrue(Pattern.matches("(.+)(\\\"pathSuffix\\\":\\\"s)" + "(\\d{8})(-)(\\d{6})(\\.)(\\d{3})(\\\")(.+)", result)); } @@ -1404,18 +1407,18 @@ public void testRenameSnapshot() throws Exception { HttpURLConnection conn = snapshotTestPreconditions("PUT", "CREATESNAPSHOT", "snapshotname=snap-to-rename"); - Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); conn = snapshotTestPreconditions("PUT", "RENAMESNAPSHOT", "oldsnapshotname=snap-to-rename" + "&snapshotname=snap-renamed"); - Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); //Validates the snapshot is properly renamed under .snapshot folder String result = getStatus("/tmp/tmp-snap-test/.snapshot", "LISTSTATUS"); - Assert.assertTrue(result.contains("snap-renamed")); + assertTrue(result.contains("snap-renamed")); //There should be no snapshot named snap-to-rename now - Assert.assertFalse(result.contains("snap-to-rename")); + assertFalse(result.contains("snap-to-rename")); } @Test @@ -1436,15 +1439,15 @@ public void testDeleteSnapshot() throws Exception { HttpURLConnection conn = snapshotTestPreconditions("PUT", "CREATESNAPSHOT", "snapshotname=snap-to-delete"); - Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); conn = snapshotTestPreconditions("DELETE", "DELETESNAPSHOT", "snapshotname=snap-to-delete"); - Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); //Validates the snapshot is not under .snapshot folder anymore String result = getStatus("/tmp/tmp-snap-test/.snapshot", "LISTSTATUS"); - Assert.assertFalse(result.contains("snap-to-delete")); + assertFalse(result.contains("snap-to-delete")); } private HttpURLConnection sendRequestToHttpFSServer(String path, String op, @@ -1481,7 +1484,7 @@ public void testGetSnapshotDiff() throws Exception { path.toUri(), TestHdfsHelper.getHdfsConf()); // Enable snapshot dfs.allowSnapshot(path); - Assert.assertTrue(dfs.getFileStatus(path).isSnapshotEnabled()); + assertTrue(dfs.getFileStatus(path).isSnapshotEnabled()); // Create a file and take a snapshot String file1 = pathStr + "/file1"; createWithHttp(file1, null); @@ -1495,7 +1498,7 @@ public void testGetSnapshotDiff() throws Exception { HttpURLConnection conn = sendRequestGetSnapshotDiff(pathStr, "snap1", "snap2"); // Should return HTTP_OK - Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); // Verify the response BufferedReader reader = new BufferedReader(new InputStreamReader(conn.getInputStream())); @@ -1504,7 +1507,7 @@ public void testGetSnapshotDiff() throws Exception { // Verify the content of diff with DFS API. SnapshotDiffReport dfsDiffReport = dfs.getSnapshotDiffReport(path, "snap1", "snap2"); - Assert.assertEquals(result, JsonUtil.toJsonString(dfsDiffReport)); + assertEquals(result, JsonUtil.toJsonString(dfsDiffReport)); // Clean up dfs.deleteSnapshot(path, "snap2"); dfs.deleteSnapshot(path, "snap1"); @@ -1526,17 +1529,17 @@ public void testGetSnapshotDiffIllegalParam() throws Exception { path.toUri(), TestHdfsHelper.getHdfsConf()); // Enable snapshot dfs.allowSnapshot(path); - Assert.assertTrue(dfs.getFileStatus(path).isSnapshotEnabled()); + assertTrue(dfs.getFileStatus(path).isSnapshotEnabled()); // Send requests with GETSNAPSHOTDIFF API // Snapshots snap1 and snap2 are not created, expect failures but not NPE HttpURLConnection conn = sendRequestGetSnapshotDiff(pathStr, "", ""); - Assert.assertNotEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertNotEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); sendRequestGetSnapshotDiff(pathStr, "snap1", ""); - Assert.assertNotEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertNotEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); sendRequestGetSnapshotDiff(pathStr, "", "snap2"); - Assert.assertNotEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertNotEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); sendRequestGetSnapshotDiff(pathStr, "snap1", "snap2"); - Assert.assertNotEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertNotEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); // Clean up dfs.delete(path, true); } @@ -1547,7 +1550,7 @@ private void verifyGetSnapshottableDirectoryList(DistributedFileSystem dfs) HttpURLConnection conn = sendRequestToHttpFSServer("/", "GETSNAPSHOTTABLEDIRECTORYLIST", ""); // Should return HTTP_OK - Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); // Verify the response BufferedReader reader = new BufferedReader(new InputStreamReader(conn.getInputStream())); @@ -1555,7 +1558,7 @@ private void verifyGetSnapshottableDirectoryList(DistributedFileSystem dfs) String dirLst = reader.readLine(); // Verify the content of diff with DFS API. SnapshottableDirectoryStatus[] dfsDirLst = dfs.getSnapshottableDirListing(); - Assert.assertEquals(dirLst, JsonUtil.toJsonString(dfsDirLst)); + assertEquals(dirLst, JsonUtil.toJsonString(dfsDirLst)); } private void verifyGetSnapshotList(DistributedFileSystem dfs, Path path) @@ -1564,7 +1567,7 @@ private void verifyGetSnapshotList(DistributedFileSystem dfs, Path path) HttpURLConnection conn = sendRequestToHttpFSServer(path.toString(), "GETSNAPSHOTLIST", ""); // Should return HTTP_OK - Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); // Verify the response BufferedReader reader = new BufferedReader(new InputStreamReader(conn.getInputStream())); @@ -1572,7 +1575,7 @@ private void verifyGetSnapshotList(DistributedFileSystem dfs, Path path) String dirLst = reader.readLine(); // Verify the content of status with DFS API. SnapshotStatus[] dfsDirLst = dfs.getSnapshotListing(path); - Assert.assertEquals(dirLst, JsonUtil.toJsonString(dfsDirLst)); + assertEquals(dirLst, JsonUtil.toJsonString(dfsDirLst)); } @Test @@ -1594,12 +1597,12 @@ public void testGetSnapshottableDirectoryList() throws Exception { verifyGetSnapshottableDirectoryList(dfs); // Enable snapshot for path1 dfs.allowSnapshot(path1); - Assert.assertTrue(dfs.getFileStatus(path1).isSnapshotEnabled()); + assertTrue(dfs.getFileStatus(path1).isSnapshotEnabled()); // Verify response when there is one snapshottable directory verifyGetSnapshottableDirectoryList(dfs); // Enable snapshot for path2 dfs.allowSnapshot(path2); - Assert.assertTrue(dfs.getFileStatus(path2).isSnapshotEnabled()); + assertTrue(dfs.getFileStatus(path2).isSnapshotEnabled()); // Verify response when there are two snapshottable directories verifyGetSnapshottableDirectoryList(dfs); @@ -1625,7 +1628,7 @@ public void testGetSnapshotList() throws Exception { path.toUri(), TestHdfsHelper.getHdfsConf()); // Enable snapshot for path1 dfs.allowSnapshot(path); - Assert.assertTrue(dfs.getFileStatus(path).isSnapshotEnabled()); + assertTrue(dfs.getFileStatus(path).isSnapshotEnabled()); // Verify response when there is one snapshottable directory verifyGetSnapshotList(dfs, path); // Create a file and take a snapshot @@ -1659,15 +1662,15 @@ public void testNoRedirect() throws Exception { conn.setRequestMethod(HttpMethod.PUT); conn.connect(); // Verify that it returned the final write location - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); JSONObject json = (JSONObject)new JSONParser().parse( new InputStreamReader(conn.getInputStream())); String location = (String)json.get("Location"); - Assert.assertTrue(location.contains(DataParam.NAME)); - Assert.assertFalse(location.contains(NoRedirectParam.NAME)); - Assert.assertTrue(location.contains("CREATE")); - Assert.assertTrue("Wrong location: " + location, - location.startsWith(TestJettyHelper.getJettyURL().toString())); + assertTrue(location.contains(DataParam.NAME)); + assertFalse(location.contains(NoRedirectParam.NAME)); + assertTrue(location.contains("CREATE")); + assertTrue(location.startsWith(TestJettyHelper.getJettyURL().toString()), + "Wrong location: " + location); // Use the location to actually write the file url = new URL(location); @@ -1681,12 +1684,12 @@ public void testNoRedirect() throws Exception { os.write(testContent.getBytes()); os.close(); // Verify that it created the file and returned the location - Assert.assertEquals( + assertEquals( HttpURLConnection.HTTP_CREATED, conn.getResponseCode()); json = (JSONObject)new JSONParser().parse( new InputStreamReader(conn.getInputStream())); location = (String)json.get("Location"); - Assert.assertEquals( + assertEquals( TestJettyHelper.getJettyURL() + "/webhdfs/v1" + path, location); @@ -1698,14 +1701,14 @@ public void testNoRedirect() throws Exception { conn.setRequestMethod(HttpMethod.GET); conn.connect(); // Verify that we got the final location to read from - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); json = (JSONObject)new JSONParser().parse( new InputStreamReader(conn.getInputStream())); location = (String)json.get("Location"); - Assert.assertTrue(!location.contains(NoRedirectParam.NAME)); - Assert.assertTrue(location.contains("OPEN")); - Assert.assertTrue("Wrong location: " + location, - location.startsWith(TestJettyHelper.getJettyURL().toString())); + assertTrue(!location.contains(NoRedirectParam.NAME)); + assertTrue(location.contains("OPEN")); + assertTrue(location.startsWith(TestJettyHelper.getJettyURL().toString()), + "Wrong location: " + location); // Use the location to actually read url = new URL(location); @@ -1713,9 +1716,9 @@ public void testNoRedirect() throws Exception { conn.setRequestMethod(HttpMethod.GET); conn.connect(); // Verify that we read what we wrote - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); String content = IOUtils.toString(conn.getInputStream(), StandardCharsets.UTF_8); - Assert.assertEquals(testContent, content); + assertEquals(testContent, content); // Get the checksum of the file which shouldn't redirect @@ -1726,14 +1729,14 @@ public void testNoRedirect() throws Exception { conn.setRequestMethod(HttpMethod.GET); conn.connect(); // Verify that we got the final location to write to - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); json = (JSONObject)new JSONParser().parse( new InputStreamReader(conn.getInputStream())); location = (String)json.get("Location"); - Assert.assertTrue(!location.contains(NoRedirectParam.NAME)); - Assert.assertTrue(location.contains("GETFILECHECKSUM")); - Assert.assertTrue("Wrong location: " + location, - location.startsWith(TestJettyHelper.getJettyURL().toString())); + assertTrue(!location.contains(NoRedirectParam.NAME)); + assertTrue(location.contains("GETFILECHECKSUM")); + assertTrue(location.startsWith(TestJettyHelper.getJettyURL().toString()), + "Wrong location: " + location); // Use the location to actually get the checksum url = new URL(location); @@ -1741,15 +1744,15 @@ public void testNoRedirect() throws Exception { conn.setRequestMethod(HttpMethod.GET); conn.connect(); // Verify that we read what we wrote - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); json = (JSONObject)new JSONParser().parse( new InputStreamReader(conn.getInputStream())); JSONObject checksum = (JSONObject)json.get("FileChecksum"); - Assert.assertEquals( + assertEquals( "0000020000000000000000001b9c0a445fed3c0bf1e1aa7438d96b1500000000", checksum.get("bytes")); - Assert.assertEquals(28L, checksum.get("length")); - Assert.assertEquals("MD5-of-0MD5-of-512CRC32C", checksum.get("algorithm")); + assertEquals(28L, checksum.get("length")); + assertEquals("MD5-of-0MD5-of-512CRC32C", checksum.get("algorithm")); } private void verifyGetServerDefaults(DistributedFileSystem dfs) @@ -1758,15 +1761,15 @@ private void verifyGetServerDefaults(DistributedFileSystem dfs) HttpURLConnection conn = sendRequestToHttpFSServer("/", "GETSERVERDEFAULTS", ""); // Should return HTTP_OK - Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); // Verify the response BufferedReader reader = new BufferedReader(new InputStreamReader(conn.getInputStream())); // The response should be a one-line JSON string. String dirLst = reader.readLine(); FsServerDefaults dfsDirLst = dfs.getServerDefaults(); - Assert.assertNotNull(dfsDirLst); - Assert.assertEquals(dirLst, JsonUtil.toJsonString(dfsDirLst)); + assertNotNull(dfsDirLst); + assertEquals(dirLst, JsonUtil.toJsonString(dfsDirLst)); } @Test @@ -1797,10 +1800,10 @@ public void testAccess() throws Exception { HttpURLConnection conn = sendRequestToHttpFSServer(dir, "CHECKACCESS", "fsaction=r--"); - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); HttpURLConnection conn1 = sendRequestToHttpFSServer(dir, "CHECKACCESS", "fsaction=-w-"); - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn1.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn1.getResponseCode()); } @Test @@ -1831,8 +1834,8 @@ public void testECPolicy() throws Exception { JSONObject jsonObject = (JSONObject) parser.parse(getFileStatusResponse); JSONObject details = (JSONObject) jsonObject.get("FileStatus"); String ecpolicyForECfile = (String) details.get("ecPolicy"); - assertEquals("EC policy for ecFile should match the set EC policy", - ecpolicyForECfile, ecPolicyName); + assertEquals(ecpolicyForECfile, ecPolicyName, + "EC policy for ecFile should match the set EC policy"); // Verify httpFs getFileStatus with WEBHDFS REST API WebHdfsFileSystem httpfsWebHdfs = (WebHdfsFileSystem) FileSystem.get( @@ -1862,19 +1865,19 @@ public void testErasureCodingPolicy() throws Exception { HttpURLConnection conn = putCmdWithReturn(dir, "SETECPOLICY", "ecpolicy=" + ecPolicyName); - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); HttpURLConnection conn1 = sendRequestToHttpFSServer(dir, "GETECPOLICY", ""); // Should return HTTP_OK - Assert.assertEquals(conn1.getResponseCode(), HttpURLConnection.HTTP_OK); + assertEquals(conn1.getResponseCode(), HttpURLConnection.HTTP_OK); // Verify the response BufferedReader reader = new BufferedReader(new InputStreamReader(conn1.getInputStream())); // The response should be a one-line JSON string. String dirLst = reader.readLine(); ErasureCodingPolicy dfsDirLst = dfs.getErasureCodingPolicy(path1); - Assert.assertNotNull(dfsDirLst); - Assert.assertEquals(dirLst, JsonUtil.toJsonString(dfsDirLst)); + assertNotNull(dfsDirLst); + assertEquals(dirLst, JsonUtil.toJsonString(dfsDirLst)); String user = HadoopUsersConfTestHelper.getHadoopUsers()[0]; URL url = new URL(TestJettyHelper.getJettyURL(), @@ -1883,18 +1886,18 @@ public void testErasureCodingPolicy() throws Exception { HttpURLConnection conn2 = (HttpURLConnection) url.openConnection(); conn2.setRequestMethod("POST"); conn2.connect(); - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn2.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn2.getResponseCode()); // response should be null dfsDirLst = dfs.getErasureCodingPolicy(path1); - Assert.assertNull(dfsDirLst); + assertNull(dfsDirLst); // test put opeartion with path as "/" final String dir1 = "/"; HttpURLConnection conn3 = putCmdWithReturn(dir1, "SETECPOLICY", "ecpolicy=" + ecPolicyName); // Should return HTTP_OK - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn3.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn3.getResponseCode()); // test post operation with path as "/" final String dir2 = "/"; @@ -1904,7 +1907,7 @@ public void testErasureCodingPolicy() throws Exception { HttpURLConnection conn4 = (HttpURLConnection) url1.openConnection(); conn4.setRequestMethod("POST"); conn4.connect(); - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn4.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn4.getResponseCode()); } @Test @@ -1927,7 +1930,7 @@ public void testStoragePolicySatisfier() throws Exception { assertEquals(HdfsConstants.COLD_STORAGE_POLICY_NAME, storagePolicy.getName()); HttpURLConnection conn = putCmdWithReturn(dir, "SATISFYSTORAGEPOLICY", ""); - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); Map xAttrs = dfs.getXAttrs(path1); assertTrue( xAttrs.containsKey(HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY)); @@ -1952,14 +1955,14 @@ public void testNoRedirectWithData() throws Exception { conn.setRequestProperty("Content-Type", MediaType.APPLICATION_OCTET_STREAM); conn.setDoOutput(true); conn.connect(); - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); JSONObject json = (JSONObject) new JSONParser() .parse(new InputStreamReader(conn.getInputStream())); // get the location to write String location = (String) json.get("Location"); - Assert.assertTrue(location.contains(DataParam.NAME)); - Assert.assertTrue(location.contains("CREATE")); + assertTrue(location.contains(DataParam.NAME)); + assertTrue(location.contains("CREATE")); url = new URL(location); conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod(HttpMethod.PUT); @@ -1971,11 +1974,11 @@ public void testNoRedirectWithData() throws Exception { os.write(writeStr.getBytes()); os.close(); // Verify that file got created - Assert.assertEquals(HttpURLConnection.HTTP_CREATED, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_CREATED, conn.getResponseCode()); json = (JSONObject) new JSONParser() .parse(new InputStreamReader(conn.getInputStream())); location = (String) json.get("Location"); - Assert.assertEquals(TestJettyHelper.getJettyURL() + "/webhdfs/v1" + path, + assertEquals(TestJettyHelper.getJettyURL() + "/webhdfs/v1" + path, location); } @@ -2024,9 +2027,9 @@ public void testGetFileBlockLocations() throws Exception { createWithHttp(file1, null); HttpURLConnection conn = sendRequestToHttpFSServer(file1, "GETFILEBLOCKLOCATIONS", "length=10&offset10"); - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); BlockLocation[] locations1 = dfs.getFileBlockLocations(new Path(file1), 0, 1); - Assert.assertNotNull(locations1); + assertNotNull(locations1); Map jsonMap = JsonSerialization.mapReader().readValue(conn.getInputStream()); diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoACLs.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoACLs.java index 6b3dfb4f5646c..ff78afabe1bdd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoACLs.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoACLs.java @@ -30,8 +30,7 @@ import org.apache.hadoop.test.TestDirHelper; import org.apache.hadoop.test.TestJetty; import org.apache.hadoop.test.TestJettyHelper; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.webapp.WebAppContext; @@ -47,6 +46,9 @@ import java.net.URL; import java.text.MessageFormat; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + /** * This test class ensures that everything works as expected when ACL * support is turned off HDFS. This is the default configuration. The other @@ -99,9 +101,9 @@ private void startMiniDFS() throws Exception { */ private void createHttpFSServer() throws Exception { File homeDir = TestDirHelper.getTestDir(); - Assert.assertTrue(new File(homeDir, "conf").mkdir()); - Assert.assertTrue(new File(homeDir, "log").mkdir()); - Assert.assertTrue(new File(homeDir, "temp").mkdir()); + assertTrue(new File(homeDir, "conf").mkdir()); + assertTrue(new File(homeDir, "log").mkdir()); + assertTrue(new File(homeDir, "temp").mkdir()); HttpFSServerWebApp.setHomeDirForCurrentThread(homeDir.getAbsolutePath()); File secretFile = new File(new File(homeDir, "conf"), "secret"); @@ -182,17 +184,17 @@ private void getStatus(String filename, String command, boolean expectOK) int resp = conn.getResponseCode(); BufferedReader reader; if (expectOK) { - Assert.assertEquals(HttpURLConnection.HTTP_OK, resp); + assertEquals(HttpURLConnection.HTTP_OK, resp); reader = new BufferedReader(new InputStreamReader(conn.getInputStream())); String res = reader.readLine(); - Assert.assertTrue(!res.contains("aclBit")); - Assert.assertTrue(res.contains("owner")); // basic sanity check + assertTrue(!res.contains("aclBit")); + assertTrue(res.contains("owner")); // basic sanity check } else { - Assert.assertEquals(HttpURLConnection.HTTP_INTERNAL_ERROR, resp); + assertEquals(HttpURLConnection.HTTP_INTERNAL_ERROR, resp); reader = new BufferedReader(new InputStreamReader(conn.getErrorStream())); String res = reader.readLine(); - Assert.assertTrue(res.contains("AclException")); - Assert.assertTrue(res.contains("Support for ACLs has been disabled")); + assertTrue(res.contains("AclException")); + assertTrue(res.contains("Support for ACLs has been disabled")); } } @@ -219,14 +221,14 @@ private void putCmd(String filename, String command, conn.connect(); int resp = conn.getResponseCode(); if (expectOK) { - Assert.assertEquals(HttpURLConnection.HTTP_OK, resp); + assertEquals(HttpURLConnection.HTTP_OK, resp); } else { - Assert.assertEquals(HttpURLConnection.HTTP_INTERNAL_ERROR, resp); + assertEquals(HttpURLConnection.HTTP_INTERNAL_ERROR, resp); BufferedReader reader; reader = new BufferedReader(new InputStreamReader(conn.getErrorStream())); String err = reader.readLine(); - Assert.assertTrue(err.contains("AclException")); - Assert.assertTrue(err.contains("Support for ACLs has been disabled")); + assertTrue(err.contains("AclException")); + assertTrue(err.contains("Support for ACLs has been disabled")); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoXAttrs.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoXAttrs.java index ac70c07fda1a6..a7dcffe06fbd1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoXAttrs.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoXAttrs.java @@ -31,8 +31,7 @@ import org.apache.hadoop.test.TestHdfs; import org.apache.hadoop.test.TestJetty; import org.apache.hadoop.test.TestJettyHelper; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.webapp.WebAppContext; @@ -48,6 +47,9 @@ import java.net.URL; import java.text.MessageFormat; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + /** * This test class ensures that everything works as expected when XAttr * support is turned off HDFS. This is the default configuration. The other @@ -100,9 +102,9 @@ private void startMiniDFS() throws Exception { */ private void createHttpFSServer() throws Exception { File homeDir = TestDirHelper.getTestDir(); - Assert.assertTrue(new File(homeDir, "conf").mkdir()); - Assert.assertTrue(new File(homeDir, "log").mkdir()); - Assert.assertTrue(new File(homeDir, "temp").mkdir()); + assertTrue(new File(homeDir, "conf").mkdir()); + assertTrue(new File(homeDir, "log").mkdir()); + assertTrue(new File(homeDir, "temp").mkdir()); HttpFSServerWebApp.setHomeDirForCurrentThread(homeDir.getAbsolutePath()); File secretFile = new File(new File(homeDir, "conf"), "secret"); @@ -181,12 +183,12 @@ private void getStatus(String filename, String command) conn.connect(); int resp = conn.getResponseCode(); BufferedReader reader; - Assert.assertEquals(HttpURLConnection.HTTP_INTERNAL_ERROR, resp); + assertEquals(HttpURLConnection.HTTP_INTERNAL_ERROR, resp); reader = new BufferedReader(new InputStreamReader(conn.getErrorStream())); String res = reader.readLine(); - Assert.assertTrue(res.contains("RemoteException")); - Assert.assertTrue(res.contains("XAttr")); - Assert.assertTrue(res.contains("rejected")); + assertTrue(res.contains("RemoteException")); + assertTrue(res.contains("XAttr")); + assertTrue(res.contains("rejected")); } /** @@ -211,13 +213,13 @@ private void putCmd(String filename, String command, conn.setRequestMethod("PUT"); conn.connect(); int resp = conn.getResponseCode(); - Assert.assertEquals(HttpURLConnection.HTTP_INTERNAL_ERROR, resp); + assertEquals(HttpURLConnection.HTTP_INTERNAL_ERROR, resp); BufferedReader reader; reader = new BufferedReader(new InputStreamReader(conn.getErrorStream())); String err = reader.readLine(); - Assert.assertTrue(err.contains("RemoteException")); - Assert.assertTrue(err.contains("XAttr")); - Assert.assertTrue(err.contains("rejected")); + assertTrue(err.contains("RemoteException")); + assertTrue(err.contains("XAttr")); + assertTrue(err.contains("rejected")); } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerWebServer.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerWebServer.java index e0fdef59889a9..d096a9b0b3a7c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerWebServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerWebServer.java @@ -37,27 +37,26 @@ import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.HadoopUsersConfTestHelper; import org.apache.hadoop.util.Shell; -import org.junit.Assert; -import org.junit.Before; -import org.junit.After; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.Timeout; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import static org.apache.hadoop.security.authentication.server.AuthenticationFilter.SIGNER_SECRET_PROVIDER_ATTRIBUTE; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Test {@link HttpFSServerWebServer}. */ +@Timeout(30) public class TestHttpFSServerWebServer { - @Rule - public Timeout timeout = new Timeout(30000); - private File secretFile; private HttpFSServerWebServer webServer; - @Before + @BeforeEach public void init() throws Exception { File homeDir = GenericTestUtils.setupTestRootDir(TestHttpFSServerWebServer.class); File confDir = new File(homeDir, "etc/hadoop"); @@ -85,7 +84,7 @@ public void init() throws Exception { "httpfs-signature-custom.secret"); } - @After + @AfterEach public void teardown() throws Exception { if (webServer != null) { webServer.stop(); @@ -187,8 +186,9 @@ private void assertSignerSecretProviderType( SignerSecretProvider secretProvider = (SignerSecretProvider) server.getWebAppContext().getServletContext() .getAttribute(SIGNER_SECRET_PROVIDER_ATTRIBUTE); - Assert.assertNotNull("The secret provider must not be null", secretProvider); - Assert.assertEquals("The secret provider must match the following", expected, secretProvider.getClass()); + assertNotNull(secretProvider, "The secret provider must not be null"); + assertEquals(expected, secretProvider.getClass(), + "The secret provider must match the following"); } private void assertServiceRespondsWithOK(URL serviceURL) @@ -197,7 +197,7 @@ private void assertServiceRespondsWithOK(URL serviceURL) URL url = new URL(serviceURL, MessageFormat.format( "/webhdfs/v1/?user.name={0}&op=liststatus", user)); HttpURLConnection conn = (HttpURLConnection) url.openConnection(); - Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); + assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode()); try (BufferedReader reader = new BufferedReader( new InputStreamReader(conn.getInputStream()))) { reader.readLine(); @@ -247,7 +247,7 @@ private HttpFSServerWebServer createWebServer(Configuration conf) } private void createSecretFile(String content) throws IOException { - Assert.assertTrue(secretFile.createNewFile()); + assertTrue(secretFile.createNewFile()); FileUtils.writeStringToFile(secretFile, content, StandardCharsets.UTF_8); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSWithKerberos.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSWithKerberos.java index fafeff076e95b..44da554a849e1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSWithKerberos.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSWithKerberos.java @@ -38,9 +38,8 @@ import org.apache.hadoop.test.TestJettyHelper; import org.json.simple.JSONObject; import org.json.simple.parser.JSONParser; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.webapp.WebAppContext; @@ -56,9 +55,12 @@ import java.security.PrivilegedExceptionAction; import java.util.concurrent.Callable; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + public class TestHttpFSWithKerberos extends HFSTestCase { - @After + @AfterEach public void resetUGI() { Configuration conf = new Configuration(); UserGroupInformation.setConfiguration(conf); @@ -66,9 +68,9 @@ public void resetUGI() { private void createHttpFSServer() throws Exception { File homeDir = TestDirHelper.getTestDir(); - Assert.assertTrue(new File(homeDir, "conf").mkdir()); - Assert.assertTrue(new File(homeDir, "log").mkdir()); - Assert.assertTrue(new File(homeDir, "temp").mkdir()); + assertTrue(new File(homeDir, "conf").mkdir()); + assertTrue(new File(homeDir, "log").mkdir()); + assertTrue(new File(homeDir, "temp").mkdir()); HttpFSServerWebApp.setHomeDirForCurrentThread(homeDir.getAbsolutePath()); File secretFile = new File(new File(homeDir, "conf"), "secret"); @@ -125,7 +127,7 @@ public Void call() throws Exception { AuthenticatedURL aUrl = new AuthenticatedURL(); AuthenticatedURL.Token aToken = new AuthenticatedURL.Token(); HttpURLConnection conn = aUrl.openConnection(url, aToken); - Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); return null; } }); @@ -141,8 +143,7 @@ public void testInvalidadHttpFSAccess() throws Exception { URL url = new URL(TestJettyHelper.getJettyURL(), "/webhdfs/v1/?op=GETHOMEDIRECTORY"); HttpURLConnection conn = (HttpURLConnection) url.openConnection(); - Assert.assertEquals(conn.getResponseCode(), - HttpURLConnection.HTTP_UNAUTHORIZED); + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_UNAUTHORIZED); } @Test @@ -161,7 +162,7 @@ public Void call() throws Exception { AuthenticatedURL aUrl = new AuthenticatedURL(); AuthenticatedURL.Token aToken = new AuthenticatedURL.Token(); HttpURLConnection conn = aUrl.openConnection(url, aToken); - Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); JSONObject json = (JSONObject) new JSONParser() .parse(new InputStreamReader(conn.getInputStream())); json = @@ -175,22 +176,21 @@ public Void call() throws Exception { "/webhdfs/v1/?op=GETHOMEDIRECTORY&delegation=" + tokenStr); conn = (HttpURLConnection) url.openConnection(); - Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); //try to renew the delegation token without SPNEGO credentials url = new URL(TestJettyHelper.getJettyURL(), "/webhdfs/v1/?op=RENEWDELEGATIONTOKEN&token=" + tokenStr); conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod("PUT"); - Assert.assertEquals(conn.getResponseCode(), - HttpURLConnection.HTTP_UNAUTHORIZED); + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_UNAUTHORIZED); //renew the delegation token with SPNEGO credentials url = new URL(TestJettyHelper.getJettyURL(), "/webhdfs/v1/?op=RENEWDELEGATIONTOKEN&token=" + tokenStr); conn = aUrl.openConnection(url, aToken); conn.setRequestMethod("PUT"); - Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); //cancel delegation token, no need for SPNEGO credentials url = new URL(TestJettyHelper.getJettyURL(), @@ -198,15 +198,14 @@ public Void call() throws Exception { tokenStr); conn = (HttpURLConnection) url.openConnection(); conn.setRequestMethod("PUT"); - Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK); //try to access httpfs with the canceled delegation token url = new URL(TestJettyHelper.getJettyURL(), "/webhdfs/v1/?op=GETHOMEDIRECTORY&delegation=" + tokenStr); conn = (HttpURLConnection) url.openConnection(); - Assert.assertEquals(conn.getResponseCode(), - HttpURLConnection.HTTP_UNAUTHORIZED); + assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_UNAUTHORIZED); return null; } }); @@ -224,7 +223,7 @@ private void testDelegationTokenWithFS(Class fileSystemClass) FileSystem fs = FileSystem.get(uri, conf); Token tokens[] = fs.addDelegationTokens("foo", null); fs.close(); - Assert.assertEquals(1, tokens.length); + assertEquals(1, tokens.length); fs = FileSystem.get(uri, conf); ((DelegationTokenRenewer.Renewable) fs).setDelegationToken(tokens[0]); fs.listStatus(new Path("/")); diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/lang/TestRunnableCallable.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/lang/TestRunnableCallable.java index 1520af8776161..1ea34f61d49c4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/lang/TestRunnableCallable.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/lang/TestRunnableCallable.java @@ -19,13 +19,12 @@ package org.apache.hadoop.lib.lang; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - import java.util.concurrent.Callable; import org.apache.hadoop.test.HTestCase; -import org.junit.Test; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.*; public class TestRunnableCallable extends HTestCase { @@ -86,11 +85,13 @@ public void callable() throws Exception { assertEquals(rc.toString(), "C"); } - @Test(expected = RuntimeException.class) + @Test public void callableExRun() throws Exception { - CEx c = new CEx(); - RunnableCallable rc = new RunnableCallable(c); - rc.run(); + assertThrows(RuntimeException.class, ()->{ + CEx c = new CEx(); + RunnableCallable rc = new RunnableCallable(c); + rc.run(); + }); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/lang/TestXException.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/lang/TestXException.java index 2869d47ca8d8b..88bf7c2d956cf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/lang/TestXException.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/lang/TestXException.java @@ -19,11 +19,11 @@ package org.apache.hadoop.lib.lang; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; import org.apache.hadoop.test.HTestCase; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestXException extends HTestCase { diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestBaseService.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestBaseService.java index 402884bfbcaac..beb7b700c5b3c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestBaseService.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestBaseService.java @@ -18,13 +18,13 @@ package org.apache.hadoop.lib.server; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.test.HTestCase; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; public class TestBaseService extends HTestCase { diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestServer.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestServer.java index ff1d1ca0ad55c..d2ee5ef8e5f17 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestServer.java @@ -18,12 +18,13 @@ package org.apache.hadoop.lib.server; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +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.FileOutputStream; @@ -43,7 +44,7 @@ import org.apache.hadoop.test.TestDirHelper; import org.apache.hadoop.test.TestException; import org.apache.hadoop.util.StringUtils; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestServer extends HTestCase { @@ -109,12 +110,14 @@ public void initNoHomeDir() throws Exception { @TestException(exception = ServerException.class, msgRegExp = "S02.*") @TestDir public void initHomeDirNotDir() throws Exception { - File homeDir = new File(TestDirHelper.getTestDir(), "home"); - new FileOutputStream(homeDir).close(); - Configuration conf = new Configuration(false); - conf.set("server.services", TestService.class.getName()); - Server server = new Server("server", homeDir.getAbsolutePath(), conf); - server.init(); + assertThrows(ServerException.class, () -> { + File homeDir = new File(TestDirHelper.getTestDir(), "home"); + new FileOutputStream(homeDir).close(); + Configuration conf = new Configuration(false); + conf.set("server.services", TestService.class.getName()); + Server server = new Server("server", homeDir.getAbsolutePath(), conf); + server.init(); + }); } @Test @@ -283,13 +286,15 @@ public void startWithStatusNotNormal() throws Exception { server.destroy(); } - @Test(expected = IllegalArgumentException.class) + @Test @TestDir public void nonSeteableStatus() throws Exception { - Configuration conf = new Configuration(false); - Server server = createServer(conf); - server.init(); - server.setStatus(Server.Status.SHUTDOWN); + assertThrows(IllegalArgumentException.class, ()->{ + Configuration conf = new Configuration(false); + Server server = createServer(conf); + server.init(); + server.setStatus(Server.Status.SHUTDOWN); + }); } public static class TestService implements Service { @@ -422,34 +427,45 @@ public void loadingSysPropConfig() throws Exception { } } - @Test(expected = IllegalStateException.class) + @Test @TestDir public void illegalState1() throws Exception { - Server server = new Server("server", TestDirHelper.getTestDir().getAbsolutePath(), new Configuration(false)); - server.destroy(); + assertThrows(IllegalStateException.class, ()->{ + Server server = new Server("server", + TestDirHelper.getTestDir().getAbsolutePath(), new Configuration(false)); + server.destroy(); + }); } - @Test(expected = IllegalStateException.class) + @Test @TestDir public void illegalState2() throws Exception { - Server server = new Server("server", TestDirHelper.getTestDir().getAbsolutePath(), new Configuration(false)); - server.get(Object.class); + assertThrows(IllegalStateException.class, () -> { + Server server = new Server("server", + TestDirHelper.getTestDir().getAbsolutePath(), new Configuration(false)); + server.get(Object.class); + }); } - @Test(expected = IllegalStateException.class) + @Test @TestDir public void illegalState3() throws Exception { - Server server = new Server("server", TestDirHelper.getTestDir().getAbsolutePath(), new Configuration(false)); - server.setService(null); + assertThrows(IllegalStateException.class, () -> { + Server server = new Server("server", + TestDirHelper.getTestDir().getAbsolutePath(), new Configuration(false)); + server.setService(null); + }); } - @Test(expected = IllegalStateException.class) + @Test @TestDir public void illegalState4() throws Exception { - String dir = TestDirHelper.getTestDir().getAbsolutePath(); - Server server = new Server("server", dir, dir, dir, dir, new Configuration(false)); - server.init(); - server.init(); + assertThrows(IllegalStateException.class, () -> { + String dir = TestDirHelper.getTestDir().getAbsolutePath(); + Server server = new Server("server", dir, dir, dir, dir, new Configuration(false)); + server.init(); + server.init(); + }); } private static List ORDER = new ArrayList(); diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestServerConstructor.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestServerConstructor.java index 6b7c6286d09cf..447dddba302ea 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestServerConstructor.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestServerConstructor.java @@ -23,14 +23,13 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.test.HTestCase; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -@RunWith(value = Parameterized.class) +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; +import static org.junit.jupiter.api.Assertions.assertThrows; + public class TestServerConstructor extends HTestCase { - @Parameterized.Parameters public static Collection constructorFailParams() { return Arrays.asList(new Object[][]{ {null, null, null, null, null, null}, @@ -57,20 +56,24 @@ public static Collection constructorFailParams() { private String tempDir; private Configuration conf; - public TestServerConstructor(String name, String homeDir, String configDir, String logDir, String tempDir, - Configuration conf) { - this.name = name; - this.homeDir = homeDir; - this.configDir = configDir; - this.logDir = logDir; - this.tempDir = tempDir; - this.conf = conf; + public void initTestServerConstructor(String pName, String pHomeDir, + String pConfigDir, String pLogDir, String pTempDir, Configuration pConf) { + this.name = pName; + this.homeDir = pHomeDir; + this.configDir = pConfigDir; + this.logDir = pLogDir; + this.tempDir = pTempDir; + this.conf = pConf; } - - @Test(expected = IllegalArgumentException.class) - public void constructorFail() { - new Server(name, homeDir, configDir, logDir, tempDir, conf); + @ParameterizedTest + @MethodSource("constructorFailParams") + public void constructorFail(String pName, String pHomeDir, + String pConfigDir, String pLogDir, String pTempDir, Configuration pConf) { + initTestServerConstructor(pName, pHomeDir, pConfigDir, pLogDir, pTempDir, pConf); + assertThrows(IllegalArgumentException.class, () -> { + new Server(name, homeDir, configDir, logDir, tempDir, conf); + }); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/hadoop/TestFileSystemAccessService.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/hadoop/TestFileSystemAccessService.java index ed9efa945f209..07a29e8759484 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/hadoop/TestFileSystemAccessService.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/hadoop/TestFileSystemAccessService.java @@ -41,9 +41,12 @@ import org.apache.hadoop.test.TestHdfs; import org.apache.hadoop.test.TestHdfsHelper; import org.apache.hadoop.util.StringUtils; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.fail; public class TestFileSystemAccessService extends HFSTestCase { @@ -55,7 +58,7 @@ private void createHadoopConf(Configuration hadoopConf) throws Exception { os.close(); } - @Before + @BeforeEach public void createHadoopConf() throws Exception { Configuration hadoopConf = new Configuration(false); hadoopConf.set("foo", "FOO"); @@ -74,7 +77,7 @@ public void simpleSecurity() throws Exception { conf.set("server.services", services); Server server = new Server("server", dir, dir, dir, dir, conf); server.init(); - Assert.assertNotNull(server.get(FileSystemAccess.class)); + assertNotNull(server.get(FileSystemAccess.class)); server.destroy(); } @@ -161,7 +164,7 @@ public void serviceHadoopConf() throws Exception { Server server = new Server("server", dir, dir, dir, dir, conf); server.init(); FileSystemAccessService fsAccess = (FileSystemAccessService) server.get(FileSystemAccess.class); - Assert.assertEquals(fsAccess.serviceHadoopConf.get("foo"), "FOO"); + assertEquals(fsAccess.serviceHadoopConf.get("foo"), "FOO"); server.destroy(); } @@ -189,7 +192,7 @@ public void serviceHadoopConfCustomDir() throws Exception { Server server = new Server("server", dir, dir, dir, dir, conf); server.init(); FileSystemAccessService fsAccess = (FileSystemAccessService) server.get(FileSystemAccess.class); - Assert.assertEquals(fsAccess.serviceHadoopConf.get("foo"), "BAR"); + assertEquals(fsAccess.serviceHadoopConf.get("foo"), "BAR"); server.destroy(); } @@ -267,15 +270,15 @@ public void createFileSystem() throws Exception { server.init(); FileSystemAccess hadoop = server.get(FileSystemAccess.class); FileSystem fs = hadoop.createFileSystem("u", hadoop.getFileSystemConfiguration()); - Assert.assertNotNull(fs); + assertNotNull(fs); fs.mkdirs(new Path("/tmp/foo")); hadoop.releaseFileSystem(fs); try { fs.mkdirs(new Path("/tmp/foo")); - Assert.fail(); + fail(); } catch (IOException ex) { } catch (Exception ex) { - Assert.fail(); + fail(); } server.destroy(); } @@ -313,10 +316,10 @@ public Void execute(FileSystem fs) throws IOException { }); try { fsa[0].mkdirs(new Path("/tmp/foo")); - Assert.fail(); + fail(); } catch (IOException ex) { } catch (Exception ex) { - Assert.fail(); + fail(); } server.destroy(); } @@ -381,19 +384,19 @@ public Void execute(FileSystem fs) throws IOException { throw new IOException(); } }); - Assert.fail(); + fail(); } catch (FileSystemAccessException ex) { - Assert.assertEquals(ex.getError(), FileSystemAccessException.ERROR.H03); + assertEquals(ex.getError(), FileSystemAccessException.ERROR.H03); } catch (Exception ex) { - Assert.fail(); + fail(); } try { fsa[0].mkdirs(new Path("/tmp/foo")); - Assert.fail(); + fail(); } catch (IOException ex) { } catch (Exception ex) { - Assert.fail(); + fail(); } server.destroy(); } @@ -424,7 +427,7 @@ public void fileSystemCache() throws Exception { FileSystem fs1 = hadoop.createFileSystem("u", hadoop.getFileSystemConfiguration()); - Assert.assertNotNull(fs1); + assertNotNull(fs1); fs1.mkdirs(new Path("/tmp/foo1")); hadoop.releaseFileSystem(fs1); @@ -435,7 +438,7 @@ public void fileSystemCache() throws Exception { hadoop.createFileSystem("u", hadoop.getFileSystemConfiguration()); //should be same instance because of caching - Assert.assertEquals(fs1, fs2); + assertEquals(fs1, fs2); Thread.sleep(4 * 1000); @@ -453,10 +456,10 @@ public void fileSystemCache() throws Exception { //should not be around as lease count is 0 try { fs2.mkdirs(new Path("/tmp/foo")); - Assert.fail(); + fail(); } catch (IOException ex) { } catch (Exception ex) { - Assert.fail(); + fail(); } } finally { server.destroy(); diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/instrumentation/TestInstrumentationService.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/instrumentation/TestInstrumentationService.java index c609fefc80bc5..86d42ccdf2d79 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/instrumentation/TestInstrumentationService.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/instrumentation/TestInstrumentationService.java @@ -18,10 +18,10 @@ package org.apache.hadoop.lib.service.instrumentation; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import java.io.StringWriter; import java.util.Arrays; @@ -39,7 +39,7 @@ import org.apache.hadoop.util.Time; import org.json.simple.JSONObject; import org.json.simple.parser.JSONParser; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestInstrumentationService extends HTestCase { diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/scheduler/TestSchedulerService.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/scheduler/TestSchedulerService.java index f8abb48e7aa9b..074cbfba036e6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/scheduler/TestSchedulerService.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/scheduler/TestSchedulerService.java @@ -18,7 +18,7 @@ package org.apache.hadoop.lib.service.scheduler; -import static org.junit.Assert.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNotNull; import java.util.Arrays; @@ -30,7 +30,7 @@ import org.apache.hadoop.test.TestDir; import org.apache.hadoop.test.TestDirHelper; import org.apache.hadoop.util.StringUtils; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestSchedulerService extends HTestCase { diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/security/TestGroupsService.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/security/TestGroupsService.java index 445192b66fde1..7e916c6ed5107 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/security/TestGroupsService.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/security/TestGroupsService.java @@ -18,8 +18,9 @@ package org.apache.hadoop.lib.service.security; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNotSame; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNotSame; +import static org.junit.jupiter.api.Assertions.assertThrows; import java.util.Arrays; import java.util.List; @@ -31,7 +32,7 @@ import org.apache.hadoop.test.TestDir; import org.apache.hadoop.test.TestDirHelper; import org.apache.hadoop.util.StringUtils; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestGroupsService extends HTestCase { @@ -50,15 +51,18 @@ public void service() throws Exception { server.destroy(); } - @Test(expected = RuntimeException.class) + @Test @TestDir public void invalidGroupsMapping() throws Exception { - String dir = TestDirHelper.getTestDir().getAbsolutePath(); - Configuration conf = new Configuration(false); - conf.set("server.services", StringUtils.join(",", Arrays.asList(GroupsService.class.getName()))); - conf.set("server.groups.hadoop.security.group.mapping", String.class.getName()); - Server server = new Server("server", dir, dir, dir, dir, conf); - server.init(); + assertThrows(RuntimeException.class, () -> { + String dir = TestDirHelper.getTestDir().getAbsolutePath(); + Configuration conf = new Configuration(false); + conf.set("server.services", StringUtils.join(",", + Arrays.asList(GroupsService.class.getName()))); + conf.set("server.groups.hadoop.security.group.mapping", String.class.getName()); + Server server = new Server("server", dir, dir, dir, dir, conf); + server.init(); + }); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/servlet/TestHostnameFilter.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/servlet/TestHostnameFilter.java index 203796ead8ebf..189c89afa0941 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/servlet/TestHostnameFilter.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/servlet/TestHostnameFilter.java @@ -18,8 +18,8 @@ package org.apache.hadoop.lib.servlet; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.IOException; import java.util.concurrent.atomic.AtomicBoolean; @@ -31,7 +31,7 @@ import javax.servlet.ServletResponse; import org.apache.hadoop.test.HTestCase; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/servlet/TestMDCFilter.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/servlet/TestMDCFilter.java index 911cc0ad23012..89ac127770899 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/servlet/TestMDCFilter.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/servlet/TestMDCFilter.java @@ -18,9 +18,9 @@ package org.apache.hadoop.lib.servlet; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.IOException; import java.security.Principal; @@ -34,7 +34,7 @@ import javax.servlet.http.HttpServletRequest; import org.apache.hadoop.test.HTestCase; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import org.slf4j.MDC; diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/servlet/TestServerWebApp.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/servlet/TestServerWebApp.java index 889d20b75848c..6eafe400fadfa 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/servlet/TestServerWebApp.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/servlet/TestServerWebApp.java @@ -18,22 +18,25 @@ package org.apache.hadoop.lib.servlet; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; import org.apache.hadoop.lib.server.Server; import org.apache.hadoop.test.HTestCase; import org.apache.hadoop.test.TestDir; import org.apache.hadoop.test.TestDirHelper; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.net.InetSocketAddress; public class TestServerWebApp extends HTestCase { - @Test(expected = IllegalArgumentException.class) + @Test public void getHomeDirNotDef() { - ServerWebApp.getHomeDir("TestServerWebApp00"); + assertThrows(IllegalArgumentException.class, () -> { + ServerWebApp.getHomeDir("TestServerWebApp00"); + }); } @Test @@ -63,19 +66,21 @@ public void lifecycle() throws Exception { assertEquals(server.getStatus(), Server.Status.SHUTDOWN); } - @Test(expected = RuntimeException.class) + @Test @TestDir public void failedInit() throws Exception { - String dir = TestDirHelper.getTestDir().getAbsolutePath(); - System.setProperty("TestServerWebApp2.home.dir", dir); - System.setProperty("TestServerWebApp2.config.dir", dir); - System.setProperty("TestServerWebApp2.log.dir", dir); - System.setProperty("TestServerWebApp2.temp.dir", dir); - System.setProperty("testserverwebapp2.services", "FOO"); - ServerWebApp server = new ServerWebApp("TestServerWebApp2") { - }; + assertThrows(RuntimeException.class, () -> { + String dir = TestDirHelper.getTestDir().getAbsolutePath(); + System.setProperty("TestServerWebApp2.home.dir", dir); + System.setProperty("TestServerWebApp2.config.dir", dir); + System.setProperty("TestServerWebApp2.log.dir", dir); + System.setProperty("TestServerWebApp2.temp.dir", dir); + System.setProperty("testserverwebapp2.services", "FOO"); + ServerWebApp server = new ServerWebApp("TestServerWebApp2") { + }; - server.contextInitialized(null); + server.contextInitialized(null); + }); } @Test @@ -92,8 +97,8 @@ public void testResolveAuthority() throws Exception { }; InetSocketAddress address = server.resolveAuthority(); - Assert.assertEquals("localhost", address.getHostName()); - Assert.assertEquals(14000, address.getPort()); + Assertions.assertEquals("localhost", address.getHostName()); + Assertions.assertEquals(14000, address.getPort()); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/util/TestCheck.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/util/TestCheck.java index a6ff4788f69f4..06baf8655512b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/util/TestCheck.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/util/TestCheck.java @@ -19,13 +19,14 @@ package org.apache.hadoop.lib.util; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; import java.util.ArrayList; import java.util.Arrays; import org.apache.hadoop.test.HTestCase; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestCheck extends HTestCase { @@ -34,9 +35,11 @@ public void notNullNotNull() { assertEquals(Check.notNull("value", "name"), "value"); } - @Test(expected = IllegalArgumentException.class) + @Test public void notNullNull() { - Check.notNull(null, "name"); + assertThrows(IllegalArgumentException.class, () -> { + Check.notNull(null, "name"); + }); } @Test @@ -45,14 +48,18 @@ public void notNullElementsNotNull() { Check.notNullElements(Arrays.asList("a"), "name"); } - @Test(expected = IllegalArgumentException.class) + @Test public void notNullElementsNullList() { - Check.notNullElements(null, "name"); + assertThrows(IllegalArgumentException.class, () -> { + Check.notNullElements(null, "name"); + }); } - @Test(expected = IllegalArgumentException.class) + @Test public void notNullElementsNullElements() { - Check.notNullElements(Arrays.asList("a", "", null), "name"); + assertThrows(IllegalArgumentException.class, () -> { + Check.notNullElements(Arrays.asList("a", "", null), "name"); + }); } @Test @@ -61,20 +68,25 @@ public void notEmptyElementsNotNull() { Check.notEmptyElements(Arrays.asList("a"), "name"); } - @Test(expected = IllegalArgumentException.class) + @Test public void notEmptyElementsNullList() { - Check.notEmptyElements(null, "name"); + assertThrows(IllegalArgumentException.class, () -> { + Check.notEmptyElements(null, "name"); + }); } - @Test(expected = IllegalArgumentException.class) + @Test public void notEmptyElementsNullElements() { - Check.notEmptyElements(Arrays.asList("a", null), "name"); + assertThrows(IllegalArgumentException.class, () -> { + Check.notEmptyElements(Arrays.asList("a", null), "name"); + }); } - - @Test(expected = IllegalArgumentException.class) + @Test public void notEmptyElementsEmptyElements() { - Check.notEmptyElements(Arrays.asList("a", ""), "name"); + assertThrows(IllegalArgumentException.class, () -> { + Check.notEmptyElements(Arrays.asList("a", ""), "name"); + }); } @@ -83,14 +95,18 @@ public void notEmptyNotEmtpy() { assertEquals(Check.notEmpty("value", "name"), "value"); } - @Test(expected = IllegalArgumentException.class) + @Test public void notEmptyNull() { - Check.notEmpty(null, "name"); + assertThrows(IllegalArgumentException.class, () -> { + Check.notEmpty(null, "name"); + }); } - @Test(expected = IllegalArgumentException.class) + @Test public void notEmptyEmpty() { - Check.notEmpty("", "name"); + assertThrows(IllegalArgumentException.class, () -> { + Check.notEmpty("", "name"); + }); } @Test @@ -101,29 +117,39 @@ public void validIdentifierValid() throws Exception { assertEquals(Check.validIdentifier("_", 1, ""), "_"); } - @Test(expected = IllegalArgumentException.class) + @Test public void validIdentifierInvalid1() throws Exception { - Check.validIdentifier("!", 1, ""); + assertThrows(IllegalArgumentException.class, () -> { + Check.validIdentifier("!", 1, ""); + }); } - @Test(expected = IllegalArgumentException.class) + @Test public void validIdentifierInvalid2() throws Exception { - Check.validIdentifier("a1", 1, ""); + assertThrows(IllegalArgumentException.class, () -> { + Check.validIdentifier("a1", 1, ""); + }); } - @Test(expected = IllegalArgumentException.class) + @Test public void validIdentifierInvalid3() throws Exception { - Check.validIdentifier("1", 1, ""); + assertThrows(IllegalArgumentException.class, () -> { + Check.validIdentifier("1", 1, ""); + }); } - @Test(expected = IllegalArgumentException.class) + @Test public void validIdentifierInvalid4() throws Exception { - Check.validIdentifier("`a", 2, ""); + assertThrows(IllegalArgumentException.class, () -> { + Check.validIdentifier("`a", 2, ""); + }); } - @Test(expected = IllegalArgumentException.class) + @Test public void validIdentifierInvalid5() throws Exception { - Check.validIdentifier("[a", 2, ""); + assertThrows(IllegalArgumentException.class, () -> { + Check.validIdentifier("[a", 2, ""); + }); } @Test @@ -131,14 +157,18 @@ public void checkGTZeroGreater() { assertEquals(Check.gt0(120, "test"), 120); } - @Test(expected = IllegalArgumentException.class) + @Test public void checkGTZeroZero() { - Check.gt0(0, "test"); + assertThrows(IllegalArgumentException.class, () -> { + Check.gt0(0, "test"); + }); } - @Test(expected = IllegalArgumentException.class) + @Test public void checkGTZeroLessThanZero() { - Check.gt0(-1, "test"); + assertThrows(IllegalArgumentException.class, () -> { + Check.gt0(-1, "test"); + }); } @Test @@ -147,9 +177,11 @@ public void checkGEZero() { assertEquals(Check.ge0(0, "test"), 0); } - @Test(expected = IllegalArgumentException.class) + @Test public void checkGELessThanZero() { - Check.ge0(-1, "test"); + assertThrows(IllegalArgumentException.class, () -> { + Check.ge0(-1, "test"); + }); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/util/TestConfigurationUtils.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/util/TestConfigurationUtils.java index b868d0b3a2b21..7a86dca32e5eb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/util/TestConfigurationUtils.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/util/TestConfigurationUtils.java @@ -18,15 +18,15 @@ package org.apache.hadoop.lib.util; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.InputStream; import org.apache.hadoop.conf.Configuration; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestConfigurationUtils { diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/wsrs/TestInputStreamEntity.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/wsrs/TestInputStreamEntity.java index 0fa94093064eb..64b4b48c1d4e6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/wsrs/TestInputStreamEntity.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/wsrs/TestInputStreamEntity.java @@ -18,13 +18,13 @@ package org.apache.hadoop.lib.wsrs; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.InputStream; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestInputStreamEntity { diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/wsrs/TestJSONMapProvider.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/wsrs/TestJSONMapProvider.java index 099378032ddef..dee9012e40b14 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/wsrs/TestJSONMapProvider.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/wsrs/TestJSONMapProvider.java @@ -18,15 +18,15 @@ package org.apache.hadoop.lib.wsrs; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.ByteArrayOutputStream; import java.util.Map; import org.json.simple.JSONObject; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestJSONMapProvider { diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/wsrs/TestJSONProvider.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/wsrs/TestJSONProvider.java index 5f747500ed520..f4b346e61c0f2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/wsrs/TestJSONProvider.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/wsrs/TestJSONProvider.java @@ -18,14 +18,14 @@ package org.apache.hadoop.lib.wsrs; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.ByteArrayOutputStream; import org.json.simple.JSONObject; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestJSONProvider { diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/wsrs/TestParam.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/wsrs/TestParam.java index 553ce9e43f2f5..8cadb1c550ea6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/wsrs/TestParam.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/wsrs/TestParam.java @@ -18,12 +18,12 @@ package org.apache.hadoop.lib.wsrs; -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; import java.util.regex.Pattern; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestParam { diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/HFSTestCase.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/HFSTestCase.java index a13a9904ef2aa..470c6d55906be 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/HFSTestCase.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/HFSTestCase.java @@ -17,12 +17,11 @@ */ package org.apache.hadoop.test; -import org.junit.Rule; -import org.junit.rules.MethodRule; +import org.junit.jupiter.api.extension.RegisterExtension; public abstract class HFSTestCase extends HTestCase { - @Rule - public MethodRule hdfsTestHelper = new TestHdfsHelper(); + @RegisterExtension + public TestHdfsHelper hdfsTestHelper = new TestHdfsHelper(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/HTestCase.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/HTestCase.java index 38956994d535c..c2dc194fd0e86 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/HTestCase.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/HTestCase.java @@ -17,13 +17,12 @@ */ package org.apache.hadoop.test; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.fail; import java.text.MessageFormat; import org.apache.hadoop.util.Time; -import org.junit.Rule; -import org.junit.rules.MethodRule; +import org.junit.jupiter.api.extension.RegisterExtension; public abstract class HTestCase { @@ -37,14 +36,14 @@ public abstract class HTestCase { private float waitForRatio = WAITFOR_RATIO_DEFAULT; - @Rule - public MethodRule testDir = new TestDirHelper(); + @RegisterExtension + public TestDirHelper testDir = new TestDirHelper(); - @Rule - public MethodRule jettyTestHelper = new TestJettyHelper(); + @RegisterExtension + public TestJettyHelper jettyTestHelper = new TestJettyHelper(); - @Rule - public MethodRule exceptionHelper = new TestExceptionHelper(); + @RegisterExtension + public TestExceptionHelper exceptionHelper = new TestExceptionHelper(); /** * Sets the 'wait for ratio' used in the {@link #sleep(long)}, diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestDirHelper.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestDirHelper.java index 8eafbf82c6a2a..43d5d0d9a8d49 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestDirHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestDirHelper.java @@ -19,15 +19,16 @@ import java.io.File; import java.io.IOException; +import java.lang.reflect.Method; import java.text.MessageFormat; import java.util.concurrent.atomic.AtomicInteger; -import org.junit.Test; -import org.junit.rules.MethodRule; -import org.junit.runners.model.FrameworkMethod; -import org.junit.runners.model.Statement; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.AfterEachCallback; +import org.junit.jupiter.api.extension.BeforeEachCallback; +import org.junit.jupiter.api.extension.ExtensionContext; -public class TestDirHelper implements MethodRule { +public class TestDirHelper implements BeforeEachCallback, AfterEachCallback { @Test public void dummy() { @@ -91,26 +92,6 @@ private static void delete(File file) throws IOException { private static final ThreadLocal TEST_DIR_TL = new InheritableThreadLocal(); - @Override - public Statement apply(final Statement statement, final FrameworkMethod frameworkMethod, final Object o) { - return new Statement() { - @Override - public void evaluate() throws Throwable { - File testDir = null; - TestDir testDirAnnotation = frameworkMethod.getAnnotation(TestDir.class); - if (testDirAnnotation != null) { - testDir = resetTestCaseDir(frameworkMethod.getName()); - } - try { - TEST_DIR_TL.set(testDir); - statement.evaluate(); - } finally { - TEST_DIR_TL.remove(); - } - } - }; - } - /** * Returns the local test directory for the current test, only available when the * test method has been annotated with {@link TestDir}. @@ -136,7 +117,7 @@ private static File resetTestCaseDir(String testName) { delete(dir); } catch (IOException ex) { throw new RuntimeException(MessageFormat.format("Could not delete test dir[{0}], {1}", - dir, ex.getMessage()), ex); + dir, ex.getMessage()), ex); } if (!dir.mkdirs()) { throw new RuntimeException(MessageFormat.format("Could not create test dir[{0}]", dir)); @@ -144,4 +125,19 @@ private static File resetTestCaseDir(String testName) { return dir; } + @Override + public void beforeEach(ExtensionContext context) throws Exception { + Method testMethod = context.getRequiredTestMethod(); + File testDir = null; + TestDir testDirAnnotation = testMethod.getAnnotation(TestDir.class); + if (testDirAnnotation != null) { + testDir = resetTestCaseDir(testMethod.getName()); + } + TEST_DIR_TL.set(testDir); + } + + @Override + public void afterEach(ExtensionContext extensionContext) throws Exception { + TEST_DIR_TL.remove(); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestExceptionHelper.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestExceptionHelper.java index e3af6435132fb..66269173b95d1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestExceptionHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestExceptionHelper.java @@ -17,51 +17,48 @@ */ package org.apache.hadoop.test; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.fail; +import java.lang.reflect.Method; import java.util.regex.Pattern; -import org.junit.Test; -import org.junit.rules.MethodRule; -import org.junit.runners.model.FrameworkMethod; -import org.junit.runners.model.Statement; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtensionContext; +import org.junit.jupiter.api.extension.TestExecutionExceptionHandler; -public class TestExceptionHelper implements MethodRule { +public class TestExceptionHelper implements TestExecutionExceptionHandler { @Test public void dummy() { } @Override - public Statement apply(final Statement statement, final FrameworkMethod frameworkMethod, final Object o) { - return new Statement() { - @Override - public void evaluate() throws Throwable { - TestException testExceptionAnnotation = frameworkMethod.getAnnotation(TestException.class); - try { - statement.evaluate(); - if (testExceptionAnnotation != null) { - Class klass = testExceptionAnnotation.exception(); - fail("Expected Exception: " + klass.getSimpleName()); - } - } catch (Throwable ex) { - if (testExceptionAnnotation != null) { - Class klass = testExceptionAnnotation.exception(); - if (klass.isInstance(ex)) { - String regExp = testExceptionAnnotation.msgRegExp(); - Pattern pattern = Pattern.compile(regExp); - if (!pattern.matcher(ex.getMessage()).find()) { - fail("Expected Exception Message pattern: " + regExp + " got message: " + ex.getMessage()); - } - } else { - fail("Expected Exception: " + klass.getSimpleName() + " got: " + ex.getClass().getSimpleName()); - } - } else { - throw ex; + public void handleTestExecutionException(ExtensionContext context, + Throwable cause) throws Throwable { + Method testMethod = context.getRequiredTestMethod(); + TestException testExceptionAnnotation = testMethod.getAnnotation(TestException.class); + try { + if (testExceptionAnnotation != null) { + Class klass = testExceptionAnnotation.exception(); + fail("Expected Exception: " + klass.getSimpleName()); + } + } catch (Throwable ex) { + if (testExceptionAnnotation != null) { + Class klass = testExceptionAnnotation.exception(); + if (klass.isInstance(cause)) { + String regExp = testExceptionAnnotation.msgRegExp(); + Pattern pattern = Pattern.compile(regExp); + if (!pattern.matcher(cause.getMessage()).find()) { + fail("Expected Exception Message pattern: " + regExp + + " got message: " + ex.getMessage()); } + } else { + fail("Expected Exception: " + klass.getSimpleName() + " got: " + + ex.getClass().getSimpleName()); } + } else { + throw ex; } - }; + } } - } diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHFSTestCase.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHFSTestCase.java index 2d09b80576f2e..737940908cf3e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHFSTestCase.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHFSTestCase.java @@ -18,9 +18,6 @@ package org.apache.hadoop.test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; - import java.io.BufferedReader; import java.io.IOException; import java.io.InputStream; @@ -39,34 +36,48 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.util.Time; import org.eclipse.jetty.servlet.ServletContextHandler; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.eclipse.jetty.server.Server; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; + public class TestHFSTestCase extends HFSTestCase { - @Test(expected = IllegalStateException.class) + @Test public void testDirNoAnnotation() throws Exception { - TestDirHelper.getTestDir(); + assertThrows(IllegalStateException.class, () -> { + TestDirHelper.getTestDir(); + }); } - @Test(expected = IllegalStateException.class) + @Test public void testJettyNoAnnotation() throws Exception { - TestJettyHelper.getJettyServer(); + assertThrows(IllegalStateException.class, () -> { + TestJettyHelper.getJettyServer(); + }); } - @Test(expected = IllegalStateException.class) + @Test public void testJettyNoAnnotation2() throws Exception { - TestJettyHelper.getJettyURL(); + assertThrows(IllegalStateException.class, () -> { + TestJettyHelper.getJettyURL(); + }); } - @Test(expected = IllegalStateException.class) + @Test public void testHdfsNoAnnotation() throws Exception { - TestHdfsHelper.getHdfsConf(); + assertThrows(IllegalStateException.class, () -> { + TestHdfsHelper.getHdfsConf(); + }); } - @Test(expected = IllegalStateException.class) + @Test public void testHdfsNoAnnotation2() throws Exception { - TestHdfsHelper.getHdfsTestDir(); + assertThrows(IllegalStateException.class, () -> { + TestHdfsHelper.getHdfsTestDir(); + }); } @Test diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHTestCase.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHTestCase.java index be01285fa8611..7354def8b05bc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHTestCase.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHTestCase.java @@ -18,9 +18,6 @@ package org.apache.hadoop.test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; - import java.io.BufferedReader; import java.io.IOException; import java.io.InputStreamReader; @@ -34,24 +31,34 @@ import org.apache.hadoop.util.Time; import org.eclipse.jetty.servlet.ServletContextHandler; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.eclipse.jetty.server.Server; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; + public class TestHTestCase extends HTestCase { - @Test(expected = IllegalStateException.class) + @Test public void testDirNoAnnotation() throws Exception { - TestDirHelper.getTestDir(); + assertThrows(IllegalStateException.class, () -> { + TestDirHelper.getTestDir(); + }); } - @Test(expected = IllegalStateException.class) + @Test public void testJettyNoAnnotation() throws Exception { - TestJettyHelper.getJettyServer(); + assertThrows(IllegalStateException.class, () -> { + TestJettyHelper.getJettyServer(); + }); } - @Test(expected = IllegalStateException.class) + @Test public void testJettyNoAnnotation2() throws Exception { - TestJettyHelper.getJettyURL(); + assertThrows(IllegalStateException.class, () -> { + TestJettyHelper.getJettyURL(); + }); } @Test diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHdfsHelper.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHdfsHelper.java index a38ea49d586d0..d127b7486ef14 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHdfsHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHdfsHelper.java @@ -18,6 +18,7 @@ package org.apache.hadoop.test; import java.io.File; +import java.lang.reflect.Method; import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.conf.Configuration; @@ -35,9 +36,8 @@ import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode; -import org.junit.Test; -import org.junit.runners.model.FrameworkMethod; -import org.junit.runners.model.Statement; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtensionContext; public class TestHdfsHelper extends TestDirHelper { @@ -54,27 +54,16 @@ public void dummy() { private static final ThreadLocal HDFS_TEST_DIR_TL = new InheritableThreadLocal(); - @Override - public Statement apply(Statement statement, FrameworkMethod frameworkMethod, Object o) { - TestHdfs testHdfsAnnotation = frameworkMethod.getAnnotation(TestHdfs.class); - if (testHdfsAnnotation != null) { - this.statement = new HdfsStatement(statement, frameworkMethod.getName()); - statement = this.statement; - } - return super.apply(statement, frameworkMethod, o); - } - public MiniDFSCluster getMiniDFSCluster() { return statement.getMiniDFSCluster(); } - private static class HdfsStatement extends Statement { - private Statement statement; + private static class HdfsStatement { + private String testName; private MiniDFSCluster miniHdfs = null; - public HdfsStatement(Statement statement, String testName) { - this.statement = statement; + HdfsStatement(String testName) { this.testName = testName; } @@ -82,21 +71,15 @@ public MiniDFSCluster getMiniDFSCluster() { return miniHdfs; } - @Override - public void evaluate() throws Throwable { + + public void evaluate() throws Exception { Configuration conf = HadoopUsersConfTestHelper.getBaseConf(); if (Boolean.parseBoolean(System.getProperty(HADOOP_MINI_HDFS, "true"))) { miniHdfs = startMiniHdfs(conf); conf = miniHdfs.getConfiguration(0); } - try { - HDFS_CONF_TL.set(conf); - HDFS_TEST_DIR_TL.set(resetHdfsTestDir(conf)); - statement.evaluate(); - } finally { - HDFS_CONF_TL.remove(); - HDFS_TEST_DIR_TL.remove(); - } + HDFS_CONF_TL.set(conf); + HDFS_TEST_DIR_TL.set(resetHdfsTestDir(conf)); } private static AtomicInteger counter = new AtomicInteger(); @@ -223,4 +206,22 @@ private static synchronized MiniDFSCluster startMiniHdfs(Configuration conf) thr return MINI_DFS; } + @Override + public void beforeEach(ExtensionContext context) throws Exception { + super.beforeEach(context); + Method testMethod = context.getRequiredTestMethod(); + TestHdfs testHdfsAnnotation = testMethod.getAnnotation(TestHdfs.class); + if (testHdfsAnnotation != null) { + this.statement = new HdfsStatement(testMethod.getName()); + this.statement.evaluate(); + } + } + + @Override + public void afterEach(ExtensionContext extensionContext) throws Exception { + + super.afterEach(extensionContext); + HDFS_CONF_TL.remove(); + HDFS_TEST_DIR_TL.remove(); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestJettyHelper.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestJettyHelper.java index e4df118435684..14028914d91f5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestJettyHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestJettyHelper.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.test; +import java.lang.reflect.Method; import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.MalformedURLException; @@ -34,11 +35,11 @@ import org.eclipse.jetty.server.ServerConnector; import org.eclipse.jetty.server.SslConnectionFactory; import org.eclipse.jetty.util.ssl.SslContextFactory; -import org.junit.rules.MethodRule; -import org.junit.runners.model.FrameworkMethod; -import org.junit.runners.model.Statement; +import org.junit.jupiter.api.extension.AfterEachCallback; +import org.junit.jupiter.api.extension.BeforeEachCallback; +import org.junit.jupiter.api.extension.ExtensionContext; -public class TestJettyHelper implements MethodRule { +public class TestJettyHelper implements BeforeEachCallback, AfterEachCallback { private boolean ssl; private String keyStoreType; private String keyStore; @@ -60,32 +61,6 @@ public TestJettyHelper(String keyStoreType, String keyStore, private static final ThreadLocal TEST_JETTY_TL = new InheritableThreadLocal(); - @Override - public Statement apply(final Statement statement, final FrameworkMethod frameworkMethod, final Object o) { - return new Statement() { - @Override - public void evaluate() throws Throwable { - TestJetty testJetty = frameworkMethod.getAnnotation(TestJetty.class); - if (testJetty != null) { - server = createJettyServer(); - } - try { - TEST_JETTY_TL.set(TestJettyHelper.this); - statement.evaluate(); - } finally { - TEST_JETTY_TL.remove(); - if (server != null && server.isRunning()) { - try { - server.stop(); - } catch (Exception ex) { - throw new RuntimeException("Could not stop embedded servlet container, " + ex.getMessage(), ex); - } - } - } - } - }; - } - private Server createJettyServer() { try { InetAddress localhost = InetAddress.getByName("localhost"); @@ -177,4 +152,26 @@ public static URL getJettyURL() { } } + @Override + public void afterEach(ExtensionContext context) throws Exception { + TEST_JETTY_TL.remove(); + if (server != null && server.isRunning()) { + try { + server.stop(); + } catch (Exception ex) { + throw new RuntimeException("Could not stop embedded servlet container, " + + ex.getMessage(), ex); + } + } + } + + @Override + public void beforeEach(ExtensionContext context) throws Exception { + Method testMethod = context.getRequiredTestMethod(); + TestJetty testJetty = testMethod.getAnnotation(TestJetty.class); + if (testJetty != null) { + server = createJettyServer(); + } + TEST_JETTY_TL.set(TestJettyHelper.this); + } }