diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRemove.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRemove.java index 7630dd650cebc..71af5e99a621f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRemove.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRemove.java @@ -16,8 +16,8 @@ * limitations under the License. */ package org.apache.hadoop.hdfs; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.DataOutputStream; import java.io.IOException; @@ -28,7 +28,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestDFSRemove { final Path dir = new Path("/test/remove/"); @@ -80,8 +80,8 @@ public void testRemove() throws Exception { Thread.sleep(3 * DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT * 1000); // all blocks should be gone now. long dfsUsedFinal = getTotalDfsUsed(cluster); - assertEquals("All blocks should be gone. start=" + dfsUsedStart - + " max=" + dfsUsedMax + " final=" + dfsUsedFinal, dfsUsedStart, dfsUsedFinal); + assertEquals(dfsUsedStart, dfsUsedFinal, "All blocks should be gone. start=" + dfsUsedStart + + " max=" + dfsUsedMax + " final=" + dfsUsedFinal); } fs.delete(dir, true); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRename.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRename.java index b16f0237b1e45..973c01c61c300 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRename.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRename.java @@ -16,9 +16,9 @@ * limitations under the License. */ package org.apache.hadoop.hdfs; -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.DataOutputStream; import java.io.IOException; @@ -34,7 +34,8 @@ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter; import org.apache.hadoop.test.GenericTestUtils; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; public class TestDFSRename { static int countLease(MiniDFSCluster cluster) { @@ -136,7 +137,8 @@ public void testRename() throws Exception { * Check the blocks of dst file are cleaned after rename with overwrite * Restart NN to check the rename successfully */ - @Test(timeout = 120000) + @Test + @Timeout(value = 120) public void testRenameWithOverwrite() throws Exception { final short replFactor = 2; final long blockSize = 512; @@ -194,9 +196,9 @@ public void testRename2Options() throws Exception { dfs.rename(path, new Path("/dir1"), new Rename[] {Rename.OVERWRITE, Rename.TO_TRASH}); String auditOut = auditLog.getOutput(); - assertTrue("Rename should have both OVERWRITE and TO_TRASH " - + "flags at namenode but had only " + auditOut, - auditOut.contains("options=[OVERWRITE, TO_TRASH]")); + assertTrue(auditOut.contains("options=[OVERWRITE, TO_TRASH]"), + "Rename should have both OVERWRITE and TO_TRASH " + + "flags at namenode but had only " + auditOut); } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java index 7fc6cd9cfafda..c42fb0fb4e077 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java @@ -19,7 +19,10 @@ import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.DATA_NODE; import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.NAME_NODE; -import static org.junit.Assert.*; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import java.io.File; import java.io.IOException; @@ -39,8 +42,8 @@ import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil; import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.util.StringUtils; -import org.junit.After; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; /** @@ -80,8 +83,7 @@ void checkResult(NodeType nodeType, String[] baseDirs) throws Exception { FSImageTestUtil.assertReasonableNameCurrentDir(curDir); break; case DATA_NODE: - assertEquals( - UpgradeUtilities.checksumContents(nodeType, curDir, false), + assertEquals(UpgradeUtilities.checksumContents(nodeType, curDir, false), UpgradeUtilities.checksumMasterDataNodeContents()); break; } @@ -127,8 +129,8 @@ void startNameNodeShouldFail(String searchString) { void startBlockPoolShouldFail(StartupOption operation, String bpid) throws IOException { cluster.startDataNodes(conf, 1, false, operation, null); // should fail - assertFalse("Block pool " + bpid + " should have failed to start", - cluster.getDataNodes().get(0).isBPServiceAlive(bpid)); + assertFalse(cluster.getDataNodes().get(0).isBPServiceAlive(bpid), + "Block pool " + bpid + " should have failed to start"); } /** @@ -344,7 +346,7 @@ private void deleteMatchingFiles(File[] baseDirs, String regex) { } } - @After + @AfterEach public void tearDown() throws Exception { LOG.info("Shutting down MiniDFSCluster"); if (cluster != null) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java index fd279e6c29975..2598dd44a374f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java @@ -29,7 +29,6 @@ import java.util.Map; import java.util.Random; import java.util.Scanner; -import java.util.TimeZone; import java.util.concurrent.atomic.AtomicInteger; import java.util.zip.DeflaterOutputStream; import java.util.zip.GZIPOutputStream; @@ -42,7 +41,8 @@ import org.apache.hadoop.util.Lists; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.*; import org.apache.hadoop.fs.permission.AclEntry; @@ -70,11 +70,8 @@ import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.Time; import org.apache.hadoop.util.ToolRunner; -import org.junit.rules.Timeout; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Assert; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; import org.slf4j.event.Level; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY; @@ -83,14 +80,19 @@ import static org.apache.hadoop.fs.permission.AclEntryType.*; import static org.apache.hadoop.fs.permission.FsAction.*; import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.aclEntry; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.CoreMatchers.not; -import static org.junit.Assert.*; -import static org.hamcrest.core.StringContains.containsString; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; /** * This class tests commands from DFSShell. */ +@Timeout(30) public class TestDFSShell { private static final Logger LOG = LoggerFactory.getLogger(TestDFSShell.class); private static final AtomicInteger counter = new AtomicInteger(); @@ -110,7 +112,7 @@ public class TestDFSShell { private static MiniDFSCluster miniCluster; private static DistributedFileSystem dfs; - @BeforeClass + @BeforeAll public static void setup() throws IOException { final Configuration conf = new Configuration(); conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true); @@ -128,16 +130,13 @@ public static void setup() throws IOException { dfs = miniCluster.getFileSystem(); } - @AfterClass + @AfterAll public static void tearDown() { if (miniCluster != null) { miniCluster.shutdown(true, true); } } - @Rule - public Timeout globalTimeout= new Timeout(30 * 1000); // 30s - static Path writeFile(FileSystem fs, Path f) throws IOException { DataOutputStream out = fs.create(f); out.writeBytes("dhruba: " + f); @@ -194,7 +193,8 @@ static void show(String s) { System.out.println(Thread.currentThread().getStackTrace()[2] + " " + s); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testZeroSizeFile() throws IOException { //create a zero size file final File f1 = new File(TEST_ROOT_DIR, "f1"); @@ -225,7 +225,8 @@ public void testZeroSizeFile() throws IOException { f2.delete(); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testRecursiveRm() throws IOException { final Path parent = new Path("/testRecursiveRm", "parent"); final Path child = new Path(parent, "child"); @@ -240,7 +241,8 @@ public void testRecursiveRm() throws IOException { assertFalse(dfs.exists(parent)); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testDu() throws IOException { int replication = 2; PrintStream psBackup = System.out; @@ -278,18 +280,18 @@ public void testDu() throws IOException { String returnString = out.toString(); out.reset(); // Check if size matches as expected - assertThat(returnString, containsString(myFileLength.toString())); - assertThat(returnString, containsString(myFileDiskUsed.toString())); - assertThat(returnString, containsString(myFile2Length.toString())); - assertThat(returnString, containsString(myFile2DiskUsed.toString())); + assertThat(returnString).contains(myFileLength.toString()); + assertThat(returnString).contains(myFileDiskUsed.toString()); + assertThat(returnString).contains(myFile2Length.toString()); + assertThat(returnString).contains(myFile2DiskUsed.toString()); // Check that -du -s reports the state of the snapshot String snapshotName = "ss1"; Path snapshotPath = new Path(myPath, ".snapshot/" + snapshotName); dfs.allowSnapshot(myPath); - assertThat(dfs.createSnapshot(myPath, snapshotName), is(snapshotPath)); - assertThat(dfs.delete(myFile, false), is(true)); - assertThat(dfs.exists(myFile), is(false)); + assertThat(dfs.createSnapshot(myPath, snapshotName)).isEqualTo(snapshotPath); + assertThat(dfs.delete(myFile, false)).isEqualTo(true); + assertThat(dfs.exists(myFile)).isEqualTo(false); args = new String[3]; args[0] = "-du"; @@ -302,13 +304,13 @@ public void testDu() throws IOException { System.err.println("Exception raised from DFSShell.run " + e.getLocalizedMessage()); } - assertThat(val, is(0)); + assertThat(val).isEqualTo(0); returnString = out.toString(); out.reset(); Long combinedLength = myFileLength + myFile2Length; Long combinedDiskUsed = myFileDiskUsed + myFile2DiskUsed; - assertThat(returnString, containsString(combinedLength.toString())); - assertThat(returnString, containsString(combinedDiskUsed.toString())); + assertThat(returnString).contains(combinedLength.toString()); + assertThat(returnString).contains(combinedDiskUsed.toString()); // Check if output is rendered properly with multiple input paths final Path myFile3 = new Path(myPath, "file3"); @@ -325,7 +327,7 @@ public void testDu() throws IOException { System.err.println("Exception raised from DFSShell.run " + e.getLocalizedMessage()); } - assertEquals("Return code should be 0.", 0, val); + assertEquals(0, val, "Return code should be 0."); returnString = out.toString(); out.reset(); assertTrue(returnString.contains("1 2 " + myFile3.toString())); @@ -335,7 +337,8 @@ public void testDu() throws IOException { } } - @Test (timeout = 180000) + @Test + @Timeout(value = 180) public void testDuSnapshots() throws IOException { final int replication = 2; final PrintStream psBackup = System.out; @@ -387,7 +390,7 @@ public void testDuSnapshots() throws IOException { final String snapshotName = "ss1"; final Path snapshotPath = new Path(parent, ".snapshot/" + snapshotName); dfs.allowSnapshot(parent); - assertThat(dfs.createSnapshot(parent, snapshotName), is(snapshotPath)); + assertThat(dfs.createSnapshot(parent, snapshotName)).isEqualTo(snapshotPath); rmr(dfs, file); final Path newFile = new Path(dir, "newfile"); writeFile(dfs, newFile); @@ -462,7 +465,8 @@ public void testDuSnapshots() throws IOException { } } - @Test (timeout = 180000) + @Test + @Timeout(value = 180) public void testCountSnapshots() throws IOException { final PrintStream psBackup = System.out; final ByteArrayOutputStream out = new ByteArrayOutputStream(); @@ -497,7 +501,7 @@ public void testCountSnapshots() throws IOException { final String snapshotName = "s1"; final Path snapshotPath = new Path(parent, ".snapshot/" + snapshotName); dfs.allowSnapshot(parent); - assertThat(dfs.createSnapshot(parent, snapshotName), is(snapshotPath)); + assertThat(dfs.createSnapshot(parent, snapshotName)).isEqualTo(snapshotPath); rmr(dfs, file); rmr(dfs, dir2); final Path newFile = new Path(dir, "new file"); @@ -546,7 +550,8 @@ public void testCountSnapshots() throws IOException { } } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testPut() throws IOException { // remove left over crc files: new File(TEST_ROOT_DIR, ".f1.crc").delete(); @@ -561,10 +566,10 @@ public void testPut() throws IOException { try (FSDataOutputStream out = dfs.create(dst, false)) { // It should fail to create a new client writing to the same file. try(DFSClient client = new DFSClient(dfs.getUri(), dfs.getConf())) { - final RemoteException e = Assert.assertThrows(RemoteException.class, + final RemoteException e = assertThrows(RemoteException.class, () -> client.create(dst.toString(), false)); LOG.info("GOOD", e); - Assert.assertEquals(e.getClassName(), AlreadyBeingCreatedException.class.getName()); + assertEquals(e.getClassName(), AlreadyBeingCreatedException.class.getName()); } // It should succeed to continue writing to the file. out.writeUTF(hello); @@ -604,7 +609,8 @@ public void testPut() throws IOException { } /** check command error outputs and exit statuses. */ - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testErrOutPut() throws Exception { PrintStream bak = null; try { @@ -617,74 +623,74 @@ public void testErrOutPut() throws Exception { argv[0] = "-cat"; argv[1] = root.toUri().getPath(); int ret = ToolRunner.run(new FsShell(), argv); - assertEquals(" -cat returned 1 ", 1, ret); + assertEquals(1, ret, " -cat returned 1 "); String returned = out.toString(); - assertTrue("cat does not print exceptions ", - (returned.lastIndexOf("Exception") == -1)); + assertTrue((returned.lastIndexOf("Exception") == -1), + "cat does not print exceptions "); out.reset(); argv[0] = "-rm"; argv[1] = root.toString(); FsShell shell = new FsShell(dfs.getConf()); ret = ToolRunner.run(shell, argv); - assertEquals(" -rm returned 1 ", 1, ret); + assertEquals(1, ret, " -rm returned 1 "); returned = out.toString(); out.reset(); - assertTrue("rm prints reasonable error ", - (returned.lastIndexOf("No such file or directory") != -1)); + assertTrue((returned.lastIndexOf("No such file or directory") != -1), + "rm prints reasonable error "); argv[0] = "-rmr"; argv[1] = root.toString(); ret = ToolRunner.run(shell, argv); - assertEquals(" -rmr returned 1", 1, ret); + assertEquals(1, ret, " -rmr returned 1"); returned = out.toString(); - assertTrue("rmr prints reasonable error ", - (returned.lastIndexOf("No such file or directory") != -1)); + assertTrue((returned.lastIndexOf("No such file or directory") != -1), + "rmr prints reasonable error "); out.reset(); argv[0] = "-du"; argv[1] = "/nonexistentfile"; ret = ToolRunner.run(shell, argv); returned = out.toString(); - assertTrue(" -du prints reasonable error ", - (returned.lastIndexOf("No such file or directory") != -1)); + assertTrue((returned.lastIndexOf("No such file or directory") != -1), + " -du prints reasonable error "); out.reset(); argv[0] = "-dus"; argv[1] = "/nonexistentfile"; ret = ToolRunner.run(shell, argv); returned = out.toString(); - assertTrue(" -dus prints reasonable error", - (returned.lastIndexOf("No such file or directory") != -1)); + assertTrue((returned.lastIndexOf("No such file or directory") != -1), + " -dus prints reasonable error"); out.reset(); argv[0] = "-ls"; argv[1] = "/nonexistenfile"; ret = ToolRunner.run(shell, argv); returned = out.toString(); - assertTrue(" -ls does not return Found 0 items", - (returned.lastIndexOf("Found 0") == -1)); + assertTrue((returned.lastIndexOf("Found 0") == -1), + " -ls does not return Found 0 items"); out.reset(); argv[0] = "-ls"; argv[1] = "/nonexistentfile"; ret = ToolRunner.run(shell, argv); - assertEquals(" -lsr should fail ", 1, ret); + assertEquals(1, ret, " -lsr should fail "); out.reset(); dfs.mkdirs(new Path("/testdir")); argv[0] = "-ls"; argv[1] = "/testdir"; ret = ToolRunner.run(shell, argv); returned = out.toString(); - assertTrue(" -ls does not print out anything ", - (returned.lastIndexOf("Found 0") == -1)); + assertTrue((returned.lastIndexOf("Found 0") == -1), + " -ls does not print out anything "); out.reset(); argv[0] = "-ls"; argv[1] = "/user/nonxistant/*"; ret = ToolRunner.run(shell, argv); - assertEquals(" -ls on nonexistent glob returns 1", 1, ret); + assertEquals(1, ret, " -ls on nonexistent glob returns 1"); out.reset(); argv[0] = "-mkdir"; argv[1] = "/testdir"; ret = ToolRunner.run(shell, argv); returned = out.toString(); - assertEquals(" -mkdir returned 1 ", 1, ret); - assertTrue(" -mkdir returned File exists", - (returned.lastIndexOf("File exists") != -1)); + assertEquals(1, ret, " -mkdir returned 1 "); + assertTrue((returned.lastIndexOf("File exists") != -1), + " -mkdir returned File exists"); Path testFile = new Path("/testfile"); OutputStream outtmp = dfs.create(testFile); outtmp.write(testFile.toString().getBytes()); @@ -694,24 +700,24 @@ public void testErrOutPut() throws Exception { argv[1] = "/testfile"; ret = ToolRunner.run(shell, argv); returned = out.toString(); - assertEquals(" -mkdir returned 1", 1, ret); - assertTrue(" -mkdir returned this is a file ", - (returned.lastIndexOf("not a directory") != -1)); + assertEquals(1, ret, " -mkdir returned 1"); + assertTrue((returned.lastIndexOf("not a directory") != -1), + " -mkdir returned this is a file "); out.reset(); argv[0] = "-mkdir"; argv[1] = "/testParent/testChild"; ret = ToolRunner.run(shell, argv); returned = out.toString(); - assertEquals(" -mkdir returned 1", 1, ret); - assertTrue(" -mkdir returned there is No file or directory but has testChild in the path", - (returned.lastIndexOf("testChild") == -1)); + assertEquals(1, ret, " -mkdir returned 1"); + assertTrue((returned.lastIndexOf("testChild") == -1), + " -mkdir returned there is No file or directory but has testChild in the path"); out.reset(); argv = new String[3]; argv[0] = "-mv"; argv[1] = "/testfile"; argv[2] = "/no-such-dir/file"; ret = ToolRunner.run(shell, argv); - assertEquals("mv failed to rename", 1, ret); + assertEquals(1, ret, "mv failed to rename"); out.reset(); argv = new String[3]; argv[0] = "-mv"; @@ -719,25 +725,22 @@ public void testErrOutPut() throws Exception { argv[2] = "/testfiletest"; ret = ToolRunner.run(shell, argv); returned = out.toString(); - assertTrue("no output from rename", - (returned.lastIndexOf("Renamed") == -1)); + assertTrue((returned.lastIndexOf("Renamed") == -1), "no output from rename"); out.reset(); argv[0] = "-mv"; argv[1] = "/testfile"; argv[2] = "/testfiletmp"; ret = ToolRunner.run(shell, argv); returned = out.toString(); - assertTrue(" unix like output", - (returned.lastIndexOf("No such file or") != -1)); + assertTrue((returned.lastIndexOf("No such file or") != -1), " unix like output"); out.reset(); argv = new String[1]; argv[0] = "-du"; dfs.mkdirs(dfs.getHomeDirectory()); ret = ToolRunner.run(shell, argv); returned = out.toString(); - assertEquals(" no error ", 0, ret); - assertTrue("empty path specified", - (returned.lastIndexOf("empty string") == -1)); + assertEquals(0, ret, " no error "); + assertTrue((returned.lastIndexOf("empty string") == -1), "empty path specified"); out.reset(); argv = new String[3]; argv[0] = "-test"; @@ -745,7 +748,7 @@ public void testErrOutPut() throws Exception { argv[2] = "/no/such/dir"; ret = ToolRunner.run(shell, argv); returned = out.toString(); - assertEquals(" -test -d wrong result ", 1, ret); + assertEquals(1, ret, " -test -d wrong result "); assertTrue(returned.isEmpty()); } finally { if (bak != null) { @@ -778,7 +781,7 @@ public void testMoveWithTargetPortEmpty() throws Exception { argv[1] = srcFs.getUri() + "/testfile"; argv[2] = "hdfs://" + srcFs.getUri().getHost() + "/testfile2"; int ret = ToolRunner.run(shell, argv); - assertEquals("mv should have succeeded", 0, ret); + assertEquals(0, ret, "mv should have succeeded"); } finally { if (cluster != null) { cluster.shutdown(); @@ -786,7 +789,8 @@ public void testMoveWithTargetPortEmpty() throws Exception { } } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testURIPaths() throws Exception { Configuration srcConf = new HdfsConfiguration(); Configuration dstConf = new HdfsConfiguration(); @@ -807,19 +811,19 @@ public void testURIPaths() throws Exception { argv[0] = "-ls"; argv[1] = dstFs.getUri().toString() + "/"; int ret = ToolRunner.run(shell, argv); - assertEquals("ls works on remote uri ", 0, ret); + assertEquals(0, ret, "ls works on remote uri "); //check for rm -r dstFs.mkdirs(new Path("/hadoopdir")); argv = new String[2]; argv[0] = "-rmr"; argv[1] = dstFs.getUri().toString() + "/hadoopdir"; ret = ToolRunner.run(shell, argv); - assertEquals("-rmr works on remote uri " + argv[1], 0, ret); + assertEquals(0, ret, "-rmr works on remote uri " + argv[1]); //check du argv[0] = "-du"; argv[1] = dstFs.getUri().toString() + "/"; ret = ToolRunner.run(shell, argv); - assertEquals("du works on remote uri ", 0, ret); + assertEquals(0, ret, "du works on remote uri "); //check put File furi = new File(TEST_ROOT_DIR, "furi"); createLocalFile(furi); @@ -828,20 +832,20 @@ public void testURIPaths() throws Exception { argv[1] = furi.toURI().toString(); argv[2] = dstFs.getUri().toString() + "/furi"; ret = ToolRunner.run(shell, argv); - assertEquals(" put is working ", 0, ret); + assertEquals(0, ret, " put is working "); //check cp argv[0] = "-cp"; argv[1] = dstFs.getUri().toString() + "/furi"; argv[2] = srcFs.getUri().toString() + "/furi"; ret = ToolRunner.run(shell, argv); - assertEquals(" cp is working ", 0, ret); + assertEquals(0, ret, " cp is working "); assertTrue(srcFs.exists(new Path("/furi"))); //check cat argv = new String[2]; argv[0] = "-cat"; argv[1] = dstFs.getUri().toString() + "/furi"; ret = ToolRunner.run(shell, argv); - assertEquals(" cat is working ", 0, ret); + assertEquals(0, ret, " cat is working "); //check chown dstFs.delete(new Path("/furi"), true); dstFs.delete(new Path("/hadoopdir"), true); @@ -858,15 +862,15 @@ public void testURIPaths() throws Exception { argv[0] = "-cat"; argv[1] = "hdfs:///furi"; ret = ToolRunner.run(shell, argv); - assertEquals(" default works for cat", 0, ret); + assertEquals(0, ret, " default works for cat"); argv[0] = "-ls"; argv[1] = "hdfs:///"; ret = ToolRunner.run(shell, argv); - assertEquals("default works for ls ", 0, ret); + assertEquals(0, ret, "default works for ls "); argv[0] = "-rmr"; argv[1] = "hdfs:///furi"; ret = ToolRunner.run(shell, argv); - assertEquals("default works for rm/rmr", 0, ret); + assertEquals(0, ret, "default works for rm/rmr"); } finally { if (null != srcCluster) { srcCluster.shutdown(); @@ -880,7 +884,8 @@ public void testURIPaths() throws Exception { /** * Test that -head displays first kilobyte of the file to stdout. */ - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testHead() throws Exception { final int fileLen = 5 * BLOCK_SIZE; @@ -895,19 +900,20 @@ public void testHead() throws Exception { final String[] argv = new String[]{"-head", testFile.toString()}; final int ret = ToolRunner.run(new FsShell(dfs.getConf()), argv); - assertEquals(Arrays.toString(argv) + " returned " + ret, 0, ret); - assertEquals("-head returned " + out.size() + " bytes data, expected 1KB", - 1024, out.size()); + assertEquals(0, ret, Arrays.toString(argv) + " returned " + ret); + assertEquals(1024, out.size(), + "-head returned " + out.size() + " bytes data, expected 1KB"); // tailed out last 1KB of the file content - assertArrayEquals("Head output doesn't match input", - text.substring(0, 1024).getBytes(), out.toByteArray()); + assertArrayEquals(text.substring(0, 1024).getBytes(), out.toByteArray(), + "Head output doesn't match input"); out.reset(); } /** * Test that -tail displays last kilobyte of the file to stdout. */ - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testTail() throws Exception { final int fileLen = 5 * BLOCK_SIZE; @@ -922,19 +928,20 @@ public void testTail() throws Exception { final String[] argv = new String[]{"-tail", testFile.toString()}; final int ret = ToolRunner.run(new FsShell(dfs.getConf()), argv); - assertEquals(Arrays.toString(argv) + " returned " + ret, 0, ret); - assertEquals("-tail returned " + out.size() + " bytes data, expected 1KB", - 1024, out.size()); + assertEquals(0, ret, Arrays.toString(argv) + " returned " + ret); + assertEquals(1024, out.size(), "-tail returned " + out.size() + + " bytes data, expected 1KB"); // tailed out last 1KB of the file content - assertArrayEquals("Tail output doesn't match input", - text.substring(fileLen - 1024).getBytes(), out.toByteArray()); + assertArrayEquals(text.substring(fileLen - 1024).getBytes(), out.toByteArray(), + "Tail output doesn't match input"); out.reset(); } /** * Test that -tail -f outputs appended data as the file grows. */ - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testTailWithFresh() throws Exception { final Path testFile = new Path("testTailWithFresh", "file1"); dfs.create(testFile); @@ -977,7 +984,8 @@ public Boolean get() { }, 100, 10000); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testText() throws Exception { final Configuration conf = dfs.getConf(); textTest(new Path("/texttest").makeQualified(dfs.getUri(), @@ -1014,9 +1022,9 @@ private void textTest(Path root, Configuration conf) throws Exception { argv[0] = "-text"; argv[1] = new Path(root, "file.gz").toString(); int ret = ToolRunner.run(new FsShell(conf), argv); - assertEquals("'-text " + argv[1] + " returned " + ret, 0, ret); - assertTrue("Output doesn't match input", - Arrays.equals(file.toByteArray(), out.toByteArray())); + assertEquals(0, ret, "'-text " + argv[1] + " returned " + ret); + assertTrue(Arrays.equals(file.toByteArray(), out.toByteArray()), + "Output doesn't match input"); // Create a sequence file with a gz extension, to test proper // container detection. Magic detection. @@ -1033,9 +1041,9 @@ private void textTest(Path root, Configuration conf) throws Exception { argv[0] = "-text"; argv[1] = new Path(root, "file.gz").toString(); ret = ToolRunner.run(new FsShell(conf), argv); - assertEquals("'-text " + argv[1] + " returned " + ret, 0, ret); - assertTrue("Output doesn't match input", - Arrays.equals("Foo\tBar\n".getBytes(), out.toByteArray())); + assertEquals(0, ret, "'-text " + argv[1] + " returned " + ret); + assertTrue(Arrays.equals("Foo\tBar\n".getBytes(), out.toByteArray()), + "Output doesn't match input"); out.reset(); // Test deflate. Extension-based detection. @@ -1050,9 +1058,9 @@ private void textTest(Path root, Configuration conf) throws Exception { argv[0] = "-text"; argv[1] = new Path(root, "file.deflate").toString(); ret = ToolRunner.run(new FsShell(conf), argv); - assertEquals("'-text " + argv[1] + " returned " + ret, 0, ret); - assertTrue("Output doesn't match input", - Arrays.equals(outbytes, out.toByteArray())); + assertEquals(0, ret, "'-text " + argv[1] + " returned " + ret); + assertTrue(Arrays.equals(outbytes, out.toByteArray()), + "Output doesn't match input"); out.reset(); // Test a simple codec. Extension based detection. We use @@ -1071,9 +1079,9 @@ private void textTest(Path root, Configuration conf) throws Exception { argv[0] = "-text"; argv[1] = new Path(root, p).toString(); ret = ToolRunner.run(new FsShell(conf), argv); - assertEquals("'-text " + argv[1] + " returned " + ret, 0, ret); - assertTrue("Output doesn't match input", - Arrays.equals(writebytes, out.toByteArray())); + assertEquals(0, ret, "'-text " + argv[1] + " returned " + ret); + assertTrue(Arrays.equals(writebytes, out.toByteArray()), + "Output doesn't match input"); out.reset(); // Test a plain text. @@ -1087,9 +1095,9 @@ private void textTest(Path root, Configuration conf) throws Exception { argv[0] = "-text"; argv[1] = new Path(root, "file.txt").toString(); ret = ToolRunner.run(new FsShell(conf), argv); - assertEquals("'-text " + argv[1] + " returned " + ret, 0, ret); - assertTrue("Output doesn't match input", - Arrays.equals(writebytes, out.toByteArray())); + assertEquals(0, ret, "'-text " + argv[1] + " returned " + ret); + assertTrue(Arrays.equals(writebytes, out.toByteArray()), + "Output doesn't match input"); out.reset(); } finally { if (null != bak) { @@ -1098,7 +1106,8 @@ private void textTest(Path root, Configuration conf) throws Exception { } } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testChecksum() throws Exception { PrintStream printStream = System.out; try { @@ -1112,18 +1121,18 @@ public void testChecksum() throws Exception { String[] args = {"-checksum", "-v", filePath.toString()}; assertEquals(0, shell.run(args)); // verify block size is printed in the output - assertTrue(out.toString() - .contains(String.format("BlockSize=%s", fileStatus.getBlockSize()))); + assertTrue(out.toString().contains(String.format("BlockSize=%s", fileStatus.getBlockSize()))); // verify checksum is printed in the output - assertTrue(out.toString().contains(StringUtils - .byteToHexString(checksum.getBytes(), 0, checksum.getLength()))); + assertTrue(out.toString().contains(StringUtils.byteToHexString(checksum.getBytes(), + 0, checksum.getLength()))); } finally { - Assert.assertNotNull(printStream); + assertNotNull(printStream); System.setOut(printStream); } } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testCopyToLocal() throws IOException { FsShell shell = new FsShell(dfs.getConf()); @@ -1143,22 +1152,22 @@ public void testCopyToLocal() throws IOException { File localroot2 = new File(TEST_ROOT_DIR, "copyToLocal2"); File f1 = new File(localroot, "f1"); - assertTrue("Copying failed.", f1.isFile()); + assertTrue(f1.isFile(), "Copying failed."); File f2 = new File(localroot, "f2"); - assertTrue("Copying failed.", f2.isFile()); + assertTrue(f2.isFile(), "Copying failed."); File sub = new File(localroot, "sub"); - assertTrue("Copying failed.", sub.isDirectory()); + assertTrue(sub.isDirectory(), "Copying failed."); File f3 = new File(sub, "f3"); - assertTrue("Copying failed.", f3.isFile()); + assertTrue(f3.isFile(), "Copying failed."); File f4 = new File(sub, "f4"); - assertTrue("Copying failed.", f4.isFile()); + assertTrue(f4.isFile(), "Copying failed."); File f5 = new File(localroot2, "f1"); - assertTrue("Copying failed.", f5.isFile()); + assertTrue(f5.isFile(), "Copying failed."); f1.delete(); f2.delete(); @@ -1206,7 +1215,8 @@ static String createTree(FileSystem fs, String name) throws IOException { return path; } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testCount() throws Exception { FsShell shell = new FsShell(dfs.getConf()); @@ -1230,7 +1240,8 @@ public void testCount() throws Exception { assertEquals(0, runCmd(shell, "-count", root, localstr)); } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testTotalSizeOfAllFiles() throws Exception { final Path root = new Path("/testTotalSizeOfAllFiles"); dfs.mkdirs(root); @@ -1385,7 +1396,8 @@ private void confirmOwner(String owner, String group, } } - @Test (timeout = 60000) + @Test + @Timeout(value = 60) public void testFilePermissions() throws IOException { Configuration conf = new HdfsConfiguration(); @@ -1452,7 +1464,9 @@ public void testFilePermissions() throws IOException { /** * Tests various options of DFSShell. */ - @Test (timeout = 120000) + @SuppressWarnings("checkstyle:MethodLength") + @Test + @Timeout(value = 120) public void testDFSShell() throws Exception { /* This tests some properties of ChecksumFileSystem as well. * Make sure that we create ChecksumDFS */ @@ -1899,7 +1913,8 @@ static interface TestGetRunner { String run(int exitcode, String... options) throws IOException; } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testRemoteException() throws Exception { UserGroupInformation tmpUGI = UserGroupInformation.createUserForTesting("tmpname", new String[] {"mygroup"}); @@ -1921,10 +1936,10 @@ public Object run() throws Exception { args[0] = "-ls"; args[1] = "/foo"; int ret = ToolRunner.run(fshell, args); - assertEquals("returned should be 1", 1, ret); + assertEquals(1, ret, "returned should be 1"); String str = out.toString(); - assertTrue("permission denied printed", - str.indexOf("Permission denied") != -1); + assertTrue( + str.indexOf("Permission denied") != -1, "permission denied printed"); out.reset(); return null; } @@ -1936,7 +1951,8 @@ public Object run() throws Exception { } } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testGet() throws IOException { GenericTestUtils.setLogLevel(FSInputChecker.LOG, Level.TRACE); @@ -1947,28 +1963,28 @@ public void testGet() throws IOException { // Set short retry timeouts so this test runs faster conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 10); TestGetRunner runner = new TestGetRunner() { - private int count = 0; - private final FsShell shell = new FsShell(conf); + private int count = 0; + private final FsShell shell = new FsShell(conf); - public String run(int exitcode, String... options) throws IOException { - String dst = new File(TEST_ROOT_DIR, fname + ++count) + public String run(int exitcode, String... options) throws IOException { + String dst = new File(TEST_ROOT_DIR, fname + ++count) .getAbsolutePath(); - String[] args = new String[options.length + 3]; - args[0] = "-get"; - args[args.length - 2] = remotef.toString(); - args[args.length - 1] = dst; - for(int i = 0; i < options.length; i++) { - args[i + 1] = options[i]; - } - show("args=" + Arrays.asList(args)); - - try { - assertEquals(exitcode, shell.run(args)); - } catch (Exception e) { - assertTrue(StringUtils.stringifyException(e), false); - } - return exitcode == 0? DFSTestUtil.readFile(new File(dst)): null; - } + String[] args = new String[options.length + 3]; + args[0] = "-get"; + args[args.length - 2] = remotef.toString(); + args[args.length - 1] = dst; + for (int i = 0; i < options.length; i++) { + args[i + 1] = options[i]; + } + show("args=" + Arrays.asList(args)); + + try { + assertEquals(exitcode, shell.run(args)); + } catch (Exception e) { + assertTrue(false, StringUtils.stringifyException(e)); + } + return exitcode == 0 ? DFSTestUtil.readFile(new File(dst)) : null; + } }; File localf = createLocalFile(new File(TEST_ROOT_DIR, fname)); @@ -1977,7 +1993,7 @@ public String run(int exitcode, String... options) throws IOException { try { cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).format(true) - .build(); + .build(); dfs = cluster.getFileSystem(); mkdir(dfs, root); @@ -2006,13 +2022,13 @@ public String run(int exitcode, String... options) throws IOException { // Start the miniCluster again, but do not reformat, so prior files remain. cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).format(false) - .build(); + .build(); dfs = cluster.getFileSystem(); assertEquals(null, runner.run(1)); String corruptedcontent = runner.run(0, "-ignoreCrc"); assertEquals(localfcontent.substring(1), corruptedcontent.substring(1)); - assertEquals(localfcontent.charAt(0)+1, corruptedcontent.charAt(0)); + assertEquals(localfcontent.charAt(0) + 1, corruptedcontent.charAt(0)); } finally { if (null != dfs) { try { @@ -2031,10 +2047,10 @@ public String run(int exitcode, String... options) throws IOException { * Test -stat [format] ... prints statistics about the file/directory * at in the specified format. */ - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testStat() throws Exception { final SimpleDateFormat fmt = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); - fmt.setTimeZone(TimeZone.getTimeZone("UTC")); final Path testDir1 = new Path("testStat", "dir1"); dfs.mkdirs(testDir1); final Path testFile2 = new Path(testDir1, "file2"); @@ -2055,43 +2071,39 @@ public void testStat() throws Exception { out.reset(); doFsStat(dfs.getConf(), null, testDir1); - assertEquals("Unexpected -stat output: " + out, - out.toString(), String.format("%s%n", mtime1)); + assertEquals(out.toString(), String.format("%s%n", mtime1), + "Unexpected -stat output: " + out); out.reset(); doFsStat(dfs.getConf(), null, testDir1, testFile2); - assertEquals("Unexpected -stat output: " + out, - out.toString(), String.format("%s%n%s%n", mtime1, mtime2)); + assertEquals(out.toString(), String.format("%s%n%s%n", mtime1, mtime2), + "Unexpected -stat output: " + out); doFsStat(dfs.getConf(), "%F %u:%g %b %y %n"); out.reset(); doFsStat(dfs.getConf(), "%F %a %A %u:%g %b %y %n", testDir1); - assertTrue(out.toString(), out.toString().contains(mtime1)); - assertTrue(out.toString(), out.toString().contains("directory")); - assertTrue(out.toString(), out.toString().contains(status1.getGroup())); - assertTrue(out.toString(), - out.toString().contains(status1.getPermission().toString())); + assertTrue(out.toString().contains(mtime1), out.toString()); + assertTrue(out.toString().contains("directory"), out.toString()); + assertTrue(out.toString().contains(status1.getGroup()), out.toString()); + assertTrue(out.toString().contains(status1.getPermission().toString()), out.toString()); int n = status1.getPermission().toShort(); int octal = (n>>>9&1)*1000 + (n>>>6&7)*100 + (n>>>3&7)*10 + (n&7); - assertTrue(out.toString(), - out.toString().contains(String.valueOf(octal))); + assertTrue(out.toString().contains(String.valueOf(octal)), out.toString()); out.reset(); doFsStat(dfs.getConf(), "%F %a %A %u:%g %b %x %y %n", testDir1, testFile2); n = status2.getPermission().toShort(); octal = (n>>>9&1)*1000 + (n>>>6&7)*100 + (n>>>3&7)*10 + (n&7); - assertTrue(out.toString(), out.toString().contains(mtime1)); - assertTrue(out.toString(), out.toString().contains(atime1)); - assertTrue(out.toString(), out.toString().contains("regular file")); - assertTrue(out.toString(), - out.toString().contains(status2.getPermission().toString())); - assertTrue(out.toString(), - out.toString().contains(String.valueOf(octal))); - assertTrue(out.toString(), out.toString().contains(mtime2)); - assertTrue(out.toString(), out.toString().contains(atime2)); + assertTrue(out.toString().contains(mtime1), out.toString()); + assertTrue(out.toString().contains(atime1), out.toString()); + assertTrue(out.toString().contains("regular file"), out.toString()); + assertTrue(out.toString().contains(status2.getPermission().toString()), out.toString()); + assertTrue(out.toString().contains(String.valueOf(octal)), out.toString()); + assertTrue(out.toString().contains(mtime2), out.toString()); + assertTrue(out.toString().contains(atime2), out.toString()); } private static void doFsStat(Configuration conf, String format, Path... files) @@ -2099,8 +2111,8 @@ private static void doFsStat(Configuration conf, String format, Path... files) if (files == null || files.length == 0) { final String[] argv = (format == null ? new String[] {"-stat"} : new String[] {"-stat", format}); - assertEquals("Should have failed with missing arguments", - -1, ToolRunner.run(new FsShell(conf), argv)); + assertEquals(-1, ToolRunner.run(new FsShell(conf), argv), + "Should have failed with missing arguments"); } else { List argv = new LinkedList<>(); argv.add("-stat"); @@ -2112,11 +2124,12 @@ private static void doFsStat(Configuration conf, String format, Path... files) } int ret = ToolRunner.run(new FsShell(conf), argv.toArray(new String[0])); - assertEquals(argv + " returned non-zero status " + ret, 0, ret); + assertEquals(0, ret, argv + " returned non-zero status " + ret); } } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testLsr() throws Exception { final Configuration conf = dfs.getConf(); final String root = createTree(dfs, "lsr"); @@ -2168,18 +2181,20 @@ private static String runLsr(final FsShell shell, String root, int returnvalue * and return -1 exit code. * @throws Exception */ - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testInvalidShell() throws Exception { Configuration conf = new Configuration(); // default FS (non-DFS) DFSAdmin admin = new DFSAdmin(); admin.setConf(conf); int res = admin.run(new String[] {"-refreshNodes"}); - assertEquals("expected to fail -1", res , -1); + assertEquals(res, -1, "expected to fail -1"); } // Preserve Copy Option is -ptopxa (timestamps, ownership, permission, XATTR, // ACLs) - @Test (timeout = 120000) + @Test + @Timeout(value = 120) public void testCopyCommandsWithPreserveOption() throws Exception { FsShell shell = null; final String testdir = "/tmp/TestDFSShell-testCopyCommandsWithPreserveOption-" @@ -2214,7 +2229,7 @@ public void testCopyCommandsWithPreserveOption() throws Exception { String[] argv = new String[] { "-cp", "-p", src.toUri().toString(), target1.toUri().toString() }; int ret = ToolRunner.run(shell, argv); - assertEquals("cp -p is not working", SUCCESS, ret); + assertEquals(SUCCESS, ret, "cp -p is not working"); FileStatus targetStatus = dfs.getFileStatus(target1); assertEquals(mtime, targetStatus.getModificationTime()); assertEquals(atime, targetStatus.getAccessTime()); @@ -2233,7 +2248,7 @@ public void testCopyCommandsWithPreserveOption() throws Exception { argv = new String[] { "-cp", "-ptop", src.toUri().toString(), target2.toUri().toString() }; ret = ToolRunner.run(shell, argv); - assertEquals("cp -ptop is not working", SUCCESS, ret); + assertEquals(SUCCESS, ret, "cp -ptop is not working"); targetStatus = dfs.getFileStatus(target2); assertEquals(mtime, targetStatus.getModificationTime()); assertEquals(atime, targetStatus.getAccessTime()); @@ -2252,7 +2267,7 @@ public void testCopyCommandsWithPreserveOption() throws Exception { argv = new String[] { "-cp", "-ptopx", src.toUri().toString(), target3.toUri().toString() }; ret = ToolRunner.run(shell, argv); - assertEquals("cp -ptopx is not working", SUCCESS, ret); + assertEquals(SUCCESS, ret, "cp -ptopx is not working"); targetStatus = dfs.getFileStatus(target3); assertEquals(mtime, targetStatus.getModificationTime()); assertEquals(atime, targetStatus.getAccessTime()); @@ -2273,7 +2288,7 @@ public void testCopyCommandsWithPreserveOption() throws Exception { argv = new String[] { "-cp", "-ptopa", src.toUri().toString(), target4.toUri().toString() }; ret = ToolRunner.run(shell, argv); - assertEquals("cp -ptopa is not working", SUCCESS, ret); + assertEquals(SUCCESS, ret, "cp -ptopa is not working"); targetStatus = dfs.getFileStatus(target4); assertEquals(mtime, targetStatus.getModificationTime()); assertEquals(atime, targetStatus.getAccessTime()); @@ -2293,7 +2308,7 @@ public void testCopyCommandsWithPreserveOption() throws Exception { argv = new String[] { "-cp", "-ptoa", src.toUri().toString(), target5.toUri().toString() }; ret = ToolRunner.run(shell, argv); - assertEquals("cp -ptoa is not working", SUCCESS, ret); + assertEquals(SUCCESS, ret, "cp -ptoa is not working"); targetStatus = dfs.getFileStatus(target5); assertEquals(mtime, targetStatus.getModificationTime()); assertEquals(atime, targetStatus.getAccessTime()); @@ -2314,7 +2329,8 @@ public void testCopyCommandsWithPreserveOption() throws Exception { } } - @Test (timeout = 120000) + @Test + @Timeout(value = 120) public void testCopyCommandsWithRawXAttrs() throws Exception { FsShell shell = null; final String testdir = "/tmp/TestDFSShell-testCopyCommandsWithRawXAttrs-" @@ -2436,7 +2452,7 @@ private Path doCopyAndTest(FsShell shell, Path dest, Path src, new String[] { "-cp", cpArgs, src.toUri().toString(), target.toUri().toString() }; final int ret = ToolRunner.run(shell, argv); - assertEquals("cp -p is not working", expectedExitCode, ret); + assertEquals(expectedExitCode, ret, "cp -p is not working"); return target; } @@ -2445,20 +2461,19 @@ private void checkXAttrs(FileSystem fs, Path target, boolean expectRaw, final Map xattrs = fs.getXAttrs(target); int expectedCount = 0; if (expectRaw) { - assertArrayEquals("raw.a1 has incorrect value", - RAW_A1_VALUE, xattrs.get(RAW_A1)); + assertArrayEquals(RAW_A1_VALUE, xattrs.get(RAW_A1), "raw.a1 has incorrect value"); expectedCount++; } if (expectVanillaXAttrs) { - assertArrayEquals("user.a1 has incorrect value", - USER_A1_VALUE, xattrs.get(USER_A1)); + assertArrayEquals(USER_A1_VALUE, xattrs.get(USER_A1), "user.a1 has incorrect value"); expectedCount++; } - assertEquals("xattrs size mismatch", expectedCount, xattrs.size()); + assertEquals(expectedCount, xattrs.size(), "xattrs size mismatch"); } // verify cp -ptopxa option will preserve directory attributes. - @Test (timeout = 120000) + @Test + @Timeout(value = 120) public void testCopyCommandsToDirectoryWithPreserveOption() throws Exception { FsShell shell = null; @@ -2505,7 +2520,7 @@ public void testCopyCommandsToDirectoryWithPreserveOption() String[] argv = new String[] { "-cp", "-p", srcDir.toUri().toString(), targetDir1.toUri().toString() }; int ret = ToolRunner.run(shell, argv); - assertEquals("cp -p is not working", SUCCESS, ret); + assertEquals(SUCCESS, ret, "cp -p is not working"); FileStatus targetStatus = dfs.getFileStatus(targetDir1); assertEquals(mtime, targetStatus.getModificationTime()); assertEquals(atime, targetStatus.getAccessTime()); @@ -2524,7 +2539,7 @@ public void testCopyCommandsToDirectoryWithPreserveOption() argv = new String[] { "-cp", "-ptop", srcDir.toUri().toString(), targetDir2.toUri().toString() }; ret = ToolRunner.run(shell, argv); - assertEquals("cp -ptop is not working", SUCCESS, ret); + assertEquals(SUCCESS, ret, "cp -ptop is not working"); targetStatus = dfs.getFileStatus(targetDir2); assertEquals(mtime, targetStatus.getModificationTime()); assertEquals(atime, targetStatus.getAccessTime()); @@ -2543,7 +2558,7 @@ public void testCopyCommandsToDirectoryWithPreserveOption() argv = new String[] { "-cp", "-ptopx", srcDir.toUri().toString(), targetDir3.toUri().toString() }; ret = ToolRunner.run(shell, argv); - assertEquals("cp -ptopx is not working", SUCCESS, ret); + assertEquals(SUCCESS, ret, "cp -ptopx is not working"); targetStatus = dfs.getFileStatus(targetDir3); assertEquals(mtime, targetStatus.getModificationTime()); assertEquals(atime, targetStatus.getAccessTime()); @@ -2564,7 +2579,7 @@ public void testCopyCommandsToDirectoryWithPreserveOption() argv = new String[] { "-cp", "-ptopa", srcDir.toUri().toString(), targetDir4.toUri().toString() }; ret = ToolRunner.run(shell, argv); - assertEquals("cp -ptopa is not working", SUCCESS, ret); + assertEquals(SUCCESS, ret, "cp -ptopa is not working"); targetStatus = dfs.getFileStatus(targetDir4); assertEquals(mtime, targetStatus.getModificationTime()); assertEquals(atime, targetStatus.getAccessTime()); @@ -2584,7 +2599,7 @@ public void testCopyCommandsToDirectoryWithPreserveOption() argv = new String[] { "-cp", "-ptoa", srcDir.toUri().toString(), targetDir5.toUri().toString() }; ret = ToolRunner.run(shell, argv); - assertEquals("cp -ptoa is not working", SUCCESS, ret); + assertEquals(SUCCESS, ret, "cp -ptoa is not working"); targetStatus = dfs.getFileStatus(targetDir5); assertEquals(mtime, targetStatus.getModificationTime()); assertEquals(atime, targetStatus.getAccessTime()); @@ -2606,7 +2621,8 @@ public void testCopyCommandsToDirectoryWithPreserveOption() } // Verify cp -pa option will preserve both ACL and sticky bit. - @Test (timeout = 120000) + @Test + @Timeout(value = 120) public void testCopyCommandsPreserveAclAndStickyBit() throws Exception { FsShell shell = null; final String testdir = @@ -2642,7 +2658,7 @@ public void testCopyCommandsPreserveAclAndStickyBit() throws Exception { String[] argv = new String[] { "-cp", "-p", src.toUri().toString(), target1.toUri().toString() }; int ret = ToolRunner.run(shell, argv); - assertEquals("cp is not working", SUCCESS, ret); + assertEquals(SUCCESS, ret, "cp is not working"); FileStatus targetStatus = dfs.getFileStatus(target1); assertEquals(mtime, targetStatus.getModificationTime()); assertEquals(atime, targetStatus.getAccessTime()); @@ -2659,7 +2675,7 @@ public void testCopyCommandsPreserveAclAndStickyBit() throws Exception { argv = new String[] { "-cp", "-ptopa", src.toUri().toString(), target2.toUri().toString() }; ret = ToolRunner.run(shell, argv); - assertEquals("cp -ptopa is not working", SUCCESS, ret); + assertEquals(SUCCESS, ret, "cp -ptopa is not working"); targetStatus = dfs.getFileStatus(target2); assertEquals(mtime, targetStatus.getModificationTime()); assertEquals(atime, targetStatus.getAccessTime()); @@ -2679,7 +2695,8 @@ public void testCopyCommandsPreserveAclAndStickyBit() throws Exception { } // force Copy Option is -f - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testCopyCommandsWithForceOption() throws Exception { FsShell shell = null; final File localFile = new File(TEST_ROOT_DIR, "testFileForPut"); @@ -2696,33 +2713,30 @@ public void testCopyCommandsWithForceOption() throws Exception { // Tests for put String[] argv = new String[] { "-put", "-f", localfilepath, testdir }; int res = ToolRunner.run(shell, argv); - assertEquals("put -f is not working", SUCCESS, res); + assertEquals(SUCCESS, res, "put -f is not working"); argv = new String[] { "-put", localfilepath, testdir }; res = ToolRunner.run(shell, argv); - assertEquals("put command itself is able to overwrite the file", ERROR, - res); + assertEquals(ERROR, res, "put command itself is able to overwrite the file"); // Tests for copyFromLocal argv = new String[] { "-copyFromLocal", "-f", localfilepath, testdir }; res = ToolRunner.run(shell, argv); - assertEquals("copyFromLocal -f is not working", SUCCESS, res); + assertEquals(SUCCESS, res, "copyFromLocal -f is not working"); argv = new String[] { "-copyFromLocal", localfilepath, testdir }; res = ToolRunner.run(shell, argv); - assertEquals( - "copyFromLocal command itself is able to overwrite the file", ERROR, - res); + assertEquals(ERROR, res, + "copyFromLocal command itself is able to overwrite the file"); // Tests for cp argv = new String[] { "-cp", "-f", localfilepath, testdir }; res = ToolRunner.run(shell, argv); - assertEquals("cp -f is not working", SUCCESS, res); + assertEquals(SUCCESS, res, "cp -f is not working"); argv = new String[] { "-cp", localfilepath, testdir }; res = ToolRunner.run(shell, argv); - assertEquals("cp command itself is able to overwrite the file", ERROR, - res); + assertEquals(ERROR, res, "cp command itself is able to overwrite the file"); } finally { if (null != shell) shell.close(); @@ -2738,7 +2752,8 @@ public void testCopyCommandsWithForceOption() throws Exception { * when trying to put/copyFromLocal a file that doesn't have read access * */ - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testCopyFromLocalWithPermissionDenied() throws Exception { FsShell shell = null; PrintStream bak = null; @@ -2764,18 +2779,18 @@ public void testCopyFromLocalWithPermissionDenied() throws Exception { // Tests for put String[] argv = new String[] { "-put", localfilepath, testdir }; int res = ToolRunner.run(shell, argv); - assertEquals("put is working", ERROR, res); + assertEquals(ERROR, res, "put is working"); String returned = out.toString(); - assertTrue(" outputs Permission denied error message", - (returned.lastIndexOf("Permission denied") != -1)); + assertTrue((returned.lastIndexOf("Permission denied") != -1), + " outputs Permission denied error message"); // Tests for copyFromLocal argv = new String[] { "-copyFromLocal", localfilepath, testdir }; res = ToolRunner.run(shell, argv); - assertEquals("copyFromLocal -f is working", ERROR, res); + assertEquals(ERROR, res, "copyFromLocal -f is working"); returned = out.toString(); - assertTrue(" outputs Permission denied error message", - (returned.lastIndexOf("Permission denied") != -1)); + assertTrue((returned.lastIndexOf("Permission denied") != -1), + " outputs Permission denied error message"); } finally { if (bak != null) { @@ -2797,7 +2812,8 @@ public void testCopyFromLocalWithPermissionDenied() throws Exception { * this here because the mini-miniCluster used with testHDFSConf.xml uses a * replication factor of 1 (for good reason). */ - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testSetrepLow() throws Exception { Configuration conf = new Configuration(); @@ -2817,8 +2833,7 @@ public void testSetrepLow() throws Exception { try { final FileSystem fs = cluster.getFileSystem(); - assertTrue("Unable to create test directory", - fs.mkdirs(new Path(testdir))); + assertTrue(fs.mkdirs(new Path(testdir)), "Unable to create test directory"); fs.create(hdfsFile, true).close(); @@ -2832,18 +2847,19 @@ public void testSetrepLow() throws Exception { final String[] argv = new String[] { "-setrep", "1", hdfsFile.toString() }; try { - assertEquals("Command did not return the expected exit code", - 1, shell.run(argv)); + assertEquals(1, shell.run(argv), + "Command did not return the expected exit code"); } finally { System.setOut(origOut); System.setErr(origErr); } - assertTrue("Error message is not the expected error message" - + bao.toString(), bao.toString().startsWith( - "setrep: Requested replication factor of 1 is less than " - + "the required minimum of 2 for /tmp/TestDFSShell-" - + "testSetrepLow/testFileForSetrepLow")); + assertTrue(bao.toString().startsWith( + "setrep: Requested replication factor of 1 is less than " + + "the required minimum of 2 for /tmp/TestDFSShell-" + + "testSetrepLow/testFileForSetrepLow"), + "Error message is not the expected error message" + + bao.toString()); } finally { shell.close(); cluster.shutdown(); @@ -2851,7 +2867,8 @@ public void testSetrepLow() throws Exception { } // setrep for file and directory. - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testSetrep() throws Exception { FsShell shell = null; final String testdir1 = "/tmp/TestDFSShell-testSetrep-" + counter.getAndIncrement(); @@ -2862,7 +2879,7 @@ public void testSetrep() throws Exception { final Short newRepFactor = new Short((short) 3); try { String[] argv; - assertThat(dfs.mkdirs(new Path(testdir2)), is(true)); + assertThat(dfs.mkdirs(new Path(testdir2))).isEqualTo(true); shell = new FsShell(dfs.getConf()); dfs.create(hdfsFile1, true).close(); @@ -2870,17 +2887,17 @@ public void testSetrep() throws Exception { // Tests for setrep on a file. argv = new String[] { "-setrep", newRepFactor.toString(), hdfsFile1.toString() }; - assertThat(shell.run(argv), is(SUCCESS)); - assertThat(dfs.getFileStatus(hdfsFile1).getReplication(), is(newRepFactor)); - assertThat(dfs.getFileStatus(hdfsFile2).getReplication(), is(oldRepFactor)); + assertThat(shell.run(argv)).isEqualTo(SUCCESS); + assertThat(dfs.getFileStatus(hdfsFile1).getReplication()).isEqualTo(newRepFactor); + assertThat(dfs.getFileStatus(hdfsFile2).getReplication()).isEqualTo(oldRepFactor); // Tests for setrep // Tests for setrep on a directory and make sure it is applied recursively. argv = new String[] { "-setrep", newRepFactor.toString(), testdir1 }; - assertThat(shell.run(argv), is(SUCCESS)); - assertThat(dfs.getFileStatus(hdfsFile1).getReplication(), is(newRepFactor)); - assertThat(dfs.getFileStatus(hdfsFile2).getReplication(), is(newRepFactor)); + assertThat(shell.run(argv)).isEqualTo(SUCCESS); + assertThat(dfs.getFileStatus(hdfsFile1).getReplication()).isEqualTo(newRepFactor); + assertThat(dfs.getFileStatus(hdfsFile2).getReplication()).isEqualTo(newRepFactor); } finally { if (shell != null) { @@ -2927,19 +2944,19 @@ private void deleteFileUsingTrash( final String trashFile = shell.getCurrentTrashDir() + "/" + testFile; String[] argv = new String[] { "-rm", testFile }; int res = ToolRunner.run(shell, argv); - assertEquals("rm failed", 0, res); + assertEquals(0, res, "rm failed"); if (serverTrash) { // If the server config was set we should use it unconditionally - assertTrue("File not in trash", fs.exists(new Path(trashFile))); + assertTrue(fs.exists(new Path(trashFile)), "File not in trash"); } else if (clientTrash) { // If the server config was not set but the client config was // set then we should use it - assertTrue("File not in trashed", fs.exists(new Path(trashFile))); + assertTrue(fs.exists(new Path(trashFile)), "File not in trashed"); } else { // If neither was set then we should not have trashed the file - assertFalse("File was not removed", fs.exists(new Path(testFile))); - assertFalse("File was trashed", fs.exists(new Path(trashFile))); + assertFalse(fs.exists(new Path(testFile)), "File was not removed"); + assertFalse(fs.exists(new Path(trashFile)), "File was trashed"); } } finally { if (fs != null) { @@ -2952,7 +2969,8 @@ private void deleteFileUsingTrash( } - @Test (timeout = 300000) + @Test + @Timeout(value = 300) public void testAppendToFile() throws Exception { final int inputFileLength = 1024 * 1024; File testRoot = new File(TEST_ROOT_DIR, "testAppendtoFileDir"); @@ -2969,8 +2987,7 @@ public void testAppendToFile() throws Exception { try { FileSystem dfs = cluster.getFileSystem(); - assertTrue("Not a HDFS: " + dfs.getUri(), - dfs instanceof DistributedFileSystem); + assertTrue(dfs instanceof DistributedFileSystem, "Not a HDFS: " + dfs.getUri()); // Run appendToFile once, make sure that the target file is // created and is of the right size. @@ -2980,20 +2997,21 @@ public void testAppendToFile() throws Exception { String[] argv = new String[] { "-appendToFile", file1.toString(), file2.toString(), remoteFile.toString() }; int res = ToolRunner.run(shell, argv); - assertThat(res, is(0)); - assertThat(dfs.getFileStatus(remoteFile).getLen(), is((long) inputFileLength * 2)); + assertThat(res).isEqualTo(0); + assertThat(dfs.getFileStatus(remoteFile).getLen()).isEqualTo((long) inputFileLength * 2); // Run the command once again and make sure that the target file // size has been doubled. res = ToolRunner.run(shell, argv); - assertThat(res, is(0)); - assertThat(dfs.getFileStatus(remoteFile).getLen(), is((long) inputFileLength * 4)); + assertThat(res).isEqualTo(0); + assertThat(dfs.getFileStatus(remoteFile).getLen()).isEqualTo((long) inputFileLength * 4); } finally { cluster.shutdown(); } } - @Test (timeout = 300000) + @Test + @Timeout(value = 300) public void testAppendToFileBadArgs() throws Exception { final int inputFileLength = 1024 * 1024; File testRoot = new File(TEST_ROOT_DIR, "testAppendToFileBadArgsDir"); @@ -3008,17 +3026,19 @@ public void testAppendToFileBadArgs() throws Exception { String[] argv = new String[] { "-appendToFile", file1.toString() }; int res = ToolRunner.run(shell, argv); - assertThat(res, not(0)); + assertThat(res).isNotEqualTo(0); // Mix stdin with other input files. Must fail. Path remoteFile = new Path("/remoteFile"); argv = new String[] { "-appendToFile", file1.toString(), "-", remoteFile.toString() }; res = ToolRunner.run(shell, argv); - assertThat(res, not(0)); + assertThat(res).isNotEqualTo(0); } - @Test (timeout = 300000) + @SuppressWarnings("checkstyle:MethodLength") + @Test + @Timeout(value = 300) public void testAppendToFileWithOptionN() throws Exception { final int inputFileLength = 1024 * 1024; File testRoot = new File(TEST_ROOT_DIR, "testAppendToFileWithOptionN"); @@ -3031,47 +3051,46 @@ public void testAppendToFileWithOptionN() throws Exception { try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(6).build()) { cluster.waitActive(); FileSystem hdfs = cluster.getFileSystem(); - assertTrue("Not a HDFS: " + hdfs.getUri(), - hdfs instanceof DistributedFileSystem); + assertTrue(hdfs instanceof DistributedFileSystem, "Not a HDFS: " + hdfs.getUri()); // Run appendToFile with option n by replica policy once, make sure that the target file is // created and is of the right size and block number is correct. String dir = "/replica"; boolean mkdirs = hdfs.mkdirs(new Path(dir)); - assertTrue("Mkdir fail", mkdirs); + assertTrue(mkdirs, "Mkdir fail"); Path remoteFile = new Path(dir + "/remoteFile"); FsShell shell = new FsShell(); shell.setConf(conf); String[] argv = new String[] { "-appendToFile", "-n", file1.toString(), remoteFile.toString() }; int res = ToolRunner.run(shell, argv); - assertEquals("Run appendToFile command fail", 0, res); + assertEquals(0, res, "Run appendToFile command fail"); FileStatus fileStatus = hdfs.getFileStatus(remoteFile); - assertEquals("File size should be " + inputFileLength, - inputFileLength, fileStatus.getLen()); + assertEquals(inputFileLength, fileStatus.getLen(), + "File size should be " + inputFileLength); BlockLocation[] fileBlockLocations = hdfs.getFileBlockLocations(fileStatus, 0, fileStatus.getLen()); - assertEquals("Block Num should be 1", 1, fileBlockLocations.length); + assertEquals(1, fileBlockLocations.length, "Block Num should be 1"); // Run appendToFile with option n by replica policy again and // make sure that the target file size has been doubled and block number has been doubled. res = ToolRunner.run(shell, argv); - assertEquals("Run appendToFile command fail", 0, res); + assertEquals(0, res, "Run appendToFile command fail"); fileStatus = hdfs.getFileStatus(remoteFile); - assertEquals("File size should be " + inputFileLength * 2, - inputFileLength * 2, fileStatus.getLen()); + assertEquals(inputFileLength * 2, fileStatus.getLen(), + "File size should be " + inputFileLength * 2); fileBlockLocations = hdfs.getFileBlockLocations(fileStatus, 0, fileStatus.getLen()); - assertEquals("Block Num should be 2", 2, fileBlockLocations.length); + assertEquals(2, fileBlockLocations.length, "Block Num should be 2"); // Before run appendToFile with option n by ec policy, set ec policy for the dir. dir = "/ecPolicy"; final String ecPolicyName = "RS-6-3-1024k"; mkdirs = hdfs.mkdirs(new Path(dir)); - assertTrue("Mkdir fail", mkdirs); + assertTrue(mkdirs, "Mkdir fail"); ((DistributedFileSystem) hdfs).setErasureCodingPolicy(new Path(dir), ecPolicyName); ErasureCodingPolicy erasureCodingPolicy = ((DistributedFileSystem) hdfs).getErasureCodingPolicy(new Path(dir)); - assertEquals("Set ec policy fail", ecPolicyName, erasureCodingPolicy.getName()); + assertEquals(ecPolicyName, erasureCodingPolicy.getName(), "Set ec policy fail"); // Run appendToFile with option n by ec policy once, make sure that the target file is // created and is of the right size and block group number is correct. @@ -3079,19 +3098,19 @@ public void testAppendToFileWithOptionN() throws Exception { argv = new String[] { "-appendToFile", "-n", file1.toString(), remoteFile.toString() }; res = ToolRunner.run(shell, argv); - assertEquals("Run appendToFile command fail", 0, res); + assertEquals(0, res, "Run appendToFile command fail"); fileStatus = hdfs.getFileStatus(remoteFile); - assertEquals("File size should be " + inputFileLength, - inputFileLength, fileStatus.getLen()); + assertEquals(inputFileLength, fileStatus.getLen(), + "File size should be " + inputFileLength); fileBlockLocations = hdfs.getFileBlockLocations(fileStatus, 0, fileStatus.getLen()); - assertEquals("Block Group Num should be 1", 1, fileBlockLocations.length); + assertEquals(1, fileBlockLocations.length, "Block Group Num should be 1"); // Run appendToFile without option n by ec policy again and make sure that // append on EC file without new block must fail. argv = new String[] { "-appendToFile", file1.toString(), remoteFile.toString() }; res = ToolRunner.run(shell, argv); - assertTrue("Run appendToFile command must fail", res != 0); + assertTrue(res != 0, "Run appendToFile command must fail"); // Run appendToFile with option n by ec policy again and // make sure that the target file size has been doubled @@ -3099,16 +3118,17 @@ public void testAppendToFileWithOptionN() throws Exception { argv = new String[] { "-appendToFile", "-n", file1.toString(), remoteFile.toString() }; res = ToolRunner.run(shell, argv); - assertEquals("Run appendToFile command fail", 0, res); + assertEquals(0, res, "Run appendToFile command fail"); fileStatus = hdfs.getFileStatus(remoteFile); - assertEquals("File size should be " + inputFileLength * 2, - inputFileLength * 2, fileStatus.getLen()); + assertEquals(inputFileLength * 2, fileStatus.getLen(), + "File size should be " + inputFileLength * 2); fileBlockLocations = hdfs.getFileBlockLocations(fileStatus, 0, fileStatus.getLen()); - assertEquals("Block Group Num should be 2", 2, fileBlockLocations.length); + assertEquals(2, fileBlockLocations.length, "Block Group Num should be 2"); } } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testSetXAttrPermission() throws Exception { UserGroupInformation user = UserGroupInformation. createUserForTesting("user", new String[] {"mygroup"}); @@ -3129,10 +3149,10 @@ public void testSetXAttrPermission() throws Exception { public Object run() throws Exception { int ret = ToolRunner.run(fshell, new String[]{ "-setfattr", "-n", "user.a1", "-v", "1234", "/foo"}); - assertEquals("Returned should be 1", 1, ret); + assertEquals(1, ret, "Returned should be 1"); String str = out.toString(); - assertTrue("Permission denied printed", - str.indexOf("Permission denied") != -1); + assertTrue(str.indexOf("Permission denied") != -1, + "Permission denied printed"); out.reset(); return null; } @@ -3140,7 +3160,7 @@ public Object run() throws Exception { int ret = ToolRunner.run(fshell, new String[]{ "-setfattr", "-n", "user.a1", "-v", "1234", "/foo"}); - assertEquals("Returned should be 0", 0, ret); + assertEquals(0, ret, "Returned should be 0"); out.reset(); // No permission to read and remove @@ -3151,18 +3171,18 @@ public Object run() throws Exception { // Read int ret = ToolRunner.run(fshell, new String[]{ "-getfattr", "-n", "user.a1", "/foo"}); - assertEquals("Returned should be 1", 1, ret); + assertEquals(1, ret, "Returned should be 1"); String str = out.toString(); - assertTrue("Permission denied printed", - str.indexOf("Permission denied") != -1); + assertTrue(str.indexOf("Permission denied") != -1, + "Permission denied printed"); out.reset(); // Remove ret = ToolRunner.run(fshell, new String[]{ "-setfattr", "-x", "user.a1", "/foo"}); - assertEquals("Returned should be 1", 1, ret); + assertEquals(1, ret, "Returned should be 1"); str = out.toString(); - assertTrue("Permission denied printed", - str.indexOf("Permission denied") != -1); + assertTrue(str.indexOf("Permission denied") != -1, + "Permission denied printed"); out.reset(); return null; } @@ -3175,7 +3195,8 @@ public Object run() throws Exception { } /* HDFS-6413 xattr names erroneously handled as case-insensitive */ - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testSetXAttrCaseSensitivity() throws Exception { PrintStream bak = null; try { @@ -3249,8 +3270,7 @@ private void doSetXattr(ByteArrayOutputStream out, FsShell fshell, ("Incorrect results from getfattr. Expected: "); sb.append(expect).append(" Full Result: "); sb.append(str); - assertTrue(sb.toString(), - str.indexOf(expect) != -1); + assertTrue(str.indexOf(expect) != -1, sb.toString()); } for (int i = 0; i < dontExpectArr.length; i++) { @@ -3259,8 +3279,7 @@ private void doSetXattr(ByteArrayOutputStream out, FsShell fshell, ("Incorrect results from getfattr. Didn't Expect: "); sb.append(dontExpect).append(" Full Result: "); sb.append(str); - assertTrue(sb.toString(), - str.indexOf(dontExpect) == -1); + assertTrue(str.indexOf(dontExpect) == -1, sb.toString()); } out.reset(); } @@ -3290,7 +3309,9 @@ private void doSetXattr(ByteArrayOutputStream out, FsShell fshell, * * As SuperUser: Set an Xattr with Trusted (Should pass) */ - @Test (timeout = 30000) + @SuppressWarnings("checkstyle:MethodLength") + @Test + @Timeout(value = 30) public void testSetXAttrPermissionAsDifferentOwner() throws Exception { final String root = "/testSetXAttrPermissionAsDifferentOwner"; final String USER1 = "user1"; @@ -3316,7 +3337,7 @@ public void testSetXAttrPermissionAsDifferentOwner() throws Exception { public Object run() throws Exception { final int ret = ToolRunner.run(fshell, new String[]{ "-mkdir", root + "/foo"}); - assertEquals("Return should be 0", 0, ret); + assertEquals(0, ret, "Return should be 0"); out.reset(); return null; } @@ -3329,7 +3350,7 @@ public Object run() throws Exception { // Give access to "other" final int ret = ToolRunner.run(fshell, new String[]{ "-chmod", "707", root + "/foo"}); - assertEquals("Return should be 0", 0, ret); + assertEquals(0, ret, "Return should be 0"); out.reset(); return null; } @@ -3342,7 +3363,7 @@ public Object run() throws Exception { public Object run() throws Exception { final int ret = ToolRunner.run(fshell, new String[]{ "-setfattr", "-n", "user.a1", "-v", "1234", root + "/foo"}); - assertEquals("Returned should be 0", 0, ret); + assertEquals(0, ret, "Returned should be 0"); out.reset(); return null; } @@ -3355,7 +3376,7 @@ public Object run() throws Exception { public Object run() throws Exception { final int ret = ToolRunner.run(fshell, new String[]{ "-setfattr", "-n", "user.a1", "-v", "1234", root + "/foo"}); - assertEquals("Returned should be 0", 0, ret); + assertEquals(0, ret, "Returned should be 0"); out.reset(); return null; } @@ -3369,12 +3390,12 @@ public Object run() throws Exception { // Read int ret = ToolRunner.run(fshell, new String[] { "-getfattr", "-n", "user.a1", root + "/foo" }); - assertEquals("Returned should be 0", 0, ret); + assertEquals(0, ret, "Returned should be 0"); out.reset(); // Remove ret = ToolRunner.run(fshell, new String[] { "-setfattr", "-x", "user.a1", root + "/foo" }); - assertEquals("Returned should be 0", 0, ret); + assertEquals(0, ret, "Returned should be 0"); out.reset(); return null; } @@ -3396,7 +3417,7 @@ public Object run() throws Exception { // Give access to "other" final int ret = ToolRunner.run(fshell, new String[]{ "-chmod", "700", root + "/foo"}); - assertEquals("Return should be 0", 0, ret); + assertEquals(0, ret, "Return should be 0"); out.reset(); return null; } @@ -3410,10 +3431,10 @@ public Object run() throws Exception { // set int ret = ToolRunner.run(fshell, new String[] { "-setfattr", "-n", "user.a2", root + "/foo" }); - assertEquals("Returned should be 1", 1, ret); + assertEquals(1, ret, "Returned should be 1"); final String str = out.toString(); - assertTrue("Permission denied printed", - str.indexOf("Permission denied") != -1); + assertTrue(str.indexOf("Permission denied") != -1, + "Permission denied printed"); out.reset(); return null; } @@ -3427,10 +3448,10 @@ public Object run() throws Exception { // set int ret = ToolRunner.run(fshell, new String[] { "-setfattr", "-x", "user.a2", root + "/foo" }); - assertEquals("Returned should be 1", 1, ret); + assertEquals(1, ret, "Returned should be 1"); final String str = out.toString(); - assertTrue("Permission denied printed", - str.indexOf("Permission denied") != -1); + assertTrue(str.indexOf("Permission denied") != -1, + "Permission denied printed"); out.reset(); return null; } @@ -3443,7 +3464,7 @@ public Object run() throws Exception { // set int ret = ToolRunner.run(fshell, new String[] { "-setfattr", "-n", "trusted.a3", root + "/foo" }); - assertEquals("Returned should be 0", 0, ret); + assertEquals(0, ret, "Returned should be 0"); out.reset(); return null; } @@ -3461,7 +3482,8 @@ public Object run() throws Exception { * 2. Test that CLI throws an exception and returns non-0 when a non-existent * xattr is requested. */ - @Test (timeout = 120000) + @Test + @Timeout(value = 120) public void testGetFAttrErrors() throws Exception { final UserGroupInformation user = UserGroupInformation. createUserForTesting("user", new String[] {"mygroup"}); @@ -3481,7 +3503,7 @@ public void testGetFAttrErrors() throws Exception { { final int ret = ToolRunner.run(fshell, new String[] { "-setfattr", "-n", "user.a1", "-v", "1234", p.toString()}); - assertEquals("Returned should be 0", 0, ret); + assertEquals(0, ret, "Returned should be 0"); out.reset(); } @@ -3491,8 +3513,8 @@ public Object run() throws Exception { int ret = ToolRunner.run(fshell, new String[] { "-getfattr", "-n", "user.a1", p.toString()}); String str = out.toString(); - assertTrue("xattr value was incorrectly returned", - str.indexOf("1234") == -1); + assertTrue(str.indexOf("1234") == -1, + "xattr value was incorrectly returned"); out.reset(); return null; } @@ -3502,10 +3524,8 @@ public Object run() throws Exception { final int ret = ToolRunner.run(fshell, new String[]{ "-getfattr", "-n", "user.nonexistent", p.toString()}); String str = out.toString(); - assertTrue("xattr value was incorrectly returned", - str.indexOf( - "getfattr: " + XAttrNotFoundException.DEFAULT_EXCEPTION_MSG) - >= 0); + assertTrue(str.indexOf("getfattr: " + XAttrNotFoundException.DEFAULT_EXCEPTION_MSG) + >= 0, "xattr value was incorrectly returned"); out.reset(); } } finally { @@ -3519,7 +3539,8 @@ public Object run() throws Exception { * Test that the server trash configuration is respected when * the client configuration is not set. */ - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testServerConfigRespected() throws Exception { deleteFileUsingTrash(true, false); } @@ -3528,7 +3549,8 @@ public void testServerConfigRespected() throws Exception { * Test that server trash configuration is respected even when the * client configuration is set. */ - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testServerConfigRespectedWithClient() throws Exception { deleteFileUsingTrash(true, true); } @@ -3537,7 +3559,8 @@ public void testServerConfigRespectedWithClient() throws Exception { * Test that the client trash configuration is respected when * the server configuration is not set. */ - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testClientConfigRespected() throws Exception { deleteFileUsingTrash(false, true); } @@ -3545,12 +3568,14 @@ public void testClientConfigRespected() throws Exception { /** * Test that trash is disabled by default. */ - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testNoTrashConfig() throws Exception { deleteFileUsingTrash(false, false); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testListReserved() throws IOException { Configuration conf = new HdfsConfiguration(); MiniDFSCluster cluster = @@ -3592,7 +3617,8 @@ public void testListReserved() throws IOException { } } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testMkdirReserved() throws IOException { try { dfs.mkdirs(new Path("/.reserved")); @@ -3603,7 +3629,8 @@ public void testMkdirReserved() throws IOException { } } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testRmReserved() throws IOException { try { dfs.delete(new Path("/.reserved"), true); @@ -3650,7 +3677,8 @@ public void testCopyReserved() throws IOException { } } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testChmodReserved() throws IOException { // runCmd prints error into System.err, thus verify from there. PrintStream syserr = System.err; @@ -3666,7 +3694,8 @@ public void testChmodReserved() throws IOException { } } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testChownReserved() throws IOException { // runCmd prints error into System.err, thus verify from there. PrintStream syserr = System.err; @@ -3682,7 +3711,8 @@ public void testChownReserved() throws IOException { } } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testSymLinkReserved() throws IOException { try { dfs.createSymlink(new Path("/.reserved"), new Path("/rl1"), false); @@ -3693,7 +3723,8 @@ public void testSymLinkReserved() throws IOException { } } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testSnapshotReserved() throws IOException { final Path reserved = new Path("/.reserved"); try { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShellGenericOptions.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShellGenericOptions.java index 282dcf7b0066c..73e14c4a2aad9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShellGenericOptions.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShellGenericOptions.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hdfs; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.File; import java.io.FileNotFoundException; @@ -29,7 +29,7 @@ import org.apache.hadoop.fs.FsShell; import org.apache.hadoop.fs.Path; import org.apache.hadoop.util.ToolRunner; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestDFSShellGenericOptions { @@ -103,8 +103,8 @@ private void execute(String [] args, String namenode) { ToolRunner.run(shell, args); fs = FileSystem.get(DFSUtilClient.getNNUri( DFSUtilClient.getNNAddress(namenode)), shell.getConf()); - assertTrue("Directory does not get created", - fs.isDirectory(new Path("/data"))); + assertTrue(fs.isDirectory(new Path("/data")), + "Directory does not get created"); fs.delete(new Path("/data"), true); } catch (Exception e) { System.err.println(e.getMessage()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShellTouch.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShellTouch.java index d0334cbf42747..fdac08c8e3006 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShellTouch.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShellTouch.java @@ -22,9 +22,9 @@ import java.text.ParseException; import java.util.Date; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,7 +49,7 @@ public class TestDFSShellTouch { private static DistributedFileSystem dfs; private static FsShell shell; - @BeforeClass + @BeforeAll public static void setup() throws IOException { final Configuration conf = new Configuration(); conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, @@ -61,7 +61,7 @@ public static void setup() throws IOException { shell = new FsShell(dfs.getConf()); } - @AfterClass + @AfterAll public static void tearDown() { if (miniCluster != null) { miniCluster.shutdown(true, true); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStartupVersions.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStartupVersions.java index 860794c083e0e..6f16e7a1ab723 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStartupVersions.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStartupVersions.java @@ -19,8 +19,8 @@ import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.DATA_NODE; import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.NAME_NODE; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.File; @@ -33,8 +33,9 @@ import org.apache.hadoop.hdfs.server.common.Storage; import org.apache.hadoop.hdfs.server.common.StorageInfo; import org.apache.hadoop.hdfs.server.datanode.DataNodeLayoutVersion; -import org.junit.After; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; /** * This test ensures the appropriate response (successful or failure) from @@ -238,7 +239,8 @@ boolean isVersionCompatible(StorageData namenodeSd, StorageData datanodeSd) { * this iterations version 3-tuple * */ - @Test (timeout=300000) + @Test + @Timeout(value = 300) public void testVersions() throws Exception { UpgradeUtilities.initialize(); Configuration conf = UpgradeUtilities.initializeStorageStateConf(1, @@ -280,7 +282,7 @@ public void testVersions() throws Exception { } } - @After + @AfterEach public void tearDown() throws Exception { LOG.info("Shutting down MiniDFSCluster"); if (cluster != null) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStorageStateRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStorageStateRecovery.java index 60839dc17bd1b..79aa16dc4a96b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStorageStateRecovery.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStorageStateRecovery.java @@ -19,10 +19,10 @@ import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.DATA_NODE; import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.NAME_NODE; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.File; import java.io.IOException; @@ -33,9 +33,9 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.apache.hadoop.hdfs.server.common.Storage; import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** * This test ensures the appropriate response (successful or failure) from @@ -443,13 +443,13 @@ public void testBlockPoolStorageStates() throws Exception { } // end numDirs loop } - @Before + @BeforeEach public void setUp() throws Exception { LOG.info("Setting up the directory structures."); UpgradeUtilities.initialize(); } - @After + @AfterEach public void tearDown() throws Exception { LOG.info("Shutting down MiniDFSCluster"); if (cluster != null) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java index 9d3bdbf4e9520..f9646e9ee162a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hdfs; -import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.HadoopIllegalArgumentException; @@ -41,12 +40,9 @@ import org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawErasureCoderFactory; import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; import org.apache.hadoop.test.GenericTestUtils; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.Timeout; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.nio.ByteBuffer; @@ -57,15 +53,18 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -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.assertArrayEquals; +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 static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.spy; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.io.TempDir; +@Timeout(300) public class TestDFSStripedInputStream { public static final Logger LOG = @@ -84,17 +83,15 @@ public class TestDFSStripedInputStream { private int blockSize; private int blockGroupSize; - @Rule - public Timeout globalTimeout = new Timeout(300000); - - @Rule - public TemporaryFolder baseDir = new TemporaryFolder(); + @SuppressWarnings("checkstyle:VisibilityModifier") + @TempDir + java.nio.file.Path baseDir; public ErasureCodingPolicy getEcPolicy() { return StripedFileTestUtil.getDefaultECPolicy(); } - @Before + @BeforeEach public void setup() throws IOException { /* * Initialize erasure coding policy. @@ -119,7 +116,7 @@ public void setup() throws IOException { } private void startUp() throws IOException { - cluster = new MiniDFSCluster.Builder(conf, baseDir.getRoot()).numDataNodes( + cluster = new MiniDFSCluster.Builder(conf, baseDir.toFile()).numDataNodes( dataBlocks + parityBlocks).build(); cluster.waitActive(); for (DataNode dn : cluster.getDataNodes()) { @@ -132,7 +129,7 @@ private void startUp() throws IOException { .setErasureCodingPolicy(dirPath.toString(), ecPolicy.getName()); } - @After + @AfterEach public void tearDown() { if (cluster != null) { cluster.shutdown(); @@ -218,9 +215,8 @@ public void testPread() throws Exception { int ret = in.read(startOffset, buf, 0, fileLen); assertEquals(remaining, ret); for (int i = 0; i < remaining; i++) { - Assert.assertEquals("Byte at " + (startOffset + i) + " should be the " + - "same", - expected[startOffset + i], buf[i]); + assertEquals(expected[startOffset + i], buf[i], + "Byte at " + (startOffset + i) + " should be the " + "same"); } } in.close(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java index 9044a6d0cb066..791a1afc01f79 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java @@ -18,9 +18,9 @@ package org.apache.hadoop.hdfs; import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Write.RECOVER_LEASE_ON_CLOSE_EXCEPTION_KEY; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.Mockito.doThrow; import java.io.ByteArrayInputStream; @@ -47,13 +47,13 @@ import org.apache.hadoop.io.erasurecode.ErasureCodeNative; import org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawErasureCoderFactory; import org.apache.hadoop.test.GenericTestUtils; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.Timeout; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; +import org.junit.jupiter.api.Timeout; +@Timeout(300) public class TestDFSStripedOutputStream { public static final Logger LOG = LoggerFactory.getLogger( TestDFSStripedOutputStream.class); @@ -74,14 +74,11 @@ public class TestDFSStripedOutputStream { private final int stripesPerBlock = 4; private int blockSize; - @Rule - public Timeout globalTimeout = new Timeout(300000); - public ErasureCodingPolicy getEcPolicy() { return StripedFileTestUtil.getDefaultECPolicy(); } - @Before + @BeforeEach public void setup() throws IOException { /* * Initialize erasure coding policy. @@ -110,7 +107,7 @@ public void setup() throws IOException { fs.getClient().setErasureCodingPolicy("/", ecPolicy.getName()); } - @After + @AfterEach public void tearDown() { if (cluster != null) { cluster.shutdown(); @@ -206,12 +203,10 @@ public void testStreamFlush() throws Exception { final byte[] bytes = StripedFileTestUtil.generateBytes(blockSize * dataBlocks * 3 + cellSize * dataBlocks + cellSize + 123); try (FSDataOutputStream os = fs.create(new Path("/ec-file-1"))) { - assertFalse( - "DFSStripedOutputStream should not have hflush() capability yet!", - os.hasCapability(StreamCapability.HFLUSH.getValue())); - assertFalse( - "DFSStripedOutputStream should not have hsync() capability yet!", - os.hasCapability(StreamCapability.HSYNC.getValue())); + assertFalse(os.hasCapability(StreamCapability.HFLUSH.getValue()), + "DFSStripedOutputStream should not have hflush() capability yet!"); + assertFalse(os.hasCapability(StreamCapability.HSYNC.getValue()), + "DFSStripedOutputStream should not have hsync() capability yet!"); try (InputStream is = new ByteArrayInputStream(bytes)) { IOUtils.copyBytes(is, os, bytes.length); os.hflush(); @@ -219,8 +214,8 @@ public void testStreamFlush() throws Exception { os.hsync(); IOUtils.copyBytes(is, os, bytes.length); } - assertTrue("stream is not a DFSStripedOutputStream", - os.getWrappedStream() instanceof DFSStripedOutputStream); + assertTrue(os.getWrappedStream() instanceof DFSStripedOutputStream, + "stream is not a DFSStripedOutputStream"); final DFSStripedOutputStream dfssos = (DFSStripedOutputStream) os.getWrappedStream(); dfssos.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH)); @@ -265,8 +260,8 @@ public void testExceptionInCloseECFileWithRecoverLease() throws Exception { spyClient.create("/testExceptionInCloseECFileWithRecoverLease", FsPermission.getFileDefault(), EnumSet.of(CreateFlag.CREATE), (short) 3, 1024*1024, null, 1024, null); - assertTrue("stream should be a DFSStripedOutputStream", - dfsOutputStream instanceof DFSStripedOutputStream); + assertTrue(dfsOutputStream instanceof DFSStripedOutputStream, + "stream should be a DFSStripedOutputStream"); DFSOutputStream spyDFSOutputStream = Mockito.spy(dfsOutputStream); doThrow(new IOException("Emulated IOException in close")) .when(spyDFSOutputStream).completeFile(Mockito.any()); @@ -290,8 +285,8 @@ public void testExceptionInCloseECFileWithoutRecoverLease() throws Exception { spyClient.create("/testExceptionInCloseECFileWithoutRecoverLease", FsPermission.getFileDefault(), EnumSet.of(CreateFlag.CREATE), (short) 3, 1024*1024, null, 1024, null); - assertTrue("stream should be a DFSStripedOutputStream", - dfsOutputStream instanceof DFSStripedOutputStream); + assertTrue(dfsOutputStream instanceof DFSStripedOutputStream, + "stream should be a DFSStripedOutputStream"); DFSOutputStream spyDFSOutputStream = Mockito.spy(dfsOutputStream); doThrow(new IOException("Emulated IOException in close")) .when(spyDFSOutputStream).completeFile(Mockito.any()); @@ -303,8 +298,7 @@ public void testExceptionInCloseECFileWithoutRecoverLease() throws Exception { try { waitForFileClosed("/testExceptionInCloseECFileWithoutRecoverLease"); } catch (TimeoutException e) { - assertFalse( - isFileClosed("/testExceptionInCloseECFileWithoutRecoverLease")); + assertFalse(isFileClosed("/testExceptionInCloseECFileWithoutRecoverLease")); } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamUpdatePipeline.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamUpdatePipeline.java index 41a1dcabaddfc..0670a4418afb2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamUpdatePipeline.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamUpdatePipeline.java @@ -21,7 +21,8 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.IOUtils; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY; @@ -66,7 +67,8 @@ public void testDFSStripedOutputStreamUpdatePipeline() throws Exception { * Test writing ec file hang when applying the second block group occurs * an addBlock exception (e.g. quota exception). */ - @Test(timeout = 90000) + @Test + @Timeout(value = 90) public void testECWriteHangWhenAddBlockWithException() throws Exception { Configuration conf = new HdfsConfiguration(); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1 * 1024 * 1024); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java index ebc0379bc60be..8ee44ef7c3b01 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java @@ -22,16 +22,18 @@ import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getImageFileName; import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getInProgressEditsFileName; import static org.apache.hadoop.test.GenericTestUtils.assertExists; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import java.io.File; import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.regex.Pattern; +import org.junit.jupiter.api.Disabled; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -46,9 +48,9 @@ import org.apache.hadoop.hdfs.server.namenode.TestParallelImageWrite; import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.util.StringUtils; -import org.junit.BeforeClass; -import org.junit.Ignore; -import org.junit.Test; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.apache.hadoop.thirdparty.com.google.common.base.Joiner; @@ -171,16 +173,14 @@ void startNameNodeShouldFail(StartupOption operation, } catch (Exception e) { // expect exception if (exceptionClass != null) { - assertTrue("Caught exception is not of expected class " - + exceptionClass.getSimpleName() + ": " - + StringUtils.stringifyException(e), - exceptionClass.isInstance(e)); + assertTrue(exceptionClass.isInstance(e), + "Caught exception is not of expected class " + + exceptionClass.getSimpleName() + ": " + StringUtils.stringifyException(e)); } if (messagePattern != null) { - assertTrue("Caught exception message string does not match expected pattern \"" - + messagePattern.pattern() + "\" : " - + StringUtils.stringifyException(e), - messagePattern.matcher(e.getMessage()).find()); + assertTrue(messagePattern.matcher(e.getMessage()).find(), + "Caught exception message string does not match expected pattern \"" + + messagePattern.pattern() + "\" : " + StringUtils.stringifyException(e)); } LOG.info("Successfully detected expected NameNode startup failure."); } @@ -195,8 +195,8 @@ void startNameNodeShouldFail(StartupOption operation, */ void startBlockPoolShouldFail(StartupOption operation, String bpid) throws IOException { cluster.startDataNodes(conf, 1, false, operation, null); // should fail - assertFalse("Block pool " + bpid + " should have failed to start", - cluster.getDataNodes().get(0).isBPServiceAlive(bpid)); + assertFalse(cluster.getDataNodes().get(0).isBPServiceAlive(bpid), + "Block pool " + bpid + " should have failed to start"); } /** @@ -212,7 +212,7 @@ private MiniDFSCluster createCluster() throws IOException { .build(); } - @BeforeClass + @BeforeAll public static void initialize() throws Exception { UpgradeUtilities.initialize(); } @@ -221,7 +221,9 @@ public static void initialize() throws Exception { * This test attempts to upgrade the NameNode and DataNode under * a number of valid and invalid conditions. */ - @Test(timeout = 60000) + @SuppressWarnings("checkstyle:MethodLength") + @Test + @Timeout(value = 60) public void testUpgrade() throws Exception { File[] baseDirs; StorageInfo storageInfo = null; @@ -404,7 +406,7 @@ public void testUpgrade() throws Exception { * Stand-alone test to detect failure of one SD during parallel upgrade. * At this time, can only be done with manual hack of {@link FSImage.doUpgrade()} */ - @Ignore + @Disabled public void testUpgrade4() throws Exception { int numDirs = 4; conf = new HdfsConfiguration(); @@ -432,21 +434,22 @@ private void deleteStorageFilesWithPrefix(String[] nameNodeDirs, String prefix) File currentDir = new File(baseDir, "current"); for (File f : currentDir.listFiles()) { if (f.getName().startsWith(prefix)) { - assertTrue("Deleting " + f, f.delete()); + assertTrue(f.delete(), "Deleting " + f); } } } } - @Test(expected=IOException.class) + @Test public void testUpgradeFromPreUpgradeLVFails() throws IOException { + assertThrows(IOException.class, () -> { + Storage.checkVersionUpgradable(Storage.LAST_PRE_UPGRADE_LAYOUT_VERSION + 1); + fail("Expected IOException is not thrown"); + }); // Upgrade from versions prior to Storage#LAST_UPGRADABLE_LAYOUT_VERSION - // is not allowed - Storage.checkVersionUpgradable(Storage.LAST_PRE_UPGRADE_LAYOUT_VERSION + 1); - fail("Expected IOException is not thrown"); } - @Ignore + @Disabled public void test203LayoutVersion() { for (int lv : Storage.LAYOUT_VERSIONS_203) { assertTrue(Storage.is203LayoutVersion(lv)); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java index 5469ebbb757c2..5453a7465e86d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java @@ -48,9 +48,12 @@ import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.StringUtils; import org.apache.log4j.Logger; -import org.junit.Test; +import org.junit.jupiter.api.Test; -import static org.junit.Assert.*; +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 static org.junit.jupiter.api.Assertions.fail; /** * This tests data transfer protocol handling in the Datanode. It sends @@ -161,7 +164,7 @@ private void verifyChecksum(String path, long checksum) throws IOException { // The paths are expected to be listed in the same order // as they are traversed here. assertEquals(info.path, path); - assertEquals("Checking checksum for " + path, info.checksum, checksum); + assertEquals(info.checksum, checksum, "Checking checksum for " + path); } } @@ -251,9 +254,9 @@ public void testFailOnPreUpgradeImage() throws IOException { // Set up a fake NN storage that looks like an ancient Hadoop dir circa 0.3.0 FileUtil.fullyDelete(namenodeStorage); - assertTrue("Make " + namenodeStorage, namenodeStorage.mkdirs()); + assertTrue(namenodeStorage.mkdirs(), "Make " + namenodeStorage); File imageDir = new File(namenodeStorage, "image"); - assertTrue("Make " + imageDir, imageDir.mkdirs()); + assertTrue(imageDir.mkdirs(), "Make " + imageDir); // Hex dump of a formatted image from Hadoop 0.3.0 File imageFile = new File(imageDir, "fsimage"); @@ -333,7 +336,7 @@ public void testUpgradeFromCorruptRel22Image() throws IOException { } int md5failures = appender.countExceptionsWithMessage( " is corrupt with MD5 checksum of "); - assertEquals("Upgrade did not fail with bad MD5", 1, md5failures); + assertEquals(1, md5failures, "Upgrade did not fail with bad MD5"); } } @@ -395,10 +398,10 @@ public void testUpgradeFromRel1ReservedImage() throws Exception { } } for (String s: expected) { - assertTrue("Did not find expected path " + s, found.contains(s)); + assertTrue(found.contains(s), "Did not find expected path " + s); } - assertEquals("Found an unexpected path while listing filesystem", - found.size(), expected.length); + assertEquals(found.size(), expected.length, + "Found an unexpected path while listing filesystem"); } } finally { if (cluster != null) { @@ -459,10 +462,10 @@ public void testUpgradeFromRel023ReservedImage() throws Exception { } } for (String s: expected) { - assertTrue("Did not find expected path " + s, found.contains(s)); + assertTrue(found.contains(s), "Did not find expected path " + s); } - assertEquals("Found an unexpected path while listing filesystem", - found.size(), expected.length); + assertEquals(found.size(), expected.length, + "Found an unexpected path while listing filesystem"); } } finally { if (cluster != null) { @@ -554,10 +557,10 @@ public void testUpgradeFromRel2ReservedImage() throws Exception { } } for (String s: expected) { - assertTrue("Did not find expected path " + s, found.contains(s)); + assertTrue(found.contains(s), "Did not find expected path " + s); } - assertEquals("Found an unexpected path while listing filesystem", - found.size(), expected.length); + assertEquals(found.size(), expected.length, + "Found an unexpected path while listing filesystem"); } } finally { if (cluster != null) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java index 3837a6aab43b5..fce57c842fd95 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java @@ -6,9 +6,9 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

* Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. @@ -37,15 +37,15 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_TRUSTSTORE_PASSWORD_KEY; import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains; import static org.apache.hadoop.test.PlatformAssumptions.assumeNotWindows; -import static org.hamcrest.CoreMatchers.not; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +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.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 static org.mockito.Mockito.*; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; @@ -84,26 +84,26 @@ import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.LambdaTestUtils; import org.apache.hadoop.util.Shell; -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 org.junit.jupiter.api.Timeout; public class TestDFSUtil { - static final String NS1_NN_ADDR = "ns1-nn.example.com:8020"; - static final String NS1_NN1_ADDR = "ns1-nn1.example.com:8020"; - static final String NS1_NN2_ADDR = "ns1-nn2.example.com:8020"; - static final String NS1_NN1_HTTPS_ADDR = "ns1-nn1.example.com:50740"; - static final String NS1_NN1_HTTP_ADDR = "ns1-nn1.example.com:50070"; + static final String NS1_NN_ADDR = "ns1-nn.example.com:8020"; + static final String NS1_NN1_ADDR = "ns1-nn1.example.com:8020"; + static final String NS1_NN2_ADDR = "ns1-nn2.example.com:8020"; + static final String NS1_NN1_HTTPS_ADDR = "ns1-nn1.example.com:50740"; + static final String NS1_NN1_HTTP_ADDR = "ns1-nn1.example.com:50070"; /** * Reset to default UGI settings since some tests change them. */ - @Before + @BeforeEach public void resetUGI() { UserGroupInformation.setConfiguration(new Configuration()); } - + /** * Test conversion of LocatedBlock to BlockLocation */ @@ -130,18 +130,17 @@ public void testLocatedBlocks2Locations() { BlockLocation[] bs = DFSUtilClient.locatedBlocks2Locations(lbs); - assertTrue("expected 2 blocks but got " + bs.length, - bs.length == 2); + assertTrue(bs.length == 2, "expected 2 blocks but got " + bs.length); int corruptCount = 0; - for (BlockLocation b: bs) { + for (BlockLocation b : bs) { if (b.isCorrupt()) { corruptCount++; } } - assertTrue("expected 1 corrupt files but got " + corruptCount, - corruptCount == 1); + assertTrue(corruptCount == 1, + "expected 1 corrupt files but got " + corruptCount); // test an empty location bs = DFSUtilClient.locatedBlocks2Locations(new LocatedBlocks()); @@ -156,7 +155,7 @@ public void testLocatedBlockConstructorWithNullCachedLocs() { DatanodeInfo d = DFSTestUtil.getLocalDatanodeInfo(); DatanodeInfo[] ds = new DatanodeInfo[1]; ds[0] = d; - + ExtendedBlock b1 = new ExtendedBlock("bpid", 1, 1, 1); LocatedBlock l1 = new LocatedBlock(b1, ds, null, null, 0, false, null); final DatanodeInfo[] cachedLocs = l1.getCachedLocations(); @@ -180,7 +179,7 @@ public void getNameServiceId() { conf.set(DFS_NAMESERVICE_ID, "nn1"); assertEquals("nn1", DFSUtil.getNamenodeNameServiceId(conf)); } - + /** * Test {@link DFSUtil#getNamenodeNameServiceId(Configuration)} to ensure * nameserviceId for namenode is determined based on matching the address with @@ -219,16 +218,18 @@ public void getSecondaryNameServiceId() { * exception is thrown when multiple rpc addresses match the local node's * address */ - @Test(expected = HadoopIllegalArgumentException.class) + @Test public void testGetNameServiceIdException() { - HdfsConfiguration conf = new HdfsConfiguration(); - conf.set(DFS_NAMESERVICES, "nn1,nn2"); - conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY, "nn1"), - "localhost:9000"); - conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY, "nn2"), - "localhost:9001"); - DFSUtil.getNamenodeNameServiceId(conf); - fail("Expected exception is not thrown"); + assertThrows(HadoopIllegalArgumentException.class, () -> { + HdfsConfiguration conf = new HdfsConfiguration(); + conf.set(DFS_NAMESERVICES, "nn1,nn2"); + conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY, "nn1"), + "localhost:9000"); + conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY, "nn2"), + "localhost:9001"); + DFSUtil.getNamenodeNameServiceId(conf); + fail("Expected exception is not thrown"); + }); } /** @@ -244,7 +245,7 @@ public void testGetNameServiceIds() { assertEquals("nn1", it.next().toString()); assertEquals("nn2", it.next().toString()); } - + @Test public void testGetOnlyNameServiceIdOrNull() { HdfsConfiguration conf = new HdfsConfiguration(); @@ -277,13 +278,13 @@ public void testMultipleNamenodes() throws IOException { Map> nnMap = DFSUtil .getNNServiceRpcAddresses(conf); assertEquals(2, nnMap.size()); - + Map nn1Map = nnMap.get("nn1"); assertEquals(1, nn1Map.size()); InetSocketAddress addr = nn1Map.get(null); assertEquals("localhost", addr.getHostName()); assertEquals(9000, addr.getPort()); - + Map nn2Map = nnMap.get("nn2"); assertEquals(1, nn2Map.size()); addr = nn2Map.get(null); @@ -301,7 +302,7 @@ public void testMultipleNamenodes() throws IOException { } public void checkNameServiceId(Configuration conf, String addr, - String expectedNameServiceId) { + String expectedNameServiceId) { InetSocketAddress s = NetUtils.createSocketAddr(addr); String nameserviceId = DFSUtil.getNameServiceIdFromAddress(conf, s, DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, DFS_NAMENODE_RPC_ADDRESS_KEY); @@ -317,15 +318,15 @@ public void testDefaultNamenode() throws IOException { // If DFS_FEDERATION_NAMESERVICES is not set, verify that // default namenode address is returned. Map> addrMap = - DFSUtil.getNNServiceRpcAddresses(conf); + DFSUtil.getNNServiceRpcAddresses(conf); assertEquals(1, addrMap.size()); - + Map defaultNsMap = addrMap.get(null); assertEquals(1, defaultNsMap.size()); - + assertEquals(9999, defaultNsMap.get(null).getPort()); } - + /** * Test to ensure nameservice specific keys in the configuration are * copied to generic keys when the namenode starts. @@ -334,7 +335,7 @@ public void testDefaultNamenode() throws IOException { public void testConfModificationFederationOnly() { final HdfsConfiguration conf = new HdfsConfiguration(); String nsId = "ns1"; - + conf.set(DFS_NAMESERVICES, nsId); conf.set(DFS_NAMESERVICE_ID, nsId); @@ -353,7 +354,7 @@ public void testConfModificationFederationOnly() { assertEquals(key, conf.get(key)); } } - + /** * Test to ensure nameservice specific keys in the configuration are * copied to generic keys when the namenode starts. @@ -363,7 +364,7 @@ public void testConfModificationFederationAndHa() { final HdfsConfiguration conf = new HdfsConfiguration(); String nsId = "ns1"; String nnId = "nn1"; - + conf.set(DFS_NAMESERVICES, nsId); conf.set(DFS_NAMESERVICE_ID, nsId); conf.set(DFS_HA_NAMENODES_KEY_PREFIX + "." + nsId, nnId); @@ -387,7 +388,7 @@ public void testConfModificationFederationAndHa() { /** * Ensure that fs.defaultFS is set in the configuration even if neither HA nor * Federation is enabled. - * + * * Regression test for HDFS-3351. */ @Test @@ -395,7 +396,7 @@ public void testConfModificationNoFederationOrHa() { final HdfsConfiguration conf = new HdfsConfiguration(); String nsId = null; String nnId = null; - + conf.set(DFS_NAMENODE_RPC_ADDRESS_KEY, "localhost:1234"); assertFalse("hdfs://localhost:1234".equals(conf.get(FS_DEFAULT_NAME_KEY))); @@ -419,23 +420,23 @@ public void testSomeConfsNNSpecificSomeNSSpecific() { Configuration newConf = new Configuration(conf); NameNode.initializeGenericKeys(newConf, "ns2", "nn1"); assertEquals("global-default", newConf.get(key)); - + // A namenode in another non-HA nameservice should get global default. newConf = new Configuration(conf); NameNode.initializeGenericKeys(newConf, "ns2", null); - assertEquals("global-default", newConf.get(key)); - + assertEquals("global-default", newConf.get(key)); + // A namenode in the same nameservice should get the ns setting newConf = new Configuration(conf); NameNode.initializeGenericKeys(newConf, "ns1", "nn2"); - assertEquals("ns1-override", newConf.get(key)); + assertEquals("ns1-override", newConf.get(key)); // The nn with the nn-specific setting should get its own override newConf = new Configuration(conf); NameNode.initializeGenericKeys(newConf, "ns1", "nn1"); - assertEquals("nn1-override", newConf.get(key)); + assertEquals("nn1-override", newConf.get(key)); } - + /** * Tests for empty configuration, an exception is thrown from * {@link DFSUtil#getNNServiceRpcAddresses(Configuration)} @@ -456,7 +457,7 @@ public void testEmptyConf() { try { Map> map = - DFSUtil.getBackupNodeAddresses(conf); + DFSUtil.getBackupNodeAddresses(conf); fail("Expected IOException is not thrown, result was: " + DFSUtil.addressMapToString(map)); } catch (IOException expected) { @@ -465,7 +466,7 @@ public void testEmptyConf() { try { Map> map = - DFSUtil.getSecondaryNameNodeAddresses(conf); + DFSUtil.getSecondaryNameNodeAddresses(conf); fail("Expected IOException is not thrown, result was: " + DFSUtil.addressMapToString(map)); } catch (IOException expected) { @@ -502,11 +503,11 @@ public void testGetNamenodeWebAddr() { @Test public void testGetInfoServer() throws IOException, URISyntaxException { HdfsConfiguration conf = new HdfsConfiguration(); - + URI httpsport = DFSUtil.getInfoServer(null, conf, "https"); assertEquals(new URI("https", null, "0.0.0.0", DFS_NAMENODE_HTTPS_PORT_DEFAULT, null, null, null), httpsport); - + URI httpport = DFSUtil.getInfoServer(null, conf, "http"); assertEquals(new URI("http", null, "0.0.0.0", DFS_NAMENODE_HTTP_PORT_DEFAULT, null, null, null), httpport); @@ -517,17 +518,17 @@ public void testGetInfoServer() throws IOException, URISyntaxException { URI.create("http://localhost:" + DFS_NAMENODE_HTTP_PORT_DEFAULT), httpAddress); } - + @Test public void testHANameNodesWithFederation() throws URISyntaxException { HdfsConfiguration conf = new HdfsConfiguration(); - + final String NS1_NN1_HOST = "ns1-nn1.example.com:8020"; final String NS1_NN2_HOST = "ns1-nn2.example.com:8020"; final String NS2_NN1_HOST = "ns2-nn1.example.com:8020"; final String NS2_NN2_HOST = "ns2-nn2.example.com:8020"; conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "hdfs://ns1"); - + // Two nameservices, each with two NNs. conf.set(DFS_NAMESERVICES, "ns1,ns2"); conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, "ns1"), @@ -535,35 +536,35 @@ public void testHANameNodesWithFederation() throws URISyntaxException { conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, "ns2"), "ns2-nn1,ns2-nn2"); conf.set(DFSUtil.addKeySuffixes( - DFS_NAMENODE_RPC_ADDRESS_KEY, "ns1", "ns1-nn1"), + DFS_NAMENODE_RPC_ADDRESS_KEY, "ns1", "ns1-nn1"), NS1_NN1_HOST); conf.set(DFSUtil.addKeySuffixes( - DFS_NAMENODE_RPC_ADDRESS_KEY, "ns1", "ns1-nn2"), + DFS_NAMENODE_RPC_ADDRESS_KEY, "ns1", "ns1-nn2"), NS1_NN2_HOST); conf.set(DFSUtil.addKeySuffixes( - DFS_NAMENODE_RPC_ADDRESS_KEY, "ns2", "ns2-nn1"), + DFS_NAMENODE_RPC_ADDRESS_KEY, "ns2", "ns2-nn1"), NS2_NN1_HOST); conf.set(DFSUtil.addKeySuffixes( - DFS_NAMENODE_RPC_ADDRESS_KEY, "ns2", "ns2-nn2"), + DFS_NAMENODE_RPC_ADDRESS_KEY, "ns2", "ns2-nn2"), NS2_NN2_HOST); - + Map> map = DFSUtilClient.getHaNnRpcAddresses(conf); assertTrue(HAUtil.isHAEnabled(conf, "ns1")); assertTrue(HAUtil.isHAEnabled(conf, "ns2")); assertFalse(HAUtil.isHAEnabled(conf, "ns3")); - + assertEquals(NS1_NN1_HOST, map.get("ns1").get("ns1-nn1").toString()); assertEquals(NS1_NN2_HOST, map.get("ns1").get("ns1-nn2").toString()); assertEquals(NS2_NN1_HOST, map.get("ns2").get("ns2-nn1").toString()); assertEquals(NS2_NN2_HOST, map.get("ns2").get("ns2-nn2").toString()); - - assertEquals(NS1_NN1_HOST, + + assertEquals(NS1_NN1_HOST, DFSUtil.getNamenodeServiceAddr(conf, "ns1", "ns1-nn1")); - assertEquals(NS1_NN2_HOST, + assertEquals(NS1_NN2_HOST, DFSUtil.getNamenodeServiceAddr(conf, "ns1", "ns1-nn2")); - assertEquals(NS2_NN1_HOST, + assertEquals(NS2_NN1_HOST, DFSUtil.getNamenodeServiceAddr(conf, "ns2", "ns2-nn1")); // No nameservice was given and we can't determine which service addr @@ -587,15 +588,15 @@ public void testHANameNodesWithFederation() throws URISyntaxException { @Test public void getNameNodeServiceAddr() throws IOException { HdfsConfiguration conf = new HdfsConfiguration(); - + // One nameservice with two NNs final String NS1_NN1_HOST = "ns1-nn1.example.com:8020"; final String NS1_NN1_HOST_SVC = "ns1-nn2.example.com:9821"; final String NS1_NN2_HOST = "ns1-nn1.example.com:8020"; final String NS1_NN2_HOST_SVC = "ns1-nn2.example.com:9821"; - + conf.set(DFS_NAMESERVICES, "ns1"); - conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, "ns1"),"nn1,nn2"); + conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, "ns1"), "nn1,nn2"); conf.set(DFSUtil.addKeySuffixes( DFS_NAMENODE_RPC_ADDRESS_KEY, "ns1", "nn1"), NS1_NN1_HOST); @@ -609,7 +610,7 @@ public void getNameNodeServiceAddr() throws IOException { // A nameservice is specified explicitly assertEquals(NS1_NN1_HOST, DFSUtil.getNamenodeServiceAddr(conf, "ns1", "nn1")); assertEquals(null, DFSUtil.getNamenodeServiceAddr(conf, "invalid", "nn1")); - + // The service addrs are used when they are defined conf.set(DFSUtil.addKeySuffixes( DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, "ns1", "nn1"), NS1_NN1_HOST_SVC); @@ -641,7 +642,7 @@ private static Configuration createWebHDFSHAConfiguration(String logicalHostName HdfsConfiguration conf = new HdfsConfiguration(); conf.set(DFS_NAMESERVICES, "ns1"); - conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, "ns1"),"nn1,nn2"); + conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, "ns1"), "nn1,nn2"); conf.set(DFSUtil.addKeySuffixes( DFS_NAMENODE_HTTP_ADDRESS_KEY, "ns1", "nn1"), nnaddr1); conf.set(DFSUtil.addKeySuffixes( @@ -661,7 +662,7 @@ public void testSubstituteForWildcardAddress() throws IOException { } private static Collection getInternalNameServiceUris(Configuration conf, - String... keys) { + String... keys) { final Collection ids = DFSUtil.getInternalNameServices(conf); return DFSUtil.getNameServiceUris(conf, ids, keys); } @@ -671,14 +672,15 @@ private static Collection getInternalNameServiceUris(Configuration conf, * settings * @throws Exception */ + @SuppressWarnings("LocalFinalVariableName") @Test public void testGetNNUris() throws Exception { HdfsConfiguration conf = new HdfsConfiguration(); - final String NS2_NN_ADDR = "ns2-nn.example.com:8020"; - final String NN1_ADDR = "nn.example.com:8020"; - final String NN1_SRVC_ADDR = "nn.example.com:9821"; - final String NN2_ADDR = "nn2.example.com:8020"; + final String NS2_NN_ADDR = "ns2-nn.example.com:8020"; + final String NN1_ADDR = "nn.example.com:8020"; + final String NN1_SRVC_ADDR = "nn.example.com:9821"; + final String NN2_ADDR = "nn2.example.com:8020"; conf.set(DFS_NAMESERVICES, "ns1"); conf.set(DFSUtil.addKeySuffixes( @@ -687,11 +689,11 @@ public void testGetNNUris() throws Exception { conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "hdfs://" + NN1_ADDR); Collection uris = DFSUtil.getInternalNsRpcUris(conf); - assertEquals("Incorrect number of URIs returned", 2, uris.size()); - assertTrue("Missing URI for name service ns1", - uris.contains(new URI("hdfs://" + NS1_NN1_ADDR))); - assertTrue("Missing URI for service address", - uris.contains(new URI("hdfs://" + NN2_ADDR))); + assertEquals(2, uris.size(), "Incorrect number of URIs returned"); + assertTrue(uris.contains(new URI("hdfs://" + NS1_NN1_ADDR)), + "Missing URI for name service ns1"); + assertTrue(uris.contains(new URI("hdfs://" + NN2_ADDR)), + "Missing URI for service address"); conf = new HdfsConfiguration(); conf.set(DFS_NAMESERVICES, "ns1,ns2"); @@ -724,15 +726,13 @@ public void testGetNNUris() throws Exception { + "IPFailoverProxyProvider"); uris = DFSUtil.getInternalNsRpcUris(conf); - assertEquals("Incorrect number of URIs returned", 3, uris.size()); - assertTrue("Missing URI for RPC address", - uris.contains(new URI("hdfs://" + NN1_ADDR))); - assertTrue("Missing URI for name service ns2", - uris.contains(new URI(HdfsConstants.HDFS_URI_SCHEME + "://" + - NS1_NN_ADDR))); - assertTrue("Missing URI for name service ns2", - uris.contains(new URI(HdfsConstants.HDFS_URI_SCHEME + "://" + - NS2_NN_ADDR))); + assertEquals(3, uris.size(), "Incorrect number of URIs returned"); + assertTrue(uris.contains(new URI("hdfs://" + NN1_ADDR)), + "Missing URI for RPC address"); + assertTrue(uris.contains(new URI(HdfsConstants.HDFS_URI_SCHEME + "://" + + NS1_NN_ADDR)), "Missing URI for name service ns2"); + assertTrue(uris.contains(new URI(HdfsConstants.HDFS_URI_SCHEME + "://" + + NS2_NN_ADDR)), "Missing URI for name service ns2"); /** * Second, test ns1 with {@link ConfiguredFailoverProxyProvider} which does @@ -743,57 +743,57 @@ public void testGetNNUris() throws Exception { + "ConfiguredFailoverProxyProvider"); uris = DFSUtil.getInternalNsRpcUris(conf); - assertEquals("Incorrect number of URIs returned", 3, uris.size()); - assertTrue("Missing URI for name service ns1", - uris.contains(new URI("hdfs://ns1"))); - assertTrue("Missing URI for name service ns2", - uris.contains(new URI("hdfs://" + NS2_NN_ADDR))); - assertTrue("Missing URI for RPC address", - uris.contains(new URI("hdfs://" + NN1_ADDR))); + assertEquals(3, uris.size(), "Incorrect number of URIs returned"); + assertTrue(uris.contains(new URI("hdfs://ns1")), "" + + "Missing URI for name service ns1"); + assertTrue(uris.contains(new URI("hdfs://" + NS2_NN_ADDR)), + "Missing URI for name service ns2"); + assertTrue(uris.contains(new URI("hdfs://" + NN1_ADDR)), + "Missing URI for RPC address"); // Make sure that non-HDFS URIs in fs.defaultFS don't get included. conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "viewfs://vfs-name.example.com"); uris = DFSUtil.getInternalNsRpcUris(conf); - assertEquals("Incorrect number of URIs returned", 3, uris.size()); - assertTrue("Missing URI for name service ns1", - uris.contains(new URI("hdfs://ns1"))); - assertTrue("Missing URI for name service ns2", - uris.contains(new URI("hdfs://" + NS2_NN_ADDR))); - assertTrue("Missing URI for RPC address", - uris.contains(new URI("hdfs://" + NN1_ADDR))); + assertEquals(3, uris.size(), "Incorrect number of URIs returned"); + assertTrue(uris.contains(new URI("hdfs://ns1")), + "Missing URI for name service ns1"); + assertTrue(uris.contains(new URI("hdfs://" + NS2_NN_ADDR)), + "Missing URI for name service ns2"); + assertTrue(uris.contains(new URI("hdfs://" + NN1_ADDR)), + "Missing URI for RPC address"); // Make sure that an HA URI being the default URI doesn't result in multiple // entries being returned. conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "hdfs://ns1"); - + uris = DFSUtil.getInternalNsRpcUris(conf); - assertEquals("Incorrect number of URIs returned", 3, uris.size()); - assertTrue("Missing URI for name service ns1", - uris.contains(new URI("hdfs://ns1"))); - assertTrue("Missing URI for name service ns2", - uris.contains(new URI("hdfs://" + NS2_NN_ADDR))); - assertTrue("Missing URI for RPC address", - uris.contains(new URI("hdfs://" + NN1_ADDR))); + assertEquals(3, uris.size(), "Incorrect number of URIs returned"); + assertTrue(uris.contains(new URI("hdfs://ns1")), + "Missing URI for name service ns1"); + assertTrue(uris.contains(new URI("hdfs://" + NS2_NN_ADDR)), + "Missing URI for name service ns2"); + assertTrue(uris.contains(new URI("hdfs://" + NN1_ADDR)), + "Missing URI for RPC address"); // Check that the default URI is returned if there's nothing else to return. conf = new HdfsConfiguration(); conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "hdfs://" + NN1_ADDR); uris = DFSUtil.getInternalNsRpcUris(conf); - assertEquals("Incorrect number of URIs returned", 1, uris.size()); - assertTrue("Missing URI for RPC address (defaultFS)", - uris.contains(new URI("hdfs://" + NN1_ADDR))); + assertEquals(1, uris.size(), "Incorrect number of URIs returned"); + assertTrue(uris.contains(new URI("hdfs://" + NN1_ADDR)), + "Missing URI for RPC address (defaultFS)"); // Check that the RPC address is the only address returned when the RPC // and the default FS is given. conf.set(DFS_NAMENODE_RPC_ADDRESS_KEY, NN2_ADDR); uris = DFSUtil.getInternalNsRpcUris(conf); - assertEquals("Incorrect number of URIs returned", 1, uris.size()); - assertTrue("Missing URI for RPC address", - uris.contains(new URI("hdfs://" + NN2_ADDR))); + assertEquals(1, uris.size(), "Incorrect number of URIs returned"); + assertTrue(uris.contains(new URI("hdfs://" + NN2_ADDR)), + "Missing URI for RPC address"); // Make sure that when a service RPC address is used that is distinct from // the client RPC address, and that client RPC address is also used as the @@ -802,20 +802,20 @@ public void testGetNNUris() throws Exception { conf.set(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, NN1_ADDR); uris = DFSUtil.getInternalNsRpcUris(conf); - assertEquals("Incorrect number of URIs returned", 1, uris.size()); - assertTrue("Missing URI for service ns1", - uris.contains(new URI("hdfs://" + NN1_ADDR))); + assertEquals(1, uris.size(), "Incorrect number of URIs returned"); + assertTrue(uris.contains(new URI("hdfs://" + NN1_ADDR)), + "Missing URI for service ns1"); // Check that when the default FS and service address are given, but // the RPC address isn't, that only the service address is returned. conf = new HdfsConfiguration(); conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "hdfs://" + NN1_ADDR); conf.set(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, NN1_SRVC_ADDR); - + uris = DFSUtil.getInternalNsRpcUris(conf); - assertEquals("Incorrect number of URIs returned", 1, uris.size()); - assertTrue("Missing URI for service address", - uris.contains(new URI("hdfs://" + NN1_SRVC_ADDR))); + assertEquals(1, uris.size(), "Incorrect number of URIs returned"); + assertTrue(uris.contains(new URI("hdfs://" + NN1_SRVC_ADDR)), + "Missing URI for service address"); } @Test @@ -843,12 +843,13 @@ public void testGetNNUris2() throws Exception { Collection uris = DFSUtil.getInternalNsRpcUris(conf); - assertEquals("Incorrect number of URIs returned", 1, uris.size()); - assertTrue("Missing URI for name service ns1", - uris.contains(new URI("hdfs://ns1"))); + assertEquals(1, uris.size(), "Incorrect number of URIs returned"); + assertTrue(uris.contains(new URI("hdfs://ns1")), + "Missing URI for name service ns1"); } - @Test (timeout=15000) + @Test + @Timeout(value = 15) public void testLocalhostReverseLookup() { // 127.0.0.1 -> localhost reverse resolution does not happen on Windows. assumeNotWindows(); @@ -860,67 +861,71 @@ public void testLocalhostReverseLookup() { Collection uris = getInternalNameServiceUris(conf); assertEquals(1, uris.size()); for (URI uri : uris) { - assertThat(uri.getHost(), not("127.0.0.1")); + assertThat(uri.getHost()).isNotEqualTo("127.0.0.1"); } } - @Test (timeout=15000) + @Test + @Timeout(value = 15) public void testIsValidName() { String validPaths[] = new String[]{"/", "/bar/"}; for (String path : validPaths) { - assertTrue("Should have been accepted '" + path + "'", DFSUtil.isValidName(path)); + assertTrue(DFSUtil.isValidName(path), "Should have been accepted '" + path + "'"); } String invalidPaths[] = new String[]{"/foo/../bar", "/foo/./bar", "/foo//bar", "/foo/:/bar", "/foo:bar"}; for (String path : invalidPaths) { - assertFalse("Should have been rejected '" + path + "'", DFSUtil.isValidName(path)); + assertFalse(DFSUtil.isValidName(path), "Should have been rejected '" + path + "'"); } String windowsPath = "/C:/foo/bar"; if (Shell.WINDOWS) { - assertTrue("Should have been accepted '" + windowsPath + "' in windows os.", - DFSUtil.isValidName(windowsPath)); + assertTrue(DFSUtil.isValidName(windowsPath), "Should have been accepted '" + + windowsPath + "' in windows os."); } else { - assertFalse("Should have been rejected '" + windowsPath + "' in unix os.", - DFSUtil.isValidName(windowsPath)); + assertFalse(DFSUtil.isValidName(windowsPath), "Should have been rejected '" + + windowsPath + "' in unix os."); } } - - @Test(timeout=5000) + + @Test + @Timeout(value = 5) public void testGetSpnegoKeytabKey() { HdfsConfiguration conf = new HdfsConfiguration(); String defaultKey = "default.spengo.key"; conf.unset(DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY); - assertEquals("Test spnego key in config is null", defaultKey, - DFSUtil.getSpnegoKeytabKey(conf, defaultKey)); + assertEquals(defaultKey, DFSUtil.getSpnegoKeytabKey(conf, defaultKey), + "Test spnego key in config is null"); conf.set(DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY, ""); - assertEquals("Test spnego key is empty", defaultKey, - DFSUtil.getSpnegoKeytabKey(conf, defaultKey)); + assertEquals(defaultKey, DFSUtil.getSpnegoKeytabKey(conf, defaultKey), + "Test spnego key is empty"); String spengoKey = "spengo.key"; conf.set(DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY, spengoKey); - assertEquals("Test spnego key is NOT null", + assertEquals( DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY, - DFSUtil.getSpnegoKeytabKey(conf, defaultKey)); + DFSUtil.getSpnegoKeytabKey(conf, defaultKey), "Test spnego key is NOT null"); } - @Test(timeout=10000) + @Test + @Timeout(value = 10) public void testDurationToString() throws Exception { assertEquals("000:00:00:00.000", DFSUtil.durationToString(0)); assertEquals("001:01:01:01.000", - DFSUtil.durationToString(((24*60*60)+(60*60)+(60)+1)*1000)); + DFSUtil.durationToString(((24 * 60 * 60) + (60 * 60) + (60) + 1) * 1000)); assertEquals("000:23:59:59.999", - DFSUtil.durationToString(((23*60*60)+(59*60)+(59))*1000+999)); + DFSUtil.durationToString(((23 * 60 * 60) + (59 * 60) + (59)) * 1000 + 999)); assertEquals("-001:01:01:01.000", - DFSUtil.durationToString(-((24*60*60)+(60*60)+(60)+1)*1000)); + DFSUtil.durationToString(-((24 * 60 * 60) + (60 * 60) + (60) + 1) * 1000)); assertEquals("-000:23:59:59.574", - DFSUtil.durationToString(-(((23*60*60)+(59*60)+(59))*1000+574))); + DFSUtil.durationToString(-(((23 * 60 * 60) + (59 * 60) + (59)) * 1000 + 574))); } - @Test(timeout=5000) + @Test + @Timeout(value = 5) public void testRelativeTimeConversion() throws Exception { try { DFSUtil.parseRelativeTime("1"); @@ -937,14 +942,14 @@ public void testRelativeTimeConversion() throws Exception { } catch (IOException e) { assertExceptionContains("is not a number", e); } - assertEquals(61*1000, DFSUtil.parseRelativeTime("61s")); - assertEquals(61*60*1000, DFSUtil.parseRelativeTime("61m")); + assertEquals(61 * 1000, DFSUtil.parseRelativeTime("61s")); + assertEquals(61 * 60 * 1000, DFSUtil.parseRelativeTime("61m")); assertEquals(0, DFSUtil.parseRelativeTime("0s")); - assertEquals(25*60*60*1000, DFSUtil.parseRelativeTime("25h")); - assertEquals(4*24*60*60*1000l, DFSUtil.parseRelativeTime("4d")); - assertEquals(999*24*60*60*1000l, DFSUtil.parseRelativeTime("999d")); + assertEquals(25 * 60 * 60 * 1000, DFSUtil.parseRelativeTime("25h")); + assertEquals(4 * 24 * 60 * 60 * 1000L, DFSUtil.parseRelativeTime("4d")); + assertEquals(999 * 24 * 60 * 60 * 1000L, DFSUtil.parseRelativeTime("999d")); } - + @Test public void testAssertAllResultsEqual() { checkAllResults(new Long[]{}, true); @@ -953,7 +958,7 @@ public void testAssertAllResultsEqual() { checkAllResults(new Long[]{1l, 1l, 1l}, true); checkAllResults(new Long[]{new Long(1), new Long(1)}, true); checkAllResults(new Long[]{null, null, null}, true); - + checkAllResults(new Long[]{1l, 2l}, false); checkAllResults(new Long[]{2l, 1l}, false); checkAllResults(new Long[]{1l, 2l, 1l}, false); @@ -963,7 +968,7 @@ public void testAssertAllResultsEqual() { checkAllResults(new Long[]{null, 1l}, false); checkAllResults(new Long[]{1l, null, 1l}, false); } - + private static void checkAllResults(Long[] toCheck, boolean shouldSucceed) { if (shouldSucceed) { DFSUtil.assertAllResultsEqual(Arrays.asList(toCheck)); @@ -985,7 +990,7 @@ public void testGetPassword() throws Exception { Configuration conf = new Configuration(); final Path jksPath = new Path(testDir.toString(), "test.jks"); final String ourUrl = - JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri(); + JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri(); File file = new File(testDir, "test.jks"); file.delete(); @@ -1031,15 +1036,15 @@ public void testGetPassword() throws Exception { DFS_SERVER_HTTPS_TRUSTSTORE_PASSWORD_KEY).getCredential()); // use WebAppUtils as would be used by loadSslConfiguration - Assert.assertEquals("keypass", + assertEquals("keypass", DFSUtil.getPassword(conf, DFS_SERVER_HTTPS_KEYPASSWORD_KEY)); - Assert.assertEquals("storepass", + assertEquals("storepass", DFSUtil.getPassword(conf, DFS_SERVER_HTTPS_KEYSTORE_PASSWORD_KEY)); - Assert.assertEquals("trustpass", + assertEquals("trustpass", DFSUtil.getPassword(conf, DFS_SERVER_HTTPS_TRUSTSTORE_PASSWORD_KEY)); // let's make sure that a password that doesn't exist returns null - Assert.assertEquals(null, DFSUtil.getPassword(conf,"invalid-alias")); + assertEquals(null, DFSUtil.getPassword(conf, "invalid-alias")); } @Test @@ -1051,9 +1056,9 @@ public void testGetNNServiceRpcAddressesForNsIds() throws IOException { final String NN1_ADDRESS = "localhost:9000"; final String NN2_ADDRESS = "localhost:9001"; conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY, "nn1"), - NN1_ADDRESS); + NN1_ADDRESS); conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY, "nn2"), - NN2_ADDRESS); + NN2_ADDRESS); { Collection internal = DFSUtil.getInternalNameServices(conf); @@ -1064,7 +1069,7 @@ public void testGetNNServiceRpcAddressesForNsIds() throws IOException { } Map> nnMap = DFSUtil - .getNNServiceRpcAddressesForCluster(conf); + .getNNServiceRpcAddressesForCluster(conf); assertEquals(1, nnMap.size()); assertTrue(nnMap.containsKey("nn1")); @@ -1080,20 +1085,21 @@ public void testGetNNServiceRpcAddressesForNsIds() throws IOException { public void testEncryptionProbe() throws Throwable { Configuration conf = new Configuration(false); conf.unset(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH); - assertFalse("encryption enabled on no provider key", - DFSUtilClient.isHDFSEncryptionEnabled(conf)); + assertFalse( + DFSUtilClient.isHDFSEncryptionEnabled(conf), + "encryption enabled on no provider key"); conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH, ""); - assertFalse("encryption enabled on empty provider key", - DFSUtilClient.isHDFSEncryptionEnabled(conf)); + assertFalse(DFSUtilClient.isHDFSEncryptionEnabled(conf), + "encryption enabled on empty provider key"); conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH, "\n\t\n"); - assertFalse("encryption enabled on whitespace provider key", - DFSUtilClient.isHDFSEncryptionEnabled(conf)); + assertFalse(DFSUtilClient.isHDFSEncryptionEnabled(conf), + "encryption enabled on whitespace provider key"); conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH, "http://hadoop.apache.org"); - assertTrue("encryption disabled on valid provider key", - DFSUtilClient.isHDFSEncryptionEnabled(conf)); + assertTrue(DFSUtilClient.isHDFSEncryptionEnabled(conf), + "encryption disabled on valid provider key"); } @@ -1101,8 +1107,8 @@ public void testEncryptionProbe() throws Throwable { public void testFileIdPath() throws Throwable { // /.reserved/.inodes/ String prefix = Path.SEPARATOR + HdfsConstants.DOT_RESERVED_STRING + - Path.SEPARATOR + HdfsConstants.DOT_INODES_STRING + - Path.SEPARATOR; + Path.SEPARATOR + HdfsConstants.DOT_INODES_STRING + + Path.SEPARATOR; Random r = new Random(); for (int i = 0; i < 100; ++i) { long inode = r.nextLong() & Long.MAX_VALUE; @@ -1116,12 +1122,12 @@ public void testErrorMessageForInvalidNameservice() throws Exception { Configuration conf = new HdfsConfiguration(); conf.set(DFSConfigKeys.DFS_NAMESERVICES, "ns1, ns2"); String expectedErrorMessage = "Incorrect configuration: namenode address " - + DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY + ".[ns1, ns2]" - + " or " - + DFS_NAMENODE_RPC_ADDRESS_KEY + ".[ns1, ns2]" - + " is not configured."; + + DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY + ".[ns1, ns2]" + + " or " + + DFS_NAMENODE_RPC_ADDRESS_KEY + ".[ns1, ns2]" + + " is not configured."; LambdaTestUtils.intercept(IOException.class, expectedErrorMessage, - ()->DFSUtil.getNNServiceRpcAddressesForCluster(conf)); + () -> DFSUtil.getNNServiceRpcAddressesForCluster(conf)); } @Test @@ -1198,7 +1204,7 @@ private void testLazyResolved(boolean isLazy) { if (isLazy) { // Lazy resolved. There is no need to change host->ip in advance. assertTrue(inetSocketAddress.isUnresolved()); - }else { + } else { // Need resolve all host->ip. assertFalse(inetSocketAddress.isUnresolved()); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDisableConnCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDisableConnCache.java index 51a28d294f9ba..567f89778904c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDisableConnCache.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDisableConnCache.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hdfs; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -25,7 +25,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; import org.apache.hadoop.hdfs.client.impl.BlockReaderTestUtil; -import org.junit.Test; +import org.junit.jupiter.api.Test; /** * This class tests disabling client connection caching in a single node diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java index 669224818f07c..158f454fe5409 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java @@ -23,12 +23,12 @@ import static org.apache.hadoop.hdfs.client.HdfsAdmin.TRASH_PERMISSION; import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CONTEXT; import static org.assertj.core.api.Assertions.assertThat; -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.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.anyString; @@ -132,8 +132,8 @@ import org.apache.hadoop.util.Time; import org.apache.hadoop.util.concurrent.HadoopExecutors; import org.apache.hadoop.util.functional.RemoteIterators; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.mockito.InOrder; import org.mockito.Mockito; import org.slf4j.Logger; @@ -234,13 +234,13 @@ public void testDFSClose() throws Exception { types.add(OpenFilesIterator.OpenFilesType.ALL_OPEN_FILES); RemoteIterator listOpenFiles = fileSys.listOpenFiles(EnumSet.copyOf(types)); - assertTrue("Two files should be open", listOpenFiles.hasNext()); + assertTrue(listOpenFiles.hasNext(), "Two files should be open"); int countOpenFiles = 0; while (listOpenFiles.hasNext()) { listOpenFiles.next(); ++countOpenFiles; } - assertEquals("Mismatch of open files count", 2, countOpenFiles); + assertEquals(2, countOpenFiles, "Mismatch of open files count"); // create another file, close it, and read it, so // the client gets a socket in its SocketCache @@ -452,7 +452,7 @@ public void testDFSSeekExceptions() throws IOException { // success threw = true; } - assertTrue("Failed to throw IOE when seeking past end", threw); + assertTrue(threw, "Failed to throw IOE when seeking past end"); input.close(); threw = false; try { @@ -461,7 +461,7 @@ public void testDFSSeekExceptions() throws IOException { //success threw = true; } - assertTrue("Failed to throw IOE when seeking after close", threw); + assertTrue(threw, "Failed to throw IOE when seeking after close"); fileSys.close(); } finally { @@ -570,7 +570,7 @@ public void testDFSClient() throws Exception { // Check to see if opening a non-existent file triggers a FNF FileSystem fs = cluster.getFileSystem(); Path dir = new Path("/wrwelkj"); - assertFalse("File should not exist for test.", fs.exists(dir)); + assertFalse(fs.exists(dir), "File should not exist for test."); try { FSDataInputStream in = fs.open(dir); @@ -578,8 +578,8 @@ public void testDFSClient() throws Exception { in.close(); fs.close(); } finally { - assertTrue("Did not get a FileNotFoundException for non-existing" + - " file.", false); + assertTrue(false, + "Did not get a FileNotFoundException for non-existing" + " file."); } } catch (FileNotFoundException fnf) { // This is the proper exception to catch; move on. @@ -629,11 +629,11 @@ public void testDFSClient() throws Exception { fs.create(new Path("/tmp/nonEmptyDir/emptyFile")).close(); try { fs.delete(new Path("/tmp/nonEmptyDir"), false); - Assert.fail("Expecting PathIsNotEmptyDirectoryException"); + fail("Expecting PathIsNotEmptyDirectoryException"); } catch (PathIsNotEmptyDirectoryException ex) { // This is the proper exception to catch; move on. } - Assert.assertTrue(fs.exists(new Path("/test/nonEmptyDir"))); + assertTrue(fs.exists(new Path("/test/nonEmptyDir"))); fs.delete(new Path("/tmp/nonEmptyDir"), true); } @@ -1057,7 +1057,8 @@ public void testECStatistics() throws IOException { } @SuppressWarnings("ThrowableResultOfMethodCallIgnored") - @Test (timeout = 180000) + @Test + @Timeout(value = 180) public void testConcurrentStatistics() throws IOException, InterruptedException { FileSystem.getStatistics(HdfsConstants.HDFS_URI_SCHEME, @@ -1106,8 +1107,7 @@ public void run() { // wait until all threads are done allDone.await(); - assertNull("Child failed with exception " + childError.get(), - childError.get()); + assertNull(childError.get(), "Child failed with exception " + childError.get()); checkStatistics(fs, 0, numThreads, 0); // check the single operation count stat @@ -1119,8 +1119,8 @@ public void run() { opCountIter.hasNext();) { final LongStatistic opCount = opCountIter.next(); if (OpType.MKDIRS.getSymbol().equals(opCount.getName())) { - assertEquals("Unexpected op count from iterator!", - numThreads + oldMkdirOpCount, opCount.getValue()); + assertEquals(numThreads + oldMkdirOpCount, opCount.getValue(), + "Unexpected op count from iterator!"); } LOG.info(opCount.getName() + "\t" + opCount.getValue()); } @@ -1238,8 +1238,8 @@ private void testReadFileSystemStatistics(int expectedDistance, } public static void checkOpStatistics(OpType op, long count) { - assertEquals("Op " + op.getSymbol() + " has unexpected count!", - count, getOpStatistics(op)); + assertEquals(count, getOpStatistics(op), "Op " + op.getSymbol() + + " has unexpected count!"); } public static long getOpStatistics(OpType op) { @@ -1270,8 +1270,8 @@ public void testFileChecksum() throws Exception { "/test/TestNonExistingFile")); fail("Expecting FileNotFoundException"); } catch (FileNotFoundException e) { - assertTrue("Not throwing the intended exception message", e.getMessage() - .contains("File does not exist: /test/TestNonExistingFile")); + assertTrue(e.getMessage().contains("File does not exist: /test/TestNonExistingFile"), + "Not throwing the intended exception message"); } try { @@ -1280,8 +1280,8 @@ public void testFileChecksum() throws Exception { hdfs.getFileChecksum(path); fail("Expecting FileNotFoundException"); } catch (FileNotFoundException e) { - assertTrue("Not throwing the intended exception message", e.getMessage() - .contains("Path is not a file: /test/TestExistingDir")); + assertTrue(e.getMessage().contains("Path is not a file: /test/TestExistingDir"), + "Not throwing the intended exception message"); } //webhdfs @@ -1417,7 +1417,8 @@ public void testAllWithNoXmlDefaults() throws Exception { } } - @Test(timeout=120000) + @Test + @Timeout(value = 120) public void testLocatedFileStatusStorageIdsTypes() throws Exception { final Configuration conf = getTestConfiguration(); final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf) @@ -1434,10 +1435,10 @@ public void testLocatedFileStatusStorageIdsTypes() throws Exception { DFSTestUtil.waitForReplication(fs, testFile, (short) repl, 30000); // Get the listing RemoteIterator it = fs.listLocatedStatus(testFile); - assertTrue("Expected file to be present", it.hasNext()); + assertTrue(it.hasNext(), "Expected file to be present"); LocatedFileStatus stat = it.next(); BlockLocation[] locs = stat.getBlockLocations(); - assertEquals("Unexpected number of locations", numBlocks, locs.length); + assertEquals(numBlocks, locs.length, "Unexpected number of locations"); Set dnStorageIds = new HashSet<>(); for (DataNode d : cluster.getDataNodes()) { @@ -1454,15 +1455,14 @@ public void testLocatedFileStatusStorageIdsTypes() throws Exception { // Run it through a set to deduplicate, since there should be no dupes Set storageIds = new HashSet<>(); Collections.addAll(storageIds, ids); - assertEquals("Unexpected num storage ids", repl, storageIds.size()); + assertEquals(repl, storageIds.size(), "Unexpected num storage ids"); // Make sure these are all valid storage IDs - assertTrue("Unknown storage IDs found!", dnStorageIds.containsAll - (storageIds)); + assertTrue(dnStorageIds.containsAll(storageIds), "Unknown storage IDs found!"); // Check storage types are the default, since we didn't set any StorageType[] types = loc.getStorageTypes(); - assertEquals("Unexpected num storage types", repl, types.length); + assertEquals(repl, types.length, "Unexpected num storage types"); for (StorageType t: types) { - assertEquals("Unexpected storage type", StorageType.DEFAULT, t); + assertEquals(StorageType.DEFAULT, t, "Unexpected storage type"); } } } finally { @@ -1528,7 +1528,8 @@ public void testCreateWithCustomChecksum() throws Exception { } } - @Test(timeout=60000) + @Test + @Timeout(value = 60) public void testFileCloseStatus() throws IOException { Configuration conf = getTestConfiguration(); MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build(); @@ -1540,9 +1541,9 @@ public void testFileCloseStatus() throws IOException { // write to file output.writeBytes("Some test data"); output.flush(); - assertFalse("File status should be open", fs.isFileClosed(file)); + assertFalse(fs.isFileClosed(file), "File status should be open"); output.close(); - assertTrue("File status should be closed", fs.isFileClosed(file)); + assertTrue(fs.isFileClosed(file), "File status should be closed"); } finally { cluster.shutdown(); } @@ -1587,7 +1588,8 @@ public void testCreateWithStoragePolicy() throws Throwable { } } - @Test(timeout=60000) + @Test + @Timeout(value = 60) public void testListFiles() throws IOException { Configuration conf = getTestConfiguration(); MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build(); @@ -1679,7 +1681,8 @@ public void testListStatusOfSnapshotDirs() throws IOException { } } - @Test(timeout=10000) + @Test + @Timeout(value = 10) public void testDFSClientPeerReadTimeout() throws IOException { final int timeout = 1000; final Configuration conf = getTestConfiguration(); @@ -1697,7 +1700,7 @@ public void testDFSClientPeerReadTimeout() throws IOException { long start = Time.now(); try { peer.getInputStream().read(); - Assert.fail("read should timeout"); + fail("read should timeout"); } catch (SocketTimeoutException ste) { long delta = Time.now() - start; if (delta < timeout*0.9) { @@ -1714,7 +1717,8 @@ public void testDFSClientPeerReadTimeout() throws IOException { } } - @Test(timeout=60000) + @Test + @Timeout(value = 60) public void testGetServerDefaults() throws IOException { Configuration conf = getTestConfiguration(); MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build(); @@ -1728,7 +1732,8 @@ public void testGetServerDefaults() throws IOException { } } - @Test(timeout=10000) + @Test + @Timeout(value = 10) public void testDFSClientPeerWriteTimeout() throws IOException { final int timeout = 1000; final Configuration conf = getTestConfiguration(); @@ -1748,7 +1753,7 @@ public void testDFSClientPeerWriteTimeout() throws IOException { byte[] buf = new byte[10 * 1024 * 1024]; peer.getOutputStream().write(buf); long delta = Time.now() - start; - Assert.fail("write finish in " + delta + " ms" + "but should timedout"); + fail("write finish in " + delta + " ms" + "but should timedout"); } catch (SocketTimeoutException ste) { long delta = Time.now() - start; @@ -1766,7 +1771,8 @@ public void testDFSClientPeerWriteTimeout() throws IOException { } } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testTotalDfsUsed() throws Exception { Configuration conf = getTestConfiguration(); MiniDFSCluster cluster = null; @@ -1906,15 +1912,15 @@ public void testDFSDataOutputStreamBuilderForCreation() throws Exception { } catch (FileNotFoundException e) { // As expected. } - assertFalse("parent directory should not be created", - fs.exists(new Path("/parent"))); + assertFalse(fs.exists(new Path("/parent")), + "parent directory should not be created"); try (FSDataOutputStream out = fs.createFile(nonParentFile).recursive() .build()) { out.write(1); } - assertTrue("parent directory has not been created", - fs.exists(new Path("/parent"))); + assertTrue(fs.exists(new Path("/parent")), + "parent directory has not been created"); } } @@ -2108,7 +2114,7 @@ public void testEnableAndDisableErasureCodingPolicy() throws Exception { //test enable a policy that doesn't exist try { fs.enableErasureCodingPolicy("notExistECName"); - Assert.fail("enable the policy that doesn't exist should fail"); + fail("enable the policy that doesn't exist should fail"); } catch (Exception e) { GenericTestUtils.assertExceptionContains("does not exist", e); // pass @@ -2117,7 +2123,7 @@ public void testEnableAndDisableErasureCodingPolicy() throws Exception { //test disable a policy that doesn't exist try { fs.disableErasureCodingPolicy("notExistECName"); - Assert.fail("disable the policy that doesn't exist should fail"); + fail("disable the policy that doesn't exist should fail"); } catch (Exception e) { GenericTestUtils.assertExceptionContains("does not exist", e); // pass @@ -2179,7 +2185,7 @@ public void testStorageFavouredNodes() .getBlockLocations(file1.toUri().getPath(), 0, Long.MAX_VALUE); int numSSD = Collections.frequency( Arrays.asList(locations[0].getStorageTypes()), StorageType.SSD); - assertEquals("Number of SSD should be 1 but was : " + numSSD, 1, numSSD); + assertEquals(1, numSSD, "Number of SSD should be 1 but was : " + numSSD); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystemWithECFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystemWithECFile.java index 1a2c4de3974f6..727d8a4c1d2cc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystemWithECFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystemWithECFile.java @@ -28,20 +28,19 @@ import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies; import org.apache.hadoop.io.IOUtils; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.Timeout; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.io.IOException; import java.io.InputStream; import java.util.ArrayList; import java.util.List; -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; /** * Testing correctness of FileSystem.getFileBlockLocations and @@ -66,10 +65,7 @@ public ErasureCodingPolicy getEcPolicy() { return StripedFileTestUtil.getDefaultECPolicy(); } - @Rule - public final Timeout globalTimeout = new Timeout(60000 * 3); - - @Before + @BeforeEach public void setup() throws IOException { ecPolicy = getEcPolicy(); cellSize = ecPolicy.getCellSize(); @@ -92,7 +88,7 @@ public void setup() throws IOException { ecPolicy.getName()); } - @After + @AfterEach public void tearDown() throws IOException { if (cluster != null) { cluster.shutdown(); @@ -108,7 +104,8 @@ private void createFile(String path, int size) throws Exception { StripedFileTestUtil.verifyLength(fs, src, size); } - @Test(timeout=60000) + @Test + @Timeout(value = 60) public void testListECFilesSmallerThanOneCell() throws Exception { createFile("/ec/smallcell", 1); final List retVal = new ArrayList<>(); @@ -142,7 +139,8 @@ private void assertSmallerThanOneCell(BlockLocation[] locations) assertTrue(blockLocation.getHosts().length == 1 + parityBlocks); } - @Test(timeout=60000) + @Test + @Timeout(value = 60) public void testListECFilesSmallerThanOneStripe() throws Exception { int dataBlocksNum = dataBlocks; createFile("/ec/smallstripe", cellSize * dataBlocksNum); @@ -173,7 +171,8 @@ private void assertSmallerThanOneStripe(BlockLocation[] locations, assertTrue(blockLocation.getLength() == dataBlocksNum * cellSize); } - @Test(timeout=60000) + @Test + @Timeout(value = 60) public void testListECFilesMoreThanOneBlockGroup() throws Exception { createFile("/ec/group", blockGroupSize + 123); RemoteIterator iter = @@ -207,7 +206,8 @@ private void assertMoreThanOneBlockGroup(BlockLocation[] locations, assertTrue(lastBlock.getLength() == lastBlockSize); } - @Test(timeout=60000) + @Test + @Timeout(value = 60) public void testReplayEditLogsForReplicatedFile() throws Exception { cluster.shutdown(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptedTransfer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptedTransfer.java index f9336fcfdc74e..3f57913eadf33 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptedTransfer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptedTransfer.java @@ -17,10 +17,10 @@ */ package org.apache.hadoop.hdfs; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.Mockito.times; import java.io.IOException; @@ -33,6 +33,9 @@ import java.util.concurrent.TimeoutException; import java.util.function.Supplier; + +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -53,19 +56,10 @@ import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils.LogCapturer; import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.Assert; -import org.junit.rules.Timeout; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; +import org.junit.jupiter.api.AfterEach; import org.mockito.Mockito; import org.slf4j.event.Level; -@RunWith(Parameterized.class) public class TestEncryptedTransfer { { GenericTestUtils.setLogLevel( @@ -73,11 +67,7 @@ public class TestEncryptedTransfer { GenericTestUtils.setLogLevel( LoggerFactory.getLogger(DataTransferSaslUtil.class), Level.DEBUG); } - - @Rule - public Timeout timeout = new Timeout(300000); - @Parameters public static Collection data() { Collection params = new ArrayList(); params.add(new Object[]{null}); @@ -114,16 +104,16 @@ private static FileSystem getFileSystem(Configuration conf) throws IOException { } String resolverClazz; - public TestEncryptedTransfer(String resolverClazz){ - this.resolverClazz = resolverClazz; + public void initTestEncryptedTransfer(String pResolverClazz) throws IOException { + this.resolverClazz = pResolverClazz; + setup(); } - @Before public void setup() throws IOException { conf = new Configuration(); } - @After + @AfterEach public void teardown() throws IOException { if (fs != null) { fs.close(); @@ -206,29 +196,39 @@ private void testEncryptedRead(String algorithm, String cipherSuite, } } - @Test - public void testEncryptedReadDefaultAlgorithmCipherSuite() + @MethodSource("data") + @ParameterizedTest + public void testEncryptedReadDefaultAlgorithmCipherSuite(String pResolverClazz) throws IOException { + initTestEncryptedTransfer(pResolverClazz); testEncryptedRead("", "", false, false); } - @Test - public void testEncryptedReadWithRC4() throws IOException { + @MethodSource("data") + @ParameterizedTest + public void testEncryptedReadWithRC4(String pResolverClazz) throws IOException { + initTestEncryptedTransfer(pResolverClazz); testEncryptedRead("rc4", "", false, false); } - @Test - public void testEncryptedReadWithAES() throws IOException { + @MethodSource("data") + @ParameterizedTest + public void testEncryptedReadWithAES(String pResolverClazz) throws IOException { + initTestEncryptedTransfer(pResolverClazz); testEncryptedRead("", "AES/CTR/NoPadding", true, false); } - @Test - public void testEncryptedReadAfterNameNodeRestart() throws IOException { + @MethodSource("data") + @ParameterizedTest + public void testEncryptedReadAfterNameNodeRestart(String pResolverClazz) throws IOException { + initTestEncryptedTransfer(pResolverClazz); testEncryptedRead("", "", false, true); } - @Test - public void testClientThatDoesNotSupportEncryption() throws IOException { + @MethodSource("data") + @ParameterizedTest + public void testClientThatDoesNotSupportEncryption(String pResolverClazz) throws IOException { + initTestEncryptedTransfer(pResolverClazz); // Set short retry timeouts so this test runs faster conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 10); @@ -260,8 +260,10 @@ public void testClientThatDoesNotSupportEncryption() throws IOException { } } - @Test - public void testLongLivedReadClientAfterRestart() throws IOException { + @MethodSource("data") + @ParameterizedTest + public void testLongLivedReadClientAfterRestart(String pResolverClazz) throws IOException { + initTestEncryptedTransfer(pResolverClazz); FileChecksum checksum = writeUnencryptedAndThenRestartEncryptedCluster(); assertEquals(PLAIN_TEXT, DFSTestUtil.readFile(fs, TEST_PATH)); @@ -276,8 +278,10 @@ public void testLongLivedReadClientAfterRestart() throws IOException { assertEquals(checksum, fs.getFileChecksum(TEST_PATH)); } - @Test - public void testLongLivedWriteClientAfterRestart() throws IOException { + @MethodSource("data") + @ParameterizedTest + public void testLongLivedWriteClientAfterRestart(String pResolverClazz) throws IOException { + initTestEncryptedTransfer(pResolverClazz); setEncryptionConfigKeys(); cluster = new MiniDFSCluster.Builder(conf).build(); @@ -295,9 +299,11 @@ public void testLongLivedWriteClientAfterRestart() throws IOException { writeTestDataToFile(fs); assertEquals(PLAIN_TEXT + PLAIN_TEXT, DFSTestUtil.readFile(fs, TEST_PATH)); } - - @Test - public void testLongLivedClient() throws IOException, InterruptedException { + + @MethodSource("data") + @ParameterizedTest + public void testLongLivedClient(String pResolverClazz) throws IOException, InterruptedException { + initTestEncryptedTransfer(pResolverClazz); FileChecksum checksum = writeUnencryptedAndThenRestartEncryptedCluster(); BlockTokenSecretManager btsm = cluster.getNamesystem().getBlockManager() @@ -321,9 +327,11 @@ public void testLongLivedClient() throws IOException, InterruptedException { assertEquals(checksum, fs.getFileChecksum(TEST_PATH)); } - @Test - public void testFileChecksumWithInvalidEncryptionKey() + @MethodSource("data") + @ParameterizedTest + public void testFileChecksumWithInvalidEncryptionKey(String pResolverClazz) throws IOException, InterruptedException, TimeoutException { + initTestEncryptedTransfer(pResolverClazz); if (resolverClazz != null) { // TestTrustedChannelResolver does not use encryption keys. return; @@ -365,16 +373,18 @@ public Boolean get() { LOG.info("The encryption key is invalid on all nodes now."); fs.getFileChecksum(TEST_PATH); // verify that InvalidEncryptionKeyException is handled properly - Assert.assertTrue(client.getEncryptionKey() == null); + assertTrue(client.getEncryptionKey() == null); Mockito.verify(spyClient, times(1)).clearDataEncryptionKey(); // Retry the operation after clearing the encryption key FileChecksum verifyChecksum = fs.getFileChecksum(TEST_PATH); - Assert.assertEquals(checksum, verifyChecksum); + assertEquals(checksum, verifyChecksum); } - @Test - public void testLongLivedClientPipelineRecovery() + @MethodSource("data") + @ParameterizedTest + public void testLongLivedClientPipelineRecovery(String pResolverClazz) throws IOException, InterruptedException, TimeoutException { + initTestEncryptedTransfer(pResolverClazz); if (resolverClazz != null) { // TestTrustedChannelResolver does not use encryption keys. return; @@ -428,25 +438,31 @@ public Boolean get() { // write data to induce pipeline recovery out.write(PLAIN_TEXT.getBytes()); out.hflush(); - assertFalse("The first datanode in the pipeline was not replaced.", - Arrays.asList(dfstream.getPipeline()).contains(targets[0])); + assertFalse(Arrays.asList(dfstream.getPipeline()).contains(targets[0]), + "The first datanode in the pipeline was not replaced."); } // verify that InvalidEncryptionKeyException is handled properly Mockito.verify(spyClient, times(1)).clearDataEncryptionKey(); } - @Test - public void testEncryptedWriteWithOneDn() throws IOException { + @MethodSource("data") + @ParameterizedTest + public void testEncryptedWriteWithOneDn(String pResolverClazz) throws IOException { + initTestEncryptedTransfer(pResolverClazz); testEncryptedWrite(1); } - @Test - public void testEncryptedWriteWithTwoDns() throws IOException { + @MethodSource("data") + @ParameterizedTest + public void testEncryptedWriteWithTwoDns(String pResolverClazz) throws IOException { + initTestEncryptedTransfer(pResolverClazz); testEncryptedWrite(2); } - @Test - public void testEncryptedWriteWithMultipleDns() throws IOException { + @MethodSource("data") + @ParameterizedTest + public void testEncryptedWriteWithMultipleDns(String pResolverClazz) throws IOException { + initTestEncryptedTransfer(pResolverClazz); testEncryptedWrite(10); } @@ -479,8 +495,10 @@ private void testEncryptedWrite(int numDns) throws IOException { } } - @Test - public void testEncryptedAppend() throws IOException { + @MethodSource("data") + @ParameterizedTest + public void testEncryptedAppend(String pResolverClazz) throws IOException { + initTestEncryptedTransfer(pResolverClazz); setEncryptionConfigKeys(); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build(); @@ -494,8 +512,10 @@ public void testEncryptedAppend() throws IOException { assertEquals(PLAIN_TEXT + PLAIN_TEXT, DFSTestUtil.readFile(fs, TEST_PATH)); } - @Test - public void testEncryptedAppendRequiringBlockTransfer() throws IOException { + @MethodSource("data") + @ParameterizedTest + public void testEncryptedAppendRequiringBlockTransfer(String pResolverClazz) throws IOException { + initTestEncryptedTransfer(pResolverClazz); setEncryptionConfigKeys(); // start up 4 DNs diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesWithHA.java index bb8b79b723f9f..063021ae32069 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesWithHA.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesWithHA.java @@ -28,15 +28,17 @@ import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil; import org.apache.hadoop.hdfs.client.HdfsAdmin; import org.apache.hadoop.hdfs.server.namenode.NameNode; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.io.File; import java.io.IOException; import java.util.EnumSet; +import static org.junit.jupiter.api.Assertions.assertEquals; + /** * Tests interaction of encryption zones with HA failover. */ @@ -55,7 +57,7 @@ public class TestEncryptionZonesWithHA { protected static final EnumSet< CreateEncryptionZoneFlag > NO_TRASH = EnumSet.of(CreateEncryptionZoneFlag.NO_TRASH); - @Before + @BeforeEach public void setupCluster() throws Exception { conf = new Configuration(); conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1); @@ -87,7 +89,7 @@ public void setupCluster() throws Exception { fs.getClient().setKeyProvider(nn0Provider); } - @After + @AfterEach public void shutdownCluster() throws IOException { if (cluster != null) { cluster.shutdown(); @@ -98,7 +100,8 @@ public void shutdownCluster() throws IOException { /** * Test that encryption zones are properly tracked by the standby. */ - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testEncryptionZonesTrackedOnStandby() throws Exception { final int len = 8196; final Path dir = new Path("/enc"); @@ -115,12 +118,12 @@ public void testEncryptionZonesTrackedOnStandby() throws Exception { cluster.shutdownNameNode(0); cluster.transitionToActive(1); - Assert.assertEquals("Got unexpected ez path", dir.toString(), - dfsAdmin1.getEncryptionZoneForPath(dir).getPath().toString()); - Assert.assertEquals("Got unexpected ez path", dir.toString(), - dfsAdmin1.getEncryptionZoneForPath(dirChild).getPath().toString()); - Assert.assertEquals("File contents after failover were changed", - contents, DFSTestUtil.readFile(fs, dirFile)); + assertEquals(dir.toString(), dfsAdmin1.getEncryptionZoneForPath(dir).getPath().toString(), + "Got unexpected ez path"); + assertEquals(dir.toString(), dfsAdmin1.getEncryptionZoneForPath(dirChild).getPath().toString(), + "Got unexpected ez path"); + assertEquals(contents, DFSTestUtil.readFile(fs, dirFile), + "File contents after failover were changed"); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodeBenchmarkThroughput.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodeBenchmarkThroughput.java index da3407d2fc12a..2b0744a17be63 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodeBenchmarkThroughput.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodeBenchmarkThroughput.java @@ -23,27 +23,26 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.util.ToolRunner; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.Rule; -import org.junit.rules.Timeout; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import java.io.IOException; +import org.junit.jupiter.api.Timeout; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; /** * To test {@link org.apache.hadoop.hdfs.ErasureCodeBenchmarkThroughput}. */ +@Timeout(300) public class TestErasureCodeBenchmarkThroughput { private static MiniDFSCluster cluster; private static Configuration conf; private static FileSystem fs; - @Rule - public Timeout globalTimeout = new Timeout(300000); - - @BeforeClass + @BeforeAll public static void setup() throws IOException { conf = new HdfsConfiguration(); int numDN = ErasureCodeBenchmarkThroughput.getEcPolicy().getNumDataUnits() + @@ -55,7 +54,7 @@ public static void setup() throws IOException { ErasureCodeBenchmarkThroughput.getEcPolicy().getName()); } - @AfterClass + @AfterAll public static void tearDown() { if (cluster != null) { cluster.shutdown(true); @@ -63,9 +62,9 @@ public static void tearDown() { } private static void runBenchmark(String[] args) throws Exception { - Assert.assertNotNull(conf); - Assert.assertNotNull(fs); - Assert.assertEquals(0, ToolRunner.run(conf, + assertNotNull(conf); + assertNotNull(fs); + assertEquals(0, ToolRunner.run(conf, new ErasureCodeBenchmarkThroughput(fs), args)); } @@ -80,7 +79,7 @@ public boolean accept(Path path) { ErasureCodeBenchmarkThroughput.getFilePath(dataSize, isEc)); } }); - Assert.assertEquals(numFile, statuses.length); + assertEquals(numFile, statuses.length); } @Test diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingAddConfig.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingAddConfig.java index 24c88bd629129..79ac8aa83e934 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingAddConfig.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingAddConfig.java @@ -17,10 +17,10 @@ */ package org.apache.hadoop.hdfs; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -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.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.IOException; @@ -28,7 +28,7 @@ import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse; import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.io.erasurecode.ECSchema; -import org.junit.Test; +import org.junit.jupiter.api.Test; /** * Test that ensures addition of user defined EC policies is allowed only when diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingExerciseAPIs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingExerciseAPIs.java index e4b09a8b336c4..2342c540e542c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingExerciseAPIs.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingExerciseAPIs.java @@ -33,9 +33,9 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.Lists; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -58,7 +58,12 @@ import static org.apache.hadoop.fs.permission.FsAction.NONE; import static org.apache.hadoop.fs.permission.FsAction.READ_EXECUTE; import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.aclEntry; -import static org.junit.Assert.*; +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.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; /** * Test after enable Erasure Coding on cluster, exercise Java API make sure they @@ -82,7 +87,7 @@ private static ErasureCodingPolicy getEcPolicy() { LoggerFactory.getLogger(TestErasureCodingExerciseAPIs.class); - @Before + @BeforeEach public void setupCluster() throws IOException { ecPolicy = getEcPolicy(); conf = new HdfsConfiguration(); @@ -252,8 +257,7 @@ public void testACLAPI() throws IOException { AclStatus as = fs.getAclStatus(p); for (AclEntry entry : aclSpec) { - assertTrue(String.format("as: %s, entry: %s", as, entry), - as.getEntries().contains(entry)); + assertTrue(as.getEntries().contains(entry), String.format("as: %s, entry: %s", as, entry)); } List maclSpec = Lists.newArrayList( aclEntry(ACCESS, USER, "bar", READ_EXECUTE), @@ -262,8 +266,7 @@ public void testACLAPI() throws IOException { as = fs.getAclStatus(p); for (AclEntry entry : maclSpec) { - assertTrue(String.format("as: %s, entry: %s", as, entry), - as.getEntries().contains(entry)); + assertTrue(as.getEntries().contains(entry), String.format("as: %s, entry: %s", as, entry)); } fs.removeAclEntries(p, maclSpec); @@ -539,7 +542,7 @@ public void testTruncate() throws IOException { } } - @After + @AfterEach public void shutdownCluster() { if (cluster != null) { cluster.shutdown(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingMultipleRacks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingMultipleRacks.java index e47cbf0c879e2..bf8c7c870c776 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingMultipleRacks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingMultipleRacks.java @@ -27,11 +27,9 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFaultTolerant; import org.apache.hadoop.net.NetworkTopology; import org.apache.hadoop.test.GenericTestUtils; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.Timeout; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.event.Level; @@ -40,12 +38,14 @@ import java.util.Map; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import org.junit.jupiter.api.Timeout; /** * Test erasure coding block placement with skewed # nodes per rack. */ +@Timeout(300) public class TestErasureCodingMultipleRacks { public static final Logger LOG = LoggerFactory.getLogger(TestErasureCodingMultipleRacks.class); @@ -58,9 +58,6 @@ public class TestErasureCodingMultipleRacks { GenericTestUtils.setLogLevel(NetworkTopology.LOG, Level.DEBUG); } - @Rule - public Timeout globalTimeout = new Timeout(300000); - public ErasureCodingPolicy getPolicy() { return StripedFileTestUtil.getDefaultECPolicy(); } @@ -70,7 +67,7 @@ public ErasureCodingPolicy getPolicy() { private Configuration conf; private DistributedFileSystem dfs; - @Before + @BeforeEach public void setup() { ecPolicy = getPolicy(); conf = new HdfsConfiguration(); @@ -98,7 +95,7 @@ public void setupCluster(final int numDatanodes, final int numRacks, dfs.setErasureCodingPolicy(new Path("/"), ecPolicy.getName()); } - @After + @AfterEach public void teardown() throws Exception { if (cluster != null) { cluster.shutdown(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java index 938bbebbf8504..54ee14dc6a3eb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java @@ -41,12 +41,9 @@ import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.test.GenericTestUtils; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.Timeout; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.FileNotFoundException; import java.io.IOException; @@ -58,8 +55,15 @@ import java.util.Map; import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains; -import static org.junit.Assert.*; - +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.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 org.junit.jupiter.api.Timeout; + +@Timeout(60) public class TestErasureCodingPolicies { private Configuration conf; private MiniDFSCluster cluster; @@ -72,10 +76,7 @@ public ErasureCodingPolicy getEcPolicy() { return StripedFileTestUtil.getDefaultECPolicy(); } - @Rule - public Timeout timeout = new Timeout(60 * 1000); - - @Before + @BeforeEach public void setupCluster() throws IOException { ecPolicy = getEcPolicy(); conf = new HdfsConfiguration(); @@ -89,7 +90,7 @@ public void setupCluster() throws IOException { DFSTestUtil.enableAllECPolicies(fs); } - @After + @AfterEach public void shutdownCluster() throws IOException { if (cluster != null) { cluster.shutdown(); @@ -164,7 +165,7 @@ public void testContentSummaryOfECSubdir() throws IOException { final Path subdir = new Path("/ec/sub"); fs.mkdir(subdir, FsPermission.getDirDefault()); ContentSummary contentSummary = fs.getContentSummary(subdir); - assertEquals(ecPolicy.getName(),contentSummary.getErasureCodingPolicy()); + assertEquals(ecPolicy.getName(), contentSummary.getErasureCodingPolicy()); } @Test @@ -227,13 +228,11 @@ public void testBasicSetECPolicy() // Already set directory-level policies should still be in effect Path disabledPolicy = new Path(dir1, "afterDisabled"); - Assert.assertEquals("Dir does not have policy set", - ecPolicy, - fs.getErasureCodingPolicy(dir1)); + assertEquals(ecPolicy, fs.getErasureCodingPolicy(dir1), + "Dir does not have policy set"); fs.create(disabledPolicy).close(); - Assert.assertEquals("File did not inherit dir's policy", - ecPolicy, - fs.getErasureCodingPolicy(disabledPolicy)); + assertEquals(ecPolicy, fs.getErasureCodingPolicy(disabledPolicy), + "File did not inherit dir's policy"); // Also check loading disabled EC policies from fsimage fs.setSafeMode(SafeModeAction.ENTER); @@ -241,12 +240,10 @@ public void testBasicSetECPolicy() fs.setSafeMode(SafeModeAction.LEAVE); cluster.restartNameNodes(); - Assert.assertEquals("Dir does not have policy set", - ecPolicy, - fs.getErasureCodingPolicy(dir1)); - Assert.assertEquals("File does not have policy set", - ecPolicy, - fs.getErasureCodingPolicy(disabledPolicy)); + assertEquals(ecPolicy, fs.getErasureCodingPolicy(dir1), + "Dir does not have policy set"); + assertEquals(ecPolicy, fs.getErasureCodingPolicy(disabledPolicy), + "File does not have policy set"); } @Test @@ -325,19 +322,19 @@ public void testErasureCodingPolicyOnReservedDir() throws IOException { final Path reserveDir = new Path("/.reserved"); // verify the EC policy is null, not an exception ErasureCodingPolicy policy = fs.getErasureCodingPolicy(reserveDir); - assertNull("Got unexpected erasure coding policy", policy); + assertNull(policy, "Got unexpected erasure coding policy"); // root EC policy before being set is null, verify the reserved raw dir // is treated as root final Path root = new Path("/"); final Path rawRoot = new Path("/.reserved/raw"); final Path rawRootSlash = new Path("/.reserved/raw/"); - assertNull("Got unexpected erasure coding policy", - fs.getErasureCodingPolicy(root)); - assertNull("Got unexpected erasure coding policy", - fs.getErasureCodingPolicy(rawRoot)); - assertNull("Got unexpected erasure coding policy", - fs.getErasureCodingPolicy(rawRootSlash)); + assertNull(fs.getErasureCodingPolicy(root), + "Got unexpected erasure coding policy"); + assertNull(fs.getErasureCodingPolicy(rawRoot), + "Got unexpected erasure coding policy"); + assertNull(fs.getErasureCodingPolicy(rawRootSlash), + "Got unexpected erasure coding policy"); // verify the EC policy correctness under the reserved raw dir final Path ecDir = new Path("/ec"); @@ -345,21 +342,18 @@ public void testErasureCodingPolicyOnReservedDir() throws IOException { fs.setErasureCodingPolicy(ecDir, ecPolicy.getName()); ErasureCodingPolicy policyBase = fs.getErasureCodingPolicy(ecDir); - assertEquals("Got unexpected erasure coding policy", ecPolicy, - policyBase); + assertEquals(ecPolicy, policyBase, "Got unexpected erasure coding policy"); final Path rawRootEc = new Path("/.reserved/raw/ec"); ErasureCodingPolicy policyMap = fs.getErasureCodingPolicy(rawRootEc); - assertEquals("Got unexpected erasure coding policy", ecPolicy, - policyMap); + assertEquals(ecPolicy, policyMap, "Got unexpected erasure coding policy"); } @Test public void testGetErasureCodingPolicy() throws Exception { List sysECPolicies = SystemErasureCodingPolicies.getPolicies(); - assertTrue("System ecPolicies should exist", - sysECPolicies.size() > 0); + assertTrue(sysECPolicies.size() > 0, "System ecPolicies should exist"); ErasureCodingPolicy usingECPolicy = sysECPolicies.get(0); String src = "/ec2"; @@ -380,8 +374,8 @@ private void verifyErasureCodingInfo( HdfsFileStatus hdfsFileStatus = fs.getClient().getFileInfo(src); ErasureCodingPolicy actualPolicy = hdfsFileStatus.getErasureCodingPolicy(); assertNotNull(actualPolicy); - assertEquals("Actually used ecPolicy should be equal with target ecPolicy", - usingECPolicy, actualPolicy); + assertEquals(usingECPolicy, actualPolicy, + "Actually used ecPolicy should be equal with target ecPolicy"); } @Test @@ -434,7 +428,7 @@ public void testGetAllErasureCodingPolicies() throws Exception { sysPolicies.remove(ecpi.getPolicy()); } } - assertTrue("All system policies should be enabled", sysPolicies.isEmpty()); + assertTrue(sysPolicies.isEmpty(), "All system policies should be enabled"); // Query after add a new policy ECSchema toAddSchema = new ECSchema("rs", 5, 2); @@ -443,9 +437,8 @@ public void testGetAllErasureCodingPolicies() throws Exception { ErasureCodingPolicy[] policyArray = new ErasureCodingPolicy[]{newPolicy}; fs.addErasureCodingPolicies(policyArray); allECPolicies = fs.getAllErasureCodingPolicies(); - assertEquals("Should return new added policy", - SystemErasureCodingPolicies.getPolicies().size() + 1, - allECPolicies.size()); + assertEquals(SystemErasureCodingPolicies.getPolicies().size() + 1, + allECPolicies.size(), "Should return new added policy"); } @@ -515,13 +508,11 @@ public HdfsAdmin run() throws Exception { userfs.mkdirs(ecdir); final String ecPolicyName = ecPolicy.getName(); useradmin.setErasureCodingPolicy(ecdir, ecPolicyName); - assertEquals("Policy not present on dir", - ecPolicyName, - useradmin.getErasureCodingPolicy(ecdir).getName()); + assertEquals(ecPolicyName, useradmin.getErasureCodingPolicy(ecdir).getName(), + "Policy not present on dir"); userfs.create(ecfile).close(); - assertEquals("Policy not present on file", - ecPolicyName, - useradmin.getErasureCodingPolicy(ecfile).getName()); + assertEquals(ecPolicyName, useradmin.getErasureCodingPolicy(ecfile).getName(), + "Policy not present on file"); // Unset and re-set useradmin.unsetErasureCodingPolicy(ecdir); @@ -631,7 +622,7 @@ public void testFileLevelECPolicy() throws Exception { final String illegalPolicyName = "RS-DEFAULT-1-2-64k"; try { fs.createFile(filePath1).ecPolicyName(illegalPolicyName).build().close(); - Assert.fail("illegal erasure coding policy should not be found"); + fail("illegal erasure coding policy should not be found"); } catch (Exception e) { GenericTestUtils.assertExceptionContains("Policy '" + illegalPolicyName + "' does not match any enabled erasure coding policies", e); @@ -691,7 +682,7 @@ public void testEnforceAsReplicatedFile() throws Exception { .ecPolicyName(ecPolicyName) .replicate() .build().close(); - Assert.fail("shouldReplicate and ecPolicyName are exclusive " + + fail("shouldReplicate and ecPolicyName are exclusive " + "parameters. Set both is not allowed."); }catch (Exception e){ GenericTestUtils.assertExceptionContains("SHOULD_REPLICATE flag and " + @@ -704,7 +695,7 @@ public void testEnforceAsReplicatedFile() throws Exception { EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE, CreateFlag.SHOULD_REPLICATE), false, (short) 1, 1024, null, 1024, null, null, ecPolicyName); - Assert.fail("SHOULD_REPLICATE flag and ecPolicyName are exclusive " + + fail("SHOULD_REPLICATE flag and ecPolicyName are exclusive " + "parameters. Set both is not allowed."); }catch (Exception e){ GenericTestUtils.assertExceptionContains("SHOULD_REPLICATE flag and " + @@ -723,8 +714,8 @@ public void testEnforceAsReplicatedFile() throws Exception { public void testGetAllErasureCodingCodecs() throws Exception { Map allECCodecs = fs .getAllErasureCodingCodecs(); - assertTrue("At least 3 system codecs should be enabled", - allECCodecs.size() >= 3); + assertTrue(allECCodecs.size() >= 3, + "At least 3 system codecs should be enabled"); System.out.println("Erasure Coding Codecs: Codec [Coder List]"); for (String codec : allECCodecs.keySet()) { String coders = allECCodecs.get(codec); @@ -771,7 +762,7 @@ public void testAddErasureCodingPolicies() throws Exception { for (int cellSize: cellSizes) { try { new ErasureCodingPolicy(toAddSchema, cellSize); - Assert.fail("Invalid cell size should be detected."); + fail("Invalid cell size should be detected."); } catch (Exception e){ GenericTestUtils.assertExceptionContains("cellSize must be", e); } @@ -860,67 +851,64 @@ public void testReplicationPolicy() throws Exception { // 1. At first, child directory will inherit parent's EC policy fs.mkdirs(replicaDir); fs.createFile(replicaFile).build().close(); - HdfsFileStatus fileStatus = (HdfsFileStatus)fs.getFileStatus(replicaFile); - assertEquals("File should inherit EC policy.", ecPolicy, fileStatus - .getErasureCodingPolicy()); - assertEquals("File should be a EC file.", true, fileStatus - .isErasureCoded()); - assertEquals("File should have the same EC policy as its ancestor.", - ecPolicy, fs.getErasureCodingPolicy(replicaFile)); + HdfsFileStatus fileStatus = (HdfsFileStatus) fs.getFileStatus(replicaFile); + assertEquals(ecPolicy, fileStatus + .getErasureCodingPolicy(), "File should inherit EC policy."); + assertEquals(true, fileStatus + .isErasureCoded(), "File should be a EC file."); + assertEquals(ecPolicy, fs.getErasureCodingPolicy(replicaFile), + "File should have the same EC policy as its ancestor."); fs.delete(replicaFile, false); // 2. Set replication policy on child directory, then get back the policy fs.setErasureCodingPolicy(replicaDir, replicaPolicy.getName()); ErasureCodingPolicy temp = fs.getErasureCodingPolicy(replicaDir); - assertEquals("Directory should hide replication EC policy.", - null, temp); + assertEquals(null, temp, "Directory should hide replication EC policy."); // 3. New file will be replication file. Please be noted that replication // policy only set on directory, not on file fs.createFile(replicaFile).build().close(); - assertEquals("Replication file should have default replication factor.", - fs.getDefaultReplication(), - fs.getFileStatus(replicaFile).getReplication()); + assertEquals(fs.getDefaultReplication(), fs.getFileStatus(replicaFile).getReplication(), + "Replication file should have default replication factor."); fs.setReplication(replicaFile, (short) 2); - assertEquals("File should have replication factor as expected.", - 2, fs.getFileStatus(replicaFile).getReplication()); - fileStatus = (HdfsFileStatus)fs.getFileStatus(replicaFile); - assertEquals("File should not have EC policy.", null, fileStatus - .getErasureCodingPolicy()); - assertEquals("File should not be a EC file.", false, - fileStatus.isErasureCoded()); + assertEquals(2, fs.getFileStatus(replicaFile).getReplication(), + "File should have replication factor as expected."); + fileStatus = (HdfsFileStatus) fs.getFileStatus(replicaFile); + assertEquals(null, fileStatus + .getErasureCodingPolicy(), "File should not have EC policy."); + assertEquals(false, fileStatus.isErasureCoded(), + "File should not be a EC file."); ErasureCodingPolicy ecPolicyOnFile = fs.getErasureCodingPolicy(replicaFile); - assertEquals("File should not have EC policy.", null, ecPolicyOnFile); + assertEquals(null, ecPolicyOnFile, "File should not have EC policy."); fs.delete(replicaFile, false); // 4. New directory under replication directory, is also replication // directory fs.mkdirs(subReplicaDir); - assertEquals("Directory should inherit hiding replication EC policy.", - null, fs.getErasureCodingPolicy(subReplicaDir)); + assertEquals(null, fs.getErasureCodingPolicy(subReplicaDir), + "Directory should inherit hiding replication EC policy."); fs.createFile(subReplicaFile).build().close(); - assertEquals("File should have default replication factor.", - fs.getDefaultReplication(), - fs.getFileStatus(subReplicaFile).getReplication()); - fileStatus = (HdfsFileStatus)fs.getFileStatus(subReplicaFile); - assertEquals("File should not have EC policy.", null, - fileStatus.getErasureCodingPolicy()); - assertEquals("File should not be a EC file.", false, - fileStatus.isErasureCoded()); - assertEquals("File should not have EC policy.", null, - fs.getErasureCodingPolicy(subReplicaFile)); + assertEquals(fs.getDefaultReplication(), fs.getFileStatus(subReplicaFile).getReplication(), + "File should have default replication factor."); + fileStatus = (HdfsFileStatus) fs.getFileStatus(subReplicaFile); + assertEquals(null, fileStatus.getErasureCodingPolicy(), + "File should not have EC policy."); + assertEquals(false, fileStatus.isErasureCoded(), + "File should not be a EC file."); + assertEquals(null, fs.getErasureCodingPolicy(subReplicaFile), + "File should not have EC policy."); fs.delete(subReplicaFile, false); // 5. Unset replication policy on directory, new file will be EC file fs.unsetErasureCodingPolicy(replicaDir); fs.createFile(subReplicaFile).build().close(); - fileStatus = (HdfsFileStatus)fs.getFileStatus(subReplicaFile); - assertEquals("File should inherit EC policy.", ecPolicy, - fileStatus.getErasureCodingPolicy()); - assertEquals("File should be a EC file.", true, - fileStatus.isErasureCoded()); - assertEquals("File should have the same EC policy as its ancestor", - ecPolicy, fs.getErasureCodingPolicy(subReplicaFile)); + fileStatus = (HdfsFileStatus) fs.getFileStatus(subReplicaFile); + assertEquals(ecPolicy, fileStatus.getErasureCodingPolicy(), + "File should inherit EC policy."); + assertEquals(true, fileStatus.isErasureCoded(), + "File should be a EC file."); + assertEquals(ecPolicy, fs.getErasureCodingPolicy(subReplicaFile), + "File should have the same EC policy as its ancestor"); fs.delete(subReplicaFile, false); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicyWithSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicyWithSnapshot.java index 6ea2bbab81d23..fd8eb5f1a89a9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicyWithSnapshot.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicyWithSnapshot.java @@ -17,8 +17,8 @@ */ package org.apache.hadoop.hdfs; -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.IOException; @@ -31,12 +31,12 @@ import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies; import org.apache.hadoop.util.ToolRunner; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.junit.Rule; -import org.junit.rules.Timeout; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +@Timeout(120) public class TestErasureCodingPolicyWithSnapshot { private MiniDFSCluster cluster; private DistributedFileSystem fs; @@ -50,10 +50,7 @@ public ErasureCodingPolicy getEcPolicy() { return StripedFileTestUtil.getDefaultECPolicy(); } - @Rule - public Timeout globalTimeout = new Timeout(120000); - - @Before + @BeforeEach public void setupCluster() throws IOException { ecPolicy = getEcPolicy(); groupSize = (short) (ecPolicy.getNumDataUnits() @@ -65,7 +62,7 @@ public void setupCluster() throws IOException { fs.enableErasureCodingPolicy(ecPolicy.getName()); } - @After + @AfterEach public void shutdownCluster() throws IOException { if (cluster != null) { cluster.shutdown(); @@ -91,8 +88,8 @@ public void testSnapshotsOnErasureCodingDirsParentDir() throws Exception { String contents = DFSTestUtil.readFile(fs, ecFile); final Path snap1 = fs.createSnapshot(ecDirParent, "snap1"); final Path snap1ECDir = new Path(snap1, ecDir.getName()); - assertEquals("Got unexpected erasure coding policy", ecPolicy, - fs.getErasureCodingPolicy(snap1ECDir)); + assertEquals(ecPolicy, fs.getErasureCodingPolicy(snap1ECDir), + "Got unexpected erasure coding policy"); // Now delete the dir which has erasure coding policy. Re-create the dir again, and // take another snapshot @@ -100,8 +97,8 @@ public void testSnapshotsOnErasureCodingDirsParentDir() throws Exception { fs.mkdir(ecDir, FsPermission.getDirDefault()); final Path snap2 = fs.createSnapshot(ecDirParent, "snap2"); final Path snap2ECDir = new Path(snap2, ecDir.getName()); - assertNull("Expected null erasure coding policy", - fs.getErasureCodingPolicy(snap2ECDir)); + assertNull(fs.getErasureCodingPolicy(snap2ECDir), + "Expected null erasure coding policy"); // Make dir again with system default ec policy fs.setErasureCodingPolicy(ecDir, ecPolicy.getName()); @@ -109,31 +106,30 @@ public void testSnapshotsOnErasureCodingDirsParentDir() throws Exception { final Path snap3ECDir = new Path(snap3, ecDir.getName()); // Check that snap3's ECPolicy has the correct settings ErasureCodingPolicy ezSnap3 = fs.getErasureCodingPolicy(snap3ECDir); - assertEquals("Got unexpected erasure coding policy", ecPolicy, - ezSnap3); + assertEquals(ecPolicy, ezSnap3, "Got unexpected erasure coding policy"); // Check that older snapshots still have the old ECPolicy settings - assertEquals("Got unexpected erasure coding policy", ecPolicy, - fs.getErasureCodingPolicy(snap1ECDir)); - assertNull("Expected null erasure coding policy", - fs.getErasureCodingPolicy(snap2ECDir)); + assertEquals(ecPolicy, fs.getErasureCodingPolicy(snap1ECDir), + "Got unexpected erasure coding policy"); + assertNull(fs.getErasureCodingPolicy(snap2ECDir), + "Expected null erasure coding policy"); // Verify contents of the snapshotted file final Path snapshottedECFile = new Path(snap1.toString() + "/" + ecDir.getName() + "/" + ecFile.getName()); - assertEquals("Contents of snapshotted file have changed unexpectedly", - contents, DFSTestUtil.readFile(fs, snapshottedECFile)); + assertEquals(contents, DFSTestUtil.readFile(fs, snapshottedECFile), + "Contents of snapshotted file have changed unexpectedly"); // Now delete the snapshots out of order and verify the EC policy // correctness fs.deleteSnapshot(ecDirParent, snap2.getName()); - assertEquals("Got unexpected erasure coding policy", ecPolicy, - fs.getErasureCodingPolicy(snap1ECDir)); - assertEquals("Got unexpected erasure coding policy", ecPolicy, - fs.getErasureCodingPolicy(snap3ECDir)); + assertEquals(ecPolicy, fs.getErasureCodingPolicy(snap1ECDir), + "Got unexpected erasure coding policy"); + assertEquals(ecPolicy, fs.getErasureCodingPolicy(snap3ECDir), + "Got unexpected erasure coding policy"); fs.deleteSnapshot(ecDirParent, snap1.getName()); - assertEquals("Got unexpected erasure coding policy", ecPolicy, - fs.getErasureCodingPolicy(snap3ECDir)); + assertEquals(ecPolicy, fs.getErasureCodingPolicy(snap3ECDir), + "Got unexpected erasure coding policy"); } /** @@ -147,8 +143,8 @@ public void testSnapshotsOnErasureCodingDir() throws Exception { fs.setErasureCodingPolicy(ecDir, ecPolicy.getName()); final Path snap1 = fs.createSnapshot(ecDir, "snap1"); - assertEquals("Got unexpected erasure coding policy", ecPolicy, - fs.getErasureCodingPolicy(snap1)); + assertEquals(ecPolicy, fs.getErasureCodingPolicy(snap1), + "Got unexpected erasure coding policy"); } /** @@ -164,8 +160,7 @@ public void testSnapshotsOnErasureCodingDirAfterNNRestart() throws Exception { fs.setErasureCodingPolicy(ecDir, ecPolicy.getName()); final Path snap1 = fs.createSnapshot(ecDir, "snap1"); ErasureCodingPolicy ecSnap = fs.getErasureCodingPolicy(snap1); - assertEquals("Got unexpected erasure coding policy", ecPolicy, - ecSnap); + assertEquals(ecPolicy, ecSnap, "Got unexpected erasure coding policy"); // save namespace, restart namenode, and check ec policy correctness. fs.setSafeMode(SafeModeAction.ENTER); @@ -174,10 +169,8 @@ public void testSnapshotsOnErasureCodingDirAfterNNRestart() throws Exception { cluster.restartNameNode(true); ErasureCodingPolicy ecSnap1 = fs.getErasureCodingPolicy(snap1); - assertEquals("Got unexpected erasure coding policy", ecPolicy, - ecSnap1); - assertEquals("Got unexpected ecSchema", ecSnap.getSchema(), - ecSnap1.getSchema()); + assertEquals(ecPolicy, ecSnap1, "Got unexpected erasure coding policy"); + assertEquals(ecSnap.getSchema(), ecSnap1.getSchema(), "Got unexpected ecSchema"); } /** @@ -202,15 +195,16 @@ public void testCopySnapshotWillNotPreserveErasureCodingPolicy() String[] argv = new String[] { "-cp", "-px", snap1.toUri().toString(), snap1Copy.toUri().toString() }; int ret = ToolRunner.run(new FsShell(conf), argv); - assertEquals("cp -px is not working on a snapshot", SUCCESS, ret); + assertEquals(SUCCESS, ret, "cp -px is not working on a snapshot"); - assertNull("Got unexpected erasure coding policy", - fs.getErasureCodingPolicy(snap1CopyECDir)); - assertEquals("Got unexpected erasure coding policy", ecPolicy, - fs.getErasureCodingPolicy(snap1)); + assertNull(fs.getErasureCodingPolicy(snap1CopyECDir), + "Got unexpected erasure coding policy"); + assertEquals(ecPolicy, fs.getErasureCodingPolicy(snap1), + "Got unexpected erasure coding policy"); } - @Test (timeout = 300000) + @Test + @Timeout(value = 300) public void testFileStatusAcrossNNRestart() throws IOException { final int len = 1024; final Path normalFile = new Path("/", "normalFile"); @@ -247,13 +241,12 @@ public void testErasureCodingPolicyOnDotSnapshotDir() throws IOException { // verify the EC policy correctness ErasureCodingPolicy ecSnap = fs.getErasureCodingPolicy(snap); - assertEquals("Got unexpected erasure coding policy", ecPolicy, - ecSnap); + assertEquals(ecPolicy, ecSnap, "Got unexpected erasure coding policy"); // verify the EC policy is null, not an exception final Path ecDotSnapshotDir = new Path(ecDir, ".snapshot"); ErasureCodingPolicy ecSnap1 = fs.getErasureCodingPolicy(ecDotSnapshotDir); - assertNull("Got unexpected erasure coding policy", ecSnap1); + assertNull(ecSnap1, "Got unexpected erasure coding policy"); } /** @@ -268,22 +261,20 @@ public void testSnapshotsOnErasureCodingDirAfterECPolicyChanges() fs.allowSnapshot(ecDir); final Path snap1 = fs.createSnapshot(ecDir, "snap1"); - assertNull("Expected null erasure coding policy", - fs.getErasureCodingPolicy(snap1)); + assertNull(fs.getErasureCodingPolicy(snap1), "Expected null erasure coding policy"); // Set erasure coding policy final ErasureCodingPolicy ec63Policy = SystemErasureCodingPolicies .getByID(SystemErasureCodingPolicies.RS_6_3_POLICY_ID); fs.setErasureCodingPolicy(ecDir, ec63Policy.getName()); final Path snap2 = fs.createSnapshot(ecDir, "snap2"); - assertEquals("Got unexpected erasure coding policy", ec63Policy, - fs.getErasureCodingPolicy(snap2)); + assertEquals(ec63Policy, fs.getErasureCodingPolicy(snap2), + "Got unexpected erasure coding policy"); // Verify the EC policy correctness after the unset operation fs.unsetErasureCodingPolicy(ecDir); final Path snap3 = fs.createSnapshot(ecDir, "snap3"); - assertNull("Expected null erasure coding policy", - fs.getErasureCodingPolicy(snap3)); + assertNull(fs.getErasureCodingPolicy(snap3), "Expected null erasure coding policy"); // Change the erasure coding policy and take another snapshot final ErasureCodingPolicy ec32Policy = SystemErasureCodingPolicies @@ -291,15 +282,13 @@ public void testSnapshotsOnErasureCodingDirAfterECPolicyChanges() fs.enableErasureCodingPolicy(ec32Policy.getName()); fs.setErasureCodingPolicy(ecDir, ec32Policy.getName()); final Path snap4 = fs.createSnapshot(ecDir, "snap4"); - assertEquals("Got unexpected erasure coding policy", ec32Policy, - fs.getErasureCodingPolicy(snap4)); + assertEquals(ec32Policy, fs.getErasureCodingPolicy(snap4), + "Got unexpected erasure coding policy"); // Check that older snapshot still have the old ECPolicy settings - assertNull("Expected null erasure coding policy", - fs.getErasureCodingPolicy(snap1)); - assertEquals("Got unexpected erasure coding policy", ec63Policy, - fs.getErasureCodingPolicy(snap2)); - assertNull("Expected null erasure coding policy", - fs.getErasureCodingPolicy(snap3)); + assertNull(fs.getErasureCodingPolicy(snap1), "Expected null erasure coding policy"); + assertEquals(ec63Policy, fs.getErasureCodingPolicy(snap2), + "Got unexpected erasure coding policy"); + assertNull(fs.getErasureCodingPolicy(snap3), "Expected null erasure coding policy"); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestExtendedAcls.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestExtendedAcls.java index b4baadfa041ca..f9511fc3c00df 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestExtendedAcls.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestExtendedAcls.java @@ -27,9 +27,9 @@ import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Lists; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.security.PrivilegedExceptionAction; @@ -49,9 +49,9 @@ import static org.apache.hadoop.fs.permission.FsAction.ALL; import static org.apache.hadoop.fs.permission.AclEntryType.GROUP; import static org.apache.hadoop.fs.permission.AclEntryType.OTHER; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * A class for testing the behavior of HDFS directory and file ACL. @@ -65,7 +65,7 @@ public class TestExtendedAcls { private static DistributedFileSystem hdfs; - @BeforeClass + @BeforeAll public static void setup() throws IOException { conf = new Configuration(); conf.setBoolean(DFS_NAMENODE_ACLS_ENABLED_KEY, true); @@ -76,7 +76,7 @@ public static void setup() throws IOException { hdfs = cluster.getFileSystem(); } - @AfterClass + @AfterAll public static void shutdown() throws IOException { if (cluster != null) { cluster.shutdown(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestExternalBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestExternalBlockReader.java index f153b2c9d1724..5da4d33bd2ddf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestExternalBlockReader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestExternalBlockReader.java @@ -28,8 +28,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.net.NetUtils; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.nio.BufferOverflowException; @@ -38,6 +37,11 @@ import java.util.LinkedList; import java.util.UUID; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + public class TestExternalBlockReader { private static final Logger LOG = LoggerFactory.getLogger(TestExternalBlockReader.class); @@ -63,7 +67,7 @@ public void testMisconfiguredExternalBlockReader() throws Exception { IOUtils.readFully(stream, buf, 0, TEST_LENGTH); byte expected[] = DFSTestUtil. calculateFileContentsFromSeed(SEED, TEST_LENGTH); - Assert.assertArrayEquals(expected, buf); + assertArrayEquals(expected, buf); stream.close(); } finally { dfs.close(); @@ -293,36 +297,36 @@ public void testExternalBlockReader() throws Exception { byte expected[] = DFSTestUtil. calculateFileContentsFromSeed(SEED, TEST_LENGTH); ReadStatistics stats = stream.getReadStatistics(); - Assert.assertEquals(1024, stats.getTotalShortCircuitBytesRead()); - Assert.assertEquals(2047, stats.getTotalLocalBytesRead()); - Assert.assertEquals(2047, stats.getTotalBytesRead()); - Assert.assertArrayEquals(expected, buf); + assertEquals(1024, stats.getTotalShortCircuitBytesRead()); + assertEquals(2047, stats.getTotalLocalBytesRead()); + assertEquals(2047, stats.getTotalBytesRead()); + assertArrayEquals(expected, buf); stream.close(); ExtendedBlock block = DFSTestUtil.getFirstBlock(dfs, new Path("/a")); - Assert.assertNotNull(block); + assertNotNull(block); LinkedList accessorList = accessors.get(uuid); - Assert.assertNotNull(accessorList); - Assert.assertEquals(3, accessorList.size()); + assertNotNull(accessorList); + assertEquals(3, accessorList.size()); SyntheticReplicaAccessor accessor = accessorList.get(0); - Assert.assertTrue(accessor.builder.allowShortCircuit); - Assert.assertEquals(block.getBlockPoolId(), + assertTrue(accessor.builder.allowShortCircuit); + assertEquals(block.getBlockPoolId(), accessor.builder.blockPoolId); - Assert.assertEquals(block.getBlockId(), + assertEquals(block.getBlockId(), accessor.builder.blockId); - Assert.assertEquals(dfs.getClient().clientName, + assertEquals(dfs.getClient().clientName, accessor.builder.clientName); - Assert.assertEquals("/a", accessor.builder.fileName); - Assert.assertEquals(block.getGenerationStamp(), + assertEquals("/a", accessor.builder.fileName); + assertEquals(block.getGenerationStamp(), accessor.getGenerationStamp()); - Assert.assertTrue(accessor.builder.verifyChecksum); - Assert.assertEquals(1024L, accessor.builder.visibleLength); - Assert.assertEquals(24L, accessor.totalRead); - Assert.assertEquals("", accessor.getError()); - Assert.assertEquals(1, accessor.numCloses); + assertTrue(accessor.builder.verifyChecksum); + assertEquals(1024L, accessor.builder.visibleLength); + assertEquals(24L, accessor.totalRead); + assertEquals("", accessor.getError()); + assertEquals(1, accessor.numCloses); byte[] tempBuf = new byte[5]; - Assert.assertEquals(-1, accessor.read(TEST_LENGTH, + assertEquals(-1, accessor.read(TEST_LENGTH, tempBuf, 0, 0)); - Assert.assertEquals(-1, accessor.read(TEST_LENGTH, + assertEquals(-1, accessor.read(TEST_LENGTH, tempBuf, 0, tempBuf.length)); accessors.remove(uuid); } finally { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSInputChecker.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSInputChecker.java index e7f3b9fc34238..86663b785db30 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSInputChecker.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSInputChecker.java @@ -17,10 +17,10 @@ */ package org.apache.hadoop.hdfs; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import java.io.EOFException; import java.io.File; @@ -40,7 +40,7 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.test.PathUtils; -import org.junit.Test; +import org.junit.jupiter.api.Test; /** * This class tests if FSInputChecker works correctly. @@ -71,9 +71,8 @@ private void writeFile(FileSystem fileSys, Path name) throws IOException { private void checkAndEraseData(byte[] actual, int from, byte[] expected, String message) throws Exception { for (int idx = 0; idx < actual.length; idx++) { - assertEquals(message+" byte "+(from+idx)+" differs. expected "+ - expected[from+idx]+" actual "+actual[idx], - actual[idx], expected[from+idx]); + assertEquals(actual[idx], expected[from + idx], message + " byte " + (from + idx) + + " differs. expected " + expected[from + idx] + " actual " + actual[idx]); actual[idx] = 0; } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFetchImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFetchImage.java index 7e1e5938a13e4..cb9b502e1bca0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFetchImage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFetchImage.java @@ -20,7 +20,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import java.io.File; import java.io.IOException; @@ -41,10 +41,11 @@ import org.apache.hadoop.io.MD5Hash; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.PathUtils; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; public class TestFetchImage { @@ -58,17 +59,17 @@ public class TestFetchImage { private NameNode nn1 = null; private Configuration conf = null; - @BeforeClass + @BeforeAll public static void setupImageDir() { FETCHED_IMAGE_FILE.mkdirs(); } - @AfterClass + @AfterAll public static void cleanup() { FileUtil.fullyDelete(FETCHED_IMAGE_FILE); } - @Before + @BeforeEach public void setupCluster() throws IOException, URISyntaxException { conf = new Configuration(); conf.setInt(DFS_HEARTBEAT_INTERVAL_KEY, 1); @@ -89,7 +90,8 @@ public void setupCluster() throws IOException, URISyntaxException { * Download a few fsimages using `hdfs dfsadmin -fetchImage ...' and verify * the results. */ - @Test(timeout=30000) + @Test + @Timeout(value = 30) public void testFetchImageHA() throws Exception { final Path parent = new Path( PathUtils.getTestPath(getClass()), diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java index b65301f8c576b..e3e10ce4a29d7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java @@ -17,9 +17,10 @@ */ package org.apache.hadoop.hdfs; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import java.io.File; import java.io.FileNotFoundException; @@ -58,8 +59,8 @@ import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.Time; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; /** * This class tests the building blocks that are needed to @@ -137,8 +138,8 @@ public void testBreakHardlinksIfNeeded() throws IOException { // Get a handle to the datanode DataNode[] dn = cluster.listDataNodes(); - assertTrue("There should be only one datanode but found " + dn.length, - dn.length == 1); + assertTrue( + dn.length == 1, "There should be only one datanode but found " + dn.length); LocatedBlocks locations = client.getNamenode().getBlockLocations( file1.toString(), 0, Long.MAX_VALUE); @@ -161,8 +162,8 @@ public void testBreakHardlinksIfNeeded() throws IOException { for (int i = 0; i < blocks.size(); i++) { ExtendedBlock b = blocks.get(i).getBlock(); System.out.println("breakHardlinksIfNeeded detaching block " + b); - assertTrue("breakHardlinksIfNeeded(" + b + ") should have returned true", - FsDatasetTestUtil.breakHardlinksIfNeeded(fsd, b)); + assertTrue(FsDatasetTestUtil.breakHardlinksIfNeeded(fsd, b), + "breakHardlinksIfNeeded(" + b + ") should have returned true"); } // Since the blocks were already detached earlier, these calls should @@ -171,8 +172,8 @@ public void testBreakHardlinksIfNeeded() throws IOException { ExtendedBlock b = blocks.get(i).getBlock(); System.out.println("breakHardlinksIfNeeded re-attempting to " + "detach block " + b); - assertTrue("breakHardlinksIfNeeded(" + b + ") should have returned false", - FsDatasetTestUtil.breakHardlinksIfNeeded(fsd, b)); + assertTrue(FsDatasetTestUtil.breakHardlinksIfNeeded(fsd, b), + "breakHardlinksIfNeeded(" + b + ") should have returned false"); } } finally { client.close(); @@ -290,20 +291,22 @@ public void testComplexFlush() throws IOException { * * @throws FileNotFoundException as the result */ - @Test(expected = FileNotFoundException.class) + @Test public void testFileNotFound() throws IOException { - Configuration conf = new HdfsConfiguration(); - File builderBaseDir = new File(GenericTestUtils.getRandomizedTempPath()); - MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf, builderBaseDir) - .build(); - FileSystem fs = cluster.getFileSystem(); - try { - Path file1 = new Path("/nonexistingfile.dat"); - fs.append(file1); - } finally { - fs.close(); - cluster.shutdown(); - } + assertThrows(FileNotFoundException.class, () -> { + Configuration conf = new HdfsConfiguration(); + File builderBaseDir = new File(GenericTestUtils.getRandomizedTempPath()); + MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf, builderBaseDir) + .build(); + FileSystem fs = cluster.getFileSystem(); + try { + Path file1 = new Path("/nonexistingfile.dat"); + fs.append(file1); + } finally { + fs.close(); + cluster.shutdown(); + } + }); } /** Test two consecutive appends on a file with a full block. */ @@ -335,10 +338,10 @@ public void testAppendTwice() throws Exception { //2nd append should get AlreadyBeingCreatedException fs1.append(p); - Assert.fail(); + fail(); } catch(RemoteException re) { AppendTestUtil.LOG.info("Got an exception:", re); - Assert.assertEquals(AlreadyBeingCreatedException.class.getName(), + assertEquals(AlreadyBeingCreatedException.class.getName(), re.getClassName()); } finally { fs2.close(); @@ -376,10 +379,10 @@ public void testAppend2Twice() throws Exception { // 2nd append should get AlreadyBeingCreatedException fs1.append(p); - Assert.fail(); + fail(); } catch(RemoteException re) { AppendTestUtil.LOG.info("Got an exception:", re); - Assert.assertEquals(AlreadyBeingCreatedException.class.getName(), + assertEquals(AlreadyBeingCreatedException.class.getName(), re.getClassName()); } finally { fs2.close(); @@ -428,13 +431,13 @@ public void testMultipleAppends() throws Exception { fileLen += appendLen; } - Assert.assertEquals(fileLen, fs.getFileStatus(p).getLen()); + assertEquals(fileLen, fs.getFileStatus(p).getLen()); final byte[] actual = new byte[fileLen]; final FSDataInputStream in = fs.open(p); in.readFully(actual); in.close(); for(int i = 0; i < fileLen; i++) { - Assert.assertEquals(data[i], actual[i]); + assertEquals(data[i], actual[i]); } } finally { fs.close(); @@ -662,7 +665,8 @@ public void testMultiAppend2() throws Exception { } } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testAppendCorruptedBlock() throws Exception { Configuration conf = new HdfsConfiguration(); conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1024); @@ -675,7 +679,7 @@ public void testAppendCorruptedBlock() throws Exception { Path fileName = new Path("/appendCorruptBlock"); DFSTestUtil.createFile(fs, fileName, 512, (short) 1, 0); DFSTestUtil.waitReplication(fs, fileName, (short) 1); - Assert.assertTrue("File not created", fs.exists(fileName)); + assertTrue(fs.exists(fileName), "File not created"); ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, fileName); cluster.corruptBlockOnDataNodes(block); DFSTestUtil.appendFile(fs, fileName, "appendCorruptBlock"); @@ -684,7 +688,8 @@ public void testAppendCorruptedBlock() throws Exception { } } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testConcurrentAppendRead() throws IOException, TimeoutException, InterruptedException { // Create a finalized replica and append to it @@ -707,7 +712,7 @@ public void testConcurrentAppendRead() Path fileName = new Path("/appendCorruptBlock"); DFSTestUtil.createFile(fs, fileName, initialFileLength, (short) 1, 0); DFSTestUtil.waitReplication(fs, fileName, (short) 1); - Assert.assertTrue("File not created", fs.exists(fileName)); + assertTrue(fs.exists(fileName), "File not created"); // Call FsDatasetImpl#append to append the block file, // which converts it to a rbw replica. @@ -738,7 +743,7 @@ public void testConcurrentAppendRead() // checksum, rather than on-disk checksum. Otherwise it will see a // checksum mismatch error. final byte[] readBlock = DFSTestUtil.readFileBuffer(fs, fileName); - assertEquals("should have read only one byte!", 1, readBlock.length); + assertEquals(1, readBlock.length, "should have read only one byte!"); } finally { cluster.shutdown(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend2.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend2.java index 9929cb24b23f5..b7916ec560730 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend2.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend2.java @@ -17,10 +17,10 @@ */ package org.apache.hadoop.hdfs; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import java.io.IOException; import java.util.ArrayList; @@ -42,7 +42,7 @@ import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.test.GenericTestUtils; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; /** @@ -445,10 +445,9 @@ public void run() { } catch (InterruptedException e) {} } - assertTrue("File " + testfile + " size is " + - fs.getFileStatus(testfile).getLen() + - " but expected " + (len + sizeToAppend), - fs.getFileStatus(testfile).getLen() == (len + sizeToAppend)); + assertTrue(fs.getFileStatus(testfile).getLen() == (len + sizeToAppend), + "File " + testfile + " size is " + fs.getFileStatus(testfile).getLen() + + " but expected " + (len + sizeToAppend)); AppendTestUtil.checkFullFile(fs, testfile, (int) (len + sizeToAppend), fileContents, "Read 2"); @@ -460,9 +459,8 @@ public void run() { " " + e); e.printStackTrace(); } - assertTrue("Workload exception " + id + " testfile " + testfile + - " expected size " + (len + sizeToAppend), - false); + assertTrue(false, "Workload exception " + id + " testfile " + testfile + + " expected size " + (len + sizeToAppend)); } // Add testfile back to the pool of files. @@ -530,7 +528,7 @@ private void testComplexAppend(boolean appendToNewBlock) throws IOException { // If any of the worker thread failed in their job, indicate that // this test failed. // - assertTrue("testComplexAppend Worker encountered exceptions.", globalStatus); + assertTrue(globalStatus, "testComplexAppend Worker encountered exceptions."); } @Test diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java index a3b4ce88f2ee8..5b3a96305a77d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java @@ -17,9 +17,9 @@ */ package org.apache.hadoop.hdfs; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import java.io.IOException; import java.util.EnumSet; @@ -45,10 +45,9 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; /** This class implements some of tests posted in HADOOP-2658. */ @@ -69,7 +68,7 @@ public class TestFileAppend3 { private static MiniDFSCluster cluster; private static DistributedFileSystem fs; - @BeforeClass + @BeforeAll public static void setUp() throws java.lang.Exception { AppendTestUtil.LOG.info("setUp()"); conf = new HdfsConfiguration(); @@ -79,7 +78,7 @@ public static void setUp() throws java.lang.Exception { fs = cluster.getFileSystem(); } - @AfterClass + @AfterAll public static void tearDown() throws Exception { AppendTestUtil.LOG.info("tearDown()"); if(fs != null) fs.close(); @@ -200,10 +199,10 @@ public void testTC2ForAppend2() throws Exception { AppendTestUtil.check(fs, p, len1 + len2); List blocks = fs.getClient().getLocatedBlocks( p.toString(), 0L).getLocatedBlocks(); - Assert.assertEquals(3, blocks.size()); - Assert.assertEquals(BLOCK_SIZE, blocks.get(0).getBlockSize()); - Assert.assertEquals(BLOCK_SIZE / 2, blocks.get(1).getBlockSize()); - Assert.assertEquals(BLOCK_SIZE / 4, blocks.get(2).getBlockSize()); + assertEquals(3, blocks.size()); + assertEquals(BLOCK_SIZE, blocks.get(0).getBlockSize()); + assertEquals(BLOCK_SIZE / 2, blocks.get(1).getBlockSize()); + assertEquals(BLOCK_SIZE / 4, blocks.get(2).getBlockSize()); } /** @@ -428,9 +427,9 @@ private void testTC12(boolean appendToNewBlock) throws Exception { AppendTestUtil.check(fs, p, len1 + len2); if (appendToNewBlock) { LocatedBlocks blks = fs.dfs.getLocatedBlocks(p.toString(), 0); - Assert.assertEquals(2, blks.getLocatedBlocks().size()); - Assert.assertEquals(len1, blks.getLocatedBlocks().get(0).getBlockSize()); - Assert.assertEquals(len2, blks.getLocatedBlocks().get(1).getBlockSize()); + assertEquals(2, blks.getLocatedBlocks().size()); + assertEquals(len1, blks.getLocatedBlocks().get(0).getBlockSize()); + assertEquals(len2, blks.getLocatedBlocks().get(1).getBlockSize()); AppendTestUtil.check(fs, p, 0, len1); AppendTestUtil.check(fs, p, len1, len2); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java index d2045252c6cbb..b10439b248bfb 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java @@ -18,9 +18,9 @@ package org.apache.hadoop.hdfs; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY; -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.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; @@ -47,8 +47,9 @@ import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols; import org.apache.hadoop.test.GenericTestUtils; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.event.Level; /* File Append tests for HDFS-200 & HDFS-142, specifically focused on: @@ -72,7 +73,7 @@ public class TestFileAppend4 { GenericTestUtils.setLogLevel(DFSClient.LOG, Level.TRACE); } - @Before + @BeforeEach public void setUp() throws Exception { this.conf = new Configuration(); @@ -142,7 +143,8 @@ private void recoverFile(final FileSystem fs) throws Exception { * before calling completeFile, and then tries to recover * the lease from another thread. */ - @Test(timeout=60000) + @Test + @Timeout(value = 60) public void testRecoverFinalizedBlock() throws Throwable { cluster = new MiniDFSCluster.Builder(conf).numDataNodes(5).build(); @@ -212,7 +214,8 @@ public void run() { * starts writing from that writer, and then has the old lease holder * call completeFile */ - @Test(timeout=60000) + @Test + @Timeout(value = 60) public void testCompleteOtherLeaseHoldersFile() throws Throwable { cluster = new MiniDFSCluster.Builder(conf).numDataNodes(5).build(); @@ -290,7 +293,8 @@ public void run() { /** * Test the updation of NeededReplications for the Appended Block */ - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testUpdateNeededReplicationsForAppendedFile() throws Exception { Configuration conf = new Configuration(); MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1) @@ -326,7 +330,8 @@ public void testUpdateNeededReplicationsForAppendedFile() throws Exception { * Test that an append with no locations fails with an exception * showing insufficient locations. */ - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testAppendInsufficientLocations() throws Exception { Configuration conf = new Configuration(); @@ -379,7 +384,7 @@ public void testAppendInsufficientLocations() throws Exception { FSDirectory dir = cluster.getNamesystem().getFSDirectory(); final INodeFile inode = INodeFile. valueOf(dir.getINode("/testAppend"), "/testAppend"); - assertTrue("File should remain closed", !inode.isUnderConstruction()); + assertTrue(!inode.isUnderConstruction(), "File should remain closed"); } finally { if (null != fileSystem) { fileSystem.close(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java index a2b344cb65b94..ca72e91e7d57e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java @@ -18,7 +18,7 @@ package org.apache.hadoop.hdfs; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import java.io.File; import java.io.IOException; @@ -38,7 +38,7 @@ import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.PathUtils; -import org.junit.Test; +import org.junit.jupiter.api.Test; /** * Unit test to make sure that Append properly logs the right diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileChecksum.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileChecksum.java index 86f71da63d7db..afadbeccac790 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileChecksum.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileChecksum.java @@ -30,12 +30,10 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.DataNodeFaultInjector; import org.apache.hadoop.test.GenericTestUtils; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; @@ -45,6 +43,11 @@ import java.util.Random; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; @@ -54,7 +57,6 @@ * layout. For simple, it assumes 6 data blocks in both files and the block size * are the same. */ -@RunWith(Parameterized.class) public class TestFileChecksum { private static final Logger LOG = LoggerFactory .getLogger(TestFileChecksum.class); @@ -84,21 +86,17 @@ public class TestFileChecksum { private static String checksumCombineMode; - public TestFileChecksum(String mode) { - checksumCombineMode = mode; + public void initTestFileChecksum(String pMode) throws IOException { + checksumCombineMode = pMode; + setup(pMode); } - @Parameterized.Parameters public static Object[] getParameters() { return new Object[] { ChecksumCombineMode.MD5MD5CRC.name(), ChecksumCombineMode.COMPOSITE_CRC.name()}; } - @Rule - public ExpectedException exception = ExpectedException.none(); - - @Parameterized.BeforeParam public static void setup(String mode) throws IOException { checksumCombineMode = mode; int numDNs = dataBlocks + parityBlocks + 2; @@ -123,58 +121,79 @@ public static void setup(String mode) throws IOException { GenericTestUtils.setLogLevel(FileChecksumHelper.LOG, Level.DEBUG); } - @Parameterized.AfterParam - public static void tearDown() { + @AfterEach + public void tearDown() { if (cluster != null) { cluster.shutdown(); cluster = null; } } - @Test(timeout = 90000) - public void testStripedFileChecksum1() throws Exception { + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksum1(String pMode) throws Exception { + initTestFileChecksum(pMode); int length = 0; prepareTestFiles(fileSize, new String[] {stripedFile1, stripedFile2}); testStripedFileChecksum(length, length + 10); } - @Test(timeout = 90000) - public void testStripedFileChecksum2() throws Exception { + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksum2(String pMode) throws Exception { + initTestFileChecksum(pMode); int length = stripSize - 1; prepareTestFiles(fileSize, new String[] {stripedFile1, stripedFile2}); testStripedFileChecksum(length, length - 10); } - @Test(timeout = 90000) - public void testStripedFileChecksum3() throws Exception { + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksum3(String pMode) throws Exception { + initTestFileChecksum(pMode); int length = stripSize; prepareTestFiles(fileSize, new String[] {stripedFile1, stripedFile2}); testStripedFileChecksum(length, length - 10); } - @Test(timeout = 90000) - public void testStripedFileChecksum4() throws Exception { + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksum4(String pMode) throws Exception { + initTestFileChecksum(pMode); int length = stripSize + cellSize * 2; prepareTestFiles(fileSize, new String[] {stripedFile1, stripedFile2}); testStripedFileChecksum(length, length - 10); } - @Test(timeout = 90000) - public void testStripedFileChecksum5() throws Exception { + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksum5(String pMode) throws Exception { + initTestFileChecksum(pMode); int length = blockGroupSize; prepareTestFiles(fileSize, new String[] {stripedFile1, stripedFile2}); testStripedFileChecksum(length, length - 10); } - @Test(timeout = 90000) - public void testStripedFileChecksum6() throws Exception { + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksum6(String pMode) throws Exception { + initTestFileChecksum(pMode); int length = blockGroupSize + blockSize; prepareTestFiles(fileSize, new String[] {stripedFile1, stripedFile2}); testStripedFileChecksum(length, length - 10); } - @Test(timeout = 90000) - public void testStripedFileChecksum7() throws Exception { + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksum7(String pMode) throws Exception { + initTestFileChecksum(pMode); int length = -1; // whole file prepareTestFiles(fileSize, new String[] {stripedFile1, stripedFile2}); testStripedFileChecksum(length, fileSize); @@ -193,14 +212,17 @@ private void testStripedFileChecksum(int range1, int range2) LOG.info("stripedFileChecksum2:" + stripedFileChecksum2); LOG.info("stripedFileChecksum3:" + stripedFileChecksum3); - Assert.assertTrue(stripedFileChecksum1.equals(stripedFileChecksum2)); + assertTrue(stripedFileChecksum1.equals(stripedFileChecksum2)); if (range1 >=0 && range1 != range2) { - Assert.assertFalse(stripedFileChecksum1.equals(stripedFileChecksum3)); + assertFalse(stripedFileChecksum1.equals(stripedFileChecksum3)); } } - @Test(timeout = 90000) - public void testStripedAndReplicatedFileChecksum() throws Exception { + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedAndReplicatedFileChecksum(String pMode) throws Exception { + initTestFileChecksum(pMode); prepareTestFiles(fileSize, new String[] {stripedFile1, replicatedFile}); FileChecksum stripedFileChecksum1 = getFileChecksum(stripedFile1, 10, false); @@ -208,9 +230,9 @@ public void testStripedAndReplicatedFileChecksum() throws Exception { 10, false); if (checksumCombineMode.equals(ChecksumCombineMode.COMPOSITE_CRC.name())) { - Assert.assertEquals(stripedFileChecksum1, replicatedFileChecksum); + assertEquals(stripedFileChecksum1, replicatedFileChecksum); } else { - Assert.assertNotEquals(stripedFileChecksum1, replicatedFileChecksum); + assertNotEquals(stripedFileChecksum1, replicatedFileChecksum); } } @@ -222,8 +244,11 @@ public void testStripedAndReplicatedFileChecksum() throws Exception { * but the last block size in the check length is ((dataBlocks - 1) * blockSize * + (int) (blockSize * 0.6)) */ - @Test(timeout = 90000) - public void testStripedAndReplicatedFileChecksum2() throws Exception { + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedAndReplicatedFileChecksum2(String pMode) throws Exception { + initTestFileChecksum(pMode); final int lastBlockSize = (int) (blockSize * 0.5); final int fullStripeLength = dataBlocks * blockSize; final int testFileSize = fullStripeLength + lastBlockSize; @@ -232,8 +257,8 @@ public void testStripedAndReplicatedFileChecksum2() throws Exception { final int specialLength = (dataBlocks - 1) * blockSize + (int) (blockSize * 0.6); - Assert.assertTrue(specialLength % blockSize > lastBlockSize); - Assert.assertTrue(specialLength % fullStripeLength > lastBlockSize); + assertTrue(specialLength % blockSize > lastBlockSize); + assertTrue(specialLength % fullStripeLength > lastBlockSize); FileChecksum stripedFileChecksum = getFileChecksum(stripedFile1, specialLength, false); @@ -241,14 +266,17 @@ public void testStripedAndReplicatedFileChecksum2() throws Exception { specialLength, false); if (checksumCombineMode.equals(ChecksumCombineMode.COMPOSITE_CRC.name())) { - Assert.assertEquals(replicatedFileChecksum, stripedFileChecksum); + assertEquals(replicatedFileChecksum, stripedFileChecksum); } else { - Assert.assertNotEquals(replicatedFileChecksum, stripedFileChecksum); + assertNotEquals(replicatedFileChecksum, stripedFileChecksum); } } - @Test(timeout = 90000) - public void testDifferentBlockSizeReplicatedFileChecksum() throws Exception { + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testDifferentBlockSizeReplicatedFileChecksum(String pMode) throws Exception { + initTestFileChecksum(pMode); byte[] fileData = StripedFileTestUtil.generateBytes(fileSize); String replicatedFile1 = "/replicatedFile1"; String replicatedFile2 = "/replicatedFile2"; @@ -260,14 +288,17 @@ public void testDifferentBlockSizeReplicatedFileChecksum() throws Exception { FileChecksum checksum2 = getFileChecksum(replicatedFile2, -1, false); if (checksumCombineMode.equals(ChecksumCombineMode.COMPOSITE_CRC.name())) { - Assert.assertEquals(checksum1, checksum2); + assertEquals(checksum1, checksum2); } else { - Assert.assertNotEquals(checksum1, checksum2); + assertNotEquals(checksum1, checksum2); } } - @Test(timeout = 90000) - public void testStripedFileChecksumWithMissedDataBlocks1() throws Exception { + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksumWithMissedDataBlocks1(String pMode) throws Exception { + initTestFileChecksum(pMode); prepareTestFiles(fileSize, new String[] {stripedFile1}); FileChecksum stripedFileChecksum1 = getFileChecksum(stripedFile1, fileSize, false); @@ -277,12 +308,15 @@ public void testStripedFileChecksumWithMissedDataBlocks1() throws Exception { LOG.info("stripedFileChecksum1:" + stripedFileChecksum1); LOG.info("stripedFileChecksumRecon:" + stripedFileChecksumRecon); - Assert.assertTrue("Checksum mismatches!", - stripedFileChecksum1.equals(stripedFileChecksumRecon)); + assertTrue(stripedFileChecksum1.equals(stripedFileChecksumRecon), + "Checksum mismatches!"); } - @Test(timeout = 90000) - public void testStripedFileChecksumWithMissedDataBlocks2() throws Exception { + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksumWithMissedDataBlocks2(String pMode) throws Exception { + initTestFileChecksum(pMode); prepareTestFiles(fileSize, new String[] {stripedFile1, stripedFile2}); FileChecksum stripedFileChecksum1 = getFileChecksum(stripedFile1, -1, false); @@ -295,12 +329,12 @@ public void testStripedFileChecksumWithMissedDataBlocks2() throws Exception { LOG.info("stripedFileChecksum2:" + stripedFileChecksum1); LOG.info("stripedFileChecksum2Recon:" + stripedFileChecksum2Recon); - Assert.assertTrue("Checksum mismatches!", - stripedFileChecksum1.equals(stripedFileChecksum2)); - Assert.assertTrue("Checksum mismatches!", - stripedFileChecksum1.equals(stripedFileChecksum2Recon)); - Assert.assertTrue("Checksum mismatches!", - stripedFileChecksum2.equals(stripedFileChecksum2Recon)); + assertTrue(stripedFileChecksum1.equals(stripedFileChecksum2), + "Checksum mismatches!"); + assertTrue(stripedFileChecksum1.equals(stripedFileChecksum2Recon), + "Checksum mismatches!"); + assertTrue(stripedFileChecksum2.equals(stripedFileChecksum2Recon), + "Checksum mismatches!"); } private void testStripedFileChecksumWithMissedDataBlocksRangeQuery( @@ -316,17 +350,20 @@ private void testStripedFileChecksumWithMissedDataBlocksRangeQuery( LOG.info("stripedFileChecksum1:" + stripedFileChecksum1); LOG.info("stripedFileChecksumRecon:" + stripedFileChecksumRecon); - Assert.assertTrue("Checksum mismatches!", - stripedFileChecksum1.equals(stripedFileChecksumRecon)); + assertTrue(stripedFileChecksum1.equals(stripedFileChecksumRecon), + "Checksum mismatches!"); } /** * Test to verify that the checksum can be computed for a small file less than * bytesPerCRC size. */ - @Test(timeout = 90000) - public void testStripedFileChecksumWithMissedDataBlocksRangeQuery1() + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksumWithMissedDataBlocksRangeQuery1(String pMode) throws Exception { + initTestFileChecksum(pMode); testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1, 1); } @@ -334,9 +371,12 @@ public void testStripedFileChecksumWithMissedDataBlocksRangeQuery1() * Test to verify that the checksum can be computed for a small file less than * bytesPerCRC size. */ - @Test(timeout = 90000) - public void testStripedFileChecksumWithMissedDataBlocksRangeQuery2() + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksumWithMissedDataBlocksRangeQuery2(String pMode) throws Exception { + initTestFileChecksum(pMode); testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1, 10); } @@ -345,9 +385,12 @@ public void testStripedFileChecksumWithMissedDataBlocksRangeQuery2() * length of file range for checksum calculation. 512 is the value of * bytesPerCRC. */ - @Test(timeout = 90000) - public void testStripedFileChecksumWithMissedDataBlocksRangeQuery3() + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksumWithMissedDataBlocksRangeQuery3(String pMode) throws Exception { + initTestFileChecksum(pMode); testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1, bytesPerCRC); } @@ -356,9 +399,12 @@ public void testStripedFileChecksumWithMissedDataBlocksRangeQuery3() * Test to verify that the checksum can be computed by giving 'cellsize' * length of file range for checksum calculation. */ - @Test(timeout = 90000) - public void testStripedFileChecksumWithMissedDataBlocksRangeQuery4() + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksumWithMissedDataBlocksRangeQuery4(String pMode) throws Exception { + initTestFileChecksum(pMode); testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1, cellSize); } @@ -367,9 +413,12 @@ public void testStripedFileChecksumWithMissedDataBlocksRangeQuery4() * Test to verify that the checksum can be computed by giving less than * cellsize length of file range for checksum calculation. */ - @Test(timeout = 90000) - public void testStripedFileChecksumWithMissedDataBlocksRangeQuery5() + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksumWithMissedDataBlocksRangeQuery5(String pMode) throws Exception { + initTestFileChecksum(pMode); testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1, cellSize - 1); } @@ -378,9 +427,12 @@ public void testStripedFileChecksumWithMissedDataBlocksRangeQuery5() * Test to verify that the checksum can be computed by giving greater than * cellsize length of file range for checksum calculation. */ - @Test(timeout = 90000) - public void testStripedFileChecksumWithMissedDataBlocksRangeQuery6() + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksumWithMissedDataBlocksRangeQuery6(String pMode) throws Exception { + initTestFileChecksum(pMode); testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1, cellSize + 1); } @@ -389,9 +441,12 @@ public void testStripedFileChecksumWithMissedDataBlocksRangeQuery6() * Test to verify that the checksum can be computed by giving two times * cellsize length of file range for checksum calculation. */ - @Test(timeout = 90000) - public void testStripedFileChecksumWithMissedDataBlocksRangeQuery7() + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksumWithMissedDataBlocksRangeQuery7(String pMode) throws Exception { + initTestFileChecksum(pMode); testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1, cellSize * 2); } @@ -400,9 +455,12 @@ public void testStripedFileChecksumWithMissedDataBlocksRangeQuery7() * Test to verify that the checksum can be computed by giving stripSize * length of file range for checksum calculation. */ - @Test(timeout = 90000) - public void testStripedFileChecksumWithMissedDataBlocksRangeQuery8() + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksumWithMissedDataBlocksRangeQuery8(String pMode) throws Exception { + initTestFileChecksum(pMode); testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1, stripSize); } @@ -411,9 +469,12 @@ public void testStripedFileChecksumWithMissedDataBlocksRangeQuery8() * Test to verify that the checksum can be computed by giving less than * stripSize length of file range for checksum calculation. */ - @Test(timeout = 90000) - public void testStripedFileChecksumWithMissedDataBlocksRangeQuery9() + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksumWithMissedDataBlocksRangeQuery9(String pMode) throws Exception { + initTestFileChecksum(pMode); testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1, stripSize - 1); } @@ -422,9 +483,12 @@ public void testStripedFileChecksumWithMissedDataBlocksRangeQuery9() * Test to verify that the checksum can be computed by giving greater than * stripSize length of file range for checksum calculation. */ - @Test(timeout = 90000) - public void testStripedFileChecksumWithMissedDataBlocksRangeQuery10() + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksumWithMissedDataBlocksRangeQuery10(String pMode) throws Exception { + initTestFileChecksum(pMode); testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1, stripSize + 1); } @@ -433,9 +497,12 @@ public void testStripedFileChecksumWithMissedDataBlocksRangeQuery10() * Test to verify that the checksum can be computed by giving less than * blockGroupSize length of file range for checksum calculation. */ - @Test(timeout = 90000) - public void testStripedFileChecksumWithMissedDataBlocksRangeQuery11() + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksumWithMissedDataBlocksRangeQuery11(String pMode) throws Exception { + initTestFileChecksum(pMode); testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1, blockGroupSize - 1); } @@ -444,9 +511,12 @@ public void testStripedFileChecksumWithMissedDataBlocksRangeQuery11() * Test to verify that the checksum can be computed by giving greaterthan * blockGroupSize length of file range for checksum calculation. */ - @Test(timeout = 90000) - public void testStripedFileChecksumWithMissedDataBlocksRangeQuery12() + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksumWithMissedDataBlocksRangeQuery12(String pMode) throws Exception { + initTestFileChecksum(pMode); testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1, blockGroupSize + 1); } @@ -455,9 +525,12 @@ public void testStripedFileChecksumWithMissedDataBlocksRangeQuery12() * Test to verify that the checksum can be computed by giving greater than * blockGroupSize length of file range for checksum calculation. */ - @Test(timeout = 90000) - public void testStripedFileChecksumWithMissedDataBlocksRangeQuery13() + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksumWithMissedDataBlocksRangeQuery13(String pMode) throws Exception { + initTestFileChecksum(pMode); testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1, blockGroupSize * numBlockGroups / 2); } @@ -466,9 +539,12 @@ public void testStripedFileChecksumWithMissedDataBlocksRangeQuery13() * Test to verify that the checksum can be computed by giving lessthan * fileSize length of file range for checksum calculation. */ - @Test(timeout = 90000) - public void testStripedFileChecksumWithMissedDataBlocksRangeQuery14() + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksumWithMissedDataBlocksRangeQuery14(String pMode) throws Exception { + initTestFileChecksum(pMode); testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1, fileSize - 1); } @@ -477,9 +553,12 @@ public void testStripedFileChecksumWithMissedDataBlocksRangeQuery14() * Test to verify that the checksum can be computed for a length greater than * file size. */ - @Test(timeout = 90000) - public void testStripedFileChecksumWithMissedDataBlocksRangeQuery15() + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksumWithMissedDataBlocksRangeQuery15(String pMode) throws Exception { + initTestFileChecksum(pMode); testStripedFileChecksumWithMissedDataBlocksRangeQuery(stripedFile1, fileSize * 2); } @@ -488,9 +567,12 @@ public void testStripedFileChecksumWithMissedDataBlocksRangeQuery15() * Test to verify that the checksum can be computed for a small file less than * bytesPerCRC size. */ - @Test(timeout = 90000) - public void testStripedFileChecksumWithMissedDataBlocksRangeQuery16() + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksumWithMissedDataBlocksRangeQuery16(String pMode) throws Exception { + initTestFileChecksum(pMode); int fileLength = 100; String stripedFile3 = ecDir + "/stripedFileChecksum3"; prepareTestFiles(fileLength, new String[] {stripedFile3}); @@ -502,9 +584,12 @@ public void testStripedFileChecksumWithMissedDataBlocksRangeQuery16() * Test to verify that the checksum can be computed for a small file less than * bytesPerCRC size. */ - @Test(timeout = 90000) - public void testStripedFileChecksumWithMissedDataBlocksRangeQuery17() + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksumWithMissedDataBlocksRangeQuery17(String pMode) throws Exception { + initTestFileChecksum(pMode); int fileLength = 100; String stripedFile3 = ecDir + "/stripedFileChecksum3"; prepareTestFiles(fileLength, new String[] {stripedFile3}); @@ -515,9 +600,12 @@ public void testStripedFileChecksumWithMissedDataBlocksRangeQuery17() * Test to verify that the checksum can be computed for a small file less than * bytesPerCRC size. */ - @Test(timeout = 90000) - public void testStripedFileChecksumWithMissedDataBlocksRangeQuery18() + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksumWithMissedDataBlocksRangeQuery18(String pMode) throws Exception { + initTestFileChecksum(pMode); int fileLength = 100; String stripedFile3 = ecDir + "/stripedFileChecksum3"; prepareTestFiles(fileLength, new String[] {stripedFile3}); @@ -528,9 +616,12 @@ public void testStripedFileChecksumWithMissedDataBlocksRangeQuery18() * Test to verify that the checksum can be computed with greater than file * length. */ - @Test(timeout = 90000) - public void testStripedFileChecksumWithMissedDataBlocksRangeQuery19() + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksumWithMissedDataBlocksRangeQuery19(String pMode) throws Exception { + initTestFileChecksum(pMode); int fileLength = 100; String stripedFile3 = ecDir + "/stripedFileChecksum3"; prepareTestFiles(fileLength, new String[] {stripedFile3}); @@ -542,9 +633,12 @@ public void testStripedFileChecksumWithMissedDataBlocksRangeQuery19() * Test to verify that the checksum can be computed for small file with less * than file length. */ - @Test(timeout = 90000) - public void testStripedFileChecksumWithMissedDataBlocksRangeQuery20() + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksumWithMissedDataBlocksRangeQuery20(String pMode) throws Exception { + initTestFileChecksum(pMode); int fileLength = bytesPerCRC; String stripedFile3 = ecDir + "/stripedFileChecksum3"; prepareTestFiles(fileLength, new String[] {stripedFile3}); @@ -552,9 +646,12 @@ public void testStripedFileChecksumWithMissedDataBlocksRangeQuery20() bytesPerCRC - 1); } - @Test(timeout = 90000) - public void testStripedFileChecksumWithReconstructFail() + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testStripedFileChecksumWithReconstructFail(String pMode) throws Exception { + initTestFileChecksum(pMode); String stripedFile4 = ecDir + "/stripedFileChecksum4"; prepareTestFiles(fileSize, new String[] {stripedFile4}); @@ -576,15 +673,17 @@ public void testStripedFileChecksumWithReconstructFail() // getting result. FileChecksum fileChecksum1 = getFileChecksum(stripedFile4, -1, true); - Assert.assertEquals("checksum should be same", fileChecksum, - fileChecksum1); + assertEquals(fileChecksum, fileChecksum1, "checksum should be same"); } finally { DataNodeFaultInjector.set(oldInjector); } } - @Test(timeout = 90000) - public void testMixedBytesPerChecksum() throws Exception { + @MethodSource("getParameters") + @ParameterizedTest + @Timeout(value = 90) + public void testMixedBytesPerChecksum(String pMode) throws Exception { + initTestFileChecksum(pMode); int fileLength = bytesPerCRC * 3; byte[] fileData = StripedFileTestUtil.generateBytes(fileLength); String replicatedFile1 = "/replicatedFile1"; @@ -610,10 +709,11 @@ public void testMixedBytesPerChecksum() throws Exception { DFSTestUtil.writeFile(fs, new Path(replicatedFile2), fileData); FileChecksum checksum1 = getFileChecksum(replicatedFile1, -1, false); FileChecksum checksum2 = getFileChecksum(replicatedFile2, -1, false); - Assert.assertEquals(checksum1, checksum2); + assertEquals(checksum1, checksum2); } else { - exception.expect(IOException.class); - FileChecksum checksum = getFileChecksum(replicatedFile1, -1, false); + assertThrows(IOException.class, () -> { + FileChecksum checksum = getFileChecksum(replicatedFile1, -1, false); + }); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileConcurrentReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileConcurrentReader.java index 0c7a3fcaae22d..a8d3c52fc6fef 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileConcurrentReader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileConcurrentReader.java @@ -16,10 +16,10 @@ * limitations under the License. */ package org.apache.hadoop.hdfs; -import static org.junit.Assert.assertFalse; -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.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import java.io.IOException; import java.util.Arrays; @@ -38,10 +38,11 @@ import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.StringUtils; -import org.junit.After; -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.event.Level; @@ -77,13 +78,13 @@ private enum SyncType { private FileSystem fileSystem; - @Before + @BeforeEach public void setUp() throws IOException { conf = new Configuration(); init(conf); } - @After + @AfterEach public void tearDown() throws Exception { if (cluster != null) { cluster.shutdown(); @@ -124,9 +125,8 @@ private void assertBytesAvailable( IOUtils.readFully(inputStream, buffer, 0, numBytes); inputStream.close(); - assertTrue( - "unable to validate bytes", - validateSequentialBytes(buffer, 0, numBytes) + assertTrue(validateSequentialBytes(buffer, 0, numBytes), + "unable to validate bytes" ); } @@ -153,7 +153,8 @@ private void waitForBlocks(FileSystem fileSys, Path name) /** * Test that that writes to an incomplete block are available to a reader */ - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testUnfinishedBlockRead() throws IOException { // create a new file in the root, write data, do no close @@ -176,7 +177,8 @@ public void testUnfinishedBlockRead() * would result in too small a buffer to do the buffer-copy needed * for partial chunks. */ - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testUnfinishedBlockPacketBufferOverrun() throws IOException { // check that / exists Path path = new Path("/"); @@ -202,7 +204,8 @@ public void testUnfinishedBlockPacketBufferOverrun() throws IOException { // use a small block size and a large write so that DN is busy creating // new blocks. This makes it almost 100% sure we can reproduce // case of client getting a DN that hasn't yet created the blocks - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testImmediateReadOfNewFile() throws IOException { final int blockSize = 64 * 1024; @@ -279,37 +282,41 @@ public void run() { // for some reason, using tranferTo evokes the race condition more often // so test separately - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testUnfinishedBlockCRCErrorTransferTo() throws IOException { runTestUnfinishedBlockCRCError(true, SyncType.SYNC, DEFAULT_WRITE_SIZE); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testUnfinishedBlockCRCErrorTransferToVerySmallWrite() throws IOException { runTestUnfinishedBlockCRCError(true, SyncType.SYNC, SMALL_WRITE_SIZE); } // fails due to issue w/append, disable - @Ignore + @Disabled public void _testUnfinishedBlockCRCErrorTransferToAppend() throws IOException { runTestUnfinishedBlockCRCError(true, SyncType.APPEND, DEFAULT_WRITE_SIZE); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testUnfinishedBlockCRCErrorNormalTransfer() throws IOException { runTestUnfinishedBlockCRCError(false, SyncType.SYNC, DEFAULT_WRITE_SIZE); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testUnfinishedBlockCRCErrorNormalTransferVerySmallWrite() throws IOException { runTestUnfinishedBlockCRCError(false, SyncType.SYNC, SMALL_WRITE_SIZE); } // fails due to issue w/append, disable - @Ignore + @Disabled public void _testUnfinishedBlockCRCErrorNormalTransferAppend() throws IOException { runTestUnfinishedBlockCRCError(false, SyncType.APPEND, DEFAULT_WRITE_SIZE); @@ -407,9 +414,7 @@ public void run() { writer.join(); tailer.join(); - assertFalse( - "error occurred, see log above", error.get() - ); + assertFalse(error.get(), "error occurred, see log above"); } catch (InterruptedException e) { LOG.info("interrupted waiting for writer or tailer to complete"); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java index bd04278a965f5..8cabb443daf5b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java @@ -22,9 +22,9 @@ import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; -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.DataInputStream; import java.io.DataOutputStream; @@ -56,7 +56,7 @@ import org.apache.hadoop.hdfs.util.RwLockMode; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.PathUtils; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.event.Level; @@ -87,7 +87,7 @@ public void testFileCorruption() throws Exception { DataNode dn = cluster.getDataNodes().get(2); Map blockReports = dn.getFSDataset().getBlockReports(bpid); - assertTrue("Blocks do not exist on data-dir", !blockReports.isEmpty()); + assertTrue(!blockReports.isEmpty(), "Blocks do not exist on data-dir"); for (BlockListAsLongs report : blockReports.values()) { for (BlockReportReplica brr : report) { LOG.info("Deliberately removing block {}", brr.getBlockName()); @@ -95,8 +95,8 @@ public void testFileCorruption() throws Exception { new ExtendedBlock(bpid, brr)).deleteData(); } } - assertTrue("Corrupted replicas not handled properly.", - util.checkFiles(fs, "/srcdat")); + assertTrue( + util.checkFiles(fs, "/srcdat"), "Corrupted replicas not handled properly."); util.cleanup(fs, "/srcdat"); } finally { if (cluster != null) { cluster.shutdown(); } @@ -147,8 +147,8 @@ public void testArrayOutOfBoundsException() throws Exception { // get the block final String bpid = cluster.getNamesystem().getBlockPoolId(); ExtendedBlock blk = getFirstBlock(cluster.getDataNodes().get(0), bpid); - assertFalse("Data directory does not contain any blocks or there was an " - + "IO error", blk==null); + assertFalse(blk==null, "Data directory does not contain any blocks or there was an " + + "IO error"); // start a third datanode cluster.startDataNodes(conf, 1, true, null, null); @@ -198,14 +198,14 @@ public void testCorruptionWithDiskFailure() throws Exception { final String bpid = cluster.getNamesystem().getBlockPoolId(); File storageDir = cluster.getInstanceStorageDir(0, 0); File dataDir = MiniDFSCluster.getFinalizedDir(storageDir, bpid); - assertTrue("Data directory does not exist", dataDir.exists()); + assertTrue(dataDir.exists(), "Data directory does not exist"); ExtendedBlock blk = getFirstBlock(cluster.getDataNodes().get(0), bpid); if (blk == null) { blk = getFirstBlock(cluster.getDataNodes().get(0), bpid); } - assertFalse("Data directory does not contain any blocks or there was an" + + assertFalse(blk == null, "Data directory does not contain any blocks or there was an" + " " + - "IO error", blk == null); + "IO error"); ArrayList datanodes = cluster.getDataNodes(); assertEquals(datanodes.size(), 3); FSNamesystem ns = cluster.getNamesystem(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java index 3d0bbf93a4ac1..3d94ebbb6aea0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java @@ -34,10 +34,11 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY; import static org.apache.hadoop.test.MetricsAsserts.assertCounter; import static org.apache.hadoop.test.MetricsAsserts.getMetrics; -import static org.junit.Assert.assertEquals; -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.assertArrayEquals; +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 static org.junit.jupiter.api.Assertions.fail; import static org.junit.Assume.assumeTrue; import static org.mockito.Mockito.doReturn; @@ -90,8 +91,8 @@ import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.ReflectionUtils; import org.apache.hadoop.util.Time; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.mockito.Mockito; import org.slf4j.event.Level; @@ -114,7 +115,7 @@ public class TestFileCreation { static final int numBlocks = 2; static final int fileSize = numBlocks * blockSize + 1; boolean simulatedStorage = false; - + private static final String[] NON_CANONICAL_PATHS = new String[] { "//foo", "///foo2", @@ -353,8 +354,8 @@ public void checkFileCreation(String netIf, boolean useDnHostname) Path path = new Path("/"); System.out.println("Path : \"" + path.toString() + "\""); System.out.println(fs.getFileStatus(path).isDirectory()); - assertTrue("/ should be a directory", - fs.getFileStatus(path).isDirectory()); + assertTrue( + fs.getFileStatus(path).isDirectory(), "/ should be a directory"); // // Create a directory inside /, then try to overwrite it @@ -366,7 +367,7 @@ public void checkFileCreation(String netIf, boolean useDnHostname) try { fs.create(dir1, true); // Create path, overwrite=true fs.close(); - assertTrue("Did not prevent directory from being overwritten.", false); + assertTrue(false, "Did not prevent directory from being overwritten."); } catch (FileAlreadyExistsException e) { // expected } @@ -382,8 +383,8 @@ public void checkFileCreation(String netIf, boolean useDnHostname) FSDataOutputStream stm = createFile(fs, file1, 1); // verify that file exists in FS namespace - assertTrue(file1 + " should be a file", - fs.getFileStatus(file1).isFile()); + assertTrue( + fs.getFileStatus(file1).isFile(), file1 + " should be a file"); System.out.println("Path : \"" + file1 + "\""); // write to file @@ -393,14 +394,14 @@ public void checkFileCreation(String netIf, boolean useDnHostname) // verify that file size has changed to the full size long len = fs.getFileStatus(file1).getLen(); - assertTrue(file1 + " should be of size " + fileSize + - " but found to be of size " + len, - len == fileSize); + assertTrue( + len == fileSize, file1 + " should be of size " + fileSize + + " but found to be of size " + len); // verify the disk space the file occupied long diskSpace = dfs.getContentSummary(file1.getParent()).getLength(); - assertEquals(file1 + " should take " + fileSize + " bytes disk space " + - "but found to take " + diskSpace + " bytes", fileSize, diskSpace); + assertEquals(fileSize, diskSpace, file1 + " should take " + fileSize + " bytes disk space " + + "but found to take " + diskSpace + " bytes"); // Check storage usage // can't check capacities for real storage since the OS file system may be changing under us. @@ -464,12 +465,12 @@ public void testDeleteOnExit() throws IOException { fs = cluster.getFileSystem(); localfs = FileSystem.getLocal(conf); - assertTrue(file1 + " still exists inspite of deletOnExit set.", - !fs.exists(file1)); - assertTrue(file2 + " still exists inspite of deletOnExit set.", - !fs.exists(file2)); - assertTrue(file3 + " still exists inspite of deletOnExit set.", - !localfs.exists(file3)); + assertTrue( + !fs.exists(file1), file1 + " still exists inspite of deletOnExit set."); + assertTrue( + !fs.exists(file2), file2 + " still exists inspite of deletOnExit set."); + assertTrue( + !localfs.exists(file3), file3 + " still exists inspite of deletOnExit set."); System.out.println("DeleteOnExit successful."); } finally { @@ -564,8 +565,8 @@ public void testFileCreationError1() throws IOException { FSDataOutputStream stm = createFile(fs, file1, 1); // verify that file exists in FS namespace - assertTrue(file1 + " should be a file", - fs.getFileStatus(file1).isFile()); + assertTrue( + fs.getFileStatus(file1).isFile(), file1 + " should be a file"); System.out.println("Path : \"" + file1 + "\""); // kill the datanode @@ -601,8 +602,8 @@ public void testFileCreationError1() throws IOException { LocatedBlocks locations = client.getNamenode().getBlockLocations( file1.toString(), 0, Long.MAX_VALUE); System.out.println("locations = " + locations.locatedBlockCount()); - assertTrue("Error blocks were not cleaned up", - locations.locatedBlockCount() == 0); + assertTrue( + locations.locatedBlockCount() == 0, "Error blocks were not cleaned up"); } finally { cluster.shutdown(); client.close(); @@ -737,14 +738,14 @@ public void testFileCreationNamenodeRestart() HdfsDataOutputStream stm = create(fs, file1, 1); System.out.println("testFileCreationNamenodeRestart: " + "Created file " + file1); - assertEquals(file1 + " should be replicated to 1 datanode.", 1, - stm.getCurrentBlockReplication()); + assertEquals(1, stm.getCurrentBlockReplication(), + file1 + " should be replicated to 1 datanode."); // write two full blocks. writeFile(stm, numBlocks * blockSize); stm.hflush(); - assertEquals(file1 + " should still be replicated to 1 datanode.", 1, - stm.getCurrentBlockReplication()); + assertEquals(1, stm.getCurrentBlockReplication(), + file1 + " should still be replicated to 1 datanode."); // rename file wile keeping it open. Path fileRenamed = new Path("/filestatusRenamed.dat"); @@ -834,15 +835,15 @@ public void testFileCreationNamenodeRestart() LocatedBlocks locations = client.getNamenode().getBlockLocations( file1.toString(), 0, Long.MAX_VALUE); System.out.println("locations = " + locations.locatedBlockCount()); - assertTrue("Error blocks were not cleaned up for file " + file1, - locations.locatedBlockCount() == 3); + assertTrue(locations.locatedBlockCount() == 3, + "Error blocks were not cleaned up for file " + file1); // verify filestatus2.dat locations = client.getNamenode().getBlockLocations( file2.toString(), 0, Long.MAX_VALUE); System.out.println("locations = " + locations.locatedBlockCount()); - assertTrue("Error blocks were not cleaned up for file " + file2, - locations.locatedBlockCount() == 1); + assertTrue(locations.locatedBlockCount() == 1, + "Error blocks were not cleaned up for file " + file2); } finally { IOUtils.closeStream(fs); cluster.shutdown(); @@ -879,8 +880,8 @@ public void testDFSClientDeath() throws IOException, InterruptedException { dfsclient.close(); // reopen file system and verify that file exists. - assertTrue(file1 + " does not exist.", - AppendTestUtil.createHdfsWithDifferentUsername(conf).exists(file1)); + assertTrue(AppendTestUtil.createHdfsWithDifferentUsername(conf).exists(file1), + file1 + " does not exist."); } finally { cluster.shutdown(); } @@ -922,19 +923,19 @@ public static void testFileCreationNonRecursive(FileSystem fs) throws IOExceptio // Create a file when parent dir exists as file, should fail expectedException = createNonRecursive(fs, new Path(path, "Create"), 1, createFlag); - assertTrue("Create a file when parent directory exists as a file" - + " should throw ParentNotDirectoryException ", - expectedException != null - && expectedException instanceof ParentNotDirectoryException); + assertTrue(expectedException != null + && expectedException instanceof ParentNotDirectoryException, + "Create a file when parent directory exists as a file" + + " should throw ParentNotDirectoryException "); fs.delete(path, true); // Create a file in a non-exist directory, should fail final Path path2 = new Path(nonExistDir + "/testCreateNonRecursive"); expectedException = createNonRecursive(fs, path2, 1, createFlag); - assertTrue("Create a file in a non-exist dir using" - + " createNonRecursive() should throw FileNotFoundException ", - expectedException != null - && expectedException instanceof FileNotFoundException); + assertTrue(expectedException != null + && expectedException instanceof FileNotFoundException, + "Create a file in a non-exist dir using" + + " createNonRecursive() should throw FileNotFoundException "); EnumSet overwriteFlag = EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE); @@ -944,20 +945,20 @@ public static void testFileCreationNonRecursive(FileSystem fs) throws IOExceptio // Overwrite a file when parent dir exists as file, should fail expectedException = createNonRecursive(fs, new Path(path, "Overwrite"), 1, overwriteFlag); - assertTrue("Overwrite a file when parent directory exists as a file" - + " should throw ParentNotDirectoryException ", - expectedException != null - && expectedException instanceof ParentNotDirectoryException); + assertTrue(expectedException != null + && expectedException instanceof ParentNotDirectoryException, + "Overwrite a file when parent directory exists as a file" + + " should throw ParentNotDirectoryException "); fs.delete(path, true); // Overwrite a file in a non-exist directory, should fail final Path path3 = new Path(nonExistDir + "/testOverwriteNonRecursive"); expectedException = createNonRecursive(fs, path3, 1, overwriteFlag); - assertTrue("Overwrite a file in a non-exist dir using" - + " createNonRecursive() should throw FileNotFoundException ", - expectedException != null - && expectedException instanceof FileNotFoundException); + assertTrue(expectedException != null + && expectedException instanceof FileNotFoundException, + "Overwrite a file in a non-exist dir using" + + " createNonRecursive() should throw FileNotFoundException "); } // Attempts to create and close a file using FileSystem.createNonRecursive(), @@ -1086,8 +1087,8 @@ public void testLeaseExpireHardLimit() throws Exception { out.write("something".getBytes()); out.hflush(); int actualRepl = out.getCurrentBlockReplication(); - assertTrue(f + " should be replicated to " + DATANODE_NUM + " datanodes.", - actualRepl == DATANODE_NUM); + assertTrue(actualRepl == DATANODE_NUM, f + " should be replicated to " + + DATANODE_NUM + " datanodes."); // set the soft and hard limit to be 1 second so that the // namenode triggers lease recovery @@ -1110,7 +1111,7 @@ public void testLeaseExpireHardLimit() throws Exception { } } System.out.println("successcount=" + successcount); - assertTrue(successcount > 0); + assertTrue(successcount > 0); } finally { IOUtils.closeStream(dfs); cluster.shutdown(); @@ -1186,7 +1187,7 @@ public void testFsCloseAfterClusterShutdown() throws IOException { } catch (IOException e) { hasException = true; } - assertTrue("Failed to close file after cluster shutdown", hasException); + assertTrue(hasException, "Failed to close file after cluster shutdown"); } finally { System.out.println("testFsCloseAfterClusterShutdown successful"); if (cluster != null) { @@ -1321,7 +1322,8 @@ public void testFileIdMismatch() throws IOException { * 2. Restart NN, check the file * 3. Save new checkpoint and restart NN, check the file */ - @Test(timeout = 120000) + @Test + @Timeout(value = 120) public void testFileCreationWithOverwrite() throws Exception { Configuration conf = new Configuration(); conf.setInt("dfs.blocksize", blockSize); @@ -1376,7 +1378,7 @@ public void testFileCreationWithOverwrite() throws Exception { } finally { in.close(); } - Assert.assertArrayEquals(newData, result); + assertArrayEquals(newData, result); // Case 2: Restart NN, check the file cluster.restartNameNode(); @@ -1387,7 +1389,7 @@ public void testFileCreationWithOverwrite() throws Exception { } finally { in.close(); } - Assert.assertArrayEquals(newData, result); + assertArrayEquals(newData, result); // Case 3: Save new checkpoint and restart NN, check the file NameNodeAdapter.enterSafeMode(nn, false); @@ -1401,7 +1403,7 @@ public void testFileCreationWithOverwrite() throws Exception { } finally { in.close(); } - Assert.assertArrayEquals(newData, result); + assertArrayEquals(newData, result); } finally { if (dfs != null) { dfs.close(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationClient.java index 986bb560ad366..41b5b340c8805 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationClient.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hdfs; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; @@ -30,7 +30,7 @@ import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.test.GenericTestUtils; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationDelete.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationDelete.java index 728fa7557e3a0..77ff225b5e4bc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationDelete.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationDelete.java @@ -16,7 +16,7 @@ * limitations under the License. */ package org.apache.hadoop.hdfs; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.IOException; @@ -24,7 +24,7 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; public class TestFileCreationDelete { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationEmpty.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationEmpty.java index fbada206bdb71..2373ea5b3d97a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationEmpty.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationEmpty.java @@ -16,14 +16,14 @@ * limitations under the License. */ package org.apache.hadoop.hdfs; -import static org.junit.Assert.assertFalse; +import static org.junit.jupiter.api.Assertions.assertFalse; import java.util.ConcurrentModificationException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.server.namenode.LeaseManager; -import org.junit.Test; +import org.junit.jupiter.api.Test; /** * Test empty file creation. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileLengthOnClusterRestart.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileLengthOnClusterRestart.java index f993db0f1fae6..d3c6d398b46e7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileLengthOnClusterRestart.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileLengthOnClusterRestart.java @@ -23,8 +23,12 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.client.HdfsDataInputStream; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; /** Test the fileLength on cluster restarts */ public class TestFileLengthOnClusterRestart { @@ -32,7 +36,8 @@ public class TestFileLengthOnClusterRestart { * Tests the fileLength when we sync the file and restart the cluster and * Datanodes not report to Namenode yet. */ - @Test(timeout = 120000) + @Test + @Timeout(value = 120) public void testFileLengthWithHSyncAndClusterRestartWithOutDNsRegister() throws Exception { final Configuration conf = new HdfsConfiguration(); @@ -55,7 +60,7 @@ public void testFileLengthWithHSyncAndClusterRestartWithOutDNsRegister() in = (HdfsDataInputStream) dfs.open(path, 1024); // Verify the length when we just restart NN. DNs will register // immediately. - Assert.assertEquals(fileLength, in.getVisibleLength()); + assertEquals(fileLength, in.getVisibleLength()); cluster.shutdownDataNodes(); cluster.restartNameNode(false); // This is just for ensuring NN started. @@ -63,9 +68,9 @@ public void testFileLengthWithHSyncAndClusterRestartWithOutDNsRegister() try { in = (HdfsDataInputStream) dfs.open(path); - Assert.fail("Expected IOException"); + fail("Expected IOException"); } catch (IOException e) { - Assert.assertTrue(e.getLocalizedMessage().indexOf( + assertTrue(e.getLocalizedMessage().indexOf( "Name node is in safe mode") >= 0); } } finally { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java index a5f8911b97658..1e9cf9ac240d7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java @@ -17,10 +17,10 @@ */ package org.apache.hadoop.hdfs; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import java.io.FileNotFoundException; import java.io.IOException; @@ -38,9 +38,9 @@ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.test.GenericTestUtils; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; /** @@ -63,7 +63,7 @@ public class TestFileStatus { private static DFSClient dfsClient; private static Path file1; - @BeforeClass + @BeforeAll public static void testSetUp() throws Exception { conf = new HdfsConfiguration(); conf.setInt(DFSConfigKeys.DFS_LIST_LIMIT, 2); @@ -76,7 +76,7 @@ public static void testSetUp() throws Exception { seed); } - @AfterClass + @AfterAll public static void testTearDown() throws Exception { if (fs != null) { fs.close(); @@ -96,13 +96,13 @@ private void checkFile(FileSystem fileSys, Path name, int repl) public void testGetFileInfo() throws IOException { // Check that / exists Path path = new Path("/"); - assertTrue("/ should be a directory", - fs.getFileStatus(path).isDirectory()); + assertTrue( + fs.getFileStatus(path).isDirectory(), "/ should be a directory"); ContractTestUtils.assertNotErasureCoded(fs, path); // Make sure getFileInfo returns null for files which do not exist HdfsFileStatus fileInfo = dfsClient.getFileInfo("/noSuchFile"); - assertEquals("Non-existant file should result in null", null, fileInfo); + assertEquals(null, fileInfo, "Non-existant file should result in null"); Path path1 = new Path("/name1"); Path path2 = new Path("/name1/name2"); @@ -119,8 +119,8 @@ public void testGetFileInfo() throws IOException { dfsClient.getFileInfo("non-absolute"); fail("getFileInfo for a non-absolute path did not throw IOException"); } catch (RemoteException re) { - assertTrue("Wrong exception for invalid file name: "+re, - re.toString().contains("Absolute path required")); + assertTrue(re.toString().contains("Absolute path required"), + "Wrong exception for invalid file name: " + re); } } @@ -131,7 +131,7 @@ public void testGetFileStatusOnFile() throws Exception { checkFile(fs, file1, 1); // test getFileStatus on a file FileStatus status = fs.getFileStatus(file1); - assertFalse(file1 + " should be a file", status.isDirectory()); + assertFalse(status.isDirectory(), file1 + " should be a file"); assertEquals(blockSize, status.getBlockSize()); assertEquals(1, status.getReplication()); assertEquals(fileSize, status.getLen()); @@ -139,9 +139,9 @@ public void testGetFileStatusOnFile() throws Exception { assertEquals(file1.makeQualified(fs.getUri(), fs.getWorkingDirectory()).toString(), status.getPath().toString()); - assertTrue(file1 + " should have erasure coding unset in " + - "FileStatus#toString(): " + status, - status.toString().contains("isErasureCoded=false")); + assertTrue(status.toString().contains("isErasureCoded=false"), + file1 + " should have erasure coding unset in " + + "FileStatus#toString(): " + status); } /** Test the FileStatus obtained calling listStatus on a file */ @@ -150,7 +150,7 @@ public void testListStatusOnFile() throws IOException { FileStatus[] stats = fs.listStatus(file1); assertEquals(1, stats.length); FileStatus status = stats[0]; - assertFalse(file1 + " should be a file", status.isDirectory()); + assertFalse(status.isDirectory(), file1 + " should be a file"); assertEquals(blockSize, status.getBlockSize()); assertEquals(1, status.getReplication()); assertEquals(fileSize, status.getLen()); @@ -162,7 +162,7 @@ public void testListStatusOnFile() throws IOException { RemoteIterator itor = fc.listStatus(file1); status = itor.next(); assertEquals(stats[0], status); - assertFalse(file1 + " should be a file", status.isDirectory()); + assertFalse(status.isDirectory(), file1 + " should be a file"); } /** Test getting a FileStatus object using a non-existant path */ @@ -186,8 +186,8 @@ public void testGetFileStatusOnNonExistantFileDir() throws IOException { fs.getFileStatus(dir); fail("getFileStatus of non-existent path should fail"); } catch (FileNotFoundException fe) { - assertTrue("Exception doesn't indicate non-existant path", - fe.getMessage().startsWith("File does not exist")); + assertTrue(fe.getMessage().startsWith("File does not exist"), + "Exception doesn't indicate non-existant path"); } } @@ -196,13 +196,13 @@ public void testGetFileStatusOnNonExistantFileDir() throws IOException { public void testGetFileStatusOnDir() throws Exception { // Create the directory Path dir = new Path("/test/mkdirs"); - assertTrue("mkdir failed", fs.mkdirs(dir)); - assertTrue("mkdir failed", fs.exists(dir)); + assertTrue(fs.mkdirs(dir), "mkdir failed"); + assertTrue(fs.exists(dir), "mkdir failed"); // test getFileStatus on an empty directory FileStatus status = fs.getFileStatus(dir); - assertTrue(dir + " should be a directory", status.isDirectory()); - assertTrue(dir + " should be zero size ", status.getLen() == 0); + assertTrue(status.isDirectory(), dir + " should be a directory"); + assertTrue(status.getLen() == 0, dir + " should be zero size "); ContractTestUtils.assertNotErasureCoded(fs, dir); assertEquals(dir.makeQualified(fs.getUri(), fs.getWorkingDirectory()).toString(), @@ -210,15 +210,15 @@ public void testGetFileStatusOnDir() throws Exception { // test listStatus on an empty directory FileStatus[] stats = fs.listStatus(dir); - assertEquals(dir + " should be empty", 0, stats.length); - assertEquals(dir + " should be zero size ", - 0, fs.getContentSummary(dir).getLength()); + assertEquals(0, stats.length, dir + " should be empty"); + assertEquals(0, fs.getContentSummary(dir).getLength(), + dir + " should be zero size "); RemoteIterator itor = fc.listStatus(dir); - assertFalse(dir + " should be empty", itor.hasNext()); + assertFalse(itor.hasNext(), dir + " should be empty"); itor = fs.listStatusIterator(dir); - assertFalse(dir + " should be empty", itor.hasNext()); + assertFalse(itor.hasNext(), dir + " should be empty"); // create another file that is smaller than a block. Path file2 = new Path(dir, "filestatus2.dat"); @@ -242,25 +242,25 @@ public void testGetFileStatusOnDir() throws Exception { // Verify that the size of the directory increased by the size // of the two files - final int expected = blockSize/2; - assertEquals(dir + " size should be " + expected, - expected, fs.getContentSummary(dir).getLength()); + final int expected = blockSize/2; + assertEquals(expected, fs.getContentSummary(dir).getLength(), + dir + " size should be " + expected); // Test listStatus on a non-empty directory stats = fs.listStatus(dir); - assertEquals(dir + " should have two entries", 2, stats.length); + assertEquals(2, stats.length, dir + " should have two entries"); assertEquals(file2.toString(), stats[0].getPath().toString()); assertEquals(file3.toString(), stats[1].getPath().toString()); itor = fc.listStatus(dir); assertEquals(file2.toString(), itor.next().getPath().toString()); assertEquals(file3.toString(), itor.next().getPath().toString()); - assertFalse("Unexpected addtional file", itor.hasNext()); + assertFalse(itor.hasNext(), "Unexpected addtional file"); itor = fs.listStatusIterator(dir); assertEquals(file2.toString(), itor.next().getPath().toString()); assertEquals(file3.toString(), itor.next().getPath().toString()); - assertFalse("Unexpected addtional file", itor.hasNext()); + assertFalse(itor.hasNext(), "Unexpected addtional file"); // Test iterative listing. Now dir has 2 entries, create one more. @@ -268,7 +268,7 @@ public void testGetFileStatusOnDir() throws Exception { fs.mkdirs(dir3); dir3 = fs.makeQualified(dir3); stats = fs.listStatus(dir); - assertEquals(dir + " should have three entries", 3, stats.length); + assertEquals(3, stats.length, dir + " should have three entries"); assertEquals(dir3.toString(), stats[0].getPath().toString()); assertEquals(file2.toString(), stats[1].getPath().toString()); assertEquals(file3.toString(), stats[2].getPath().toString()); @@ -277,13 +277,13 @@ public void testGetFileStatusOnDir() throws Exception { assertEquals(dir3.toString(), itor.next().getPath().toString()); assertEquals(file2.toString(), itor.next().getPath().toString()); assertEquals(file3.toString(), itor.next().getPath().toString()); - assertFalse("Unexpected addtional file", itor.hasNext()); + assertFalse(itor.hasNext(), "Unexpected addtional file"); itor = fs.listStatusIterator(dir); assertEquals(dir3.toString(), itor.next().getPath().toString()); assertEquals(file2.toString(), itor.next().getPath().toString()); assertEquals(file3.toString(), itor.next().getPath().toString()); - assertFalse("Unexpected addtional file", itor.hasNext()); + assertFalse(itor.hasNext(), "Unexpected addtional file"); // Now dir has 3 entries, create two more Path dir4 = fs.makeQualified(new Path(dir, "dir4")); @@ -293,7 +293,7 @@ public void testGetFileStatusOnDir() throws Exception { fs.mkdirs(dir5); dir5 = fs.makeQualified(dir5); stats = fs.listStatus(dir); - assertEquals(dir + " should have five entries", 5, stats.length); + assertEquals(5, stats.length, dir + " should have five entries"); assertEquals(dir3.toString(), stats[0].getPath().toString()); assertEquals(dir4.toString(), stats[1].getPath().toString()); assertEquals(dir5.toString(), stats[2].getPath().toString()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusSerialization.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusSerialization.java index 444c0ec156a4d..8a9001e0c45cc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusSerialization.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusSerialization.java @@ -36,8 +36,8 @@ import org.apache.hadoop.thirdparty.protobuf.ByteString; -import org.junit.Test; -import static org.junit.Assert.*; +import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.assertEquals; /** * Verify compatible FileStatus/HdfsFileStatus serialization. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithDefaultECPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithDefaultECPolicy.java index a57777a0a1a0d..333e07bdc0888 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithDefaultECPolicy.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithDefaultECPolicy.java @@ -17,9 +17,9 @@ */ package org.apache.hadoop.hdfs; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +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 java.io.IOException; @@ -28,24 +28,21 @@ import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.Timeout; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; /** * This test ensures the statuses of EC files with the default policy. */ +@Timeout(300) public class TestFileStatusWithDefaultECPolicy { private MiniDFSCluster cluster; private DistributedFileSystem fs; private DFSClient client; - @Rule - public Timeout globalTimeout = new Timeout(300000); - - @Before + @BeforeEach public void before() throws IOException { HdfsConfiguration conf = new HdfsConfiguration(); cluster = @@ -56,7 +53,7 @@ public void before() throws IOException { fs.enableErasureCodingPolicy(getEcPolicy().getName()); } - @After + @AfterEach public void after() { if (cluster != null) { cluster.shutdown(); @@ -100,8 +97,7 @@ public void testFileStatusWithECPolicy() throws Exception { assertTrue(ecPolicy1.equals(ecPolicy3)); ContractTestUtils.assertErasureCoded(fs, file); FileStatus status = fs.getFileStatus(file); - assertTrue(file + " should have erasure coding set in " + - "FileStatus#toString(): " + status, - status.toString().contains("isErasureCoded=true")); + assertTrue(status.toString().contains("isErasureCoded=true"), + file + " should have erasure coding set in " + "FileStatus#toString(): " + status); } }