diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java index 5de131e6e411c..da89b85f6de78 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java @@ -30,13 +30,13 @@ import static org.apache.hadoop.test.MetricsAsserts.assertCounter; import static org.apache.hadoop.test.MetricsAsserts.getLongCounter; import static org.apache.hadoop.test.MetricsAsserts.getMetrics; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertSame; -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.assertNull; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import java.io.File; import java.io.IOException; @@ -87,11 +87,11 @@ import org.apache.hadoop.test.PathUtils; import org.apache.hadoop.util.Lists; import org.apache.hadoop.util.Time; -import org.junit.Before; -import org.junit.After; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.io.TempDir; import org.mockito.ArgumentCaptor; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; @@ -122,8 +122,9 @@ public class TestBPOfferService { GenericTestUtils.setLogLevel(DataNode.LOG, Level.TRACE); } - @Rule - public TemporaryFolder baseDir = new TemporaryFolder(); + @SuppressWarnings("checkstyle:VisibilityModifier") + @TempDir + java.nio.file.Path baseDir; private DatanodeProtocolClientSideTranslatorPB mockNN1; private DatanodeProtocolClientSideTranslatorPB mockNN2; @@ -138,7 +139,7 @@ public class TestBPOfferService { private boolean isSlownode; private String mockStorageID; - @Before + @BeforeEach public void setupMocks() throws Exception { mockNN1 = setupNNMock(0); mockNN2 = setupNNMock(1); @@ -166,7 +167,7 @@ public void setupMocks() throws Exception { Mockito.doReturn(dataSetLockManager).when(mockDn).getDataSetLockManager(); } - @After + @AfterEach public void checkDataSetLockManager() { dataSetLockManager.lockLeakCheck(); // make sure no lock Leak. @@ -831,9 +832,8 @@ public void testReportBadBlockWhenStandbyNNTimesOut() throws Exception { .getStorageType()); Thread.sleep(10000); long difference = secondCallTime - firstCallTime; - assertTrue("Active namenode reportBadBlock processing should be " - + "independent of standby namenode reportBadBlock processing ", - difference < 5000); + assertTrue(difference < 5000, "Active namenode reportBadBlock processing should be " + + "independent of standby namenode reportBadBlock processing "); } finally { bpos.stop(); bpos.join(); @@ -871,9 +871,9 @@ public void testTrySendErrorReportWhenStandbyNNTimesOut() throws Exception { bpos.trySendErrorReport(DatanodeProtocol.INVALID_BLOCK, errorString); Thread.sleep(10000); long difference = secondCallTime - firstCallTime; - assertTrue("Active namenode trySendErrorReport processing " + assertTrue(difference < 5000, "Active namenode trySendErrorReport processing " + "should be independent of standby namenode trySendErrorReport" - + " processing ", difference < 5000); + + " processing "); } finally { bpos.stop(); bpos.join(); @@ -911,8 +911,8 @@ public void testTrySendErrorReportWhenNNThrowsIOException() String errorString = "Can't send invalid block " + FAKE_BLOCK; bpos.trySendErrorReport(DatanodeProtocol.INVALID_BLOCK, errorString); GenericTestUtils.waitFor(() -> secondCallTime != 0, 100, 20000); - assertTrue("Active namenode didn't add the report back to the queue " - + "when errorReport threw IOException", secondCallTime != 0); + assertTrue(secondCallTime != 0, "Active namenode didn't add the report back to the queue " + + "when errorReport threw IOException"); } finally { bpos.stop(); bpos.join(); @@ -1025,9 +1025,7 @@ public Boolean get() { // Send register command back to Datanode to reRegister(). // After reRegister IBRs should be cleared. datanodeCommands[1] = new DatanodeCommand[] { new RegisterCommand() }; - assertEquals( - "IBR size before reRegister should be non-0", 1, getStandbyIBRSize( - bpos)); + assertEquals(1, getStandbyIBRSize(bpos), "IBR size before reRegister should be non-0"); bpos.triggerHeartbeatForTests(); GenericTestUtils.waitFor(new Supplier() { @Override @@ -1082,7 +1080,8 @@ public void testNNHAStateUpdateFromVersionRequest() throws Exception { } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testRefreshNameNodes() throws Exception { BPOfferService bpos = setupBPOSForNNs(mockDn, mockNN1, mockNN2); @@ -1156,7 +1155,8 @@ public void testRefreshNameNodes() throws Exception { } } - @Test(timeout = 15000) + @Test + @Timeout(value = 15) public void testRefreshLeaseId() throws Exception { Mockito.when(mockNN1.sendHeartbeat( Mockito.any(DatanodeRegistration.class), @@ -1221,7 +1221,8 @@ public Object answer(InvocationOnMock invocation) } } - @Test(timeout = 15000) + @Test + @Timeout(value = 15) public void testSetIsSlownode() throws Exception { assertEquals(mockDn.isSlownode(), false); Mockito.when(mockNN1.sendHeartbeat( @@ -1259,10 +1260,11 @@ public void testSetIsSlownode() throws Exception { } } - @Test(timeout = 15000) + @Test + @Timeout(value = 15) public void testCommandProcessingThread() throws Exception { Configuration conf = new HdfsConfiguration(); - try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf, baseDir.getRoot()).build()) { + try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf, baseDir.toFile()).build()) { List datanodes = cluster.getDataNodes(); assertEquals(datanodes.size(), 1); DataNode datanode = datanodes.get(0); @@ -1273,8 +1275,8 @@ public void testCommandProcessingThread() throws Exception { DFSTestUtil.createFile(fs, file, 10240L, (short)1, 0L); MetricsRecordBuilder mrb = getMetrics(datanode.getMetrics().name()); - assertTrue("Process command nums is not expected.", - getLongCounter("NumProcessedCommands", mrb) > 0); + assertTrue(getLongCounter("NumProcessedCommands", mrb) > 0, + "Process command nums is not expected."); assertEquals(0, getLongCounter("SumOfActorCommandQueueLength", mrb)); // Check new metric result about processedCommandsOp. // One command send back to DataNode here is #FinalizeCommand. @@ -1282,10 +1284,11 @@ public void testCommandProcessingThread() throws Exception { } } - @Test(timeout = 5000) + @Test + @Timeout(value = 5) public void testCommandProcessingThreadExit() throws Exception { Configuration conf = new HdfsConfiguration(); - try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf, baseDir.getRoot()). + try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf, baseDir.toFile()). numDataNodes(1).build()) { List datanodes = cluster.getDataNodes(); DataNode dataNode = datanodes.get(0); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBatchIbr.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBatchIbr.java index f44075014e8b2..3365f934f9a20 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBatchIbr.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBatchIbr.java @@ -45,8 +45,8 @@ import org.apache.hadoop.test.MetricsAsserts; import org.apache.hadoop.util.Time; import org.slf4j.event.Level; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; /** * This test verifies that incremental block reports are sent in batch mode @@ -161,7 +161,7 @@ public Boolean call() throws Exception { }); } for(int i = 0; i < NUM_FILES; i++) { - Assert.assertTrue(verifyService.take().get()); + Assertions.assertTrue(verifyService.take().get()); } final long testEndTime = Time.monotonicNow(); @@ -247,7 +247,7 @@ static boolean verifyFile(Path f, DistributedFileSystem dfs) { for(int i = 0; i < numBlocks; i++) { in.read(computed); nextBytes(i, seed, expected); - Assert.assertArrayEquals(expected, computed); + Assertions.assertArrayEquals(expected, computed); } return true; } catch(Exception e) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockCountersInPendingIBR.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockCountersInPendingIBR.java index 6f5e6208c40f9..ececeb07e8141 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockCountersInPendingIBR.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockCountersInPendingIBR.java @@ -36,7 +36,7 @@ import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus; import org.apache.hadoop.metrics2.MetricsRecordBuilder; import org.apache.hadoop.test.MetricsAsserts; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java index 958b3e46cf46e..e40ff2e57ca8e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java @@ -40,13 +40,11 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport; import org.apache.hadoop.test.GenericTestUtils; -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 static org.hamcrest.CoreMatchers.not; -import static org.hamcrest.core.Is.is; -import static org.junit.Assert.assertThat; +import static org.assertj.core.api.Assertions.assertThat; /** * This test verifies NameNode behavior when it gets unexpected block reports @@ -68,7 +66,7 @@ public class TestBlockHasMultipleReplicasOnSameDN { private DFSClient client; private String bpid; - @Before + @BeforeEach public void startUpCluster() throws IOException { conf = new HdfsConfiguration(); cluster = new MiniDFSCluster.Builder(conf) @@ -79,7 +77,7 @@ public void startUpCluster() throws IOException { bpid = cluster.getNamesystem().getBlockPoolId(); } - @After + @AfterEach public void shutDownCluster() throws IOException { if (cluster != null) { fs.close(); @@ -142,8 +140,8 @@ public void testBlockHasMultipleReplicasOnSameDN() throws IOException { // Make sure that each block has two replicas, one on each DataNode. for (LocatedBlock locatedBlock : locatedBlocks.getLocatedBlocks()) { DatanodeInfo[] locations = locatedBlock.getLocations(); - assertThat(locations.length, is((int) NUM_DATANODES)); - assertThat(locations[0].getDatanodeUuid(), not(locations[1].getDatanodeUuid())); + assertThat(locations.length).isEqualTo((int) NUM_DATANODES); + assertThat(locations[0].getDatanodeUuid()).isNotEqualTo(locations[1].getDatanodeUuid()); } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockPoolManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockPoolManager.java index a4f5071aac359..a406adf36ad31 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockPoolManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockPoolManager.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hdfs.server.datanode; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import java.io.IOException; import java.net.InetSocketAddress; @@ -31,9 +31,9 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSUtil; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -47,7 +47,7 @@ public class TestBlockPoolManager { private final StringBuilder log = new StringBuilder(); private int mockIdx = 1; - @Before + @BeforeEach public void setupBPM() { bpm = new BlockPoolManager(mockDN){ @@ -159,9 +159,9 @@ public void testInternalNameService() throws Exception { bpm.refreshNamenodes(conf); assertEquals("create #1\n", log.toString()); Map map = bpm.getBpByNameserviceId(); - Assert.assertFalse(map.containsKey("ns2")); - Assert.assertFalse(map.containsKey("ns3")); - Assert.assertTrue(map.containsKey("ns1")); + Assertions.assertFalse(map.containsKey("ns2")); + Assertions.assertFalse(map.containsKey("ns3")); + Assertions.assertTrue(map.containsKey("ns1")); log.setLength(0); } @@ -179,18 +179,18 @@ public void testNameServiceNeedToBeResolved() throws Exception { "create #2\n" + "create #3\n", log.toString()); Map map = bpm.getBpByNameserviceId(); - Assert.assertTrue(map.containsKey("ns1")); - Assert.assertTrue(map.containsKey("ns2")); - Assert.assertTrue(map.containsKey("ns3")); - Assert.assertEquals(2, map.get("ns3").getBPServiceActors().size()); - Assert.assertEquals("ns3-" + MockDomainNameResolver.FQDN_1 + "-8020", + Assertions.assertTrue(map.containsKey("ns1")); + Assertions.assertTrue(map.containsKey("ns2")); + Assertions.assertTrue(map.containsKey("ns3")); + Assertions.assertEquals(2, map.get("ns3").getBPServiceActors().size()); + Assertions.assertEquals("ns3-" + MockDomainNameResolver.FQDN_1 + "-8020", map.get("ns3").getBPServiceActors().get(0).getNnId()); - Assert.assertEquals("ns3-" + MockDomainNameResolver.FQDN_2 + "-8020", + Assertions.assertEquals("ns3-" + MockDomainNameResolver.FQDN_2 + "-8020", map.get("ns3").getBPServiceActors().get(1).getNnId()); - Assert.assertEquals( + Assertions.assertEquals( new InetSocketAddress(MockDomainNameResolver.FQDN_1, 8020), map.get("ns3").getBPServiceActors().get(0).getNNSocketAddress()); - Assert.assertEquals( + Assertions.assertEquals( new InetSocketAddress(MockDomainNameResolver.FQDN_2, 8020), map.get("ns3").getBPServiceActors().get(1).getNNSocketAddress()); log.setLength(0); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockPoolSliceStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockPoolSliceStorage.java index 5e850b944e10c..937d1a194cd82 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockPoolSliceStorage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockPoolSliceStorage.java @@ -18,7 +18,8 @@ package org.apache.hadoop.hdfs.server.datanode; import org.apache.hadoop.hdfs.server.common.Storage; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.mockito.Mockito; import java.io.File; @@ -27,8 +28,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.hamcrest.core.Is.is; -import static org.junit.Assert.assertThat; +import static org.assertj.core.api.Assertions.assertThat; /** * Test that BlockPoolSliceStorage can correctly generate trash and @@ -52,7 +52,7 @@ private static class StubBlockPoolSliceStorage extends BlockPoolSliceStorage { String clusterId) { super(namespaceID, bpID, cTime, clusterId); addStorageDir(new StorageDirectory(new File("/tmp/dontcare/" + bpID))); - assertThat(getStorageDirs().size(), is(1)); + assertThat(getStorageDirs().size()).isEqualTo(1); } } @@ -111,7 +111,7 @@ public void getTrashDirectoryForBlockFile(String fileName, int nestingLevel) { ReplicaInfo info = Mockito.mock(ReplicaInfo.class); Mockito.when(info.getBlockURI()).thenReturn(new File(testFilePath).toURI()); - assertThat(storage.getTrashDirectory(info), is(expectedTrashPath)); + assertThat(storage.getTrashDirectory(info)).isEqualTo(expectedTrashPath); } /* @@ -134,12 +134,13 @@ public void getRestoreDirectoryForBlockFile(String fileName, int nestingLevel) { blockFileSubdir.substring(0, blockFileSubdir.length() - 1); LOG.info("Generated deleted file path {}", deletedFilePath); - assertThat(storage.getRestoreDirectory(new File(deletedFilePath)), - is(expectedRestorePath)); + assertThat(storage.getRestoreDirectory(new File(deletedFilePath))) + .isEqualTo(expectedRestorePath); } - @Test (timeout=300000) + @Test + @Timeout(value = 300) public void testGetTrashAndRestoreDirectories() { storage = makeBlockPoolStorage(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery2.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery2.java index 9d7c820249c65..5475c7fbfa7db 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery2.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery2.java @@ -46,12 +46,13 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols; import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo; 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.TestName; +import org.apache.hadoop.test.TestName; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.RegisterExtension; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -75,7 +76,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.anyString; @@ -104,8 +105,9 @@ public class TestBlockRecovery2 { private final static InetSocketAddress NN_ADDR = new InetSocketAddress( "localhost", 5020); - @Rule - public TestName currentTestName = new TestName(); + @SuppressWarnings("checkstyle:VisibilityModifier") + @RegisterExtension + public TestName methodName = new TestName(); static { GenericTestUtils.setLogLevel(FSNamesystem.LOG, Level.TRACE); @@ -116,7 +118,7 @@ public class TestBlockRecovery2 { * Starts an instance of DataNode. * @throws IOException */ - @Before + @BeforeEach public void startUp() throws IOException { tearDownDone = false; conf = new HdfsConfiguration(); @@ -166,7 +168,7 @@ public void startUp() throws IOException { @Override DatanodeProtocolClientSideTranslatorPB connectToNN( InetSocketAddress nnAddr) throws IOException { - Assert.assertEquals(NN_ADDR, nnAddr); + Assertions.assertEquals(NN_ADDR, nnAddr); return namenode; } }; @@ -190,15 +192,15 @@ private void waitForActiveNN() { } catch (InterruptedException e) { LOG.warn("InterruptedException while waiting to see active NN", e); } - Assert.assertNotNull("Failed to get ActiveNN", - dn.getAllBpOs().get(0).getActiveNN()); + Assertions.assertNotNull(dn.getAllBpOs().get(0).getActiveNN(), + "Failed to get ActiveNN"); } /** * Cleans the resources and closes the instance of datanode. * @throws IOException if an error occurred */ - @After + @AfterEach public void tearDown() throws IOException { if (!tearDownDone && dn != null) { try { @@ -208,8 +210,7 @@ public void tearDown() throws IOException { } finally { File dir = new File(DATA_DIR); if (dir.exists()) { - Assert.assertTrue( - "Cannot delete data-node dirs", FileUtil.fullyDelete(dir)); + Assertions.assertTrue(FileUtil.fullyDelete(dir), "Cannot delete data-node dirs"); } } tearDownDone = true; @@ -221,7 +222,8 @@ public void tearDown() throws IOException { * * @throws Exception */ - @Test(timeout = 20000) + @Test + @Timeout(value = 20) public void testRaceBetweenReplicaRecoveryAndFinalizeBlock() throws Exception { // Stop the Mocked DN started in startup() @@ -262,13 +264,13 @@ public void testRaceBetweenReplicaRecoveryAndFinalizeBlock() try { out.close(); } catch (IOException e) { - Assert.assertTrue("Writing should fail", - e.getMessage().contains("are bad. Aborting...")); + Assertions.assertTrue(e.getMessage().contains("are bad. Aborting..."), + "Writing should fail"); } finally { recoveryThread.join(); } - Assert.assertTrue("Recovery should be initiated successfully", - recoveryInitResult.get()); + Assertions.assertTrue(recoveryInitResult.get(), + "Recovery should be initiated successfully"); dataNode.updateReplicaUnderRecovery(block.getBlock(), block.getBlock() .getGenerationStamp() + 1, block.getBlock().getBlockId(), @@ -284,7 +286,8 @@ public void testRaceBetweenReplicaRecoveryAndFinalizeBlock() * Test for block recovery timeout. All recovery attempts will be delayed * and the first attempt will be lost to trigger recovery timeout and retry. */ - @Test(timeout = 300000L) + @Test + @Timeout(300) public void testRecoveryTimeout() throws Exception { tearDown(); // Stop the Mocked DN started in startup() final Random r = new Random(); @@ -323,7 +326,8 @@ public Object answer(InvocationOnMock invocation) throws Throwable { /** * Test for block recovery taking longer than the heartbeat interval. */ - @Test(timeout = 300000L) + @Test + @Timeout(value = 300) public void testRecoverySlowerThanHeartbeat() throws Exception { tearDown(); // Stop the Mocked DN started in startup() @@ -332,7 +336,8 @@ public void testRecoverySlowerThanHeartbeat() throws Exception { TestBlockRecovery.testRecoveryWithDatanodeDelayed(delayer); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testEcRecoverBlocks() throws Throwable { // Stop the Mocked DN started in startup() tearDown(); @@ -392,7 +397,8 @@ public void testEcRecoverBlocks() throws Throwable { * * Check that, after recovering, the block will be successfully replicated. */ - @Test(timeout = 300000L) + @Test + @Timeout(300) public void testRecoveryWillIgnoreMinReplication() throws Exception { tearDown(); // Stop the Mocked DN started in startup() diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java index 3f99f1bb12eb2..e626da455978f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java @@ -17,10 +17,10 @@ */ package org.apache.hadoop.hdfs.server.datanode; -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.IOException; import java.net.InetSocketAddress; @@ -57,7 +57,8 @@ import org.apache.hadoop.hdfs.util.DataTransferThrottler; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.util.Time; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; /** * This class tests if block replacement request to data nodes work correctly. @@ -208,7 +209,8 @@ public void testBlockReplacement() throws Exception { * datanode will throw IOException with error code Status.ERROR_BLOCK_PINNED. * */ - @Test(timeout = 90000) + @Test + @Timeout(value = 90) public void testBlockReplacementWithPinnedBlocks() throws Exception { final Configuration conf = new HdfsConfiguration(); @@ -229,7 +231,7 @@ public void testBlockReplacementWithPinnedBlocks() throws Exception { LocatedBlock lb = dfs.getClient().getLocatedBlocks(fileName, 0).get(0); DatanodeInfo[] oldNodes = lb.getLocations(); - assertEquals("Wrong block locations", oldNodes.length, 1); + assertEquals(oldNodes.length, 1, "Wrong block locations"); DatanodeInfo source = oldNodes[0]; ExtendedBlock b = lb.getBlock(); @@ -243,10 +245,10 @@ public void testBlockReplacementWithPinnedBlocks() throws Exception { } } - assertNotNull("Failed to choose destination datanode!", destin); + assertNotNull(destin, "Failed to choose destination datanode!"); - assertFalse("Source and destin datanode should be different", - source.equals(destin)); + assertFalse(source.equals(destin), + "Source and destin datanode should be different"); // Mock FsDatasetSpi#getPinning to show that the block is pinned. for (int i = 0; i < cluster.getDataNodes().size(); i++) { @@ -257,8 +259,8 @@ public void testBlockReplacementWithPinnedBlocks() throws Exception { // Block movement to a different datanode should fail as the block is // pinned. - assertTrue("Status code mismatches!", replaceBlock(b, source, source, - destin, StorageType.ARCHIVE, Status.ERROR_BLOCK_PINNED)); + assertTrue(replaceBlock(b, source, source, destin, + StorageType.ARCHIVE, Status.ERROR_BLOCK_PINNED), "Status code mismatches!"); } finally { cluster.shutdown(); } @@ -299,10 +301,10 @@ public void testBlockMoveAcrossStorageInSameNode() throws Exception { locatedBlocks = dfs.getClient().getLocatedBlocks(file.toString(), 0); // get the current locatedBlock = locatedBlocks.get(0); - assertEquals("Storage should be only one", 1, - locatedBlock.getLocations().length); - assertTrue("Block should be moved to ARCHIVE", locatedBlock - .getStorageTypes()[0] == StorageType.ARCHIVE); + assertEquals(1, locatedBlock.getLocations().length, + "Storage should be only one"); + assertTrue(locatedBlock.getStorageTypes()[0] == StorageType.ARCHIVE, + "Block should be moved to ARCHIVE"); } finally { cluster.shutdown(); } @@ -398,14 +400,14 @@ public void testDeletedBlockWhenAddBlockIsInEdit() throws Exception { DFSClient client = null; try { cluster.waitActive(); - assertEquals("Number of namenodes is not 2", 2, - cluster.getNumNameNodes()); + assertEquals(2, cluster.getNumNameNodes(), + "Number of namenodes is not 2"); // Transitioning the namenode 0 to active. cluster.transitionToActive(0); - assertTrue("Namenode 0 should be in active state", - cluster.getNameNode(0).isActiveState()); - assertTrue("Namenode 1 should be in standby state", - cluster.getNameNode(1).isStandbyState()); + assertTrue(cluster.getNameNode(0).isActiveState(), + "Namenode 0 should be in active state"); + assertTrue(cluster.getNameNode(1).isStandbyState(), + "Namenode 1 should be in standby state"); // Trigger heartbeat to mark DatanodeStorageInfo#heartbeatedSinceFailover // to true. @@ -430,8 +432,7 @@ public void testDeletedBlockWhenAddBlockIsInEdit() throws Exception { // add a second datanode to the cluster cluster.startDataNodes(conf, 1, true, null, null, null, null); - assertEquals("Number of datanodes should be 2", 2, - cluster.getDataNodes().size()); + assertEquals(2, cluster.getDataNodes().size(), "Number of datanodes should be 2"); DataNode dn0 = cluster.getDataNodes().get(0); DataNode dn1 = cluster.getDataNodes().get(1); @@ -464,10 +465,10 @@ public void testDeletedBlockWhenAddBlockIsInEdit() throws Exception { cluster.transitionToStandby(0); cluster.transitionToActive(1); - assertTrue("Namenode 1 should be in active state", - cluster.getNameNode(1).isActiveState()); - assertTrue("Namenode 0 should be in standby state", - cluster.getNameNode(0).isStandbyState()); + assertTrue(cluster.getNameNode(1).isActiveState(), + "Namenode 1 should be in active state"); + assertTrue(cluster.getNameNode(0).isStandbyState(), + "Namenode 0 should be in standby state"); client.close(); // Opening a new client for new active namenode @@ -476,8 +477,8 @@ public void testDeletedBlockWhenAddBlockIsInEdit() throws Exception { .getBlockLocations("/tmp.txt", 0, 10L).getLocatedBlocks(); assertEquals(1, locatedBlocks1.size()); - assertEquals("The block should be only on 1 datanode ", 1, - locatedBlocks1.get(0).getLocations().length); + assertEquals(1, locatedBlocks1.get(0).getLocations().length, + "The block should be only on 1 datanode "); } finally { IOUtils.cleanupWithLogger(null, client); cluster.shutdown(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockScanner.java index 4251c15718545..9e0743acb4fed 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockScanner.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockScanner.java @@ -24,10 +24,10 @@ import static org.apache.hadoop.hdfs.server.datanode.BlockScanner.Conf.INTERNAL_DFS_DATANODE_SCAN_PERIOD_MS; import static org.apache.hadoop.hdfs.server.datanode.BlockScanner.Conf.INTERNAL_VOLUME_SCANNER_SCAN_RESULT_HANDLER; import static org.apache.hadoop.hdfs.server.datanode.BlockScanner.Conf.INTERNAL_DFS_BLOCK_SCANNER_CURSOR_SAVE_INTERVAL_MS; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertFalse; -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.assertFalse; +import static org.junit.jupiter.api.Assertions.fail; import java.io.Closeable; import java.io.File; @@ -62,9 +62,10 @@ import org.apache.hadoop.hdfs.server.datanode.VolumeScanner.Statistics; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.Time; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +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; @@ -73,7 +74,7 @@ public class TestBlockScanner { public static final Logger LOG = LoggerFactory.getLogger(TestBlockScanner.class); - @Before + @BeforeEach public void before() { BlockScanner.Conf.allowUnitTestSettings = true; GenericTestUtils.setLogLevel(BlockScanner.LOG, Level.TRACE); @@ -220,7 +221,7 @@ private void testVolumeIteratorImpl(int numFiles, assertEquals(savedBlock, loadedBlock); } boolean blockRemoved = blocks.remove(block); - assertTrue("Found unknown block " + block, blockRemoved); + assertTrue(blockRemoved, "Found unknown block " + block); if (blocksProcessed > (numFiles / 3)) { if (!testedSave) { LOG.info("Processed {} blocks out of {}. Saving iterator.", @@ -268,17 +269,20 @@ private void testVolumeIteratorImpl(int numFiles, } } - @Test(timeout=60000) + @Test + @Timeout(value = 60) public void testVolumeIteratorWithoutCaching() throws Exception { testVolumeIteratorImpl(5, 0); } - @Test(timeout=300000) + @Test + @Timeout(value = 300) public void testVolumeIteratorWithCaching() throws Exception { testVolumeIteratorImpl(600, 100); } - @Test(timeout=60000) + @Test + @Timeout(value = 60) public void testDisableVolumeScanner() throws Exception { Configuration conf = new Configuration(); disableBlockScanner(conf); @@ -287,7 +291,8 @@ public void testDisableVolumeScanner() throws Exception { } } - @Test(timeout=60000) + @Test + @Timeout(value = 60) public void testDisableVolumeScanner2() throws Exception { Configuration conf = new Configuration(); conf.setLong(DFS_BLOCK_SCANNER_VOLUME_BYTES_PER_SECOND, -1L); @@ -444,7 +449,8 @@ public Boolean get() { * Test scanning all blocks. Set the scan period high enough that * we shouldn't rescan any block during this test. */ - @Test(timeout=60000) + @Test + @Timeout(value = 60) public void testScanAllBlocksNoRescan() throws Exception { testScanAllBlocksImpl(false); } @@ -453,7 +459,8 @@ public void testScanAllBlocksNoRescan() throws Exception { * Test scanning all blocks. Set the scan period high enough that * we should rescan all blocks at least twice during this test. */ - @Test(timeout=60000) + @Test + @Timeout(value = 60) public void testScanAllBlocksWithRescan() throws Exception { testScanAllBlocksImpl(true); } @@ -461,7 +468,8 @@ public void testScanAllBlocksWithRescan() throws Exception { /** * Test that we don't scan too many blocks per second. */ - @Test(timeout=120000) + @Test + @Timeout(value = 120) public void testScanRateLimit() throws Exception { Configuration conf = new Configuration(); // Limit scan bytes per second dramatically @@ -494,15 +502,16 @@ public Boolean get() { // Should scan no more than one block a second. long seconds = ((endMs + 999 - startMs) / 1000); long maxBlocksScanned = seconds * 1; - assertTrue("The number of blocks scanned is too large. Scanned " + - info.blocksScanned + " blocks; only expected to scan at most " + - maxBlocksScanned + " in " + seconds + " seconds.", - info.blocksScanned <= maxBlocksScanned); + assertTrue(info.blocksScanned <= maxBlocksScanned, + "The number of blocks scanned is too large. Scanned " + + info.blocksScanned + " blocks; only expected to scan at most " + + maxBlocksScanned + " in " + seconds + " seconds."); } ctx.close(); } - @Test(timeout=120000) + @Test + @Timeout(value = 120) public void testCorruptBlockHandling() throws Exception { Configuration conf = new Configuration(); conf.setLong(DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, 100L); @@ -544,7 +553,8 @@ public Boolean get() { * Test that we save the scan cursor when shutting down the datanode, and * restart scanning from there when the datanode is restarted. */ - @Test(timeout=120000) + @Test + @Timeout(value = 120) public void testDatanodeCursor() throws Exception { Configuration conf = new Configuration(); conf.setLong(DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, 100L); @@ -579,8 +589,8 @@ public Boolean get() { URI vURI = ctx.volumes.get(0).getStorageLocation().getUri(); File cursorPath = new File(new File(new File(new File(vURI), "current"), ctx.bpids[0]), "scanner.cursor"); - assertTrue("Failed to find cursor save file in " + - cursorPath.getAbsolutePath(), cursorPath.exists()); + assertTrue(cursorPath.exists(), + "Failed to find cursor save file in " + cursorPath.getAbsolutePath()); Set prevGoodBlocks = new HashSet(); synchronized (info) { info.sem = new Semaphore(4); @@ -636,7 +646,8 @@ public Boolean get() { ctx.close(); } - @Test(timeout=120000) + @Test + @Timeout(value = 120) public void testMultipleBlockPoolScanning() throws Exception { Configuration conf = new Configuration(); conf.setLong(DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, 100L); @@ -681,47 +692,48 @@ public Boolean get() { Statistics stats = ctx.blockScanner.getVolumeStats( ctx.volumes.get(0).getStorageID()); assertEquals(TOTAL_FILES, stats.blocksScannedSinceRestart); - assertEquals(BYTES_SCANNED_PER_FILE * TOTAL_FILES, - stats.bytesScannedInPastHour); + assertEquals(BYTES_SCANNED_PER_FILE * TOTAL_FILES, stats.bytesScannedInPastHour); ctx.close(); } - @Test(timeout=120000) + @Test + @Timeout(value = 120) public void testNextSorted() throws Exception { List arr = new LinkedList(); arr.add("1"); arr.add("3"); arr.add("5"); arr.add("7"); - Assert.assertEquals("3", FsVolumeImpl.nextSorted(arr, "2")); - Assert.assertEquals("3", FsVolumeImpl.nextSorted(arr, "1")); - Assert.assertEquals("1", FsVolumeImpl.nextSorted(arr, "")); - Assert.assertEquals("1", FsVolumeImpl.nextSorted(arr, null)); - Assert.assertEquals(null, FsVolumeImpl.nextSorted(arr, "9")); + Assertions.assertEquals("3", FsVolumeImpl.nextSorted(arr, "2")); + Assertions.assertEquals("3", FsVolumeImpl.nextSorted(arr, "1")); + Assertions.assertEquals("1", FsVolumeImpl.nextSorted(arr, "")); + Assertions.assertEquals("1", FsVolumeImpl.nextSorted(arr, null)); + Assertions.assertEquals(null, FsVolumeImpl.nextSorted(arr, "9")); } - @Test(timeout=120000) + @Test + @Timeout(value = 120) public void testCalculateNeededBytesPerSec() throws Exception { // If we didn't check anything the last hour, we should scan now. - Assert.assertTrue( + Assertions.assertTrue( VolumeScanner.calculateShouldScan("test", 100, 0, 0, 60)); // If, on average, we checked 101 bytes/s checked during the last hour, // stop checking now. - Assert.assertFalse(VolumeScanner. + Assertions.assertFalse(VolumeScanner. calculateShouldScan("test", 100, 101 * 3600, 1000, 5000)); // Target is 1 byte / s, but we didn't scan anything in the last minute. // Should scan now. - Assert.assertTrue(VolumeScanner. + Assertions.assertTrue(VolumeScanner. calculateShouldScan("test", 1, 3540, 0, 60)); // Target is 1000000 byte / s, but we didn't scan anything in the last // minute. Should scan now. - Assert.assertTrue(VolumeScanner. + Assertions.assertTrue(VolumeScanner. calculateShouldScan("test", 100000L, 354000000L, 0, 60)); - Assert.assertFalse(VolumeScanner. + Assertions.assertFalse(VolumeScanner. calculateShouldScan("test", 100000L, 365000000L, 0, 60)); } @@ -729,7 +741,8 @@ public void testCalculateNeededBytesPerSec() throws Exception { * Test that we can mark certain blocks as suspect, and get them quickly * rescanned that way. See HDFS-7686 and HDFS-7548. */ - @Test(timeout=120000) + @Test + @Timeout(value = 120) public void testMarkSuspectBlock() throws Exception { Configuration conf = new Configuration(); // Set a really long scan period. @@ -766,10 +779,10 @@ public Boolean get() { }, 50, 30000); // We should have scanned 4 blocks synchronized (info) { - assertEquals("Expected 4 good blocks.", 4, info.goodBlocks.size()); + assertEquals(4, info.goodBlocks.size(), "Expected 4 good blocks."); info.goodBlocks.clear(); - assertEquals("Expected 4 blocksScanned", 4, info.blocksScanned); - assertEquals("Did not expect bad blocks.", 0, info.badBlocks.size()); + assertEquals(4, info.blocksScanned, "Expected 4 blocksScanned"); + assertEquals(0, info.badBlocks.size(), "Did not expect bad blocks."); info.blocksScanned = 0; } ExtendedBlock first = ctx.getFileBlock(0, 0); @@ -799,11 +812,11 @@ public Boolean get() { }, 50, 30000); synchronized (info) { - assertTrue("Expected block " + first + " to have been scanned.", - info.goodBlocks.contains(first)); + assertTrue(info.goodBlocks.contains(first), + "Expected block " + first + " to have been scanned."); assertEquals(2, info.goodBlocks.size()); info.goodBlocks.clear(); - assertEquals("Did not expect bad blocks.", 0, info.badBlocks.size()); + assertEquals(0, info.badBlocks.size(), "Did not expect bad blocks."); assertEquals(2, info.blocksScanned); info.blocksScanned = 0; } @@ -834,9 +847,9 @@ public Boolean get() { // We should not have rescanned the "suspect block", // because it was recently rescanned by the suspect block system. // This is a test of the "suspect block" rate limiting. - Assert.assertFalse("We should not " + + Assertions.assertFalse(info.goodBlocks.contains(first), "We should not " + "have rescanned block " + first + ", because it should have been " + - "in recentSuspectBlocks.", info.goodBlocks.contains(first)); + "in recentSuspectBlocks."); info.blocksScanned = 0; } ctx.close(); @@ -845,7 +858,8 @@ public Boolean get() { /** * Test that blocks which are in the wrong location are ignored. */ - @Test(timeout=120000) + @Test + @Timeout(value = 120) public void testIgnoreMisplacedBlock() throws Exception { Configuration conf = new Configuration(); // Set a really long scan period. @@ -889,10 +903,10 @@ public Boolean get() { synchronized (info) { assertFalse(info.goodBlocks.contains(unreachableBlock)); assertFalse(info.badBlocks.contains(unreachableBlock)); - assertEquals("Expected 3 good blocks.", 3, info.goodBlocks.size()); + assertEquals(3, info.goodBlocks.size(), "Expected 3 good blocks."); info.goodBlocks.clear(); - assertEquals("Expected 3 blocksScanned", 3, info.blocksScanned); - assertEquals("Did not expect bad blocks.", 0, info.badBlocks.size()); + assertEquals(3, info.blocksScanned, "Expected 3 blocksScanned"); + assertEquals(0, info.badBlocks.size(), "Did not expect bad blocks."); info.blocksScanned = 0; } info.sem.release(1); @@ -903,7 +917,8 @@ public Boolean get() { * Test concurrent append and scan. * @throws Exception */ - @Test(timeout=120000) + @Test + @Timeout(value = 120) public void testAppendWhileScanning() throws Exception { GenericTestUtils.setLogLevel(DataNode.LOG, Level.TRACE); Configuration conf = new Configuration(); @@ -985,12 +1000,12 @@ public Boolean get() { }, 1000, 30000); synchronized (info) { - assertEquals("Expected 1 good block.", - numExpectedBlocks, info.goodBlocks.size()); + assertEquals(numExpectedBlocks, info.goodBlocks.size(), + "Expected 1 good block."); info.goodBlocks.clear(); - assertEquals("Expected 1 blocksScanned", - numExpectedBlocks, info.blocksScanned); - assertEquals("Did not expect bad blocks.", 0, info.badBlocks.size()); + assertEquals(numExpectedBlocks, info.blocksScanned, "Expected 1 blocksScanned"); + assertEquals(0, info.badBlocks.size(), + "Did not expect bad blocks."); info.blocksScanned = 0; } } @@ -1029,8 +1044,8 @@ public void testSkipRecentAccessFile() throws Exception { info.shouldRun = false; info.notify(); } - assertEquals("Should not scan block accessed in last period", - 0, info.blocksScanned); + assertEquals(0, info.blocksScanned, + "Should not scan block accessed in last period"); ctx.close(); } @@ -1040,7 +1055,8 @@ public void testSkipRecentAccessFile() throws Exception { * * @throws Exception */ - @Test(timeout=120000) + @Test + @Timeout(value = 120) public void testFastDatanodeShutdown() throws Exception { // set the joinTimeOut to a value smaller than the completion time of the // VolumeScanner. @@ -1052,7 +1068,8 @@ public void testFastDatanodeShutdown() throws Exception { * * @throws Exception */ - @Test(timeout=120000) + @Test + @Timeout(value = 120) public void testSlowDatanodeShutdown() throws Exception { // Set the joinTimeOut to a value larger than the completion time of the // volume scanner @@ -1101,18 +1118,19 @@ public Boolean get() { long totalTimeShutdown = endShutdownTime - startShutdownTime; if (isFastShutdown) { - assertTrue("total shutdown time of DN must be smaller than " - + "VolumeScanner Response time: " + totalTimeShutdown, - totalTimeShutdown < delayMS - && totalTimeShutdown >= joinTimeOutMS); + assertTrue(totalTimeShutdown < delayMS + && totalTimeShutdown >= joinTimeOutMS, + "total shutdown time of DN must be smaller than " + + "VolumeScanner Response time: " + totalTimeShutdown + ); // wait for scanners to terminate before we move to the next test. injectDelay.waitForScanners(); return; } - assertTrue("total shutdown time of DN must be larger than " + - "VolumeScanner Response time: " + totalTimeShutdown, - totalTimeShutdown >= delayMS - && totalTimeShutdown < joinTimeOutMS); + assertTrue(totalTimeShutdown >= delayMS + && totalTimeShutdown < joinTimeOutMS, + "total shutdown time of DN must be larger than " + + "VolumeScanner Response time: " + totalTimeShutdown); } finally { // restore the VolumeScanner callback injector. VolumeScannerCBInjector.set(prevVolumeScannerCBInject); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBpServiceActorScheduler.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBpServiceActorScheduler.java index 166ae118f784e..791fe05e0a15a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBpServiceActorScheduler.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBpServiceActorScheduler.java @@ -19,38 +19,33 @@ package org.apache.hadoop.hdfs.server.datanode; import org.apache.hadoop.util.Time; +import org.junit.jupiter.api.Timeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.hdfs.server.datanode.BPServiceActor.Scheduler; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.Timeout; +import org.junit.jupiter.api.Test; import java.util.Arrays; import java.util.List; import java.util.Random; import static java.lang.Math.abs; -import static org.hamcrest.core.Is.is; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertThat; -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 static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.spy; - +import static org.assertj.core.api.Assertions.assertThat; /** * Verify the block report and heartbeat scheduling logic of BPServiceActor * using a few different values . */ +@Timeout(300) public class TestBpServiceActorScheduler { protected static final Logger LOG = LoggerFactory.getLogger(TestBpServiceActorScheduler.class); - @Rule - public Timeout timeout = new Timeout(300000); - private static final long HEARTBEAT_INTERVAL_MS = 5000; // 5 seconds private static final long LIFELINE_INTERVAL_MS = 3 * HEARTBEAT_INTERVAL_MS; private static final long BLOCK_REPORT_INTERVAL_MS = 10000; // 10 seconds @@ -72,7 +67,7 @@ public void testScheduleBlockReportImmediate() { Scheduler scheduler = makeMockScheduler(now); scheduler.scheduleBlockReport(0, true); assertTrue(scheduler.resetBlockReportTime); - assertThat(scheduler.getNextBlockReportTime(), is(now)); + assertThat(scheduler.getNextBlockReportTime()).isEqualTo(now); } } @@ -113,8 +108,7 @@ public void testScheduleNextBlockReport2() { Scheduler scheduler = makeMockScheduler(now); scheduler.resetBlockReportTime = false; scheduler.scheduleNextBlockReport(); - assertThat(scheduler.getNextBlockReportTime(), - is(now + BLOCK_REPORT_INTERVAL_MS)); + assertThat(scheduler.getNextBlockReportTime()).isEqualTo(now + BLOCK_REPORT_INTERVAL_MS); } } @@ -137,8 +131,8 @@ public void testScheduleNextBlockReport3() { scheduler.scheduleNextBlockReport(); assertTrue((scheduler.getNextBlockReportTime() - now) < BLOCK_REPORT_INTERVAL_MS); - assertEquals(0, ((scheduler.getNextBlockReportTime() - origBlockReportTime) - % BLOCK_REPORT_INTERVAL_MS)); + assertEquals(0, + ((scheduler.getNextBlockReportTime() - origBlockReportTime) % BLOCK_REPORT_INTERVAL_MS)); } } @@ -198,10 +192,10 @@ public void testScheduleLifeline() { Scheduler scheduler = makeMockScheduler(now); scheduler.scheduleNextLifeline(now); assertFalse(scheduler.isLifelineDue(now)); - assertThat(scheduler.getLifelineWaitTime(), is(LIFELINE_INTERVAL_MS)); + assertThat(scheduler.getLifelineWaitTime()).isEqualTo(LIFELINE_INTERVAL_MS); scheduler.scheduleNextLifeline(now - LIFELINE_INTERVAL_MS); assertTrue(scheduler.isLifelineDue(now)); - assertThat(scheduler.getLifelineWaitTime(), is(0L)); + assertThat(scheduler.getLifelineWaitTime()).isEqualTo(0L); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestCachingStrategy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestCachingStrategy.java index 011df46900c22..cfaf9d9a9177a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestCachingStrategy.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestCachingStrategy.java @@ -43,9 +43,10 @@ import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.POSIX_FADV_DONTNEED; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; public class TestCachingStrategy { private static final Logger LOG = @@ -56,7 +57,7 @@ public class TestCachingStrategy { private final static TestRecordingCacheTracker tracker = new TestRecordingCacheTracker(); - @BeforeClass + @BeforeAll public static void setupTest() { EditLogFileOutputStream.setShouldSkipFsyncForTesting(true); @@ -211,7 +212,8 @@ static long readHdfsFile(FileSystem fs, Path p, long length, throw new RuntimeException("unreachable"); } - @Test(timeout=120000) + @Test + @Timeout(value = 120) public void testFadviseAfterWriteThenRead() throws Exception { // start a cluster LOG.info("testFadviseAfterWriteThenRead"); @@ -239,7 +241,7 @@ public void testFadviseAfterWriteThenRead() throws Exception { // read file readHdfsFile(fs, new Path(TEST_PATH), Long.MAX_VALUE, true); // verify that we dropped everything from the cache. - Assert.assertNotNull(stats); + Assertions.assertNotNull(stats); stats.assertDroppedInRange(0, TEST_PATH_LEN - WRITE_PACKET_SIZE); } finally { if (cluster != null) { @@ -252,7 +254,8 @@ public void testFadviseAfterWriteThenRead() throws Exception { * Test the scenario where the DataNode defaults to not dropping the cache, * but our client defaults are set. */ - @Test(timeout=120000) + @Test + @Timeout(value = 120) public void testClientDefaults() throws Exception { // start a cluster LOG.info("testClientDefaults"); @@ -284,7 +287,7 @@ public void testClientDefaults() throws Exception { // read file readHdfsFile(fs, new Path(TEST_PATH), Long.MAX_VALUE, null); // verify that we dropped everything from the cache. - Assert.assertNotNull(stats); + Assertions.assertNotNull(stats); stats.assertDroppedInRange(0, TEST_PATH_LEN - WRITE_PACKET_SIZE); } finally { if (cluster != null) { @@ -293,7 +296,8 @@ public void testClientDefaults() throws Exception { } } - @Test(timeout=120000) + @Test + @Timeout(value = 120) public void testFadviseSkippedForSmallReads() throws Exception { // start a cluster LOG.info("testFadviseSkippedForSmallReads"); @@ -339,7 +343,8 @@ public void testFadviseSkippedForSmallReads() throws Exception { } } - @Test(timeout=120000) + @Test + @Timeout(value = 120) public void testNoFadviseAfterWriteThenRead() throws Exception { // start a cluster LOG.info("testNoFadviseAfterWriteThenRead"); @@ -361,7 +366,7 @@ public void testNoFadviseAfterWriteThenRead() throws Exception { TEST_PATH, 0, Long.MAX_VALUE).get(0).getBlock(); String fadvisedFileName = cluster.getBlockFile(0, block).getName(); Stats stats = tracker.getStats(fadvisedFileName); - Assert.assertNull(stats); + Assertions.assertNull(stats); // read file readHdfsFile(fs, new Path(TEST_PATH), Long.MAX_VALUE, false); @@ -372,7 +377,8 @@ public void testNoFadviseAfterWriteThenRead() throws Exception { } } - @Test(timeout=120000) + @Test + @Timeout(value = 120) public void testSeekAfterSetDropBehind() throws Exception { // start a cluster LOG.info("testSeekAfterSetDropBehind"); @@ -388,7 +394,7 @@ public void testSeekAfterSetDropBehind() throws Exception { createHdfsFile(fs, new Path(TEST_PATH), TEST_PATH_LEN, false); // verify that we can seek after setDropBehind try (FSDataInputStream fis = fs.open(new Path(TEST_PATH))) { - Assert.assertTrue(fis.read() != -1); // create BlockReader + Assertions.assertTrue(fis.read() != -1); // create BlockReader fis.setDropBehind(false); // clear BlockReader fis.seek(2); // seek } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestCorruptMetadataFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestCorruptMetadataFile.java index a71dbdbd30c5d..3190a277d0341 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestCorruptMetadataFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestCorruptMetadataFile.java @@ -28,14 +28,15 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.LambdaTestUtils; -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.junit.jupiter.api.Timeout; import java.io.File; import java.io.RandomAccessFile; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; /** * Tests to ensure that a block is not read successfully from a datanode @@ -47,7 +48,7 @@ public class TestCorruptMetadataFile { private MiniDFSCluster.Builder clusterBuilder; private Configuration conf; - @Before + @BeforeEach public void setUp() throws Exception { conf = new HdfsConfiguration(); // Reduce block acquire retries as we only have 1 DN and it allows the @@ -57,7 +58,7 @@ public void setUp() throws Exception { clusterBuilder = new MiniDFSCluster.Builder(conf).numDataNodes(1); } - @After + @AfterEach public void tearDown() throws Exception { if (cluster != null) { cluster.shutdown(); @@ -65,7 +66,8 @@ public void tearDown() throws Exception { } } - @Test(timeout=60000) + @Test + @Timeout(value = 60) public void testReadBlockFailsWhenMetaIsCorrupt() throws Exception { cluster = clusterBuilder.build(); cluster.waitActive(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataDirs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataDirs.java index 550a83e5ebda7..aba70620e4a60 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataDirs.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataDirs.java @@ -28,15 +28,17 @@ import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.util.Shell; import org.opentest4j.TestAbortedException; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY; -import static org.hamcrest.CoreMatchers.is; -import static org.junit.Assert.*; +import static org.junit.jupiter.api.Assertions.*; +import static org.assertj.core.api.Assertions.assertThat; public class TestDataDirs { - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testDataDirParsing() throws Throwable { Configuration conf = new Configuration(); List locations; @@ -56,28 +58,28 @@ public void testDataDirParsing() throws Throwable { "[ram_disk]/dir4,[disk]/dir5, [disk] /dir6, [disk] , [nvdimm]/dir7"; conf.set(DFS_DATANODE_DATA_DIR_KEY, locations1); locations = DataNode.getStorageLocations(conf); - assertThat(locations.size(), is(9)); - assertThat(locations.get(0).getStorageType(), is(StorageType.DISK)); - assertThat(locations.get(0).getUri(), is(dir0.toURI())); - assertThat(locations.get(1).getStorageType(), is(StorageType.DISK)); - assertThat(locations.get(1).getUri(), is(dir1.toURI())); - assertThat(locations.get(2).getStorageType(), is(StorageType.SSD)); - assertThat(locations.get(2).getUri(), is(dir2.toURI())); - assertThat(locations.get(3).getStorageType(), is(StorageType.DISK)); - assertThat(locations.get(3).getUri(), is(dir3.toURI())); - assertThat(locations.get(4).getStorageType(), is(StorageType.RAM_DISK)); - assertThat(locations.get(4).getUri(), is(dir4.toURI())); - assertThat(locations.get(5).getStorageType(), is(StorageType.DISK)); - assertThat(locations.get(5).getUri(), is(dir5.toURI())); - assertThat(locations.get(6).getStorageType(), is(StorageType.DISK)); - assertThat(locations.get(6).getUri(), is(dir6.toURI())); + assertThat(locations.size()).isEqualTo(9); + assertThat(locations.get(0).getStorageType()).isEqualTo(StorageType.DISK); + assertThat(locations.get(0).getUri()).isEqualTo(dir0.toURI()); + assertThat(locations.get(1).getStorageType()).isEqualTo(StorageType.DISK); + assertThat(locations.get(1).getUri()).isEqualTo(dir1.toURI()); + assertThat(locations.get(2).getStorageType()).isEqualTo(StorageType.SSD); + assertThat(locations.get(2).getUri()).isEqualTo(dir2.toURI()); + assertThat(locations.get(3).getStorageType()).isEqualTo(StorageType.DISK); + assertThat(locations.get(3).getUri()).isEqualTo(dir3.toURI()); + assertThat(locations.get(4).getStorageType()).isEqualTo(StorageType.RAM_DISK); + assertThat(locations.get(4).getUri()).isEqualTo(dir4.toURI()); + assertThat(locations.get(5).getStorageType()).isEqualTo(StorageType.DISK); + assertThat(locations.get(5).getUri()).isEqualTo(dir5.toURI()); + assertThat(locations.get(6).getStorageType()).isEqualTo(StorageType.DISK); + assertThat(locations.get(6).getUri()).isEqualTo(dir6.toURI()); // not asserting the 8th URI since it is incomplete and it in the // test set to make sure that we don't fail if we get URIs like that. - assertThat(locations.get(7).getStorageType(), is(StorageType.DISK)); + assertThat(locations.get(7).getStorageType()).isEqualTo(StorageType.DISK); - assertThat(locations.get(8).getStorageType(), is(StorageType.NVDIMM)); - assertThat(locations.get(8).getUri(), is(dir7.toURI())); + assertThat(locations.get(8).getStorageType()).isEqualTo(StorageType.NVDIMM); + assertThat(locations.get(8).getUri()).isEqualTo(dir7.toURI()); // Verify that an unrecognized storage type result in an exception. String locations2 = "[BadMediaType]/dir0,[ssd]/dir1,[disk]/dir2"; @@ -94,11 +96,11 @@ public void testDataDirParsing() throws Throwable { String locations3 = "/dir0,/dir1"; conf.set(DFS_DATANODE_DATA_DIR_KEY, locations3); locations = DataNode.getStorageLocations(conf); - assertThat(locations.size(), is(2)); - assertThat(locations.get(0).getStorageType(), is(StorageType.DISK)); - assertThat(locations.get(0).getUri(), is(dir0.toURI())); - assertThat(locations.get(1).getStorageType(), is(StorageType.DISK)); - assertThat(locations.get(1).getUri(), is(dir1.toURI())); + assertThat(locations.size()).isEqualTo(2); + assertThat(locations.get(0).getStorageType()).isEqualTo(StorageType.DISK); + assertThat(locations.get(0).getUri()).isEqualTo(dir0.toURI()); + assertThat(locations.get(1).getStorageType()).isEqualTo(StorageType.DISK); + assertThat(locations.get(1).getUri()).isEqualTo(dir1.toURI()); } @Test @@ -136,10 +138,8 @@ public void testCapacityRatioForDataDir() { // Good case String config = "[0.9 ]/disk /2, [0.1]/disk2/1"; Map map = StorageLocation.parseCapacityRatio(config); - assertEquals(0.9, - map.get(new Path("/disk/2").toUri()), 0); - assertEquals(0.1, - map.get(new Path("/disk2/1").toUri()), 0); + assertEquals(0.9, map.get(new Path("/disk/2").toUri()), 0); + assertEquals(0.1, map.get(new Path("/disk2/1").toUri()), 0); // config without capacity ratio config = "[0.9 ]/disk /2, /disk2/1"; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeECN.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeECN.java index cb727e2e14c53..ce81872c22f99 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeECN.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeECN.java @@ -21,18 +21,15 @@ import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.Timeout; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.io.IOException; +@Timeout(300) public class TestDataNodeECN { - @Rule - public Timeout globalTimeout = new Timeout(300000); - @Test public void testECNFlag() throws IOException { Configuration conf = new Configuration(); @@ -41,7 +38,7 @@ public void testECNFlag() throws IOException { try { cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build(); PipelineAck.ECN ecn = cluster.getDataNodes().get(0).getECN(); - Assert.assertNotEquals(PipelineAck.ECN.DISABLED, ecn); + Assertions.assertNotEquals(PipelineAck.ECN.DISABLED, ecn); } finally { if (cluster != null) { cluster.shutdown(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeErasureCodingMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeErasureCodingMetrics.java index ce65b6b674906..de6c56e6d432c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeErasureCodingMetrics.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeErasureCodingMetrics.java @@ -38,13 +38,14 @@ import static org.apache.hadoop.test.MetricsAsserts.getLongCounter; import static org.apache.hadoop.test.MetricsAsserts.getLongCounterWithoutCheck; import static org.apache.hadoop.test.MetricsAsserts.getMetrics; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; - -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.io.IOException; /** @@ -67,7 +68,7 @@ public class TestDataNodeErasureCodingMetrics { private Configuration conf; private DistributedFileSystem fs; - @Before + @BeforeEach public void setup() throws IOException { conf = new Configuration(); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); @@ -81,80 +82,84 @@ public void setup() throws IOException { StripedFileTestUtil.getDefaultECPolicy().getName()); } - @After + @AfterEach public void tearDown() { if (cluster != null) { cluster.shutdown(); } } - @Test(timeout = 120000) + @Test + @Timeout(value = 120) public void testFullBlock() throws Exception { - Assert.assertEquals(0, getLongMetric("EcReconstructionReadTimeMillis")); - Assert.assertEquals(0, getLongMetric("EcReconstructionDecodingTimeMillis")); - Assert.assertEquals(0, getLongMetric("EcReconstructionWriteTimeMillis")); + Assertions.assertEquals(0, getLongMetric("EcReconstructionReadTimeMillis")); + Assertions.assertEquals(0, getLongMetric("EcReconstructionDecodingTimeMillis")); + Assertions.assertEquals(0, getLongMetric("EcReconstructionWriteTimeMillis")); doTest("/testEcMetrics", blockGroupSize, 0); - Assert.assertEquals("EcReconstructionTasks should be ", - 1, getLongMetric("EcReconstructionTasks")); - Assert.assertEquals("EcFailedReconstructionTasks should be ", - 0, getLongMetric("EcFailedReconstructionTasks")); - Assert.assertTrue(getLongMetric("EcDecodingTimeNanos") > 0); - Assert.assertEquals("EcReconstructionBytesRead should be ", - blockGroupSize, getLongMetric("EcReconstructionBytesRead")); - Assert.assertEquals("EcReconstructionBytesWritten should be ", - blockSize, getLongMetric("EcReconstructionBytesWritten")); - Assert.assertEquals("EcReconstructionRemoteBytesRead should be ", - 0, getLongMetricWithoutCheck("EcReconstructionRemoteBytesRead")); - Assert.assertTrue(getLongMetric("EcReconstructionReadTimeMillis") > 0); - Assert.assertTrue(getLongMetric("EcReconstructionDecodingTimeMillis") > 0); - Assert.assertTrue(getLongMetric("EcReconstructionWriteTimeMillis") > 0); + Assertions.assertEquals(1, getLongMetric("EcReconstructionTasks"), + "EcReconstructionTasks should be "); + Assertions.assertEquals(0, getLongMetric("EcFailedReconstructionTasks"), + "EcFailedReconstructionTasks should be "); + Assertions.assertTrue(getLongMetric("EcDecodingTimeNanos") > 0); + Assertions.assertEquals(blockGroupSize, getLongMetric("EcReconstructionBytesRead"), + "EcReconstructionBytesRead should be "); + Assertions.assertEquals(blockSize, getLongMetric("EcReconstructionBytesWritten"), + "EcReconstructionBytesWritten should be "); + Assertions.assertEquals(0, getLongMetricWithoutCheck("EcReconstructionRemoteBytesRead"), + "EcReconstructionRemoteBytesRead should be "); + Assertions.assertTrue(getLongMetric("EcReconstructionReadTimeMillis") > 0); + Assertions.assertTrue(getLongMetric("EcReconstructionDecodingTimeMillis") > 0); + Assertions.assertTrue(getLongMetric("EcReconstructionWriteTimeMillis") > 0); } // A partial block, reconstruct the partial block - @Test(timeout = 120000) + @Test + @Timeout(value = 120) public void testReconstructionBytesPartialGroup1() throws Exception { final int fileLen = blockSize / 10; doTest("/testEcBytes", fileLen, 0); - Assert.assertEquals("EcReconstructionBytesRead should be ", - fileLen, getLongMetric("EcReconstructionBytesRead")); - Assert.assertEquals("EcReconstructionBytesWritten should be ", - fileLen, getLongMetric("EcReconstructionBytesWritten")); - Assert.assertEquals("EcReconstructionRemoteBytesRead should be ", - 0, getLongMetricWithoutCheck("EcReconstructionRemoteBytesRead")); + Assertions.assertEquals(fileLen, getLongMetric("EcReconstructionBytesRead"), + "EcReconstructionBytesRead should be "); + Assertions.assertEquals(fileLen, getLongMetric("EcReconstructionBytesWritten"), + "EcReconstructionBytesWritten should be "); + Assertions.assertEquals(0, getLongMetricWithoutCheck("EcReconstructionRemoteBytesRead"), + "EcReconstructionRemoteBytesRead should be "); } // 1 full block + 5 partial block, reconstruct the full block - @Test(timeout = 120000) + @Test + @Timeout(value = 120) public void testReconstructionBytesPartialGroup2() throws Exception { final int fileLen = cellSize * dataBlocks + cellSize + cellSize / 10; doTest("/testEcBytes", fileLen, 0); - Assert.assertEquals("ecReconstructionBytesRead should be ", - cellSize * dataBlocks + cellSize + cellSize / 10, - getLongMetric("EcReconstructionBytesRead")); - Assert.assertEquals("EcReconstructionBytesWritten should be ", - blockSize, getLongMetric("EcReconstructionBytesWritten")); - Assert.assertEquals("EcReconstructionRemoteBytesRead should be ", - 0, getLongMetricWithoutCheck("EcReconstructionRemoteBytesRead")); + Assertions.assertEquals(cellSize * dataBlocks + + cellSize + cellSize / 10, + getLongMetric("EcReconstructionBytesRead"), "ecReconstructionBytesRead should be "); + Assertions.assertEquals(blockSize, getLongMetric("EcReconstructionBytesWritten"), + "EcReconstructionBytesWritten should be "); + Assertions.assertEquals(0, getLongMetricWithoutCheck("EcReconstructionRemoteBytesRead"), + "EcReconstructionRemoteBytesRead should be "); } // 1 full block + 5 partial block, reconstruct the partial block - @Test(timeout = 120000) + @Test + @Timeout(value = 120) public void testReconstructionBytesPartialGroup3() throws Exception { final int fileLen = cellSize * dataBlocks + cellSize + cellSize / 10; doTest("/testEcBytes", fileLen, 1); - Assert.assertEquals("ecReconstructionBytesRead should be ", - cellSize * dataBlocks + (cellSize / 10) * 2 , - getLongMetric("EcReconstructionBytesRead")); - Assert.assertEquals("ecReconstructionBytesWritten should be ", - cellSize + cellSize / 10, - getLongMetric("EcReconstructionBytesWritten")); - Assert.assertEquals("EcReconstructionRemoteBytesRead should be ", - 0, getLongMetricWithoutCheck("EcReconstructionRemoteBytesRead")); + Assertions.assertEquals(cellSize * dataBlocks + (cellSize / 10) * 2, + getLongMetric("EcReconstructionBytesRead"), + "ecReconstructionBytesRead should be "); + Assertions.assertEquals(cellSize + cellSize / 10, + getLongMetric("EcReconstructionBytesWritten"), + "ecReconstructionBytesWritten should be "); + Assertions.assertEquals(0, getLongMetricWithoutCheck("EcReconstructionRemoteBytesRead"), + "EcReconstructionRemoteBytesRead should be "); } private long getLongMetric(String metricName) { @@ -195,14 +200,14 @@ private void doTest(String fileName, int fileLen, final DataNode toCorruptDn = cluster.getDataNode( lastBlock.getLocations()[deadNodeIndex].getIpcPort()); LOG.info("Datanode to be corrupted: " + toCorruptDn); - assertNotNull("Failed to find a datanode to be corrupted", toCorruptDn); + assertNotNull(toCorruptDn, "Failed to find a datanode to be corrupted"); toCorruptDn.shutdown(); setDataNodeDead(toCorruptDn.getDatanodeId()); DFSTestUtil.waitForDatanodeState(cluster, toCorruptDn.getDatanodeUuid(), false, 10000); final int workCount = getComputedDatanodeWork(); - assertTrue("Wrongly computed block reconstruction work", workCount > 0); + assertTrue(workCount > 0, "Wrongly computed block reconstruction work"); cluster.triggerHeartbeats(); int totalBlocks = (fileLen / blockGroupSize) * groupSize; final int remainder = fileLen % blockGroupSize; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeExit.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeExit.java index 8c2fe37a58737..644fae7589a06 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeExit.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeExit.java @@ -18,10 +18,10 @@ package org.apache.hadoop.hdfs.server.datanode; -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.List; @@ -31,9 +31,9 @@ import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSNNTopology; -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.mockito.Mockito; /** @@ -44,7 +44,7 @@ public class TestDataNodeExit { Configuration conf; MiniDFSCluster cluster = null; - @Before + @BeforeEach public void setUp() throws IOException { conf = new HdfsConfiguration(); conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 100); @@ -57,7 +57,7 @@ public void setUp() throws IOException { } } - @After + @AfterEach public void tearDown() throws Exception { if (cluster != null) { cluster.shutdown(); @@ -78,8 +78,7 @@ private void stopBPServiceThreads(int numStopThreads, DataNode dn) Thread.sleep(WAIT_TIME_IN_MILLIS); iterations--; } - assertEquals("Mismatch in number of BPServices running", expected, - dn.getBpOsCount()); + assertEquals(expected, dn.getBpOsCount(), "Mismatch in number of BPServices running"); } @Test @@ -100,9 +99,9 @@ public void testBPServiceState() { public void testBPServiceExit() throws Exception { DataNode dn = cluster.getDataNodes().get(0); stopBPServiceThreads(1, dn); - assertTrue("DataNode should not exit", dn.isDatanodeUp()); + assertTrue(dn.isDatanodeUp(), "DataNode should not exit"); stopBPServiceThreads(2, dn); - assertFalse("DataNode should exit", dn.isDatanodeUp()); + assertFalse(dn.isDatanodeUp(), "DataNode should exit"); } @Test diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeFSDataSetSink.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeFSDataSetSink.java index 27f5f02c92f8f..b4e6ef5dd8ea5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeFSDataSetSink.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeFSDataSetSink.java @@ -25,12 +25,12 @@ import org.apache.hadoop.metrics2.MetricsSink; import org.apache.hadoop.metrics2.MetricsTag; import org.apache.hadoop.metrics2.impl.MetricsSystemImpl; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.Set; import java.util.TreeSet; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; public class TestDataNodeFSDataSetSink { private static final MetricsSystemImpl ms = new diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeFaultInjector.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeFaultInjector.java index 6879dc0eab5a1..4ba39affdde80 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeFaultInjector.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeFaultInjector.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hdfs.server.datanode; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.IOException; @@ -33,7 +33,8 @@ import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.PathUtils; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; /** * This class tests various cases where faults are injected to DataNode. @@ -76,7 +77,8 @@ void logDelay(final long duration) { } } - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testDelaySendingAckToUpstream() throws Exception { final MetricsDataNodeFaultInjector mdnFaultInjector = new MetricsDataNodeFaultInjector() { @@ -95,7 +97,8 @@ public void logDelaySendingAckToUpstream(final String upstreamAddr, verifyFaultInjectionDelayPipeline(mdnFaultInjector); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testDelaySendingPacketDownstream() throws Exception { final MetricsDataNodeFaultInjector mdnFaultInjector = new MetricsDataNodeFaultInjector() { @@ -161,8 +164,8 @@ private void verifyFaultInjectionDelayPipeline( } LOG.info("delay info: " + mdnFaultInjector.getDelayMs() + ":" + datanodeSlowLogThresholdMs); - assertTrue("Injected delay should be longer than the configured one", - mdnFaultInjector.getDelayMs() > datanodeSlowLogThresholdMs); + assertTrue(mdnFaultInjector.getDelayMs() > datanodeSlowLogThresholdMs, + "Injected delay should be longer than the configured one"); } finally { if (cluster != null) { cluster.shutdown(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeInitStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeInitStorage.java index 9d6958ea2ab94..ff403c6531054 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeInitStorage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeInitStorage.java @@ -27,7 +27,8 @@ import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; /** * Test to verify that the DataNode Uuid is correctly initialized before @@ -68,7 +69,8 @@ public SimulatedFsDatasetVerifier(DataStorage storage, Configuration conf) { } - @Test (timeout = 60000) + @Test + @Timeout(value = 60) public void testDataNodeInitStorage() throws Throwable { // Create configuration to use SimulatedFsDatasetVerifier#Factory. Configuration conf = new HdfsConfiguration(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeLifeline.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeLifeline.java index ef6cf05dcdbda..73d0d05c4d23d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeLifeline.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeLifeline.java @@ -31,9 +31,9 @@ import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports; import static org.apache.hadoop.test.MetricsAsserts.getLongCounter; import static org.apache.hadoop.test.MetricsAsserts.getMetrics; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyBoolean; import static org.mockito.Mockito.anyInt; @@ -60,12 +60,11 @@ import org.apache.hadoop.hdfs.server.protocol.StorageReport; 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.junit.jupiter.api.Timeout; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -78,6 +77,7 @@ /** * Test suite covering lifeline protocol handling in the DataNode. */ +@Timeout(60) public class TestDataNodeLifeline { private static final Logger LOG = LoggerFactory.getLogger( @@ -87,9 +87,6 @@ public class TestDataNodeLifeline { GenericTestUtils.setLogLevel(DataNode.LOG, Level.TRACE); } - @Rule - public Timeout timeout = new Timeout(60000); - private MiniDFSCluster cluster; private HdfsConfiguration conf; private DatanodeLifelineProtocolClientSideTranslatorPB lifelineNamenode; @@ -105,7 +102,7 @@ public void mockAnException() { } }; - @Before + @BeforeEach public void setup() throws Exception { // Configure cluster with lifeline RPC server enabled, and down-tune // heartbeat timings to try to force quick dead/stale DataNodes. @@ -152,7 +149,7 @@ public Boolean get() { bpsa.setNameNode(namenode); } - @After + @AfterEach public void shutdown() { if (cluster != null) { cluster.shutdown(); @@ -198,12 +195,12 @@ public void testSendLifelineIfHeartbeatBlocked() throws Exception { // poll DataNode tracking information. Thanks to the lifeline, we expect // that the DataNode always stays alive, and never goes stale or dead. while (!lifelinesSent.await(1, SECONDS)) { - assertEquals("Expect DataNode to be kept alive by lifeline.", 1, - namesystem.getNumLiveDataNodes()); - assertEquals("Expect DataNode not marked dead due to lifeline.", 0, - namesystem.getNumDeadDataNodes()); - assertEquals("Expect DataNode not marked stale due to lifeline.", 0, - namesystem.getNumStaleDataNodes()); + assertEquals(1, namesystem.getNumLiveDataNodes(), + "Expect DataNode to be kept alive by lifeline."); + assertEquals(0, namesystem.getNumDeadDataNodes(), + "Expect DataNode not marked dead due to lifeline."); + assertEquals(0, namesystem.getNumStaleDataNodes(), + "Expect DataNode not marked stale due to lifeline."); // add a new volume on the next heartbeat cluster.getDataNodes().get(0).reconfigurePropertyImpl( DFS_DATANODE_DATA_DIR_KEY, @@ -225,9 +222,9 @@ public void testSendLifelineIfHeartbeatBlocked() throws Exception { // numLifelines, guaranteed by waiting on the latch. There is a small // possibility of extra lifeline calls depending on timing, so we allow // slack in the assertion. - assertTrue("Expect metrics to count at least " + numLifelines + " calls.", - getLongCounter("LifelinesNumOps", getMetrics(metrics.name())) >= - numLifelines); + assertTrue(getLongCounter("LifelinesNumOps", + getMetrics(metrics.name())) >= numLifelines, + "Expect metrics to count at least " + numLifelines + " calls."); } @Test @@ -255,12 +252,12 @@ public void testNoLifelineSentIfHeartbeatsOnTime() throws Exception { // poll DataNode tracking information. We expect that the DataNode always // stays alive, and never goes stale or dead. while (!heartbeatsSent.await(1, SECONDS)) { - assertEquals("Expect DataNode to be kept alive by lifeline.", 1, - namesystem.getNumLiveDataNodes()); - assertEquals("Expect DataNode not marked dead due to lifeline.", 0, - namesystem.getNumDeadDataNodes()); - assertEquals("Expect DataNode not marked stale due to lifeline.", 0, - namesystem.getNumStaleDataNodes()); + assertEquals(1, namesystem.getNumLiveDataNodes(), + "Expect DataNode to be kept alive by lifeline."); + assertEquals(0, namesystem.getNumDeadDataNodes(), + "Expect DataNode not marked dead due to lifeline."); + assertEquals(0, namesystem.getNumStaleDataNodes(), + "Expect DataNode not marked stale due to lifeline."); } // Verify that we did not call the lifeline RPC. @@ -275,8 +272,8 @@ public void testNoLifelineSentIfHeartbeatsOnTime() throws Exception { any()); // Also verify no lifeline calls through metrics. - assertEquals("Expect metrics to count no lifeline calls.", 0, - getLongCounter("LifelinesNumOps", getMetrics(metrics.name()))); + assertEquals(0, getLongCounter("LifelinesNumOps", getMetrics(metrics.name())), + "Expect metrics to count no lifeline calls."); } @Test @@ -285,11 +282,11 @@ public void testLifelineForDeadNode() throws Exception { assertTrue(initialCapacity > 0); dn.setHeartbeatsDisabledForTests(true); cluster.setDataNodesDead(); - assertEquals("Capacity should be 0 after all DNs dead", 0, cluster - .getNamesystem(0).getCapacityTotal()); + assertEquals(0, cluster.getNamesystem(0).getCapacityTotal(), + "Capacity should be 0 after all DNs dead"); bpsa.sendLifelineForTests(); - assertEquals("Lifeline should be ignored for dead node", 0, cluster - .getNamesystem(0).getCapacityTotal()); + assertEquals(0, cluster.getNamesystem(0).getCapacityTotal(), + "Lifeline should be ignored for dead node"); // Wait for re-registration and heartbeat dn.setHeartbeatsDisabledForTests(false); final DatanodeDescriptor dnDesc = cluster.getNamesystem(0).getBlockManager() @@ -301,8 +298,8 @@ public Boolean get() { return dnDesc.isAlive() && dnDesc.isHeartbeatedSinceRegistration(); } }, 100, 5000); - assertEquals("Capacity should include only live capacity", initialCapacity, - cluster.getNamesystem(0).getCapacityTotal()); + assertEquals(initialCapacity, cluster.getNamesystem(0).getCapacityTotal(), + "Capacity should include only live capacity"); } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java index 3a74f835f4ad8..b4457d13c3927 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java @@ -24,7 +24,8 @@ import static org.apache.hadoop.test.MetricsAsserts.assertQuantileGauges; import static org.apache.hadoop.test.MetricsAsserts.getLongCounter; import static org.apache.hadoop.test.MetricsAsserts.getMetrics; -import static org.junit.Assert.*; +import static org.junit.jupiter.api.Assertions.*; +import static org.junit.jupiter.api.Assumptions.assumeTrue; import java.io.Closeable; import java.io.File; @@ -46,7 +47,6 @@ import org.apache.hadoop.net.unix.DomainSocket; import org.apache.hadoop.net.unix.TemporarySocketDirectory; import org.apache.hadoop.util.Lists; -import org.junit.Assume; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -69,7 +69,8 @@ import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.MetricsAsserts; import org.apache.hadoop.util.Time; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -97,8 +98,8 @@ public void testDataNodeMetrics() throws Exception { DataNode datanode = datanodes.get(0); MetricsRecordBuilder rb = getMetrics(datanode.getMetrics().name()); assertCounter("BytesWritten", LONG_FILE_LEN, rb); - assertTrue("Expected non-zero number of incremental block reports", - getLongCounter("IncrementalBlockReportsNumOps", rb) > 0); + assertTrue(getLongCounter("IncrementalBlockReportsNumOps", rb) > 0, + "Expected non-zero number of incremental block reports"); } finally { if (cluster != null) {cluster.shutdown();} } @@ -209,13 +210,11 @@ public Object answer(InvocationOnMock invocationOnMock) List datanodes = cluster.getDataNodes(); DataNode headNode = datanodes.stream().filter(d -> d.getDatanodeId().equals(headDatanodeInfo)) .findFirst().orElseGet(null); - assertNotNull("Could not find the head of the datanode write pipeline", - headNode); + assertNotNull(headNode, "Could not find the head of the datanode write pipeline"); MetricsRecordBuilder dnMetrics = getMetrics(headNode.getMetrics().name()); - assertTrue("More than 1 packet received", - getLongCounter("PacketsReceived", dnMetrics) > 1L); - assertTrue("More than 1 slow packet to mirror", - getLongCounter("PacketsSlowWriteToMirror", dnMetrics) > 1L); + assertTrue(getLongCounter("PacketsReceived", dnMetrics) > 1L, "More than 1 packet received"); + assertTrue(getLongCounter("PacketsSlowWriteToMirror", dnMetrics) > 1L, + "More than 1 slow packet to mirror"); assertCounter("PacketsSlowWriteToDisk", 1L, dnMetrics); assertCounter("PacketsSlowWriteToOsCache", 0L, dnMetrics); } finally { @@ -310,15 +309,14 @@ public void testRoundTripAckMetric() throws Exception { break; } } - assertNotNull("Could not find the head of the datanode write pipeline", - headNode); + assertNotNull(headNode, "Could not find the head of the datanode write pipeline"); // Close the file and wait for the metrics to rollover Thread.sleep((interval + 1) * 1000); // Check the ack was received MetricsRecordBuilder dnMetrics = getMetrics(headNode.getMetrics() .name()); - assertTrue("Expected non-zero number of acks", - getLongCounter("PacketAckRoundTripTimeNanosNumOps", dnMetrics) > 0); + assertTrue(getLongCounter("PacketAckRoundTripTimeNanosNumOps", dnMetrics) > 0, + "Expected non-zero number of acks"); assertQuantileGauges("PacketAckRoundTripTimeNanos" + interval + "s", dnMetrics); } finally { @@ -328,7 +326,8 @@ public void testRoundTripAckMetric() throws Exception { } } - @Test(timeout=60000) + @Test + @Timeout(value = 60) public void testTimeoutMetric() throws Exception { final Configuration conf = new HdfsConfiguration(); final Path path = new Path("/test"); @@ -363,10 +362,8 @@ public void testTimeoutMetric() throws Exception { final Object dnc = mbs.getAttribute(mxbeanName, "DatanodeNetworkCounts"); final String allDnc = dnc.toString(); - assertTrue("expected to see loopback address", - allDnc.indexOf("127.0.0.1") >= 0); - assertTrue("expected to see networkErrors", - allDnc.indexOf("networkErrors") >= 0); + assertTrue(allDnc.indexOf("127.0.0.1") >= 0, "expected to see loopback address"); + assertTrue(allDnc.indexOf("networkErrors") >= 0, "expected to see networkErrors"); } finally { IOUtils.cleanupWithLogger(LOG, streams.toArray(new Closeable[0])); if (cluster != null) { @@ -381,7 +378,8 @@ public void testTimeoutMetric() throws Exception { * and reading causes totalReadTime to move. * @throws Exception */ - @Test(timeout=120000) + @Test + @Timeout(value = 120) public void testDataNodeTimeSpend() throws Exception { Configuration conf = new HdfsConfiguration(); conf.set(DFSConfigKeys.DFS_METRICS_PERCENTILES_INTERVALS_KEY, "" + 60); @@ -441,7 +439,7 @@ public void testDatanodeBlocksReplicatedMetric() throws Exception { MetricsRecordBuilder rb = getMetrics(datanode.getMetrics().name()); long blocksReplicated = getLongCounter("BlocksReplicated", rb); - assertEquals("No blocks replicated yet", 0, blocksReplicated); + assertEquals(0, blocksReplicated, "No blocks replicated yet"); Path path = new Path("/counter.txt"); DFSTestUtil.createFile(fs, path, 1024, (short) 2, Time.monotonicNow()); @@ -451,7 +449,7 @@ public void testDatanodeBlocksReplicatedMetric() throws Exception { MetricsRecordBuilder rbNew = getMetrics(datanode.getMetrics().name()); blocksReplicated = getLongCounter("BlocksReplicated", rbNew); - assertEquals("blocks replicated counter incremented", 1, blocksReplicated); + assertEquals(1, blocksReplicated, "blocks replicated counter incremented"); } finally { if (cluster != null) { cluster.shutdown(); @@ -571,8 +569,8 @@ public void testDNShouldNotDeleteBlockONTooManyOpenFiles() CachingStrategy.newDefaultStrategy()); fail("Must throw FileNotFoundException"); } catch (FileNotFoundException fe) { - assertTrue("Should throw too many open files", - fe.getMessage().contains("Too many open files")); + assertTrue(fe.getMessage().contains("Too many open files"), + "Should throw too many open files"); } cluster.triggerHeartbeats(); // IBR delete ack //After DN throws too many open files @@ -611,7 +609,9 @@ public void testNNRpcMetricsWithNonHA() throws IOException { MetricsRecordBuilder rb = getMetrics(dn.getMetrics().name()); assertCounter("HeartbeatsNumOps", 1L, rb); } - @Test(timeout = 60000) + + @Test + @Timeout(value = 60) public void testSlowMetrics() throws Exception { DataNodeFaultInjector dnFaultInjector = new DataNodeFaultInjector() { @Override public void delay() { @@ -723,7 +723,7 @@ public void testNNRpcMetricsWithFederationAndHA() throws IOException { @Test public void testNodeLocalMetrics() throws Exception { - Assume.assumeTrue(null == DomainSocket.getLoadingFailureReason()); + assumeTrue(null == DomainSocket.getLoadingFailureReason()); Configuration conf = new HdfsConfiguration(); conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true); TemporarySocketDirectory sockDir = new TemporarySocketDirectory(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java index 5167042f9be35..ef0562517dc8b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java @@ -18,10 +18,10 @@ package org.apache.hadoop.hdfs.server.datanode; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNotSame; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNotSame; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.IOException; import java.net.InetSocketAddress; @@ -44,16 +44,17 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.StringUtils; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; public class TestDataNodeMultipleRegistrations { private static final Logger LOG = LoggerFactory.getLogger(TestDataNodeMultipleRegistrations.class); Configuration conf; - @Before + @BeforeEach public void setUp() throws Exception { conf = new HdfsConfiguration(); } @@ -73,8 +74,8 @@ public void test2NNRegistration() throws IOException { cluster.waitActive(); NameNode nn1 = cluster.getNameNode(0); NameNode nn2 = cluster.getNameNode(1); - assertNotNull("cannot create nn1", nn1); - assertNotNull("cannot create nn2", nn2); + assertNotNull(nn1, "cannot create nn1"); + assertNotNull(nn2, "cannot create nn2"); String bpid1 = FSImageTestUtil.getFSImage(nn1).getBlockPoolID(); String bpid2 = FSImageTestUtil.getFSImage(nn2).getBlockPoolID(); @@ -84,7 +85,7 @@ public void test2NNRegistration() throws IOException { int lv2 = FSImageTestUtil.getFSImage(nn2).getLayoutVersion(); int ns1 = FSImageTestUtil.getFSImage(nn1).getNamespaceID(); int ns2 = FSImageTestUtil.getFSImage(nn2).getNamespaceID(); - assertNotSame("namespace ids should be different", ns1, ns2); + assertNotSame(ns1, ns2, "namespace ids should be different"); LOG.info("nn1: lv=" + lv1 + ";cid=" + cid1 + ";bpid=" + bpid1 + ";uri=" + nn1.getNameNodeAddress()); LOG.info("nn2: lv=" + lv2 + ";cid=" + cid2 + ";bpid=" + bpid2 + ";uri=" @@ -93,15 +94,14 @@ public void test2NNRegistration() throws IOException { // check number of volumes in fsdataset DataNode dn = cluster.getDataNodes().get(0); final Map volInfos = dn.data.getVolumeInfoMap(); - Assert.assertTrue("No volumes in the fsdataset", volInfos.size() > 0); + Assertions.assertTrue(volInfos.size() > 0, "No volumes in the fsdataset"); int i = 0; for (Map.Entry e : volInfos.entrySet()) { LOG.info("vol " + i++ + ") " + e.getKey() + ": " + e.getValue()); } // number of volumes should be 2 - [data1, data2] - assertEquals("number of volumes is wrong", - cluster.getFsDatasetTestUtils(0).getDefaultNumOfDataDirs(), - volInfos.size()); + assertEquals(cluster.getFsDatasetTestUtils(0).getDefaultNumOfDataDirs(), volInfos.size(), + "number of volumes is wrong"); for (BPOfferService bpos : dn.getAllBpOs()) { LOG.info("BP: " + bpos); @@ -117,18 +117,14 @@ public void test2NNRegistration() throws IOException { bpos2 = tmp; } - assertEquals("wrong nn address", getNNSocketAddress(bpos1), - nn1.getNameNodeAddress()); - assertEquals("wrong nn address", getNNSocketAddress(bpos2), - nn2.getNameNodeAddress()); - assertEquals("wrong bpid", bpos1.getBlockPoolId(), bpid1); - assertEquals("wrong bpid", bpos2.getBlockPoolId(), bpid2); - assertEquals("wrong cid", dn.getClusterId(), cid1); - assertEquals("cid should be same", cid2, cid1); - assertEquals("namespace should be same", - bpos1.bpNSInfo.namespaceID, ns1); - assertEquals("namespace should be same", - bpos2.bpNSInfo.namespaceID, ns2); + assertEquals(getNNSocketAddress(bpos1), nn1.getNameNodeAddress(), "wrong nn address"); + assertEquals(getNNSocketAddress(bpos2), nn2.getNameNodeAddress(), "wrong nn address"); + assertEquals(bpos1.getBlockPoolId(), bpid1, "wrong bpid"); + assertEquals(bpos2.getBlockPoolId(), bpid2, "wrong bpid"); + assertEquals(dn.getClusterId(), cid1, "wrong cid"); + assertEquals(cid2, cid1, "cid should be same"); + assertEquals(bpos1.bpNSInfo.namespaceID, ns1, "namespace should be same"); + assertEquals(bpos2.bpNSInfo.namespaceID, ns2, "namespace should be same"); } finally { cluster.shutdown(); } @@ -151,7 +147,7 @@ public void testFedSingleNN() throws IOException { .nameNodePort(9927).build(); try { NameNode nn1 = cluster.getNameNode(); - assertNotNull("cannot create nn1", nn1); + assertNotNull(nn1, "cannot create nn1"); String bpid1 = FSImageTestUtil.getFSImage(nn1).getBlockPoolID(); String cid1 = FSImageTestUtil.getFSImage(nn1).getClusterID(); @@ -162,15 +158,14 @@ public void testFedSingleNN() throws IOException { // check number of vlumes in fsdataset DataNode dn = cluster.getDataNodes().get(0); final Map volInfos = dn.data.getVolumeInfoMap(); - Assert.assertTrue("No volumes in the fsdataset", volInfos.size() > 0); + Assertions.assertTrue(volInfos.size() > 0, "No volumes in the fsdataset"); int i = 0; for (Map.Entry e : volInfos.entrySet()) { LOG.info("vol " + i++ + ") " + e.getKey() + ": " + e.getValue()); } // number of volumes should be 2 - [data1, data2] - assertEquals("number of volumes is wrong", - cluster.getFsDatasetTestUtils(0).getDefaultNumOfDataDirs(), - volInfos.size()); + assertEquals(cluster.getFsDatasetTestUtils(0).getDefaultNumOfDataDirs(), volInfos.size(), + "number of volumes is wrong"); for (BPOfferService bpos : dn.getAllBpOs()) { LOG.info("reg: bpid=" + "; name=" + bpos.bpRegistration + "; sid=" @@ -182,11 +177,11 @@ public void testFedSingleNN() throws IOException { BPOfferService bpos1 = dn.getAllBpOs().get(0); bpos1.triggerBlockReportForTests(); - assertEquals("wrong nn address", - getNNSocketAddress(bpos1), - nn1.getNameNodeAddress()); - assertEquals("wrong bpid", bpos1.getBlockPoolId(), bpid1); - assertEquals("wrong cid", dn.getClusterId(), cid1); + assertEquals(getNNSocketAddress(bpos1), + nn1.getNameNodeAddress(), + "wrong nn address"); + assertEquals(bpos1.getBlockPoolId(), bpid1, "wrong bpid"); + assertEquals(dn.getClusterId(), cid1, "wrong cid"); cluster.shutdown(); // Ensure all the BPOfferService threads are shutdown @@ -210,31 +205,32 @@ public void testClusterIdMismatch() throws Exception { DataNode dn = cluster.getDataNodes().get(0); List bposs = dn.getAllBpOs(); LOG.info("dn bpos len (should be 2):" + bposs.size()); - Assert.assertEquals("should've registered with two namenodes", bposs.size(),2); + Assertions.assertEquals(bposs.size(), 2, "should've registered with two namenodes"); // add another namenode cluster.addNameNode(conf, 9938); Thread.sleep(500);// lets wait for the registration to happen bposs = dn.getAllBpOs(); LOG.info("dn bpos len (should be 3):" + bposs.size()); - Assert.assertEquals("should've registered with three namenodes", bposs.size(),3); + Assertions.assertEquals(bposs.size(), 3, "should've registered with three namenodes"); // change cluster id and another Namenode StartupOption.FORMAT.setClusterId("DifferentCID"); cluster.addNameNode(conf, 9948); NameNode nn4 = cluster.getNameNode(3); - assertNotNull("cannot create nn4", nn4); + assertNotNull(nn4, "cannot create nn4"); Thread.sleep(500);// lets wait for the registration to happen bposs = dn.getAllBpOs(); LOG.info("dn bpos len (still should be 3):" + bposs.size()); - Assert.assertEquals("should've registered with three namenodes", 3, bposs.size()); + Assertions.assertEquals(3, bposs.size(), "should've registered with three namenodes"); } finally { cluster.shutdown(); } } - @Test(timeout = 20000) + @Test + @Timeout(value = 20) public void testClusterIdMismatchAtStartupWithHA() throws Exception { MiniDFSNNTopology top = new MiniDFSNNTopology() .addNameservice(new MiniDFSNNTopology.NSConf("ns1") @@ -254,9 +250,8 @@ public void testClusterIdMismatchAtStartupWithHA() throws Exception { // let the initialization be complete cluster.waitActive(); DataNode dn = cluster.getDataNodes().get(0); - assertTrue("Datanode should be running", dn.isDatanodeUp()); - assertEquals("Only one BPOfferService should be running", 1, - dn.getAllBpOs().size()); + assertTrue(dn.isDatanodeUp(), "Datanode should be running"); + assertEquals(1, dn.getAllBpOs().size(), "Only one BPOfferService should be running"); } finally { cluster.shutdown(); } @@ -278,9 +273,8 @@ public void testDNWithInvalidStorageWithHA() throws Exception { // let the initialization be complete cluster.waitActive(); DataNode dn = cluster.getDataNodes().get(0); - assertTrue("Datanode should be running", dn.isDatanodeUp()); - assertEquals("BPOfferService should be running", 1, - dn.getAllBpOs().size()); + assertTrue(dn.isDatanodeUp(), "Datanode should be running"); + assertEquals(1, dn.getAllBpOs().size(), "BPOfferService should be running"); DataNodeProperties dnProp = cluster.stopDataNode(0); cluster.getNameNode(0).stop(); @@ -328,12 +322,12 @@ public void testMiniDFSClusterWithMultipleNN() throws IOException { // add a node try { cluster.waitActive(); - Assert.assertEquals("(1)Should be 2 namenodes", 2, cluster.getNumNameNodes()); + Assertions.assertEquals(2, cluster.getNumNameNodes(), "(1)Should be 2 namenodes"); cluster.addNameNode(conf, 0); - Assert.assertEquals("(1)Should be 3 namenodes", 3, cluster.getNumNameNodes()); + Assertions.assertEquals(3, cluster.getNumNameNodes(), "(1)Should be 3 namenodes"); } catch (IOException ioe) { - Assert.fail("Failed to add NN to cluster:" + StringUtils.stringifyException(ioe)); + Assertions.fail("Failed to add NN to cluster:" + StringUtils.stringifyException(ioe)); } finally { cluster.shutdown(); } @@ -345,15 +339,15 @@ public void testMiniDFSClusterWithMultipleNN() throws IOException { .build(); try { - Assert.assertNotNull(cluster); + Assertions.assertNotNull(cluster); cluster.waitActive(); - Assert.assertEquals("(2)Should be 1 namenodes", 1, cluster.getNumNameNodes()); + Assertions.assertEquals(1, cluster.getNumNameNodes(), "(2)Should be 1 namenodes"); // add a node cluster.addNameNode(conf, 0); - Assert.assertEquals("(2)Should be 2 namenodes", 2, cluster.getNumNameNodes()); + Assertions.assertEquals(2, cluster.getNumNameNodes(), "(2)Should be 2 namenodes"); } catch (IOException ioe) { - Assert.fail("Failed to add NN to cluster:" + StringUtils.stringifyException(ioe)); + Assertions.fail("Failed to add NN to cluster:" + StringUtils.stringifyException(ioe)); } finally { cluster.shutdown(); } @@ -365,15 +359,15 @@ public void testMiniDFSClusterWithMultipleNN() throws IOException { // add a node try { cluster.waitActive(); - Assert.assertNotNull(cluster); - Assert.assertEquals("(2)Should be 1 namenodes", 1, cluster.getNumNameNodes()); + Assertions.assertNotNull(cluster); + Assertions.assertEquals(1, cluster.getNumNameNodes(), "(2)Should be 1 namenodes"); cluster.addNameNode(conf, 9929); - Assert.fail("shouldn't be able to add another NN to non federated cluster"); + Assertions.fail("shouldn't be able to add another NN to non federated cluster"); } catch (IOException e) { // correct - Assert.assertTrue(e.getMessage().startsWith("cannot add namenode")); - Assert.assertEquals("(3)Should be 1 namenodes", 1, cluster.getNumNameNodes()); + Assertions.assertTrue(e.getMessage().startsWith("cannot add namenode")); + Assertions.assertEquals(1, cluster.getNumNameNodes(), "(3)Should be 1 namenodes"); } finally { cluster.shutdown(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodePeerMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodePeerMetrics.java index 9ea8a08a7629c..bff29543c32c8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodePeerMetrics.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodePeerMetrics.java @@ -29,20 +29,21 @@ import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.Time; import org.apache.hadoop.conf.Configuration; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_PEER_METRICS_MIN_OUTLIER_DETECTION_SAMPLES_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_PEER_STATS_ENABLED_KEY; -import static org.hamcrest.CoreMatchers.containsString; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThat; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.assertj.core.api.Assertions.assertThat; /** * This class tests various cases of DataNode peer metrics. */ public class TestDataNodePeerMetrics { - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testGetSendPacketDownstreamAvgInfo() throws Exception { final int windowSize = 5; // 5s roll over interval final int numWindows = 2; // 2 rolling windows @@ -83,11 +84,12 @@ public void testGetSendPacketDownstreamAvgInfo() throws Exception { * "[49.236.149.246:9801]RollingAvgTime":504.463, * "[84.125.113.65:9801]RollingAvgTime":497.954} */ - assertThat(json, containsString(peerAddr)); + assertThat(json).contains(peerAddr); } } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testRemoveStaleRecord() throws Exception { final int numWindows = 5; final long scheduleInterval = 1000; @@ -124,7 +126,7 @@ public void testRemoveStaleRecord() throws Exception { assertEquals(3, rollingAverages.getStats(numSamples).size()); String json = peerMetrics.dumpSendPacketDownstreamAvgInfoAsJson(); for (String peerAddr : peerAddrList) { - assertThat(json, containsString(peerAddr)); + assertThat(json).contains(peerAddr); } /* wait for stale report to be removed */ GenericTestUtils.waitFor( @@ -146,7 +148,7 @@ public void testRemoveStaleRecord() throws Exception { assertEquals(3, rollingAverages.getStats(numSamples).size()); json = peerMetrics.dumpSendPacketDownstreamAvgInfoAsJson(); for (String peerAddr : peerAddrList) { - assertThat(json, containsString(peerAddr)); + assertThat(json).contains(peerAddr); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java index 69f0565f2b6b6..1101368ecbe19 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java @@ -55,11 +55,11 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DISK_BALANCER_ENABLED_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DISK_BALANCER_PLAN_VALID_INTERVAL; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DISK_BALANCER_PLAN_VALID_INTERVAL_DEFAULT; -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.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.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import java.io.File; import java.io.IOException; @@ -81,10 +81,10 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.BlockPoolSlice; import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl; import org.apache.hadoop.test.LambdaTestUtils; -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.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** * Test to reconfigure some parameters for DataNode without restart @@ -100,12 +100,12 @@ public class TestDataNodeReconfiguration { private MiniDFSCluster cluster; private static long counter = 0; - @Before + @BeforeEach public void Setup() throws IOException { startDFSCluster(NUM_NAME_NODE, NUM_DATA_NODE); } - @After + @AfterEach public void tearDown() throws Exception { if (cluster != null) { cluster.shutdown(); @@ -114,8 +114,8 @@ public void tearDown() throws Exception { File dir = new File(DATA_DIR); if (dir.exists()) - Assert.assertTrue("Cannot delete data-node dirs", - FileUtil.fullyDelete(dir)); + Assertions.assertTrue(FileUtil.fullyDelete(dir), + "Cannot delete data-node dirs"); } private void startDFSCluster(int numNameNodes, int numDataNodes) @@ -164,8 +164,8 @@ public void testMaxConcurrentMoversReconfiguration() DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY, "text"); fail("ReconfigurationException expected"); } catch (ReconfigurationException expected) { - assertTrue("expecting NumberFormatException", - expected.getCause() instanceof NumberFormatException); + assertTrue(expected.getCause() instanceof NumberFormatException, + "expecting NumberFormatException"); } try { dn.reconfigureProperty( @@ -173,8 +173,8 @@ public void testMaxConcurrentMoversReconfiguration() String.valueOf(-1)); fail("ReconfigurationException expected"); } catch (ReconfigurationException expected) { - assertTrue("expecting IllegalArgumentException", - expected.getCause() instanceof IllegalArgumentException); + assertTrue(expected.getCause() instanceof IllegalArgumentException, + "expecting IllegalArgumentException"); } try { dn.reconfigureProperty( @@ -182,8 +182,8 @@ public void testMaxConcurrentMoversReconfiguration() String.valueOf(0)); fail("ReconfigurationException expected"); } catch (ReconfigurationException expected) { - assertTrue("expecting IllegalArgumentException", - expected.getCause() instanceof IllegalArgumentException); + assertTrue(expected.getCause() instanceof IllegalArgumentException, + "expecting IllegalArgumentException"); } // change properties @@ -191,28 +191,25 @@ public void testMaxConcurrentMoversReconfiguration() String.valueOf(maxConcurrentMovers)); // verify change - assertEquals(String.format("%s has wrong value", - DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY), - maxConcurrentMovers, dn.xserver.balanceThrottler.getMaxConcurrentMovers()); + assertEquals(maxConcurrentMovers, dn.xserver.balanceThrottler.getMaxConcurrentMovers(), + String.format("%s has wrong value", DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY)); - assertEquals(String.format("%s has wrong value", - DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY), - maxConcurrentMovers, Integer.parseInt(dn.getConf().get( - DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY))); + assertEquals(maxConcurrentMovers, + Integer.parseInt(dn.getConf().get(DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY)), + String.format("%s has wrong value", DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY)); // revert to default dn.reconfigureProperty(DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY, null); // verify default - assertEquals(String.format("%s has wrong value", - DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY), - DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_DEFAULT, - dn.xserver.balanceThrottler.getMaxConcurrentMovers()); - - assertEquals(String.format("expect %s is not configured", - DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY), null, dn - .getConf().get(DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY)); + assertEquals(DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_DEFAULT, + dn.xserver.balanceThrottler.getMaxConcurrentMovers(), + String.format("%s has wrong value", DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY)); + + assertEquals(null, dn.getConf().get(DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY), + String.format("expect %s is not configured", + DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY)); } } @@ -263,7 +260,7 @@ public void testFailedDecreaseConcurrentMovers() // Attempt to set new maximum to 1 final boolean success = dataNode.xserver.updateBalancerMaxConcurrentMovers(1); - Assert.assertFalse(success); + Assertions.assertFalse(success); } finally { dataNode.shutdown(); } @@ -272,32 +269,34 @@ public void testFailedDecreaseConcurrentMovers() /** * Test with invalid configuration. */ - @Test(expected = ReconfigurationException.class) + @Test public void testFailedDecreaseConcurrentMoversReconfiguration() throws IOException, ReconfigurationException { - final DataNode[] dns = createDNsForTest(1); - final DataNode dataNode = dns[0]; - try { - // Set the current max to 2 - dataNode.xserver.updateBalancerMaxConcurrentMovers(2); - - // Simulate grabbing 2 threads - dataNode.xserver.balanceThrottler.acquire(); - dataNode.xserver.balanceThrottler.acquire(); - - dataNode.xserver.setMaxReconfigureWaitTime(1); - - // Now try reconfigure maximum downwards with threads released - dataNode.reconfigurePropertyImpl( - DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY, "1"); - } catch (ReconfigurationException e) { - Assert.assertEquals(DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY, - e.getProperty()); - Assert.assertEquals("1", e.getNewValue()); - throw e; - } finally { - dataNode.shutdown(); - } + Assertions.assertThrows(ReconfigurationException.class, () -> { + final DataNode[] dns = createDNsForTest(1); + final DataNode dataNode = dns[0]; + try { + // Set the current max to 2 + dataNode.xserver.updateBalancerMaxConcurrentMovers(2); + + // Simulate grabbing 2 threads + dataNode.xserver.balanceThrottler.acquire(); + dataNode.xserver.balanceThrottler.acquire(); + + dataNode.xserver.setMaxReconfigureWaitTime(1); + + // Now try reconfigure maximum downwards with threads released + dataNode.reconfigurePropertyImpl( + DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY, "1"); + } catch (ReconfigurationException e) { + Assertions.assertEquals(DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY, + e.getProperty()); + Assertions.assertEquals("1", e.getNewValue()); + throw e; + } finally { + dataNode.shutdown(); + } + }); } private void testAcquireOnMaxConcurrentMoversReconfiguration( @@ -310,12 +309,12 @@ private void testAcquireOnMaxConcurrentMoversReconfiguration( /** Test that the default setup is working */ for (int i = 0; i < defaultMaxThreads; i++) { - assertEquals("should be able to get thread quota", true, - dataNode.xserver.balanceThrottler.acquire()); + assertEquals(true, dataNode.xserver.balanceThrottler.acquire(), + "should be able to get thread quota"); } - assertEquals("should not be able to get thread quota", false, - dataNode.xserver.balanceThrottler.acquire()); + assertEquals(false, dataNode.xserver.balanceThrottler.acquire(), + "should not be able to get thread quota"); // Give back the threads for (int i = 0; i < defaultMaxThreads; i++) { @@ -329,16 +328,16 @@ private void testAcquireOnMaxConcurrentMoversReconfiguration( DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY, String.valueOf(maxConcurrentMovers)); - assertEquals("thread quota is wrong", maxConcurrentMovers, - dataNode.xserver.balanceThrottler.getMaxConcurrentMovers()); + assertEquals(maxConcurrentMovers, dataNode.xserver.balanceThrottler.getMaxConcurrentMovers(), + "thread quota is wrong"); for (int i = 0; i < maxConcurrentMovers; i++) { - assertEquals("should be able to get thread quota", true, - dataNode.xserver.balanceThrottler.acquire()); + assertEquals(true, dataNode.xserver.balanceThrottler.acquire(), + "should be able to get thread quota"); } - assertEquals("should not be able to get thread quota", false, - dataNode.xserver.balanceThrottler.acquire()); + assertEquals(false, dataNode.xserver.balanceThrottler.acquire(), + "should not be able to get thread quota"); } @Test @@ -360,8 +359,8 @@ public void testBlockReportIntervalReconfiguration() dn.reconfigureProperty(blockReportParameter, "text"); fail("ReconfigurationException expected"); } catch (ReconfigurationException expected) { - assertTrue("expecting NumberFormatException", - expected.getCause() instanceof NumberFormatException); + assertTrue(expected.getCause() instanceof NumberFormatException, + "expecting NumberFormatException"); } } @@ -369,15 +368,15 @@ public void testBlockReportIntervalReconfiguration() dn.reconfigureProperty(DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, String.valueOf(-1)); fail("ReconfigurationException expected"); } catch (ReconfigurationException expected) { - assertTrue("expecting IllegalArgumentException", - expected.getCause() instanceof IllegalArgumentException); + assertTrue(expected.getCause() instanceof IllegalArgumentException, + "expecting IllegalArgumentException"); } try { dn.reconfigureProperty(DFS_BLOCKREPORT_SPLIT_THRESHOLD_KEY, String.valueOf(-1)); fail("ReconfigurationException expected"); } catch (ReconfigurationException expected) { - assertTrue("expecting IllegalArgumentException", - expected.getCause() instanceof IllegalArgumentException); + assertTrue(expected.getCause() instanceof IllegalArgumentException, + "expecting IllegalArgumentException"); } dn.reconfigureProperty(DFS_BLOCKREPORT_INITIAL_DELAY_KEY, String.valueOf(-1)); assertEquals(0, dn.getDnConf().initialBlockReportDelayMs); @@ -388,10 +387,8 @@ public void testBlockReportIntervalReconfiguration() for (BPOfferService bpos : blockPoolManager.getAllNamenodeThreads()) { if (bpos != null) { for (BPServiceActor actor : bpos.getBPServiceActors()) { - assertEquals(String.format("%s has wrong value", - DFS_BLOCKREPORT_INTERVAL_MSEC_KEY), - blockReportInterval, - actor.getScheduler().getBlockReportIntervalMs()); + assertEquals(blockReportInterval, actor.getScheduler().getBlockReportIntervalMs(), + String.format("%s has wrong value", DFS_BLOCKREPORT_INTERVAL_MSEC_KEY)); } } } @@ -407,23 +404,22 @@ public void testBlockReportIntervalReconfiguration() for (BPOfferService bpos : blockPoolManager.getAllNamenodeThreads()) { if (bpos != null) { for (BPServiceActor actor : bpos.getBPServiceActors()) { - assertEquals(String.format("%s has wrong value", - DFS_BLOCKREPORT_INTERVAL_MSEC_KEY), - DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT, - actor.getScheduler().getBlockReportIntervalMs()); + assertEquals(DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT, + actor.getScheduler().getBlockReportIntervalMs(), + String.format("%s has wrong value", DFS_BLOCKREPORT_INTERVAL_MSEC_KEY)); } } } - assertNull(String.format("expect %s is not configured", DFS_BLOCKREPORT_INTERVAL_MSEC_KEY), - dn.getConf().get(DFS_BLOCKREPORT_INTERVAL_MSEC_KEY)); + assertNull(dn.getConf().get(DFS_BLOCKREPORT_INTERVAL_MSEC_KEY), + String.format("expect %s is not configured", DFS_BLOCKREPORT_INTERVAL_MSEC_KEY)); dn.reconfigureProperty(DFS_BLOCKREPORT_SPLIT_THRESHOLD_KEY, null); - assertNull(String.format("expect %s is not configured", DFS_BLOCKREPORT_SPLIT_THRESHOLD_KEY), - dn.getConf().get(DFS_BLOCKREPORT_SPLIT_THRESHOLD_KEY)); + assertNull(dn.getConf().get(DFS_BLOCKREPORT_SPLIT_THRESHOLD_KEY), + String.format("expect %s is not configured", DFS_BLOCKREPORT_SPLIT_THRESHOLD_KEY)); dn.reconfigureProperty(DFS_BLOCKREPORT_INITIAL_DELAY_KEY, null); - assertNull(String.format("expect %s is not configured", DFS_BLOCKREPORT_INITIAL_DELAY_KEY), - dn.getConf().get(DFS_BLOCKREPORT_INITIAL_DELAY_KEY)); + assertNull(dn.getConf().get(DFS_BLOCKREPORT_INITIAL_DELAY_KEY), + String.format("expect %s is not configured", DFS_BLOCKREPORT_INITIAL_DELAY_KEY)); } } @@ -438,92 +434,83 @@ public void testDataXceiverReconfiguration() dn.reconfigureProperty(DFS_DATANODE_MAX_RECEIVER_THREADS_KEY, "text"); fail("ReconfigurationException expected"); } catch (ReconfigurationException expected) { - assertTrue("expecting NumberFormatException", - expected.getCause() instanceof NumberFormatException); + assertTrue(expected.getCause() instanceof NumberFormatException, + "expecting NumberFormatException"); } try { dn.reconfigureProperty(DFS_DATANODE_MAX_RECEIVER_THREADS_KEY, String.valueOf(-1)); fail("ReconfigurationException expected"); } catch (ReconfigurationException expected) { - assertTrue("expecting IllegalArgumentException", - expected.getCause() instanceof IllegalArgumentException); + assertTrue(expected.getCause() instanceof IllegalArgumentException, + "expecting IllegalArgumentException"); } try { dn.reconfigureProperty(DFS_DATANODE_DATA_TRANSFER_BANDWIDTHPERSEC_KEY, "text"); fail("ReconfigurationException expected"); } catch (ReconfigurationException expected) { - assertTrue("expecting NumberFormatException", - expected.getCause() instanceof NumberFormatException); + assertTrue(expected.getCause() instanceof NumberFormatException, + "expecting NumberFormatException"); } try { dn.reconfigureProperty(DFS_DATANODE_DATA_WRITE_BANDWIDTHPERSEC_KEY, "text"); fail("ReconfigurationException expected"); } catch (ReconfigurationException expected) { - assertTrue("expecting NumberFormatException", - expected.getCause() instanceof NumberFormatException); + assertTrue(expected.getCause() instanceof NumberFormatException, + "expecting NumberFormatException"); } try { dn.reconfigureProperty(DFS_DATANODE_DATA_READ_BANDWIDTHPERSEC_KEY, "text"); fail("ReconfigurationException expected"); } catch (ReconfigurationException expected) { - assertTrue("expecting NumberFormatException", - expected.getCause() instanceof NumberFormatException); + assertTrue(expected.getCause() instanceof NumberFormatException, + "expecting NumberFormatException"); } // Change properties and verify change. dn.reconfigureProperty(DFS_DATANODE_MAX_RECEIVER_THREADS_KEY, String.valueOf(123)); - assertEquals(String.format("%s has wrong value", DFS_DATANODE_MAX_RECEIVER_THREADS_KEY), - 123, dn.getXferServer().getMaxXceiverCount()); + assertEquals(123, dn.getXferServer().getMaxXceiverCount(), + String.format("%s has wrong value", DFS_DATANODE_MAX_RECEIVER_THREADS_KEY)); dn.reconfigureProperty(DFS_DATANODE_DATA_TRANSFER_BANDWIDTHPERSEC_KEY, String.valueOf(1000)); - assertEquals(String.format("%s has wrong value", - DFS_DATANODE_DATA_TRANSFER_BANDWIDTHPERSEC_KEY), - 1000, dn.getXferServer().getTransferThrottler().getBandwidth()); + assertEquals(1000, dn.getXferServer().getTransferThrottler().getBandwidth(), + String.format("%s has wrong value", DFS_DATANODE_DATA_TRANSFER_BANDWIDTHPERSEC_KEY)); dn.reconfigureProperty(DFS_DATANODE_DATA_WRITE_BANDWIDTHPERSEC_KEY, String.valueOf(1000)); - assertEquals(String.format("%s has wrong value", - DFS_DATANODE_DATA_WRITE_BANDWIDTHPERSEC_KEY), - 1000, dn.getXferServer().getWriteThrottler().getBandwidth()); + assertEquals(1000, dn.getXferServer().getWriteThrottler().getBandwidth(), + String.format("%s has wrong value", DFS_DATANODE_DATA_WRITE_BANDWIDTHPERSEC_KEY)); dn.reconfigureProperty(DFS_DATANODE_DATA_READ_BANDWIDTHPERSEC_KEY, String.valueOf(1000)); - assertEquals(String.format("%s has wrong value", - DFS_DATANODE_DATA_READ_BANDWIDTHPERSEC_KEY), - 1000, dn.getXferServer().getReadThrottler().getBandwidth()); + assertEquals(1000, dn.getXferServer().getReadThrottler().getBandwidth(), + String.format("%s has wrong value", DFS_DATANODE_DATA_READ_BANDWIDTHPERSEC_KEY)); // Revert to default. dn.reconfigureProperty(DFS_DATANODE_MAX_RECEIVER_THREADS_KEY, null); - assertEquals(String.format("%s has wrong value", DFS_DATANODE_MAX_RECEIVER_THREADS_KEY), - DFS_DATANODE_MAX_RECEIVER_THREADS_DEFAULT, dn.getXferServer().getMaxXceiverCount()); - assertNull(String.format("expect %s is not configured", - DFS_DATANODE_MAX_RECEIVER_THREADS_KEY), - dn.getConf().get(DFS_DATANODE_MAX_RECEIVER_THREADS_KEY)); + assertEquals(DFS_DATANODE_MAX_RECEIVER_THREADS_DEFAULT, + dn.getXferServer().getMaxXceiverCount(), + String.format("%s has wrong value", DFS_DATANODE_MAX_RECEIVER_THREADS_KEY)); + assertNull(dn.getConf().get(DFS_DATANODE_MAX_RECEIVER_THREADS_KEY), + String.format("expect %s is not configured", DFS_DATANODE_MAX_RECEIVER_THREADS_KEY)); dn.reconfigureProperty(DFS_DATANODE_DATA_TRANSFER_BANDWIDTHPERSEC_KEY, null); - assertEquals(String.format("%s has wrong value", - DFS_DATANODE_DATA_TRANSFER_BANDWIDTHPERSEC_KEY), - null, dn.getXferServer().getTransferThrottler()); - assertNull(String.format("expect %s is not configured", - DFS_DATANODE_DATA_TRANSFER_BANDWIDTHPERSEC_KEY), - dn.getConf().get(DFS_DATANODE_DATA_TRANSFER_BANDWIDTHPERSEC_KEY)); + assertEquals(null, dn.getXferServer().getTransferThrottler(), + String.format("%s has wrong value", DFS_DATANODE_DATA_TRANSFER_BANDWIDTHPERSEC_KEY)); + assertNull(dn.getConf().get(DFS_DATANODE_DATA_TRANSFER_BANDWIDTHPERSEC_KEY), String + .format("expect %s is not configured", DFS_DATANODE_DATA_TRANSFER_BANDWIDTHPERSEC_KEY)); dn.reconfigureProperty(DFS_DATANODE_DATA_WRITE_BANDWIDTHPERSEC_KEY, null); - assertEquals(String.format("%s has wrong value", - DFS_DATANODE_DATA_WRITE_BANDWIDTHPERSEC_KEY), - null, dn.getXferServer().getWriteThrottler()); - assertNull(String.format("expect %s is not configured", - DFS_DATANODE_DATA_WRITE_BANDWIDTHPERSEC_KEY), - dn.getConf().get(DFS_DATANODE_DATA_WRITE_BANDWIDTHPERSEC_KEY)); + assertEquals(null, dn.getXferServer().getWriteThrottler(), + String.format("%s has wrong value", DFS_DATANODE_DATA_WRITE_BANDWIDTHPERSEC_KEY)); + assertNull(dn.getConf().get(DFS_DATANODE_DATA_WRITE_BANDWIDTHPERSEC_KEY), String + .format("expect %s is not configured", DFS_DATANODE_DATA_WRITE_BANDWIDTHPERSEC_KEY)); dn.reconfigureProperty(DFS_DATANODE_DATA_READ_BANDWIDTHPERSEC_KEY, null); - assertEquals(String.format("%s has wrong value", - DFS_DATANODE_DATA_READ_BANDWIDTHPERSEC_KEY), - null, dn.getXferServer().getReadThrottler()); - assertNull(String.format("expect %s is not configured", - DFS_DATANODE_DATA_READ_BANDWIDTHPERSEC_KEY), - dn.getConf().get(DFS_DATANODE_DATA_READ_BANDWIDTHPERSEC_KEY)); + assertEquals(null, dn.getXferServer().getReadThrottler(), + String.format("%s has wrong value", DFS_DATANODE_DATA_READ_BANDWIDTHPERSEC_KEY)); + assertNull(dn.getConf().get(DFS_DATANODE_DATA_READ_BANDWIDTHPERSEC_KEY), + String.format("expect %s is not configured", DFS_DATANODE_DATA_READ_BANDWIDTHPERSEC_KEY)); } } @@ -539,15 +526,15 @@ public void testCacheReportReconfiguration() dn.reconfigureProperty(DFS_CACHEREPORT_INTERVAL_MSEC_KEY, "text"); fail("ReconfigurationException expected"); } catch (ReconfigurationException expected) { - assertTrue("expecting NumberFormatException", - expected.getCause() instanceof NumberFormatException); + assertTrue(expected.getCause() instanceof NumberFormatException, + "expecting NumberFormatException"); } try { dn.reconfigureProperty(DFS_CACHEREPORT_INTERVAL_MSEC_KEY, String.valueOf(-1)); fail("ReconfigurationException expected"); } catch (ReconfigurationException expected) { - assertTrue("expecting IllegalArgumentException", - expected.getCause() instanceof IllegalArgumentException); + assertTrue(expected.getCause() instanceof IllegalArgumentException, + "expecting IllegalArgumentException"); } // Change properties. @@ -555,16 +542,16 @@ public void testCacheReportReconfiguration() String.valueOf(cacheReportInterval)); // Verify change. - assertEquals(String.format("%s has wrong value", DFS_CACHEREPORT_INTERVAL_MSEC_KEY), - cacheReportInterval, dn.getDnConf().getCacheReportInterval()); + assertEquals(cacheReportInterval, dn.getDnConf().getCacheReportInterval(), + String.format("%s has wrong value", DFS_CACHEREPORT_INTERVAL_MSEC_KEY)); // Revert to default. dn.reconfigureProperty(DFS_CACHEREPORT_INTERVAL_MSEC_KEY, null); - assertEquals(String.format("%s has wrong value", DFS_CACHEREPORT_INTERVAL_MSEC_KEY), - DFS_CACHEREPORT_INTERVAL_MSEC_DEFAULT, dn.getDnConf().getCacheReportInterval()); + assertEquals(DFS_CACHEREPORT_INTERVAL_MSEC_DEFAULT, dn.getDnConf().getCacheReportInterval(), + String.format("%s has wrong value", DFS_CACHEREPORT_INTERVAL_MSEC_KEY)); - assertNull(String.format("expect %s is not configured", DFS_CACHEREPORT_INTERVAL_MSEC_KEY), - dn.getConf().get(DFS_CACHEREPORT_INTERVAL_MSEC_KEY)); + assertNull(dn.getConf().get(DFS_CACHEREPORT_INTERVAL_MSEC_KEY), + String.format("expect %s is not configured", DFS_CACHEREPORT_INTERVAL_MSEC_KEY)); } } @@ -588,16 +575,16 @@ public void testSlowPeerParameters() throws Exception { dn.reconfigureProperty(parameter, "text"); fail("ReconfigurationException expected"); } catch (ReconfigurationException expected) { - assertTrue("expecting NumberFormatException", - expected.getCause() instanceof NumberFormatException); + assertTrue(expected.getCause() instanceof NumberFormatException, + "expecting NumberFormatException"); } try { dn.reconfigureProperty(parameter, String.valueOf(-1)); fail("ReconfigurationException expected"); } catch (ReconfigurationException expected) { - assertTrue("expecting IllegalArgumentException", - expected.getCause() instanceof IllegalArgumentException); + assertTrue(expected.getCause() instanceof IllegalArgumentException, + "expecting IllegalArgumentException"); } } @@ -613,16 +600,13 @@ public void testSlowPeerParameters() throws Exception { assertEquals(123, dn.getPeerMetrics().getMinOutlierDetectionNodes()); assertEquals(123, dn.getPeerMetrics().getLowThresholdMs()); assertEquals(123, dn.getPeerMetrics().getMinOutlierDetectionSamples()); - assertEquals(123, - dn.getPeerMetrics().getSlowNodeDetector().getMinOutlierDetectionNodes()); - assertEquals(123, - dn.getPeerMetrics().getSlowNodeDetector().getLowThresholdMs()); + assertEquals(123, dn.getPeerMetrics().getSlowNodeDetector().getMinOutlierDetectionNodes()); + assertEquals(123, dn.getPeerMetrics().getSlowNodeDetector().getLowThresholdMs()); // Revert to default and verify. dn.reconfigureProperty(DFS_DATANODE_PEER_STATS_ENABLED_KEY, null); - assertEquals(String.format("expect %s is not configured", - DFS_DATANODE_PEER_STATS_ENABLED_KEY), null, - dn.getConf().get(DFS_DATANODE_PEER_STATS_ENABLED_KEY)); + assertEquals(null, dn.getConf().get(DFS_DATANODE_PEER_STATS_ENABLED_KEY), + String.format("expect %s is not configured", DFS_DATANODE_PEER_STATS_ENABLED_KEY)); // Reset DFS_DATANODE_PEER_STATS_ENABLED_KEY to true. dn.reconfigureProperty(DFS_DATANODE_PEER_STATS_ENABLED_KEY, "true"); @@ -630,15 +614,14 @@ public void testSlowPeerParameters() throws Exception { for (String parameter : slowPeersParameters) { dn.reconfigureProperty(parameter, null); } - assertEquals(String.format("expect %s is not configured", - DFS_DATANODE_MIN_OUTLIER_DETECTION_NODES_KEY), null, - dn.getConf().get(DFS_DATANODE_MIN_OUTLIER_DETECTION_NODES_KEY)); - assertEquals(String.format("expect %s is not configured", - DFS_DATANODE_SLOWPEER_LOW_THRESHOLD_MS_KEY), null, - dn.getConf().get(DFS_DATANODE_SLOWPEER_LOW_THRESHOLD_MS_KEY)); - assertEquals(String.format("expect %s is not configured", - DFS_DATANODE_PEER_METRICS_MIN_OUTLIER_DETECTION_SAMPLES_KEY), null, - dn.getConf().get(DFS_DATANODE_PEER_METRICS_MIN_OUTLIER_DETECTION_SAMPLES_KEY)); + assertEquals(null, dn.getConf().get(DFS_DATANODE_MIN_OUTLIER_DETECTION_NODES_KEY), String + .format("expect %s is not configured", DFS_DATANODE_MIN_OUTLIER_DETECTION_NODES_KEY)); + assertEquals(null, dn.getConf().get(DFS_DATANODE_SLOWPEER_LOW_THRESHOLD_MS_KEY), + String.format("expect %s is not configured", DFS_DATANODE_SLOWPEER_LOW_THRESHOLD_MS_KEY)); + assertEquals(null, + dn.getConf().get(DFS_DATANODE_PEER_METRICS_MIN_OUTLIER_DETECTION_SAMPLES_KEY), + String.format("expect %s is not configured", + DFS_DATANODE_PEER_METRICS_MIN_OUTLIER_DETECTION_SAMPLES_KEY)); assertEquals(dn.getPeerMetrics().getSlowNodeDetector().getMinOutlierDetectionNodes(), DFS_DATANODE_MIN_OUTLIER_DETECTION_NODES_DEFAULT); assertEquals(dn.getPeerMetrics().getSlowNodeDetector().getLowThresholdMs(), @@ -659,15 +642,15 @@ public void testSlowDiskParameters() throws ReconfigurationException, IOExceptio try { dn.reconfigureProperty(DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY, "text"); } catch (ReconfigurationException expected) { - assertTrue("expecting NumberFormatException", - expected.getCause() instanceof NumberFormatException); + assertTrue(expected.getCause() instanceof NumberFormatException, + "expecting NumberFormatException"); } try { dn.reconfigureProperty(DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY, "text"); } catch (ReconfigurationException expected) { - assertTrue("expecting NumberFormatException", - expected.getCause() instanceof NumberFormatException); + assertTrue(expected.getCause() instanceof NumberFormatException, + "expecting NumberFormatException"); } // Enable disk stats, make DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY > 0. @@ -677,16 +660,16 @@ public void testSlowDiskParameters() throws ReconfigurationException, IOExceptio dn.reconfigureProperty(parameter, "text"); fail("ReconfigurationException expected"); } catch (ReconfigurationException expected) { - assertTrue("expecting NumberFormatException", - expected.getCause() instanceof NumberFormatException); + assertTrue(expected.getCause() instanceof NumberFormatException, + "expecting NumberFormatException"); } try { dn.reconfigureProperty(parameter, String.valueOf(-1)); fail("ReconfigurationException expected"); } catch (ReconfigurationException expected) { - assertTrue("expecting IllegalArgumentException", - expected.getCause() instanceof IllegalArgumentException); + assertTrue(expected.getCause() instanceof IllegalArgumentException, + "expecting IllegalArgumentException"); } } @@ -699,9 +682,8 @@ public void testSlowDiskParameters() throws ReconfigurationException, IOExceptio for (BPOfferService bpos : blockPoolManager.getAllNamenodeThreads()) { if (bpos != null) { for (BPServiceActor actor : bpos.getBPServiceActors()) { - assertEquals(String.format("%s has wrong value", - DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY), - 1, actor.getScheduler().getOutliersReportIntervalMs()); + assertEquals(1, actor.getScheduler().getOutliersReportIntervalMs(), + String.format("%s has wrong value", DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY)); } } } @@ -725,39 +707,32 @@ public void testSlowDiskParameters() throws ReconfigurationException, IOExceptio assertEquals((int) ((double) 99 / 100 * Integer.MAX_VALUE), dn.getFileIoProvider().getProfilingEventHook().getSampleRangeMax()); // Assert slowDiskDetector. - assertEquals(99, - dn.getDiskMetrics().getSlowDiskDetector().getMinOutlierDetectionNodes()); - assertEquals(99, - dn.getDiskMetrics().getSlowDiskDetector().getLowThresholdMs()); + assertEquals(99, dn.getDiskMetrics().getSlowDiskDetector().getMinOutlierDetectionNodes()); + assertEquals(99, dn.getDiskMetrics().getSlowDiskDetector().getLowThresholdMs()); // Revert to default and verify. dn.reconfigureProperty(DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY, null); - assertEquals(String.format("expect %s is not configured", - DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY), null, - dn.getConf().get(DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY)); + assertEquals(null, dn.getConf().get(DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY), + String.format("expect %s is not configured", DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY)); dn.reconfigureProperty(DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY, null); - assertEquals(String.format("expect %s is not configured", - DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY), null, - dn.getConf().get(DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY)); + assertEquals(null, dn.getConf().get(DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY), + String.format("expect %s is not configured", + DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY)); assertFalse(dn.getFileIoProvider().getProfilingEventHook().getDiskStatsEnabled()); - assertEquals(0, - dn.getFileIoProvider().getProfilingEventHook().getSampleRangeMax()); + assertEquals(0, dn.getFileIoProvider().getProfilingEventHook().getSampleRangeMax()); // Enable disk stats, make DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY > 0. dn.reconfigureProperty(DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY, "1"); dn.reconfigureProperty(DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY, null); dn.reconfigureProperty(DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY, null); dn.reconfigureProperty(DFS_DATANODE_MAX_SLOWDISKS_TO_EXCLUDE_KEY, null); - assertEquals(String.format("expect %s is not configured", - DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY), null, - dn.getConf().get(DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY)); - assertEquals(String.format("expect %s is not configured", - DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY), null, - dn.getConf().get(DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY)); - assertEquals(String.format("expect %s is not configured", - DFS_DATANODE_MAX_SLOWDISKS_TO_EXCLUDE_KEY), null, - dn.getConf().get(DFS_DATANODE_MAX_SLOWDISKS_TO_EXCLUDE_KEY)); + assertEquals(null, dn.getConf().get(DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY), String + .format("expect %s is not configured", DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY)); + assertEquals(null, dn.getConf().get(DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY), + String.format("expect %s is not configured", DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY)); + assertEquals(null, dn.getConf().get(DFS_DATANODE_MAX_SLOWDISKS_TO_EXCLUDE_KEY), + String.format("expect %s is not configured", DFS_DATANODE_MAX_SLOWDISKS_TO_EXCLUDE_KEY)); assertEquals(DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_DEFAULT, dn.getDiskMetrics().getSlowDiskDetector().getMinOutlierDetectionNodes()); assertEquals(DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_DEFAULT, @@ -780,16 +755,16 @@ public void testDfsUsageParameters() throws ReconfigurationException { dn.reconfigureProperty(parameter, "text"); fail("ReconfigurationException expected"); } catch (ReconfigurationException expected) { - assertTrue("expecting NumberFormatException", - expected.getCause() instanceof NumberFormatException); + assertTrue(expected.getCause() instanceof NumberFormatException, + "expecting NumberFormatException"); } try { dn.reconfigureProperty(parameter, String.valueOf(-1)); fail("ReconfigurationException expected"); } catch (ReconfigurationException expected) { - assertTrue("expecting IllegalArgumentException", - expected.getCause() instanceof IllegalArgumentException); + assertTrue(expected.getCause() instanceof IllegalArgumentException, + "expecting IllegalArgumentException"); } } @@ -805,8 +780,7 @@ public void testDfsUsageParameters() throws ReconfigurationException { if (dfsUsage instanceof CachingGetSpaceUsed) { assertEquals(99, ((CachingGetSpaceUsed) entry.getValue().getDfsUsage()).getRefreshInterval()); - assertEquals(99, - ((CachingGetSpaceUsed) entry.getValue().getDfsUsage()).getJitter()); + assertEquals(99, ((CachingGetSpaceUsed) entry.getValue().getDfsUsage()).getJitter()); } } } @@ -820,19 +794,17 @@ public void testDfsUsageParameters() throws ReconfigurationException { for (Map.Entry entry : blockPoolSlices.entrySet()) { GetSpaceUsed dfsUsage = entry.getValue().getDfsUsage(); if (dfsUsage instanceof CachingGetSpaceUsed) { - assertEquals(String.format("expect %s is not configured", - FS_DU_INTERVAL_KEY), FS_DU_INTERVAL_DEFAULT, - ((CachingGetSpaceUsed) entry.getValue().getDfsUsage()).getRefreshInterval()); - assertEquals(String.format("expect %s is not configured", - FS_GETSPACEUSED_JITTER_KEY), FS_GETSPACEUSED_JITTER_DEFAULT, - ((CachingGetSpaceUsed) entry.getValue().getDfsUsage()).getJitter()); + assertEquals(FS_DU_INTERVAL_DEFAULT, + ((CachingGetSpaceUsed) entry.getValue().getDfsUsage()).getRefreshInterval(), + String.format("expect %s is not configured", FS_DU_INTERVAL_KEY)); + assertEquals(FS_GETSPACEUSED_JITTER_DEFAULT, + ((CachingGetSpaceUsed) entry.getValue().getDfsUsage()).getJitter(), + String.format("expect %s is not configured", FS_GETSPACEUSED_JITTER_KEY)); } - assertEquals(String.format("expect %s is not configured", - FS_DU_INTERVAL_KEY), null, - dn.getConf().get(FS_DU_INTERVAL_KEY)); - assertEquals(String.format("expect %s is not configured", - FS_GETSPACEUSED_JITTER_KEY), null, - dn.getConf().get(FS_GETSPACEUSED_JITTER_KEY)); + assertEquals(null, dn.getConf().get(FS_DU_INTERVAL_KEY), + String.format("expect %s is not configured", FS_DU_INTERVAL_KEY)); + assertEquals(null, dn.getConf().get(FS_GETSPACEUSED_JITTER_KEY), + String.format("expect %s is not configured", FS_GETSPACEUSED_JITTER_KEY)); } } } @@ -880,8 +852,9 @@ public void testDiskBalancerParameters() throws Exception { // Set default value. dn.reconfigureProperty(DFS_DISK_BALANCER_ENABLED, null); - assertEquals(dn.getConf().getBoolean(DFS_DISK_BALANCER_ENABLED, - DFS_DISK_BALANCER_ENABLED_DEFAULT), dn.getDiskBalancer().isDiskBalancerEnabled()); + assertEquals( + dn.getConf().getBoolean(DFS_DISK_BALANCER_ENABLED, DFS_DISK_BALANCER_ENABLED_DEFAULT), + dn.getDiskBalancer().isDiskBalancerEnabled()); // Set DFS_DISK_BALANCER_ENABLED to false. dn.reconfigureProperty(DFS_DISK_BALANCER_ENABLED, "false"); @@ -900,10 +873,12 @@ public void testDiskBalancerParameters() throws Exception { // Set default value. dn.reconfigureProperty(DFS_DISK_BALANCER_PLAN_VALID_INTERVAL, null); - assertEquals(dn.getConf().getTimeDuration(DFS_DISK_BALANCER_PLAN_VALID_INTERVAL, - DFS_DISK_BALANCER_PLAN_VALID_INTERVAL_DEFAULT, TimeUnit.MILLISECONDS), + assertEquals( + dn.getConf().getTimeDuration(DFS_DISK_BALANCER_PLAN_VALID_INTERVAL, + DFS_DISK_BALANCER_PLAN_VALID_INTERVAL_DEFAULT, TimeUnit.MILLISECONDS), dn.getDiskBalancer().getPlanValidityInterval()); - assertEquals(dn.getConf().getTimeDuration(DFS_DISK_BALANCER_PLAN_VALID_INTERVAL, + assertEquals( + dn.getConf().getTimeDuration(DFS_DISK_BALANCER_PLAN_VALID_INTERVAL, DFS_DISK_BALANCER_PLAN_VALID_INTERVAL_DEFAULT, TimeUnit.MILLISECONDS), dn.getDiskBalancer().getPlanValidityIntervalInConfig()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeTcpNoDelay.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeTcpNoDelay.java index ead7baa72c766..2b1040ae58770 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeTcpNoDelay.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeTcpNoDelay.java @@ -18,8 +18,8 @@ package org.apache.hadoop.hdfs.server.datanode; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_RPC_SOCKET_FACTORY_CLASS_DEFAULT_KEY; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,9 +34,9 @@ import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.StandardSocketFactory; -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 javax.net.SocketFactory; import java.io.IOException; @@ -59,12 +59,12 @@ public class TestDataNodeTcpNoDelay { LoggerFactory.getLogger(TestDataNodeTcpNoDelay.class); private static Configuration baseConf; - @BeforeClass + @BeforeAll public static void setUpBeforeClass() throws Exception { baseConf = new HdfsConfiguration(); } - @AfterClass + @AfterAll public static void tearDownAfterClass() throws Exception { } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeTransferSocketSize.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeTransferSocketSize.java index 0e98b86a45429..65ba74899867c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeTransferSocketSize.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeTransferSocketSize.java @@ -17,8 +17,8 @@ */ package org.apache.hadoop.hdfs.server.datanode; -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.util.List; @@ -26,7 +26,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestDataNodeTransferSocketSize { @@ -40,8 +40,8 @@ public void testSpecifiedDataSocketSize() throws Exception { try { List datanodes = cluster.getDataNodes(); DataNode datanode = datanodes.get(0); - assertEquals("Receive buffer size should be 4K", - 4 * 1024, datanode.getXferServer().getPeerServer().getReceiveBufferSize()); + assertEquals(4 * 1024, datanode.getXferServer().getPeerServer().getReceiveBufferSize(), + "Receive buffer size should be 4K"); } finally { if (cluster != null) { cluster.shutdown(); @@ -60,8 +60,8 @@ public void testAutoTuningDataSocketSize() throws Exception { List datanodes = cluster.getDataNodes(); DataNode datanode = datanodes.get(0); assertTrue( - "Receive buffer size should be a default value (determined by kernel)", - datanode.getXferServer().getPeerServer().getReceiveBufferSize() > 0); + datanode.getXferServer().getPeerServer().getReceiveBufferSize() > 0, + "Receive buffer size should be a default value (determined by kernel)"); } finally { if (cluster != null) { cluster.shutdown(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java index 6b5faee984914..f4d66f8c8d001 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java @@ -20,13 +20,12 @@ import static org.apache.hadoop.test.MetricsAsserts.getLongCounter; import static org.apache.hadoop.test.MetricsAsserts.getMetrics; import static org.apache.hadoop.test.PlatformAssumptions.assumeNotWindows; -import static org.hamcrest.core.Is.is; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -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.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; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; @@ -83,11 +82,10 @@ import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.token.Token; 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.junit.jupiter.api.Timeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -96,6 +94,7 @@ /** * Fine-grain testing of block files and locations after volume failure. */ +@Timeout(120) public class TestDataNodeVolumeFailure { private final static Logger LOG = LoggerFactory.getLogger( TestDataNodeVolumeFailure.class); @@ -118,11 +117,7 @@ private class BlockLocs { // block id to BlockLocs final Map block_map = new HashMap (); - // specific the timeout for entire test class - @Rule - public Timeout timeout = new Timeout(120 * 1000); - - @Before + @BeforeEach public void setUp() throws Exception { // bring up a cluster of 2 conf = new HdfsConfiguration(); @@ -138,7 +133,7 @@ public void setUp() throws Exception { dataDir = new File(cluster.getDataDirectory()); } - @After + @AfterEach public void tearDown() throws Exception { if(data_fail != null) { FileUtil.setWritable(data_fail, true); @@ -159,7 +154,8 @@ public void tearDown() throws Exception { * and that we can replicate to both datanodes even after a single volume * failure if the configuration parameter allows this. */ - @Test(timeout = 120000) + @Test + @Timeout(value = 120) public void testVolumeFailure() throws Exception { System.out.println("Data dir: is " + dataDir.getPath()); @@ -242,7 +238,8 @@ public Boolean get() { * NN (HDFS-14333). This is done by using a simulated FsDataset that throws * an exception for a failed volume when the block pool is initialized. */ - @Test(timeout=15000) + @Test + @Timeout(value = 15) public void testDnStartsAfterDiskErrorScanningBlockPool() throws Exception { // Don't use the cluster configured in the setup() method for this test. cluster.shutdown(true); @@ -282,8 +279,9 @@ public void testDnStartsAfterDiskErrorScanningBlockPool() throws Exception { * Test that DataStorage and BlockPoolSliceStorage remove the failed volume * after failure. */ - @Test(timeout=150000) - public void testFailedVolumeBeingRemovedFromDataNode() + @Test + @Timeout(value = 150) + public void testFailedVolumeBeingRemovedFromDataNode() throws Exception { // The test uses DataNodeTestUtils#injectDataDirFailure() to simulate // volume failures which is currently not supported on Windows. @@ -348,7 +346,8 @@ public void testFailedVolumeBeingRemovedFromDataNode() * Test DataNode stops when the number of failed volumes exceeds * dfs.datanode.failed.volumes.tolerated . */ - @Test(timeout=10000) + @Test + @Timeout(value = 10) public void testDataNodeShutdownAfterNumFailedVolumeExceedsTolerated() throws Exception { // The test uses DataNodeTestUtils#injectDataDirFailure() to simulate @@ -396,15 +395,15 @@ public void testVolumeFailureRecoveredByHotSwappingVolume() String dataDirs = dn0Vol2.getPath(); assertThat( dn0.reconfigurePropertyImpl( - DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dataDirs), - is(dn0.getConf().get(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY))); + DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dataDirs)) + .isEqualTo(dn0.getConf().get(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY)); // Fix failure volume dn0Vol1 and remount it back. DataNodeTestUtils.restoreDataDirFromFailure(dn0Vol1); assertThat( dn0.reconfigurePropertyImpl( - DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, oldDataDirs), - is(dn0.getConf().get(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY))); + DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, oldDataDirs)) + .isEqualTo(dn0.getConf().get(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY)); // Fail dn0Vol2. Now since dn0Vol1 has been fixed, DN0 has sufficient // resources, thus it should keep running. @@ -419,7 +418,8 @@ public void testVolumeFailureRecoveredByHotSwappingVolume() * {@link BPOfferService#registrationSucceeded(BPServiceActor, * DatanodeRegistration)}. */ - @Test(timeout=10000) + @Test + @Timeout(value = 10) public void testRefreshDeadLock() throws Exception { CountDownLatch latch = new CountDownLatch(1); DataNodeFaultInjector.set(new DataNodeFaultInjector() { @@ -480,8 +480,8 @@ public void testTolerateVolumeFailuresAfterAddingMoreVolumes() assertThat( dn0.reconfigurePropertyImpl( DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, - oldDataDirs + "," + dn0VolNew.getAbsolutePath()), - is(dn0.getConf().get(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY))); + oldDataDirs + "," + dn0VolNew.getAbsolutePath())) + .isEqualTo(dn0.getConf().get(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY)); // Fail dn0Vol1 first and hot swap it. DataNodeTestUtils.injectDataDirFailure(dn0Vol1); @@ -546,14 +546,15 @@ public Boolean get() { * * We fail a volume by setting the parent directory non-writable. */ - @Test (timeout = 120000) + @Test + @Timeout(value = 120) public void testDataNodeFailToStartWithVolumeFailure() throws Exception { // Method to simulate volume failures is currently not supported on Windows. assumeNotWindows(); failedDir = new File(dataDir, "failedDir"); - assertTrue("Failed to fail a volume by setting it non-writable", - failedDir.mkdir() && failedDir.setReadOnly()); + assertTrue(failedDir.mkdir() && failedDir.setReadOnly(), + "Failed to fail a volume by setting it non-writable"); startNewDataNodeWithDiskFailure(new File(failedDir, "newDir1"), false); } @@ -563,14 +564,15 @@ public void testDataNodeFailToStartWithVolumeFailure() throws Exception { * * We fail a volume by setting the parent directory non-writable. */ - @Test (timeout = 120000) + @Test + @Timeout(value = 120) public void testDNStartAndTolerateOneVolumeFailure() throws Exception { // Method to simulate volume failures is currently not supported on Windows. assumeNotWindows(); failedDir = new File(dataDir, "failedDir"); - assertTrue("Failed to fail a volume by setting it non-writable", - failedDir.mkdir() && failedDir.setReadOnly()); + assertTrue(failedDir.mkdir() && failedDir.setReadOnly(), + "Failed to fail a volume by setting it non-writable"); startNewDataNodeWithDiskFailure(new File(failedDir, "newDir1"), true); } @@ -578,14 +580,15 @@ public void testDNStartAndTolerateOneVolumeFailure() throws Exception { /** * Test if data directory is not readable/writable, DataNode won't start. */ - @Test (timeout = 120000) + @Test + @Timeout(value = 120) public void testDNFailToStartWithDataDirNonWritable() throws Exception { // Method to simulate volume failures is currently not supported on Windows. assumeNotWindows(); final File readOnlyDir = new File(dataDir, "nonWritable"); - assertTrue("Set the data dir permission non-writable", - readOnlyDir.mkdir() && readOnlyDir.setReadOnly()); + assertTrue(readOnlyDir.mkdir() && readOnlyDir.setReadOnly(), + "Set the data dir permission non-writable"); startNewDataNodeWithDiskFailure(new File(readOnlyDir, "newDir1"), false); } @@ -594,14 +597,15 @@ public void testDNFailToStartWithDataDirNonWritable() throws Exception { * DataNode will start and tolerate one non-writable data directory * according to config. */ - @Test (timeout = 120000) + @Test + @Timeout(value = 120) public void testDNStartAndTolerateOneDataDirNonWritable() throws Exception { // Method to simulate volume failures is currently not supported on Windows. assumeNotWindows(); final File readOnlyDir = new File(dataDir, "nonWritable"); - assertTrue("Set the data dir permission non-writable", - readOnlyDir.mkdir() && readOnlyDir.setReadOnly()); + assertTrue(readOnlyDir.mkdir() && readOnlyDir.setReadOnly(), + "Set the data dir permission non-writable"); startNewDataNodeWithDiskFailure(new File(readOnlyDir, "newDir1"), true); } @@ -624,9 +628,9 @@ private void startNewDataNodeWithDiskFailure(File badDataDir, try { cluster.startDataNodes(newConf, 1, false, null, null); - assertTrue("Failed to get expected IOException", tolerated); + assertTrue(tolerated, "Failed to get expected IOException"); } catch (IOException ioe) { - assertFalse("Unexpected IOException " + ioe, tolerated); + assertFalse(tolerated, "Unexpected IOException " + ioe); return; } @@ -662,11 +666,9 @@ private void verify(String fn, int fs) throws IOException{ // System.out.println(bid + "->" + bl.num_files + "vs." + bl.num_locs); // number of physical files (1 or 2) should be same as number of datanodes // in the list of the block locations - assertEquals("Num files should match num locations", - bl.num_files, bl.num_locs); + assertEquals(bl.num_files, bl.num_locs, "Num files should match num locations"); } - assertEquals("Num physical blocks should match num stored in the NN", - totalReal, totalNN); + assertEquals(totalReal, totalNN, "Num physical blocks should match num stored in the NN"); // now check the number of under-replicated blocks FSNamesystem fsn = cluster.getNamesystem(); @@ -683,8 +685,7 @@ private void verify(String fn, int fs) throws IOException{ (totalReal + totalRepl) + " vs. all files blocks " + blocks_num*2); // together all the blocks should be equal to all real + all underreplicated - assertEquals("Incorrect total block count", - totalReal + totalRepl, blocks_num * repl); + assertEquals(totalReal + totalRepl, blocks_num * repl, "Incorrect total block count"); } /** @@ -844,7 +845,7 @@ private int countRealBlocks(Map map) { for(File f: res) { String s = f.getName(); // cut off "blk_-" at the beginning and ".meta" at the end - assertNotNull("Block file name should not be null", s); + assertNotNull(s, "Block file name should not be null"); String bid = s.substring(s.indexOf("_")+1, s.lastIndexOf("_")); //System.out.println(ii++ + ". block " + s + "; id=" + bid); BlockLocs val = map.get(bid); @@ -927,7 +928,8 @@ public boolean isSimulated() { /* * Verify the failed volume can be cheched during dn startup */ - @Test(timeout = 120000) + @Test + @Timeout(value = 120) public void testVolumeFailureDuringStartup() throws Exception { LOG.debug("Data dir: is " + dataDir.getPath()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java index 6011d6eeef1a3..91521f9be1e47 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java @@ -20,12 +20,12 @@ import static org.apache.hadoop.test.MetricsAsserts.getLongCounter; import static org.apache.hadoop.test.MetricsAsserts.getMetrics; import static org.apache.hadoop.test.PlatformAssumptions.assumeNotWindows; -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.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.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import java.io.File; import java.lang.management.ManagementFactory; @@ -37,6 +37,7 @@ import javax.management.MBeanServer; import javax.management.ObjectName; +import org.junit.jupiter.api.Timeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -56,16 +57,15 @@ import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.test.GenericTestUtils; import org.slf4j.event.Level; -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.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** * Test reporting of DN volume failure counts and metrics. */ +@Timeout(120) public class TestDataNodeVolumeFailureReporting { private static final Logger LOG = @@ -88,11 +88,7 @@ public class TestDataNodeVolumeFailureReporting { // a datanode to be considered dead by the namenode. final int WAIT_FOR_DEATH = 15000; - // specific the timeout for entire test class - @Rule - public Timeout timeout = new Timeout(120 * 1000); - - @Before + @BeforeEach public void setUp() throws Exception { // These tests use DataNodeTestUtils#injectDataDirFailure() to simulate // volume failures which is currently not supported on Windows. @@ -101,7 +97,7 @@ public void setUp() throws Exception { initCluster(1, 2, 1); } - @After + @AfterEach public void tearDown() throws Exception { IOUtils.cleanupWithLogger(LOG, fs); if (cluster != null) { @@ -156,9 +152,9 @@ public void testSuccessiveVolumeFailures() throws Exception { DFSTestUtil.createFile(fs, file1, 1024, (short)3, 1L); DFSTestUtil.waitReplication(fs, file1, (short)3); ArrayList dns = cluster.getDataNodes(); - assertTrue("DN1 should be up", dns.get(0).isDatanodeUp()); - assertTrue("DN2 should be up", dns.get(1).isDatanodeUp()); - assertTrue("DN3 should be up", dns.get(2).isDatanodeUp()); + assertTrue(dns.get(0).isDatanodeUp(), "DN1 should be up"); + assertTrue(dns.get(1).isDatanodeUp(), "DN2 should be up"); + assertTrue(dns.get(2).isDatanodeUp(), "DN3 should be up"); /* * The metrics should confirm the volume failures. @@ -186,7 +182,7 @@ public void testSuccessiveVolumeFailures() throws Exception { Path file2 = new Path("/test2"); DFSTestUtil.createFile(fs, file2, 1024, (short)3, 1L); DFSTestUtil.waitReplication(fs, file2, (short)3); - assertTrue("DN3 should still be up", dns.get(2).isDatanodeUp()); + assertTrue(dns.get(2).isDatanodeUp(), "DN3 should still be up"); checkFailuresAtDataNode(dns.get(2), 1, true, dn3Vol1.getAbsolutePath()); DataNodeTestUtils.triggerHeartbeat(dns.get(2)); @@ -338,9 +334,9 @@ public void testMultipleVolFailuresOnNode() throws Exception { DFSTestUtil.waitReplication(fs, file2, (short)3); ArrayList dns = cluster.getDataNodes(); - assertTrue("DN1 should be up", dns.get(0).isDatanodeUp()); - assertTrue("DN2 should be up", dns.get(1).isDatanodeUp()); - assertTrue("DN3 should be up", dns.get(2).isDatanodeUp()); + assertTrue(dns.get(0).isDatanodeUp(), "DN1 should be up"); + assertTrue(dns.get(1).isDatanodeUp(), "DN2 should be up"); + assertTrue(dns.get(2).isDatanodeUp(), "DN3 should be up"); checkFailuresAtDataNode(dns.get(0), 1, true, dn1Vol1.getAbsolutePath(), dn1Vol2.getAbsolutePath()); @@ -387,9 +383,9 @@ public void testDataNodeReconfigureWithVolumeFailures() throws Exception { DFSTestUtil.waitReplication(fs, file1, (short)2); ArrayList dns = cluster.getDataNodes(); - assertTrue("DN1 should be up", dns.get(0).isDatanodeUp()); - assertTrue("DN2 should be up", dns.get(1).isDatanodeUp()); - assertTrue("DN3 should be up", dns.get(2).isDatanodeUp()); + assertTrue(dns.get(0).isDatanodeUp(), "DN1 should be up"); + assertTrue(dns.get(1).isDatanodeUp(), "DN2 should be up"); + assertTrue(dns.get(2).isDatanodeUp(), "DN3 should be up"); checkFailuresAtDataNode(dns.get(0), 1, true, dn1Vol1.getAbsolutePath()); checkFailuresAtDataNode(dns.get(1), 1, true, dn2Vol1.getAbsolutePath()); @@ -478,8 +474,7 @@ public void testAutoFormatEmptyDirectory() throws Exception { cluster.waitActive(); ArrayList dns = cluster.getDataNodes(); DataNode dn = dns.get(0); - assertFalse("DataNode should not reformat if VERSION is missing", - currentVersion.exists()); + assertFalse(currentVersion.exists(), "DataNode should not reformat if VERSION is missing"); // Make sure DN's JMX sees the failed volume final String[] expectedFailedVolumes = {dn1Vol1.getAbsolutePath()}; @@ -516,8 +511,7 @@ public void testAutoFormatEmptyBlockPoolDirectory() throws Exception { assertTrue(cluster.restartDataNodes(true)); // the DN should tolerate one volume failure. cluster.waitActive(); - assertFalse("DataNode should not reformat if VERSION is missing", - currentVersion.exists()); + assertFalse(currentVersion.exists(), "DataNode should not reformat if VERSION is missing"); } /** @@ -538,8 +532,7 @@ public void testHotSwapOutFailedVolumeAndReporting() "Hadoop:service=DataNode,name=FSDatasetState-" + dn0.getDatanodeUuid()); int numFailedVolumes = (int) mbs.getAttribute(mxbeanName, "NumFailedVolumes"); - Assert.assertEquals(dn0.getFSDataset().getNumFailedVolumes(), - numFailedVolumes); + Assertions.assertEquals(dn0.getFSDataset().getNumFailedVolumes(), numFailedVolumes); checkFailuresAtDataNode(dn0, 0, false, new String[] {}); // Fail dn0Vol1 first. @@ -548,9 +541,8 @@ public void testHotSwapOutFailedVolumeAndReporting() DataNodeTestUtils.waitForDiskError(dn0, DataNodeTestUtils.getVolume(dn0, dn0Vol1)); numFailedVolumes = (int) mbs.getAttribute(mxbeanName, "NumFailedVolumes"); - Assert.assertEquals(1, numFailedVolumes); - Assert.assertEquals(dn0.getFSDataset().getNumFailedVolumes(), - numFailedVolumes); + Assertions.assertEquals(1, numFailedVolumes); + Assertions.assertEquals(dn0.getFSDataset().getNumFailedVolumes(), numFailedVolumes); checkFailuresAtDataNode(dn0, 1, true, new String[] {dn0Vol1.getAbsolutePath()}); @@ -561,13 +553,12 @@ public void testHotSwapOutFailedVolumeAndReporting() oldDataDirs); fail("Reconfigure with failed disk should throw exception."); } catch (ReconfigurationException e) { - Assert.assertTrue("Reconfigure exception doesn't have expected path!", - e.getCause().getMessage().contains(dn0Vol1.getAbsolutePath())); + Assertions.assertTrue(e.getCause().getMessage().contains(dn0Vol1.getAbsolutePath()), + "Reconfigure exception doesn't have expected path!"); } numFailedVolumes = (int) mbs.getAttribute(mxbeanName, "NumFailedVolumes"); - Assert.assertEquals(1, numFailedVolumes); - Assert.assertEquals(dn0.getFSDataset().getNumFailedVolumes(), - numFailedVolumes); + Assertions.assertEquals(1, numFailedVolumes); + Assertions.assertEquals(dn0.getFSDataset().getNumFailedVolumes(), numFailedVolumes); checkFailuresAtDataNode(dn0, 1, true, new String[] {dn0Vol1.getAbsolutePath()}); @@ -577,9 +568,9 @@ public void testHotSwapOutFailedVolumeAndReporting() dn0.reconfigurePropertyImpl(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dataDirs); numFailedVolumes = (int) mbs.getAttribute(mxbeanName, "NumFailedVolumes"); - Assert.assertEquals(0, numFailedVolumes); - Assert.assertEquals(dn0.getFSDataset().getNumFailedVolumes(), - numFailedVolumes); + Assertions.assertEquals(0, numFailedVolumes); + Assertions.assertEquals(dn0.getFSDataset().getNumFailedVolumes(), + numFailedVolumes); checkFailuresAtDataNode(dn0, 0, true, new String[] {}); // Fix failure volume dn0Vol1 and remount it back. @@ -588,8 +579,8 @@ public void testHotSwapOutFailedVolumeAndReporting() dn0.reconfigurePropertyImpl(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, oldDataDirs); numFailedVolumes = (int) mbs.getAttribute(mxbeanName, "NumFailedVolumes"); - Assert.assertEquals(0, numFailedVolumes); - Assert.assertEquals(dn0.getFSDataset().getNumFailedVolumes(), + Assertions.assertEquals(0, numFailedVolumes); + Assertions.assertEquals(dn0.getFSDataset().getNumFailedVolumes(), numFailedVolumes); checkFailuresAtDataNode(dn0, 0, true, new String[] {}); @@ -599,8 +590,8 @@ public void testHotSwapOutFailedVolumeAndReporting() DataNodeTestUtils.waitForDiskError(dn0, DataNodeTestUtils.getVolume(dn0, dn0Vol2)); numFailedVolumes = (int) mbs.getAttribute(mxbeanName, "NumFailedVolumes"); - Assert.assertEquals(1, numFailedVolumes); - Assert.assertEquals(dn0.getFSDataset().getNumFailedVolumes(), + Assertions.assertEquals(1, numFailedVolumes); + Assertions.assertEquals(dn0.getFSDataset().getNumFailedVolumes(), numFailedVolumes); checkFailuresAtDataNode(dn0, 1, true, new String[] {dn0Vol2.getAbsolutePath()}); @@ -658,9 +649,9 @@ private void checkFailuresAtDataNode(DataNode dn, LOG.info(strBuilder.toString()); final long actualVolumeFailures = getLongCounter("VolumeFailures", getMetrics(dn.getMetrics().name())); - assertTrue("Actual async detected volume failures should be greater or " + - "equal than " + expectedFailedVolumes, - actualVolumeFailures >= expectedVolumeFailuresCounter); + assertTrue(actualVolumeFailures >= expectedVolumeFailuresCounter, + "Actual async detected volume failures should be greater or " + "equal than " + + expectedFailedVolumes); assertEquals(expectedFailedVolumes.length, fsd.getNumFailedVolumes()); assertArrayEquals(expectedFailedVolumes, convertToAbsolutePaths(fsd.getFailedStorageLocations())); @@ -696,13 +687,11 @@ private void checkFailuresAtNameNode(DatanodeManager dm, DataNode dn, VolumeFailureSummary volumeFailureSummary = dd.getVolumeFailureSummary(); if (expectedFailedVolumes.length > 0) { assertArrayEquals(expectedFailedVolumes, - convertToAbsolutePaths(volumeFailureSummary - .getFailedStorageLocations())); + convertToAbsolutePaths(volumeFailureSummary.getFailedStorageLocations())); assertTrue(volumeFailureSummary.getLastVolumeFailureDate() > 0); long expectedCapacityLost = getExpectedCapacityLost(expectCapacityKnown, expectedFailedVolumes.length); - assertEquals(expectedCapacityLost, - volumeFailureSummary.getEstimatedCapacityLostTotal()); + assertEquals(expectedCapacityLost, volumeFailureSummary.getEstimatedCapacityLostTotal()); } else { assertNull(volumeFailureSummary); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataSetLockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataSetLockManager.java index 6cb12d2681f82..81f6020088965 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataSetLockManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataSetLockManager.java @@ -19,21 +19,23 @@ import org.apache.hadoop.hdfs.server.common.AutoCloseDataSetLock; import org.apache.hadoop.hdfs.server.common.DataNodeLockManager.LockLevel; -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 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; public class TestDataSetLockManager { private DataSetLockManager manager; - @Before + @BeforeEach public void init() { manager = new DataSetLockManager(); } - @Test(timeout = 5000) + @Test + @Timeout(value = 5) public void testBaseFunc() { manager.addLock(LockLevel.BLOCK_POOl, "BPtest"); manager.addLock(LockLevel.VOLUME, "BPtest", "Volumetest"); @@ -80,7 +82,8 @@ public void testBaseFunc() { assertEquals(lastException.getMessage(), "lock Leak"); } - @Test(timeout = 5000) + @Test + @Timeout(value = 5) public void testAcquireWriteLockError() throws InterruptedException { Thread t = new Thread(() -> { manager.readLock(LockLevel.BLOCK_POOl, "test"); @@ -93,7 +96,8 @@ public void testAcquireWriteLockError() throws InterruptedException { assertEquals(lastException.getMessage(), "lock Leak"); } - @Test(timeout = 5000) + @Test + @Timeout(value = 5) public void testLockLeakCheck() { manager.writeLock(LockLevel.BLOCK_POOl, "test"); manager.lockLeakCheck(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataStorage.java index f82462a384f39..fd9a43bb389fc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataStorage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataStorage.java @@ -26,9 +26,9 @@ import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory; import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo; import org.apache.hadoop.test.GenericTestUtils; -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.mockito.Mockito; import java.io.File; @@ -37,9 +37,9 @@ import java.util.ArrayList; import java.util.List; -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; public class TestDataStorage { private final static String DEFAULT_BPID = "bp-0"; @@ -55,18 +55,18 @@ public class TestDataStorage { private NamespaceInfo nsInfo; private DataStorage storage; - @Before + @BeforeEach public void setUp() throws IOException { Configuration conf = new HdfsConfiguration(); storage = new DataStorage(); nsInfo = new NamespaceInfo(0, CLUSTER_ID, DEFAULT_BPID, CTIME, BUILD_VERSION, SOFTWARE_VERSION); FileUtil.fullyDelete(TEST_DIR); - assertTrue("Failed to make test dir.", TEST_DIR.mkdirs()); + assertTrue(TEST_DIR.mkdirs(), "Failed to make test dir."); Mockito.when(mockDN.getConf()).thenReturn(conf); } - @After + @AfterEach public void tearDown() throws IOException { storage.unlockAll(); FileUtil.fullyDelete(TEST_DIR); @@ -206,7 +206,7 @@ public void testMissingVersion() throws IOException, // create a fake directory under current/ File currentDir = new File(sd.getCurrentDir(), "BP-787466439-172.26.24.43-1462305406642"); - assertTrue("unable to mkdir " + currentDir.getName(), currentDir.mkdirs()); + assertTrue(currentDir.mkdirs(), "unable to mkdir " + currentDir.getName()); // Add volumes for multiple namespaces. List namespaceInfos = createNamespaceInfos(numNamespace); @@ -215,7 +215,7 @@ public void testMissingVersion() throws IOException, } // It should not format the directory because VERSION is missing. - assertTrue("Storage directory was formatted", currentDir.exists()); + assertTrue(currentDir.exists(), "Storage directory was formatted"); } @Test diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataTransferThrottler.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataTransferThrottler.java index a3e4222d75b66..bdcfef81d02e5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataTransferThrottler.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataTransferThrottler.java @@ -22,12 +22,12 @@ import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import static org.apache.hadoop.util.Time.monotonicNow; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_READ_BANDWIDTHPERSEC_KEY; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Tests throttle the data transfers related functions. @@ -54,10 +54,10 @@ public void testReadDataTransferThrottler() throws Exception { DataNode dataNode = cluster.getDataNodes().get(0); // DataXceiverServer#readThrottler is null if // dfs.datanode.data.read.bandwidthPerSec default value is 0. - Assert.assertNull(dataNode.xserver.getReadThrottler()); + Assertions.assertNull(dataNode.xserver.getReadThrottler()); // Read file. - Assert.assertEquals(fileLength, DFSTestUtil.readFileAsBytes(fs, file).length); + Assertions.assertEquals(fileLength, DFSTestUtil.readFileAsBytes(fs, file).length); // Set dfs.datanode.data.read.bandwidthPerSec. long bandwidthPerSec = 1024 * 1024 * 8; @@ -67,11 +67,11 @@ public void testReadDataTransferThrottler() throws Exception { cluster.stopDataNode(0); cluster.startDataNodes(conf, 1, true, null, null); dataNode = cluster.getDataNodes().get(0); - Assert.assertEquals(bandwidthPerSec, dataNode.xserver.getReadThrottler().getBandwidth()); + Assertions.assertEquals(bandwidthPerSec, dataNode.xserver.getReadThrottler().getBandwidth()); // Read file with throttler. long start = monotonicNow(); - Assert.assertEquals(fileLength, DFSTestUtil.readFileAsBytes(fs, file).length); + Assertions.assertEquals(fileLength, DFSTestUtil.readFileAsBytes(fs, file).length); long elapsedTime = monotonicNow() - start; // Ensure throttler is effective, read 1024 * 1024 * 10 * 8 bytes, // should take approximately 10 seconds (1024 * 1024 * 8 bytes per second). diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataXceiverBackwardsCompat.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataXceiverBackwardsCompat.java index 43b149a905819..c6b5592c3c01d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataXceiverBackwardsCompat.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataXceiverBackwardsCompat.java @@ -35,9 +35,8 @@ import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.DataChecksum; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.Timeout; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.mockito.ArgumentCaptor; import java.io.ByteArrayOutputStream; @@ -49,7 +48,7 @@ import java.net.Socket; import java.util.UUID; -import static org.junit.Assert.fail; +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.anyLong; @@ -62,9 +61,8 @@ * Mock-based unit test to verify that DataXceiver does not fail when no * storageId or targetStorageTypes are passed - as is the case in Hadoop 2.x. */ +@Timeout(60) public class TestDataXceiverBackwardsCompat { - @Rule - public Timeout timeout = new Timeout(60000); private void failWithException(String message, Exception exception) { ByteArrayOutputStream buffer = new ByteArrayOutputStream(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java index 905cc2a1ecc19..d118c7e50c231 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java @@ -50,10 +50,11 @@ import org.apache.hadoop.hdfs.server.protocol.RegisterCommand; import org.apache.hadoop.test.GenericTestUtils; import org.slf4j.event.Level; -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.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -85,7 +86,7 @@ public class TestDatanodeProtocolRetryPolicy { * Starts an instance of DataNode * @throws IOException */ - @Before + @BeforeEach public void startUp() throws IOException, URISyntaxException { tearDownDone = false; conf = new HdfsConfiguration(); @@ -107,7 +108,7 @@ public void startUp() throws IOException, URISyntaxException { * Cleans the resources and closes the instance of datanode * @throws IOException if an error occurred */ - @After + @AfterEach public void tearDown() throws IOException { if (!tearDownDone && dn != null) { try { @@ -117,8 +118,8 @@ public void tearDown() throws IOException { } finally { File dir = new File(DATA_DIR); if (dir.exists()) - Assert.assertTrue( - "Cannot delete data-node dirs", FileUtil.fullyDelete(dir)); + Assertions.assertTrue(FileUtil.fullyDelete(dir), + "Cannot delete data-node dirs"); } tearDownDone = true; } @@ -154,7 +155,8 @@ public Boolean get() { * 6. DN retries. * 7. DatanodeProtocol.registerDatanode succeeds. */ - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testDatanodeRegistrationRetry() throws Exception { final DatanodeProtocolClientSideTranslatorPB namenode = mock(DatanodeProtocolClientSideTranslatorPB.class); @@ -222,7 +224,7 @@ public HeartbeatResponse answer(InvocationOnMock invocation) @Override DatanodeProtocolClientSideTranslatorPB connectToNN( InetSocketAddress nnAddr) throws IOException { - Assert.assertEquals(NN_ADDR, nnAddr); + Assertions.assertEquals(NN_ADDR, nnAddr); return namenode; } }; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java index 772452decc0c2..52905d833bf62 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java @@ -19,8 +19,8 @@ package org.apache.hadoop.hdfs.server.datanode; import static org.apache.hadoop.test.LambdaTestUtils.intercept; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -43,9 +43,9 @@ import org.apache.hadoop.util.Lists; import org.apache.hadoop.util.VersionInfo; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -61,7 +61,7 @@ public class TestDatanodeRegister { NamespaceInfo fakeNsInfo; DNConf mockDnConf; - @Before + @BeforeEach public void setUp() throws IOException { mockDnConf = mock(DNConf.class); doReturn(VersionInfo.getVersion()).when(mockDnConf).getMinimumNameNodeVersion(); @@ -91,8 +91,7 @@ public void setUp() throws IOException { @Test public void testSoftwareVersionDifferences() throws Exception { // We expect no exception to be thrown when the software versions match. - assertEquals(VersionInfo.getVersion(), - actor.retrieveNamespaceInfo().getSoftwareVersion()); + assertEquals(VersionInfo.getVersion(), actor.retrieveNamespaceInfo().getSoftwareVersion()); // We expect no exception to be thrown when the min NN version is below the // reported NN version. @@ -158,8 +157,7 @@ public void testDNShutdwonBeforeRegister() throws Exception { localActor.stop(); localActor.register(nsInfo); } catch (IOException e) { - Assert.assertEquals("DN shut down before block pool registered", - e.getMessage()); + Assertions.assertEquals("DN shut down before block pool registered", e.getMessage()); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeStartupOptions.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeStartupOptions.java index 1a868fe0c2a5d..a8454ddd8d850 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeStartupOptions.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeStartupOptions.java @@ -21,11 +21,11 @@ import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; -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 static org.hamcrest.core.Is.is; -import static org.junit.Assert.assertThat; +import static org.assertj.core.api.Assertions.assertThat; /** * This test verifies DataNode command line processing. @@ -58,10 +58,10 @@ private static void checkExpected(boolean expectSuccess, boolean returnValue = DataNode.parseArguments(args, conf); StartupOption option = DataNode.getStartupOption(conf); - assertThat(returnValue, is(expectSuccess)); + assertThat(returnValue).isEqualTo(expectSuccess); if (expectSuccess) { - assertThat(option, is(expectedOption)); + assertThat(option).isEqualTo(expectedOption); } } @@ -69,7 +69,7 @@ private static void checkExpected(boolean expectSuccess, * Reinitialize configuration before every test since DN stores the * parsed StartupOption in the configuration. */ - @Before + @BeforeEach public void initConfiguration() { conf = new HdfsConfiguration(); } @@ -77,7 +77,8 @@ public void initConfiguration() { /** * A few options that should all parse successfully. */ - @Test (timeout=60000) + @Test + @Timeout(value = 60) public void testStartupSuccess() { checkExpected(true, StartupOption.REGULAR, conf); checkExpected(true, StartupOption.REGULAR, conf, "-regular"); @@ -88,7 +89,8 @@ public void testStartupSuccess() { /** * A few options that should all fail to parse. */ - @Test (timeout=60000) + @Test + @Timeout(value = 60) public void testStartupFailure() { checkExpected(false, StartupOption.REGULAR, conf, "unknownoption"); checkExpected(false, StartupOption.REGULAR, conf, "-regular -rollback"); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDeleteBlockPool.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDeleteBlockPool.java index e3b4267e526a9..7c7e9c7f74281 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDeleteBlockPool.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDeleteBlockPool.java @@ -18,9 +18,9 @@ package org.apache.hadoop.hdfs.server.datanode; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -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.fail; import java.io.IOException; @@ -32,7 +32,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSNNTopology; import org.apache.hadoop.hdfs.tools.DFSAdmin; -import org.junit.Test; +import org.junit.jupiter.api.Test; /** * Tests deleteBlockPool functionality. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java index 3392410d1fe0d..55a2862870b36 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java @@ -19,14 +19,13 @@ import static org.apache.hadoop.hdfs.protocol.Block.BLOCK_FILE_PREFIX; import static org.apache.hadoop.util.Shell.getMemlockLimit; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.core.Is.is; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; +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.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.ByteArrayOutputStream; import java.io.File; @@ -88,8 +87,9 @@ import org.apache.hadoop.util.Time; import org.apache.log4j.SimpleLayout; import org.apache.log4j.WriterAppender; -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.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -123,7 +123,7 @@ public Configuration getConfiguration() { return configuration; } - @Before + @BeforeEach public void setup() { LazyPersistTestCase.initCacheManipulator(); } @@ -375,7 +375,8 @@ private void verifyStats(long totalBlocks, int diffsize, long missingMetaFile, assertEquals(duplicateBlocks, stats.duplicateBlocks); } - @Test(timeout = 300000) + @Test + @Timeout(value = 300) public void testRetainBlockOnPersistentStorage() throws Exception { Configuration conf = getConfiguration(); cluster = new MiniDFSCluster.Builder(conf) @@ -418,7 +419,8 @@ public void testRetainBlockOnPersistentStorage() throws Exception { /** * test scan only meta file NOT generate wrong folder structure warn log. */ - @Test(timeout=600000) + @Test + @Timeout(value = 600) public void testScanDirectoryStructureWarn() throws Exception { //add a logger stream to check what has printed to log @@ -464,10 +466,9 @@ public void testScanDirectoryStructureWarn() throws Exception { " for the deleted block"; String dirStructureWarnLog = " found in invalid directory." + " Expected directory: "; - assertFalse("directory check print meaningless warning message", - logContent.contains(dirStructureWarnLog)); - assertTrue("missing block warn log not appear", - logContent.contains(missingBlockWarn)); + assertFalse(logContent.contains(dirStructureWarnLog), + "directory check print meaningless warning message"); + assertTrue(logContent.contains(missingBlockWarn), "missing block warn log not appear"); LOG.info("check pass"); } finally { @@ -480,7 +481,8 @@ public void testScanDirectoryStructureWarn() throws Exception { } } - @Test(timeout = 300000) + @Test + @Timeout(value = 300) public void testDeleteBlockOnTransientStorage() throws Exception { Configuration conf = getConfiguration(); cluster = new MiniDFSCluster.Builder(conf) @@ -522,7 +524,8 @@ public void testDeleteBlockOnTransientStorage() throws Exception { } } - @Test(timeout = 600000) + @Test + @Timeout(value = 600) public void testRegularBlock() throws Exception { Configuration conf = getConfiguration(); cluster = new MiniDFSCluster.Builder(conf).build(); @@ -569,7 +572,8 @@ public void testRegularBlock() throws Exception { } } - @Test(timeout = 600000) + @Test + @Timeout(value = 600) public void testDirectoryScannerDuringUpdateBlockMeta() throws Exception { Configuration conf = getConfiguration(); DataNodeFaultInjector oldDnInjector = DataNodeFaultInjector.get(); @@ -651,7 +655,8 @@ public void delayDiffRecord() { } } - @Test(timeout = 600000) + @Test + @Timeout(value = 600) public void testDirectoryScanner() throws Exception { // Run the test with and without parallel scanning for (int parallelism = 1; parallelism < 3; parallelism++) { @@ -792,10 +797,9 @@ public void runTest(int parallelism) throws Exception { scan(totalBlocks + 1, 0, 0, 0, 0, 0); // Test14: make sure no throttling is happening - assertTrue("Throttle appears to be engaged", - scanner.timeWaitingMs.get() < 10L); - assertTrue("Report complier threads logged no execution time", - scanner.timeRunningMs.get() > 0L); + assertTrue(scanner.timeWaitingMs.get() < 10L, "Throttle appears to be engaged"); + assertTrue(scanner.timeRunningMs.get() > 0L, + "Report complier threads logged no execution time"); scanner.shutdown(); assertFalse(scanner.getRunStatus()); @@ -862,8 +866,8 @@ public void testThrottling() throws Exception { // Waiting should be about 9x running. LOG.info("RATIO: " + ratio); - assertTrue("Throttle is too restrictive", ratio <= 10f); - assertTrue("Throttle is too permissive" + ratio, ratio >= 7f); + assertTrue(ratio <= 10f, "Throttle is too restrictive"); + assertTrue(ratio >= 7f, "Throttle is too permissive" + ratio); // Test with a different limit conf.setInt( @@ -880,8 +884,8 @@ public void testThrottling() throws Exception { // Waiting should be about 4x running. LOG.info("RATIO: " + ratio); - assertTrue("Throttle is too restrictive", ratio <= 4.5f); - assertTrue("Throttle is too permissive", ratio >= 2.75f); + assertTrue(ratio <= 4.5f, "Throttle is too restrictive"); + assertTrue(ratio >= 2.75f, "Throttle is too permissive"); // Test with more than 1 thread conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY, 3); @@ -899,8 +903,8 @@ public void testThrottling() throws Exception { // Waiting should be about 9x running. LOG.info("RATIO: " + ratio); - assertTrue("Throttle is too restrictive", ratio <= 10f); - assertTrue("Throttle is too permissive", ratio >= 7f); + assertTrue(ratio <= 10f, "Throttle is too restrictive"); + assertTrue(ratio >= 7f, "Throttle is too permissive"); // Test with no limit scanner = new DirectoryScanner(fds, getConfiguration()); @@ -909,10 +913,9 @@ public void testThrottling() throws Exception { scanner.shutdown(); assertFalse(scanner.getRunStatus()); - assertTrue("Throttle appears to be engaged", - scanner.timeWaitingMs.get() < 10L); - assertTrue("Report complier threads logged no execution time", - scanner.timeRunningMs.get() > 0L); + assertTrue(scanner.timeWaitingMs.get() < 10L, "Throttle appears to be engaged"); + assertTrue(scanner.timeRunningMs.get() > 0L, + "Report complier threads logged no execution time"); // Test with a 1ms limit. This also tests whether the scanner can be // shutdown cleanly in mid stride. @@ -951,8 +954,8 @@ public void run() { if (finalMs > 0) { LOG.info("Scanner took " + (Time.monotonicNow() - finalMs) + "ms to shutdown"); - assertTrue("Scanner took too long to shutdown", - Time.monotonicNow() - finalMs < 1000L); + assertTrue(Time.monotonicNow() - finalMs < 1000L, + "Scanner took too long to shutdown"); } ratio = @@ -965,9 +968,9 @@ public void run() { // We just want to test that it waits a lot, but it also runs some LOG.info("RATIO: " + ratio); - assertTrue("Throttle is too permissive", ratio > 8); - assertTrue("Report complier threads logged no execution time", - scanner.timeRunningMs.get() > 0L); + assertTrue(ratio > 8, "Throttle is too permissive"); + assertTrue(scanner.timeRunningMs.get() > 0L, + "Report complier threads logged no execution time"); // Test with a 0 limit, i.e. disabled conf.setInt( @@ -979,10 +982,9 @@ public void run() { scanner.shutdown(); assertFalse(scanner.getRunStatus()); - assertTrue("Throttle appears to be engaged", - scanner.timeWaitingMs.get() < 10L); - assertTrue("Report complier threads logged no execution time", - scanner.timeRunningMs.get() > 0L); + assertTrue(scanner.timeWaitingMs.get() < 10L, "Throttle appears to be engaged"); + assertTrue(scanner.timeRunningMs.get() > 0L, + "Report complier threads logged no execution time"); // Test with a 1000 limit, i.e. disabled conf.setInt( @@ -994,10 +996,9 @@ public void run() { scanner.shutdown(); assertFalse(scanner.getRunStatus()); - assertTrue("Throttle appears to be engaged", - scanner.timeWaitingMs.get() < 10L); - assertTrue("Report complier threads logged no execution time", - scanner.timeRunningMs.get() > 0L); + assertTrue(scanner.timeWaitingMs.get() < 10L, "Throttle appears to be engaged"); + assertTrue(scanner.timeRunningMs.get() > 0L, + "Report complier threads logged no execution time"); // Test that throttle works from regular start conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY, 1); @@ -1018,7 +1019,7 @@ public void run() { scanner.shutdown(); assertFalse(scanner.getRunStatus()); - assertTrue("Throttle does not appear to be engaged", count > 0); + assertTrue(count > 0, "Throttle does not appear to be engaged"); } finally { cluster.shutdown(); } @@ -1041,8 +1042,7 @@ private void verifyAddition(long blockId, long genStamp, long size) { // Added block has the same file as the one created by the test File file = new File(getBlockFile(blockId)); - assertEquals(file.getName(), - FsDatasetTestUtil.getFile(fds, bpid, blockId).getName()); + assertEquals(file.getName(), FsDatasetTestUtil.getFile(fds, bpid, blockId).getName()); // Generation stamp is same as that of created file assertEquals(genStamp, replicainfo.getGenerationStamp()); @@ -1067,7 +1067,7 @@ private void verifyStorageType(long blockId, boolean expectTransient) { final ReplicaInfo memBlock; memBlock = FsDatasetTestUtil.fetchReplicaInfo(fds, bpid, blockId); assertNotNull(memBlock); - assertThat(memBlock.getVolume().isTransientStorage(), is(expectTransient)); + assertThat(memBlock.getVolume().isTransientStorage()).isEqualTo(expectTransient); } private static class TestFsVolumeSpi implements FsVolumeSpi { @@ -1219,7 +1219,8 @@ void testScanInfoObject(long blockId) throws Exception { assertNull(scanInfo.getMetaFile()); } - @Test(timeout = 120000) + @Test + @Timeout(value = 120) public void TestScanInfo() throws Exception { testScanInfoObject(123, new File(TEST_VOLUME.getFinalizedDir(BPID_1).getAbsolutePath()), @@ -1242,7 +1243,8 @@ public void TestScanInfo() throws Exception { * Directory scanner shouldn't abort the scan on every directory just because * one had an error. */ - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testExceptionHandlingWhileDirectoryScan() throws Exception { Configuration conf = getConfiguration(); cluster = new MiniDFSCluster.Builder(conf).build(); @@ -1334,7 +1336,8 @@ public void testDirectoryScannerInFederatedCluster() throws Exception { * Test parsing LocalReplica. We should be able to find the replica's path * even if the replica's dir doesn't match the idToBlockDir. */ - @Test(timeout = 3000) + @Test + @Timeout(value = 3) public void testLocalReplicaParsing() { String baseDir = GenericTestUtils.getRandomizedTempPath(); long blkId = getRandomBlockId(); @@ -1369,7 +1372,8 @@ public void testLocalReplicaParsing() { * Test whether can LocalReplica.updateWithReplica() correct the wrongly * recorded replica location. */ - @Test(timeout = 3000) + @Test + @Timeout(value = 3) public void testLocalReplicaUpdateWithReplica() throws Exception { String baseDir = GenericTestUtils.getRandomizedTempPath(); long blkId = getRandomBlockId(); @@ -1393,7 +1397,8 @@ public void testLocalReplicaUpdateWithReplica() throws Exception { assertEquals(realBlkFile, localReplica.getBlockFile()); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testLastDirScannerFinishTimeIsUpdated() throws Exception { Configuration conf = getConfiguration(); conf.setLong(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY, 3L); @@ -1423,7 +1428,8 @@ private void writeFile(FileSystem fs, int numFiles) throws IOException { } } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testNullStorage() throws Exception { DataNodeFaultInjector oldInjector = DataNodeFaultInjector.get(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java index 53be71f9a34c3..47588d4b2e557 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java @@ -17,8 +17,8 @@ */ package org.apache.hadoop.hdfs.server.datanode; -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.File; @@ -53,9 +53,10 @@ import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.DataChecksum; -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.junit.jupiter.api.Timeout; import org.mockito.Mockito; /** @@ -67,7 +68,7 @@ public class TestDiskError { private MiniDFSCluster cluster; private Configuration conf; - @Before + @BeforeEach public void setUp() throws Exception { conf = new HdfsConfiguration(); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 512L); @@ -79,7 +80,7 @@ public void setUp() throws Exception { fs = cluster.getFileSystem(); } - @After + @AfterEach public void tearDown() throws Exception { if (cluster != null) { cluster.shutdown(); @@ -112,8 +113,8 @@ public void testShutdown() throws Exception { File dir2 = MiniDFSCluster.getRbwDir(storageDir, bpid); try { // make the data directory of the first datanode to be readonly - assertTrue("Couldn't chmod local vol", dir1.setReadOnly()); - assertTrue("Couldn't chmod local vol", dir2.setReadOnly()); + assertTrue(dir1.setReadOnly(), "Couldn't chmod local vol"); + assertTrue(dir2.setReadOnly(), "Couldn't chmod local vol"); // create files and make sure that first datanode will be down DataNode dn = cluster.getDataNodes().get(dnIndex); @@ -145,7 +146,7 @@ public void testReplicationError() throws Exception { // get the block belonged to the created file LocatedBlocks blocks = NameNodeAdapter.getBlockLocations( cluster.getNameNode(), fileName.toString(), 0, (long)fileLen); - assertEquals("Should only find 1 block", blocks.locatedBlockCount(), 1); + assertEquals(blocks.locatedBlockCount(), 1, "Should only find 1 block"); LocatedBlock block = blocks.get(0); // bring up a second datanode @@ -207,8 +208,8 @@ public void testLocalDirs() throws Exception { for (FsVolumeSpi vol : volumes) { Path dataDir = new Path(vol.getStorageLocation().getNormalizedUri()); FsPermission actual = localFS.getFileStatus(dataDir).getPermission(); - assertEquals("Permission for dir: " + dataDir + ", is " + actual + - ", while expected is " + expected, expected, actual); + assertEquals(expected, actual, "Permission for dir: " + dataDir + ", is " + actual + + ", while expected is " + expected); } } } @@ -219,7 +220,8 @@ public void testLocalDirs() throws Exception { * Before refactoring the code the above function was not getting called * @throws IOException, InterruptedException */ - @Test(timeout=60000) + @Test + @Timeout(value = 60) public void testcheckDiskError() throws Exception { if(cluster.getDataNodes().size() <= 0) { cluster.startDataNodes(conf, 1, true, null, null); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDnRespectsBlockReportSplitThreshold.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDnRespectsBlockReportSplitThreshold.java index 9d24ccb0c4578..152b935303d81 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDnRespectsBlockReportSplitThreshold.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDnRespectsBlockReportSplitThreshold.java @@ -34,16 +34,16 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_SPLIT_THRESHOLD_KEY; import org.apache.hadoop.test.GenericTestUtils; -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; import org.mockito.ArgumentCaptor; import org.mockito.Mockito; -import static org.hamcrest.core.Is.is; -import static org.junit.Assert.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.times; +import static org.assertj.core.api.Assertions.assertThat; /** * Tests that the DataNode respects @@ -73,7 +73,7 @@ public void startUpCluster(long splitThreshold) throws IOException { bpid = cluster.getNamesystem().getBlockPoolId(); } - @After + @AfterEach public void shutDownCluster() throws IOException { if (cluster != null) { fs.close(); @@ -97,7 +97,7 @@ private void verifyCapturedArguments( List listOfReports = captor.getAllValues(); int numBlocksReported = 0; for (StorageBlockReport[] reports : listOfReports) { - assertThat(reports.length, is(expectedReportsPerCall)); + assertThat(reports.length).isEqualTo(expectedReportsPerCall); for (StorageBlockReport report : reports) { BlockListAsLongs blockList = report.getBlocks(); @@ -112,7 +112,8 @@ private void verifyCapturedArguments( * Test that if splitThreshold is zero, then we always get a separate * call per storage. */ - @Test(timeout=300000) + @Test + @Timeout(value = 300) public void testAlwaysSplit() throws IOException, InterruptedException { startUpCluster(0); NameNode nn = cluster.getNameNode(); @@ -144,7 +145,8 @@ public void testAlwaysSplit() throws IOException, InterruptedException { * Tests the behavior when the count of blocks is exactly one less than * the threshold. */ - @Test(timeout=300000) + @Test + @Timeout(value = 300) public void testCornerCaseUnderThreshold() throws IOException, InterruptedException { startUpCluster(BLOCKS_IN_FILE + 1); NameNode nn = cluster.getNameNode(); @@ -176,7 +178,8 @@ public void testCornerCaseUnderThreshold() throws IOException, InterruptedExcept * Tests the behavior when the count of blocks is exactly equal to the * threshold. */ - @Test(timeout=300000) + @Test + @Timeout(value = 300) public void testCornerCaseAtThreshold() throws IOException, InterruptedException { startUpCluster(BLOCKS_IN_FILE); NameNode nn = cluster.getNameNode(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestHSync.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestHSync.java index 10f371bc3ec8e..46dd31c31e2f1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestHSync.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestHSync.java @@ -39,7 +39,7 @@ import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.io.SequenceFile.Writer; import org.apache.hadoop.io.compress.DefaultCodec; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestHSync { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestHdfsServerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestHdfsServerConstants.java index 0d359d8520ab2..d6f6991c149a8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestHdfsServerConstants.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestHdfsServerConstants.java @@ -17,13 +17,12 @@ */ package org.apache.hadoop.hdfs.server.datanode; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.fail; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; -import org.junit.Test; - +import org.junit.jupiter.api.Test; /** * Test enumerations in TestHdfsServerConstants. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBrVariations.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBrVariations.java index c556d0d3810a5..adac73fffa2d6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBrVariations.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBrVariations.java @@ -20,8 +20,8 @@ import static org.apache.hadoop.test.MetricsAsserts.assertCounter; import static org.apache.hadoop.test.MetricsAsserts.getLongCounter; import static org.apache.hadoop.test.MetricsAsserts.getMetrics; -import static org.hamcrest.core.Is.is; -import static org.junit.Assert.*; +import static org.junit.jupiter.api.Assertions.*; +import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; import java.net.InetSocketAddress; @@ -46,9 +46,10 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus; import org.apache.hadoop.test.GenericTestUtils; -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.junit.jupiter.api.Timeout; /** * This test verifies that incremental block reports from a single DataNode are @@ -89,7 +90,7 @@ public class TestIncrementalBrVariations { GenericTestUtils.setLogLevel(TestIncrementalBrVariations.LOG, Level.TRACE); } - @Before + @BeforeEach public void startUpCluster() throws IOException { conf = new Configuration(); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES).build(); @@ -101,7 +102,7 @@ public void startUpCluster() throws IOException { dn0Reg = dn0.getDNRegistrationForBP(poolId); } - @After + @AfterEach public void shutDownCluster() throws IOException { if (cluster != null) { client.close(); @@ -138,7 +139,7 @@ private LocatedBlocks createFileGetBlocks(String filenamePrefix) throws IOExcept // Get the block list for the file with the block locations. LocatedBlocks blocks = client.getLocatedBlocks( filePath.toString(), 0, BLOCK_SIZE * NUM_BLOCKS); - assertThat(cluster.getNamesystem().getUnderReplicatedBlocks(), is(0L)); + assertThat(cluster.getNamesystem().getUnderReplicatedBlocks()).isEqualTo(0L); return blocks; } @@ -195,8 +196,7 @@ public void verifyIncrementalBlockReports(boolean splitReports) throws IOExcepti // by the NameNode. IBRs are async, make sure the NN processes // all of them. cluster.getNamesystem().getBlockManager().flushBlockOps(); - assertThat(cluster.getNamesystem().getMissingBlocksCount(), - is((long) reports.length)); + assertThat(cluster.getNamesystem().getMissingBlocksCount()).isEqualTo((long) reports.length); } } @@ -206,11 +206,12 @@ public void verifyIncrementalBlockReports(boolean splitReports) throws IOExcepti * @throws IOException * @throws InterruptedException */ - @Test (timeout=60000) + @Test + @Timeout(value = 60) public void testDataNodeDoesNotSplitReports() throws IOException, InterruptedException { LocatedBlocks blocks = createFileGetBlocks(GenericTestUtils.getMethodName()); - assertThat(cluster.getDataNodes().size(), is(1)); + assertThat(cluster.getDataNodes().size()).isEqualTo(1); // Remove all blocks from the DataNode. for (LocatedBlock block : blocks.getLocatedBlocks()) { @@ -242,7 +243,8 @@ private static Block getDummyBlock() { * @throws IOException * @throws InterruptedException */ - @Test (timeout=60000) + @Test + @Timeout(value = 60) public void testNnLearnsNewStorages() throws IOException, InterruptedException { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestLargeBlockReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestLargeBlockReport.java index 1ea52a0cffe40..5926a1a1cac7b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestLargeBlockReport.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestLargeBlockReport.java @@ -19,7 +19,7 @@ import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_MAXIMUM_DATA_LENGTH; import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_MAXIMUM_DATA_LENGTH_DEFAULT; -import static org.junit.Assert.*; +import static org.junit.jupiter.api.Assertions.*; import java.util.ArrayList; import java.util.List; @@ -37,9 +37,9 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport; -import org.junit.After; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; /** @@ -59,13 +59,13 @@ public class TestLargeBlockReport { private final long reportId = 1; private final long fullBrLeaseId = 0; - @BeforeClass + @BeforeAll public static void init() { DFSTestUtil.setNameNodeLogLevel(Level.WARN); FsDatasetImplTestUtils.setFsDatasetImplLogLevel(Level.WARN); } - @After + @AfterEach public void tearDown() { if (cluster != null) { cluster.shutdown(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java index 788e91b02578c..6f18d80783d61 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java @@ -18,9 +18,9 @@ package org.apache.hadoop.hdfs.server.datanode; -import static org.hamcrest.CoreMatchers.*; -import static org.junit.Assert.*; +import static org.junit.jupiter.api.Assertions.*; import static org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State.*; +import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; import java.net.InetSocketAddress; @@ -47,9 +47,9 @@ import org.apache.hadoop.hdfs.server.blockmanagement.NumberReplicas; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.StorageReport; -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.apache.hadoop.thirdparty.com.google.common.collect.Iterables; @@ -92,7 +92,7 @@ public class TestReadOnlySharedStorage { * Setup a {@link MiniDFSCluster}. * Create a block with both {@link State#NORMAL} and {@link State#READ_ONLY_SHARED} replicas. */ - @Before + @BeforeEach public void setup() throws IOException, InterruptedException { conf = new HdfsConfiguration(); SimulatedFSDataset.setFactory(conf); @@ -137,10 +137,10 @@ public void setup() throws IOException, InterruptedException { block = extendedBlock.getLocalBlock(); storedBlock = blockManager.getStoredBlock(block); - assertThat(locatedBlock.getLocations().length, is(1)); + assertThat(locatedBlock.getLocations().length).isEqualTo(1); normalDataNode = locatedBlock.getLocations()[0]; readOnlyDataNode = datanodeManager.getDatanode(cluster.getDataNodes().get(RO_NODE_INDEX).getDatanodeId()); - assertThat(normalDataNode, is(not(readOnlyDataNode))); + assertThat(normalDataNode).isNotEqualTo(readOnlyDataNode); validateNumberReplicas(1); @@ -152,7 +152,7 @@ public void setup() throws IOException, InterruptedException { waitForLocations(2); } - @After + @AfterEach public void tearDown() throws IOException { fs.delete(PATH, false); @@ -167,7 +167,7 @@ private void waitForLocations(int locations) throws IOException, InterruptedExce for (int tries = 0; tries < RETRIES; ) try { LocatedBlock locatedBlock = getLocatedBlock(); - assertThat(locatedBlock.getLocations().length, is(locations)); + assertThat(locatedBlock.getLocations().length).isEqualTo(locations); break; } catch (AssertionError e) { if (++tries < RETRIES) { @@ -180,28 +180,28 @@ private void waitForLocations(int locations) throws IOException, InterruptedExce private LocatedBlock getLocatedBlock() throws IOException { LocatedBlocks locatedBlocks = client.getLocatedBlocks(PATH.toString(), 0, BLOCK_SIZE); - assertThat(locatedBlocks.getLocatedBlocks().size(), is(1)); + assertThat(locatedBlocks.getLocatedBlocks().size()).isEqualTo(1); return Iterables.getOnlyElement(locatedBlocks.getLocatedBlocks()); } private void validateStorageState(StorageReport[] storageReports, DatanodeStorage.State state) { for (StorageReport storageReport : storageReports) { DatanodeStorage storage = storageReport.getStorage(); - assertThat(storage.getState(), is(state)); + assertThat(storage.getState()).isEqualTo(state); } } private void validateNumberReplicas(int expectedReplicas) throws IOException { NumberReplicas numberReplicas = blockManager.countNodes(storedBlock); - assertThat(numberReplicas.liveReplicas(), is(expectedReplicas)); - assertThat(numberReplicas.excessReplicas(), is(0)); - assertThat(numberReplicas.corruptReplicas(), is(0)); - assertThat(numberReplicas.decommissionedAndDecommissioning(), is(0)); - assertThat(numberReplicas.replicasOnStaleNodes(), is(0)); + assertThat(numberReplicas.liveReplicas()).isEqualTo(expectedReplicas); + assertThat(numberReplicas.excessReplicas()).isEqualTo(0); + assertThat(numberReplicas.corruptReplicas()).isEqualTo(0); + assertThat(numberReplicas.decommissionedAndDecommissioning()).isEqualTo(0); + assertThat(numberReplicas.replicasOnStaleNodes()).isEqualTo(0); BlockManagerTestUtil.updateState(blockManager); - assertThat(blockManager.getLowRedundancyBlocksCount(), is(0L)); - assertThat(blockManager.getExcessBlocksCount(), is(0L)); + assertThat(blockManager.getLowRedundancyBlocksCount()).isEqualTo(0L); + assertThat(blockManager.getExcessBlocksCount()).isEqualTo(0L); } /** @@ -235,11 +235,11 @@ public void testNormalReplicaOffline() throws Exception { // The live replica count should now be zero (since the NORMAL replica is offline) NumberReplicas numberReplicas = blockManager.countNodes(storedBlock); - assertThat(numberReplicas.liveReplicas(), is(0)); + assertThat(numberReplicas.liveReplicas()).isEqualTo(0); // The block should be reported as under-replicated BlockManagerTestUtil.updateState(blockManager); - assertThat(blockManager.getLowRedundancyBlocksCount(), is(1L)); + assertThat(blockManager.getLowRedundancyBlocksCount()).isEqualTo(1L); // The BlockManager should be able to heal the replication count back to 1 // by triggering an inter-datanode replication from one of the READ_ONLY_SHARED replicas @@ -248,7 +248,7 @@ public void testNormalReplicaOffline() throws Exception { DFSTestUtil.waitForReplication(cluster, extendedBlock, 1, 1, 0); // There should now be 2 *locations* for the block, and 1 *replica* - assertThat(getLocatedBlock().getLocations().length, is(2)); + assertThat(getLocatedBlock().getLocations().length).isEqualTo(2); validateNumberReplicas(1); } @@ -268,7 +268,7 @@ public void testReadOnlyReplicaCorrupt() throws Exception { // However, the corrupt READ_ONLY_SHARED replica should *not* affect the overall corrupt replicas count NumberReplicas numberReplicas = blockManager.countNodes(storedBlock); - assertThat(numberReplicas.corruptReplicas(), is(0)); + assertThat(numberReplicas.corruptReplicas()).isEqualTo(0); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestRefreshNamenodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestRefreshNamenodes.java index 1f18b7bde0f4f..a8ba069322257 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestRefreshNamenodes.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestRefreshNamenodes.java @@ -18,8 +18,8 @@ package org.apache.hadoop.hdfs.server.datanode; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.IOException; import java.net.InetSocketAddress; @@ -34,7 +34,8 @@ import org.apache.hadoop.hdfs.MiniDFSNNTopology.NNConf; import org.apache.hadoop.hdfs.MiniDFSNNTopology.NSConf; import org.apache.hadoop.util.Sets; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.apache.hadoop.thirdparty.com.google.common.base.Joiner; @@ -97,7 +98,8 @@ public void testRefreshNamenodes() throws IOException { } } - @Test(timeout=10000) + @Test + @Timeout(value = 10) public void testRefreshNameNodeDeadLock() throws Exception { Configuration conf = new HdfsConfiguration(); MiniDFSCluster cluster = null; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java index dde9ad50fa02e..200c30a5abb0e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java @@ -17,10 +17,10 @@ */ package org.apache.hadoop.hdfs.server.datanode; -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.DataInputStream; import java.io.IOException; @@ -39,8 +39,8 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetFactory; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.util.DataChecksum; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** * this class tests the methods of the SimulatedFSDataset. @@ -52,17 +52,13 @@ public class TestSimulatedFSDataset { static final int BLOCK_LENGTH_MULTIPLIER = 79; static final long FIRST_BLK_ID = 1; - private final int storageCount; + private int storageCount = 1; - public TestSimulatedFSDataset() { - this(1); + protected void pTestSimulatedFSDataset(int pStorageCount) { + this.storageCount = pStorageCount; } - protected TestSimulatedFSDataset(int storageCount) { - this.storageCount = storageCount; - } - - @Before + @BeforeEach public void setUp() throws Exception { conf = new HdfsConfiguration(); SimulatedFSDataset.setFactory(conf); @@ -141,7 +137,7 @@ public void testGetMetaData() throws IOException { ExtendedBlock b = new ExtendedBlock(bpid, FIRST_BLK_ID, 5, 0); try { assertTrue(fsdataset.getMetaDataInputStream(b) == null); - assertTrue("Expected an IO exception", false); + assertTrue(false, "Expected an IO exception"); } catch (IOException e) { // ok - as expected } @@ -250,7 +246,7 @@ public void testInjectionNonEmpty() throws IOException { sfsdataset = getSimulatedFSDataset(); sfsdataset.addBlockPool(bpid, conf); injectBlocksFromBlockReport(fsdataset, sfsdataset); - assertTrue("Expected an IO exception", false); + assertTrue(false, "Expected an IO exception"); } catch (IOException e) { // ok - as expected } @@ -261,21 +257,21 @@ public void checkInvalidBlock(ExtendedBlock b) { assertFalse(fsdataset.isValidBlock(b)); try { fsdataset.getLength(b); - assertTrue("Expected an IO exception", false); + assertTrue(false, "Expected an IO exception"); } catch (IOException e) { // ok - as expected } try { fsdataset.getBlockInputStream(b); - assertTrue("Expected an IO exception", false); + assertTrue(false, "Expected an IO exception"); } catch (IOException e) { // ok - as expected } try { fsdataset.finalizeBlock(b, false); - assertTrue("Expected an IO exception", false); + assertTrue(false, "Expected an IO exception"); } catch (IOException e) { // ok - as expected } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDatasetWithMultipleStorages.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDatasetWithMultipleStorages.java index b31ae9857136f..0f7e0070ce84f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDatasetWithMultipleStorages.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDatasetWithMultipleStorages.java @@ -17,12 +17,11 @@ */ package org.apache.hadoop.hdfs.server.datanode; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY; -import static org.junit.Assert.assertEquals; - +import static org.junit.jupiter.api.Assertions.assertEquals; /** * Test that the {@link SimulatedFSDataset} works correctly when configured @@ -32,10 +31,10 @@ public class TestSimulatedFSDatasetWithMultipleStorages extends TestSimulatedFSDataset { public TestSimulatedFSDatasetWithMultipleStorages() { - super(2); + pTestSimulatedFSDataset(2); } - @Before + @BeforeEach public void setUp() throws Exception { super.setUp(); conf.set(DFS_DATANODE_DATA_DIR_KEY, "data1,data2"); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStartSecureDataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStartSecureDataNode.java index 51a843bd42256..8b64ac46a9b6e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStartSecureDataNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStartSecureDataNode.java @@ -17,8 +17,9 @@ package org.apache.hadoop.hdfs.server.datanode; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; +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 org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; @@ -26,11 +27,10 @@ import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; import static org.apache.hadoop.security.SecurityUtilTestHelper.isExternalKdcRunning; +import static org.junit.jupiter.api.Assumptions.assumeTrue; + import org.apache.hadoop.net.NetUtils; -import org.junit.Assume; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; +import org.junit.jupiter.api.Test; import java.net.BindException; import java.net.InetSocketAddress; import java.net.ServerSocket; @@ -53,13 +53,12 @@ * dfs.datanode.keytab.file */ public class TestStartSecureDataNode { - @Rule - public ExpectedException thrown = ExpectedException.none(); + final static private int NUM_OF_DATANODES = 1; private void testExternalKdcRunning() { // Tests are skipped if external KDC is not running. - Assume.assumeTrue(isExternalKdcRunning()); + assumeTrue(isExternalKdcRunning()); } @Test @@ -72,15 +71,14 @@ public void testSecureNameNode() throws Exception { String nnSpnegoPrincipal = System.getProperty("dfs.namenode.kerberos.internal.spnego.principal"); String nnKeyTab = System.getProperty("dfs.namenode.keytab.file"); - assertNotNull("NameNode principal was not specified", nnPrincipal); - assertNotNull("NameNode SPNEGO principal was not specified", - nnSpnegoPrincipal); - assertNotNull("NameNode keytab was not specified", nnKeyTab); + assertNotNull(nnPrincipal, "NameNode principal was not specified"); + assertNotNull(nnSpnegoPrincipal, "NameNode SPNEGO principal was not specified"); + assertNotNull(nnKeyTab, "NameNode keytab was not specified"); String dnPrincipal = System.getProperty("dfs.datanode.kerberos.principal"); String dnKeyTab = System.getProperty("dfs.datanode.keytab.file"); - assertNotNull("DataNode principal was not specified", dnPrincipal); - assertNotNull("DataNode keytab was not specified", dnKeyTab); + assertNotNull(dnPrincipal, "DataNode principal was not specified"); + assertNotNull(dnKeyTab, "DataNode keytab was not specified"); Configuration conf = new HdfsConfiguration(); conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, @@ -124,13 +122,14 @@ public void testStreamingAddrBindException() throws Exception { ServerSocket ss = new ServerSocket(); try { ss.bind(new InetSocketAddress("localhost", 0)); - thrown.expect(BindException.class); - thrown.expectMessage("localhost/127.0.0.1:" + ss.getLocalPort()); + BindException ex = assertThrows(BindException.class, () -> { + Configuration conf = new HdfsConfiguration(); + conf.set(DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY, + "localhost:" + ss.getLocalPort()); - Configuration conf = new HdfsConfiguration(); - conf.set(DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY, - "localhost:" + ss.getLocalPort()); - SecureDataNodeStarter.getSecureResources(conf); + SecureDataNodeStarter.getSecureResources(conf); + }); + assertTrue(ex.getMessage().contains("localhost/127.0.0.1:" + ss.getLocalPort())); } finally { ss.close(); } @@ -148,16 +147,16 @@ public void testWebServerAddrBindException() throws Exception { ServerSocket ss = new ServerSocket(); try { ss.bind(new InetSocketAddress("localhost", 0)); - thrown.expect(BindException.class); - thrown.expectMessage("localhost/127.0.0.1:" + ss.getLocalPort()); - - Configuration conf = new HdfsConfiguration(); - conf.set(DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY, - "localhost:" + NetUtils.getFreeSocketPort()); - conf.set(DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY, - "localhost:" + ss.getLocalPort()); + BindException ex = assertThrows(BindException.class, () -> { + Configuration conf = new HdfsConfiguration(); + conf.set(DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY, + "localhost:" + NetUtils.getFreeSocketPort()); + conf.set(DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY, + "localhost:" + ss.getLocalPort()); - SecureDataNodeStarter.getSecureResources(conf); + SecureDataNodeStarter.getSecureResources(conf); + }); + assertTrue(ex.getMessage().contains("localhost/127.0.0.1:" + ss.getLocalPort())); } finally { ss.close(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java index 9df6209e21a4b..8e72744010abe 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java @@ -34,18 +34,17 @@ import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports; import org.apache.hadoop.hdfs.server.protocol.SlowPeerReports; import org.apache.hadoop.hdfs.server.protocol.StorageReport; -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.mockito.ArgumentCaptor; import org.mockito.Mockito; -import static org.hamcrest.core.Is.is; -import static org.junit.Assert.assertNotSame; -import static org.junit.Assert.assertThat; +import static org.junit.jupiter.api.Assertions.assertNotSame; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyLong; +import static org.assertj.core.api.Assertions.assertThat; public class TestStorageReport { public static final Logger LOG = @@ -59,7 +58,7 @@ public class TestStorageReport { private DistributedFileSystem fs; static String bpid; - @Before + @BeforeEach public void startUpCluster() throws IOException { conf = new HdfsConfiguration(); cluster = new MiniDFSCluster.Builder(conf) @@ -70,7 +69,7 @@ public void startUpCluster() throws IOException { bpid = cluster.getNamesystem().getBlockPoolId(); } - @After + @AfterEach public void shutDownCluster() throws IOException { if (cluster != null) { fs.close(); @@ -115,8 +114,8 @@ public void testStorageReportHasStorageTypeAndState() throws IOException { StorageReport[] reports = captor.getValue(); for (StorageReport report: reports) { - assertThat(report.getStorage().getStorageType(), is(storageType)); - assertThat(report.getStorage().getState(), is(DatanodeStorage.State.NORMAL)); + assertThat(report.getStorage().getStorageType()).isEqualTo(storageType); + assertThat(report.getStorage().getState()).isEqualTo(DatanodeStorage.State.NORMAL); } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestTriggerBlockReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestTriggerBlockReport.java index 14af74df1b83b..c2ecaf1d24686 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestTriggerBlockReport.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestTriggerBlockReport.java @@ -40,7 +40,7 @@ import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus; import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport; import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import java.net.InetSocketAddress;