diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterServiceTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterServiceTestBase.java index 3f2ecff06a1cf..46bb2da26cbb6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterServiceTestBase.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterServiceTestBase.java @@ -52,9 +52,9 @@ import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceUtils; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.ArrayList; import java.util.Collections; @@ -66,7 +66,10 @@ import static java.lang.Thread.sleep; import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB; import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; /** * Base class for Application Master test classes. @@ -122,14 +125,15 @@ private void requestResources(MockAM am, long memory, int vCores, .build()), null); } - @Before + @BeforeEach public void setup() { conf = new YarnConfiguration(); conf.setClass(YarnConfiguration.RM_SCHEDULER, FifoScheduler.class, ResourceScheduler.class); } - @Test(timeout = 3000000) + @Test + @Timeout(value = 3000) public void testRMIdentifierOnContainerAllocation() throws Exception { MockRM rm = new MockRM(conf); rm.start(); @@ -163,12 +167,13 @@ public void testRMIdentifierOnContainerAllocation() throws Exception { ContainerTokenIdentifier tokenId = BuilderUtils.newContainerTokenIdentifier(allocatedContainer .getContainerToken()); - Assert.assertEquals(MockRM.getClusterTimeStamp(), + assertEquals(MockRM.getClusterTimeStamp(), tokenId.getRMIdentifier()); rm.stop(); } - @Test(timeout = 3000000) + @Test + @Timeout(value = 3000) public void testAllocateResponseIdOverflow() throws Exception { MockRM rm = new MockRM(conf); @@ -188,21 +193,22 @@ public void testAllocateResponseIdOverflow() throws Exception { am1.registerAppAttempt(); // Set the last responseId to be Integer.MAX_VALUE - Assert.assertTrue(am1.setApplicationLastResponseId(Integer.MAX_VALUE)); + assertTrue(am1.setApplicationLastResponseId(Integer.MAX_VALUE)); // Both allocate should succeed am1.schedule(); // send allocate with responseId = Integer.MAX_VALUE - Assert.assertEquals(0, am1.getResponseId()); + assertEquals(0, am1.getResponseId()); am1.schedule(); // send allocate with responseId = 0 - Assert.assertEquals(1, am1.getResponseId()); + assertEquals(1, am1.getResponseId()); } finally { rm.stop(); } } - @Test(timeout=600000) + @Test + @Timeout(value = 600) public void testInvalidContainerReleaseRequest() throws Exception { MockRM rm = new MockRM(conf); @@ -232,7 +238,7 @@ public void testInvalidContainerReleaseRequest() throws Exception { alloc1Response = am1.schedule(); } - Assert.assertTrue(alloc1Response.getAllocatedContainers().size() > 0); + assertTrue(alloc1Response.getAllocatedContainers().size() > 0); RMApp app2 = MockRMAppSubmitter.submitWithMemory(1024, rm); @@ -252,14 +258,15 @@ public void testInvalidContainerReleaseRequest() throws Exception { sb.append(cId.toString()); sb.append(" not belonging to this application attempt : "); sb.append(attempt2.getAppAttemptId().toString()); - Assert.assertTrue(e.getMessage().contains(sb.toString())); + assertTrue(e.getMessage().contains(sb.toString())); } } finally { rm.stop(); } } - @Test(timeout=1200000) + @Test + @Timeout(value = 1200) public void testProgressFilter() throws Exception{ MockRM rm = new MockRM(conf); rm.start(); @@ -324,7 +331,8 @@ public void testProgressFilter() throws Exception{ } } - @Test(timeout=1200000) + @Test + @Timeout(value = 1200) public void testFinishApplicationMasterBeforeRegistering() throws Exception { MockRM rm = new MockRM(conf); @@ -342,9 +350,9 @@ public void testFinishApplicationMasterBeforeRegistering() throws Exception { am1.unregisterAppAttempt(req, false); fail("ApplicationMasterNotRegisteredException should be thrown"); } catch (ApplicationMasterNotRegisteredException e) { - Assert.assertNotNull(e); - Assert.assertNotNull(e.getMessage()); - Assert.assertTrue(e.getMessage().contains( + assertNotNull(e); + assertNotNull(e.getMessage()); + assertTrue(e.getMessage().contains( "Application Master is trying to unregister before registering for:" )); } catch (Exception e) { @@ -361,7 +369,8 @@ public void testFinishApplicationMasterBeforeRegistering() throws Exception { } } - @Test(timeout = 1200000) + @Test + @Timeout(value = 1200) public void testRepeatedFinishApplicationMaster() throws Exception { CountingDispatcher dispatcher = new CountingDispatcher(); @@ -387,8 +396,8 @@ protected Dispatcher createDispatcher() { am1.unregisterAppAttempt(req, false); } rm.drainEvents(); - Assert.assertEquals("Expecting only one event", 1, - dispatcher.getEventCount()); + assertEquals(1, dispatcher.getEventCount(), + "Expecting only one event"); } finally { rm.stop(); } @@ -412,7 +421,8 @@ public int getEventCount() { } } - @Test(timeout = 3000000) + @Test + @Timeout(value = 3000) public void testResourceTypes() throws Exception { HashMap> driver = @@ -464,12 +474,13 @@ public void testResourceTypes() throws Exception { EnumSet types = resp.getSchedulerResourceTypes(); LOG.info("types = " + types.toString()); - Assert.assertEquals(expectedValue, types); + assertEquals(expectedValue, types); rm.stop(); } } - @Test(timeout=1200000) + @Test + @Timeout(value = 1200) public void testAllocateAfterUnregister() throws Exception { MockRM rm = new MockRM(conf); rm.start(); @@ -495,10 +506,11 @@ public void testAllocateAfterUnregister() throws Exception { nm1.nodeHeartbeat(true); rm.drainEvents(); alloc1Response = am1.schedule(); - Assert.assertEquals(0, alloc1Response.getAllocatedContainers().size()); + assertEquals(0, alloc1Response.getAllocatedContainers().size()); } - @Test(timeout = 300000) + @Test + @Timeout(value = 300) public void testUpdateTrackingUrl() throws Exception { conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, ResourceScheduler.class); @@ -514,7 +526,7 @@ public void testUpdateTrackingUrl() throws Exception { RMAppAttempt attempt1 = app1.getCurrentAppAttempt(); MockAM am1 = rm.sendAMLaunched(attempt1.getAppAttemptId()); am1.registerAppAttempt(); - Assert.assertEquals("N/A", rm.getRMContext().getRMApps().get( + assertEquals("N/A", rm.getRMContext().getRMApps().get( app1.getApplicationId()).getOriginalTrackingUrl()); AllocateRequestPBImpl allocateRequest = new AllocateRequestPBImpl(); @@ -526,17 +538,18 @@ public void testUpdateTrackingUrl() throws Exception { // wait until RMAppAttemptEventType.STATUS_UPDATE is handled rm.drainEvents(); - Assert.assertEquals(newTrackingUrl, rm.getRMContext().getRMApps().get( + assertEquals(newTrackingUrl, rm.getRMContext().getRMApps().get( app1.getApplicationId()).getOriginalTrackingUrl()); // Send it again am1.allocate(allocateRequest); - Assert.assertEquals(newTrackingUrl, rm.getRMContext().getRMApps().get( + assertEquals(newTrackingUrl, rm.getRMContext().getRMApps().get( app1.getApplicationId()).getOriginalTrackingUrl()); rm.stop(); } - @Test(timeout = 300000) + @Test + @Timeout(value = 300) public void testValidateRequestCapacityAgainstMinMaxAllocation() throws Exception { Map riMap = @@ -582,7 +595,7 @@ public void testValidateRequestCapacityAgainstMinMaxAllocation() } catch (InvalidResourceRequestException e) { exception = true; } - Assert.assertTrue(exception); + assertTrue(exception); exception = false; try { @@ -595,12 +608,13 @@ public void testValidateRequestCapacityAgainstMinMaxAllocation() } catch (InvalidResourceRequestException e) { exception = true; } - Assert.assertTrue(exception); + assertTrue(exception); rm.close(); } - @Test(timeout = 300000) + @Test + @Timeout(value = 300) public void testRequestCapacityMinMaxAllocationForResourceTypes() throws Exception { Map riMap = initializeMandatoryResources(); @@ -637,25 +651,25 @@ public void testRequestCapacityMinMaxAllocationForResourceTypes() RMApp app1 = MockRMAppSubmitter.submit(rm, data); MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm1); - Assert.assertEquals(Resource.newInstance(GB, 1), + assertEquals(Resource.newInstance(GB, 1), getResourceUsageForQueue(rm, getDefaultQueueName())); // Request memory > allowed try { requestResources(am1, 9 * GB, 1, ImmutableMap.of()); - Assert.fail("Should throw InvalidResourceRequestException"); + fail("Should throw InvalidResourceRequestException"); } catch (InvalidResourceRequestException ignored) {} try { // Request vcores > allowed requestResources(am1, GB, 18, ImmutableMap.of()); - Assert.fail("Should throw InvalidResourceRequestException"); + fail("Should throw InvalidResourceRequestException"); } catch (InvalidResourceRequestException ignored) {} try { // Request custom resource 'res_1' > allowed requestResources(am1, GB, 2, ImmutableMap.of(CUSTOM_RES, 100)); - Assert.fail("Should throw InvalidResourceRequestException"); + fail("Should throw InvalidResourceRequestException"); } catch (InvalidResourceRequestException ignored) {} rm.close(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java index 6c24a0563aff8..1c1e5466bcfac 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java @@ -19,6 +19,9 @@ package org.apache.hadoop.yarn.server.resourcemanager; import static org.apache.hadoop.yarn.server.resourcemanager.MockNM.createMockNodeStatus; +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 org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.conf.Configuration; @@ -100,7 +103,6 @@ import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.YarnVersionInfo; import org.apache.hadoop.yarn.util.resource.ResourceUtils; -import org.junit.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.event.Level; @@ -239,7 +241,7 @@ private void waitForState(ApplicationId appId, EnumSet finalStates) throws InterruptedException { drainEventsImplicitly(); RMApp app = getRMContext().getRMApps().get(appId); - Assert.assertNotNull("app shouldn't be null", app); + assertNotNull(app, "app shouldn't be null"); final int timeoutMsecs = 80 * SECOND; int timeWaiting = 0; while (!finalStates.contains(app.getState())) { @@ -254,8 +256,8 @@ private void waitForState(ApplicationId appId, EnumSet finalStates) } LOG.info("App State is : " + app.getState()); - Assert.assertTrue("App State is not correct (timeout).", - finalStates.contains(app.getState())); + assertTrue(finalStates.contains(app.getState()), + "App State is not correct (timeout)."); } /** @@ -270,7 +272,7 @@ public void waitForState(ApplicationId appId, RMAppState finalState) throws InterruptedException { drainEventsImplicitly(); RMApp app = getRMContext().getRMApps().get(appId); - Assert.assertNotNull("app shouldn't be null", app); + assertNotNull(app, "app shouldn't be null"); final int timeoutMsecs = 80 * SECOND; int timeWaiting = 0; while (!finalState.equals(app.getState())) { @@ -285,8 +287,8 @@ public void waitForState(ApplicationId appId, RMAppState finalState) } LOG.info("App State is : " + app.getState()); - Assert.assertEquals("App State is not correct (timeout).", finalState, - app.getState()); + assertEquals(finalState, app.getState(), + "App State is not correct (timeout)."); } /** @@ -316,7 +318,7 @@ public void waitForState(ApplicationAttemptId attemptId, throws InterruptedException { drainEventsImplicitly(); RMApp app = getRMContext().getRMApps().get(attemptId.getApplicationId()); - Assert.assertNotNull("app shouldn't be null", app); + assertNotNull(app, "app shouldn't be null"); RMAppAttempt attempt = app.getRMAppAttempt(attemptId); MockRM.waitForState(attempt, finalState, timeoutMsecs); } @@ -359,8 +361,8 @@ public static void waitForState(RMAppAttempt attempt, } LOG.info("Attempt State is : " + attempt.getAppAttemptState()); - Assert.assertEquals("Attempt state is not correct (timeout).", finalState, - attempt.getState()); + assertEquals(finalState, attempt.getState(), + "Attempt state is not correct (timeout)."); } public void waitForContainerToComplete(RMAppAttempt attempt, @@ -384,7 +386,7 @@ public void waitForContainerToComplete(RMAppAttempt attempt, public MockAM waitForNewAMToLaunchAndRegister(ApplicationId appId, int attemptSize, MockNM nm) throws Exception { RMApp app = getRMContext().getRMApps().get(appId); - Assert.assertNotNull(app); + assertNotNull(app); int timeWaiting = 0; while (app.getAppAttempts().size() != attemptSize) { if (timeWaiting >= TIMEOUT_MS_FOR_ATTEMPT) { @@ -607,7 +609,7 @@ public void waitForState(NodeId nodeId, NodeState finalState) Thread.sleep(WAIT_MS_PER_LOOP); timeWaiting += WAIT_MS_PER_LOOP; } - Assert.assertNotNull("node shouldn't be null (timedout)", node); + assertNotNull(node, "node shouldn't be null (timedout)"); while (!finalState.equals(node.getState())) { if (timeWaiting >= TIMEOUT_MS_FOR_CONTAINER_AND_NODE) { break; @@ -620,22 +622,22 @@ public void waitForState(NodeId nodeId, NodeState finalState) } LOG.info("Node " + nodeId + " State is : " + node.getState()); - Assert.assertEquals("Node state is not correct (timedout)", finalState, - node.getState()); + assertEquals(finalState, node.getState(), + "Node state is not correct (timedout)"); } public void sendNodeGracefulDecommission( MockNM nm, int timeout) throws Exception { RMNodeImpl node = (RMNodeImpl) getRMContext().getRMNodes().get(nm.getNodeId()); - Assert.assertNotNull("node shouldn't be null", node); + assertNotNull(node, "node shouldn't be null"); node.handle(new RMNodeDecommissioningEvent(nm.getNodeId(), timeout)); } public void sendNodeEvent(MockNM nm, RMNodeEventType event) throws Exception { RMNodeImpl node = (RMNodeImpl) getRMContext().getRMNodes().get(nm.getNodeId()); - Assert.assertNotNull("node shouldn't be null", node); + assertNotNull(node, "node shouldn't be null"); node.handle(new RMNodeEvent(nm.getNodeId(), event)); } @@ -883,9 +885,10 @@ private static void waitForSchedulerAppAttemptAdded( } tick++; } - Assert.assertNotNull("Timed out waiting for SchedulerApplicationAttempt=" + - attemptId + " to be added.", ((AbstractYarnScheduler) - rm.getResourceScheduler()).getApplicationAttempt(attemptId)); + assertNotNull(((AbstractYarnScheduler) + rm.getResourceScheduler()).getApplicationAttempt(attemptId), + "Timed out waiting for SchedulerApplicationAttempt=" + + attemptId + " to be added."); } public static MockAM launchAMWhenAsyncSchedulingEnabled(RMApp app, MockRM rm) @@ -1033,8 +1036,8 @@ public void waitForAppRemovedFromScheduler(ApplicationId appId) Thread.sleep(WAIT_MS_PER_LOOP); timeWaiting += WAIT_MS_PER_LOOP; } - Assert.assertTrue("app is not removed from scheduler (timeout).", - !apps.containsKey(appId)); + assertTrue(!apps.containsKey(appId), + "app is not removed from scheduler (timeout)."); LOG.info("app is removed from scheduler, " + appId); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java index 06c4527e5ba73..68550f94684d4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java @@ -18,6 +18,8 @@ package org.apache.hadoop.yarn.server.resourcemanager; +import static org.junit.jupiter.api.Assertions.assertEquals; + import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; @@ -38,7 +40,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.ResourceLocalizationResponse; import org.apache.hadoop.yarn.api.protocolrecords.RestartContainerResponse; import org.apache.hadoop.yarn.api.protocolrecords.RollbackResponse; -import org.junit.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience.Private; @@ -225,10 +226,10 @@ synchronized public StartContainersResponse startContainers( synchronized public void checkResourceUsage() { LOG.info("Checking resource usage for " + containerManagerAddress); - Assert.assertEquals(available.getMemorySize(), + assertEquals(available.getMemorySize(), resourceManager.getResourceScheduler().getNodeReport( this.nodeId).getAvailableResource().getMemorySize()); - Assert.assertEquals(used.getMemorySize(), + assertEquals(used.getMemorySize(), resourceManager.getResourceScheduler().getNodeReport( this.nodeId).getUsedResource().getMemorySize()); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/QueueACLsTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/QueueACLsTestBase.java index deb6644e2b63e..59c1cc810c111 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/QueueACLsTestBase.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/QueueACLsTestBase.java @@ -18,13 +18,16 @@ package org.apache.hadoop.yarn.server.resourcemanager; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; + import java.io.IOException; import java.util.HashMap; import java.util.Map; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.api.records.QueueACL; -import org.junit.Assert; import org.apache.hadoop.security.authorize.AccessControlList; import org.apache.hadoop.yarn.api.ApplicationClientProtocol; @@ -211,14 +214,12 @@ private void checkAccess(boolean access, String queueName) UserGroupInformation user = UserGroupInformation.getCurrentUser(); String failureMsg = "Wrong %s access to %s queue"; - Assert.assertEquals( - String.format(failureMsg, QueueACL.ADMINISTER_QUEUE, queueName), - access, resourceManager.getResourceScheduler() - .checkAccess(user, QueueACL.ADMINISTER_QUEUE, queueName)); - Assert.assertEquals( - String.format(failureMsg, QueueACL.SUBMIT_APPLICATIONS, queueName), - access, resourceManager.getResourceScheduler() - .checkAccess(user, QueueACL.SUBMIT_APPLICATIONS, queueName)); + assertEquals(access, resourceManager.getResourceScheduler() + .checkAccess(user, QueueACL.ADMINISTER_QUEUE, queueName), + String.format(failureMsg, QueueACL.ADMINISTER_QUEUE, queueName)); + assertEquals(access, resourceManager.getResourceScheduler() + .checkAccess(user, QueueACL.SUBMIT_APPLICATIONS, queueName), + String.format(failureMsg, QueueACL.SUBMIT_APPLICATIONS, queueName)); } private void verifyGetClientAMToken(String submitter, String queueAdmin, @@ -236,7 +237,7 @@ private void verifyGetClientAMToken(String submitter, String queueAdmin, GetApplicationReportResponse adMinUserGetReport = adMinUserClient.getApplicationReport(appReportRequest); - Assert.assertEquals(submitterGetReport.getApplicationReport() + assertEquals(submitterGetReport.getApplicationReport() .getClientToAMToken(), adMinUserGetReport.getApplicationReport() .getClientToAMToken()); } @@ -255,12 +256,12 @@ private void verifyKillAppFailure(String submitter, String killer, // Kill app as the killer try { killerClient.forceKillApplication(finishAppRequest); - Assert.fail("App killing by the enemy should fail!!"); + fail("App killing by the enemy should fail!!"); } catch (YarnException e) { LOG.info("Got exception while killing app as the enemy", e); - Assert.assertTrue(e.getMessage().contains( - "User " + killer + " cannot perform operation MODIFY_APP on " - + applicationId)); + assertTrue(e.getMessage().contains( + "User " + killer + " cannot perform operation MODIFY_APP on " + + applicationId)); } getRMClientForUser(submitter).forceKillApplication(finishAppRequest); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ReservationACLsTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ReservationACLsTestBase.java index a0fe708c77e20..b84ae1f1826a1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ReservationACLsTestBase.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ReservationACLsTestBase.java @@ -18,6 +18,9 @@ package org.apache.hadoop.yarn.server.resourcemanager; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; + import java.io.File; import java.io.IOException; import java.util.Arrays; @@ -58,13 +61,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair .allocationfile.AllocationFileWriter; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; -@RunWith(Parameterized.class) public class ReservationACLsTestBase extends ACLsTestBase { private final int defaultDuration = 600000; @@ -77,28 +77,36 @@ public class ReservationACLsTestBase extends ACLsTestBase { private Configuration configuration; private boolean useFullQueuePath; - public ReservationACLsTestBase(Configuration conf, boolean useFullPath) { + @Override + public void setup() throws InterruptedException, IOException { + super.setup(); + } + + public void initReservationACLsTestBase(Configuration conf, boolean useFullPath) + throws IOException, InterruptedException { configuration = conf; useFullQueuePath = useFullPath; + setup(); } - @After + @AfterEach public void tearDown() { if (resourceManager != null) { resourceManager.stop(); } } - @Parameterized.Parameters public static Collection data() throws IOException { - return Arrays.asList(new Object[][] { - { createCapacitySchedulerConfiguration(), false }, - { createFairSchedulerConfiguration(), true } + return Arrays.asList(new Object[][]{ + {createCapacitySchedulerConfiguration(), true}, + {createFairSchedulerConfiguration(), true} }); } - @Test - public void testApplicationACLs() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testApplicationACLs(Configuration conf, boolean useFullPath) throws Exception { + initReservationACLsTestBase(conf, useFullPath); registerNode("test:1234", 8192, 8); String queueA = !useFullQueuePath? QUEUEA : CapacitySchedulerConfiguration .ROOT + "." + QUEUEA; @@ -258,7 +266,7 @@ private void verifySubmitReservationFailure(String submitter, String try { ReservationId reservationId = createReservation(submitter); submitReservation(submitter, queueName, reservationId); - Assert.fail("Submit reservation by the enemy should fail!"); + fail("Submit reservation by the enemy should fail!"); } catch (YarnException e) { handleAdministerException(e, submitter, queueName, ReservationACL .SUBMIT_RESERVATIONS.name()); @@ -286,7 +294,7 @@ private void verifyListReservationFailure(String lister, try { listReservation(lister, queueName); - Assert.fail("List reservation by the enemy should fail!"); + fail("List reservation by the enemy should fail!"); } catch (YarnException e) { handleAdministerException(e, lister, queueName, ReservationACL .LIST_RESERVATIONS.name()); @@ -316,7 +324,7 @@ private void verifyListReservationByIdFailure(String lister, submitReservation(originalSubmitter, queueName, reservationId); try { listReservationById(lister, reservationId, queueName); - Assert.fail("List reservation by the enemy should fail!"); + fail("List reservation by the enemy should fail!"); } catch (YarnException e) { handleAdministerException(e, lister, queueName, ReservationACL .LIST_RESERVATIONS.name()); @@ -341,7 +349,7 @@ private void verifyDeleteReservationFailure(String killer, try { deleteReservation(killer, reservationId); - Assert.fail("Reservation deletion by the enemy should fail!"); + fail("Reservation deletion by the enemy should fail!"); } catch (YarnException e) { handleAdministerException(e, killer, queueName, ReservationACL .ADMINISTER_RESERVATIONS.name()); @@ -378,7 +386,7 @@ private void verifyUpdateReservationFailure(String updater, ApplicationClientProtocol unauthorizedClient = getRMClientForUser(updater); try { unauthorizedClient.updateReservation(updateRequest); - Assert.fail("Reservation updating by the enemy should fail."); + fail("Reservation updating by the enemy should fail."); } catch (YarnException e) { handleAdministerException(e, updater, queueName, ReservationACL .ADMINISTER_RESERVATIONS.name()); @@ -454,9 +462,9 @@ private void submitReservation(String submitter, private void handleAdministerException(Exception e, String user, String queue, String operation) { LOG.info("Got exception while killing app as the enemy", e); - Assert.assertTrue(e.getMessage().contains("User " + user - + " cannot perform operation " + operation + " on queue " - + queue)); + assertTrue(e.getMessage().contains("User " + user + + " cannot perform operation " + operation + " on queue " + + queue)); } private void registerNode(String host, int memory, int vCores) throws @@ -477,13 +485,13 @@ private void registerNode(String host, int memory, int vCores) throws Thread.sleep(100); } while (attempts-- > 0); if (attempts <= 0) { - Assert.fail("Exhausted attempts in checking if node capacity was " - + "added to the plan"); + fail("Exhausted attempts in checking if node capacity was " + + "added to the plan"); } } catch (Exception e) { e.printStackTrace(); - Assert.fail(e.getMessage()); + fail(e.getMessage()); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java index 79b0cabce107a..686754fa819ad 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java @@ -18,6 +18,11 @@ package org.apache.hadoop.yarn.server.resourcemanager; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; + import java.io.IOException; import java.net.InetSocketAddress; import java.nio.ByteBuffer; @@ -27,6 +32,8 @@ import java.util.HashMap; import java.util.Map; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -75,28 +82,22 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; import org.apache.hadoop.yarn.util.Records; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; + public class TestAMAuthorization { private static final Logger LOG = LoggerFactory.getLogger(TestAMAuthorization.class); - private final Configuration conf; + private Configuration conf; private MockRM rm; // Note : Any test case in ResourceManager package that creates a proxy has // to be run with enabling hadoop.security.token.service.use_ip. And reset // to false at the end of test class. See YARN-5208 - @BeforeClass + @BeforeAll public static void setUp() { Configuration conf = new Configuration(); conf.setBoolean( @@ -104,7 +105,7 @@ public static void setUp() { SecurityUtil.setConfiguration(conf); } - @AfterClass + @AfterAll public static void resetConf() { Configuration conf = new Configuration(); conf.setBoolean( @@ -112,7 +113,6 @@ public static void resetConf() { SecurityUtil.setConfiguration(conf); } - @Parameters public static Collection configs() { Configuration conf = new Configuration(); Configuration confWithSecurity = new Configuration(); @@ -122,12 +122,12 @@ public static Collection configs() { return Arrays.asList(new Object[][] {{ conf }, { confWithSecurity} }); } - public TestAMAuthorization(Configuration conf) { - this.conf = conf; + public void initTestAMAuthorization(Configuration pConf) { + this.conf = pConf; UserGroupInformation.setConfiguration(conf); } - @After + @AfterEach public void tearDown() { if (rm != null) { rm.stop(); @@ -258,8 +258,10 @@ public static Token setupAndReturnAMRMToken( } } - @Test - public void testAuthorizedAccess() throws Exception { + @ParameterizedTest + @MethodSource("configs") + public void testAuthorizedAccess(Configuration pConf) throws Exception { + initTestAMAuthorization(pConf); MyContainerManager containerManager = new MyContainerManager(); rm = new MockRMWithAMS(conf, containerManager); @@ -285,7 +287,7 @@ public void testAuthorizedAccess() throws Exception { LOG.info("Waiting for AM Launch to happen.."); Thread.sleep(1000); } - Assert.assertNotNull(containerManager.containerTokens); + assertNotNull(containerManager.containerTokens); RMAppAttempt attempt = app.getCurrentAppAttempt(); ApplicationAttemptId applicationAttemptId = attempt.getAppAttemptId(); @@ -317,17 +319,18 @@ public ApplicationMasterProtocol run() { .newRecord(RegisterApplicationMasterRequest.class); RegisterApplicationMasterResponse response = client.registerApplicationMaster(request); - Assert.assertNotNull(response.getClientToAMTokenMasterKey()); + assertNotNull(response.getClientToAMTokenMasterKey()); if (UserGroupInformation.isSecurityEnabled()) { - Assert - .assertTrue(response.getClientToAMTokenMasterKey().array().length > 0); + assertTrue(response.getClientToAMTokenMasterKey().array().length > 0); } - Assert.assertEquals("Register response has bad ACLs", "*", - response.getApplicationACLs().get(ApplicationAccessType.VIEW_APP)); + assertEquals("*", response.getApplicationACLs().get(ApplicationAccessType.VIEW_APP), + "Register response has bad ACLs"); } - @Test - public void testUnauthorizedAccess() throws Exception { + @ParameterizedTest + @MethodSource("configs") + public void testUnauthorizedAccess(Configuration pConf) throws Exception { + initTestAMAuthorization(pConf); MyContainerManager containerManager = new MyContainerManager(); rm = new MockRMWithAMS(conf, containerManager); rm.start(); @@ -343,7 +346,7 @@ public void testUnauthorizedAccess() throws Exception { LOG.info("Waiting for AM Launch to happen.."); Thread.sleep(1000); } - Assert.assertNotNull(containerManager.containerTokens); + assertNotNull(containerManager.containerTokens); RMAppAttempt attempt = app.getCurrentAppAttempt(); ApplicationAttemptId applicationAttemptId = attempt.getAppAttemptId(); @@ -373,7 +376,7 @@ public ApplicationMasterProtocol run() { .newRecord(RegisterApplicationMasterRequest.class); try { client.registerApplicationMaster(request); - Assert.fail("Should fail with authorization error"); + fail("Should fail with authorization error"); } catch (Exception e) { if (isCause(AccessControlException.class, e)) { // Because there are no tokens, the request should be rejected as the @@ -385,7 +388,7 @@ public ApplicationMasterProtocol run() { expectedMessage = "SIMPLE authentication is not enabled. Available:[TOKEN]"; } - Assert.assertTrue(e.getCause().getMessage().contains(expectedMessage)); + assertTrue(e.getCause().getMessage().contains(expectedMessage)); } else { throw e; } @@ -441,7 +444,7 @@ private void waitForLaunchedState(RMAppAttempt attempt) + "Current state is " + attempt.getAppAttemptState()); Thread.sleep(1000); } - Assert.assertEquals(attempt.getAppAttemptState(), + assertEquals(attempt.getAppAttemptState(), RMAppAttemptState.LAUNCHED); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java index 50fc8fd67deac..5a9e3cae92780 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java @@ -84,16 +84,16 @@ import org.apache.hadoop.thirdparty.com.google.common.collect.Maps; +import org.junit.jupiter.api.extension.BeforeEachCallback; +import org.junit.jupiter.api.extension.ExtensionContext; +import org.junit.jupiter.api.extension.RegisterExtension; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TestWatcher; -import org.junit.runner.Description; +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.ArgumentCaptor; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -101,6 +101,7 @@ import java.io.IOException; import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; +import java.lang.reflect.Method; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; @@ -118,8 +119,11 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.PREFIX; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePrefixes.getAutoCreatedQueueTemplateConfPrefix; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePrefixes.getQueuePrefix; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertEquals; +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.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; @@ -138,8 +142,9 @@ */ public class TestAppManager extends AppManagerTestBase{ - @Rule - public UseCapacitySchedulerRule shouldUseCs = new UseCapacitySchedulerRule(); + + @RegisterExtension + private UseCapacitySchedulerRule shouldUseCs = new UseCapacitySchedulerRule(); private static final Logger LOG = LoggerFactory.getLogger(TestAppManager.class); @@ -248,7 +253,7 @@ protected void addToCompletedApps(TestRMAppManager appMonitor, RMContext rmConte private QueueInfo mockDefaultQueueInfo; @SuppressWarnings("deprecation") - @Before + @BeforeEach public void setUp() throws IOException { long now = System.currentTimeMillis(); @@ -510,7 +515,7 @@ public void testLegacyAutoCreatedQueuesWithACLTemplates() ((CapacityScheduler) newMockRM.getResourceScheduler()); cs.getCapacitySchedulerQueueManager().createQueue(user2QueuePath); AutoCreatedLeafQueue autoCreatedLeafQueue = (AutoCreatedLeafQueue) cs.getQueue("user2"); - Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedLeafQueue); + assertNotNull(autoCreatedLeafQueue, "Auto Creation of Queue failed"); ManagedParentQueue parentQueue = (ManagedParentQueue) cs.getQueue("parent"); assertEquals(parentQueue, autoCreatedLeafQueue.getParent()); // reinitialize to load the ACLs for the queue @@ -599,12 +604,12 @@ public void testFlexibleAutoCreatedQueuesWithSpecializedACLTemplatesAndDynamicPa cs.getCapacitySchedulerQueueManager().createQueue(new QueuePath("root.parent.user2.user3")); ParentQueue autoCreatedParentQueue = (ParentQueue) cs.getQueue("user2"); - Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedParentQueue); + assertNotNull(autoCreatedParentQueue, "Auto Creation of Queue failed"); ParentQueue parentQueue = (ParentQueue) cs.getQueue("parent"); assertEquals(parentQueue, autoCreatedParentQueue.getParent()); LeafQueue autoCreatedLeafQueue = (LeafQueue) cs.getQueue("user3"); - Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedLeafQueue); + assertNotNull(autoCreatedLeafQueue, "Auto Creation of Queue failed"); assertEquals(autoCreatedParentQueue, autoCreatedLeafQueue.getParent()); // reinitialize to load the ACLs for the queue @@ -722,12 +727,12 @@ private void testFlexibleAQCDWithMixedTemplatesDynamicParentACLScenario( cs.getCapacitySchedulerQueueManager().createQueue(new QueuePath("root.parent.user2.user3")); ParentQueue autoCreatedParentQueue = (ParentQueue) cs.getQueue("user2"); - Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedParentQueue); + assertNotNull(autoCreatedParentQueue, "Auto Creation of Queue failed"); ParentQueue parentQueue = (ParentQueue) cs.getQueue("parent"); assertEquals(parentQueue, autoCreatedParentQueue.getParent()); LeafQueue autoCreatedLeafQueue = (LeafQueue) cs.getQueue("user3"); - Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedLeafQueue); + assertNotNull(autoCreatedLeafQueue, "Auto Creation of Queue failed"); assertEquals(autoCreatedParentQueue, autoCreatedLeafQueue.getParent()); // reinitialize to load the ACLs for the queue @@ -822,7 +827,7 @@ private void testFlexibleAQCLeafOnly( ParentQueue autoCreatedParentQueue = (ParentQueue) cs.getQueue("parent"); LeafQueue autoCreatedLeafQueue = (LeafQueue) cs.getQueue("user2"); - Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedLeafQueue); + assertNotNull(autoCreatedLeafQueue, "Auto Creation of Queue failed"); assertEquals(autoCreatedParentQueue, autoCreatedLeafQueue.getParent()); // reinitialize to load the ACLs for the queue @@ -916,12 +921,12 @@ public void testFlexibleAutoCreatedQueuesWithSpecializedACLTemplatesAndDynamicRo cs.getCapacitySchedulerQueueManager().createQueue(new QueuePath("root.user2.user3")); ParentQueue autoCreatedParentQueue = (ParentQueue) cs.getQueue("user2"); - Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedParentQueue); + assertNotNull(autoCreatedParentQueue, "Auto Creation of Queue failed"); ParentQueue parentQueue = (ParentQueue) cs.getQueue("root"); assertEquals(parentQueue, autoCreatedParentQueue.getParent()); LeafQueue autoCreatedLeafQueue = (LeafQueue) cs.getQueue("user3"); - Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedLeafQueue); + assertNotNull(autoCreatedLeafQueue, "Auto Creation of Queue failed"); assertEquals(autoCreatedParentQueue, autoCreatedLeafQueue.getParent()); // reinitialize to load the ACLs for the queue @@ -994,16 +999,16 @@ public void testFlexibleAutoCreatedQueuesMultiLevelDynamicParentACL() cs.getCapacitySchedulerQueueManager().createQueue(new QueuePath("root.user2.user3.queue")); ParentQueue autoCreatedParentQueue = (ParentQueue) cs.getQueue("user2"); - Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedParentQueue); + assertNotNull(autoCreatedParentQueue, "Auto Creation of Queue failed"); ParentQueue parentQueue = (ParentQueue) cs.getQueue("root"); assertEquals(parentQueue, autoCreatedParentQueue.getParent()); ParentQueue autoCreatedParentQueue2 = (ParentQueue) cs.getQueue("user3"); - Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedParentQueue2); + assertNotNull(autoCreatedParentQueue2, "Auto Creation of Queue failed"); assertEquals(autoCreatedParentQueue, autoCreatedParentQueue2.getParent()); LeafQueue autoCreatedLeafQueue = (LeafQueue) cs.getQueue("queue"); - Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedLeafQueue); + assertNotNull(autoCreatedLeafQueue, "Auto Creation of Queue failed"); assertEquals(autoCreatedParentQueue, autoCreatedParentQueue2.getParent()); // reinitialize to load the ACLs for the queue @@ -1046,7 +1051,7 @@ private static void verifyAppSubmissionFailure(TestRMAppManager appManager, String user) { try { appManager.submitApplication(submission, user); - Assert.fail( + fail( String.format("should fail since %s does not have permission to submit to queue", user)); } catch (YarnException e) { assertTrue(e.getCause() instanceof AccessControlException); @@ -1060,10 +1065,9 @@ private static void verifyAppSubmission(ApplicationSubmissionContext submission, String expectedQueue) throws YarnException { appManager.submitApplication(submission, user); RMApp app = rmContext.getRMApps().get(submission.getApplicationId()); - Assert.assertNotNull("app should not be null", app); - Assert.assertEquals(String.format("the queue should be placed on '%s' queue", expectedQueue), - expectedQueue, - app.getQueue()); + assertNotNull(app, "app should not be null"); + assertEquals(expectedQueue, app.getQueue(), + String.format("the queue should be placed on '%s' queue", expectedQueue)); } private static ApplicationSubmissionContext createAppSubmissionContext(ApplicationId id) { @@ -1078,7 +1082,7 @@ private static ApplicationSubmissionContext createAppSubmissionContext(Applicati return appSubmission; } - @After + @AfterEach public void tearDown() { setAppEventType(RMAppEventType.KILL); ((Service)rmContext.getDispatcher()).stop(); @@ -1096,18 +1100,18 @@ public void testRMAppRetireNone() throws Exception { conf.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, 10); TestRMAppManager appMonitor = new TestRMAppManager(rmContext,conf); - Assert.assertEquals("Number of apps incorrect before checkAppTimeLimit", - 10, rmContext.getRMApps().size()); + assertEquals(10, rmContext.getRMApps().size(), + "Number of apps incorrect before checkAppTimeLimit"); // add them to completed apps list addToCompletedApps(appMonitor, rmContext); // shouldn't have to many apps appMonitor.checkAppNumCompletedLimit(); - Assert.assertEquals("Number of apps incorrect after # completed check", 10, - rmContext.getRMApps().size()); - Assert.assertEquals("Number of completed apps incorrect after check", 10, - appMonitor.getCompletedAppsListSize()); + assertEquals(10, rmContext.getRMApps().size(), + "Number of apps incorrect after # completed check"); + assertEquals(10, appMonitor.getCompletedAppsListSize(), + "Number of completed apps incorrect after check"); verify(rmContext.getStateStore(), never()).removeApplication( isA(RMApp.class)); } @@ -1141,7 +1145,7 @@ public void testQueueSubmitWithNoPermission() throws IOException { if (e instanceof YarnException) { assertTrue(e.getCause() instanceof AccessControlException); } else { - Assert.fail("Yarn exception is expected : " + e.getMessage()); + fail("Yarn exception is expected : " + e.getMessage()); } } finally { mockRM.close(); @@ -1158,18 +1162,18 @@ public void testRMAppRetireSome() throws Exception { conf.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, 3); TestRMAppManager appMonitor = new TestRMAppManager(rmContext, conf); - Assert.assertEquals("Number of apps incorrect before", 10, rmContext - .getRMApps().size()); + assertEquals(10, rmContext + .getRMApps().size(), "Number of apps incorrect before"); // add them to completed apps list addToCompletedApps(appMonitor, rmContext); // shouldn't have to many apps appMonitor.checkAppNumCompletedLimit(); - Assert.assertEquals("Number of apps incorrect after # completed check", 3, - rmContext.getRMApps().size()); - Assert.assertEquals("Number of completed apps incorrect after check", 3, - appMonitor.getCompletedAppsListSize()); + assertEquals(3, rmContext.getRMApps().size(), + "Number of apps incorrect after # completed check"); + assertEquals(3, appMonitor.getCompletedAppsListSize(), + "Number of completed apps incorrect after check"); verify(rmContext.getStateStore(), times(7)).removeApplication( isA(RMApp.class)); } @@ -1188,7 +1192,7 @@ public void testRMAppRetireSomeDifferentStates() throws Exception { // clear out applications map rmContext.getRMApps().clear(); - Assert.assertEquals("map isn't empty", 0, rmContext.getRMApps().size()); + assertEquals(0, rmContext.getRMApps().size(), "map isn't empty"); // 6 applications are in final state, 4 are not in final state. // / set with various finished states @@ -1215,18 +1219,18 @@ public void testRMAppRetireSomeDifferentStates() throws Exception { app = new MockRMApp(9, now - 20000, RMAppState.FAILED); rmContext.getRMApps().put(app.getApplicationId(), app); - Assert.assertEquals("Number of apps incorrect before", 10, rmContext - .getRMApps().size()); + assertEquals(10, rmContext + .getRMApps().size(), "Number of apps incorrect before"); // add them to completed apps list addToCompletedApps(appMonitor, rmContext); // shouldn't have to many apps appMonitor.checkAppNumCompletedLimit(); - Assert.assertEquals("Number of apps incorrect after # completed check", 6, - rmContext.getRMApps().size()); - Assert.assertEquals("Number of completed apps incorrect after check", 2, - appMonitor.getCompletedAppsListSize()); + assertEquals(6, rmContext.getRMApps().size(), + "Number of apps incorrect after # completed check"); + assertEquals(2, appMonitor.getCompletedAppsListSize(), + "Number of completed apps incorrect after check"); // 6 applications in final state, 4 of them are removed verify(rmContext.getStateStore(), times(4)).removeApplication( isA(RMApp.class)); @@ -1239,13 +1243,13 @@ public void testRMAppRetireNullApp() throws Exception { RMContext rmContext = mockRMContext(10, now - 20000); TestRMAppManager appMonitor = new TestRMAppManager(rmContext, new Configuration()); - Assert.assertEquals("Number of apps incorrect before", 10, rmContext - .getRMApps().size()); + assertEquals(10, rmContext + .getRMApps().size(), "Number of apps incorrect before"); appMonitor.finishApplication(null); - Assert.assertEquals("Number of completed apps incorrect after check", 0, - appMonitor.getCompletedAppsListSize()); + assertEquals(0, appMonitor.getCompletedAppsListSize(), + "Number of completed apps incorrect after check"); } @Test @@ -1257,19 +1261,19 @@ public void testRMAppRetireZeroSetting() throws Exception { conf.setInt(YarnConfiguration.RM_STATE_STORE_MAX_COMPLETED_APPLICATIONS, 0); conf.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, 0); TestRMAppManager appMonitor = new TestRMAppManager(rmContext, conf); - Assert.assertEquals("Number of apps incorrect before", 10, rmContext - .getRMApps().size()); + assertEquals(10, rmContext + .getRMApps().size(), "Number of apps incorrect before"); addToCompletedApps(appMonitor, rmContext); - Assert.assertEquals("Number of completed apps incorrect", 10, - appMonitor.getCompletedAppsListSize()); + assertEquals(10, appMonitor.getCompletedAppsListSize(), + "Number of completed apps incorrect"); appMonitor.checkAppNumCompletedLimit(); - Assert.assertEquals("Number of apps incorrect after # completed check", 0, - rmContext.getRMApps().size()); - Assert.assertEquals("Number of completed apps incorrect after check", 0, - appMonitor.getCompletedAppsListSize()); + assertEquals(0, rmContext.getRMApps().size(), + "Number of apps incorrect after # completed check"); + assertEquals(0, appMonitor.getCompletedAppsListSize(), + "Number of completed apps incorrect after check"); verify(rmContext.getStateStore(), times(10)).removeApplication( isA(RMApp.class)); } @@ -1288,19 +1292,19 @@ public void testStateStoreAppLimitLessThanMemoryAppLimit() { TestRMAppManager appMonitor = new TestRMAppManager(rmContext, conf); addToCompletedApps(appMonitor, rmContext); - Assert.assertEquals("Number of completed apps incorrect", allApps, - appMonitor.getCompletedAppsListSize()); + assertEquals(allApps, appMonitor.getCompletedAppsListSize(), + "Number of completed apps incorrect"); appMonitor.checkAppNumCompletedLimit(); - Assert.assertEquals("Number of apps incorrect after # completed check", - maxAppsInMemory, rmContext.getRMApps().size()); - Assert.assertEquals("Number of completed apps incorrect after check", - maxAppsInMemory, appMonitor.getCompletedAppsListSize()); + assertEquals(maxAppsInMemory, rmContext.getRMApps().size(), + "Number of apps incorrect after # completed check"); + assertEquals(maxAppsInMemory, appMonitor.getCompletedAppsListSize(), + "Number of completed apps incorrect after check"); int numRemoveAppsFromStateStore = 10 - maxAppsInStateStore; verify(rmContext.getStateStore(), times(numRemoveAppsFromStateStore)) .removeApplication(isA(RMApp.class)); - Assert.assertEquals(maxAppsInStateStore, + assertEquals(maxAppsInStateStore, appMonitor.getNumberOfCompletedAppsInStateStore()); } @@ -1317,18 +1321,18 @@ public void testStateStoreAppLimitGreaterThanMemoryAppLimit() { TestRMAppManager appMonitor = new TestRMAppManager(rmContext, conf); addToCompletedApps(appMonitor, rmContext); - Assert.assertEquals("Number of completed apps incorrect", allApps, - appMonitor.getCompletedAppsListSize()); + assertEquals(allApps, appMonitor.getCompletedAppsListSize(), + "Number of completed apps incorrect"); appMonitor.checkAppNumCompletedLimit(); int numRemoveApps = allApps - maxAppsInMemory; - Assert.assertEquals("Number of apps incorrect after # completed check", - maxAppsInMemory, rmContext.getRMApps().size()); - Assert.assertEquals("Number of completed apps incorrect after check", - maxAppsInMemory, appMonitor.getCompletedAppsListSize()); + assertEquals(maxAppsInMemory, rmContext.getRMApps().size(), + "Number of apps incorrect after # completed check"); + assertEquals(maxAppsInMemory, appMonitor.getCompletedAppsListSize(), + "Number of completed apps incorrect after check"); verify(rmContext.getStateStore(), times(numRemoveApps)).removeApplication( isA(RMApp.class)); - Assert.assertEquals(maxAppsInMemory, + assertEquals(maxAppsInMemory, appMonitor.getNumberOfCompletedAppsInStateStore()); } @@ -1340,7 +1344,7 @@ protected void setupDispatcher(RMContext rmContext, Configuration conf) { rmContext.getDispatcher().register(RMAppManagerEventType.class, testAppManagerDispatcher); ((Service)rmContext.getDispatcher()).init(conf); ((Service)rmContext.getDispatcher()).start(); - Assert.assertEquals("app event type is wrong before", RMAppEventType.KILL, appEventType); + assertEquals(RMAppEventType.KILL, appEventType, "app event type is wrong before"); } @SuppressWarnings("deprecation") @@ -1360,8 +1364,8 @@ public void testRMAppSubmitAMContainerResourceRequests() throws Exception { asContext.setAMContainerResourceRequests(cloneResourceRequests(reqs)); // getAMContainerResourceRequest uses the first entry of // getAMContainerResourceRequests - Assert.assertEquals(reqs.get(0), asContext.getAMContainerResourceRequest()); - Assert.assertEquals(reqs, asContext.getAMContainerResourceRequests()); + assertEquals(reqs.get(0), asContext.getAMContainerResourceRequest()); + assertEquals(reqs, asContext.getAMContainerResourceRequests()); RMApp app = testRMAppSubmit(); for (ResourceRequest req : reqs) { req.setNodeLabelExpression(RMNodeLabelsManager.NO_LABEL); @@ -1369,7 +1373,7 @@ public void testRMAppSubmitAMContainerResourceRequests() throws Exception { // setAMContainerResourceRequests has priority over // setAMContainerResourceRequest and setResource - Assert.assertEquals(reqs, app.getAMResourceRequests()); + assertEquals(reqs, app.getAMResourceRequests()); } @SuppressWarnings("deprecation") @@ -1384,12 +1388,12 @@ public void testRMAppSubmitAMContainerResourceRequest() throws Exception { asContext.setAMContainerResourceRequest(ResourceRequest.clone(req)); // getAMContainerResourceRequests uses a singleton list of // getAMContainerResourceRequest - Assert.assertEquals(req, asContext.getAMContainerResourceRequest()); - Assert.assertEquals(req, asContext.getAMContainerResourceRequests().get(0)); - Assert.assertEquals(1, asContext.getAMContainerResourceRequests().size()); + assertEquals(req, asContext.getAMContainerResourceRequest()); + assertEquals(req, asContext.getAMContainerResourceRequests().get(0)); + assertEquals(1, asContext.getAMContainerResourceRequests().size()); RMApp app = testRMAppSubmit(); // setAMContainerResourceRequest has priority over setResource - Assert.assertEquals(Collections.singletonList(req), + assertEquals(Collections.singletonList(req), app.getAMResourceRequests()); } @@ -1416,7 +1420,7 @@ public void testRMAppSubmitAMContainerWithNoLabelByRMDefaultAMNodeLabel() throws RMApp app = rmContext.getRMApps().get(appId); waitUntilEventProcessed(); - Assert.assertEquals(defaultAMNodeLabel, + assertEquals(defaultAMNodeLabel, app.getAMResourceRequests().get(0).getNodeLabelExpression()); } @@ -1427,7 +1431,7 @@ public void testRMAppSubmitResource() throws Exception { RMApp app = testRMAppSubmit(); // setResource - Assert.assertEquals(Collections.singletonList( + assertEquals(Collections.singletonList( ResourceRequest.newInstance(RMAppAttemptImpl.AM_CONTAINER_PRIORITY, ResourceRequest.ANY, Resources.createResource(1024), 1, true, "")), @@ -1440,9 +1444,9 @@ public void testRMAppSubmitNoResourceRequests() throws Exception { asContext.setAMContainerResourceRequests(null); try { testRMAppSubmit(); - Assert.fail("Should have failed due to no ResourceRequest"); + fail("Should have failed due to no ResourceRequest"); } catch (InvalidResourceRequestException e) { - Assert.assertEquals( + assertEquals( "Invalid resource request, no resources requested", e.getMessage()); } @@ -1476,7 +1480,7 @@ public void testRMAppSubmitAMContainerResourceRequestsDisagree() req.setNumContainers(1); req.setPriority(Priority.newInstance(0)); } - Assert.assertEquals(reqs, app.getAMResourceRequests()); + assertEquals(reqs, app.getAMResourceRequests()); } @Test @@ -1491,12 +1495,12 @@ public void testRMAppSubmitAMContainerResourceRequestsNoAny() asContext.setAMContainerResourceRequests(cloneResourceRequests(reqs)); // getAMContainerResourceRequest uses the first entry of // getAMContainerResourceRequests - Assert.assertEquals(reqs, asContext.getAMContainerResourceRequests()); + assertEquals(reqs, asContext.getAMContainerResourceRequests()); try { testRMAppSubmit(); - Assert.fail("Should have failed due to missing ANY ResourceRequest"); + fail("Should have failed due to missing ANY ResourceRequest"); } catch (InvalidResourceRequestException e) { - Assert.assertEquals( + assertEquals( "Invalid resource request, no resource request specified with *", e.getMessage()); } @@ -1514,12 +1518,12 @@ public void testRMAppSubmitAMContainerResourceRequestsTwoManyAny() asContext.setAMContainerResourceRequests(cloneResourceRequests(reqs)); // getAMContainerResourceRequest uses the first entry of // getAMContainerResourceRequests - Assert.assertEquals(reqs, asContext.getAMContainerResourceRequests()); + assertEquals(reqs, asContext.getAMContainerResourceRequests()); try { testRMAppSubmit(); - Assert.fail("Should have failed due to too many ANY ResourceRequests"); + fail("Should have failed due to too many ANY ResourceRequests"); } catch (InvalidResourceRequestException e) { - Assert.assertEquals( + assertEquals( "Invalid resource request, only one resource request with * is " + "allowed", e.getMessage()); } @@ -1532,17 +1536,17 @@ private RMApp testRMAppSubmit() throws Exception { private RMApp waitUntilEventProcessed() throws InterruptedException { RMApp app = rmContext.getRMApps().get(appId); - Assert.assertNotNull("app is null", app); - Assert.assertEquals("app id doesn't match", appId, app.getApplicationId()); - Assert.assertEquals("app state doesn't match", RMAppState.NEW, app.getState()); + assertNotNull(app, "app is null"); + assertEquals(appId, app.getApplicationId(), "app id doesn't match"); + assertEquals(RMAppState.NEW, app.getState(), "app state doesn't match"); // wait for event to be processed int timeoutSecs = 0; while ((getAppEventType() == RMAppEventType.KILL) && timeoutSecs++ < 20) { Thread.sleep(1000); } - Assert.assertEquals("app event type sent is wrong", RMAppEventType.START, - getAppEventType()); + assertEquals(RMAppEventType.START, getAppEventType(), + "app event type sent is wrong"); return app; } @@ -1559,25 +1563,25 @@ public void testRMAppSubmitWithInvalidTokens() throws Exception { asContext.getAMContainerSpec().setTokens(securityTokens); try { appMonitor.submitApplication(asContext, "test"); - Assert.fail("Application submission should fail because" + + fail("Application submission should fail because" + " Tokens are invalid."); } catch (YarnException e) { // Exception is expected - assertTrue("The thrown exception is not" + - " java.io.EOFException", - e.getMessage().contains("java.io.EOFException")); + assertTrue(e.getMessage().contains("java.io.EOFException"), + "The thrown exception is not java.io.EOFException"); } int timeoutSecs = 0; while ((getAppEventType() == RMAppEventType.KILL) && timeoutSecs++ < 20) { Thread.sleep(1000); } - Assert.assertEquals("app event type sent is wrong", - RMAppEventType.APP_REJECTED, getAppEventType()); + assertEquals(RMAppEventType.APP_REJECTED, getAppEventType(), + "app event type sent is wrong"); asContext.getAMContainerSpec().setTokens(null); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testRMAppSubmitMaxAppAttempts() throws Exception { int[] globalMaxAppAttempts = new int[] { 10, 1 }; int[] rmAmMaxAttempts = new int[] { 8, 1 }; @@ -1607,8 +1611,8 @@ public void testRMAppSubmitMaxAppAttempts() throws Exception { } appMonitor.submitApplication(asContext, "test"); RMApp app = rmContext.getRMApps().get(appID); - Assert.assertEquals("max application attempts doesn't match", - expectedNums[i][j], app.getMaxAppAttempts()); + assertEquals(expectedNums[i][j], app.getMaxAppAttempts(), + "max application attempts doesn't match"); // wait for event to be processed int timeoutSecs = 0; @@ -1621,34 +1625,34 @@ public void testRMAppSubmitMaxAppAttempts() throws Exception { } } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testRMAppSubmitDuplicateApplicationId() throws Exception { ApplicationId appId = MockApps.newAppID(0); asContext.setApplicationId(appId); RMApp appOrig = rmContext.getRMApps().get(appId); - assertTrue("app name matches " - + "but shouldn't", "testApp1" != appOrig.getName()); + assertTrue("testApp1" != appOrig.getName(), "app name matches " + + "but shouldn't"); // our testApp1 should be rejected and original app with same id should be left in place try { appMonitor.submitApplication(asContext, "test"); - Assert.fail("Exception is expected when applicationId is duplicate."); + fail("Exception is expected when applicationId is duplicate."); } catch (YarnException e) { - assertTrue("The thrown exception is not the expectd one.", - e.getMessage().contains("Cannot add a duplicate!")); + assertTrue(e.getMessage().contains("Cannot add a duplicate!"), + "The thrown exception is not the expectd one."); } // make sure original app didn't get removed RMApp app = rmContext.getRMApps().get(appId); - Assert.assertNotNull("app is null", app); - Assert.assertEquals("app id doesn't match", - appId, app.getApplicationId()); - Assert.assertEquals("app state doesn't match", - RMAppState.FINISHED, app.getState()); + assertNotNull(app, "app is null"); + assertEquals(appId, app.getApplicationId(), "app id doesn't match"); + assertEquals(RMAppState.FINISHED, app.getState(), "app state doesn't match"); } @SuppressWarnings("deprecation") - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testRMAppSubmitInvalidResourceRequest() throws Exception { asContext.setResource(Resources.createResource( YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB + 1)); @@ -1656,19 +1660,20 @@ public void testRMAppSubmitInvalidResourceRequest() throws Exception { // submit an app try { appMonitor.submitApplication(asContext, "test"); - Assert.fail("Application submission should fail because resource" + + fail("Application submission should fail because resource" + " request is invalid."); } catch (YarnException e) { // Exception is expected // TODO Change this to assert the expected exception type - post YARN-142 // sub-task related to specialized exceptions. - assertTrue("The thrown exception is not" + - " InvalidResourceRequestException", - e.getMessage().contains("Invalid resource request")); + assertTrue(e.getMessage().contains("Invalid resource request"), + "The thrown exception is not" + + " InvalidResourceRequestException"); } } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testEscapeApplicationSummary() { RMApp app = mock(RMAppImpl.class); ApplicationSubmissionContext asc = mock(ApplicationSubmissionContext.class); @@ -1709,8 +1714,8 @@ public void testEscapeApplicationSummary() { new RMAppManager.ApplicationSummary().createAppSummary(app); String msg = summary.toString(); LOG.info("summary: " + msg); - Assert.assertFalse(msg.contains("\n")); - Assert.assertFalse(msg.contains("\r")); + assertFalse(msg.contains("\n")); + assertFalse(msg.contains("\r")); String escaped = "\\n\\n\\r\\r"; assertTrue(msg.contains("Multiline" + escaped +"AppName")); @@ -1759,16 +1764,16 @@ public ApplicationPlacementContext answer(InvocationOnMock invocation) event = new RMAppEvent(appId, RMAppEventType.APP_NEW_SAVED); rmContext.getRMApps().get(appId).handle(event); - Assert.assertNotNull("app is null", app); - Assert.assertEquals("newQueue", asContext.getQueue()); + assertNotNull(app, "app is null"); + assertEquals("newQueue", asContext.getQueue()); // wait for event to be processed int timeoutSecs = 0; while ((getAppEventType() == RMAppEventType.KILL) && timeoutSecs++ < 20) { Thread.sleep(1000); } - Assert.assertEquals("app event type sent is wrong", RMAppEventType.START, - getAppEventType()); + assertEquals(RMAppEventType.START, getAppEventType(), + "app event type sent is wrong"); } private static ResourceScheduler mockResourceScheduler() { @@ -2012,7 +2017,7 @@ public void testGetUserNameForPlacementNoRuleDefined() .thenReturn(null); String userNameForPlacement = appMonitor .getUserNameForPlacement(user, asContext, placementMgr); - Assert.assertEquals(expectedUser, userNameForPlacement); + assertEquals(expectedUser, userNameForPlacement); } @Test @@ -2056,8 +2061,8 @@ private void testCheckAccess(String parent, String queue) verify(scheduler).checkAccess(any(UserGroupInformation.class), any(QueueACL.class), queueNameCaptor.capture()); - assertEquals("Expected access check for queue", - expectedQueue, queueNameCaptor.getValue()); + assertEquals(expectedQueue, queueNameCaptor.getValue(), + "Expected access check for queue"); } private void enableApplicationTagPlacement(boolean userHasAccessToQueue, @@ -2090,7 +2095,7 @@ private void verifyPlacementUsername(final String queue, .thenReturn(appContext); String userNameForPlacement = appMonitor .getUserNameForPlacement(submittingUser, asContext, placementMgr); - Assert.assertEquals(expectedUser, userNameForPlacement); + assertEquals(expectedUser, userNameForPlacement); } private void setApplicationTags(String... tags) { @@ -2099,13 +2104,13 @@ private void setApplicationTags(String... tags) { asContext.setApplicationTags(applicationTags); } - private class UseCapacitySchedulerRule extends TestWatcher { + private class UseCapacitySchedulerRule implements BeforeEachCallback { private boolean useCapacityScheduler; @Override - protected void starting(Description d) { - useCapacityScheduler = - d.getAnnotation(UseMockCapacityScheduler.class) != null; + public void beforeEach(ExtensionContext context) throws Exception { + Method testMethod = context.getRequiredTestMethod(); + useCapacityScheduler = testMethod.getAnnotation(UseMockCapacityScheduler.class) != null; } public boolean useCapacityScheduler() { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManagerWithFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManagerWithFairScheduler.java index b3ed5ef58ce48..66208cddf4ce8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManagerWithFairScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManagerWithFairScheduler.java @@ -18,7 +18,9 @@ package org.apache.hadoop.yarn.server.resourcemanager; -import static junit.framework.TestCase.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException.InvalidResourceType.GREATER_THEN_MAX_ALLOCATION; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; @@ -55,10 +57,9 @@ import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** * Testing RMAppManager application submission with fair scheduler. @@ -74,7 +75,7 @@ public class TestAppManagerWithFairScheduler extends AppManagerTestBase { private static String allocFileName = GenericTestUtils.getTestDir(TEST_FOLDER).getAbsolutePath(); - @Before + @BeforeEach public void setup() throws IOException { // Basic config with one queue (override in test if needed) AllocationFileWriter.create() @@ -98,7 +99,7 @@ public void setup() throws IOException { masterService, new ApplicationACLsManager(conf), conf); } - @After + @AfterEach public void teardown(){ File allocFile = GenericTestUtils.getTestDir(TEST_FOLDER); allocFile.delete(); @@ -131,9 +132,9 @@ public void testQueueSubmitWithHighQueueContainerSize() .thenReturn(new ApplicationPlacementContext("limited")); try { rmAppManager.submitApplication(asContext, "test"); - Assert.fail("Test should fail on too high allocation!"); + fail("Test should fail on too high allocation!"); } catch (InvalidResourceRequestException e) { - Assert.assertEquals(GREATER_THEN_MAX_ALLOCATION, + assertEquals(GREATER_THEN_MAX_ALLOCATION, e.getInvalidResourceType()); } @@ -176,10 +177,10 @@ public void testQueueSubmitWithPermissionLimits() .thenReturn(new ApplicationPlacementContext("noaccess")); try { rmAppManager.submitApplication(asContext, "test"); - Assert.fail("Test should have failed with access denied"); + fail("Test should have failed with access denied"); } catch (YarnException e) { - assertTrue("Access exception not found", - e.getCause() instanceof AccessControlException); + assertTrue(e.getCause() instanceof AccessControlException, + "Access exception not found"); } // Submit to submit access queue when(placementMgr.placeApplication(any(), any(), any(Boolean.class))) @@ -249,10 +250,10 @@ public void testQueueSubmitWithAutoCreateQueue() .thenReturn(new ApplicationPlacementContext("root.noaccess.child")); try { rmAppManager.submitApplication(asContext, "test"); - Assert.fail("Test should have failed with access denied"); + fail("Test should have failed with access denied"); } catch (YarnException e) { - assertTrue("Access exception not found", - e.getCause() instanceof AccessControlException); + assertTrue(e.getCause() instanceof AccessControlException, + "Access exception not found"); } // Submit to submitonly parent with non existent child queue when(placementMgr.placeApplication(any(), any(), any(Boolean.class))) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java index 807000a981021..aee3bfcbc79a1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java @@ -18,6 +18,10 @@ package org.apache.hadoop.yarn.server.resourcemanager; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; + import java.net.UnknownHostException; import java.util.ArrayList; import java.util.Arrays; @@ -51,9 +55,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.utils.BuilderUtils; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; public class TestApplicationCleanup { @@ -62,14 +66,14 @@ public class TestApplicationCleanup { private YarnConfiguration conf; - @Before + @BeforeEach public void setup() throws UnknownHostException { GenericTestUtils.setRootLogLevel(Level.DEBUG); conf = new YarnConfiguration(); UserGroupInformation.setConfiguration(conf); conf.set(YarnConfiguration.RECOVERY_ENABLED, "true"); conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName()); - Assert.assertTrue(YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS > 1); + assertTrue(YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS > 1); } @SuppressWarnings("resource") @@ -110,7 +114,7 @@ public void testAppCleanup() throws Exception { contReceived += conts.size(); nm1.nodeHeartbeat(true); } - Assert.assertEquals(request, contReceived); + assertEquals(request, contReceived); am.unregisterAppAttempt(); NodeHeartbeatResponse resp = nm1.nodeHeartbeat(attempt.getAppAttemptId(), 1, @@ -142,10 +146,10 @@ public void testAppCleanup() throws Exception { numCleanedApps = appsToCleanup.size(); } - Assert.assertEquals(1, appsToCleanup.size()); - Assert.assertEquals(app.getApplicationId(), appsToCleanup.get(0)); - Assert.assertEquals(1, numCleanedApps); - Assert.assertEquals(2, numCleanedContainers); + assertEquals(1, appsToCleanup.size()); + assertEquals(app.getApplicationId(), appsToCleanup.get(0)); + assertEquals(1, numCleanedApps); + assertEquals(2, numCleanedContainers); rm.stop(); } @@ -190,7 +194,7 @@ public void testContainerCleanup() throws Exception { contReceived += conts.size(); nm1.nodeHeartbeat(true); } - Assert.assertEquals(request, contReceived); + assertEquals(request, contReceived); // Release a container. ArrayList release = new ArrayList(); @@ -252,7 +256,7 @@ protected void waitForContainerCleanup(MockRM rm, MockNM nm, } else { LOG.info("Got cleanup for " + contsToClean.get(0)); } - Assert.assertEquals(1, cleanedConts); + assertEquals(1, cleanedConts); } private void waitForAppCleanupMessageRecved(MockNM nm, ApplicationId appId) @@ -283,7 +287,8 @@ private MockAM launchAM(RMApp app, MockRM rm, MockNM nm) } @SuppressWarnings("resource") - @Test (timeout = 60000) + @Test + @Timeout(value = 60) public void testAppCleanupWhenRMRestartedAfterAppFinished() throws Exception { conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1); // start RM @@ -318,7 +323,8 @@ public void testAppCleanupWhenRMRestartedAfterAppFinished() throws Exception { } @SuppressWarnings("resource") - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testAppCleanupWhenRMRestartedBeforeAppFinished() throws Exception { conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1); @@ -375,7 +381,8 @@ public void testAppCleanupWhenRMRestartedBeforeAppFinished() throws Exception { rm2.stop(); } - @Test (timeout = 60000) + @Test + @Timeout(value = 60) public void testContainerCleanupWhenRMRestartedAppNotRegistered() throws Exception { conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1); @@ -411,7 +418,8 @@ public void testContainerCleanupWhenRMRestartedAppNotRegistered() throws rm2.stop(); } - @Test (timeout = 60000) + @Test + @Timeout(value = 60) public void testAppCleanupWhenNMReconnects() throws Exception { conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1); @@ -449,7 +457,8 @@ public void testAppCleanupWhenNMReconnects() throws Exception { // 6. Verify for Memory Used, it should be 1024 // 7. Send AM heatbeat to RM. Allocated response should contain completed // container. - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testProcessingNMContainerStatusesOnNMRestart() throws Exception { conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1); @@ -472,7 +481,7 @@ public void testProcessingNMContainerStatusesOnNMRestart() throws Exception { am0.allocateAndWaitForContainers(noOfContainers, containerMemory, nm1); // 3. Verify for number of container allocated by RM - Assert.assertEquals(noOfContainers, allocateContainers.size()); + assertEquals(noOfContainers, allocateContainers.size()); Container container = allocateContainers.get(0); nm1.nodeHeartbeat(am0.getApplicationAttemptId(), 1, ContainerState.RUNNING); @@ -485,7 +494,7 @@ public void testProcessingNMContainerStatusesOnNMRestart() throws Exception { // requested memory. 1024 + 2048=3072 ResourceScheduler rs = rm1.getRMContext().getScheduler(); long allocatedMB = rs.getRootQueueMetrics().getAllocatedMB(); - Assert.assertEquals(amMemory + containerMemory, allocatedMB); + assertEquals(amMemory + containerMemory, allocatedMB); // 5. Re-register NM by sending completed container status List nMContainerStatusForApp = @@ -496,7 +505,7 @@ public void testProcessingNMContainerStatusesOnNMRestart() throws Exception { waitForClusterMemory(nm1, rs, amMemory); // 6. Verify for Memory Used, it should be 1024 - Assert.assertEquals(amMemory, rs.getRootQueueMetrics().getAllocatedMB()); + assertEquals(amMemory, rs.getRootQueueMetrics().getAllocatedMB()); // 7. Send AM heatbeat to RM. Allocated response should contain completed // container @@ -506,7 +515,7 @@ public void testProcessingNMContainerStatusesOnNMRestart() throws Exception { AllocateResponse allocate = am0.allocate(req); List completedContainersStatuses = allocate.getCompletedContainersStatuses(); - Assert.assertEquals(noOfContainers, completedContainersStatuses.size()); + assertEquals(noOfContainers, completedContainersStatuses.size()); // Application clean up should happen Cluster memory used is 0 nm1.nodeHeartbeat(am0.getApplicationAttemptId(), 1, ContainerState.COMPLETE); @@ -523,7 +532,7 @@ private void waitForClusterMemory(MockNM nm1, ResourceScheduler rs, Thread.sleep(100); if (counter++ == 50) { - Assert.fail("Wait for cluster memory is timed out.Expected=" + fail("Wait for cluster memory is timed out.Expected=" + clusterMemory + " Actual=" + rs.getRootQueueMetrics().getAllocatedMB()); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java index 355f28877b7d7..ed922f91d9211 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java @@ -89,12 +89,16 @@ import org.apache.hadoop.yarn.server.security.AMSecretKeys; import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.server.webproxy.ProxyCA; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.function.Supplier; -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.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; @@ -244,20 +248,20 @@ public void testAMLaunchAndCleanup() throws Exception { } catch (TimeoutException e) { fail("timed out while waiting for AM Launch to happen."); } - Assert.assertTrue(containerManager.launched); + assertTrue(containerManager.launched); RMAppAttempt attempt = app.getCurrentAppAttempt(); ApplicationAttemptId appAttemptId = attempt.getAppAttemptId(); - Assert.assertEquals(appAttemptId.toString(), + assertEquals(appAttemptId.toString(), containerManager.attemptIdAtContainerManager); - Assert.assertEquals(app.getSubmitTime(), + assertEquals(app.getSubmitTime(), containerManager.submitTimeAtContainerManager); - Assert.assertEquals(app.getRMAppAttempt(appAttemptId) + assertEquals(app.getRMAppAttempt(appAttemptId) .getMasterContainer().getId() .toString(), containerManager.containerIdAtContainerManager); - Assert.assertEquals(nm1.getNodeId().toString(), + assertEquals(nm1.getNodeId().toString(), containerManager.nmHostAtContainerManager); - Assert.assertEquals(YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS, + assertEquals(YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS, containerManager.maxAppAttempts); MockAM am = new MockAM(rm.getRMContext(), rm @@ -278,7 +282,7 @@ public void testAMLaunchAndCleanup() throws Exception { } catch (TimeoutException e) { fail("timed out while waiting for AM cleanup to happen."); } - Assert.assertTrue(containerManager.cleanedup); + assertTrue(containerManager.cleanedup); rm.waitForState(am.getApplicationAttemptId(), RMAppAttemptState.FINISHED); rm.stop(); @@ -312,9 +316,9 @@ public void testAMCleanupBeforeLaunch() throws Exception { attempt, AMLauncherEventType.LAUNCH, rm.getConfig()) { @Override public void onAMLaunchFailed(ContainerId containerId, Exception e) { - Assert.assertFalse("NullPointerException happens " - + " while launching " + containerId, - e instanceof NullPointerException); + assertFalse(e instanceof NullPointerException, + "NullPointerException happens " + + " while launching " + containerId); } @Override protected ContainerManagementProtocol getContainerMgrProxy( @@ -378,7 +382,8 @@ protected YarnRPC getYarnRPC() { @SuppressWarnings("unused") - @Test(timeout = 100000) + @Test + @Timeout(value = 100) public void testallocateBeforeAMRegistration() throws Exception { boolean thrown = false; GenericTestUtils.setRootLogLevel(Level.DEBUG); @@ -396,7 +401,7 @@ public void testallocateBeforeAMRegistration() throws Exception { AllocateResponse ar = null; try { ar = am.allocate("h1", 1000, request, new ArrayList()); - Assert.fail(); + fail(); } catch (ApplicationMasterNotRegisteredException e) { } @@ -407,16 +412,16 @@ public void testallocateBeforeAMRegistration() throws Exception { try { amrs = am.allocate(new ArrayList(), new ArrayList()); - Assert.fail(); + fail(); } catch (ApplicationMasterNotRegisteredException e) { } am.registerAppAttempt(); try { am.registerAppAttempt(false); - Assert.fail(); + fail(); } catch (Exception e) { - Assert.assertEquals(AMRMClientUtils.APP_ALREADY_REGISTERED_MESSAGE + assertEquals(AMRMClientUtils.APP_ALREADY_REGISTERED_MESSAGE + attempt.getAppAttemptId().getApplicationId(), e.getMessage()); } @@ -430,7 +435,7 @@ public void testallocateBeforeAMRegistration() throws Exception { try { amrs = am.allocate(new ArrayList(), new ArrayList()); - Assert.fail(); + fail(); } catch (ApplicationAttemptNotFoundException e) { } } @@ -477,7 +482,7 @@ public void testAMMasterContainerHost() throws Exception { fail("timed out while waiting for AM Launch to happen."); } - Assert.assertEquals( + assertEquals( app.getCurrentAppAttempt().getMasterContainer().getNodeId().getHost(), host); @@ -531,7 +536,7 @@ private void testSetupTokens(boolean https, YarnConfiguration conf) try { launcher.setupTokens(amContainer, containerId); } catch (java.io.EOFException e) { - Assert.fail("EOFException should not happen."); + fail("EOFException should not happen."); } // verify token @@ -539,30 +544,30 @@ private void testSetupTokens(boolean https, YarnConfiguration conf) dibb.reset(amContainer.getTokens()); Credentials credentials = new Credentials(); credentials.readTokenStorageStream(dibb); - Assert.assertEquals(1, credentials.numberOfTokens()); + assertEquals(1, credentials.numberOfTokens()); org.apache.hadoop.security.token.Token token = credentials.getAllTokens().iterator().next(); - Assert.assertEquals(tokenIdentifier.getKind(), token.getKind()); - Assert.assertArrayEquals(tokenIdentifier.getBytes(), token.getIdentifier()); - Assert.assertArrayEquals("password".getBytes(), token.getPassword()); + assertEquals(tokenIdentifier.getKind(), token.getKind()); + assertArrayEquals(tokenIdentifier.getBytes(), token.getIdentifier()); + assertArrayEquals("password".getBytes(), token.getPassword()); // verify keystore and truststore if (https) { - Assert.assertEquals(4, credentials.numberOfSecretKeys()); - Assert.assertArrayEquals("keystore".getBytes(), + assertEquals(4, credentials.numberOfSecretKeys()); + assertArrayEquals("keystore".getBytes(), credentials.getSecretKey( AMSecretKeys.YARN_APPLICATION_AM_KEYSTORE)); - Assert.assertArrayEquals("kPassword".getBytes(), + assertArrayEquals("kPassword".getBytes(), credentials.getSecretKey( AMSecretKeys.YARN_APPLICATION_AM_KEYSTORE_PASSWORD)); - Assert.assertArrayEquals("truststore".getBytes(), + assertArrayEquals("truststore".getBytes(), credentials.getSecretKey( AMSecretKeys.YARN_APPLICATION_AM_TRUSTSTORE)); - Assert.assertArrayEquals("tPassword".getBytes(), + assertArrayEquals("tPassword".getBytes(), credentials.getSecretKey( AMSecretKeys.YARN_APPLICATION_AM_TRUSTSTORE_PASSWORD)); } else { - Assert.assertEquals(0, credentials.numberOfSecretKeys()); + assertEquals(0, credentials.numberOfSecretKeys()); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceCapacity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceCapacity.java index ab68515911651..fef2379e0d817 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceCapacity.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceCapacity.java @@ -47,8 +47,8 @@ import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.ArrayList; import java.util.Arrays; @@ -57,7 +57,8 @@ import java.util.Set; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils.toSet; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.fail; /** * Unit tests for {@link ApplicationMasterService} @@ -103,7 +104,8 @@ private void sentRMContainerLaunched(MockRM rm, ContainerId containerId) { } } - @Test(timeout=60000) + @Test + @Timeout(value = 60) public void testInvalidIncreaseDecreaseRequest() throws Exception { conf = new YarnConfiguration(); conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, @@ -142,8 +144,8 @@ public void testInvalidIncreaseDecreaseRequest() throws Exception { ContainerId.newContainerId(attempt1.getAppAttemptId(), 1), ContainerUpdateType.INCREASE_RESOURCE, Resources.createResource(-1), null))); - Assert.assertEquals(1, response.getUpdateErrors().size()); - Assert.assertEquals("RESOURCE_OUTSIDE_ALLOWED_RANGE", + assertEquals(1, response.getUpdateErrors().size()); + assertEquals("RESOURCE_OUTSIDE_ALLOWED_RANGE", response.getUpdateErrors().get(0).getReason()); // Target resource is more than maxAllocation, should fail @@ -154,8 +156,8 @@ public void testInvalidIncreaseDecreaseRequest() throws Exception { Resources.add( registerResponse.getMaximumResourceCapability(), Resources.createResource(1)), null))); - Assert.assertEquals(1, response.getUpdateErrors().size()); - Assert.assertEquals("RESOURCE_OUTSIDE_ALLOWED_RANGE", + assertEquals(1, response.getUpdateErrors().size()); + assertEquals("RESOURCE_OUTSIDE_ALLOWED_RANGE", response.getUpdateErrors().get(0).getReason()); // Contains multiple increase/decrease requests for same containerId @@ -168,13 +170,14 @@ public void testInvalidIncreaseDecreaseRequest() throws Exception { ContainerId.newContainerId(attempt1.getAppAttemptId(), 1), ContainerUpdateType.DECREASE_RESOURCE, Resources.createResource(1024, 1), null))); - Assert.assertEquals(1, response.getUpdateErrors().size()); - Assert.assertEquals("UPDATE_OUTSTANDING_ERROR", + assertEquals(1, response.getUpdateErrors().size()); + assertEquals("UPDATE_OUTSTANDING_ERROR", response.getUpdateErrors().get(0).getReason()); } } - @Test(timeout = 300000) + @Test + @Timeout(value = 300) public void testPriorityInAllocatedResponse() throws Exception { conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, ResourceScheduler.class); @@ -206,7 +209,7 @@ public void testPriorityInAllocatedResponse() throws Exception { allocateRequest.setAskList(ask); AllocateResponse response1 = am1.allocate(allocateRequest); - Assert.assertEquals(appPriority1, response1.getApplicationPriority()); + assertEquals(appPriority1, response1.getApplicationPriority()); // Change the priority of App1 to 8 Priority appPriority2 = Priority.newInstance(8); @@ -216,11 +219,12 @@ public void testPriorityInAllocatedResponse() throws Exception { appPriority2); AllocateResponse response2 = am1.allocate(allocateRequest); - Assert.assertEquals(appPriority2, response2.getApplicationPriority()); + assertEquals(appPriority2, response2.getApplicationPriority()); rm.stop(); } - @Test(timeout = 300000) + @Test + @Timeout(value = 300) public void testGetNMNumInAllocatedResponseWithOutNodeLabel() throws Exception { conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, ResourceScheduler.class); MockRM rm = new MockRM(conf); @@ -252,7 +256,7 @@ public void testGetNMNumInAllocatedResponseWithOutNodeLabel() throws Exception { allocateRequest.setAskList(ask); AllocateResponse response1 = am1.allocate(allocateRequest); - Assert.assertEquals(3, response1.getNumClusterNodes()); + assertEquals(3, response1.getNumClusterNodes()); rm.stop(); } @@ -285,7 +289,8 @@ private Configuration getConfigurationWithQueueLabels(Configuration config) { return conf; } - @Test(timeout = 300000) + @Test + @Timeout(value = 300) public void testGetNMNumInAllocatedResponseWithNodeLabel() throws Exception { conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, ResourceScheduler.class); conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true); @@ -368,10 +373,10 @@ protected RMNodeLabelsManager createNodeLabelManager() { } //has 3 nodes with node label "x" - Assert.assertEquals(3, response1.getNumClusterNodes()); + assertEquals(3, response1.getNumClusterNodes()); //has 1 node with node label "y" - Assert.assertEquals(1, response2.getNumClusterNodes()); + assertEquals(1, response2.getNumClusterNodes()); rm.stop(); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceFair.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceFair.java index 6b268381fd4a4..17f860c8467ba 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceFair.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceFair.java @@ -30,14 +30,14 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedulerConfiguration; import org.apache.hadoop.yarn.util.resource.ResourceUtils; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.Collections; import java.util.Map; import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB; import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.fail; /** * Unit tests for {@link ApplicationMasterService} with {@link FairScheduler}. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceInterceptor.java index e1d6defa63d9c..15d75f2e772ca 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceInterceptor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceInterceptor.java @@ -16,6 +16,8 @@ package org.apache.hadoop.yarn.server.resourcemanager; +import static org.junit.jupiter.api.Assertions.assertEquals; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.yarn.ams.ApplicationMasterServiceContext; @@ -37,9 +39,9 @@ .ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler; import org.apache.hadoop.yarn.server.utils.BuilderUtils; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.io.IOException; import java.util.concurrent.atomic.AtomicInteger; @@ -152,14 +154,15 @@ public void finishApplicationMaster( private static YarnConfiguration conf; private static final int GB = 1024; - @Before + @BeforeEach public void setup() { conf = new YarnConfiguration(); conf.setClass(YarnConfiguration.RM_SCHEDULER, FifoScheduler.class, ResourceScheduler.class); } - @Test(timeout = 300000) + @Test + @Timeout(value = 300) public void testApplicationMasterInterceptor() throws Exception { conf.set(YarnConfiguration.RM_APPLICATION_MASTER_SERVICE_PROCESSORS, TestInterceptor1.class.getName() + "," @@ -201,17 +204,17 @@ public void testApplicationMasterInterceptor() throws Exception { .getContainerToken()); am1.unregisterAppAttempt(); - Assert.assertEquals(1, beforeRegCount.get()); - Assert.assertEquals(1, afterRegCount.get()); + assertEquals(1, beforeRegCount.get()); + assertEquals(1, afterRegCount.get()); // The allocate calls should be incremented twice - Assert.assertEquals(allocCount * 2, beforeAllocCount.get()); - Assert.assertEquals(allocCount * 2, afterAllocCount.get()); + assertEquals(allocCount * 2, beforeAllocCount.get()); + assertEquals(allocCount * 2, afterAllocCount.get()); // Finish should only be called once, since the FirstInterceptor // does not forward the call. - Assert.assertEquals(1, beforeFinishCount.get()); - Assert.assertEquals(1, afterFinishCount.get()); + assertEquals(1, beforeFinishCount.get()); + assertEquals(1, afterFinishCount.get()); rm.stop(); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestCapacitySchedulerMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestCapacitySchedulerMetrics.java index 99b3983f863eb..894e8fd2f5ada 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestCapacitySchedulerMetrics.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestCapacitySchedulerMetrics.java @@ -18,6 +18,11 @@ package org.apache.hadoop.yarn.server.resourcemanager; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; + import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager; @@ -27,9 +32,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerMetrics; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.concurrent.TimeoutException; @@ -67,7 +71,7 @@ public RMNodeLabelsManager createNodeLabelManager() { nm2.nodeHeartbeat(true); CapacitySchedulerMetrics csMetrics = CapacitySchedulerMetrics.getMetrics(); - Assert.assertNotNull(csMetrics); + assertNotNull(csMetrics); try { GenericTestUtils.waitFor(() -> csMetrics.getNumOfNodeUpdate() == 2, 100, 3000); @@ -75,11 +79,11 @@ public RMNodeLabelsManager createNodeLabelManager() { .waitFor(() -> csMetrics.getNumOfSchedulerNodeHBInterval() == 2, 100, 3000); } catch(TimeoutException e) { - Assert.fail("CS metrics not updated on node-update events."); + fail("CS metrics not updated on node-update events."); } - Assert.assertEquals(0, csMetrics.getNumOfAllocates()); - Assert.assertEquals(0, csMetrics.getNumOfCommitSuccess()); + assertEquals(0, csMetrics.getNumOfAllocates()); + assertEquals(0, csMetrics.getNumOfCommitSuccess()); RMApp rmApp = MockRMAppSubmitter.submit(rm, MockRMAppSubmissionData.Builder.createWithMemory(1024, rm) @@ -108,16 +112,16 @@ public RMNodeLabelsManager createNodeLabelManager() { .waitFor(() -> csMetrics.getNumOfSchedulerNodeHBInterval() == 4, 100, 3000); // For async mode, the number of alloc might be bigger than 1 - Assert.assertTrue(csMetrics.getNumOfAllocates() > 0); + assertTrue(csMetrics.getNumOfAllocates() > 0); // But there will be only 2 successful commit (1 AM + 1 task) GenericTestUtils.waitFor(() -> csMetrics.getNumOfCommitSuccess() == 2, 100, 3000); } catch(TimeoutException e) { - Assert.fail("CS metrics not updated on node-update events."); + fail("CS metrics not updated on node-update events."); } } - @After + @AfterEach public void tearDown() { if (rm != null) { rm.stop(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java index f31a79112161b..e0ebf4e6caed8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java @@ -20,8 +20,15 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static org.assertj.core.api.Assertions.assertThat; -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.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNotSame; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; +import static org.junit.jupiter.api.Assumptions.assumeFalse; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.anyList; @@ -185,10 +192,9 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceUtils; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.After; -import org.junit.Assert; -import org.junit.Assume; -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.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -251,10 +257,10 @@ protected ClientRMService createClientRMService() { GetClusterNodesRequest.newInstance( EnumSet.of(NodeState.DECOMMISSIONING))) .getNodeReports(); - Assert.assertEquals(1, nodeReports.size()); + assertEquals(1, nodeReports.size()); NodeReport nr = nodeReports.iterator().next(); - Assert.assertEquals(decommissioningTimeout, nr.getDecommissioningTimeout()); - Assert.assertNull(nr.getNodeUpdateType()); + assertEquals(decommissioningTimeout, nr.getDecommissioningTimeout()); + assertNull(nr.getNodeUpdateType()); } @Test @@ -299,14 +305,14 @@ protected ClientRMService createClientRMService() { GetClusterNodesRequest.newInstance(EnumSet.of(NodeState.RUNNING)); List nodeReports = client.getClusterNodes(request).getNodeReports(); - Assert.assertEquals(1, nodeReports.size()); - Assert.assertNotSame("Node is expected to be healthy!", NodeState.UNHEALTHY, - nodeReports.get(0).getNodeState()); + assertEquals(1, nodeReports.size()); + assertNotSame(NodeState.UNHEALTHY, nodeReports.get(0).getNodeState(), + "Node is expected to be healthy!"); // Check node's label = x - Assert.assertTrue(nodeReports.get(0).getNodeLabels().contains("x")); - Assert.assertNull(nodeReports.get(0).getDecommissioningTimeout()); - Assert.assertNull(nodeReports.get(0).getNodeUpdateType()); + assertTrue(nodeReports.get(0).getNodeLabels().contains("x")); + assertNull(nodeReports.get(0).getDecommissioningTimeout()); + assertNull(nodeReports.get(0).getNodeUpdateType()); // Now make the node unhealthy. node.nodeHeartbeat(false); @@ -314,8 +320,8 @@ protected ClientRMService createClientRMService() { // Call again nodeReports = client.getClusterNodes(request).getNodeReports(); - Assert.assertEquals("Unhealthy nodes should not show up by default", 0, - nodeReports.size()); + assertEquals(0, nodeReports.size(), + "Unhealthy nodes should not show up by default"); // Change label of host1 to y map = new HashMap>(); @@ -325,13 +331,13 @@ protected ClientRMService createClientRMService() { // Now query for UNHEALTHY nodes request = GetClusterNodesRequest.newInstance(EnumSet.of(NodeState.UNHEALTHY)); nodeReports = client.getClusterNodes(request).getNodeReports(); - Assert.assertEquals(1, nodeReports.size()); - Assert.assertEquals("Node is expected to be unhealthy!", NodeState.UNHEALTHY, - nodeReports.get(0).getNodeState()); + assertEquals(1, nodeReports.size()); + assertEquals(NodeState.UNHEALTHY, nodeReports.get(0).getNodeState(), + "Node is expected to be unhealthy!"); - Assert.assertTrue(nodeReports.get(0).getNodeLabels().contains("y")); - Assert.assertNull(nodeReports.get(0).getDecommissioningTimeout()); - Assert.assertNull(nodeReports.get(0).getNodeUpdateType()); + assertTrue(nodeReports.get(0).getNodeLabels().contains("y")); + assertNull(nodeReports.get(0).getDecommissioningTimeout()); + assertNull(nodeReports.get(0).getNodeUpdateType()); // Remove labels of host1 map = new HashMap>(); @@ -342,14 +348,14 @@ protected ClientRMService createClientRMService() { rm.registerNode("host3:1236", 1024); request = GetClusterNodesRequest.newInstance(EnumSet.allOf(NodeState.class)); nodeReports = client.getClusterNodes(request).getNodeReports(); - Assert.assertEquals(3, nodeReports.size()); + assertEquals(3, nodeReports.size()); // All host1-3's label should be empty (instead of null) for (NodeReport report : nodeReports) { - Assert.assertTrue(report.getNodeLabels() != null + assertTrue(report.getNodeLabels() != null && report.getNodeLabels().isEmpty()); - Assert.assertNull(report.getDecommissioningTimeout()); - Assert.assertNull(report.getNodeUpdateType()); + assertNull(report.getDecommissioningTimeout()); + assertNull(report.getNodeUpdateType()); } } @@ -365,9 +371,9 @@ public void testNonExistingApplicationReport() throws YarnException { request.setApplicationId(ApplicationId.newInstance(0, 0)); try { rmService.getApplicationReport(request); - Assert.fail(); + fail(); } catch (ApplicationNotFoundException ex) { - Assert.assertEquals(ex.getMessage(), + assertEquals(ex.getMessage(), "Application with id '" + request.getApplicationId() + "' doesn't exist in RM. Please check that the " + "job submission was successful."); @@ -398,10 +404,10 @@ public void testGetApplicationReport() throws Exception { ApplicationReport report = response.getApplicationReport(); ApplicationResourceUsageReport usageReport = report.getApplicationResourceUsageReport(); - Assert.assertEquals(10, usageReport.getMemorySeconds()); - Assert.assertEquals(3, usageReport.getVcoreSeconds()); - Assert.assertEquals("", report.getAmNodeLabelExpression()); - Assert.assertEquals("", report.getAppNodeLabelExpression()); + assertEquals(10, usageReport.getMemorySeconds()); + assertEquals(3, usageReport.getVcoreSeconds()); + assertEquals("", report.getAmNodeLabelExpression()); + assertEquals("", report.getAppNodeLabelExpression()); // if application has am node label set to blank ApplicationId appId2 = getApplicationId(2); @@ -411,9 +417,9 @@ public void testGetApplicationReport() throws Exception { response = rmService.getApplicationReport(request); report = response.getApplicationReport(); - Assert.assertEquals(NodeLabel.DEFAULT_NODE_LABEL_PARTITION, + assertEquals(NodeLabel.DEFAULT_NODE_LABEL_PARTITION, report.getAmNodeLabelExpression()); - Assert.assertEquals(NodeLabel.NODE_LABEL_EXPRESSION_NOT_SET, + assertEquals(NodeLabel.NODE_LABEL_EXPRESSION_NOT_SET, report.getAppNodeLabelExpression()); // if application has am node label set to blank @@ -425,8 +431,8 @@ public void testGetApplicationReport() throws Exception { response = rmService.getApplicationReport(request); report = response.getApplicationReport(); - Assert.assertEquals("high-mem", report.getAmNodeLabelExpression()); - Assert.assertEquals("high-mem", report.getAppNodeLabelExpression()); + assertEquals("high-mem", report.getAmNodeLabelExpression()); + assertEquals("high-mem", report.getAppNodeLabelExpression()); // if application id is null GetApplicationReportRequest invalidRequest = recordFactory @@ -437,7 +443,7 @@ public void testGetApplicationReport() throws Exception { } catch (YarnException e) { // rmService should return a ApplicationNotFoundException // when a null application id is provided - Assert.assertTrue(e instanceof ApplicationNotFoundException); + assertTrue(e instanceof ApplicationNotFoundException); } } finally { rmService.close(); @@ -457,10 +463,10 @@ public void testGetApplicationAttemptReport() throws YarnException, try { GetApplicationAttemptReportResponse response = rmService .getApplicationAttemptReport(request); - Assert.assertEquals(attemptId, response.getApplicationAttemptReport() + assertEquals(attemptId, response.getApplicationAttemptReport() .getApplicationAttemptId()); } catch (ApplicationNotFoundException ex) { - Assert.fail(ex.getMessage()); + fail(ex.getMessage()); } } @@ -498,12 +504,12 @@ public void testGetApplicationAttempts() throws YarnException, IOException { try { GetApplicationAttemptsResponse response = rmService .getApplicationAttempts(request); - Assert.assertEquals(1, response.getApplicationAttemptList().size()); - Assert.assertEquals(attemptId, response.getApplicationAttemptList() + assertEquals(1, response.getApplicationAttemptList().size()); + assertEquals(attemptId, response.getApplicationAttemptList() .get(0).getApplicationAttemptId()); } catch (ApplicationNotFoundException ex) { - Assert.fail(ex.getMessage()); + fail(ex.getMessage()); } } @@ -520,10 +526,10 @@ public void testGetContainerReport() throws YarnException, IOException { try { GetContainerReportResponse response = rmService .getContainerReport(request); - Assert.assertEquals(containerId, response.getContainerReport() + assertEquals(containerId, response.getContainerReport() .getContainerId()); } catch (ApplicationNotFoundException ex) { - Assert.fail(ex.getMessage()); + fail(ex.getMessage()); } } @@ -538,10 +544,10 @@ public void testGetContainers() throws YarnException, IOException { request.setApplicationAttemptId(attemptId); try { GetContainersResponse response = rmService.getContainers(request); - Assert.assertEquals(containerId, response.getContainerList().get(0) + assertEquals(containerId, response.getContainerList().get(0) .getContainerId()); } catch (ApplicationNotFoundException ex) { - Assert.fail(ex.getMessage()); + fail(ex.getMessage()); } } @@ -584,9 +590,9 @@ public void testForceKillNonExistingApplication() throws YarnException { KillApplicationRequest.newInstance(applicationId); try { rmService.forceKillApplication(request); - Assert.fail(); + fail(); } catch (ApplicationNotFoundException ex) { - Assert.assertEquals(ex.getMessage(), + assertEquals(ex.getMessage(), "Trying to kill an absent " + "application " + request.getApplicationId()); } @@ -631,9 +637,9 @@ private void validateApplicationTag(ClientRMService rmService, new HashSet(tags)); try { rmService.submitApplication(submitRequest); - Assert.fail(); + fail(); } catch (Exception ex) { - Assert.assertTrue(ex.getMessage().contains(errorMsg)); + assertTrue(ex.getMessage().contains(errorMsg)); } } @@ -657,8 +663,8 @@ public void testForceKillApplication() throws Exception { .build(); RMApp app2 = MockRMAppSubmitter.submit(rm, data); - assertEquals("Incorrect number of apps in the RM", 0, - rmService.getApplications(getRequest).getApplicationList().size()); + assertEquals(0, rmService.getApplications(getRequest).getApplicationList().size(), + "Incorrect number of apps in the RM"); KillApplicationRequest killRequest1 = KillApplicationRequest.newInstance(app1.getApplicationId()); @@ -677,17 +683,18 @@ public void testForceKillApplication() throws Exception { } Thread.sleep(10); } - assertTrue("Kill attempt count should be greater than 1 for managed AMs", - killAttemptCount > 1); - assertEquals("Incorrect number of apps in the RM", 1, - rmService.getApplications(getRequest).getApplicationList().size()); - assertTrue("Diagnostic message is incorrect", - app1.getDiagnostics().toString().contains(diagnostic)); + assertTrue(killAttemptCount > 1, + "Kill attempt count should be greater than 1 for managed AMs"); + assertEquals(1, + rmService.getApplications(getRequest).getApplicationList().size(), + "Incorrect number of apps in the RM"); + assertTrue(app1.getDiagnostics().toString().contains(diagnostic), + "Diagnostic message is incorrect"); KillApplicationResponse killResponse2 = rmService.forceKillApplication(killRequest2); - assertTrue("Killing UnmanagedAM should falsely acknowledge true", - killResponse2.getIsKillCompleted()); + assertTrue(killResponse2.getIsKillCompleted(), + "Killing UnmanagedAM should falsely acknowledge true"); for (int i = 0; i < 100; i++) { if (2 == rmService.getApplications(getRequest).getApplicationList().size()) { @@ -695,23 +702,25 @@ public void testForceKillApplication() throws Exception { } Thread.sleep(10); } - assertEquals("Incorrect number of apps in the RM", 2, - rmService.getApplications(getRequest).getApplicationList().size()); + assertEquals(2, rmService.getApplications(getRequest).getApplicationList().size(), + "Incorrect number of apps in the RM"); } - @Test (expected = ApplicationNotFoundException.class) + @Test public void testMoveAbsentApplication() throws YarnException { - RMContext rmContext = mock(RMContext.class); - when(rmContext.getRMApps()).thenReturn( - new ConcurrentHashMap()); - ClientRMService rmService = new ClientRMService(rmContext, null, null, - null, null, null); - ApplicationId applicationId = - BuilderUtils.newApplicationId(System.currentTimeMillis(), 0); - MoveApplicationAcrossQueuesRequest request = - MoveApplicationAcrossQueuesRequest.newInstance(applicationId, - "newqueue"); - rmService.moveApplicationAcrossQueues(request); + assertThrows(ApplicationNotFoundException.class, () -> { + RMContext rmContext = mock(RMContext.class); + when(rmContext.getRMApps()).thenReturn( + new ConcurrentHashMap()); + ClientRMService rmService = new ClientRMService(rmContext, null, null, + null, null, null); + ApplicationId applicationId = + BuilderUtils.newApplicationId(System.currentTimeMillis(), 0); + MoveApplicationAcrossQueuesRequest request = + MoveApplicationAcrossQueuesRequest.newInstance(applicationId, + "newqueue"); + rmService.moveApplicationAcrossQueues(request); + }); } @Test @@ -739,10 +748,10 @@ public void testMoveApplicationSubmitTargetQueue() throws Exception { try { rmService.moveApplicationAcrossQueues(moveAppRequest); - Assert.fail("The request should fail with an AccessControlException"); + fail("The request should fail with an AccessControlException"); } catch (YarnException rex) { - Assert.assertTrue("AccessControlException is expected", - rex.getCause() instanceof AccessControlException); + assertTrue(rex.getCause() instanceof AccessControlException, + "AccessControlException is expected"); } // ACL is owned by "moveuser", move is performed as a different user @@ -761,10 +770,10 @@ public void testMoveApplicationSubmitTargetQueue() throws Exception { newInstance(applicationId, "move_queue"); try { rmService2.moveApplicationAcrossQueues(moveAppRequest2); - Assert.fail("The request should fail with an AccessControlException"); + fail("The request should fail with an AccessControlException"); } catch (YarnException rex) { - Assert.assertTrue("AccessControlException is expected", - rex.getCause() instanceof AccessControlException); + assertTrue(rex.getCause() instanceof AccessControlException, + "AccessControlException is expected"); } // execute the move as the acl owner @@ -800,10 +809,10 @@ public void testMoveApplicationAdminTargetQueue() throws Exception { try { rmService.moveApplicationAcrossQueues(moveAppRequest); - Assert.fail("The request should fail with an AccessControlException"); + fail("The request should fail with an AccessControlException"); } catch (YarnException rex) { - Assert.assertTrue("AccessControlException is expected", - rex.getCause() instanceof AccessControlException); + assertTrue(rex.getCause() instanceof AccessControlException, + "AccessControlException is expected"); } // ACL is owned by "moveuser", move is performed as a different user @@ -823,10 +832,10 @@ public void testMoveApplicationAdminTargetQueue() throws Exception { try { rmService2.moveApplicationAcrossQueues(moveAppRequest2); - Assert.fail("The request should fail with an AccessControlException"); + fail("The request should fail with an AccessControlException"); } catch (YarnException rex) { - Assert.assertTrue("AccessControlException is expected", - rex.getCause() instanceof AccessControlException); + assertTrue(rex.getCause() instanceof AccessControlException, + "AccessControlException is expected"); } // execute the move as the acl owner @@ -839,20 +848,21 @@ public Object run() throws Exception { }); } - @Test (expected = YarnException.class) + @Test public void testNonExistingQueue() throws Exception { - ApplicationId applicationId = getApplicationId(1); - UserGroupInformation aclUGI = UserGroupInformation.getCurrentUser(); - QueueACLsManager queueAclsManager = getQueueAclManager(); - ApplicationACLsManager appAclsManager = getAppAclManager(); - ClientRMService rmService = - createClientRMServiceForMoveApplicationRequest(applicationId, - aclUGI.getShortUserName(), appAclsManager, queueAclsManager); - - MoveApplicationAcrossQueuesRequest moveAppRequest = - MoveApplicationAcrossQueuesRequest.newInstance(applicationId, - "unknown_queue"); - rmService.moveApplicationAcrossQueues(moveAppRequest); + assertThrows(YarnException.class, () -> { + ApplicationId applicationId = getApplicationId(1); + UserGroupInformation aclUGI = UserGroupInformation.getCurrentUser(); + QueueACLsManager queueAclsManager = getQueueAclManager(); + ApplicationACLsManager appAclsManager = getAppAclManager(); + ClientRMService rmService = + createClientRMServiceForMoveApplicationRequest(applicationId, + aclUGI.getShortUserName(), appAclsManager, queueAclsManager); + MoveApplicationAcrossQueuesRequest moveAppRequest = + MoveApplicationAcrossQueuesRequest.newInstance(applicationId, + "unknown_queue"); + rmService.moveApplicationAcrossQueues(moveAppRequest); + }); } /** @@ -990,17 +1000,17 @@ public void testGetQueueInfo() throws Exception { GetQueueInfoResponse queueInfo = rmService.getQueueInfo(request); List applications = queueInfo.getQueueInfo() .getApplications(); - Assert.assertEquals(2, applications.size()); + assertEquals(2, applications.size()); Map queueConfigsByPartition = queueInfo.getQueueInfo().getQueueConfigurations(); - Assert.assertEquals(1, queueConfigsByPartition.size()); - Assert.assertTrue(queueConfigsByPartition.containsKey("*")); + assertEquals(1, queueConfigsByPartition.size()); + assertTrue(queueConfigsByPartition.containsKey("*")); QueueConfigurations queueConfigs = queueConfigsByPartition.get("*"); - Assert.assertEquals(0.5f, queueConfigs.getCapacity(), 0.0001f); - Assert.assertEquals(0.1f, queueConfigs.getAbsoluteCapacity(), 0.0001f); - Assert.assertEquals(1.0f, queueConfigs.getMaxCapacity(), 0.0001f); - Assert.assertEquals(1.0f, queueConfigs.getAbsoluteMaxCapacity(), 0.0001f); - Assert.assertEquals(0.2f, queueConfigs.getMaxAMPercentage(), 0.0001f); + assertEquals(0.5f, queueConfigs.getCapacity(), 0.0001f); + assertEquals(0.1f, queueConfigs.getAbsoluteCapacity(), 0.0001f); + assertEquals(1.0f, queueConfigs.getMaxCapacity(), 0.0001f); + assertEquals(1.0f, queueConfigs.getAbsoluteMaxCapacity(), 0.0001f); + assertEquals(0.2f, queueConfigs.getMaxAMPercentage(), 0.0001f); request.setQueueName("nonexistentqueue"); request.setIncludeApplications(true); @@ -1027,10 +1037,11 @@ public void testGetQueueInfo() throws Exception { GetQueueInfoResponse queueInfo1 = rmService1.getQueueInfo(request); List applications1 = queueInfo1.getQueueInfo() .getApplications(); - Assert.assertEquals(0, applications1.size()); + assertEquals(0, applications1.size()); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) @SuppressWarnings ("rawtypes") public void testAppSubmitWithSubmissionPreProcessor() throws Exception { ResourceScheduler scheduler = mockResourceScheduler(); @@ -1095,17 +1106,18 @@ public void handle(Event event) {} try { rmService.submitApplication(submitRequest1); } catch (YarnException e) { - Assert.fail("Exception is not expected."); + fail("Exception is not expected."); } RMApp app1 = rmContext.getRMApps().get(appId1); - Assert.assertNotNull("app doesn't exist", app1); - Assert.assertEquals("app name doesn't match", - YarnConfiguration.DEFAULT_APPLICATION_NAME, app1.getName()); - Assert.assertTrue("custom tag not present", - app1.getApplicationTags().contains("cluster:cluster1")); - Assert.assertEquals("app queue doesn't match", "bar", app1.getQueue()); - Assert.assertEquals("app node label doesn't match", - "foo", app1.getApplicationSubmissionContext().getNodeLabelExpression()); + assertNotNull(app1, "app doesn't exist"); + assertEquals(YarnConfiguration.DEFAULT_APPLICATION_NAME, app1.getName(), + "app name doesn't match"); + assertTrue(app1.getApplicationTags().contains("cluster:cluster1"), + "custom tag not present"); + assertEquals("bar", app1.getQueue(), "app queue doesn't match"); + assertEquals("foo", + app1.getApplicationSubmissionContext().getNodeLabelExpression(), + "app node label doesn't match"); setupCurrentCall("host.cluster2.com"); ApplicationId appId2 = getApplicationId(101); SubmitApplicationRequest submitRequest2 = mockSubmitAppRequest( @@ -1118,20 +1130,20 @@ public void handle(Event event) {} try { rmService.submitApplication(submitRequest2); } catch (YarnException e) { - Assert.fail("Exception is not expected."); + fail("Exception is not expected."); } RMApp app2 = rmContext.getRMApps().get(appId2); - Assert.assertNotNull("app doesn't exist", app2); - Assert.assertEquals("app name doesn't match", - YarnConfiguration.DEFAULT_APPLICATION_NAME, app2.getName()); - Assert.assertTrue("client tag not present", - app2.getApplicationTags().contains(APPLICATION_TAG_SC_PREPROCESSOR)); - Assert.assertTrue("custom tag not present", - app2.getApplicationTags().contains("cluster:cluster2")); - Assert.assertEquals("app queue doesn't match", "hello", app2.getQueue()); - Assert.assertEquals("app node label doesn't match", - "zuess", - app2.getApplicationSubmissionContext().getNodeLabelExpression()); + assertNotNull(app2, "app doesn't exist"); + assertEquals(YarnConfiguration.DEFAULT_APPLICATION_NAME, app2.getName(), + "app name doesn't match"); + assertTrue(app2.getApplicationTags().contains(APPLICATION_TAG_SC_PREPROCESSOR), + "client tag not present"); + assertTrue(app2.getApplicationTags().contains("cluster:cluster2"), + "custom tag not present"); + assertEquals("hello", app2.getQueue(), "app queue doesn't match"); + assertEquals("zuess", + app2.getApplicationSubmissionContext().getNodeLabelExpression(), + "app node label doesn't match"); // Test Default commands setupCurrentCall("host2.cluster3.com"); ApplicationId appId3 = getApplicationId(102); @@ -1143,20 +1155,19 @@ public void handle(Event event) {} try { rmService.submitApplication(submitRequest3); } catch (YarnException e) { - Assert.fail("Exception is not expected."); + fail("Exception is not expected."); } RMApp app3 = rmContext.getRMApps().get(appId3); - Assert.assertNotNull("app doesn't exist", app3); - Assert.assertEquals("app name doesn't match", - YarnConfiguration.DEFAULT_APPLICATION_NAME, app3.getName()); - Assert.assertTrue("client tag not present", - app3.getApplicationTags().contains(APPLICATION_TAG_SC_PREPROCESSOR)); - Assert.assertTrue("custom tag not present", - app3.getApplicationTags().contains("cluster:other")); - Assert.assertEquals("app queue doesn't match", "default", app3.getQueue()); - Assert.assertEquals("app node label doesn't match", - "barfoo", - app3.getApplicationSubmissionContext().getNodeLabelExpression()); + assertNotNull(app3, "app doesn't exist"); + assertEquals(YarnConfiguration.DEFAULT_APPLICATION_NAME, app3.getName(), + "app name doesn't match"); + assertTrue(app3.getApplicationTags().contains(APPLICATION_TAG_SC_PREPROCESSOR), + "client tag not present"); + assertTrue(app3.getApplicationTags().contains("cluster:other"), + "custom tag not present"); + assertEquals("default", app3.getQueue(), "app queue doesn't match"); + assertEquals("barfoo", app3.getApplicationSubmissionContext().getNodeLabelExpression(), + "app node label doesn't match"); // Test regex setupCurrentCall("host.cluster100.com"); ApplicationId appId4 = getApplicationId(103); @@ -1165,13 +1176,14 @@ public void handle(Event event) {} try { rmService.submitApplication(submitRequest4); } catch (YarnException e) { - Assert.fail("Exception is not expected."); + fail("Exception is not expected."); } RMApp app4 = rmContext.getRMApps().get(appId4); - Assert.assertTrue("custom tag not present", - app4.getApplicationTags().contains("cluster:reg")); - Assert.assertEquals("app node label doesn't match", - "reg", app4.getApplicationSubmissionContext().getNodeLabelExpression()); + assertTrue(app4.getApplicationTags().contains("cluster:reg"), + "custom tag not present"); + assertEquals("reg", + app4.getApplicationSubmissionContext().getNodeLabelExpression(), + "app node label doesn't match"); testSubmissionContextWithAbsentTAG(rmService, rmContext); rmService.serviceStop(); } @@ -1185,15 +1197,14 @@ private void testSubmissionContextWithAbsentTAG(ClientRMService rmService, try { rmService.submitApplication(submitRequest5); } catch (YarnException e) { - Assert.fail("Exception is not expected."); + fail("Exception is not expected."); } RMApp app5 = rmContext.getRMApps().get(appId5); - Assert.assertEquals("custom tag present", - app5.getApplicationTags().size(), 0); - Assert.assertNull("app node label present", - app5.getApplicationSubmissionContext().getNodeLabelExpression()); - Assert.assertEquals("Queue name is not present", - app5.getQueue(), "default"); + assertEquals(app5.getApplicationTags().size(), 0, "custom tag present"); + assertNull(app5.getApplicationSubmissionContext().getNodeLabelExpression(), + "app node label present"); + assertEquals(app5.getQueue(), "default", + "Queue name is not present"); } private void setupCurrentCall(String hostName) throws UnknownHostException { Server.Call mockCall = mock(Server.Call.class); @@ -1203,7 +1214,8 @@ private void setupCurrentCall(String hostName) throws UnknownHostException { Server.getCurCall().set(mockCall); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) @SuppressWarnings ("rawtypes") public void testAppSubmit() throws Exception { ResourceScheduler scheduler = mockResourceScheduler(); @@ -1244,14 +1256,14 @@ public void handle(Event event) {} try { rmService.submitApplication(submitRequest1); } catch (YarnException e) { - Assert.fail("Exception is not expected."); + fail("Exception is not expected."); } RMApp app1 = rmContext.getRMApps().get(appId1); - Assert.assertNotNull("app doesn't exist", app1); - Assert.assertEquals("app name doesn't match", - YarnConfiguration.DEFAULT_APPLICATION_NAME, app1.getName()); - Assert.assertEquals("app queue doesn't match", - YarnConfiguration.DEFAULT_QUEUE_NAME, app1.getQueue()); + assertNotNull(app1, "app doesn't exist"); + assertEquals(YarnConfiguration.DEFAULT_APPLICATION_NAME, app1.getName(), + "app name doesn't match"); + assertEquals(YarnConfiguration.DEFAULT_QUEUE_NAME, app1.getQueue(), + "app queue doesn't match"); // with name and queue String name = MockApps.newAppName(); @@ -1264,25 +1276,25 @@ public void handle(Event event) {} try { rmService.submitApplication(submitRequest2); } catch (YarnException e) { - Assert.fail("Exception is not expected."); + fail("Exception is not expected."); } RMApp app2 = rmContext.getRMApps().get(appId2); - Assert.assertNotNull("app doesn't exist", app2); - Assert.assertEquals("app name doesn't match", name, app2.getName()); - Assert.assertEquals("app queue doesn't match", queue, app2.getQueue()); + assertNotNull(app2, "app doesn't exist"); + assertEquals(name, app2.getName(), "app name doesn't match"); + assertEquals(queue, app2.getQueue(), "app queue doesn't match"); // duplicate appId try { rmService.submitApplication(submitRequest2); } catch (YarnException e) { - Assert.fail("Exception is not expected."); + fail("Exception is not expected."); } GetApplicationsRequest getAllAppsRequest = GetApplicationsRequest.newInstance(new HashSet()); GetApplicationsResponse getAllApplicationsResponse = rmService.getApplications(getAllAppsRequest); - Assert.assertEquals(5, + assertEquals(5, getAllApplicationsResponse.getApplicationList().size()); Set appTypes = new HashSet(); @@ -1291,9 +1303,9 @@ public void handle(Event event) {} getAllAppsRequest = GetApplicationsRequest.newInstance(appTypes); getAllApplicationsResponse = rmService.getApplications(getAllAppsRequest); - Assert.assertEquals(1, + assertEquals(1, getAllApplicationsResponse.getApplicationList().size()); - Assert.assertEquals(appId2, + assertEquals(appId2, getAllApplicationsResponse.getApplicationList() .get(0).getApplicationId()); @@ -1303,9 +1315,9 @@ public void handle(Event event) {} getAllAppsRequest = GetApplicationsRequest.newInstance(appTypes); getAllApplicationsResponse = rmService.getApplications(getAllAppsRequest); - Assert.assertEquals(1, + assertEquals(1, getAllApplicationsResponse.getApplicationList().size()); - Assert.assertEquals(appId2, + assertEquals(appId2, getAllApplicationsResponse.getApplicationList() .get(0).getApplicationId()); } @@ -1370,31 +1382,33 @@ public void handle(Event event) {} // Test different cases of ClientRMService#getApplications() GetApplicationsRequest request = GetApplicationsRequest.newInstance(); - assertEquals("Incorrect total number of apps", 6, - rmService.getApplications(request).getApplicationList().size()); + assertEquals(6, + rmService.getApplications(request).getApplicationList().size(), + "Incorrect total number of apps"); // Check limit request.setLimit(1L); - assertEquals("Failed to limit applications", 1, - rmService.getApplications(request).getApplicationList().size()); + assertEquals(1, + rmService.getApplications(request).getApplicationList().size(), + "Failed to limit applications"); // Check start range request = GetApplicationsRequest.newInstance(); request.setStartRange(submitTimeMillis[0] + 1, System.currentTimeMillis()); // 2 applications are submitted after first timeMills - assertEquals("Incorrect number of matching start range", - 2, rmService.getApplications(request).getApplicationList().size()); + assertEquals(2, rmService.getApplications(request).getApplicationList().size(), + "Incorrect number of matching start range"); // 1 application is submitted after the second timeMills request.setStartRange(submitTimeMillis[1] + 1, System.currentTimeMillis()); - assertEquals("Incorrect number of matching start range", - 1, rmService.getApplications(request).getApplicationList().size()); + assertEquals(1, rmService.getApplications(request).getApplicationList().size(), + "Incorrect number of matching start range"); // no application is submitted after the third timeMills request.setStartRange(submitTimeMillis[2] + 1, System.currentTimeMillis()); - assertEquals("Incorrect number of matching start range", - 0, rmService.getApplications(request).getApplicationList().size()); + assertEquals(0, rmService.getApplications(request).getApplicationList().size(), + "Incorrect number of matching start range"); // Check queue request = GetApplicationsRequest.newInstance(); @@ -1402,14 +1416,14 @@ public void handle(Event event) {} request.setQueues(queueSet); queueSet.add(queues[0]); - assertEquals("Incorrect number of applications in queue", 3, - rmService.getApplications(request).getApplicationList().size()); - assertEquals("Incorrect number of applications in queue", 3, - rmService.getApplications(request).getApplicationList().size()); + assertEquals(3, rmService.getApplications(request).getApplicationList().size(), + "Incorrect number of applications in queue"); + assertEquals(3, rmService.getApplications(request).getApplicationList().size(), + "Incorrect number of applications in queue"); queueSet.add(queues[1]); - assertEquals("Incorrect number of applications in queue", 3, - rmService.getApplications(request).getApplicationList().size()); + assertEquals(3, rmService.getApplications(request).getApplicationList().size(), + "Incorrect number of applications in queue"); // Check user request = GetApplicationsRequest.newInstance(); @@ -1417,17 +1431,17 @@ public void handle(Event event) {} request.setUsers(userSet); userSet.add("random-user-name"); - assertEquals("Incorrect number of applications for user", 0, - rmService.getApplications(request).getApplicationList().size()); + assertEquals(0, rmService.getApplications(request).getApplicationList().size(), + "Incorrect number of applications for user"); userSet.add(UserGroupInformation.getCurrentUser().getShortUserName()); - assertEquals("Incorrect number of applications for user", 3, - rmService.getApplications(request).getApplicationList().size()); + assertEquals(3, rmService.getApplications(request).getApplicationList().size(), + "Incorrect number of applications for user"); rmService.setDisplayPerUserApps(true); userSet.clear(); - assertEquals("Incorrect number of applications for user", 6, - rmService.getApplications(request).getApplicationList().size()); + assertEquals(6, rmService.getApplications(request).getApplicationList().size(), + "Incorrect number of applications for user"); rmService.setDisplayPerUserApps(false); // Check tags @@ -1436,37 +1450,38 @@ public void handle(Event event) {} null, null); Set tagSet = new HashSet(); request.setApplicationTags(tagSet); - assertEquals("Incorrect number of matching tags", 6, - rmService.getApplications(request).getApplicationList().size()); + assertEquals(6, rmService.getApplications(request).getApplicationList().size(), + "Incorrect number of matching tags"); tagSet = Sets.newHashSet(tags.get(0)); request.setApplicationTags(tagSet); - assertEquals("Incorrect number of matching tags", 3, - rmService.getApplications(request).getApplicationList().size()); + assertEquals(3, rmService.getApplications(request).getApplicationList().size(), + "Incorrect number of matching tags"); tagSet = Sets.newHashSet(tags.get(1)); request.setApplicationTags(tagSet); - assertEquals("Incorrect number of matching tags", 2, - rmService.getApplications(request).getApplicationList().size()); + assertEquals(2, rmService.getApplications(request).getApplicationList().size(), + "Incorrect number of matching tags"); tagSet = Sets.newHashSet(tags.get(2)); request.setApplicationTags(tagSet); - assertEquals("Incorrect number of matching tags", 1, - rmService.getApplications(request).getApplicationList().size()); + assertEquals(1, rmService.getApplications(request).getApplicationList().size(), + "Incorrect number of matching tags"); // Check scope request = GetApplicationsRequest.newInstance( ApplicationsRequestScope.VIEWABLE); - assertEquals("Incorrect number of applications for the scope", 6, - rmService.getApplications(request).getApplicationList().size()); + assertEquals(6, rmService.getApplications(request).getApplicationList().size(), + "Incorrect number of applications for the scope"); request = GetApplicationsRequest.newInstance( ApplicationsRequestScope.OWN); - assertEquals("Incorrect number of applications for the scope", 3, - rmService.getApplications(request).getApplicationList().size()); + assertEquals(3, rmService.getApplications(request).getApplicationList().size(), + "Incorrect number of applications for the scope"); } - @Test(timeout=4000) + @Test + @Timeout(value = 4) public void testConcurrentAppSubmit() throws IOException, InterruptedException, BrokenBarrierException, YarnException { @@ -1753,7 +1768,7 @@ private ResourceManager setupResourceManager() { // allow plan follower to synchronize Thread.sleep(1050); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } return rm; } @@ -1769,7 +1784,7 @@ private ReservationSubmissionRequest submitReservationTestHelper( reservationID = clientService.getNewReservation(newReservationRequest) .getReservationId(); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } ReservationSubmissionRequest sRequest = ReservationSystemTestUtil.createSimpleReservationRequest(reservationID, @@ -1777,10 +1792,10 @@ private ReservationSubmissionRequest submitReservationTestHelper( try { sResponse = clientService.submitReservation(sRequest); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } - Assert.assertNotNull(sResponse); - Assert.assertNotNull(reservationID); + assertNotNull(sResponse); + assertNotNull(reservationID); System.out.println("Submit reservation response: " + reservationID); return sRequest; } @@ -1801,7 +1816,7 @@ public void testCreateReservation() { try { clientService.submitReservation(sRequest); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } // Submit the reservation with the same reservation id but different @@ -1812,11 +1827,11 @@ public void testCreateReservation() { sRequest.setReservationDefinition(rDef); try { clientService.submitReservation(sRequest); - Assert.fail("Reservation submission should fail if a duplicate " + fail("Reservation submission should fail if a duplicate " + "reservation id is used, but the reservation definition has been " + "updated."); } catch (Exception e) { - Assert.assertTrue(e instanceof YarnException); + assertTrue(e instanceof YarnException); } } @@ -1848,9 +1863,9 @@ public void testUpdateReservation() { try { uResponse = clientService.updateReservation(uRequest); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } - Assert.assertNotNull(uResponse); + assertNotNull(uResponse); System.out.println("Update reservation response: " + uResponse); } @@ -1873,13 +1888,13 @@ public void testListReservationsByReservationId() { try { response = clientService.listReservations(request); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } - Assert.assertNotNull(response); - Assert.assertEquals(1, response.getReservationAllocationState().size()); - Assert.assertEquals(response.getReservationAllocationState().get(0) + assertNotNull(response); + assertEquals(1, response.getReservationAllocationState().size()); + assertEquals(response.getReservationAllocationState().get(0) .getReservationId().getId(), reservationID.getId()); - Assert.assertEquals(response.getReservationAllocationState().get(0) + assertEquals(response.getReservationAllocationState().get(0) .getResourceAllocationRequests().size(), 0); } @@ -1906,11 +1921,11 @@ public void testListReservationsByTimeInterval() { try { response = clientService.listReservations(request); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } - Assert.assertNotNull(response); - Assert.assertEquals(1, response.getReservationAllocationState().size()); - Assert.assertEquals(response.getReservationAllocationState().get(0) + assertNotNull(response); + assertEquals(1, response.getReservationAllocationState().size()); + assertEquals(response.getReservationAllocationState().get(0) .getReservationId().getId(), reservationID.getId()); // List reservations, search by time within reservation interval. request = ReservationListRequest.newInstance( @@ -1920,23 +1935,23 @@ public void testListReservationsByTimeInterval() { try { response = clientService.listReservations(request); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } - Assert.assertNotNull(response); - Assert.assertEquals(1, response.getReservationAllocationState().size()); - Assert.assertEquals(response.getReservationAllocationState().get(0) + assertNotNull(response); + assertEquals(1, response.getReservationAllocationState().size()); + assertEquals(response.getReservationAllocationState().get(0) .getReservationId().getId(), reservationID.getId()); // Verify that the full resource allocations exist. - Assert.assertTrue(response.getReservationAllocationState().get(0) + assertTrue(response.getReservationAllocationState().get(0) .getResourceAllocationRequests().size() > 0); // Verify that the full RDL is returned. ReservationRequests reservationRequests = response.getReservationAllocationState().get(0) .getReservationDefinition().getReservationRequests(); - Assert.assertEquals("R_ALL", + assertEquals("R_ALL", reservationRequests.getInterpreter().toString()); - Assert.assertTrue(reservationRequests.getReservationResources().get(0) + assertTrue(reservationRequests.getReservationResources().get(0) .getDuration() == duration); } @@ -1959,11 +1974,11 @@ public void testListReservationsByInvalidTimeInterval() { try { response = clientService.listReservations(request); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } - Assert.assertNotNull(response); - Assert.assertEquals(1, response.getReservationAllocationState().size()); - Assert.assertEquals(response.getReservationAllocationState().get(0) + assertNotNull(response); + assertEquals(1, response.getReservationAllocationState().size()); + assertEquals(response.getReservationAllocationState().get(0) .getReservationId().getId(), sRequest.getReservationId().getId()); // List reservations, search by invalid end time < -1. @@ -1974,11 +1989,11 @@ public void testListReservationsByInvalidTimeInterval() { try { response = clientService.listReservations(request); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } - Assert.assertNotNull(response); - Assert.assertEquals(1, response.getReservationAllocationState().size()); - Assert.assertEquals(response.getReservationAllocationState().get(0) + assertNotNull(response); + assertEquals(1, response.getReservationAllocationState().size()); + assertEquals(response.getReservationAllocationState().get(0) .getReservationId().getId(), sRequest.getReservationId().getId()); } @@ -2001,11 +2016,11 @@ public void testListReservationsByTimeIntervalContainingNoReservations() { try { response = clientService.listReservations(request); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } // Ensure all reservations are filtered out. - Assert.assertNotNull(response); + assertNotNull(response); assertThat(response.getReservationAllocationState()).isEmpty(); duration = 30000; @@ -2021,11 +2036,11 @@ public void testListReservationsByTimeIntervalContainingNoReservations() { try { response = clientService.listReservations(request); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } // Ensure all reservations are filtered out. - Assert.assertNotNull(response); + assertNotNull(response); assertThat(response.getReservationAllocationState()).isEmpty(); arrival = clock.getTime(); @@ -2039,11 +2054,11 @@ public void testListReservationsByTimeIntervalContainingNoReservations() { try { response = clientService.listReservations(request); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } // Ensure all reservations are filtered out. - Assert.assertNotNull(response); + assertNotNull(response); assertThat(response.getReservationAllocationState()).isEmpty(); // List reservations, search by very small end time. @@ -2054,11 +2069,11 @@ public void testListReservationsByTimeIntervalContainingNoReservations() { try { response = clientService.listReservations(request); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } // Ensure all reservations are filtered out. - Assert.assertNotNull(response); + assertNotNull(response); assertThat(response.getReservationAllocationState()).isEmpty(); } @@ -2081,9 +2096,9 @@ public void testReservationDelete() { try { dResponse = clientService.deleteReservation(dRequest); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } - Assert.assertNotNull(dResponse); + assertNotNull(dResponse); System.out.println("Delete reservation response: " + dResponse); // List reservations, search by non-existent reservationID @@ -2095,10 +2110,10 @@ public void testReservationDelete() { try { response = clientService.listReservations(request); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } - Assert.assertNotNull(response); - Assert.assertEquals(0, response.getReservationAllocationState().size()); + assertNotNull(response); + assertEquals(0, response.getReservationAllocationState().size()); } @Test @@ -2136,21 +2151,21 @@ protected ClientRMService createClientRMService() { // Get node labels collection GetClusterNodeLabelsResponse response = client .getClusterNodeLabels(GetClusterNodeLabelsRequest.newInstance()); - Assert.assertTrue(response.getNodeLabelList().containsAll( + assertTrue(response.getNodeLabelList().containsAll( Arrays.asList(labelX, labelY))); // Get node labels mapping GetNodesToLabelsResponse response1 = client .getNodeToLabels(GetNodesToLabelsRequest.newInstance()); Map> nodeToLabels = response1.getNodeToLabels(); - Assert.assertTrue(nodeToLabels.keySet().containsAll( + assertTrue(nodeToLabels.keySet().containsAll( Arrays.asList(node1, node2))); - Assert.assertTrue(nodeToLabels.get(node1) + assertTrue(nodeToLabels.get(node1) .containsAll(Arrays.asList(labelX.getName()))); - Assert.assertTrue(nodeToLabels.get(node2) + assertTrue(nodeToLabels.get(node2) .containsAll(Arrays.asList(labelY.getName()))); // Below label "x" is not present in the response as exclusivity is true - Assert.assertFalse(nodeToLabels.get(node1).containsAll( + assertFalse(nodeToLabels.get(node1).containsAll( Arrays.asList(NodeLabel.newInstance("x")))); } @@ -2197,20 +2212,20 @@ protected ClientRMService createClientRMService() { // Get node labels collection GetClusterNodeLabelsResponse response = client .getClusterNodeLabels(GetClusterNodeLabelsRequest.newInstance()); - Assert.assertTrue(response.getNodeLabelList().containsAll( + assertTrue(response.getNodeLabelList().containsAll( Arrays.asList(labelX, labelY, labelZ))); // Get labels to nodes mapping GetLabelsToNodesResponse response1 = client .getLabelsToNodes(GetLabelsToNodesRequest.newInstance()); Map> labelsToNodes = response1.getLabelsToNodes(); - Assert.assertTrue(labelsToNodes.keySet().containsAll( + assertTrue(labelsToNodes.keySet().containsAll( Arrays.asList(labelX.getName(), labelY.getName(), labelZ.getName()))); - Assert.assertTrue(labelsToNodes.get(labelX.getName()).containsAll( + assertTrue(labelsToNodes.get(labelX.getName()).containsAll( Arrays.asList(node1A))); - Assert.assertTrue(labelsToNodes.get(labelY.getName()).containsAll( + assertTrue(labelsToNodes.get(labelY.getName()).containsAll( Arrays.asList(node2A, node3A))); - Assert.assertTrue(labelsToNodes.get(labelZ.getName()).containsAll( + assertTrue(labelsToNodes.get(labelZ.getName()).containsAll( Arrays.asList(node1B, node3B))); // Get labels to nodes mapping for specific labels @@ -2219,16 +2234,17 @@ protected ClientRMService createClientRMService() { GetLabelsToNodesResponse response2 = client .getLabelsToNodes(GetLabelsToNodesRequest.newInstance(setlabels)); labelsToNodes = response2.getLabelsToNodes(); - Assert.assertTrue(labelsToNodes.keySet().containsAll( + assertTrue(labelsToNodes.keySet().containsAll( Arrays.asList(labelX.getName(), labelZ.getName()))); - Assert.assertTrue(labelsToNodes.get(labelX.getName()).containsAll( + assertTrue(labelsToNodes.get(labelX.getName()).containsAll( Arrays.asList(node1A))); - Assert.assertTrue(labelsToNodes.get(labelZ.getName()).containsAll( + assertTrue(labelsToNodes.get(labelZ.getName()).containsAll( Arrays.asList(node1B, node3B))); assertThat(labelsToNodes.get(labelY.getName())).isNull(); } - @Test(timeout = 120000) + @Test + @Timeout(value = 120) public void testGetClusterNodeAttributes() throws IOException, YarnException { Configuration newConf = NodeAttributeTestUtils.getRandomDirConf(null); MockRM rm = new MockRM(newConf) { @@ -2270,14 +2286,14 @@ protected ClientRMService createClientRMService() { GetClusterNodeAttributesResponse response = client.getClusterNodeAttributes(request); Set attributes = response.getNodeAttributes(); - Assert.assertEquals("Size not correct", 3, attributes.size()); - Assert.assertTrue(attributes.contains(NodeAttributeInfo.newInstance(gpu))); - Assert.assertTrue(attributes.contains(NodeAttributeInfo.newInstance(os))); - Assert - .assertTrue(attributes.contains(NodeAttributeInfo.newInstance(docker))); + assertEquals(3, attributes.size(), "Size not correct"); + assertTrue(attributes.contains(NodeAttributeInfo.newInstance(gpu))); + assertTrue(attributes.contains(NodeAttributeInfo.newInstance(os))); + assertTrue(attributes.contains(NodeAttributeInfo.newInstance(docker))); } - @Test(timeout = 120000) + @Test + @Timeout(value = 120) public void testGetAttributesToNodes() throws IOException, YarnException { Configuration newConf = NodeAttributeTestUtils.getRandomDirConf(null); MockRM rm = new MockRM(newConf) { @@ -2327,11 +2343,11 @@ protected ClientRMService createClientRMService() { assertThat(attrs.get(dist.getAttributeKey())).hasSize(2); assertThat(attrs.get(os.getAttributeKey())).hasSize(1); assertThat(attrs.get(gpu.getAttributeKey())).hasSize(1); - Assert.assertTrue(findHostnameAndValInMapping(node1, "3_0_2", + assertTrue(findHostnameAndValInMapping(node1, "3_0_2", attrs.get(dist.getAttributeKey()))); - Assert.assertTrue(findHostnameAndValInMapping(node2, "3_0_2", + assertTrue(findHostnameAndValInMapping(node2, "3_0_2", attrs.get(dist.getAttributeKey()))); - Assert.assertTrue(findHostnameAndValInMapping(node2, "docker0", + assertTrue(findHostnameAndValInMapping(node2, "docker0", attrs.get(docker.getAttributeKey()))); GetAttributesToNodesRequest request2 = GetAttributesToNodesRequest @@ -2341,7 +2357,7 @@ protected ClientRMService createClientRMService() { Map> attrs2 = response2.getAttributesToNodes(); assertThat(attrs2).hasSize(1); - Assert.assertTrue(findHostnameAndValInMapping(node2, "docker0", + assertTrue(findHostnameAndValInMapping(node2, "docker0", attrs2.get(docker.getAttributeKey()))); GetAttributesToNodesRequest request3 = @@ -2352,9 +2368,9 @@ protected ClientRMService createClientRMService() { Map> attrs3 = response3.getAttributesToNodes(); assertThat(attrs3).hasSize(2); - Assert.assertTrue(findHostnameAndValInMapping(node1, "windows64", + assertTrue(findHostnameAndValInMapping(node1, "windows64", attrs3.get(os.getAttributeKey()))); - Assert.assertTrue(findHostnameAndValInMapping(node2, "docker0", + assertTrue(findHostnameAndValInMapping(node2, "docker0", attrs3.get(docker.getAttributeKey()))); } @@ -2368,7 +2384,8 @@ private boolean findHostnameAndValInMapping(String hostname, String attrVal, return false; } - @Test(timeout = 120000) + @Test + @Timeout(value = 120) public void testGetNodesToAttributes() throws IOException, YarnException { Configuration newConf = NodeAttributeTestUtils.getRandomDirConf(null); MockRM rm = new MockRM(newConf) { @@ -2415,11 +2432,11 @@ protected ClientRMService createClientRMService() { client.getNodesToAttributes(request1); Map> hostToAttrs = response1.getNodeToAttributes(); - Assert.assertEquals(2, hostToAttrs.size()); + assertEquals(2, hostToAttrs.size()); - Assert.assertTrue(hostToAttrs.get(node2).contains(dist)); - Assert.assertTrue(hostToAttrs.get(node2).contains(docker)); - Assert.assertTrue(hostToAttrs.get(node1).contains(dist)); + assertTrue(hostToAttrs.get(node2).contains(dist)); + assertTrue(hostToAttrs.get(node2).contains(docker)); + assertTrue(hostToAttrs.get(node1).contains(dist)); // Specify particular node GetNodesToAttributesRequest request2 = @@ -2427,8 +2444,8 @@ protected ClientRMService createClientRMService() { GetNodesToAttributesResponse response2 = client.getNodesToAttributes(request2); hostToAttrs = response2.getNodeToAttributes(); - Assert.assertEquals(1, response2.getNodeToAttributes().size()); - Assert.assertTrue(hostToAttrs.get(node1).contains(dist)); + assertEquals(1, response2.getNodeToAttributes().size()); + assertTrue(hostToAttrs.get(node1).contains(dist)); // Test queury with empty set GetNodesToAttributesRequest request3 = @@ -2436,11 +2453,11 @@ protected ClientRMService createClientRMService() { GetNodesToAttributesResponse response3 = client.getNodesToAttributes(request3); hostToAttrs = response3.getNodeToAttributes(); - Assert.assertEquals(2, hostToAttrs.size()); + assertEquals(2, hostToAttrs.size()); - Assert.assertTrue(hostToAttrs.get(node2).contains(dist)); - Assert.assertTrue(hostToAttrs.get(node2).contains(docker)); - Assert.assertTrue(hostToAttrs.get(node1).contains(dist)); + assertTrue(hostToAttrs.get(node2).contains(dist)); + assertTrue(hostToAttrs.get(node2).contains(docker)); + assertTrue(hostToAttrs.get(node1).contains(dist)); // test invalid hostname GetNodesToAttributesRequest request4 = @@ -2448,18 +2465,18 @@ protected ClientRMService createClientRMService() { GetNodesToAttributesResponse response4 = client.getNodesToAttributes(request4); hostToAttrs = response4.getNodeToAttributes(); - Assert.assertEquals(0, hostToAttrs.size()); + assertEquals(0, hostToAttrs.size()); } - @Test(timeout = 120000) + @Test + @Timeout(value = 120) public void testUpdatePriorityAndKillAppWithZeroClusterResource() throws Exception { int maxPriority = 10; int appPriority = 5; conf = new YarnConfiguration(); - Assume.assumeFalse("FairScheduler does not support Application Priorities", - conf.get(YarnConfiguration.RM_SCHEDULER) - .equals(FairScheduler.class.getName())); + assumeFalse(conf.get(YarnConfiguration.RM_SCHEDULER).equals(FairScheduler.class.getName()), + "FairScheduler does not support Application Priorities"); conf.setInt(YarnConfiguration.MAX_CLUSTER_LEVEL_APPLICATION_PRIORITY, maxPriority); MockRM rm = new MockRM(conf); @@ -2477,14 +2494,14 @@ public void testUpdatePriorityAndKillAppWithZeroClusterResource() rm.waitForState(app1.getApplicationId(), RMAppState.KILLED); } - @Test(timeout = 120000) + @Test + @Timeout(value = 120) public void testUpdateApplicationPriorityRequest() throws Exception { int maxPriority = 10; int appPriority = 5; conf = new YarnConfiguration(); - Assume.assumeFalse("FairScheduler does not support Application Priorities", - conf.get(YarnConfiguration.RM_SCHEDULER) - .equals(FairScheduler.class.getName())); + assumeFalse(conf.get(YarnConfiguration.RM_SCHEDULER).equals(FairScheduler.class.getName()), + "FairScheduler does not support Application Priorities"); conf.setInt(YarnConfiguration.MAX_CLUSTER_LEVEL_APPLICATION_PRIORITY, maxPriority); MockRM rm = new MockRM(conf); @@ -2499,8 +2516,8 @@ public void testUpdateApplicationPriorityRequest() throws Exception { .build(); RMApp app1 = MockRMAppSubmitter.submit(rm, data); - Assert.assertEquals("Incorrect priority has been set to application", - appPriority, app1.getApplicationPriority().getPriority()); + assertEquals(appPriority, app1.getApplicationPriority().getPriority(), + "Incorrect priority has been set to application"); appPriority = 11; ClientRMService rmService = rm.getClientRMService(); @@ -2520,7 +2537,7 @@ public void testUpdateApplicationPriorityRequest() throws Exception { Priority.newInstance(appPriority)); try { rmService.updateApplicationPriority(updateRequest); - Assert.fail("ApplicationNotFoundException should be thrown " + fail("ApplicationNotFoundException should be thrown " + "for invalid application id"); } catch (ApplicationNotFoundException e) { // Expected @@ -2529,9 +2546,9 @@ public void testUpdateApplicationPriorityRequest() throws Exception { updateRequest = UpdateApplicationPriorityRequest.newInstance(app1.getApplicationId(), Priority.newInstance(11)); - Assert.assertEquals("Incorrect priority has been set to application", - appPriority, rmService.updateApplicationPriority(updateRequest) - .getApplicationPriority().getPriority()); + assertEquals(appPriority, rmService.updateApplicationPriority(updateRequest) + .getApplicationPriority().getPriority(), + "Incorrect priority has been set to application"); } private void testApplicationPriorityUpdation(ClientRMService rmService, @@ -2544,11 +2561,10 @@ private void testApplicationPriorityUpdation(ClientRMService rmService, UpdateApplicationPriorityResponse updateApplicationPriority = rmService.updateApplicationPriority(updateRequest); - Assert.assertEquals("Incorrect priority has been set to application", - expected, app1.getApplicationSubmissionContext().getPriority() - .getPriority()); - Assert.assertEquals("Incorrect priority has been returned", expected, - updateApplicationPriority.getApplicationPriority().getPriority()); + assertEquals(expected, app1.getApplicationSubmissionContext().getPriority() + .getPriority(), "Incorrect priority has been set to application"); + assertEquals(expected, updateApplicationPriority.getApplicationPriority().getPriority(), + "Incorrect priority has been returned"); } private File createExcludeFile(File testDir) throws IOException { @@ -2562,7 +2578,8 @@ private File createExcludeFile(File testDir) throws IOException { @Test public void testRMStartWithDecommissionedNode() throws Exception { File testDir = GenericTestUtils.getRandomizedTestDir(); - assertTrue("Failed to create test directory: " + testDir.getAbsolutePath(), testDir.mkdirs()); + assertTrue(testDir.mkdirs(), + "Failed to create test directory: " + testDir.getAbsolutePath()); try { File excludeFile = createExcludeFile(testDir); conf = new YarnConfiguration(); @@ -2619,18 +2636,18 @@ protected ClientRMService createClientRMService() { GetAllResourceTypeInfoRequest.newInstance(); GetAllResourceTypeInfoResponse response = client.getResourceTypeInfo(request); - Assert.assertEquals(2, response.getResourceTypeInfo().size()); + assertEquals(2, response.getResourceTypeInfo().size()); // Check memory - Assert.assertEquals(ResourceInformation.MEMORY_MB.getName(), + assertEquals(ResourceInformation.MEMORY_MB.getName(), response.getResourceTypeInfo().get(0).getName()); - Assert.assertEquals(ResourceInformation.MEMORY_MB.getUnits(), + assertEquals(ResourceInformation.MEMORY_MB.getUnits(), response.getResourceTypeInfo().get(0).getDefaultUnit()); // Check vcores - Assert.assertEquals(ResourceInformation.VCORES.getName(), + assertEquals(ResourceInformation.VCORES.getName(), response.getResourceTypeInfo().get(1).getName()); - Assert.assertEquals(ResourceInformation.VCORES.getUnits(), + assertEquals(ResourceInformation.VCORES.getUnits(), response.getResourceTypeInfo().get(1).getDefaultUnit()); } @@ -2693,12 +2710,12 @@ public void handle(Event event) { // Test different cases of ClientRMService#getApplications() GetApplicationsRequest request = GetApplicationsRequest.newInstance(); - assertEquals("Incorrect total number of apps", 6, - rmService.getApplications(request).getApplicationList().size()); + assertEquals(6, rmService.getApplications(request).getApplicationList().size(), + "Incorrect total number of apps"); rmService.setDisplayPerUserApps(true); - assertEquals("Incorrect number of applications for user", 0, - rmService.getApplications(request).getApplicationList().size()); + assertEquals(0, rmService.getApplications(request).getApplicationList().size(), + "Incorrect number of applications for user"); rmService.setDisplayPerUserApps(false); } @@ -2752,30 +2769,30 @@ protected ClientRMService createClientRMService() { GetClusterNodesRequest.newInstance(EnumSet.of(NodeState.RUNNING)); List nodeReports = client.getClusterNodes(request).getNodeReports(); - Assert.assertEquals(1, nodeReports.size()); - Assert.assertNotSame("Node is expected to be healthy!", NodeState.UNHEALTHY, - nodeReports.get(0).getNodeState()); - Assert.assertEquals(1, nodeReports.size()); + assertEquals(1, nodeReports.size()); + assertNotSame(NodeState.UNHEALTHY, nodeReports.get(0).getNodeState(), + "Node is expected to be healthy!"); + assertEquals(1, nodeReports.size()); //Resource 'resource1' has been passed as 1T while registering NM. //1T should be converted to 1000G - Assert.assertEquals("G", nodeReports.get(0).getCapability(). + assertEquals("G", nodeReports.get(0).getCapability(). getResourceInformation("resource1").getUnits()); - Assert.assertEquals(1000, nodeReports.get(0).getCapability(). + assertEquals(1000, nodeReports.get(0).getCapability(). getResourceInformation("resource1").getValue()); //Resource 'resource2' has been passed as 1M while registering NM //1M should be converted to 1000000000M - Assert.assertEquals("m", nodeReports.get(0).getCapability(). + assertEquals("m", nodeReports.get(0).getCapability(). getResourceInformation("resource2").getUnits()); - Assert.assertEquals(1000000000, nodeReports.get(0).getCapability(). + assertEquals(1000000000, nodeReports.get(0).getCapability(). getResourceInformation("resource2").getValue()); //Resource 'memory-mb' has been passed as 976562G while registering NM //976562G should be converted to 976562Mi - Assert.assertEquals("Mi", nodeReports.get(0).getCapability(). + assertEquals("Mi", nodeReports.get(0).getCapability(). getResourceInformation("memory-mb").getUnits()); - Assert.assertEquals(976562, nodeReports.get(0).getCapability(). + assertEquals(976562, nodeReports.get(0).getCapability(). getResourceInformation("memory-mb").getValue()); } @@ -2811,17 +2828,17 @@ protected ClientRMService createClientRMService() { YarnClusterMetrics ymetrics = client.getClusterMetrics( GetClusterMetricsRequest.newInstance()).getClusterMetrics(); - Assert.assertEquals(0, ymetrics.getNumNodeManagers()); - Assert.assertEquals(1, ymetrics.getNumDecommissioningNodeManagers()); - Assert.assertEquals(2, ymetrics.getNumDecommissionedNodeManagers()); - Assert.assertEquals(3, ymetrics.getNumActiveNodeManagers()); - Assert.assertEquals(4, ymetrics.getNumLostNodeManagers()); - Assert.assertEquals(5, ymetrics.getNumUnhealthyNodeManagers()); - Assert.assertEquals(6, ymetrics.getNumRebootedNodeManagers()); - Assert.assertEquals(7, ymetrics.getNumShutdownNodeManagers()); + assertEquals(0, ymetrics.getNumNodeManagers()); + assertEquals(1, ymetrics.getNumDecommissioningNodeManagers()); + assertEquals(2, ymetrics.getNumDecommissionedNodeManagers()); + assertEquals(3, ymetrics.getNumActiveNodeManagers()); + assertEquals(4, ymetrics.getNumLostNodeManagers()); + assertEquals(5, ymetrics.getNumUnhealthyNodeManagers()); + assertEquals(6, ymetrics.getNumRebootedNodeManagers()); + assertEquals(7, ymetrics.getNumShutdownNodeManagers()); } - @After + @AfterEach public void tearDown() throws Exception { if (resourceTypesFile != null && resourceTypesFile.exists()) { resourceTypesFile.delete(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java index 928f5f506647d..30c30963bd96e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java @@ -17,8 +17,8 @@ package org.apache.hadoop.yarn.server.resourcemanager; -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.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; @@ -41,8 +41,7 @@ import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.RMDelegationTokenIdentifierData; -import org.junit.AfterClass; -import org.junit.Assert; +import org.junit.jupiter.api.AfterAll; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -78,9 +77,9 @@ import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; public class TestClientRMTokens { @@ -91,7 +90,7 @@ public class TestClientRMTokens { // Note : Any test case in ResourceManager package that creates a proxy has // to be run with enabling hadoop.security.token.service.use_ip. And reset // to false at the end of test class. See YARN-5208 - @BeforeClass + @BeforeAll public static void setUp() { Configuration conf = new Configuration(); conf.setBoolean( @@ -99,7 +98,7 @@ public static void setUp() { SecurityUtil.setConfiguration(conf); } - @AfterClass + @AfterAll public static void tearDown() { Configuration conf = new Configuration(); conf.setBoolean( @@ -107,7 +106,7 @@ public static void tearDown() { SecurityUtil.setConfiguration(conf); } - @Before + @BeforeEach public void resetSecretManager() { RMDelegationTokenIdentifier.Renewer.setSecretManager(null, null); } @@ -149,7 +148,7 @@ public void testDelegationToken() throws Exception { // Create a user for the renewr and fake the authentication-method UserGroupInformation loggedInUser = UserGroupInformation .createRemoteUser("testrenewer@APACHE.ORG"); - Assert.assertEquals("testrenewer", loggedInUser.getShortUserName()); + assertEquals("testrenewer", loggedInUser.getShortUserName()); // Default realm is APACHE.ORG loggedInUser.setAuthenticationMethod(AuthenticationMethod.KERBEROS); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClusterMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClusterMetrics.java index 645a2f1c40e59..c618dd730e96a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClusterMetrics.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClusterMetrics.java @@ -18,13 +18,14 @@ package org.apache.hadoop.yarn.server.resourcemanager; +import static org.junit.jupiter.api.Assertions.assertTrue; + import org.apache.hadoop.metrics2.MetricsSystem; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.test.GenericTestUtils; -import org.junit.After; -import org.junit.Before; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.util.function.Supplier; @@ -40,24 +41,24 @@ public class TestClusterMetrics { @Test public void testAmMetrics() throws Exception { assert(metrics != null); - Assert.assertTrue(!metrics.aMLaunchDelay.changed()); - Assert.assertTrue(!metrics.aMRegisterDelay.changed()); - Assert.assertTrue(!metrics.getAMContainerAllocationDelay().changed()); + assertTrue(!metrics.aMLaunchDelay.changed()); + assertTrue(!metrics.aMRegisterDelay.changed()); + assertTrue(!metrics.getAMContainerAllocationDelay().changed()); metrics.addAMLaunchDelay(1); metrics.addAMRegisterDelay(1); metrics.addAMContainerAllocationDelay(1); - Assert.assertTrue(metrics.aMLaunchDelay.changed()); - Assert.assertTrue(metrics.aMRegisterDelay.changed()); - Assert.assertTrue(metrics.getAMContainerAllocationDelay().changed()); + assertTrue(metrics.aMLaunchDelay.changed()); + assertTrue(metrics.aMRegisterDelay.changed()); + assertTrue(metrics.getAMContainerAllocationDelay().changed()); } - @Before + @BeforeEach public void setup() { DefaultMetricsSystem.initialize("ResourceManager"); metrics = ClusterMetrics.getMetrics(); } - @After + @AfterEach public void tearDown() { ClusterMetrics.destroy(); @@ -69,7 +70,7 @@ public void tearDown() { @Test public void testClusterMetrics() throws Exception { - Assert.assertTrue(!metrics.containerAssignedPerSecond.changed()); + assertTrue(!metrics.containerAssignedPerSecond.changed()); metrics.incrNumContainerAssigned(); metrics.incrNumContainerAssigned(); GenericTestUtils.waitFor(new Supplier() { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestContainerResourceUsage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestContainerResourceUsage.java index 54fd6509647e9..41fe96f327249 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestContainerResourceUsage.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestContainerResourceUsage.java @@ -18,6 +18,10 @@ package org.apache.hadoop.yarn.server.resourcemanager; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + import java.net.UnknownHostException; import java.util.ArrayList; import java.util.Collection; @@ -47,16 +51,16 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState; 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.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; public class TestContainerResourceUsage { private YarnConfiguration conf; - @Before + @BeforeEach public void setup() throws UnknownHostException { GenericTestUtils.setRootLogLevel(Level.DEBUG); conf = new YarnConfiguration(); @@ -65,11 +69,12 @@ public void setup() throws UnknownHostException { YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS); } - @After + @AfterEach public void tearDown() { } - @Test (timeout = 120000) + @Test + @Timeout(value = 120) public void testUsageWithOneAttemptAndOneContainer() throws Exception { MockRM rm = new MockRM(conf); rm.start(); @@ -81,14 +86,12 @@ public void testUsageWithOneAttemptAndOneContainer() throws Exception { RMApp app0 = MockRMAppSubmitter.submitWithMemory(200, rm); RMAppMetrics rmAppMetrics = app0.getRMAppMetrics(); - Assert.assertTrue( + assertTrue(rmAppMetrics.getMemorySeconds() == 0, "Before app submittion, memory seconds should have been 0 but was " - + rmAppMetrics.getMemorySeconds(), - rmAppMetrics.getMemorySeconds() == 0); - Assert.assertTrue( + + rmAppMetrics.getMemorySeconds()); + assertTrue(rmAppMetrics.getVcoreSeconds() == 0, "Before app submission, vcore seconds should have been 0 but was " - + rmAppMetrics.getVcoreSeconds(), - rmAppMetrics.getVcoreSeconds() == 0); + + rmAppMetrics.getVcoreSeconds()); RMAppAttempt attempt0 = app0.getCurrentAppAttempt(); @@ -109,29 +112,28 @@ public void testUsageWithOneAttemptAndOneContainer() throws Exception { } rmAppMetrics = app0.getRMAppMetrics(); - Assert.assertTrue( + assertTrue(rmAppMetrics.getMemorySeconds() > 0, "While app is running, memory seconds should be >0 but is " - + rmAppMetrics.getMemorySeconds(), - rmAppMetrics.getMemorySeconds() > 0); - Assert.assertTrue( + + rmAppMetrics.getMemorySeconds()); + assertTrue(rmAppMetrics.getVcoreSeconds() > 0, "While app is running, vcore seconds should be >0 but is " - + rmAppMetrics.getVcoreSeconds(), - rmAppMetrics.getVcoreSeconds() > 0); + + rmAppMetrics.getVcoreSeconds()); MockRM.finishAMAndVerifyAppState(app0, rm, nm, am0); AggregateAppResourceUsage ru = calculateContainerResourceMetrics(rmContainer); rmAppMetrics = app0.getRMAppMetrics(); - Assert.assertEquals("Unexpected MemorySeconds value", - ru.getMemorySeconds(), rmAppMetrics.getMemorySeconds()); - Assert.assertEquals("Unexpected VcoreSeconds value", - ru.getVcoreSeconds(), rmAppMetrics.getVcoreSeconds()); + assertEquals(ru.getMemorySeconds(), rmAppMetrics.getMemorySeconds(), + "Unexpected MemorySeconds value"); + assertEquals(ru.getVcoreSeconds(), rmAppMetrics.getVcoreSeconds(), + "Unexpected VcoreSeconds value"); rm.stop(); } - @Test (timeout = 120000) + @Test + @Timeout(value = 120) public void testUsageWithMultipleContainersAndRMRestart() throws Exception { // Set max attempts to 1 so that when the first attempt fails, the app // won't try to start a new one. @@ -223,12 +225,12 @@ public void testUsageWithMultipleContainersAndRMRestart() throws Exception { } RMAppMetrics metricsBefore = app0.getRMAppMetrics(); - Assert.assertEquals("Unexpected MemorySeconds value", - memorySeconds, metricsBefore.getMemorySeconds()); - Assert.assertEquals("Unexpected VcoreSeconds value", - vcoreSeconds, metricsBefore.getVcoreSeconds()); - Assert.assertEquals("Unexpected totalAllocatedContainers value", - NUM_CONTAINERS + 1, metricsBefore.getTotalAllocatedContainers()); + assertEquals(memorySeconds, metricsBefore.getMemorySeconds(), + "Unexpected MemorySeconds value"); + assertEquals(vcoreSeconds, metricsBefore.getVcoreSeconds(), + "Unexpected VcoreSeconds value"); + assertEquals(NUM_CONTAINERS + 1, metricsBefore.getTotalAllocatedContainers(), + "Unexpected totalAllocatedContainers value"); // create new RM to represent RM restart. Load up the state store. MockRM rm1 = new MockRM(conf, memStore); @@ -238,13 +240,14 @@ public void testUsageWithMultipleContainersAndRMRestart() throws Exception { // Compare container resource usage metrics from before and after restart. RMAppMetrics metricsAfter = app0After.getRMAppMetrics(); - Assert.assertEquals("Vcore seconds were not the same after RM Restart", - metricsBefore.getVcoreSeconds(), metricsAfter.getVcoreSeconds()); - Assert.assertEquals("Memory seconds were not the same after RM Restart", - metricsBefore.getMemorySeconds(), metricsAfter.getMemorySeconds()); - Assert.assertEquals("TotalAllocatedContainers was not the same after " + - "RM Restart", metricsBefore.getTotalAllocatedContainers(), - metricsAfter.getTotalAllocatedContainers()); + assertEquals(metricsBefore.getVcoreSeconds(), metricsAfter.getVcoreSeconds(), + "Vcore seconds were not the same after RM Restart"); + assertEquals(metricsBefore.getMemorySeconds(), metricsAfter.getMemorySeconds(), + "Memory seconds were not the same after RM Restart"); + assertEquals(metricsBefore.getTotalAllocatedContainers(), + metricsAfter.getTotalAllocatedContainers(), + "TotalAllocatedContainers was not the same after " + + "RM Restart"); rm0.stop(); rm0.close(); @@ -252,12 +255,14 @@ public void testUsageWithMultipleContainersAndRMRestart() throws Exception { rm1.close(); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testUsageAfterAMRestartWithMultipleContainers() throws Exception { amRestartTests(false); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testUsageAfterAMRestartKeepContainers() throws Exception { amRestartTests(true); } @@ -339,9 +344,9 @@ private void amRestartTests(boolean keepRunningContainers) vcoreSeconds += ru.getVcoreSeconds(); } else { // The remaining container should be RUNNING. - Assert.assertTrue("After first attempt failed, remaining container " - + "should still be running. ", - c.getContainerState().equals(ContainerState.RUNNING)); + assertTrue(c.getContainerState().equals(ContainerState.RUNNING), + "After first attempt failed, remaining container " + + "should still be running. "); } } } else { @@ -360,7 +365,7 @@ private void amRestartTests(boolean keepRunningContainers) // assert this is a new AM. RMAppAttempt attempt2 = app.getCurrentAppAttempt(); - Assert.assertFalse(attempt2.getAppAttemptId() + assertFalse(attempt2.getAppAttemptId() .equals(am0.getApplicationAttemptId())); rm.waitForState(attempt2.getAppAttemptId(), RMAppAttemptState.SCHEDULED); @@ -408,10 +413,10 @@ private void amRestartTests(boolean keepRunningContainers) RMAppMetrics rmAppMetrics = app.getRMAppMetrics(); - Assert.assertEquals("Unexpected MemorySeconds value", - memorySeconds, rmAppMetrics.getMemorySeconds()); - Assert.assertEquals("Unexpected VcoreSeconds value", - vcoreSeconds, rmAppMetrics.getVcoreSeconds()); + assertEquals(memorySeconds, rmAppMetrics.getMemorySeconds(), + "Unexpected MemorySeconds value"); + assertEquals(vcoreSeconds, rmAppMetrics.getVcoreSeconds(), + "Unexpected VcoreSeconds value"); rm.stop(); return; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestDecommissioningNodesWatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestDecommissioningNodesWatcher.java index c662b55e8db4a..9785518876f56 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestDecommissioningNodesWatcher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestDecommissioningNodesWatcher.java @@ -18,6 +18,9 @@ package org.apache.hadoop.yarn.server.resourcemanager; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; + import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -37,9 +40,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStatusEvent; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; /** * This class tests DecommissioningNodesWatcher. @@ -84,25 +86,25 @@ public void testDecommissioningNodesWatcher() throws Exception { nodeStatus = createNodeStatus(id1, app, 2); node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus)); watcher.update(node1, nodeStatus); - Assert.assertFalse(watcher.checkReadyToBeDecommissioned(id1)); + assertFalse(watcher.checkReadyToBeDecommissioned(id1)); nodeStatus = createNodeStatus(id1, app, 1); node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus)); watcher.update(node1, nodeStatus); - Assert.assertEquals(DecommissioningNodeStatus.WAIT_CONTAINER, + assertEquals(DecommissioningNodeStatus.WAIT_CONTAINER, watcher.checkDecommissioningStatus(id1)); nodeStatus = createNodeStatus(id1, app, 0); watcher.update(node1, nodeStatus); node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus)); - Assert.assertEquals(DecommissioningNodeStatus.WAIT_APP, + assertEquals(DecommissioningNodeStatus.WAIT_APP, watcher.checkDecommissioningStatus(id1)); // Set app to be FINISHED and verified DecommissioningNodeStatus is READY. MockRM.finishAMAndVerifyAppState(app, rm, nm1, am); rm.waitForState(app.getApplicationId(), RMAppState.FINISHED); watcher.update(node1, nodeStatus); - Assert.assertEquals(DecommissioningNodeStatus.READY, + assertEquals(DecommissioningNodeStatus.READY, watcher.checkDecommissioningStatus(id1)); } @@ -132,13 +134,13 @@ public void testDecommissioningNodesWatcherWithPreviousRunningApps() NodeStatus nodeStatus = createNodeStatus(id1, app, 3); node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus)); - Assert.assertEquals(1, node1.getRunningApps().size()); + assertEquals(1, node1.getRunningApps().size()); // update node with 0 running containers nodeStatus = createNodeStatus(id1, app, 0); node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus)); - Assert.assertEquals(1, node1.getRunningApps().size()); + assertEquals(1, node1.getRunningApps().size()); // Setup nm1 as DECOMMISSIONING for DecommissioningNodesWatcher. Right now // there is no container running on the node. @@ -149,20 +151,20 @@ public void testDecommissioningNodesWatcherWithPreviousRunningApps() // we should still get WAIT_APP as container for a running app previously // ran on this node. watcher.update(node1, nodeStatus); - Assert.assertFalse(watcher.checkReadyToBeDecommissioned(id1)); - Assert.assertEquals(DecommissioningNodeStatus.WAIT_APP, + assertFalse(watcher.checkReadyToBeDecommissioned(id1)); + assertEquals(DecommissioningNodeStatus.WAIT_APP, watcher.checkDecommissioningStatus(id1)); // Set app to be FINISHED and verified DecommissioningNodeStatus is READY. MockRM.finishAMAndVerifyAppState(app, rm, nm1, am); rm.waitForState(app.getApplicationId(), RMAppState.FINISHED); - Assert.assertEquals(0, node1.getRunningApps().size()); + assertEquals(0, node1.getRunningApps().size()); watcher.update(node1, nodeStatus); - Assert.assertEquals(DecommissioningNodeStatus.READY, + assertEquals(DecommissioningNodeStatus.READY, watcher.checkDecommissioningStatus(id1)); } - @After + @AfterEach public void tearDown() { if (rm != null) { rm.stop(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestLeaderElectorService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestLeaderElectorService.java index a5762680942d0..a7e7253171fe1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestLeaderElectorService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestLeaderElectorService.java @@ -33,9 +33,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData; import org.slf4j.event.Level; import org.apache.zookeeper.ZooKeeper; -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.IOException; import java.util.Collection; @@ -54,7 +55,7 @@ public class TestLeaderElectorService { MockRM rm1; MockRM rm2; TestingCluster zkCluster; - @Before + @BeforeEach public void setUp() throws Exception { GenericTestUtils.setRootLogLevel(Level.INFO); conf = new Configuration(); @@ -73,7 +74,7 @@ public void setUp() throws Exception { zkCluster.start(); } - @After + @AfterEach public void tearDown() throws Exception { if (rm1 != null) { rm1.stop(); @@ -87,7 +88,8 @@ public void tearDown() throws Exception { // 2. rm2 standby // 3. stop rm1 // 4. rm2 become active - @Test (timeout = 20000) + @Test + @Timeout(value = 20) public void testRMShutDownCauseFailover() throws Exception { rm1 = startRM("rm1", HAServiceState.ACTIVE); rm2 = startRM("rm2", HAServiceState.STANDBY); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestMoveApplication.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestMoveApplication.java index a078fb2ed7b63..36657bbc5fede 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestMoveApplication.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestMoveApplication.java @@ -18,8 +18,8 @@ package org.apache.hadoop.yarn.server.resourcemanager; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.fail; import java.security.AccessControlException; import java.security.PrivilegedExceptionAction; @@ -36,16 +36,17 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler; -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; public class TestMoveApplication { private ResourceManager resourceManager = null; private static boolean failMove; private Configuration conf; - @Before + @BeforeEach public void setUp() throws Exception { conf = new YarnConfiguration(); conf.setClass(YarnConfiguration.RM_SCHEDULER, FifoSchedulerWithMove.class, @@ -59,7 +60,7 @@ public void setUp() throws Exception { failMove = false; } - @After + @AfterEach public void tearDown() { resourceManager.stop(); } @@ -91,7 +92,8 @@ public void testMoveRejectedByScheduler() throws Exception { } } - @Test (timeout = 10000) + @Test + @Timeout(value = 10) public void testMoveTooLate() throws Exception { // Submit application Application application = new Application("user1", resourceManager); @@ -119,8 +121,9 @@ public void testMoveTooLate() throws Exception { } } - @Test (timeout = 10000) - public + @Test + @Timeout(value = 10) + public void testMoveSuccessful() throws Exception { MockRM rm1 = new MockRM(conf); rm1.start(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestNodeBlacklistingOnAMFailures.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestNodeBlacklistingOnAMFailures.java index 8e35faff08d9a..590b9ae6535c0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestNodeBlacklistingOnAMFailures.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestNodeBlacklistingOnAMFailures.java @@ -18,6 +18,9 @@ package org.apache.hadoop.yarn.server.resourcemanager; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; + import java.util.ArrayList; import java.util.List; @@ -48,9 +51,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; /** * Validate system behavior when the am-scheduling logic 'blacklists' a node for @@ -87,13 +90,14 @@ private RMApp submitAppWithAMResourceRequests(MockRM rm, return MockRMAppSubmitter.submit(rm, data); } - @Before + @BeforeEach public void setup() { QueueMetrics.clearQueueMetrics(); DefaultMetricsSystem.setMiniClusterMode(true); } - @Test(timeout = 100000) + @Test + @Timeout(value = 100) public void testNodeBlacklistingOnAMFailure() throws Exception { YarnConfiguration conf = new YarnConfiguration(); @@ -160,10 +164,9 @@ public void testNodeBlacklistingOnAMFailure() throws Exception { currentNode.nodeHeartbeat(true); rm.drainEvents(); - Assert.assertEquals( + assertEquals(RMAppAttemptState.SCHEDULED, attempt.getAppAttemptState(), "AppAttemptState should still be SCHEDULED if currentNode is " - + "blacklisted correctly", RMAppAttemptState.SCHEDULED, - attempt.getAppAttemptState()); + + "blacklisted correctly"); } // Now try the other node @@ -181,22 +184,21 @@ public void testNodeBlacklistingOnAMFailure() throws Exception { nodeWhereAMRan = rmContainer.getAllocatedNode(); // The other node should now receive the assignment - Assert.assertEquals( - "After blacklisting, AM should have run on the other node", - otherNode.getNodeId(), nodeWhereAMRan); + assertEquals(otherNode.getNodeId(), nodeWhereAMRan, + "After blacklisting, AM should have run on the other node"); am2.registerAppAttempt(); rm.waitForState(app.getApplicationId(), RMAppState.RUNNING); List allocatedContainers = TestAMRestart.allocateContainers(currentNode, am2, 1); - Assert.assertEquals( + assertEquals(currentNode.getNodeId(), allocatedContainers.get(0).getNodeId(), "Even though AM is blacklisted from the node, application can " - + "still allocate non-AM containers there", - currentNode.getNodeId(), allocatedContainers.get(0).getNodeId()); + + "still allocate non-AM containers there"); } - @Test(timeout = 100000) + @Test + @Timeout(value = 100) public void testNodeBlacklistingOnAMFailureStrictNodeLocality() throws Exception { YarnConfiguration conf = new YarnConfiguration(); @@ -251,7 +253,7 @@ public void testNodeBlacklistingOnAMFailureStrictNodeLocality() ContainerId.newContainerId(am1.getApplicationAttemptId(), 1); RMContainer rmContainer = scheduler.getRMContainer(amContainerId); NodeId nodeWhereAMRan = rmContainer.getAllocatedNode(); - Assert.assertEquals(nm2.getNodeId(), nodeWhereAMRan); + assertEquals(nm2.getNodeId(), nodeWhereAMRan); // Set the exist status to INVALID so that we can verify that the system // automatically blacklisting the node @@ -277,13 +279,14 @@ public void testNodeBlacklistingOnAMFailureStrictNodeLocality() // The second AM should be on the same node because the strict locality // made the eligible nodes only 1, so the blacklisting threshold kicked in System.out.println("AM ran on " + nodeWhereAMRan); - Assert.assertEquals(nm2.getNodeId(), nodeWhereAMRan); + assertEquals(nm2.getNodeId(), nodeWhereAMRan); am2.registerAppAttempt(); rm.waitForState(app.getApplicationId(), RMAppState.RUNNING); } - @Test(timeout = 100000) + @Test + @Timeout(value = 100) public void testNodeBlacklistingOnAMFailureRelaxedNodeLocality() throws Exception { YarnConfiguration conf = new YarnConfiguration(); @@ -338,7 +341,7 @@ public void testNodeBlacklistingOnAMFailureRelaxedNodeLocality() ContainerId.newContainerId(am1.getApplicationAttemptId(), 1); RMContainer rmContainer = scheduler.getRMContainer(amContainerId); NodeId nodeWhereAMRan = rmContainer.getAllocatedNode(); - Assert.assertEquals(nm2.getNodeId(), nodeWhereAMRan); + assertEquals(nm2.getNodeId(), nodeWhereAMRan); // Set the exist status to INVALID so that we can verify that the system // automatically blacklisting the node @@ -368,13 +371,14 @@ public void testNodeBlacklistingOnAMFailureRelaxedNodeLocality() // The second AM should be on a different node because the relaxed locality // made the app schedulable on other nodes and nm2 is blacklisted System.out.println("AM ran on " + nodeWhereAMRan); - Assert.assertNotEquals(nm2.getNodeId(), nodeWhereAMRan); + assertNotEquals(nm2.getNodeId(), nodeWhereAMRan); am2.registerAppAttempt(); rm.waitForState(app.getApplicationId(), RMAppState.RUNNING); } - @Test(timeout = 100000) + @Test + @Timeout(value = 100) public void testNoBlacklistingForNonSystemErrors() throws Exception { YarnConfiguration conf = new YarnConfiguration(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestOpportunisticContainerAllocatorAMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestOpportunisticContainerAllocatorAMService.java index b99c456fb8173..4715b5147a6b2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestOpportunisticContainerAllocatorAMService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestOpportunisticContainerAllocatorAMService.java @@ -18,6 +18,11 @@ package org.apache.hadoop.yarn.server.resourcemanager; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.ipc.ProtobufRpcEngine2; import org.apache.hadoop.ipc.RPC; @@ -98,10 +103,10 @@ import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerContext; import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.io.IOException; import java.net.InetSocketAddress; @@ -124,7 +129,7 @@ public class TestOpportunisticContainerAllocatorAMService { private OpportunisticContainersStatus oppContainersStatus = getOpportunisticStatus(); - @Before + @BeforeEach public void createAndStartRM() { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); @@ -168,7 +173,7 @@ protected Dispatcher createDispatcher() { rm.start(); } - @After + @AfterEach public void stopRM() { if (rm != null) { rm.stop(); @@ -177,7 +182,9 @@ public void stopRM() { OpportunisticSchedulerMetrics.resetMetrics(); } - @Test(timeout = 600000) + @Test + @Timeout(value = 600) + @SuppressWarnings("checkstyle:MethodLength") public void testContainerPromoteAndDemoteBeforeContainerStart() throws Exception { HashMap nodes = new HashMap<>(); MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService()); @@ -236,7 +243,7 @@ public void testContainerPromoteAndDemoteBeforeContainerStart() throws Exception null); List allocatedContainers = allocateResponse .getAllocatedContainers(); - Assert.assertEquals(2, allocatedContainers.size()); + assertEquals(2, allocatedContainers.size()); Container container = allocatedContainers.get(0); MockNM allocNode = nodes.get(container.getNodeId()); MockNM sameHostDiffNode = null; @@ -258,7 +265,7 @@ public void testContainerPromoteAndDemoteBeforeContainerStart() throws Exception sameHostDiffNode.nodeHeartbeat(oppContainersStatus, true); rm.drainEvents(); allocateResponse = am1.allocate(new ArrayList<>(), new ArrayList<>()); - Assert.assertEquals(0, allocateResponse.getUpdatedContainers().size()); + assertEquals(0, allocateResponse.getUpdatedContainers().size()); // Wait for scheduler to process all events dispatcher.waitForEventThreadToWait(); @@ -271,11 +278,11 @@ public void testContainerPromoteAndDemoteBeforeContainerStart() throws Exception Arrays.asList(UpdateContainerRequest.newInstance(0, container.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE, null, ExecutionType.GUARANTEED))); - Assert.assertEquals(0, allocateResponse.getUpdatedContainers().size()); - Assert.assertEquals(1, allocateResponse.getUpdateErrors().size()); - Assert.assertEquals("UPDATE_OUTSTANDING_ERROR", + assertEquals(0, allocateResponse.getUpdatedContainers().size()); + assertEquals(1, allocateResponse.getUpdateErrors().size()); + assertEquals("UPDATE_OUTSTANDING_ERROR", allocateResponse.getUpdateErrors().get(0).getReason()); - Assert.assertEquals(container.getId(), + assertEquals(container.getId(), allocateResponse.getUpdateErrors().get(0) .getUpdateContainerRequest().getContainerId()); @@ -286,14 +293,14 @@ public void testContainerPromoteAndDemoteBeforeContainerStart() throws Exception container.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE, null, ExecutionType.GUARANTEED))); - Assert.assertEquals(0, allocateResponse.getUpdatedContainers().size()); - Assert.assertEquals(1, allocateResponse.getUpdateErrors().size()); - Assert.assertEquals("INCORRECT_CONTAINER_VERSION_ERROR", + assertEquals(0, allocateResponse.getUpdatedContainers().size()); + assertEquals(1, allocateResponse.getUpdateErrors().size()); + assertEquals("INCORRECT_CONTAINER_VERSION_ERROR", allocateResponse.getUpdateErrors().get(0).getReason()); - Assert.assertEquals(0, + assertEquals(0, allocateResponse.getUpdateErrors().get(0) .getCurrentContainerVersion()); - Assert.assertEquals(container.getId(), + assertEquals(container.getId(), allocateResponse.getUpdateErrors().get(0) .getUpdateContainerRequest().getContainerId()); @@ -301,12 +308,12 @@ public void testContainerPromoteAndDemoteBeforeContainerStart() throws Exception allocNode.nodeHeartbeat(oppContainersStatus, true); rm.drainEvents(); allocateResponse = am1.allocate(new ArrayList<>(), new ArrayList<>()); - Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size()); + assertEquals(1, allocateResponse.getUpdatedContainers().size()); Container uc = allocateResponse.getUpdatedContainers().get(0).getContainer(); - Assert.assertEquals(ExecutionType.GUARANTEED, uc.getExecutionType()); - Assert.assertEquals(uc.getId(), container.getId()); - Assert.assertEquals(uc.getVersion(), container.getVersion() + 1); + assertEquals(ExecutionType.GUARANTEED, uc.getExecutionType()); + assertEquals(uc.getId(), container.getId()); + assertEquals(uc.getVersion(), container.getVersion() + 1); // Verify Metrics After OPP allocation : // Allocated cores+mem should have increased, available should decrease @@ -322,7 +329,7 @@ public void testContainerPromoteAndDemoteBeforeContainerStart() throws Exception RMContainer rmContainer = ((CapacityScheduler) scheduler) .getApplicationAttempt( uc.getId().getApplicationAttemptId()).getRMContainer(uc.getId()); - Assert.assertEquals(RMContainerState.ACQUIRED, rmContainer.getState()); + assertEquals(RMContainerState.ACQUIRED, rmContainer.getState()); // Now demote the container back.. allocateResponse = am1.sendContainerUpdateRequest( @@ -330,11 +337,11 @@ public void testContainerPromoteAndDemoteBeforeContainerStart() throws Exception uc.getId(), ContainerUpdateType.DEMOTE_EXECUTION_TYPE, null, ExecutionType.OPPORTUNISTIC))); // This should happen in the same heartbeat.. - Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size()); + assertEquals(1, allocateResponse.getUpdatedContainers().size()); uc = allocateResponse.getUpdatedContainers().get(0).getContainer(); - Assert.assertEquals(ExecutionType.OPPORTUNISTIC, uc.getExecutionType()); - Assert.assertEquals(uc.getId(), container.getId()); - Assert.assertEquals(uc.getVersion(), container.getVersion() + 2); + assertEquals(ExecutionType.OPPORTUNISTIC, uc.getExecutionType()); + assertEquals(uc.getId(), container.getId()); + assertEquals(uc.getVersion(), container.getVersion() + 2); // Wait for scheduler to finish processing events dispatcher.waitForEventThreadToWait(); @@ -344,7 +351,8 @@ public void testContainerPromoteAndDemoteBeforeContainerStart() throws Exception verifyMetrics(metrics, 15360, 15, 1024, 1, 1); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testContainerPromoteAfterContainerStart() throws Exception { HashMap nodes = new HashMap<>(); MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService()); @@ -393,7 +401,7 @@ public void testContainerPromoteAfterContainerStart() throws Exception { null); List allocatedContainers = allocateResponse .getAllocatedContainers(); - Assert.assertEquals(2, allocatedContainers.size()); + assertEquals(2, allocatedContainers.size()); Container container = allocatedContainers.get(0); MockNM allocNode = nodes.get(container.getNodeId()); @@ -409,7 +417,7 @@ public void testContainerPromoteAfterContainerStart() throws Exception { .getApplicationAttempt( container.getId().getApplicationAttemptId()).getRMContainer( container.getId()); - Assert.assertEquals(RMContainerState.RUNNING, rmContainer.getState()); + assertEquals(RMContainerState.RUNNING, rmContainer.getState()); // Verify Metrics After OPP allocation (Nothing should change) verifyMetrics(metrics, 7168, 7, 1024, 1, 1); @@ -427,11 +435,11 @@ public void testContainerPromoteAfterContainerStart() throws Exception { Arrays.asList(UpdateContainerRequest.newInstance(0, container.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE, null, ExecutionType.GUARANTEED))); - Assert.assertEquals(0, allocateResponse.getUpdatedContainers().size()); - Assert.assertEquals(1, allocateResponse.getUpdateErrors().size()); - Assert.assertEquals("UPDATE_OUTSTANDING_ERROR", + assertEquals(0, allocateResponse.getUpdatedContainers().size()); + assertEquals(1, allocateResponse.getUpdateErrors().size()); + assertEquals("UPDATE_OUTSTANDING_ERROR", allocateResponse.getUpdateErrors().get(0).getReason()); - Assert.assertEquals(container.getId(), + assertEquals(container.getId(), allocateResponse.getUpdateErrors().get(0) .getUpdateContainerRequest().getContainerId()); @@ -443,25 +451,26 @@ public void testContainerPromoteAfterContainerStart() throws Exception { rm.drainEvents(); allocateResponse = am1.allocate(new ArrayList<>(), new ArrayList<>()); - Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size()); + assertEquals(1, allocateResponse.getUpdatedContainers().size()); Container uc = allocateResponse.getUpdatedContainers().get(0).getContainer(); - Assert.assertEquals(ExecutionType.GUARANTEED, uc.getExecutionType()); - Assert.assertEquals(uc.getId(), container.getId()); - Assert.assertEquals(uc.getVersion(), container.getVersion() + 1); + assertEquals(ExecutionType.GUARANTEED, uc.getExecutionType()); + assertEquals(uc.getId(), container.getId()); + assertEquals(uc.getVersion(), container.getVersion() + 1); // Verify that the Container is still in RUNNING state wrt RM.. rmContainer = ((CapacityScheduler) scheduler) .getApplicationAttempt( uc.getId().getApplicationAttemptId()).getRMContainer(uc.getId()); - Assert.assertEquals(RMContainerState.RUNNING, rmContainer.getState()); + assertEquals(RMContainerState.RUNNING, rmContainer.getState()); // Verify Metrics After OPP allocation : // Allocated cores+mem should have increased, available should decrease verifyMetrics(metrics, 6144, 6, 2048, 2, 2); } - @Test(timeout = 600000) + @Test + @Timeout(value = 600) public void testContainerPromoteAfterContainerComplete() throws Exception { HashMap nodes = new HashMap<>(); MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService()); @@ -511,7 +520,7 @@ public void testContainerPromoteAfterContainerComplete() throws Exception { null); List allocatedContainers = allocateResponse .getAllocatedContainers(); - Assert.assertEquals(2, allocatedContainers.size()); + assertEquals(2, allocatedContainers.size()); Container container = allocatedContainers.get(0); MockNM allocNode = nodes.get(container.getNodeId()); @@ -527,7 +536,7 @@ public void testContainerPromoteAfterContainerComplete() throws Exception { .getApplicationAttempt( container.getId().getApplicationAttemptId()).getRMContainer( container.getId()); - Assert.assertEquals(RMContainerState.RUNNING, rmContainer.getState()); + assertEquals(RMContainerState.RUNNING, rmContainer.getState()); // Container Completed in the NM allocNode.nodeHeartbeat(Arrays.asList( @@ -541,7 +550,7 @@ public void testContainerPromoteAfterContainerComplete() throws Exception { .getApplicationAttempt( container.getId().getApplicationAttemptId()).getRMContainer( container.getId()); - Assert.assertNull(rmContainer); + assertNull(rmContainer); // Verify Metrics After OPP allocation (Nothing should change) verifyMetrics(metrics, 7168, 7, 1024, 1, 1); @@ -553,16 +562,16 @@ public void testContainerPromoteAfterContainerComplete() throws Exception { container.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE, null, ExecutionType.GUARANTEED))); - Assert.assertEquals(1, + assertEquals(1, allocateResponse.getCompletedContainersStatuses().size()); - Assert.assertEquals(container.getId(), + assertEquals(container.getId(), allocateResponse.getCompletedContainersStatuses().get(0) .getContainerId()); - Assert.assertEquals(0, allocateResponse.getUpdatedContainers().size()); - Assert.assertEquals(1, allocateResponse.getUpdateErrors().size()); - Assert.assertEquals("INVALID_CONTAINER_ID", + assertEquals(0, allocateResponse.getUpdatedContainers().size()); + assertEquals(1, allocateResponse.getUpdateErrors().size()); + assertEquals("INVALID_CONTAINER_ID", allocateResponse.getUpdateErrors().get(0).getReason()); - Assert.assertEquals(container.getId(), + assertEquals(container.getId(), allocateResponse.getUpdateErrors().get(0) .getUpdateContainerRequest().getContainerId()); @@ -570,7 +579,8 @@ public void testContainerPromoteAfterContainerComplete() throws Exception { verifyMetrics(metrics, 7168, 7, 1024, 1, 1); } - @Test(timeout = 600000) + @Test + @Timeout(value = 600) public void testContainerAutoUpdateContainer() throws Exception { rm.stop(); createAndStartRMWithAutoUpdateContainer(); @@ -608,7 +618,7 @@ public void testContainerAutoUpdateContainer() throws Exception { allocateResponse.getAllocatedContainers(); allocatedContainers.addAll( am1.allocate(null, null).getAllocatedContainers()); - Assert.assertEquals(2, allocatedContainers.size()); + assertEquals(2, allocatedContainers.size()); Container container = allocatedContainers.get(0); // Start Container in NM nm1.nodeHeartbeat(Arrays.asList(ContainerStatus @@ -620,7 +630,7 @@ public void testContainerAutoUpdateContainer() throws Exception { RMContainer rmContainer = ((CapacityScheduler) scheduler) .getApplicationAttempt(container.getId().getApplicationAttemptId()) .getRMContainer(container.getId()); - Assert.assertEquals(RMContainerState.RUNNING, rmContainer.getState()); + assertEquals(RMContainerState.RUNNING, rmContainer.getState()); // Send Promotion req... this should result in update error // Since the container doesn't exist anymore.. @@ -636,17 +646,17 @@ public void testContainerAutoUpdateContainer() throws Exception { // Get the update response on next allocate allocateResponse = am1.allocate(new ArrayList<>(), new ArrayList<>()); // Check the update response from YARNRM - Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size()); + assertEquals(1, allocateResponse.getUpdatedContainers().size()); UpdatedContainer uc = allocateResponse.getUpdatedContainers().get(0); - Assert.assertEquals(container.getId(), uc.getContainer().getId()); - Assert.assertEquals(ExecutionType.GUARANTEED, + assertEquals(container.getId(), uc.getContainer().getId()); + assertEquals(ExecutionType.GUARANTEED, uc.getContainer().getExecutionType()); // Check that the container is updated in NM through NM heartbeat response NodeHeartbeatResponse response = nm1.nodeHeartbeat(true); - Assert.assertEquals(1, response.getContainersToUpdate().size()); + assertEquals(1, response.getContainersToUpdate().size()); Container containersFromNM = response.getContainersToUpdate().get(0); - Assert.assertEquals(container.getId(), containersFromNM.getId()); - Assert.assertEquals(ExecutionType.GUARANTEED, + assertEquals(container.getId(), containersFromNM.getId()); + assertEquals(ExecutionType.GUARANTEED, containersFromNM.getExecutionType()); //Increase resources @@ -662,10 +672,10 @@ public void testContainerAutoUpdateContainer() throws Exception { if (allocateResponse.getUpdatedContainers().size() == 0) { allocateResponse = am1.allocate(new ArrayList<>(), new ArrayList<>()); } - Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size()); + assertEquals(1, allocateResponse.getUpdatedContainers().size()); uc = allocateResponse.getUpdatedContainers().get(0); - Assert.assertEquals(container.getId(), uc.getContainer().getId()); - Assert.assertEquals(Resource.newInstance(2 * GB, 1), + assertEquals(container.getId(), uc.getContainer().getId()); + assertEquals(Resource.newInstance(2 * GB, 1), uc.getContainer().getResource()); rm.drainEvents(); @@ -674,8 +684,8 @@ public void testContainerAutoUpdateContainer() throws Exception { if (response.getContainersToUpdate().size() == 0) { response = nm1.nodeHeartbeat(true); } - Assert.assertEquals(1, response.getContainersToUpdate().size()); - Assert.assertEquals(Resource.newInstance(2 * GB, 1), + assertEquals(1, response.getContainersToUpdate().size()); + assertEquals(Resource.newInstance(2 * GB, 1), response.getContainersToUpdate().get(0).getResource()); //Decrease resources @@ -683,14 +693,14 @@ public void testContainerAutoUpdateContainer() throws Exception { UpdateContainerRequest.newInstance(2, container.getId(), ContainerUpdateType.DECREASE_RESOURCE, Resources.createResource(1 * GB, 1), null))); - Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size()); + assertEquals(1, allocateResponse.getUpdatedContainers().size()); rm.drainEvents(); // Check that the container resources are decreased // in NM through NM heartbeat response response = nm1.nodeHeartbeat(true); - Assert.assertEquals(1, response.getContainersToUpdate().size()); - Assert.assertEquals(Resource.newInstance(1 * GB, 1), + assertEquals(1, response.getContainersToUpdate().size()); + assertEquals(Resource.newInstance(1 * GB, 1), response.getContainersToUpdate().get(0).getResource()); nm1.nodeHeartbeat(oppContainersStatus, true); @@ -709,30 +719,31 @@ public void testContainerAutoUpdateContainer() throws Exception { allocateResponse = am1.allocate(new ArrayList<>(), new ArrayList<>()); } // Check the update response from YARNRM - Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size()); + assertEquals(1, allocateResponse.getUpdatedContainers().size()); uc = allocateResponse.getUpdatedContainers().get(0); - Assert.assertEquals(ExecutionType.OPPORTUNISTIC, + assertEquals(ExecutionType.OPPORTUNISTIC, uc.getContainer().getExecutionType()); // Check that the container is updated in NM through NM heartbeat response if (response.getContainersToUpdate().size() == 0) { response = nm1.nodeHeartbeat(oppContainersStatus, true); } - Assert.assertEquals(1, response.getContainersToUpdate().size()); - Assert.assertEquals(ExecutionType.OPPORTUNISTIC, + assertEquals(1, response.getContainersToUpdate().size()); + assertEquals(ExecutionType.OPPORTUNISTIC, response.getContainersToUpdate().get(0).getExecutionType()); } private void verifyMetrics(QueueMetrics metrics, long availableMB, int availableVirtualCores, long allocatedMB, int allocatedVirtualCores, int allocatedContainers) { - Assert.assertEquals(availableMB, metrics.getAvailableMB()); - Assert.assertEquals(availableVirtualCores, metrics.getAvailableVirtualCores()); - Assert.assertEquals(allocatedMB, metrics.getAllocatedMB()); - Assert.assertEquals(allocatedVirtualCores, metrics.getAllocatedVirtualCores()); - Assert.assertEquals(allocatedContainers, metrics.getAllocatedContainers()); + assertEquals(availableMB, metrics.getAvailableMB()); + assertEquals(availableVirtualCores, metrics.getAvailableVirtualCores()); + assertEquals(allocatedMB, metrics.getAllocatedMB()); + assertEquals(allocatedVirtualCores, metrics.getAllocatedVirtualCores()); + assertEquals(allocatedContainers, metrics.getAllocatedContainers()); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testOpportunisticSchedulerMetrics() throws Exception { HashMap nodes = new HashMap<>(); MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService()); @@ -783,12 +794,12 @@ public void testOpportunisticSchedulerMetrics() throws Exception { List allocatedContainers = allocateResponse .getAllocatedContainers(); - Assert.assertEquals(2, allocatedContainers.size()); + assertEquals(2, allocatedContainers.size()); - Assert.assertEquals(allocContainers + 2, metrics.getAllocatedContainers()); - Assert.assertEquals(aggrAllocatedContainers + 2, + assertEquals(allocContainers + 2, metrics.getAllocatedContainers()); + assertEquals(aggrAllocatedContainers + 2, metrics.getAggregatedAllocatedContainers()); - Assert.assertEquals(aggrOffSwitchContainers + 2, + assertEquals(aggrOffSwitchContainers + 2, metrics.getAggregatedOffSwitchContainers()); Container container = allocatedContainers.get(0); @@ -806,7 +817,7 @@ public void testOpportunisticSchedulerMetrics() throws Exception { .getApplicationAttempt( container.getId().getApplicationAttemptId()).getRMContainer( container.getId()); - Assert.assertEquals(RMContainerState.RUNNING, rmContainer.getState()); + assertEquals(RMContainerState.RUNNING, rmContainer.getState()); // Container Completed in the NM allocNode.nodeHeartbeat(Arrays.asList( @@ -820,10 +831,10 @@ public void testOpportunisticSchedulerMetrics() throws Exception { .getApplicationAttempt( container.getId().getApplicationAttemptId()).getRMContainer( container.getId()); - Assert.assertNull(rmContainer); + assertNull(rmContainer); - Assert.assertEquals(allocContainers + 1, metrics.getAllocatedContainers()); - Assert.assertEquals(aggrReleasedContainers + 1, + assertEquals(allocContainers + 1, metrics.getAllocatedContainers()); + assertEquals(aggrReleasedContainers + 1, metrics.getAggregatedReleasedContainers()); } @@ -872,7 +883,7 @@ public void testMetricsRetainsAllocatedOpportunisticAfterRMRestart() ExecutionType.OPPORTUNISTIC, -1); // Make sure that numbers start with 0 - Assert.assertEquals(0, metrics.getAllocatedContainers()); + assertEquals(0, metrics.getAllocatedContainers()); // Recover one OContainer only rm.registerNode("h2:1234", 4096, 1, @@ -880,7 +891,7 @@ public void testMetricsRetainsAllocatedOpportunisticAfterRMRestart() appAttemptId.getApplicationId()), Collections.singletonList(recoverOContainerReport1)); - Assert.assertEquals(1, metrics.getAllocatedContainers()); + assertEquals(1, metrics.getAllocatedContainers()); // Recover two OContainers at once final ContainerId recoverOContainerId3 = ContainerId.newContainerId( @@ -909,7 +920,7 @@ public void testMetricsRetainsAllocatedOpportunisticAfterRMRestart() appAttemptId.getApplicationId()), Arrays.asList(recoverOContainerReport2, recoverOContainerReport3)); - Assert.assertEquals(3, metrics.getAllocatedContainers()); + assertEquals(3, metrics.getAllocatedContainers()); // Make sure that the recovered GContainer // does not increment OContainer count @@ -929,7 +940,7 @@ public void testMetricsRetainsAllocatedOpportunisticAfterRMRestart() appAttemptId.getApplicationId()), Collections.singletonList(recoverGContainerReport)); - Assert.assertEquals(3, metrics.getAllocatedContainers()); + assertEquals(3, metrics.getAllocatedContainers()); final ContainerId completedOContainerId = ContainerId.newContainerId( appAttemptId, 6); @@ -948,10 +959,11 @@ public void testMetricsRetainsAllocatedOpportunisticAfterRMRestart() appAttemptId.getApplicationId()), Collections.singletonList(completedOContainerReport)); - Assert.assertEquals(3, metrics.getAllocatedContainers()); + assertEquals(3, metrics.getAllocatedContainers()); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testAMCrashDuringAllocate() throws Exception { MockNM nm = new MockNM("h:1234", 4096, rm.getResourceTrackerService()); nm.registerNode(); @@ -986,7 +998,8 @@ public void testAMCrashDuringAllocate() throws Exception { "*", Resources.createResource(1 * GB), 2)), null); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testNodeRemovalDuringAllocate() throws Exception { MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService()); MockNM nm2 = new MockNM("h2:1234", 4096, rm.getResourceTrackerService()); @@ -1032,7 +1045,7 @@ public void testNodeRemovalDuringAllocate() throws Exception { } Thread.sleep(50); } - Assert.assertEquals(2, ctxt.getNodeMap().size()); + assertEquals(2, ctxt.getNodeMap().size()); // Remove node from scheduler but not from AM Service. scheduler.handle(new NodeRemovedSchedulerEvent(rmNode1)); // After removal of node 1, only 1 node will be applicable for scheduling. @@ -1043,17 +1056,18 @@ public void testNodeRemovalDuringAllocate() throws Exception { "*", Resources.createResource(1 * GB), 2)), null); } catch (Exception e) { - Assert.fail("Allocate request should be handled on node removal"); + fail("Allocate request should be handled on node removal"); } if (ctxt.getNodeMap().size() == 1) { break; } Thread.sleep(50); } - Assert.assertEquals(1, ctxt.getNodeMap().size()); + assertEquals(1, ctxt.getNodeMap().size()); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testAppAttemptRemovalAfterNodeRemoval() throws Exception { MockNM nm = new MockNM("h:1234", 4096, rm.getResourceTrackerService()); @@ -1189,7 +1203,7 @@ public ResourceScheduler getScheduler() { ((RegisterApplicationMasterRequestPBImpl)factory .newRecordInstance( RegisterApplicationMasterRequest.class)).getProto())); - Assert.assertEquals("dummyQueue", regResp.getQueue()); + assertEquals("dummyQueue", regResp.getQueue()); FinishApplicationMasterResponse finishResp = new FinishApplicationMasterResponsePBImpl( ampProxy.finishApplicationMaster(null, @@ -1197,7 +1211,7 @@ public ResourceScheduler getScheduler() { .newRecordInstance( FinishApplicationMasterRequest.class)).getProto() )); - Assert.assertEquals(false, finishResp.getIsUnregistered()); + assertEquals(false, finishResp.getIsUnregistered()); AllocateResponse allocResp = new AllocateResponsePBImpl( ampProxy.allocate(null, @@ -1205,10 +1219,10 @@ public ResourceScheduler getScheduler() { .newRecordInstance(AllocateRequest.class)).getProto()) ); List allocatedContainers = allocResp.getAllocatedContainers(); - Assert.assertEquals(1, allocatedContainers.size()); - Assert.assertEquals(ExecutionType.OPPORTUNISTIC, + assertEquals(1, allocatedContainers.size()); + assertEquals(ExecutionType.OPPORTUNISTIC, allocatedContainers.get(0).getExecutionType()); - Assert.assertEquals(12345, allocResp.getNumClusterNodes()); + assertEquals(12345, allocResp.getNumClusterNodes()); // Verify that the DistrubutedSchedulingService can handle the @@ -1225,12 +1239,12 @@ public ResourceScheduler getScheduler() { ((RegisterApplicationMasterRequestPBImpl)factory .newRecordInstance(RegisterApplicationMasterRequest.class)) .getProto())); - Assert.assertEquals(54321l, dsRegResp.getContainerIdStart()); - Assert.assertEquals(4, + assertEquals(54321L, dsRegResp.getContainerIdStart()); + assertEquals(4, dsRegResp.getMaxContainerResource().getVirtualCores()); - Assert.assertEquals(1024, + assertEquals(1024, dsRegResp.getMinContainerResource().getMemorySize()); - Assert.assertEquals(2, + assertEquals(2, dsRegResp.getIncrContainerResource().getVirtualCores()); DistributedSchedulingAllocateRequestPBImpl distAllReq = @@ -1242,9 +1256,9 @@ public ResourceScheduler getScheduler() { new DistributedSchedulingAllocateResponsePBImpl( dsProxy.allocateForDistributedScheduling(null, distAllReq.getProto())); - Assert.assertEquals( + assertEquals( "h1", dsAllocResp.getNodesForScheduling().get(0).getNodeId().getHost()); - Assert.assertEquals( + assertEquals( "l1", dsAllocResp.getNodesForScheduling().get(1).getNodePartition()); FinishApplicationMasterResponse dsfinishResp = @@ -1253,7 +1267,7 @@ public ResourceScheduler getScheduler() { ((FinishApplicationMasterRequestPBImpl) factory .newRecordInstance(FinishApplicationMasterRequest.class)) .getProto())); - Assert.assertEquals( + assertEquals( false, dsfinishResp.getIsUnregistered()); } @@ -1315,11 +1329,11 @@ public AllocateResponse allocate(AllocateRequest request) throws List askList = request.getAllocateRequest().getAskList(); List allocatedContainers = request.getAllocatedContainers(); - Assert.assertEquals(1, allocatedContainers.size()); - Assert.assertEquals(ExecutionType.OPPORTUNISTIC, + assertEquals(1, allocatedContainers.size()); + assertEquals(ExecutionType.OPPORTUNISTIC, allocatedContainers.get(0).getExecutionType()); - Assert.assertEquals(1, askList.size()); - Assert.assertTrue(askList.get(0) + assertEquals(1, askList.size()); + assertTrue(askList.get(0) .getExecutionTypeRequest().getEnforceExecutionType()); DistributedSchedulingAllocateResponse resp = factory .newRecordInstance(DistributedSchedulingAllocateResponse.class); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java index 457e9d47dd0b4..f370bff16d54d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java @@ -19,8 +19,11 @@ package org.apache.hadoop.yarn.server.resourcemanager; import static org.assertj.core.api.Assertions.assertThat; -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.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import java.io.DataOutputStream; import java.io.File; @@ -91,10 +94,10 @@ import static org.apache.hadoop.yarn.conf.YarnConfiguration.RM_PROXY_USER_PREFIX; import static org.apache.hadoop.yarn.server.resourcemanager.resource.DynamicResourceConfiguration.NODES; import static org.apache.hadoop.yarn.server.resourcemanager.resource.DynamicResourceConfiguration.PREFIX; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.mockito.Mockito; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList; @@ -105,8 +108,6 @@ import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager; import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.AddToClusterNodeLabelsRequestProto; -import static org.junit.Assert.assertTrue; - public class TestRMAdminService { private Configuration configuration; @@ -122,7 +123,7 @@ public class TestRMAdminService { YarnConfiguration.DR_CONFIGURATION_FILE); } - @Before + @BeforeEach public void setup() throws IOException { QueueMetrics.clearQueueMetrics(); DefaultMetricsSystem.setMiniClusterMode(true); @@ -147,7 +148,7 @@ public void setup() throws IOException { MockUnixGroupsMapping.resetGroups(); } - @After + @AfterEach public void tearDown() throws IOException { if (rm != null) { rm.stop(); @@ -169,7 +170,7 @@ public void testAdminRefreshQueuesWithLocalConfigurationProvider() try { rm.adminService.refreshQueues(RefreshQueuesRequest.newInstance()); - Assert.assertEquals(maxAppsBefore, cs.getConfiguration() + assertEquals(maxAppsBefore, cs.getConfiguration() .getMaximumSystemApplications()); } catch (Exception ex) { fail("Using localConfigurationProvider. Should not get any exception."); @@ -239,8 +240,8 @@ public void testAdminRefreshQueuesWithFileSystemBasedConfigurationProvider() rm.adminService.refreshQueues(RefreshQueuesRequest.newInstance()); int maxAppsAfter = cs.getConfiguration().getMaximumSystemApplications(); - Assert.assertEquals(maxAppsAfter, 5000); - Assert.assertTrue(maxAppsAfter != maxAppsBefore); + assertEquals(maxAppsAfter, 5000); + assertTrue(maxAppsAfter != maxAppsBefore); } @Test @@ -313,7 +314,7 @@ public void testRefreshNodesResourceWithFileSystemBasedConfigurationProvider() NodeId nid = NodeId.fromString("h1:1234"); RMNode ni = rm.getRMContext().getRMNodes().get(nid); Resource resource = ni.getTotalCapability(); - Assert.assertEquals("", resource.toString()); + assertEquals("", resource.toString()); DynamicResourceConfiguration drConf = new DynamicResourceConfiguration(); @@ -328,7 +329,7 @@ public void testRefreshNodesResourceWithFileSystemBasedConfigurationProvider() RMNode niAfter = rm.getRMContext().getRMNodes().get(nid); Resource resourceAfter = niAfter.getTotalCapability(); - Assert.assertEquals("", resourceAfter.toString()); + assertEquals("", resourceAfter.toString()); } @Test @@ -353,7 +354,7 @@ public void testRefreshNodesResourceWithResourceReturnInRegistration() NodeId nid = NodeId.fromString("h1:1234"); RMNode ni = rm.getRMContext().getRMNodes().get(nid); Resource resource = ni.getTotalCapability(); - Assert.assertEquals("", resource.toString()); + assertEquals("", resource.toString()); DynamicResourceConfiguration drConf = new DynamicResourceConfiguration(); @@ -375,10 +376,10 @@ public void testRefreshNodesResourceWithResourceReturnInRegistration() RMNode niAfter = rm.getRMContext().getRMNodes().get(nid); Resource resourceAfter = niAfter.getTotalCapability(); - Assert.assertEquals("", resourceAfter.toString()); + assertEquals("", resourceAfter.toString()); - Assert.assertEquals(4096, nm.getMemory()); - Assert.assertEquals(4, nm.getvCores()); + assertEquals(4096, nm.getMemory()); + assertEquals(4, nm.getvCores()); } @Test @@ -403,7 +404,7 @@ public void testRefreshNodesResourceWithResourceReturnInHeartbeat() NodeId nid = NodeId.fromString("h1:1234"); RMNode ni = rm.getRMContext().getRMNodes().get(nid); Resource resource = ni.getTotalCapability(); - Assert.assertEquals("", resource.toString()); + assertEquals("", resource.toString()); DynamicResourceConfiguration drConf = new DynamicResourceConfiguration(); @@ -424,10 +425,10 @@ public void testRefreshNodesResourceWithResourceReturnInHeartbeat() RMNode niAfter = rm.getRMContext().getRMNodes().get(nid); Resource resourceAfter = niAfter.getTotalCapability(); - Assert.assertEquals("", resourceAfter.toString()); + assertEquals("", resourceAfter.toString()); - Assert.assertEquals(4096, nm.getMemory()); - Assert.assertEquals(4, nm.getvCores()); + assertEquals(4096, nm.getMemory()); + assertEquals(4, nm.getvCores()); } @Test @@ -451,7 +452,7 @@ public void testResourcePersistentForNMRegistrationWithNewResource() NodeId nid = NodeId.fromString("h1:1234"); RMNode ni = rm.getRMContext().getRMNodes().get(nid); Resource resource = ni.getTotalCapability(); - Assert.assertEquals("", resource.toString()); + assertEquals("", resource.toString()); DynamicResourceConfiguration drConf = new DynamicResourceConfiguration(); @@ -473,7 +474,7 @@ public void testResourcePersistentForNMRegistrationWithNewResource() RMNode niAfter = rm.getRMContext().getRMNodes().get(nid); Resource resourceAfter = niAfter.getTotalCapability(); - Assert.assertEquals("", resourceAfter.toString()); + assertEquals("", resourceAfter.toString()); // Replace original dr file with an empty dr file, and validate node // registration with new resources will take effective now. @@ -496,7 +497,7 @@ public void testResourcePersistentForNMRegistrationWithNewResource() resourceAfter = niAfter.getTotalCapability(); // new resource in registration should take effective as we empty // dynamic resource file already. - Assert.assertEquals("", resourceAfter.toString()); + assertEquals("", resourceAfter.toString()); } @Test @@ -541,8 +542,8 @@ public void testAdminAclsWithFileSystemBasedConfigurationProvider() String aclStringAfter = rm.adminService.getAccessControlList().getAclString().trim(); - Assert.assertTrue(!aclStringAfter.equals(aclStringBefore)); - Assert.assertEquals(aclStringAfter, "world:anyone:rwcda," + + assertTrue(!aclStringAfter.equals(aclStringBefore)); + assertEquals(aclStringAfter, "world:anyone:rwcda," + UserGroupInformation.getCurrentUser().getShortUserName()); } @@ -643,7 +644,7 @@ private void verifyServiceACLsRefresh(ServiceAuthorizationManager manager, AccessControlList accessList = manager.getProtocolsAcls(protocolClass); if (protocolClass == protocol) { - Assert.assertEquals(accessList.getAclString(), + assertEquals(accessList.getAclString(), aclString); } else { assertThat(accessList.getAclString()).isEqualTo("*"); @@ -691,14 +692,14 @@ private void verifyServiceACLsRefresh(ServiceAuthorizationManager manager, rm.adminService.refreshSuperUserGroupsConfiguration( RefreshSuperUserGroupsConfigurationRequest.newInstance()); - Assert.assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyGroups() + assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyGroups() .get("hadoop.proxyuser.test.groups").size() == 1); - Assert.assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyGroups() + assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyGroups() .get("hadoop.proxyuser.test.groups").contains("test_groups")); - Assert.assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyHosts() + assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyHosts() .get("hadoop.proxyuser.test.hosts").size() == 1); - Assert.assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyHosts() + assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyHosts() .get("hadoop.proxyuser.test.hosts").contains("test_hosts")); Configuration yarnConf = new Configuration(false); @@ -709,14 +710,14 @@ private void verifyServiceACLsRefresh(ServiceAuthorizationManager manager, // RM specific configs will overwrite the common ones rm.adminService.refreshSuperUserGroupsConfiguration( RefreshSuperUserGroupsConfigurationRequest.newInstance()); - Assert.assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyGroups() + assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyGroups() .get("hadoop.proxyuser.test.groups").size() == 1); - Assert.assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyGroups() + assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyGroups() .get("hadoop.proxyuser.test.groups").contains("test_groups_1")); - Assert.assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyHosts() + assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyHosts() .get("hadoop.proxyuser.test.hosts").size() == 1); - Assert.assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyHosts() + assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyHosts() .get("hadoop.proxyuser.test.hosts").contains("test_hosts_1")); } @@ -772,11 +773,11 @@ public void testRefreshUserToGroupsMappingsWithLocalConfigurationProvider() { List groupBefore = new ArrayList(Groups.getUserToGroupsMappingService( configuration).getGroups(user)); - Assert.assertTrue(groupBefore.contains("test_group_A") + assertTrue(groupBefore.contains("test_group_A") && groupBefore.contains("test_group_B") && groupBefore.contains("test_group_C") && groupBefore.size() == 3); - Assert.assertTrue(groupWithInit.size() != groupBefore.size()); - Assert.assertFalse(groupWithInit.contains("test_group_A") + assertTrue(groupWithInit.size() != groupBefore.size()); + assertFalse(groupWithInit.contains("test_group_A") || groupWithInit.contains("test_group_B") || groupWithInit.contains("test_group_C")); @@ -790,7 +791,7 @@ public void testRefreshUserToGroupsMappingsWithLocalConfigurationProvider() { Groups.getUserToGroupsMappingService(configuration).getGroups(user); // should get the updated groups - Assert.assertTrue(groupAfter.contains("test_group_D") + assertTrue(groupAfter.contains("test_group_D") && groupAfter.contains("test_group_E") && groupAfter.contains("test_group_F") && groupAfter.size() == 3); @@ -832,7 +833,7 @@ public void testRefreshNodesWithFileSystemBasedConfigurationProvider() excludeHostsFile.delete(); } if (!excludeHostsFile.createNewFile()) { - Assert.fail("Can not create " + "excludeHosts"); + fail("Can not create " + "excludeHosts"); } PrintWriter fileWriter = new PrintWriter(excludeHostsFile); fileWriter.write("0.0.0.0:123"); @@ -849,8 +850,8 @@ public void testRefreshNodesWithFileSystemBasedConfigurationProvider() .newInstance(DecommissionType.NORMAL)); Set excludeHosts = rm.getNodesListManager().getHostsReader().getExcludedHosts(); - Assert.assertTrue(excludeHosts.size() == 1); - Assert.assertTrue(excludeHosts.contains("0.0.0.0:123")); + assertTrue(excludeHosts.size() == 1); + assertTrue(excludeHosts.contains("0.0.0.0:123")); } @Test @@ -873,17 +874,17 @@ public void testRMHAWithFileSystemBasedConfiguration() throws IOException, rm1 = new MockRM(conf1); rm1.init(conf1); rm1.start(); - Assert.assertTrue(rm1.getRMContext().getHAServiceState() + assertTrue(rm1.getRMContext().getHAServiceState() == HAServiceState.STANDBY); rm2 = new MockRM(conf2); rm2.init(conf1); rm2.start(); - Assert.assertTrue(rm2.getRMContext().getHAServiceState() + assertTrue(rm2.getRMContext().getHAServiceState() == HAServiceState.STANDBY); rm1.adminService.transitionToActive(requestInfo); - Assert.assertTrue(rm1.getRMContext().getHAServiceState() + assertTrue(rm1.getRMContext().getHAServiceState() == HAServiceState.ACTIVE); CapacitySchedulerConfiguration csConf = @@ -897,28 +898,28 @@ public void testRMHAWithFileSystemBasedConfiguration() throws IOException, int maxApps = ((CapacityScheduler) rm1.getRMContext().getScheduler()) .getConfiguration().getMaximumSystemApplications(); - Assert.assertEquals(maxApps, 5000); + assertEquals(maxApps, 5000); // Before failover happens, the maxApps is // still the default value on the standby rm : rm2 int maxAppsBeforeFailOver = ((CapacityScheduler) rm2.getRMContext().getScheduler()) .getConfiguration().getMaximumSystemApplications(); - Assert.assertEquals(maxAppsBeforeFailOver, 10000); + assertEquals(maxAppsBeforeFailOver, 10000); // Do the failover rm1.adminService.transitionToStandby(requestInfo); rm2.adminService.transitionToActive(requestInfo); - Assert.assertTrue(rm1.getRMContext().getHAServiceState() + assertTrue(rm1.getRMContext().getHAServiceState() == HAServiceState.STANDBY); - Assert.assertTrue(rm2.getRMContext().getHAServiceState() + assertTrue(rm2.getRMContext().getHAServiceState() == HAServiceState.ACTIVE); int maxAppsAfter = ((CapacityScheduler) rm2.getRMContext().getScheduler()) .getConfiguration().getMaximumSystemApplications(); - Assert.assertEquals(maxAppsAfter, 5000); + assertEquals(maxAppsAfter, 5000); } finally { if (rm1 != null) { rm1.stop(); @@ -1034,7 +1035,7 @@ public void testRMInitialsWithFileSystemBasedConfigurationProvider() excludeHostsFile.delete(); } if (!excludeHostsFile.createNewFile()) { - Assert.fail("Can not create " + "excludeHosts"); + fail("Can not create " + "excludeHosts"); } PrintWriter fileWriter = new PrintWriter(excludeHostsFile); fileWriter.write("0.0.0.0:123"); @@ -1085,21 +1086,21 @@ public void testRMInitialsWithFileSystemBasedConfigurationProvider() Set excludeHosts = resourceManager.getRMContext().getNodesListManager() .getHostsReader().getExcludedHosts(); - Assert.assertTrue(excludeHosts.size() == 1); - Assert.assertTrue(excludeHosts.contains("0.0.0.0:123")); + assertTrue(excludeHosts.size() == 1); + assertTrue(excludeHosts.contains("0.0.0.0:123")); // validate values for admin-acls String aclStringAfter = resourceManager.adminService.getAccessControlList() .getAclString().trim(); - Assert.assertEquals(aclStringAfter, "world:anyone:rwcda," + + assertEquals(aclStringAfter, "world:anyone:rwcda," + UserGroupInformation.getCurrentUser().getShortUserName()); // validate values for queue configuration CapacityScheduler cs = (CapacityScheduler) resourceManager.getRMContext().getScheduler(); int maxAppsAfter = cs.getConfiguration().getMaximumSystemApplications(); - Assert.assertEquals(maxAppsAfter, 5000); + assertEquals(maxAppsAfter, 5000); // verify service Acls for AdminService ServiceAuthorizationManager adminServiceServiceManager = @@ -1135,14 +1136,14 @@ public void testRMInitialsWithFileSystemBasedConfigurationProvider() // verify ProxyUsers and ProxyHosts ProxyUsers.refreshSuperUserGroupsConfiguration(configuration); - Assert.assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyGroups() + assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyGroups() .get("hadoop.proxyuser.test.groups").size() == 1); - Assert.assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyGroups() + assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyGroups() .get("hadoop.proxyuser.test.groups").contains("test_groups")); - Assert.assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyHosts() + assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyHosts() .get("hadoop.proxyuser.test.hosts").size() == 1); - Assert.assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyHosts() + assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyHosts() .get("hadoop.proxyuser.test.hosts").contains("test_hosts")); // verify UserToGroupsMappings @@ -1150,7 +1151,7 @@ public void testRMInitialsWithFileSystemBasedConfigurationProvider() List groupAfter = Groups.getUserToGroupsMappingService(configuration).getGroups( UserGroupInformation.getCurrentUser().getUserName()); - Assert.assertTrue(groupAfter.contains("test_group_D") + assertTrue(groupAfter.contains("test_group_D") && groupAfter.contains("test_group_E") && groupAfter.contains("test_group_F") && groupAfter.size() == 3); } finally { @@ -1219,25 +1220,27 @@ public void testModifyLabelsOnNodesWithDistributedConfigurationDisabled() rm.close(); } - @Test(expected = YarnException.class) + @Test public void testModifyLabelsOnNodesWithCentralizedConfigurationDisabled() throws IOException, YarnException { - // create RM and set it's ACTIVE, and set distributed node label - // configuration to true - MockRM rm = new MockRM(); - rm.adminService.isCentralizedNodeLabelConfiguration = false; - - ((RMContextImpl) rm.getRMContext()) - .setHAServiceState(HAServiceState.ACTIVE); - RMNodeLabelsManager labelMgr = rm.rmContext.getNodeLabelManager(); - - // by default, distributed configuration for node label is disabled, this - // should pass - labelMgr.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet.of("x", "y")); - rm.adminService.replaceLabelsOnNode(ReplaceLabelsOnNodeRequest - .newInstance(ImmutableMap.of(NodeId.newInstance("host", 0), - (Set) ImmutableSet.of("x")))); - rm.close(); + assertThrows(YarnException.class, () -> { + // create RM and set it's ACTIVE, and set distributed node label + // configuration to true + MockRM mockRM = new MockRM(); + mockRM.adminService.isCentralizedNodeLabelConfiguration = false; + + ((RMContextImpl) mockRM.getRMContext()) + .setHAServiceState(HAServiceState.ACTIVE); + RMNodeLabelsManager labelMgr = mockRM.rmContext.getNodeLabelManager(); + + // by default, distributed configuration for node label is disabled, this + // should pass + labelMgr.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet.of("x", "y")); + mockRM.adminService.replaceLabelsOnNode(ReplaceLabelsOnNodeRequest + .newInstance(ImmutableMap.of(NodeId.newInstance("host", 0), + (Set) ImmutableSet.of("x")))); + mockRM.close(); + }); } @Test @@ -1348,7 +1351,8 @@ public void testRemoveClusterNodeLabelsWithCentralizedConfigurationDisabled() rm.close(); } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testAdminRefreshClusterMaxPriority() throws Exception, YarnException { configuration.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS, @@ -1364,7 +1368,7 @@ public void testAdminRefreshClusterMaxPriority() throws Exception, rm.start(); CapacityScheduler cs = (CapacityScheduler) rm.getRMContext().getScheduler(); - Assert.assertEquals(5, cs.getMaxClusterLevelAppPriority().getPriority()); + assertEquals(5, cs.getMaxClusterLevelAppPriority().getPriority()); yarnConf = new YarnConfiguration(); yarnConf @@ -1375,7 +1379,7 @@ public void testAdminRefreshClusterMaxPriority() throws Exception, rm.adminService .refreshClusterMaxPriority(RefreshClusterMaxPriorityRequest .newInstance()); - Assert.assertEquals(10, cs.getMaxClusterLevelAppPriority().getPriority()); + assertEquals(10, cs.getMaxClusterLevelAppPriority().getPriority()); } catch (Exception ex) { fail("Could not refresh cluster max priority."); } @@ -1390,7 +1394,7 @@ private String writeConfigurationXML(Configuration conf, String confXMLName) confFile.delete(); } if (!confFile.createNewFile()) { - Assert.fail("Can not create " + confXMLName); + fail("Can not create " + confXMLName); } output = new DataOutputStream( new FileOutputStream(confFile)); @@ -1495,7 +1499,7 @@ public void testSecureRMBecomeActive() throws IOException, try { resourceManager.init(configuration); resourceManager.start(); - Assert.assertTrue(resourceManager.getRMContext().getHAServiceState() + assertTrue(resourceManager.getRMContext().getHAServiceState() == HAServiceState.STANDBY); resourceManager.adminService.transitionToActive(requestInfo); } finally { @@ -1537,12 +1541,13 @@ private void checkBadConfiguration(Configuration conf) { rm1.init(conf); fail("The RM allowed an invalid configuration"); } catch (YarnRuntimeException e) { - assertTrue("The RM initialization threw an unexpected exception", - e.getMessage().startsWith(HAUtil.BAD_CONFIG_MESSAGE_PREFIX)); + assertTrue(e.getMessage().startsWith(HAUtil.BAD_CONFIG_MESSAGE_PREFIX), + "The RM initialization threw an unexpected exception"); } } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testAdminAddToClusterNodeLabelsWithDeprecatedAPIs() throws Exception, YarnException { configuration.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS, @@ -1586,11 +1591,13 @@ protected ClientRMService createClientRMService() { .getClusterNodeLabels(GetClusterNodeLabelsRequest.newInstance()); NodeLabel labelX = NodeLabel.newInstance("a"); NodeLabel labelY = NodeLabel.newInstance("b"); - Assert.assertTrue( - response.getNodeLabelList().containsAll(Arrays.asList(labelX, labelY))); + assertTrue(response.getNodeLabelList().containsAll( + Arrays.asList(labelX, labelY))); } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) + @SuppressWarnings("checkstyle:MethodLength") public void testMapAttributesToNodes() throws Exception, YarnException { // 1. Need to test for the Invalid Node // 1.1. Need to test for active nodes @@ -1655,9 +1662,9 @@ public void testMapAttributesToNodes() throws Exception, YarnException { .removeNodeAttributes(Mockito.anyMap()); // Assert node to attributes mappings are empty. - Assert.assertTrue("Attributes of host4 should be empty", - rm.getRMContext().getNodeAttributesManager() - .getAttributesForNode("host4").isEmpty()); + assertTrue(rm.getRMContext().getNodeAttributesManager() + .getAttributesForNode("host4").isEmpty(), + "Attributes of host4 should be empty"); // remove non existing attributes. request = NodesToAttributesMappingRequest .newInstance(AttributeMappingOperationType.REMOVE, ImmutableList @@ -1669,7 +1676,7 @@ public void testMapAttributesToNodes() throws Exception, YarnException { rm.adminService.mapAttributesToNodes(request); fail("Should have failed for non exists attribute"); } catch (Exception ex) { - assertTrue("Exception expected if attributes does not exist", true); + assertTrue(true, "Exception expected if attributes does not exist"); } request = @@ -1684,9 +1691,8 @@ public void testMapAttributesToNodes() throws Exception, YarnException { rm.adminService.mapAttributesToNodes(request); fail("host5 is not a valid node, It should have failed"); } catch (YarnException ex) { - Assert.assertEquals("Exception Message is not as desired", - " Following nodes does not exist : [host5]", - ex.getCause().getMessage()); + assertEquals(" Following nodes does not exist : [host5]", + ex.getCause().getMessage(), "Exception Message is not as desired"); } request = @@ -1707,9 +1713,8 @@ public void testMapAttributesToNodes() throws Exception, YarnException { rm.adminService.mapAttributesToNodes(request); fail("host with the port should fail as only hostnames are validated"); } catch (YarnException ex) { - Assert.assertEquals("Exception Message is not as desired", - " Following nodes does not exist : [host4:8889, host2:8889]", - ex.getCause().getMessage()); + assertEquals(" Following nodes does not exist : [host4:8889, host2:8889]", + ex.getCause().getMessage(), "Exception Message is not as desired"); } request = @@ -1743,9 +1748,10 @@ public void testMapAttributesToNodes() throws Exception, YarnException { rm.adminService.mapAttributesToNodes(request); fail("This operation should fail as prefix should be \"nm.yarn.io\"."); } catch (YarnException ex) { - Assert.assertEquals("Exception Message is not as desired", + assertEquals( "Invalid Attribute Mapping for the node host5. Prefix should be " - + "rm.yarn.io", ex.getCause().getMessage()); + + "rm.yarn.io", ex.getCause().getMessage(), + "Exception Message is not as desired"); } rm.close(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAuditLogger.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAuditLogger.java index c78a7788c4dba..785f1cb93bd71 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAuditLogger.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAuditLogger.java @@ -17,7 +17,9 @@ */ package org.apache.hadoop.yarn.server.resourcemanager; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -46,9 +48,8 @@ import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.Keys; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** @@ -69,7 +70,7 @@ public class TestRMAuditLogger { private static final byte[] CALLER_SIGNATURE = "signature".getBytes(); private static final String PARTITION = "label1"; - @Before + @BeforeEach public void setUp() throws Exception { when(APPID.toString()).thenReturn("app_1"); when(ATTEMPTID.toString()).thenReturn("app_attempt_1"); @@ -242,7 +243,7 @@ private void testSuccessLogFormatHelperWithIP(boolean checkIP, } catch (UnknownHostException uhe) { // should not happen as long as IP address format // stays the same - Assert.fail("Check ip address being constructed"); + fail("Check ip address being constructed"); } testSuccessLogFormatHelperWithIP(checkIP, appId, attemptId, containerId, addr); @@ -404,8 +405,8 @@ public TestProtos.EmptyResponseProto ping( throws ServiceException { // Ensure clientId is received byte[] clientId = Server.getClientId(); - Assert.assertNotNull(clientId); - Assert.assertEquals(ClientId.BYTE_LENGTH, clientId.length); + assertNotNull(clientId); + assertEquals(ClientId.BYTE_LENGTH, clientId.length); // test with ip set testSuccessLogFormat(true); testFailureLogFormat(true); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.java index 1ea470cf0944f..149518fc5f8a6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.Mockito.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; @@ -33,13 +34,14 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerPreemptEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; public class TestRMDispatcher { @SuppressWarnings("unchecked") - @Test(timeout=10000) + @Test + @Timeout(value = 10) public void testSchedulerEventDispatcherForPreemptionEvents() { AsyncDispatcher rmDispatcher = new AsyncDispatcher(); CapacityScheduler sched = spy(new CapacityScheduler()); @@ -73,7 +75,7 @@ public void testSchedulerEventDispatcherForPreemptionEvents() { verify(sched).markContainerForPreemption(appAttemptId, container); verify(sched).markContainerForKillable(container); } catch (InterruptedException e) { - Assert.fail(); + fail(); } finally { schedulerDispatcher.stop(); rmDispatcher.stop(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java index 2f8de6b695a95..d15a02c778a86 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java @@ -27,10 +27,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.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 java.io.IOException; import java.net.InetSocketAddress; @@ -71,9 +72,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.glassfish.jersey.jettison.internal.entity.JettisonObjectProvider; @@ -96,7 +97,7 @@ public class TestRMHA { private static final String RM3_ADDRESS = "2.2.2.2:2"; private static final String RM3_NODE_ID = "rm3"; - @Before + @BeforeEach public void setUp() throws Exception { configuration = new Configuration(); UserGroupInformation.setConfiguration(configuration); @@ -128,21 +129,20 @@ private void checkMonitorHealth() throws IOException { } private void checkStandbyRMFunctionality() throws IOException { - assertEquals(STATE_ERR, HAServiceState.STANDBY, - rm.adminService.getServiceStatus().getState()); - assertFalse("Active RM services are started", - rm.areActiveServicesRunning()); - assertTrue("RM is not ready to become active", - rm.adminService.getServiceStatus().isReadyToBecomeActive()); + assertEquals(HAServiceState.STANDBY, + rm.adminService.getServiceStatus().getState(), STATE_ERR); + assertFalse(rm.areActiveServicesRunning(), + "Active RM services are started"); + assertTrue(rm.adminService.getServiceStatus().isReadyToBecomeActive(), + "RM is not ready to become active"); } private void checkActiveRMFunctionality() throws Exception { - assertEquals(STATE_ERR, HAServiceState.ACTIVE, - rm.adminService.getServiceStatus().getState()); - assertTrue("Active RM services aren't started", - rm.areActiveServicesRunning()); - assertTrue("RM is not ready to become active", - rm.adminService.getServiceStatus().isReadyToBecomeActive()); + assertEquals(HAServiceState.ACTIVE, + rm.adminService.getServiceStatus().getState(), STATE_ERR); + assertTrue(rm.areActiveServicesRunning(), "Active RM services aren't started"); + assertTrue(rm.adminService.getServiceStatus().isReadyToBecomeActive(), + "RM is not ready to become active"); try { rm.getNewAppId(); @@ -180,7 +180,7 @@ private void checkActiveRMWebServices() throws JSONException { response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject appJson = json.getJSONObject("app"); assertEquals("ACCEPTED", appJson.getString("state")); // Other stuff is verified in the regular web-services related tests @@ -197,7 +197,8 @@ private void checkActiveRMWebServices() throws JSONException { * 6. Stop the RM: All services should stop and RM should not be ready to * become Active */ - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testFailoverAndTransitions() throws Exception { configuration.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false); Configuration conf = new YarnConfiguration(configuration); @@ -207,10 +208,10 @@ public void testFailoverAndTransitions() throws Exception { StateChangeRequestInfo requestInfo = new StateChangeRequestInfo( HAServiceProtocol.RequestSource.REQUEST_BY_USER); - assertEquals(STATE_ERR, HAServiceState.INITIALIZING, - rm.adminService.getServiceStatus().getState()); - assertFalse("RM is ready to become active before being started", - rm.adminService.getServiceStatus().isReadyToBecomeActive()); + assertEquals(HAServiceState.INITIALIZING, + rm.adminService.getServiceStatus().getState(), STATE_ERR); + assertFalse(rm.adminService.getServiceStatus().isReadyToBecomeActive(), + "RM is ready to become active before being started"); checkMonitorHealth(); rm.start(); @@ -251,12 +252,11 @@ public void testFailoverAndTransitions() throws Exception { // 6. Stop the RM. All services should stop and RM should not be ready to // become active rm.stop(); - assertEquals(STATE_ERR, HAServiceState.STOPPING, - rm.adminService.getServiceStatus().getState()); - assertFalse("RM is ready to become active even after it is stopped", - rm.adminService.getServiceStatus().isReadyToBecomeActive()); - assertFalse("Active RM services are started", - rm.areActiveServicesRunning()); + assertEquals(HAServiceState.STOPPING, + rm.adminService.getServiceStatus().getState(), STATE_ERR); + assertFalse(rm.adminService.getServiceStatus().isReadyToBecomeActive(), + "RM is ready to become active even after it is stopped"); + assertFalse(rm.areActiveServicesRunning(), "Active RM services are started"); checkMonitorHealth(); } @@ -341,10 +341,10 @@ protected Dispatcher createDispatcher() { StateChangeRequestInfo requestInfo = new StateChangeRequestInfo( HAServiceProtocol.RequestSource.REQUEST_BY_USER); - assertEquals(STATE_ERR, HAServiceState.INITIALIZING, - rm.adminService.getServiceStatus().getState()); - assertFalse("RM is ready to become active before being started", - rm.adminService.getServiceStatus().isReadyToBecomeActive()); + assertEquals(HAServiceState.INITIALIZING, + rm.adminService.getServiceStatus().getState(), STATE_ERR); + assertFalse(rm.adminService.getServiceStatus().isReadyToBecomeActive(), + "RM is ready to become active before being started"); rm.start(); //call transitions to standby and active a couple of times @@ -359,11 +359,11 @@ protected Dispatcher createDispatcher() { assertTrue(!dispatcher.isStopped()); rm.adminService.transitionToActive(requestInfo); - assertEquals(errorMessageForEventHandler, expectedEventHandlerCount, + assertEquals(expectedEventHandlerCount, ((MyCountingDispatcher) rm.getRMContext().getDispatcher()) - .getEventHandlerCount()); - assertEquals(errorMessageForService, expectedServiceCount, - rm.getServices().size()); + .getEventHandlerCount(), errorMessageForEventHandler); + assertEquals(expectedServiceCount, + rm.getServices().size(), errorMessageForService); // Keep the dispatcher reference before transitioning to standby @@ -371,11 +371,11 @@ protected Dispatcher createDispatcher() { rm.adminService.transitionToStandby(requestInfo); - assertEquals(errorMessageForEventHandler, expectedEventHandlerCount, + assertEquals(expectedEventHandlerCount, ((MyCountingDispatcher) rm.getRMContext().getDispatcher()) - .getEventHandlerCount()); - assertEquals(errorMessageForService, expectedServiceCount, - rm.getServices().size()); + .getEventHandlerCount(), errorMessageForEventHandler); + assertEquals(expectedServiceCount, + rm.getServices().size(), errorMessageForService); assertTrue(dispatcher.isStopped()); @@ -408,7 +408,7 @@ public void testHAIDLookup() { rm.init(conf); fail("Should get an exception here."); } catch (Exception ex) { - Assert.assertTrue(ex.getMessage().contains( + assertTrue(ex.getMessage().contains( "Invalid configuration! Can not find valid RM_HA_ID.")); } } @@ -421,7 +421,8 @@ public void testHAWithRMHostName() throws Exception { innerTestHAWithRMHostName(true); } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testFailoverWhenTransitionToActiveThrowException() throws Exception { configuration.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false); @@ -447,10 +448,10 @@ public synchronized void startInternal() throws Exception { new StateChangeRequestInfo( HAServiceProtocol.RequestSource.REQUEST_BY_USER); - assertEquals(STATE_ERR, HAServiceState.INITIALIZING, rm.adminService - .getServiceStatus().getState()); - assertFalse("RM is ready to become active before being started", - rm.adminService.getServiceStatus().isReadyToBecomeActive()); + assertEquals(HAServiceState.INITIALIZING, rm.adminService + .getServiceStatus().getState(), STATE_ERR); + assertFalse(rm.adminService.getServiceStatus().isReadyToBecomeActive(), + "RM is ready to become active before being started"); checkMonitorHealth(); rm.start(); @@ -460,7 +461,7 @@ public synchronized void startInternal() throws Exception { // 2. Try Transition to active, throw exception try { rm.adminService.transitionToActive(requestInfo); - Assert.fail("Transitioned to Active should throw exception."); + fail("Transitioned to Active should throw exception."); } catch (Exception e) { assertTrue("Error when transitioning to Active mode".contains(e .getMessage())); @@ -500,10 +501,10 @@ void stopActiveServices() { new StateChangeRequestInfo( HAServiceProtocol.RequestSource.REQUEST_BY_USER); - assertEquals(STATE_ERR, HAServiceState.INITIALIZING, rm.adminService - .getServiceStatus().getState()); - assertFalse("RM is ready to become active before being started", - rm.adminService.getServiceStatus().isReadyToBecomeActive()); + assertEquals(HAServiceState.INITIALIZING, rm.adminService + .getServiceStatus().getState(), STATE_ERR); + assertFalse(rm.adminService.getServiceStatus().isReadyToBecomeActive(), + "RM is ready to become active before being started"); checkMonitorHealth(); rm.start(); @@ -562,7 +563,7 @@ public void testFailoverClearsRMContext() throws Exception { verifyClusterMetrics(1, 1, 1, 1, 2048, 1); assertEquals(1, rm.getRMContext().getRMNodes().size()); assertEquals(1, rm.getRMContext().getRMApps().size()); - Assert.assertNotNull("Node not registered", nm); + assertNotNull(nm, "Node not registered"); rm.adminService.transitionToStandby(requestInfo); checkMonitorHealth(); @@ -597,7 +598,7 @@ protected void serviceStart() throws Exception { // 4. Try Transition to active, throw exception try { rm.adminService.transitionToActive(requestInfo); - Assert.fail("Transitioned to Active should throw exception."); + fail("Transitioned to Active should throw exception."); } catch (Exception e) { assertTrue("Error when transitioning to Active mode".contains(e .getMessage())); @@ -608,7 +609,8 @@ protected void serviceStart() throws Exception { assertEquals(0, rm.getRMContext().getRMApps().size()); } - @Test(timeout = 9000000) + @Test + @Timeout(value = 9000) public void testTransitionedToActiveRefreshFail() throws Exception { configuration.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false); rm = new MockRM(configuration) { @@ -648,21 +650,21 @@ protected Dispatcher createDispatcher() { ((FailFastDispatcher) rm.rmContext.getDispatcher()); // Verify transition to transitionToStandby rm.adminService.transitionToStandby(requestInfo); - assertEquals("Fatal Event should be 0", 0, dispatcher.getEventCount()); - assertEquals("HA state should be in standBy State", HAServiceState.STANDBY, - rm.getRMContext().getHAServiceState()); + assertEquals(0, dispatcher.getEventCount(), "Fatal Event should be 0"); + assertEquals(HAServiceState.STANDBY, + rm.getRMContext().getHAServiceState(), "HA state should be in standBy State"); try { // Verify refreshAll call failure and check fail Event is dispatched rm.adminService.transitionToActive(requestInfo); - Assert.fail("Transition to Active should have failed for refreshAll()"); + fail("Transition to Active should have failed for refreshAll()"); } catch (Exception e) { - assertTrue("Service fail Exception expected", - e instanceof ServiceFailedException); + assertTrue(e instanceof ServiceFailedException, + "Service fail Exception expected"); } // Since refreshAll failed we are expecting fatal event to be send // Then fatal event is send RM will shutdown dispatcher.await(); - assertEquals("Fatal Event to be received", 1, dispatcher.getEventCount()); + assertEquals(1, dispatcher.getEventCount(), "Fatal Event to be received"); // Check of refreshAll success HA can be active rm.adminService.transitionToActive(requestInfo); assertEquals(HAServiceState.ACTIVE, rm.getRMContext().getHAServiceState()); @@ -706,7 +708,7 @@ public Boolean get() { } }, 100, 3000); rm.stop(); - Assert.assertEquals(1, appMaster.getLeastLoadedNodes().size()); + assertEquals(1, appMaster.getLeastLoadedNodes().size()); } @@ -744,9 +746,8 @@ public void testResourceProfilesManagerAfterRMWentStandbyThenBackToActive() checkActiveRMFunctionality(); // 5. Check ResourceProfilesManager - Assert.assertNotNull( - "ResourceProfilesManager should not be null!", - rm.getRMContext().getResourceProfilesManager()); + assertNotNull(rm.getRMContext().getResourceProfilesManager(), + "ResourceProfilesManager should not be null!"); } @Test @@ -769,12 +770,12 @@ public void testTransitionedToActiveWithExcludeFileNotExist() throws Exception { if (confFile.exists()) { hasRenamed = confFile.renameTo(backupConfFile); if (!hasRenamed) { - Assert.fail("Can not rename " + confFile.getAbsolutePath() + " to " + fail("Can not rename " + confFile.getAbsolutePath() + " to " + backupConfFile.getAbsolutePath()); } } if (!confFile.createNewFile()) { - Assert.fail( + fail( "Can not create " + YarnConfiguration.YARN_SITE_CONFIGURATION_FILE); } output = new DataOutputStream(Files.newOutputStream(confFile.toPath())); @@ -867,20 +868,20 @@ public void innerTestHAWithRMHostName(boolean includeBindHost) { rm = new MockRM(conf); rm.init(conf); for (String confKey : YarnConfiguration.getServiceAddressConfKeys(conf)) { - assertEquals("RPC address not set for " + confKey, - RM1_ADDRESS, conf.get(HAUtil.addSuffix(confKey, RM1_NODE_ID))); - assertEquals("RPC address not set for " + confKey, - RM2_ADDRESS, conf.get(HAUtil.addSuffix(confKey, RM2_NODE_ID))); - assertEquals("RPC address not set for " + confKey, - RM3_ADDRESS, conf.get(HAUtil.addSuffix(confKey, RM3_NODE_ID))); + assertEquals(RM1_ADDRESS, conf.get(HAUtil.addSuffix(confKey, RM1_NODE_ID)), + "RPC address not set for " + confKey); + assertEquals(RM2_ADDRESS, conf.get(HAUtil.addSuffix(confKey, RM2_NODE_ID)), + "RPC address not set for " + confKey); + assertEquals(RM3_ADDRESS, conf.get(HAUtil.addSuffix(confKey, RM3_NODE_ID)), + "RPC address not set for " + confKey); if (includeBindHost) { - assertEquals("Web address misconfigured WITH bind-host", - rm.webAppAddress.substring(0, 7), "9.9.9.9"); + assertEquals(rm.webAppAddress.substring(0, 7), "9.9.9.9", + "Web address misconfigured WITH bind-host"); } else { //YarnConfiguration tries to figure out which rm host it's on by binding to it, //which doesn't happen for any of these fake addresses, so we end up with 0.0.0.0 - assertEquals("Web address misconfigured WITHOUT bind-host", - rm.webAppAddress.substring(0, 7), "0.0.0.0"); + assertEquals(rm.webAppAddress.substring(0, 7), "0.0.0.0", + "Web address misconfigured WITHOUT bind-host"); } } } catch (YarnRuntimeException e) { @@ -900,13 +901,12 @@ public void innerTestHAWithRMHostName(boolean includeBindHost) { Configuration conf = new YarnConfiguration(configuration); rm = new MockRM(conf); rm.init(conf); - assertEquals("RPC address not set for " + YarnConfiguration.RM_ADDRESS, - "1.1.1.1:8032", - conf.get(HAUtil.addSuffix(YarnConfiguration.RM_ADDRESS, RM1_NODE_ID))); - assertEquals("RPC address not set for " + YarnConfiguration.RM_ADDRESS, - "0.0.0.0:8032", - conf.get(HAUtil.addSuffix(YarnConfiguration.RM_ADDRESS, RM2_NODE_ID))); - + assertEquals("1.1.1.1:8032", + conf.get(HAUtil.addSuffix(YarnConfiguration.RM_ADDRESS, RM1_NODE_ID)), + "RPC address not set for " + YarnConfiguration.RM_ADDRESS); + assertEquals("0.0.0.0:8032", + conf.get(HAUtil.addSuffix(YarnConfiguration.RM_ADDRESS, RM2_NODE_ID)), + "RPC address not set for " + YarnConfiguration.RM_ADDRESS); } catch (YarnRuntimeException e) { fail("Should not throw any exceptions."); } @@ -941,11 +941,11 @@ private void verifyClusterMetrics(int activeNodes, int appsSubmitted, Thread.sleep(1000); } } - assertTrue(message, isAllMetricAssertionDone); + assertTrue(isAllMetricAssertionDone, message); } private void assertMetric(String metricName, long expected, long actual) { - assertEquals("Incorrect value for metric " + metricName, expected, actual); + assertEquals(expected, actual, "Incorrect value for metric " + metricName); } @SuppressWarnings("rawtypes") diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHAForNodeLabels.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHAForNodeLabels.java index 5d6638ce13515..76cba5406ab46 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHAForNodeLabels.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHAForNodeLabels.java @@ -18,6 +18,8 @@ package org.apache.hadoop.yarn.server.resourcemanager; +import static org.junit.jupiter.api.Assertions.assertTrue; + import java.io.File; import java.util.Arrays; import java.util.HashMap; @@ -29,9 +31,8 @@ import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeLabel; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; @@ -39,7 +40,7 @@ public class TestRMHAForNodeLabels extends RMHATestBase { public static final Logger LOG = LoggerFactory .getLogger(TestRMHAForNodeLabels.class); - @Before + @BeforeEach @Override public void setup() throws Exception { super.setup(); @@ -81,13 +82,12 @@ public void testRMHARecoverNodeLabels() throws Exception { explicitFailover(); // Check labels in rm2 - Assert - .assertTrue(rm2.getRMContext().getNodeLabelManager() - .getClusterNodeLabelNames() - .containsAll(ImmutableSet.of("a", "b", "c"))); - Assert.assertTrue(rm2.getRMContext().getNodeLabelManager() + assertTrue(rm2.getRMContext().getNodeLabelManager() + .getClusterNodeLabelNames() + .containsAll(ImmutableSet.of("a", "b", "c"))); + assertTrue(rm2.getRMContext().getNodeLabelManager() .getNodeLabels().get(NodeId.newInstance("host1", 0)).contains("a")); - Assert.assertTrue(rm2.getRMContext().getNodeLabelManager() + assertTrue(rm2.getRMContext().getNodeLabelManager() .getNodeLabels().get(NodeId.newInstance("host2", 0)).contains("b")); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHAMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHAMetrics.java index b55cee6213f71..88be542b65a62 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHAMetrics.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHAMetrics.java @@ -23,15 +23,16 @@ import org.apache.hadoop.yarn.conf.HAUtil; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import javax.management.MBeanServer; import javax.management.ObjectName; import java.lang.management.ManagementFactory; -import static junit.framework.TestCase.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; /** * Metrics related RM HA testing. Metrics are mostly static singletons. To @@ -47,7 +48,7 @@ public class TestRMHAMetrics { private static final String RM2_ADDRESS = "0.0.0.0:0"; private static final String RM2_NODE_ID = "rm2"; - @Before + @BeforeEach public void setUp() throws Exception { configuration = new Configuration(); UserGroupInformation.setConfiguration(configuration); @@ -65,7 +66,8 @@ public void setUp() throws Exception { DefaultMetricsSystem.shutdown(); } - @Test(timeout = 300000) + @Test + @Timeout(value = 300) public void testMetricsAfterTransitionToStandby() throws Exception { configuration.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false); Configuration conf = new YarnConfiguration(configuration); @@ -76,20 +78,19 @@ public void testMetricsAfterTransitionToStandby() throws Exception { ObjectName mxbeanName = new ObjectName("Hadoop:service=ResourceManager,name=RMInfo"); - Assert.assertEquals("initializing", + assertEquals("initializing", (String) mbs.getAttribute(mxbeanName, "State")); rm.start(); - Assert.assertEquals("standby", + assertEquals("standby", (String) mbs.getAttribute(mxbeanName, "State")); rm.transitionToActive(); - Assert - .assertEquals("active", + assertEquals("active", (String) mbs.getAttribute(mxbeanName, "State")); rm.transitionToStandby(true); - Assert.assertEquals("standby", + assertEquals("standby", (String) mbs.getAttribute(mxbeanName, "State")); assertNotNull(DefaultMetricsSystem.instance().getSource("JvmMetrics")); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHATimelineCollectors.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHATimelineCollectors.java index 99bf1822ebb6a..01899e4b6714d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHATimelineCollectors.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHATimelineCollectors.java @@ -25,8 +25,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl; import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.util.function.Supplier; @@ -34,7 +34,7 @@ import java.util.List; import java.util.Map; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; /** * Test if the new active RM could recover collector status on a state @@ -42,7 +42,7 @@ */ public class TestRMHATimelineCollectors extends RMHATestBase { - @Before + @BeforeEach @Override public void setup() throws Exception { super.setup(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java index 3a0288715075e..aca9ebe14da3a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java @@ -18,9 +18,11 @@ package org.apache.hadoop.yarn.server.resourcemanager; import static org.apache.hadoop.yarn.server.resourcemanager.MockNM.createMockNodeStatus; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; @@ -86,12 +88,11 @@ import org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils; import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.mockito.ArgumentMatchers; -import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -128,7 +129,7 @@ public void handle(NodesListManagerEvent event) { } - @Before + @BeforeEach public void setUp() throws Exception { InlineDispatcher rmDispatcher = new InlineDispatcher(); @@ -172,7 +173,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { nodesListManagerEventsNodeStateSequence.clear(); } - @After + @AfterEach public void tearDown() throws Exception { } @@ -264,7 +265,8 @@ private static NMContainerStatus createNMContainerStatus( CommonNodeLabelsManager.NO_LABEL, executionType, -1); } - @Test (timeout = 5000) + @Test + @Timeout(value = 5) public void testExpiredContainer() { NodeStatus mockNodeStatus = createMockNodeStatus(); // Start the node @@ -276,7 +278,7 @@ public void testExpiredContainer() { BuilderUtils.newApplicationAttemptId( BuilderUtils.newApplicationId(0, 0), 0), 0); node.handle(new RMNodeCleanContainerEvent(null, completedContainerId)); - Assert.assertEquals(1, node.getContainersToCleanUp().size()); + assertEquals(1, node.getContainersToCleanUp().size()); // Now verify that scheduler isn't notified of an expired container // by checking number of 'completedContainers' it got in the previous event @@ -301,35 +303,35 @@ public void testStatusUpdateOnDecommissioningNode() { int initialActive = cm.getNumActiveNMs(); int initialDecommissioning = cm.getNumDecommissioningNMs(); int initialDecommissioned = cm.getNumDecommisionedNMs(); - Assert.assertEquals(NodeState.DECOMMISSIONING, node.getState()); + assertEquals(NodeState.DECOMMISSIONING, node.getState()); // Verify node in DECOMMISSIONING won't be changed by status update // with running apps RMNodeStatusEvent statusEvent = getMockRMNodeStatusEventWithRunningApps(); node.handle(statusEvent); - Assert.assertEquals(NodeState.DECOMMISSIONING, node.getState()); - Assert.assertEquals("Active Nodes", initialActive, cm.getNumActiveNMs()); - Assert.assertEquals("Decommissioning Nodes", initialDecommissioning, - cm.getNumDecommissioningNMs()); - Assert.assertEquals("Decommissioned Nodes", initialDecommissioned, - cm.getNumDecommisionedNMs()); + assertEquals(NodeState.DECOMMISSIONING, node.getState()); + assertEquals(initialActive, cm.getNumActiveNMs(), "Active Nodes"); + assertEquals(initialDecommissioning, + cm.getNumDecommissioningNMs(), "Decommissioning Nodes"); + assertEquals(initialDecommissioned, + cm.getNumDecommisionedNMs(), "Decommissioned Nodes"); } @Test public void testRecommissionNode() { RMNodeImpl node = getDecommissioningNode(); - Assert.assertEquals(NodeState.DECOMMISSIONING, node.getState()); + assertEquals(NodeState.DECOMMISSIONING, node.getState()); ClusterMetrics cm = ClusterMetrics.getMetrics(); int initialActive = cm.getNumActiveNMs(); int initialDecommissioning = cm.getNumDecommissioningNMs(); node.handle(new RMNodeEvent(node.getNodeID(), RMNodeEventType.RECOMMISSION)); - Assert.assertEquals(NodeState.RUNNING, node.getState()); - Assert - .assertEquals("Active Nodes", initialActive + 1, cm.getNumActiveNMs()); - Assert.assertEquals("Decommissioning Nodes", initialDecommissioning - 1, - cm.getNumDecommissioningNMs()); + assertEquals(NodeState.RUNNING, node.getState()); + assertEquals(initialActive + 1, cm.getNumActiveNMs(), "Active Nodes"); + assertEquals(initialDecommissioning - 1, + cm.getNumDecommissioningNMs(), "Decommissioning Nodes"); } - @Test (timeout = 5000) + @Test + @Timeout(value = 5) public void testContainerUpdate() throws InterruptedException{ NodeStatus mockNodeStatus = createMockNodeStatus(); //Start the node @@ -347,8 +349,8 @@ public void testContainerUpdate() throws InterruptedException{ BuilderUtils.newApplicationAttemptId(app1, 1), 1); ContainerId completedContainerIdFromNode2_2 = BuilderUtils.newContainerId( BuilderUtils.newApplicationAttemptId(app1, 1), 2); - rmContext.getRMApps().put(app0, Mockito.mock(RMApp.class)); - rmContext.getRMApps().put(app1, Mockito.mock(RMApp.class)); + rmContext.getRMApps().put(app0, mock(RMApp.class)); + rmContext.getRMApps().put(app1, mock(RMApp.class)); RMNodeStatusEvent statusEventFromNode1 = getMockRMNodeStatusEvent(null); RMNodeStatusEvent statusEventFromNode2_1 = getMockRMNodeStatusEvent(null); @@ -364,8 +366,8 @@ public void testContainerUpdate() throws InterruptedException{ doReturn(Collections.singletonList(containerStatusFromNode1)) .when(statusEventFromNode1).getContainers(); node.handle(statusEventFromNode1); - Assert.assertEquals(1, completedContainers.size()); - Assert.assertEquals(completedContainerIdFromNode1, + assertEquals(1, completedContainers.size()); + assertEquals(completedContainerIdFromNode1, completedContainers.get(0).getContainerId()); completedContainers.clear(); @@ -381,10 +383,10 @@ public void testContainerUpdate() throws InterruptedException{ node2.setNextHeartBeat(true); node2.handle(statusEventFromNode2_2); - Assert.assertEquals(2, completedContainers.size()); - Assert.assertEquals(completedContainerIdFromNode2_1,completedContainers.get(0) - .getContainerId()); - Assert.assertEquals(completedContainerIdFromNode2_2,completedContainers.get(1) + assertEquals(2, completedContainers.size()); + assertEquals(completedContainerIdFromNode2_1, completedContainers.get(0) + .getContainerId()); + assertEquals(completedContainerIdFromNode2_2, completedContainers.get(1) .getContainerId()); } @@ -443,11 +445,11 @@ public void testAddWithAllocatedContainers() { Arrays.asList(newContainerStatus, runningContainerStatus, newOppContainerStatus, runningOppContainerStatus), null, mockNodeStatus)); - Assert.assertEquals(NodeState.RUNNING, node.getState()); - Assert.assertNotNull(nodesListManagerEvent); - Assert.assertEquals(NodesListManagerEventType.NODE_USABLE, + assertEquals(NodeState.RUNNING, node.getState()); + assertNotNull(nodesListManagerEvent); + assertEquals(NodesListManagerEventType.NODE_USABLE, nodesListManagerEvent.getType()); - Assert.assertEquals(expectedResource, node.getAllocatedContainerResource()); + assertEquals(expectedResource, node.getAllocatedContainerResource()); } /** @@ -456,14 +458,15 @@ public void testAddWithAllocatedContainers() { * upon a node update. Resources should be counted for both GUARANTEED * and OPPORTUNISTIC containers. */ - @Test (timeout = 5000) + @Test + @Timeout(value = 5) public void testAllocatedContainerUpdate() { NodeStatus mockNodeStatus = createMockNodeStatus(); //Start the node node.handle(new RMNodeStartedEvent(null, null, null, mockNodeStatus)); // Make sure that the node starts with no allocated resources - Assert.assertEquals(Resources.none(), node.getAllocatedContainerResource()); + assertEquals(Resources.none(), node.getAllocatedContainerResource()); ApplicationId app0 = BuilderUtils.newApplicationId(0, 0); final ContainerId newContainerId = BuilderUtils.newContainerId( @@ -471,7 +474,7 @@ public void testAllocatedContainerUpdate() { final ContainerId runningContainerId = BuilderUtils.newContainerId( BuilderUtils.newApplicationAttemptId(app0, 0), 1); - rmContext.getRMApps().put(app0, Mockito.mock(RMApp.class)); + rmContext.getRMApps().put(app0, mock(RMApp.class)); RMNodeStatusEvent statusEventFromNode1 = getMockRMNodeStatusEvent(null); @@ -495,7 +498,7 @@ public void testAllocatedContainerUpdate() { newContainerStatusFromNode, runningContainerStatusFromNode)); doReturn(containerStatuses).when(statusEventFromNode1).getContainers(); node.handle(statusEventFromNode1); - Assert.assertEquals(Resource.newInstance(300, 3), + assertEquals(Resource.newInstance(300, 3), node.getAllocatedContainerResource()); final ContainerId newOppContainerId = BuilderUtils.newContainerId( @@ -524,7 +527,7 @@ public void testAllocatedContainerUpdate() { // The result here should be double the first check, // since allocated resources are doubled, just // with different execution types - Assert.assertEquals(Resource.newInstance(600, 6), + assertEquals(Resource.newInstance(600, 6), node.getAllocatedContainerResource()); RMNodeStatusEvent statusEventFromNode3 = getMockRMNodeStatusEvent(null); @@ -548,7 +551,7 @@ public void testAllocatedContainerUpdate() { // Adding completed containers should not have changed // the resources allocated - Assert.assertEquals(Resource.newInstance(600, 6), + assertEquals(Resource.newInstance(600, 6), node.getAllocatedContainerResource()); RMNodeStatusEvent emptyStatusEventFromNode = @@ -559,11 +562,12 @@ public void testAllocatedContainerUpdate() { node.handle(emptyStatusEventFromNode); // Passing an empty containers list should yield no resources allocated - Assert.assertEquals(Resources.none(), + assertEquals(Resources.none(), node.getAllocatedContainerResource()); } - @Test (timeout = 5000) + @Test + @Timeout(value = 5) public void testStatusChange(){ NodeStatus mockNodeStatus = createMockNodeStatus(); //Start the node @@ -596,9 +600,9 @@ public void testStatusChange(){ node.handle(statusEvent1); node.handle(statusEvent2); verify(scheduler, times(1)).handle(any(NodeAddedSchedulerEvent.class)); - Assert.assertEquals(2, node.getQueueSize()); + assertEquals(2, node.getQueueSize()); node.handle(new RMNodeEvent(node.getNodeID(), RMNodeEventType.EXPIRE)); - Assert.assertEquals(0, node.getQueueSize()); + assertEquals(0, node.getQueueSize()); } @Test @@ -611,15 +615,12 @@ public void testRunningExpire() { int initialDecommissioned = cm.getNumDecommisionedNMs(); int initialRebooted = cm.getNumRebootedNMs(); node.handle(new RMNodeEvent(node.getNodeID(), RMNodeEventType.EXPIRE)); - Assert.assertEquals("Active Nodes", initialActive - 1, cm.getNumActiveNMs()); - Assert.assertEquals("Lost Nodes", initialLost + 1, cm.getNumLostNMs()); - Assert.assertEquals("Unhealthy Nodes", - initialUnhealthy, cm.getUnhealthyNMs()); - Assert.assertEquals("Decommissioned Nodes", - initialDecommissioned, cm.getNumDecommisionedNMs()); - Assert.assertEquals("Rebooted Nodes", - initialRebooted, cm.getNumRebootedNMs()); - Assert.assertEquals(NodeState.LOST, node.getState()); + assertEquals(initialActive - 1, cm.getNumActiveNMs(), "Active Nodes"); + assertEquals(initialLost + 1, cm.getNumLostNMs(), "Lost Nodes"); + assertEquals(initialUnhealthy, cm.getUnhealthyNMs(), "Unhealthy Nodes"); + assertEquals(initialDecommissioned, cm.getNumDecommisionedNMs(), "Decommissioned Nodes"); + assertEquals(initialRebooted, cm.getNumRebootedNMs(), "Rebooted Nodes"); + assertEquals(NodeState.LOST, node.getState()); } @Test @@ -633,34 +634,29 @@ public void testRunningExpireMultiple() { int initialDecommissioned = cm.getNumDecommisionedNMs(); int initialRebooted = cm.getNumRebootedNMs(); node1.handle(new RMNodeEvent(node1.getNodeID(), RMNodeEventType.EXPIRE)); - Assert.assertEquals("Active Nodes", initialActive - 1, cm.getNumActiveNMs()); - Assert.assertEquals("Lost Nodes", initialLost + 1, cm.getNumLostNMs()); - Assert.assertEquals("Unhealthy Nodes", initialUnhealthy, - cm.getUnhealthyNMs()); - Assert.assertEquals("Decommissioned Nodes", initialDecommissioned, - cm.getNumDecommisionedNMs()); - Assert.assertEquals("Rebooted Nodes", initialRebooted, - cm.getNumRebootedNMs()); - Assert.assertEquals(NodeState.LOST, node1.getState()); - Assert.assertTrue("Node " + node1.toString() + " should be inactive", - rmContext.getInactiveRMNodes().containsKey(node1.getNodeID())); - Assert.assertFalse("Node " + node2.toString() + " should not be inactive", - rmContext.getInactiveRMNodes().containsKey(node2.getNodeID())); + assertEquals(initialActive - 1, cm.getNumActiveNMs(), "Active Nodes"); + assertEquals(initialLost + 1, cm.getNumLostNMs(), "Lost Nodes"); + assertEquals(initialUnhealthy, cm.getUnhealthyNMs(), "Unhealthy Nodes"); + assertEquals(initialDecommissioned, cm.getNumDecommisionedNMs(), + "Decommissioned Nodes"); + assertEquals(initialRebooted, cm.getNumRebootedNMs(), "Rebooted Nodes"); + assertEquals(NodeState.LOST, node1.getState()); + assertTrue(rmContext.getInactiveRMNodes().containsKey(node1.getNodeID()), + "Node " + node1.toString() + " should be inactive"); + assertFalse(rmContext.getInactiveRMNodes().containsKey(node2.getNodeID()), + "Node " + node2.toString() + " should not be inactive"); node2.handle(new RMNodeEvent(node1.getNodeID(), RMNodeEventType.EXPIRE)); - Assert.assertEquals("Active Nodes", initialActive - 2, cm.getNumActiveNMs()); - Assert.assertEquals("Lost Nodes", initialLost + 2, cm.getNumLostNMs()); - Assert.assertEquals("Unhealthy Nodes", initialUnhealthy, - cm.getUnhealthyNMs()); - Assert.assertEquals("Decommissioned Nodes", initialDecommissioned, - cm.getNumDecommisionedNMs()); - Assert.assertEquals("Rebooted Nodes", initialRebooted, - cm.getNumRebootedNMs()); - Assert.assertEquals(NodeState.LOST, node2.getState()); - Assert.assertTrue("Node " + node1.toString() + " should be inactive", - rmContext.getInactiveRMNodes().containsKey(node1.getNodeID())); - Assert.assertTrue("Node " + node2.toString() + " should be inactive", - rmContext.getInactiveRMNodes().containsKey(node2.getNodeID())); + assertEquals(initialActive - 2, cm.getNumActiveNMs(), "Active Nodes"); + assertEquals(initialLost + 2, cm.getNumLostNMs(), "Lost Nodes"); + assertEquals(initialUnhealthy, cm.getUnhealthyNMs(), "Unhealthy Nodes"); + assertEquals(initialDecommissioned, cm.getNumDecommisionedNMs(), "Decommissioned Nodes"); + assertEquals(initialRebooted, cm.getNumRebootedNMs(), "Rebooted Nodes"); + assertEquals(NodeState.LOST, node2.getState()); + assertTrue(rmContext.getInactiveRMNodes().containsKey(node1.getNodeID()), + "Node " + node1.toString() + " should be inactive"); + assertTrue(rmContext.getInactiveRMNodes().containsKey(node2.getNodeID()), + "Node " + node2.toString() + " should be inactive"); } @Test @@ -673,15 +669,12 @@ public void testUnhealthyExpire() { int initialDecommissioned = cm.getNumDecommisionedNMs(); int initialRebooted = cm.getNumRebootedNMs(); node.handle(new RMNodeEvent(node.getNodeID(), RMNodeEventType.EXPIRE)); - Assert.assertEquals("Active Nodes", initialActive, cm.getNumActiveNMs()); - Assert.assertEquals("Lost Nodes", initialLost + 1, cm.getNumLostNMs()); - Assert.assertEquals("Unhealthy Nodes", - initialUnhealthy - 1, cm.getUnhealthyNMs()); - Assert.assertEquals("Decommissioned Nodes", - initialDecommissioned, cm.getNumDecommisionedNMs()); - Assert.assertEquals("Rebooted Nodes", - initialRebooted, cm.getNumRebootedNMs()); - Assert.assertEquals(NodeState.LOST, node.getState()); + assertEquals(initialActive, cm.getNumActiveNMs(), "Active Nodes"); + assertEquals(initialLost + 1, cm.getNumLostNMs(), "Lost Nodes"); + assertEquals(initialUnhealthy - 1, cm.getUnhealthyNMs(), "Unhealthy Nodes"); + assertEquals(initialDecommissioned, cm.getNumDecommisionedNMs(), "Decommissioned Nodes"); + assertEquals(initialRebooted, cm.getNumRebootedNMs(), "Rebooted Nodes"); + assertEquals(NodeState.LOST, node.getState()); } @Test @@ -690,7 +683,7 @@ public void testUnhealthyExpireForSchedulerRemove() { verify(scheduler, times(1)).handle(any(NodeRemovedSchedulerEvent.class)); node.handle(new RMNodeEvent(node.getNodeID(), RMNodeEventType.EXPIRE)); verify(scheduler, times(1)).handle(any(NodeRemovedSchedulerEvent.class)); - Assert.assertEquals(NodeState.LOST, node.getState()); + assertEquals(NodeState.LOST, node.getState()); } @Test @@ -704,15 +697,12 @@ public void testRunningDecommission() { int initialRebooted = cm.getNumRebootedNMs(); node.handle(new RMNodeEvent(node.getNodeID(), RMNodeEventType.DECOMMISSION)); - Assert.assertEquals("Active Nodes", initialActive - 1, cm.getNumActiveNMs()); - Assert.assertEquals("Lost Nodes", initialLost, cm.getNumLostNMs()); - Assert.assertEquals("Unhealthy Nodes", - initialUnhealthy, cm.getUnhealthyNMs()); - Assert.assertEquals("Decommissioned Nodes", - initialDecommissioned + 1, cm.getNumDecommisionedNMs()); - Assert.assertEquals("Rebooted Nodes", - initialRebooted, cm.getNumRebootedNMs()); - Assert.assertEquals(NodeState.DECOMMISSIONED, node.getState()); + assertEquals(initialActive - 1, cm.getNumActiveNMs(), "Active Nodes"); + assertEquals(initialLost, cm.getNumLostNMs(), "Lost Nodes"); + assertEquals(initialUnhealthy, cm.getUnhealthyNMs(), "Unhealthy Nodes"); + assertEquals(initialDecommissioned + 1, cm.getNumDecommisionedNMs(), "Decommissioned Nodes"); + assertEquals(initialRebooted, cm.getNumRebootedNMs(), "Rebooted Nodes"); + assertEquals(NodeState.DECOMMISSIONED, node.getState()); } @Test @@ -726,17 +716,16 @@ public void testDecommissionOnDecommissioningNode() { int initialRebooted = cm.getNumRebootedNMs(); int initialDecommissioning = cm.getNumDecommissioningNMs(); node.handle(new RMNodeEvent(node.getNodeID(), RMNodeEventType.DECOMMISSION)); - Assert.assertEquals("Active Nodes", initialActive, cm.getNumActiveNMs()); - Assert.assertEquals("Lost Nodes", initialLost, cm.getNumLostNMs()); - Assert.assertEquals("Unhealthy Nodes", initialUnhealthy, - cm.getUnhealthyNMs()); - Assert.assertEquals("Decommissioning Nodes", initialDecommissioning - 1, - cm.getNumDecommissioningNMs()); - Assert.assertEquals("Decommissioned Nodes", initialDecommissioned + 1, - cm.getNumDecommisionedNMs()); - Assert.assertEquals("Rebooted Nodes", initialRebooted, - cm.getNumRebootedNMs()); - Assert.assertEquals(NodeState.DECOMMISSIONED, node.getState()); + assertEquals(initialActive, cm.getNumActiveNMs(), "Active Nodes"); + assertEquals(initialLost, cm.getNumLostNMs(), "Lost Nodes"); + assertEquals(initialUnhealthy, cm.getUnhealthyNMs(), "Unhealthy Nodes"); + assertEquals(initialDecommissioning - 1, + cm.getNumDecommissioningNMs(), "Decommissioning Nodes"); + assertEquals(initialDecommissioned + 1, + cm.getNumDecommisionedNMs(), "Decommissioned Nodes"); + assertEquals(initialRebooted, + cm.getNumRebootedNMs(), "Rebooted Nodes"); + assertEquals(NodeState.DECOMMISSIONED, node.getState()); } @Test @@ -750,15 +739,14 @@ public void testUnhealthyDecommission() { int initialRebooted = cm.getNumRebootedNMs(); node.handle(new RMNodeEvent(node.getNodeID(), RMNodeEventType.DECOMMISSION)); - Assert.assertEquals("Active Nodes", initialActive, cm.getNumActiveNMs()); - Assert.assertEquals("Lost Nodes", initialLost, cm.getNumLostNMs()); - Assert.assertEquals("Unhealthy Nodes", - initialUnhealthy - 1, cm.getUnhealthyNMs()); - Assert.assertEquals("Decommissioned Nodes", - initialDecommissioned + 1, cm.getNumDecommisionedNMs()); - Assert.assertEquals("Rebooted Nodes", - initialRebooted, cm.getNumRebootedNMs()); - Assert.assertEquals(NodeState.DECOMMISSIONED, node.getState()); + assertEquals(initialActive, cm.getNumActiveNMs(), "Active Nodes"); + assertEquals(initialLost, cm.getNumLostNMs(), "Lost Nodes"); + assertEquals(initialUnhealthy - 1, cm.getUnhealthyNMs(), + "Unhealthy Nodes"); + assertEquals(initialDecommissioned + 1, cm.getNumDecommisionedNMs(), + "Decommissioned Nodes"); + assertEquals(initialRebooted, cm.getNumRebootedNMs(), "Rebooted Nodes"); + assertEquals(NodeState.DECOMMISSIONED, node.getState()); } // Test Decommissioning on a unhealthy node will make it decommissioning. @@ -774,18 +762,14 @@ public void testUnhealthyDecommissioning() { int initialRebooted = cm.getNumRebootedNMs(); node.handle(new RMNodeEvent(node.getNodeID(), RMNodeEventType.GRACEFUL_DECOMMISSION)); - Assert.assertEquals("Active Nodes", initialActive, - cm.getNumActiveNMs()); - Assert.assertEquals("Lost Nodes", initialLost, cm.getNumLostNMs()); - Assert.assertEquals("Unhealthy Nodes", - initialUnhealthy - 1, cm.getUnhealthyNMs()); - Assert.assertEquals("Decommissioned Nodes", initialDecommissioned, - cm.getNumDecommisionedNMs()); - Assert.assertEquals("Decommissioning Nodes", initialDecommissioning + 1, - cm.getNumDecommissioningNMs()); - Assert.assertEquals("Rebooted Nodes", - initialRebooted, cm.getNumRebootedNMs()); - Assert.assertEquals(NodeState.DECOMMISSIONING, node.getState()); + assertEquals(initialActive, cm.getNumActiveNMs(), "Active Nodes"); + assertEquals(initialLost, cm.getNumLostNMs(), "Lost Nodes"); + assertEquals(initialUnhealthy - 1, cm.getUnhealthyNMs(), "Unhealthy Nodes"); + assertEquals(initialDecommissioned, cm.getNumDecommisionedNMs(), "Decommissioned Nodes"); + assertEquals(initialDecommissioning + 1, cm.getNumDecommissioningNMs(), + "Decommissioning Nodes"); + assertEquals(initialRebooted, cm.getNumRebootedNMs(), "Rebooted Nodes"); + assertEquals(NodeState.DECOMMISSIONING, node.getState()); } @Test @@ -799,15 +783,14 @@ public void testRunningRebooting() { int initialRebooted = cm.getNumRebootedNMs(); node.handle(new RMNodeEvent(node.getNodeID(), RMNodeEventType.REBOOTING)); - Assert.assertEquals("Active Nodes", initialActive - 1, cm.getNumActiveNMs()); - Assert.assertEquals("Lost Nodes", initialLost, cm.getNumLostNMs()); - Assert.assertEquals("Unhealthy Nodes", - initialUnhealthy, cm.getUnhealthyNMs()); - Assert.assertEquals("Decommissioned Nodes", - initialDecommissioned, cm.getNumDecommisionedNMs()); - Assert.assertEquals("Rebooted Nodes", - initialRebooted + 1, cm.getNumRebootedNMs()); - Assert.assertEquals(NodeState.REBOOTED, node.getState()); + assertEquals(initialActive - 1, cm.getNumActiveNMs(), "Active Nodes"); + assertEquals(initialLost, cm.getNumLostNMs(), "Lost Nodes"); + assertEquals(initialUnhealthy, cm.getUnhealthyNMs(), "Unhealthy Nodes"); + assertEquals(initialDecommissioned, cm.getNumDecommisionedNMs(), + "Decommissioned Nodes"); + assertEquals(initialRebooted + 1, cm.getNumRebootedNMs(), + "Rebooted Nodes"); + assertEquals(NodeState.REBOOTED, node.getState()); } @Test @@ -821,15 +804,15 @@ public void testUnhealthyRebooting() { int initialRebooted = cm.getNumRebootedNMs(); node.handle(new RMNodeEvent(node.getNodeID(), RMNodeEventType.REBOOTING)); - Assert.assertEquals("Active Nodes", initialActive, cm.getNumActiveNMs()); - Assert.assertEquals("Lost Nodes", initialLost, cm.getNumLostNMs()); - Assert.assertEquals("Unhealthy Nodes", - initialUnhealthy - 1, cm.getUnhealthyNMs()); - Assert.assertEquals("Decommissioned Nodes", - initialDecommissioned, cm.getNumDecommisionedNMs()); - Assert.assertEquals("Rebooted Nodes", - initialRebooted + 1, cm.getNumRebootedNMs()); - Assert.assertEquals(NodeState.REBOOTED, node.getState()); + assertEquals(initialActive, cm.getNumActiveNMs(), "Active Nodes"); + assertEquals(initialLost, cm.getNumLostNMs(), "Lost Nodes"); + assertEquals(initialUnhealthy - 1, cm.getUnhealthyNMs(), + "Unhealthy Nodes"); + assertEquals(initialDecommissioned, cm.getNumDecommisionedNMs(), + "Decommissioned Nodes"); + assertEquals(initialRebooted + 1, cm.getNumRebootedNMs(), + "Rebooted Nodes"); + assertEquals(NodeState.REBOOTED, node.getState()); } @Test @@ -848,32 +831,31 @@ public void testAddUnhealthyNode() { node.handle(new RMNodeStartedEvent(node.getNodeID(), null, null, nodeStatus)); - Assert.assertEquals("Unhealthy Nodes", - initialUnhealthy + 1, cm.getUnhealthyNMs()); - Assert.assertEquals("Active Nodes", initialActive, cm.getNumActiveNMs()); - Assert.assertEquals("Lost Nodes", initialLost, cm.getNumLostNMs()); - Assert.assertEquals("Decommissioned Nodes", - initialDecommissioned, cm.getNumDecommisionedNMs()); - Assert.assertEquals("Rebooted Nodes", - initialRebooted, cm.getNumRebootedNMs()); - Assert.assertEquals(NodeState.UNHEALTHY, node.getState()); + assertEquals(initialUnhealthy + 1, cm.getUnhealthyNMs(), + "Unhealthy Nodes"); + assertEquals(initialActive, cm.getNumActiveNMs(), "Active Nodes"); + assertEquals(initialLost, cm.getNumLostNMs(), "Lost Nodes"); + assertEquals(initialDecommissioned, cm.getNumDecommisionedNMs(), "Decommissioned Nodes"); + assertEquals(initialRebooted, cm.getNumRebootedNMs(), "Rebooted Nodes"); + assertEquals(NodeState.UNHEALTHY, node.getState()); } @Test public void testNMShutdown() { RMNodeImpl node = getRunningNode(); node.handle(new RMNodeEvent(node.getNodeID(), RMNodeEventType.SHUTDOWN)); - Assert.assertEquals(NodeState.SHUTDOWN, node.getState()); + assertEquals(NodeState.SHUTDOWN, node.getState()); } @Test public void testUnhealthyNMShutdown() { RMNodeImpl node = getUnhealthyNode(); node.handle(new RMNodeEvent(node.getNodeID(), RMNodeEventType.SHUTDOWN)); - Assert.assertEquals(NodeState.SHUTDOWN, node.getState()); + assertEquals(NodeState.SHUTDOWN, node.getState()); } - @Test(timeout=20000) + @Test + @Timeout(value = 20) public void testUpdateHeartbeatResponseForCleanup() { RMNodeImpl node = getRunningNode(); NodeId nodeId = node.getNodeID(); @@ -883,36 +865,37 @@ public void testUpdateHeartbeatResponseForCleanup() { BuilderUtils.newApplicationAttemptId( BuilderUtils.newApplicationId(0, 0), 0), 0); node.handle(new RMNodeCleanContainerEvent(nodeId, completedContainerId)); - Assert.assertEquals(1, node.getContainersToCleanUp().size()); + assertEquals(1, node.getContainersToCleanUp().size()); // Finish an application ApplicationId finishedAppId = BuilderUtils.newApplicationId(0, 1); node.handle(new RMNodeCleanAppEvent(nodeId, finishedAppId)); - Assert.assertEquals(1, node.getAppsToCleanup().size()); + assertEquals(1, node.getAppsToCleanup().size()); // Verify status update does not clear containers/apps to cleanup // but updating heartbeat response for cleanup does RMNodeStatusEvent statusEvent = getMockRMNodeStatusEvent(null); node.handle(statusEvent); - Assert.assertEquals(1, node.getContainersToCleanUp().size()); - Assert.assertEquals(1, node.getAppsToCleanup().size()); + assertEquals(1, node.getContainersToCleanUp().size()); + assertEquals(1, node.getAppsToCleanup().size()); NodeHeartbeatResponse hbrsp = Records.newRecord(NodeHeartbeatResponse.class); node.setAndUpdateNodeHeartbeatResponse(hbrsp); - Assert.assertEquals(0, node.getContainersToCleanUp().size()); - Assert.assertEquals(0, node.getAppsToCleanup().size()); - Assert.assertEquals(1, hbrsp.getContainersToCleanup().size()); - Assert.assertEquals(completedContainerId, hbrsp.getContainersToCleanup().get(0)); - Assert.assertEquals(1, hbrsp.getApplicationsToCleanup().size()); - Assert.assertEquals(finishedAppId, hbrsp.getApplicationsToCleanup().get(0)); + assertEquals(0, node.getContainersToCleanUp().size()); + assertEquals(0, node.getAppsToCleanup().size()); + assertEquals(1, hbrsp.getContainersToCleanup().size()); + assertEquals(completedContainerId, hbrsp.getContainersToCleanup().get(0)); + assertEquals(1, hbrsp.getApplicationsToCleanup().size()); + assertEquals(finishedAppId, hbrsp.getApplicationsToCleanup().get(0)); } - @Test(timeout=20000) + @Test + @Timeout(value = 20) public void testUpdateHeartbeatResponseForAppLifeCycle() { RMNodeImpl node = getRunningNode(); NodeId nodeId = node.getNodeID(); ApplicationId runningAppId = BuilderUtils.newApplicationId(0, 1); - rmContext.getRMApps().put(runningAppId, Mockito.mock(RMApp.class)); + rmContext.getRMApps().put(runningAppId, mock(RMApp.class)); // Create a running container ContainerId runningContainerId = BuilderUtils.newContainerId( BuilderUtils.newApplicationAttemptId( @@ -928,13 +911,13 @@ public void testUpdateHeartbeatResponseForAppLifeCycle() { nodeHealth, null, null, null); node.handle(new RMNodeStatusEvent(nodeId, nodeStatus, null)); - Assert.assertEquals(1, node.getRunningApps().size()); + assertEquals(1, node.getRunningApps().size()); // Finish an application ApplicationId finishedAppId = runningAppId; node.handle(new RMNodeCleanAppEvent(nodeId, finishedAppId)); - Assert.assertEquals(1, node.getAppsToCleanup().size()); - Assert.assertEquals(0, node.getRunningApps().size()); + assertEquals(1, node.getAppsToCleanup().size()); + assertEquals(0, node.getRunningApps().size()); } @Test @@ -946,9 +929,8 @@ public void testUnknownNodeId() { rmContext.getInactiveRMNodes().putIfAbsent(nodeId,node); node.handle( new RMNodeEvent(node.getNodeID(), RMNodeEventType.DECOMMISSION)); - Assert.assertNull( - "Must be null as there is no NODE_UNUSABLE update", - nodesListManagerEvent); + assertNull(nodesListManagerEvent, + "Must be null as there is no NODE_UNUSABLE update"); } private RMNodeImpl getRunningNode() { @@ -967,7 +949,7 @@ private RMNodeImpl getRunningNode(String nmVersion, int port) { NodeStatus mockNodeStatus = createMockNodeStatus(); node.handle(new RMNodeStartedEvent(node.getNodeID(), null, null, mockNodeStatus)); - Assert.assertEquals(NodeState.RUNNING, node.getState()); + assertEquals(NodeState.RUNNING, node.getState()); return node; } @@ -978,13 +960,12 @@ private RMNodeImpl getDecommissioningNode() { int initialDecommissioning = cm.getNumDecommissioningNMs(); node.handle(new RMNodeEvent(node.getNodeID(), RMNodeEventType.GRACEFUL_DECOMMISSION)); - Assert.assertEquals(NodeState.DECOMMISSIONING, node.getState()); - Assert.assertEquals(Arrays.asList(NodeState.NEW, NodeState.RUNNING), + assertEquals(NodeState.DECOMMISSIONING, node.getState()); + assertEquals(Arrays.asList(NodeState.NEW, NodeState.RUNNING), nodesListManagerEventsNodeStateSequence); - Assert - .assertEquals("Active Nodes", initialActive - 1, cm.getNumActiveNMs()); - Assert.assertEquals("Decommissioning Nodes", initialDecommissioning + 1, - cm.getNumDecommissioningNMs()); + assertEquals(initialActive - 1, cm.getNumActiveNMs(), "Active Nodes"); + assertEquals(initialDecommissioning + 1, + cm.getNumDecommissioningNMs(), "Decommissioning Nodes"); return node; } @@ -995,7 +976,7 @@ private RMNodeImpl getUnhealthyNode() { NodeStatus nodeStatus = NodeStatus.newInstance(node.getNodeID(), 0, new ArrayList(), null, status, null, null, null); node.handle(new RMNodeStatusEvent(node.getNodeID(), nodeStatus, null)); - Assert.assertEquals(NodeState.UNHEALTHY, node.getState()); + assertEquals(NodeState.UNHEALTHY, node.getState()); return node; } @@ -1021,9 +1002,9 @@ private RMNodeImpl getRebootedNode() { node.handle(new RMNodeStartedEvent(node.getNodeID(), null, null, mockNodeStatus)); - Assert.assertEquals(NodeState.RUNNING, node.getState()); + assertEquals(NodeState.RUNNING, node.getState()); node.handle(new RMNodeEvent(node.getNodeID(), RMNodeEventType.REBOOTING)); - Assert.assertEquals(NodeState.REBOOTED, node.getState()); + assertEquals(NodeState.REBOOTED, node.getState()); return node; } @@ -1039,17 +1020,14 @@ public void testAdd() { NodeStatus mockNodeStatus = createMockNodeStatus(); node.handle(new RMNodeStartedEvent(node.getNodeID(), null, null, mockNodeStatus)); - Assert.assertEquals("Active Nodes", initialActive + 1, cm.getNumActiveNMs()); - Assert.assertEquals("Lost Nodes", initialLost, cm.getNumLostNMs()); - Assert.assertEquals("Unhealthy Nodes", - initialUnhealthy, cm.getUnhealthyNMs()); - Assert.assertEquals("Decommissioned Nodes", - initialDecommissioned, cm.getNumDecommisionedNMs()); - Assert.assertEquals("Rebooted Nodes", - initialRebooted, cm.getNumRebootedNMs()); - Assert.assertEquals(NodeState.RUNNING, node.getState()); - Assert.assertNotNull(nodesListManagerEvent); - Assert.assertEquals(NodesListManagerEventType.NODE_USABLE, + assertEquals(initialActive + 1, cm.getNumActiveNMs(), "Active Nodes"); + assertEquals(initialLost, cm.getNumLostNMs(), "Lost Nodes"); + assertEquals(initialUnhealthy, cm.getUnhealthyNMs(), "Unhealthy Nodes"); + assertEquals(initialDecommissioned, cm.getNumDecommisionedNMs(), "Decommissioned Nodes"); + assertEquals(initialRebooted, cm.getNumRebootedNMs(), "Rebooted Nodes"); + assertEquals(NodeState.RUNNING, node.getState()); + assertNotNull(nodesListManagerEvent); + assertEquals(NodesListManagerEventType.NODE_USABLE, nodesListManagerEvent.getType()); } @@ -1063,17 +1041,14 @@ public void testReconnect() { int initialDecommissioned = cm.getNumDecommisionedNMs(); int initialRebooted = cm.getNumRebootedNMs(); node.handle(new RMNodeReconnectEvent(node.getNodeID(), node, null, null)); - Assert.assertEquals("Active Nodes", initialActive, cm.getNumActiveNMs()); - Assert.assertEquals("Lost Nodes", initialLost, cm.getNumLostNMs()); - Assert.assertEquals("Unhealthy Nodes", - initialUnhealthy, cm.getUnhealthyNMs()); - Assert.assertEquals("Decommissioned Nodes", - initialDecommissioned, cm.getNumDecommisionedNMs()); - Assert.assertEquals("Rebooted Nodes", - initialRebooted, cm.getNumRebootedNMs()); - Assert.assertEquals(NodeState.RUNNING, node.getState()); - Assert.assertNotNull(nodesListManagerEvent); - Assert.assertEquals(NodesListManagerEventType.NODE_USABLE, + assertEquals(initialActive, cm.getNumActiveNMs(), "Active Nodes"); + assertEquals(initialLost, cm.getNumLostNMs(), "Lost Nodes"); + assertEquals(initialUnhealthy, cm.getUnhealthyNMs(), "Unhealthy Nodes"); + assertEquals(initialDecommissioned, cm.getNumDecommisionedNMs(), "Decommissioned Nodes"); + assertEquals(initialRebooted, cm.getNumRebootedNMs(), "Rebooted Nodes"); + assertEquals(NodeState.RUNNING, node.getState()); + assertNotNull(nodesListManagerEvent); + assertEquals(NodesListManagerEventType.NODE_USABLE, nodesListManagerEvent.getType()); } @@ -1089,21 +1064,21 @@ public void testReconnectOnDecommissioningNode() { node.handle(new RMNodeReconnectEvent(node.getNodeID(), node, getAppIdList(), null)); // still decommissioning - Assert.assertEquals(NodeState.DECOMMISSIONING, node.getState()); - Assert.assertEquals("Active Nodes", initialActive, cm.getNumActiveNMs()); - Assert.assertEquals("Decommissioning Nodes", initialDecommissioning, - cm.getNumDecommissioningNMs()); - Assert.assertEquals("Decommissioned Nodes", initialDecommissioned, - cm.getNumDecommisionedNMs()); + assertEquals(NodeState.DECOMMISSIONING, node.getState()); + assertEquals(initialActive, cm.getNumActiveNMs(), "Active Nodes"); + assertEquals(initialDecommissioning, + cm.getNumDecommissioningNMs(), "Decommissioning Nodes"); + assertEquals(initialDecommissioned, + cm.getNumDecommisionedNMs(), "Decommissioned Nodes"); // Reconnect event without any running app node.handle(new RMNodeReconnectEvent(node.getNodeID(), node, null, null)); - Assert.assertEquals(NodeState.DECOMMISSIONED, node.getState()); - Assert.assertEquals("Active Nodes", initialActive, cm.getNumActiveNMs()); - Assert.assertEquals("Decommissioning Nodes", initialDecommissioning - 1, - cm.getNumDecommissioningNMs()); - Assert.assertEquals("Decommissioned Nodes", initialDecommissioned + 1, - cm.getNumDecommisionedNMs()); + assertEquals(NodeState.DECOMMISSIONED, node.getState()); + assertEquals(initialActive, cm.getNumActiveNMs(), "Active Nodes"); + assertEquals(initialDecommissioning - 1, + cm.getNumDecommissioningNMs(), "Decommissioning Nodes"); + assertEquals(initialDecommissioned + 1, + cm.getNumDecommisionedNMs(), "Decommissioned Nodes"); } @Test @@ -1115,30 +1090,30 @@ public void testReconnectWithNewPortOnDecommissioningNode() { node.handle(new RMNodeReconnectEvent(node.getNodeID(), node, getAppIdList(), null)); // still decommissioning - Assert.assertEquals(NodeState.DECOMMISSIONING, node.getState()); + assertEquals(NodeState.DECOMMISSIONING, node.getState()); node.setHttpPort(r.nextInt(10000)); // Reconnect event without any running app node.handle(new RMNodeReconnectEvent(node.getNodeID(), node, null, null)); - Assert.assertEquals(NodeState.DECOMMISSIONED, node.getState()); + assertEquals(NodeState.DECOMMISSIONED, node.getState()); } @Test public void testResourceUpdateOnRunningNode() { RMNodeImpl node = getRunningNode(); Resource oldCapacity = node.getTotalCapability(); - assertEquals("Memory resource is not match.", oldCapacity.getMemorySize(), 4096); - assertEquals("CPU resource is not match.", oldCapacity.getVirtualCores(), 4); + assertEquals(oldCapacity.getMemorySize(), 4096, "Memory resource is not match."); + assertEquals(oldCapacity.getVirtualCores(), 4, "CPU resource is not match."); node.handle(new RMNodeResourceUpdateEvent(node.getNodeID(), ResourceOption.newInstance(Resource.newInstance(2048, 2), ResourceOption.OVER_COMMIT_TIMEOUT_MILLIS_DEFAULT))); Resource newCapacity = node.getTotalCapability(); - assertEquals("Memory resource is not match.", newCapacity.getMemorySize(), 2048); - assertEquals("CPU resource is not match.", newCapacity.getVirtualCores(), 2); + assertEquals(newCapacity.getMemorySize(), 2048, "Memory resource is not match."); + assertEquals(newCapacity.getVirtualCores(), 2, "CPU resource is not match."); - Assert.assertEquals(NodeState.RUNNING, node.getState()); - Assert.assertNotNull(nodesListManagerEvent); - Assert.assertEquals(NodesListManagerEventType.NODE_USABLE, + assertEquals(NodeState.RUNNING, node.getState()); + assertNotNull(nodesListManagerEvent); + assertEquals(NodesListManagerEventType.NODE_USABLE, nodesListManagerEvent.getType()); } @@ -1151,16 +1126,16 @@ public void testDecommissioningOnRunningNode(){ public void testResourceUpdateOnNewNode() { RMNodeImpl node = getNewNode(Resource.newInstance(4096, 4)); Resource oldCapacity = node.getTotalCapability(); - assertEquals("Memory resource is not match.", oldCapacity.getMemorySize(), 4096); - assertEquals("CPU resource is not match.", oldCapacity.getVirtualCores(), 4); + assertEquals(oldCapacity.getMemorySize(), 4096, "Memory resource is not match."); + assertEquals(oldCapacity.getVirtualCores(), 4, "CPU resource is not match."); node.handle(new RMNodeResourceUpdateEvent(node.getNodeID(), ResourceOption.newInstance(Resource.newInstance(2048, 2), ResourceOption.OVER_COMMIT_TIMEOUT_MILLIS_DEFAULT))); Resource newCapacity = node.getTotalCapability(); - assertEquals("Memory resource is not match.", newCapacity.getMemorySize(), 2048); - assertEquals("CPU resource is not match.", newCapacity.getVirtualCores(), 2); + assertEquals(newCapacity.getMemorySize(), 2048, "Memory resource is not match."); + assertEquals(newCapacity.getVirtualCores(), 2, "CPU resource is not match."); - Assert.assertEquals(NodeState.NEW, node.getState()); + assertEquals(NodeState.NEW, node.getState()); } @Test @@ -1171,21 +1146,21 @@ public void testResourceUpdateOnRebootedNode() { int initialUnHealthy = cm.getUnhealthyNMs(); int initialDecommissioning = cm.getNumDecommissioningNMs(); Resource oldCapacity = node.getTotalCapability(); - assertEquals("Memory resource is not match.", oldCapacity.getMemorySize(), 4096); - assertEquals("CPU resource is not match.", oldCapacity.getVirtualCores(), 4); + assertEquals(oldCapacity.getMemorySize(), 4096, "Memory resource is not match."); + assertEquals(oldCapacity.getVirtualCores(), 4, "CPU resource is not match."); node.handle(new RMNodeResourceUpdateEvent(node.getNodeID(), ResourceOption .newInstance(Resource.newInstance(2048, 2), ResourceOption.OVER_COMMIT_TIMEOUT_MILLIS_DEFAULT))); Resource newCapacity = node.getTotalCapability(); - assertEquals("Memory resource is not match.", newCapacity.getMemorySize(), 2048); - assertEquals("CPU resource is not match.", newCapacity.getVirtualCores(), 2); - - Assert.assertEquals(NodeState.REBOOTED, node.getState()); - Assert.assertEquals("Active Nodes", initialActive, cm.getNumActiveNMs()); - Assert.assertEquals("Unhealthy Nodes", initialUnHealthy, - cm.getUnhealthyNMs()); - Assert.assertEquals("Decommissioning Nodes", initialDecommissioning, - cm.getNumDecommissioningNMs()); + assertEquals(newCapacity.getMemorySize(), 2048, "Memory resource is not match."); + assertEquals(newCapacity.getVirtualCores(), 2, "CPU resource is not match."); + + assertEquals(NodeState.REBOOTED, node.getState()); + assertEquals(initialActive, cm.getNumActiveNMs(), "Active Nodes"); + assertEquals(initialUnHealthy, + cm.getUnhealthyNMs(), "Unhealthy Nodes"); + assertEquals(initialDecommissioning, + cm.getNumDecommissioningNMs(), "Decommissioning Nodes"); } // Test unhealthy report on a decommissioning node will make it @@ -1201,10 +1176,10 @@ public void testDecommissioningUnhealthy() { NodeStatus nodeStatus = NodeStatus.newInstance(node.getNodeID(), 0, null, keepAliveApps, status, null, null, null); node.handle(new RMNodeStatusEvent(node.getNodeID(), nodeStatus, null)); - Assert.assertEquals(NodeState.DECOMMISSIONING, node.getState()); + assertEquals(NodeState.DECOMMISSIONING, node.getState()); nodeStatus.setKeepAliveApplications(null); node.handle(new RMNodeStatusEvent(node.getNodeID(), nodeStatus, null)); - Assert.assertEquals(NodeState.DECOMMISSIONED, node.getState()); + assertEquals(NodeState.DECOMMISSIONED, node.getState()); } @Test @@ -1212,11 +1187,11 @@ public void testReconnnectUpdate() { final String nmVersion1 = "nm version 1"; final String nmVersion2 = "nm version 2"; RMNodeImpl node = getRunningNode(nmVersion1); - Assert.assertEquals(nmVersion1, node.getNodeManagerVersion()); + assertEquals(nmVersion1, node.getNodeManagerVersion()); RMNodeImpl reconnectingNode = getRunningNode(nmVersion2); node.handle(new RMNodeReconnectEvent(node.getNodeID(), reconnectingNode, null, null)); - Assert.assertEquals(nmVersion2, node.getNodeManagerVersion()); + assertEquals(nmVersion2, node.getNodeManagerVersion()); } @Test @@ -1227,7 +1202,7 @@ public void testContainerExpire() throws Exception { ApplicationId.newInstance(System.currentTimeMillis(), 1); ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1); - rmContext.getRMApps().put(appId, Mockito.mock(RMApp.class)); + rmContext.getRMApps().put(appId, mock(RMApp.class)); ContainerId containerId1 = ContainerId.newContainerId(appAttemptId, 1L); ContainerId containerId2 = ContainerId.newContainerId(appAttemptId, 2L); AllocationExpirationInfo expirationInfo1 = @@ -1259,21 +1234,23 @@ public void testContainerExpire() throws Exception { public void testResourceUpdateOnDecommissioningNode() { RMNodeImpl node = getDecommissioningNode(); Resource oldCapacity = node.getTotalCapability(); - assertEquals("Memory resource is not match.", oldCapacity.getMemorySize(), 4096); - assertEquals("CPU resource is not match.", oldCapacity.getVirtualCores(), 4); + assertEquals(oldCapacity.getMemorySize(), 4096, "Memory resource is not match."); + assertEquals(oldCapacity.getVirtualCores(), 4, "CPU resource is not match."); node.handle(new RMNodeResourceUpdateEvent(node.getNodeID(), ResourceOption.newInstance(Resource.newInstance(2048, 2), ResourceOption.OVER_COMMIT_TIMEOUT_MILLIS_DEFAULT))); Resource originalCapacity = node.getOriginalTotalCapability(); - assertEquals("Memory resource is not match.", originalCapacity.getMemorySize(), oldCapacity.getMemorySize()); - assertEquals("CPU resource is not match.", originalCapacity.getVirtualCores(), oldCapacity.getVirtualCores()); + assertEquals(originalCapacity.getMemorySize(), oldCapacity.getMemorySize(), + "Memory resource is not match."); + assertEquals(originalCapacity.getVirtualCores(), oldCapacity.getVirtualCores(), + "CPU resource is not match."); Resource newCapacity = node.getTotalCapability(); - assertEquals("Memory resource is not match.", newCapacity.getMemorySize(), 2048); - assertEquals("CPU resource is not match.", newCapacity.getVirtualCores(), 2); + assertEquals(newCapacity.getMemorySize(), 2048, "Memory resource is not match."); + assertEquals(newCapacity.getVirtualCores(), 2, "CPU resource is not match."); - Assert.assertEquals(NodeState.DECOMMISSIONING, node.getState()); - Assert.assertNotNull(nodesListManagerEvent); - Assert.assertEquals(NodesListManagerEventType.NODE_DECOMMISSIONING, + assertEquals(NodeState.DECOMMISSIONING, node.getState()); + assertNotNull(nodesListManagerEvent); + assertEquals(NodesListManagerEventType.NODE_DECOMMISSIONING, nodesListManagerEvent.getType()); } @@ -1281,15 +1258,14 @@ public void testResourceUpdateOnDecommissioningNode() { public void testResourceUpdateOnRecommissioningNode() { RMNodeImpl node = getDecommissioningNode(); Resource oldCapacity = node.getTotalCapability(); - assertEquals("Memory resource is not match.", oldCapacity.getMemorySize(), 4096); - assertEquals("CPU resource is not match.", oldCapacity.getVirtualCores(), 4); - assertFalse("updatedCapability should be false.", - node.isUpdatedCapability()); + assertEquals(oldCapacity.getMemorySize(), 4096, "Memory resource is not match."); + assertEquals(oldCapacity.getVirtualCores(), 4, "CPU resource is not match."); + assertFalse(node.isUpdatedCapability(), "updatedCapability should be false."); node.handle(new RMNodeEvent(node.getNodeID(), RMNodeEventType.RECOMMISSION)); Resource originalCapacity = node.getOriginalTotalCapability(); - assertEquals("Original total capability not null after recommission", null, originalCapacity); - assertTrue("updatedCapability should be set.", node.isUpdatedCapability()); + assertEquals(null, originalCapacity, "Original total capability not null after recommission"); + assertTrue(node.isUpdatedCapability(), "updatedCapability should be set."); } @Test @@ -1308,41 +1284,37 @@ public void testDisappearingContainer() { ContainerState.RUNNING, "", -1)); node = getRunningNode(); node.handle(getMockRMNodeStatusEvent(containerStats)); - assertEquals("unexpected number of running containers", - 2, node.getLaunchedContainers().size()); - Assert.assertTrue("first container not running", - node.getLaunchedContainers().contains(cid1)); - Assert.assertTrue("second container not running", - node.getLaunchedContainers().contains(cid2)); - assertEquals("unexpected number of running containers", - 2, node.getUpdatedExistContainers().size()); - Assert.assertTrue("first container not running", - node.getUpdatedExistContainers().containsKey(cid1)); - Assert.assertTrue("second container not running", - node.getUpdatedExistContainers().containsKey(cid2)); - assertEquals("already completed containers", - 0, completedContainers.size()); + assertEquals(2, node.getLaunchedContainers().size(), + "unexpected number of running containers"); + assertTrue(node.getLaunchedContainers().contains(cid1), "first container not running"); + assertTrue(node.getLaunchedContainers().contains(cid2), "second container not running"); + assertEquals(2, node.getUpdatedExistContainers().size(), + "unexpected number of running containers"); + assertTrue(node.getUpdatedExistContainers().containsKey(cid1), + "first container not running"); + assertTrue(node.getUpdatedExistContainers().containsKey(cid2), + "second container not running"); + assertEquals(0, completedContainers.size(), "already completed containers"); containerStats.remove(0); node.handle(getMockRMNodeStatusEvent(containerStats)); - assertEquals("expected one container to be completed", - 1, completedContainers.size()); + assertEquals(1, completedContainers.size(), + "expected one container to be completed"); ContainerStatus cs = completedContainers.get(0); - assertEquals("first container not the one that completed", - cid1, cs.getContainerId()); - assertEquals("completed container not marked complete", - ContainerState.COMPLETE, cs.getState()); - assertEquals("completed container not marked aborted", - ContainerExitStatus.ABORTED, cs.getExitStatus()); - Assert.assertTrue("completed container not marked missing", - cs.getDiagnostics().contains("not reported")); - assertEquals("unexpected number of running containers", - 1, node.getLaunchedContainers().size()); - Assert.assertTrue("second container not running", - node.getLaunchedContainers().contains(cid2)); - assertEquals("unexpected number of running containers", - 1, node.getUpdatedExistContainers().size()); - Assert.assertTrue("second container not running", - node.getUpdatedExistContainers().containsKey(cid2)); + assertEquals(cid1, cs.getContainerId(), + "first container not the one that completed"); + assertEquals(ContainerState.COMPLETE, cs.getState(), + "completed container not marked complete"); + assertEquals(ContainerExitStatus.ABORTED, cs.getExitStatus(), + "completed container not marked aborted"); + assertTrue(cs.getDiagnostics().contains("not reported"), + "completed container not marked missing"); + assertEquals(1, node.getLaunchedContainers().size(), + "unexpected number of running containers"); + assertTrue(node.getLaunchedContainers().contains(cid2), "second container not running"); + assertEquals(1, node.getUpdatedExistContainers().size(), + "unexpected number of running containers"); + assertTrue(node.getUpdatedExistContainers().containsKey(cid2), + "second container not running"); } @Test @@ -1367,12 +1339,12 @@ public void testForHandlingDuplicatedCompltedContainers() { verify(scheduler, times(1)).handle(any(NodeAddedSchedulerEvent.class)); node.handle(statusEvent1); verify(scheduler, times(1)).handle(any(NodeAddedSchedulerEvent.class)); - Assert.assertEquals(1, node.getQueueSize()); - Assert.assertEquals(1, node.getCompletedContainers().size()); + assertEquals(1, node.getQueueSize()); + assertEquals(1, node.getCompletedContainers().size()); // test for duplicate entries node.handle(statusEvent1); - Assert.assertEquals(1, node.getQueueSize()); + assertEquals(1, node.getQueueSize()); // send clean up container event node.handle(new RMNodeFinishedContainersPulledByAMEvent(node.getNodeID(), @@ -1382,8 +1354,8 @@ public void testForHandlingDuplicatedCompltedContainers() { Records.newRecord(NodeHeartbeatResponse.class); node.setAndUpdateNodeHeartbeatResponse(hbrsp); - Assert.assertEquals(1, hbrsp.getContainersToBeRemovedFromNM().size()); - Assert.assertEquals(0, node.getCompletedContainers().size()); + assertEquals(1, hbrsp.getContainersToBeRemovedFromNM().size()); + assertEquals(0, node.getCompletedContainers().size()); } @Test @@ -1393,7 +1365,7 @@ public void testFinishedContainersPulledByAMOnNewNode() { rmNode.handle(new RMNodeFinishedContainersPulledByAMEvent(nodeId, getContainerIdList())); - Assert.assertEquals(1, rmNode.getContainersToBeRemovedFromNM().size()); + assertEquals(1, rmNode.getContainersToBeRemovedFromNM().size()); } @@ -1404,7 +1376,7 @@ private void calcIntervalTest(RMNodeImpl rmNode, ResourceUtilization nodeUtil, rmNode.setNodeUtilization(nodeUtil); long hbInterval = rmNode.calculateHeartBeatInterval(hbDefault, hbMin, hbMax, speedup, slowdown); - assertEquals("heartbeat interval incorrect", expectedHb, hbInterval); + assertEquals(expectedHb, hbInterval, "heartbeat interval incorrect"); } @Test @@ -1510,7 +1482,7 @@ public void testFinishedContainersPulledByAmOnDecommissioningNode() { RMNodeImpl rMNodeImpl = getRunningNode(); rMNodeImpl.handle( new RMNodeEvent(rMNodeImpl.getNodeID(), RMNodeEventType.GRACEFUL_DECOMMISSION)); - Assert.assertEquals(NodeState.DECOMMISSIONING, rMNodeImpl.getState()); + assertEquals(NodeState.DECOMMISSIONING, rMNodeImpl.getState()); ContainerId containerId = BuilderUtils.newContainerId( BuilderUtils.newApplicationAttemptId(BuilderUtils.newApplicationId(0, 0), 0), 0); @@ -1518,13 +1490,13 @@ public void testFinishedContainersPulledByAmOnDecommissioningNode() { rMNodeImpl.handle( new RMNodeFinishedContainersPulledByAMEvent(rMNodeImpl.getNodeID(), containerIds)); - Assert.assertEquals(NodeState.DECOMMISSIONING, rMNodeImpl.getState()); + assertEquals(NodeState.DECOMMISSIONING, rMNodeImpl.getState()); // Verify expected containersToBeRemovedFromNM from NodeHeartbeatResponse. NodeHeartbeatResponse response = YarnServerBuilderUtils.newNodeHeartbeatResponse(1, NodeAction.NORMAL, null, null, null, null, 1000); rMNodeImpl.setAndUpdateNodeHeartbeatResponse(response); - Assert.assertEquals(1, response.getContainersToBeRemovedFromNM().size()); + assertEquals(1, response.getContainersToBeRemovedFromNM().size()); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMProxyUsersConf.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMProxyUsersConf.java index 2b67a31d59120..43af49e917603 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMProxyUsersConf.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMProxyUsersConf.java @@ -18,6 +18,8 @@ package org.apache.hadoop.yarn.server.resourcemanager; +import static org.junit.jupiter.api.Assertions.fail; + import java.util.Arrays; import java.util.Collection; @@ -27,12 +29,9 @@ import org.apache.hadoop.security.authorize.ProxyUsers; import org.apache.hadoop.yarn.conf.YarnConfiguration; import static org.apache.hadoop.yarn.conf.YarnConfiguration.RM_PROXY_USER_PREFIX; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; -@RunWith(Parameterized.class) public class TestRMProxyUsersConf { private static final UserGroupInformation FOO_USER = @@ -41,14 +40,13 @@ public class TestRMProxyUsersConf { UserGroupInformation.createUserForTesting("bar", new String[] { "bar_group" }); private final String ipAddress = "127.0.0.1"; - @Parameterized.Parameters public static Collection headers() { - return Arrays.asList(new Object[][] { { 0 }, { 1 }, { 2 } }); + return Arrays.asList(new Object[][]{{0}, {1}, {2}}); } private Configuration conf; - public TestRMProxyUsersConf(int round) { + public void initTestRMProxyUsersConf(int round) { conf = new YarnConfiguration(); switch (round) { case 0: @@ -78,8 +76,10 @@ public TestRMProxyUsersConf(int round) { } } - @Test - public void testProxyUserConfiguration() throws Exception { + @ParameterizedTest + @MethodSource("headers") + public void testProxyUserConfiguration(int round) throws Exception { + initTestRMProxyUsersConf(round); MockRM rm = null; try { rm = new MockRM(conf); @@ -94,7 +94,7 @@ public void testProxyUserConfiguration() throws Exception { ipAddress); } catch (AuthorizationException e) { // Exception is not expected - Assert.fail(); + fail(); } } finally { if (rm != null) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMServerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMServerUtils.java index 95cc98339296e..0d7003d1e1cbf 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMServerUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMServerUtils.java @@ -19,6 +19,8 @@ package org.apache.hadoop.yarn.server.resourcemanager; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.apache.hadoop.yarn.api.records.ContainerUpdateType.INCREASE_RESOURCE; import static org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils.RESOURCE_OUTSIDE_ALLOWED_RANGE; @@ -53,8 +55,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -112,22 +113,22 @@ public void testValidateAndSplitUpdateResourceRequests() { ContainerUpdates containerUpdates = RMServerUtils.validateAndSplitUpdateResourceRequests(rmContext, allocateRequest, maxAllocation, updateErrors); - Assert.assertEquals(1, updateErrors.size()); - Assert.assertEquals(resource + 1, updateErrors.get(0) + assertEquals(1, updateErrors.size()); + assertEquals(resource + 1, updateErrors.get(0) .getUpdateContainerRequest().getCapability().getMemorySize()); - Assert.assertEquals(resource + 1, updateErrors.get(0) + assertEquals(resource + 1, updateErrors.get(0) .getUpdateContainerRequest().getCapability().getVirtualCores()); - Assert.assertEquals(RESOURCE_OUTSIDE_ALLOWED_RANGE, + assertEquals(RESOURCE_OUTSIDE_ALLOWED_RANGE, updateErrors.get(0).getReason()); - Assert.assertEquals(1, containerUpdates.getIncreaseRequests().size()); + assertEquals(1, containerUpdates.getIncreaseRequests().size()); UpdateContainerRequest increaseRequest = containerUpdates.getIncreaseRequests().get(0); - Assert.assertEquals(capabilityOk.getVirtualCores(), + assertEquals(capabilityOk.getVirtualCores(), increaseRequest.getCapability().getVirtualCores()); - Assert.assertEquals(capabilityOk.getMemorySize(), + assertEquals(capabilityOk.getMemorySize(), increaseRequest.getCapability().getMemorySize()); - Assert.assertEquals(containerIdOk, increaseRequest.getContainerId()); + assertEquals(containerIdOk, increaseRequest.getContainerId()); } @Test @@ -142,22 +143,22 @@ public void testQueryRMNodes() throws Exception { when(rmContext.getInactiveRMNodes()).thenReturn(inactiveList); List result = RMServerUtils.queryRMNodes(rmContext, EnumSet.of(NodeState.SHUTDOWN)); - Assert.assertTrue(result.size() != 0); + assertTrue(result.size() != 0); assertThat(result.get(0)).isEqualTo(rmNode1); when(rmNode1.getState()).thenReturn(NodeState.DECOMMISSIONED); result = RMServerUtils.queryRMNodes(rmContext, EnumSet.of(NodeState.DECOMMISSIONED)); - Assert.assertTrue(result.size() != 0); + assertTrue(result.size() != 0); assertThat(result.get(0)).isEqualTo(rmNode1); when(rmNode1.getState()).thenReturn(NodeState.LOST); result = RMServerUtils.queryRMNodes(rmContext, EnumSet.of(NodeState.LOST)); - Assert.assertTrue(result.size() != 0); + assertTrue(result.size() != 0); assertThat(result.get(0)).isEqualTo(rmNode1); when(rmNode1.getState()).thenReturn(NodeState.REBOOTED); result = RMServerUtils.queryRMNodes(rmContext, EnumSet.of(NodeState.REBOOTED)); - Assert.assertTrue(result.size() != 0); + assertTrue(result.size() != 0); assertThat(result.get(0)).isEqualTo(rmNode1); } @@ -187,46 +188,46 @@ public void testGetApplicableNodeCountForAMLocality() throws Exception { true, null); List reqs = new ArrayList<>(); reqs.add(anyReq); - Assert.assertEquals(100, + assertEquals(100, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); ResourceRequest rackReq = createResourceRequest("/rack1", true, null); reqs.add(rackReq); - Assert.assertEquals(30, + assertEquals(30, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); anyReq.setRelaxLocality(false); - Assert.assertEquals(30, + assertEquals(30, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); rackReq.setRelaxLocality(false); - Assert.assertEquals(100, + assertEquals(100, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); ResourceRequest node1Req = createResourceRequest("node1", false, null); reqs.add(node1Req); - Assert.assertEquals(100, + assertEquals(100, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); node1Req.setRelaxLocality(true); - Assert.assertEquals(1, + assertEquals(1, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); rackReq.setRelaxLocality(true); - Assert.assertEquals(31, + assertEquals(31, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); ResourceRequest node2Req = createResourceRequest("node2", false, null); reqs.add(node2Req); - Assert.assertEquals(31, + assertEquals(31, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); node2Req.setRelaxLocality(true); - Assert.assertEquals(31, + assertEquals(31, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); rackReq.setRelaxLocality(false); - Assert.assertEquals(2, + assertEquals(2, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); node1Req.setRelaxLocality(false); - Assert.assertEquals(1, + assertEquals(1, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); node2Req.setRelaxLocality(false); - Assert.assertEquals(100, + assertEquals(100, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); } @@ -261,10 +262,10 @@ public void testGetApplicableNodeCountForAMLabels() throws Exception { true, null); List reqs = new ArrayList<>(); reqs.add(anyReq); - Assert.assertEquals(80, + assertEquals(80, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); anyReq.setNodeLabelExpression("label1"); - Assert.assertEquals(10, + assertEquals(10, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); } @@ -320,46 +321,46 @@ public void testGetApplicableNodeCountForAMLocalityAndLabels() true, null); List reqs = new ArrayList<>(); reqs.add(anyReq); - Assert.assertEquals(80, + assertEquals(80, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); ResourceRequest rackReq = createResourceRequest("/rack1", true, null); reqs.add(rackReq); - Assert.assertEquals(20, + assertEquals(20, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); anyReq.setRelaxLocality(false); - Assert.assertEquals(20, + assertEquals(20, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); rackReq.setRelaxLocality(false); - Assert.assertEquals(80, + assertEquals(80, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); ResourceRequest node1Req = createResourceRequest("node1", false, null); reqs.add(node1Req); - Assert.assertEquals(80, + assertEquals(80, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); node1Req.setRelaxLocality(true); - Assert.assertEquals(0, + assertEquals(0, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); rackReq.setRelaxLocality(true); - Assert.assertEquals(20, + assertEquals(20, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); ResourceRequest node2Req = createResourceRequest("node2", false, null); reqs.add(node2Req); - Assert.assertEquals(20, + assertEquals(20, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); node2Req.setRelaxLocality(true); - Assert.assertEquals(20, + assertEquals(20, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); rackReq.setRelaxLocality(false); - Assert.assertEquals(1, + assertEquals(1, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); node1Req.setRelaxLocality(false); - Assert.assertEquals(1, + assertEquals(1, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); node2Req.setRelaxLocality(false); - Assert.assertEquals(80, + assertEquals(80, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); anyReq.setNodeLabelExpression("label1"); @@ -369,65 +370,65 @@ public void testGetApplicableNodeCountForAMLocalityAndLabels() anyReq.setRelaxLocality(true); reqs = new ArrayList<>(); reqs.add(anyReq); - Assert.assertEquals(15, + assertEquals(15, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); rackReq.setRelaxLocality(true); reqs.add(rackReq); - Assert.assertEquals(10, + assertEquals(10, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); anyReq.setRelaxLocality(false); - Assert.assertEquals(10, + assertEquals(10, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); rackReq.setRelaxLocality(false); - Assert.assertEquals(15, + assertEquals(15, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); node1Req.setRelaxLocality(false); reqs.add(node1Req); - Assert.assertEquals(15, + assertEquals(15, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); node1Req.setRelaxLocality(true); - Assert.assertEquals(1, + assertEquals(1, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); rackReq.setRelaxLocality(true); - Assert.assertEquals(11, + assertEquals(11, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); node2Req.setRelaxLocality(false); reqs.add(node2Req); - Assert.assertEquals(11, + assertEquals(11, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); node2Req.setRelaxLocality(true); - Assert.assertEquals(11, + assertEquals(11, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); rackReq.setRelaxLocality(false); - Assert.assertEquals(1, + assertEquals(1, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); node1Req.setRelaxLocality(false); - Assert.assertEquals(0, + assertEquals(0, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); node2Req.setRelaxLocality(false); - Assert.assertEquals(15, + assertEquals(15, RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs)); } @Test public void testConvertRmAppAttemptStateToYarnApplicationAttemptState() { - Assert.assertEquals( + assertEquals( YarnApplicationAttemptState.FAILED, RMServerUtils.convertRmAppAttemptStateToYarnApplicationAttemptState( RMAppAttemptState.FINAL_SAVING, RMAppAttemptState.FAILED ) ); - Assert.assertEquals( + assertEquals( YarnApplicationAttemptState.SCHEDULED, RMServerUtils.convertRmAppAttemptStateToYarnApplicationAttemptState( RMAppAttemptState.FINAL_SAVING, RMAppAttemptState.SCHEDULED ) ); - Assert.assertEquals( + assertEquals( YarnApplicationAttemptState.NEW, RMServerUtils.convertRmAppAttemptStateToYarnApplicationAttemptState( RMAppAttemptState.NEW, diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMStoreCommands.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMStoreCommands.java index e2c78345d533b..c0873036af47c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMStoreCommands.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMStoreCommands.java @@ -18,11 +18,11 @@ package org.apache.hadoop.yarn.server.resourcemanager; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.test.TestingServer; @@ -34,7 +34,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore; import org.apache.hadoop.yarn.server.resourcemanager.recovery.TestZKRMStateStore; import org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestRMStoreCommands { @@ -52,9 +52,10 @@ public void testFormatStateStoreCmdForZK() throws Exception { rm.start(); rm.getRMContext().getRMAdminService().transitionToActive(req); String zkStateRoot = ZKRMStateStore.ROOT_ZNODE_NAME; - assertEquals("RM State store parent path should have a child node " + - zkStateRoot, zkStateRoot, curatorFramework.getChildren().forPath( - YarnConfiguration.DEFAULT_ZK_RM_STATE_STORE_PARENT_PATH).get(0)); + assertEquals(zkStateRoot, curatorFramework.getChildren().forPath( + YarnConfiguration.DEFAULT_ZK_RM_STATE_STORE_PARENT_PATH).get(0), + "RM State store parent path should have a child node " + + zkStateRoot); rm.close(); try { ResourceManager.deleteRMStateStore(conf); @@ -62,9 +63,9 @@ public void testFormatStateStoreCmdForZK() throws Exception { fail("Exception should not be thrown during format rm state store" + " operation."); } - assertTrue("After store format parent path should have no child nodes", - curatorFramework.getChildren().forPath( - YarnConfiguration.DEFAULT_ZK_RM_STATE_STORE_PARENT_PATH).isEmpty()); + assertTrue(curatorFramework.getChildren().forPath( + YarnConfiguration.DEFAULT_ZK_RM_STATE_STORE_PARENT_PATH).isEmpty(), + "After store format parent path should have no child nodes"); } } @@ -86,8 +87,8 @@ public void testFormatConfStoreCmdForZK() throws Exception { YarnConfiguration.RM_SCHEDCONF_STORE_ZK_PARENT_PATH, YarnConfiguration.DEFAULT_RM_SCHEDCONF_STORE_ZK_PARENT_PATH) + "/CONF_STORE"; - assertNotNull("Failed to initialize ZKConfigurationStore", - curatorFramework.checkExists().forPath(confStorePath)); + assertNotNull(curatorFramework.checkExists().forPath(confStorePath), + "Failed to initialize ZKConfigurationStore"); rm.close(); try { @@ -96,8 +97,8 @@ public void testFormatConfStoreCmdForZK() throws Exception { fail("Exception should not be thrown during format rm conf store" + " operation."); } - assertNull("Failed to format ZKConfigurationStore", - curatorFramework.checkExists().forPath(confStorePath)); + assertNull(curatorFramework.checkExists().forPath(confStorePath), + "Failed to format ZKConfigurationStore"); } } @@ -126,8 +127,8 @@ public void testRemoveApplicationFromStateStoreCmdForZK() throws Exception { if (path.equals(ZKRMStateStore.RM_APP_ROOT_HIERARCHIES)) { continue; } - assertEquals("Application node for " + appId + " should exist", - appId, path); + assertEquals(appId, path, + "Application node for " + appId + " should exist"); } try { ResourceManager.removeApplication(conf, appId); @@ -135,11 +136,12 @@ public void testRemoveApplicationFromStateStoreCmdForZK() throws Exception { fail("Exception should not be thrown while removing app from " + "rm state store."); } - assertTrue("After remove app from store there should be no child nodes" + - " for application in app root path", + assertTrue( curatorFramework.getChildren().forPath(appRootPath).size() == 1 && curatorFramework.getChildren().forPath(appRootPath).get(0).equals( - ZKRMStateStore.RM_APP_ROOT_HIERARCHIES)); + ZKRMStateStore.RM_APP_ROOT_HIERARCHIES), + "After remove app from store there should be no child nodes" + + " for application in app root path"); } } } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMTimelineService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMTimelineService.java index 795979cd1cea1..2936057651210 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMTimelineService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMTimelineService.java @@ -18,6 +18,9 @@ package org.apache.hadoop.yarn.server.resourcemanager; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; + import java.util.List; import org.apache.hadoop.conf.Configuration; @@ -28,8 +31,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore; import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl; import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; /** * Tests that the RM creates timeline services (v1/v2) as specified by the @@ -41,7 +43,7 @@ public class TestRMTimelineService { private void setup(boolean v1Enabled, boolean v2Enabled, boolean systemMetricEnabled) { Configuration conf = new YarnConfiguration(new Configuration(false)); - Assert.assertFalse(YarnConfiguration.timelineServiceEnabled(conf)); + assertFalse(YarnConfiguration.timelineServiceEnabled(conf)); conf.setBoolean(YarnConfiguration.SYSTEM_METRICS_PUBLISHER_ENABLED, systemMetricEnabled); @@ -87,11 +89,11 @@ private void validate(boolean v1Enabled, boolean v2Enabled, } if(systemMetricEnabled) { - Assert.assertEquals(v1Enabled, v1PublisherServiceFound); - Assert.assertEquals(v2Enabled, v2PublisherServiceFound); + assertEquals(v1Enabled, v1PublisherServiceFound); + assertEquals(v2Enabled, v2PublisherServiceFound); } else { - Assert.assertEquals(false, v1PublisherServiceFound); - Assert.assertEquals(false, v2PublisherServiceFound); + assertEquals(false, v1PublisherServiceFound); + assertEquals(false, v2PublisherServiceFound); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java index b9c5500a7d20b..4d942adda6fa3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java @@ -19,8 +19,11 @@ package org.apache.hadoop.yarn.server.resourcemanager; import static org.apache.hadoop.yarn.server.resourcemanager.MockNM.createMockNodeStatus; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.fail; import java.io.IOException; import java.util.Collection; @@ -51,12 +54,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.converter.FSConfigConverterTestCommons; import org.apache.hadoop.yarn.server.security.http.RMAuthenticationFilterInitializer; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; +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; @@ -66,11 +67,9 @@ public class TestResourceManager { private ResourceManager resourceManager = null; - @Rule - public ExpectedException thrown = ExpectedException.none(); private FSConfigConverterTestCommons converterTestCommons; - @Before + @BeforeEach public void setUp() throws Exception { YarnConfiguration conf = new YarnConfiguration(); UserGroupInformation.setConfiguration(conf); @@ -84,7 +83,7 @@ public void setUp() throws Exception { converterTestCommons.setUp(); } - @After + @AfterEach public void tearDown() throws Exception { resourceManager.stop(); converterTestCommons.tearDown(); @@ -245,7 +244,8 @@ private void checkResourceUsage( } } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testResourceManagerInitConfigValidation() throws Exception { Configuration conf = new YarnConfiguration(); conf.setInt(YarnConfiguration.GLOBAL_RM_AM_MAX_ATTEMPTS, -1); @@ -287,7 +287,8 @@ public void testNMExpiryAndHeartbeatIntervalsValidation() throws Exception { } } - @Test(timeout = 50000) + @Test + @Timeout(value = 50) public void testFilterOverrides() throws Exception { String filterInitializerConfKey = "hadoop.http.filter.initializers"; String[] filterInitializers = @@ -326,11 +327,11 @@ protected void doSecureLogin() throws IOException { // just want to test filter settings String tmp = resourceManager.getConfig().get(filterInitializerConfKey); if (filterInitializer.contains(this.getClass().getName())) { - Assert.assertEquals(RMAuthenticationFilterInitializer.class.getName() + assertEquals(RMAuthenticationFilterInitializer.class.getName() + "," + this.getClass().getName(), tmp); } else { - Assert.assertEquals( - RMAuthenticationFilterInitializer.class.getName(), tmp); + assertEquals( + RMAuthenticationFilterInitializer.class.getName(), tmp); } resourceManager.stop(); } @@ -352,10 +353,10 @@ protected void doSecureLogin() throws IOException { // just want to test filter settings String tmp = resourceManager.getConfig().get(filterInitializerConfKey); if (filterInitializer.equals(StaticUserWebFilter.class.getName())) { - Assert.assertEquals(RMAuthenticationFilterInitializer.class.getName() + assertEquals(RMAuthenticationFilterInitializer.class.getName() + "," + StaticUserWebFilter.class.getName(), tmp); } else { - Assert.assertEquals( + assertEquals( RMAuthenticationFilterInitializer.class.getName(), tmp); } resourceManager.stop(); @@ -370,18 +371,21 @@ protected void doSecureLogin() throws IOException { */ @Test public void testUserProvidedUGIConf() throws Exception { - thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("Invalid attribute value for " + + IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, () -> { + Configuration dummyConf = new YarnConfiguration(); + dummyConf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, + "DUMMYAUTH"); + ResourceManager dummyResourceManager = new ResourceManager(); + try { + dummyResourceManager.init(dummyConf); + } finally { + dummyResourceManager.stop(); + } + }); + + assertThat(exception.getMessage()).contains("Invalid attribute value for " + CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION + " of DUMMYAUTH"); - Configuration dummyConf = new YarnConfiguration(); - dummyConf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, - "DUMMYAUTH"); - ResourceManager dummyResourceManager = new ResourceManager(); - try { - dummyResourceManager.init(dummyConf); - } finally { - dummyResourceManager.stop(); - } } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManagerMXBean.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManagerMXBean.java index 0e70b5f1c3897..435e6522d4312 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManagerMXBean.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManagerMXBean.java @@ -17,13 +17,14 @@ */ package org.apache.hadoop.yarn.server.resourcemanager; +import static org.junit.jupiter.api.Assertions.assertEquals; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import javax.management.MBeanServer; import javax.management.ObjectName; @@ -50,7 +51,7 @@ public void testResourceManagerMXBean() throws Exception { // Get attribute "SecurityEnabled" boolean securityEnabled = (boolean) mbs.getAttribute(mxbeanName, "SecurityEnabled"); - Assert.assertEquals(resourceManager.isSecurityEnabled(), securityEnabled); + assertEquals(resourceManager.isSecurityEnabled(), securityEnabled); } } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java index bd7548081fde4..69cf306150e06 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java @@ -36,7 +36,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.FileSystemNodeAttributeStore; import static org.apache.hadoop.yarn.server.resourcemanager.MockNM.createMockNodeStatus; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; @@ -148,9 +154,9 @@ import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.YarnVersionInfo; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -174,7 +180,8 @@ public class TestResourceTrackerService extends NodeLabelTestBase { * Test RM read NM next heartBeat Interval correctly from Configuration file, * and NM get next heartBeat Interval from RM correctly */ - @Test (timeout = 50000) + @Test + @Timeout(value = 50) public void testGetNextHeartBeatInterval() throws Exception { Configuration conf = new Configuration(); conf.set(YarnConfiguration.RM_NM_HEARTBEAT_INTERVAL_MS, "4000"); @@ -186,10 +193,10 @@ public void testGetNextHeartBeatInterval() throws Exception { MockNM nm2 = rm.registerNode("host2:5678", 10240); NodeHeartbeatResponse nodeHeartbeat = nm1.nodeHeartbeat(true); - Assert.assertEquals(4000, nodeHeartbeat.getNextHeartBeatInterval()); + assertEquals(4000, nodeHeartbeat.getNextHeartBeatInterval()); NodeHeartbeatResponse nodeHeartbeat2 = nm2.nodeHeartbeat(true); - Assert.assertEquals(4000, nodeHeartbeat2.getNextHeartBeatInterval()); + assertEquals(4000, nodeHeartbeat2.getNextHeartBeatInterval()); } @@ -216,11 +223,11 @@ public void testDecommissionWithIncludeHosts() throws Exception { int metricCount = metrics.getNumDecommisionedNMs(); NodeHeartbeatResponse nodeHeartbeat = nm1.nodeHeartbeat(true); - Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); nodeHeartbeat = nm2.nodeHeartbeat(true); - Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); nodeHeartbeat = nm3.nodeHeartbeat(true); - Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); // To test that IPs also work String ip = NetUtils.normalizeHostName("localhost"); @@ -231,17 +238,17 @@ public void testDecommissionWithIncludeHosts() throws Exception { checkShutdownNMCount(rm, ++metricCount); nodeHeartbeat = nm1.nodeHeartbeat(true); - Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); - Assert - .assertEquals(1, ClusterMetrics.getMetrics().getNumShutdownNMs()); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); + + assertEquals(1, ClusterMetrics.getMetrics().getNumShutdownNMs()); nodeHeartbeat = nm2.nodeHeartbeat(true); - Assert.assertTrue("Node is not decommisioned.", NodeAction.SHUTDOWN - .equals(nodeHeartbeat.getNodeAction())); + assertTrue(NodeAction.SHUTDOWN.equals(nodeHeartbeat.getNodeAction()), + "Node is not decommisioned."); nodeHeartbeat = nm3.nodeHeartbeat(true); - Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); - Assert.assertEquals(metricCount, ClusterMetrics.getMetrics() + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); + assertEquals(metricCount, ClusterMetrics.getMetrics() .getNumShutdownNMs()); rm.stop(); } @@ -266,9 +273,9 @@ public void testDecommissionWithExcludeHosts() throws Exception { int metricCount = ClusterMetrics.getMetrics().getNumDecommisionedNMs(); NodeHeartbeatResponse nodeHeartbeat = nm1.nodeHeartbeat(true); - Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); nodeHeartbeat = nm2.nodeHeartbeat(true); - Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); rm.drainEvents(); // To test that IPs also work @@ -280,14 +287,14 @@ public void testDecommissionWithExcludeHosts() throws Exception { checkDecommissionedNMCount(rm, metricCount + 2); nodeHeartbeat = nm1.nodeHeartbeat(true); - Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); nodeHeartbeat = nm2.nodeHeartbeat(true); - Assert.assertTrue("The decommisioned metrics are not updated", - NodeAction.SHUTDOWN.equals(nodeHeartbeat.getNodeAction())); + assertTrue(NodeAction.SHUTDOWN.equals(nodeHeartbeat.getNodeAction()), + "The decommisioned metrics are not updated"); nodeHeartbeat = nm3.nodeHeartbeat(true); - Assert.assertTrue("The decommisioned metrics are not updated", - NodeAction.SHUTDOWN.equals(nodeHeartbeat.getNodeAction())); + assertTrue(NodeAction.SHUTDOWN.equals(nodeHeartbeat.getNodeAction()), + "The decommisioned metrics are not updated"); rm.drainEvents(); writeToHostsFile(""); @@ -296,7 +303,7 @@ public void testDecommissionWithExcludeHosts() throws Exception { nm3 = rm.registerNode("localhost:4433", 1024); nodeHeartbeat = nm3.nodeHeartbeat(true); rm.drainEvents(); - Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); // decommissined node is 1 since 1 node is rejoined after updating exclude // file checkDecommissionedNMCount(rm, metricCount + 1); @@ -324,9 +331,9 @@ public void testGracefulDecommissionNoApp() throws Exception { NodeHeartbeatResponse nodeHeartbeat2 = nm2.nodeHeartbeat(true); NodeHeartbeatResponse nodeHeartbeat3 = nm3.nodeHeartbeat(true); - Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat1.getNodeAction())); - Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat2.getNodeAction())); - Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat3.getNodeAction())); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat1.getNodeAction())); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat2.getNodeAction())); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat3.getNodeAction())); rm.waitForState(nm2.getNodeId(), NodeState.RUNNING); rm.waitForState(nm3.getNodeId(), NodeState.RUNNING); @@ -346,9 +353,9 @@ public void testGracefulDecommissionNoApp() throws Exception { rm.waitForState(nm2.getNodeId(), NodeState.DECOMMISSIONED); rm.waitForState(nm3.getNodeId(), NodeState.DECOMMISSIONED); - Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat1.getNodeAction())); - Assert.assertEquals(NodeAction.SHUTDOWN, nodeHeartbeat2.getNodeAction()); - Assert.assertEquals(NodeAction.SHUTDOWN, nodeHeartbeat3.getNodeAction()); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat1.getNodeAction())); + assertEquals(NodeAction.SHUTDOWN, nodeHeartbeat2.getNodeAction()); + assertEquals(NodeAction.SHUTDOWN, nodeHeartbeat3.getNodeAction()); } @Test @@ -371,12 +378,9 @@ public void testGracefulDecommissionDefaultTimeoutResolution() NodeHeartbeatResponse nodeHeartbeat2 = nm2.nodeHeartbeat(true); NodeHeartbeatResponse nodeHeartbeat3 = nm3.nodeHeartbeat(true); - Assert.assertTrue( - NodeAction.NORMAL.equals(nodeHeartbeat1.getNodeAction())); - Assert.assertTrue( - NodeAction.NORMAL.equals(nodeHeartbeat2.getNodeAction())); - Assert.assertTrue( - NodeAction.NORMAL.equals(nodeHeartbeat3.getNodeAction())); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat1.getNodeAction())); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat2.getNodeAction())); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat3.getNodeAction())); rm.waitForState(nm1.getNodeId(), NodeState.RUNNING); rm.waitForState(nm2.getNodeId(), NodeState.RUNNING); @@ -392,13 +396,11 @@ public void testGracefulDecommissionDefaultTimeoutResolution() rm.getNodesListManager().refreshNodes(conf, true); rm.waitForState(nm1.getNodeId(), NodeState.DECOMMISSIONING); rm.waitForState(nm2.getNodeId(), NodeState.DECOMMISSIONING); - Assert.assertEquals( - nm1DecommissionTimeout, rm.getDecommissioningTimeout(nm1.getNodeId())); + assertEquals(nm1DecommissionTimeout, rm.getDecommissioningTimeout(nm1.getNodeId())); Integer defaultDecTimeout = conf.getInt(YarnConfiguration.RM_NODE_GRACEFUL_DECOMMISSION_TIMEOUT, YarnConfiguration.DEFAULT_RM_NODE_GRACEFUL_DECOMMISSION_TIMEOUT); - Assert.assertEquals( - defaultDecTimeout, rm.getDecommissioningTimeout(nm2.getNodeId())); + assertEquals(defaultDecTimeout, rm.getDecommissioningTimeout(nm2.getNodeId())); // Graceful decommission host3 with a new default timeout final Integer newDefaultDecTimeout = defaultDecTimeout + 10; @@ -408,8 +410,7 @@ public void testGracefulDecommissionDefaultTimeoutResolution() newDefaultDecTimeout); rm.getNodesListManager().refreshNodes(conf, true); rm.waitForState(nm3.getNodeId(), NodeState.DECOMMISSIONING); - Assert.assertEquals( - newDefaultDecTimeout, rm.getDecommissioningTimeout(nm3.getNodeId())); + assertEquals(newDefaultDecTimeout, rm.getDecommissioningTimeout(nm3.getNodeId())); } /** @@ -458,13 +459,13 @@ public void testGracefulDecommissionWithApp() throws Exception { // Since the app is still RUNNING, expect NodeAction.NORMAL. NodeHeartbeatResponse nodeHeartbeat1 = nm1.nodeHeartbeat(aaid, 2, ContainerState.COMPLETE); - Assert.assertEquals(NodeAction.NORMAL, nodeHeartbeat1.getNodeAction()); + assertEquals(NodeAction.NORMAL, nodeHeartbeat1.getNodeAction()); // Finish the app and verified DECOMMISSIONED. MockRM.finishAMAndVerifyAppState(app, rm, nm1, am); rm.waitForState(app.getApplicationId(), RMAppState.FINISHED); nodeHeartbeat1 = nm1.nodeHeartbeat(aaid, 2, ContainerState.COMPLETE); - Assert.assertEquals(NodeAction.SHUTDOWN, nodeHeartbeat1.getNodeAction()); + assertEquals(NodeAction.SHUTDOWN, nodeHeartbeat1.getNodeAction()); rm.waitForState(id1, NodeState.DECOMMISSIONED); } @@ -472,7 +473,8 @@ public void testGracefulDecommissionWithApp() throws Exception { * Test graceful decommission of node when an AM container is scheduled on a * node just before it is gracefully decommissioned. */ - @Test (timeout = 60000) + @Test + @Timeout(value = 60) public void testGracefulDecommissionAfterAMContainerAlloc() throws Exception { Configuration conf = new Configuration(); conf.set(YarnConfiguration.RM_NODES_EXCLUDE_FILE_PATH, hostFile @@ -540,11 +542,11 @@ public void testAddNewIncludePathToConfiguration() throws Exception { assert(metrics != null); int initialMetricCount = metrics.getNumShutdownNMs(); NodeHeartbeatResponse nodeHeartbeat = nm1.nodeHeartbeat(true); - Assert.assertEquals( + assertEquals( NodeAction.NORMAL, nodeHeartbeat.getNodeAction()); nodeHeartbeat = nm2.nodeHeartbeat(true); - Assert.assertEquals( + assertEquals( NodeAction.NORMAL, nodeHeartbeat.getNodeAction()); writeToHostsFile("host1"); @@ -553,14 +555,12 @@ public void testAddNewIncludePathToConfiguration() throws Exception { rm.getNodesListManager().refreshNodes(conf); checkShutdownNMCount(rm, ++initialMetricCount); nodeHeartbeat = nm1.nodeHeartbeat(true); - Assert.assertEquals( - "Node should not have been shutdown.", - NodeAction.NORMAL, - nodeHeartbeat.getNodeAction()); + assertEquals(NodeAction.NORMAL, + nodeHeartbeat.getNodeAction(), "Node should not have been shutdown."); NodeState nodeState = rm.getRMContext().getInactiveRMNodes().get(nm2.getNodeId()).getState(); - Assert.assertEquals("Node should have been shutdown but is in state" + - nodeState, NodeState.SHUTDOWN, nodeState); + assertEquals(NodeState.SHUTDOWN, nodeState, + "Node should have been shutdown but is in state" + nodeState); } /** @@ -577,11 +577,11 @@ public void testAddNewExcludePathToConfiguration() throws Exception { assert(metrics != null); int initialMetricCount = metrics.getNumDecommisionedNMs(); NodeHeartbeatResponse nodeHeartbeat = nm1.nodeHeartbeat(true); - Assert.assertEquals( + assertEquals( NodeAction.NORMAL, nodeHeartbeat.getNodeAction()); nodeHeartbeat = nm2.nodeHeartbeat(true); - Assert.assertEquals( + assertEquals( NodeAction.NORMAL, nodeHeartbeat.getNodeAction()); writeToHostsFile("host2"); @@ -590,15 +590,12 @@ public void testAddNewExcludePathToConfiguration() throws Exception { rm.getNodesListManager().refreshNodes(conf); checkDecommissionedNMCount(rm, ++initialMetricCount); nodeHeartbeat = nm1.nodeHeartbeat(true); - Assert.assertEquals( - "Node should not have been decommissioned.", - NodeAction.NORMAL, - nodeHeartbeat.getNodeAction()); + assertEquals(NodeAction.NORMAL, + nodeHeartbeat.getNodeAction(), "Node should not have been decommissioned."); nodeHeartbeat = nm2.nodeHeartbeat(true); - Assert.assertEquals( + assertEquals(NodeAction.SHUTDOWN, nodeHeartbeat.getNodeAction(), "Node should have been decommissioned but is in state" - + nodeHeartbeat.getNodeAction(), - NodeAction.SHUTDOWN, nodeHeartbeat.getNodeAction()); + + nodeHeartbeat.getNodeAction()); } @Test @@ -622,7 +619,7 @@ public void testNodeRegistrationSuccess() throws Exception { // trying to register a invalid node. RegisterNodeManagerResponse response = resourceTrackerService.registerNodeManager(req); - Assert.assertEquals(NodeAction.NORMAL, response.getNodeAction()); + assertEquals(NodeAction.NORMAL, response.getNodeAction()); } @Test @@ -647,7 +644,7 @@ protected RMNodeLabelsManager createNodeLabelManager() { try { nodeLabelsMgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("A", "B", "C")); } catch (IOException e) { - Assert.fail("Caught Exception while initializing"); + fail("Caught Exception while initializing"); e.printStackTrace(); } @@ -665,12 +662,12 @@ protected RMNodeLabelsManager createNodeLabelManager() { RegisterNodeManagerResponse response = resourceTrackerService.registerNodeManager(registerReq); - Assert.assertEquals("Action should be normal on valid Node Labels", - NodeAction.NORMAL, response.getNodeAction()); + assertEquals(NodeAction.NORMAL, response.getNodeAction(), + "Action should be normal on valid Node Labels"); assertCollectionEquals(nodeLabelsMgr.getNodeLabels().get(nodeId), NodeLabelsUtils.convertToStringSet(registerReq.getNodeLabels())); - Assert.assertTrue("Valid Node Labels were not accepted by RM", - response.getAreNodeLabelsAcceptedByRM()); + assertTrue(response.getAreNodeLabelsAcceptedByRM(), + "Valid Node Labels were not accepted by RM"); rm.stop(); } @@ -696,7 +693,7 @@ protected RMNodeLabelsManager createNodeLabelManager() { try { nodeLabelsMgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("X", "Y", "Z")); } catch (IOException e) { - Assert.fail("Caught Exception while initializing"); + fail("Caught Exception while initializing"); e.printStackTrace(); } @@ -714,13 +711,12 @@ protected RMNodeLabelsManager createNodeLabelManager() { RegisterNodeManagerResponse response = resourceTrackerService.registerNodeManager(registerReq); - Assert.assertEquals( - "On Invalid Node Labels action is expected to be normal", - NodeAction.NORMAL, response.getNodeAction()); - Assert.assertNull(nodeLabelsMgr.getNodeLabels().get(nodeId)); - Assert.assertNotNull(response.getDiagnosticsMessage()); - Assert.assertFalse("Node Labels should not accepted by RM If Invalid", - response.getAreNodeLabelsAcceptedByRM()); + assertEquals(NodeAction.NORMAL, response.getNodeAction(), + "On Invalid Node Labels action is expected to be normal"); + assertNull(nodeLabelsMgr.getNodeLabels().get(nodeId)); + assertNotNull(response.getDiagnosticsMessage()); + assertFalse(response.getAreNodeLabelsAcceptedByRM(), + "Node Labels should not accepted by RM If Invalid"); if (rm != null) { rm.stop(); @@ -749,7 +745,7 @@ protected RMNodeLabelsManager createNodeLabelManager() { try { nodeLabelsMgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("X", "Y", "Z")); } catch (IOException e) { - Assert.fail("Caught Exception while initializing"); + fail("Caught Exception while initializing"); e.printStackTrace(); } @@ -767,13 +763,12 @@ protected RMNodeLabelsManager createNodeLabelManager() { RegisterNodeManagerResponse response = resourceTrackerService.registerNodeManager(req); - Assert.assertEquals( - "On Invalid Node Labels action is expected to be normal", - NodeAction.NORMAL, response.getNodeAction()); - Assert.assertNull(nodeLabelsMgr.getNodeLabels().get(nodeId)); - Assert.assertNotNull(response.getDiagnosticsMessage()); - Assert.assertFalse("Node Labels should not accepted by RM If Invalid", - response.getAreNodeLabelsAcceptedByRM()); + assertEquals(NodeAction.NORMAL, response.getNodeAction(), + "On Invalid Node Labels action is expected to be normal"); + assertNull(nodeLabelsMgr.getNodeLabels().get(nodeId)); + assertNotNull(response.getDiagnosticsMessage()); + assertFalse(response.getAreNodeLabelsAcceptedByRM(), + "Node Labels should not accepted by RM If Invalid"); if (rm != null) { rm.stop(); @@ -801,7 +796,7 @@ protected RMNodeLabelsManager createNodeLabelManager() { try { nodeLabelsMgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("A", "B", "C")); } catch (IOException e) { - Assert.fail("Caught Exception while initializing"); + fail("Caught Exception while initializing"); e.printStackTrace(); } ResourceTrackerService resourceTrackerService = @@ -818,13 +813,11 @@ protected RMNodeLabelsManager createNodeLabelManager() { RegisterNodeManagerResponse response = resourceTrackerService.registerNodeManager(req); // registered to RM with central label config - Assert.assertEquals(NodeAction.NORMAL, response.getNodeAction()); - Assert.assertNull(nodeLabelsMgr.getNodeLabels().get(nodeId)); - Assert - .assertFalse( - "Node Labels should not accepted by RM If its configured with " + - "Central configuration", - response.getAreNodeLabelsAcceptedByRM()); + assertEquals(NodeAction.NORMAL, response.getNodeAction()); + assertNull(nodeLabelsMgr.getNodeLabels().get(nodeId)); + assertFalse(response.getAreNodeLabelsAcceptedByRM(), + "Node Labels should not accepted by RM If its configured with " + + "Central configuration"); if (rm != null) { rm.stop(); } @@ -867,14 +860,14 @@ public void testNodeRegistrationWithAttributes() throws Exception { RegisterNodeManagerResponse response = resourceTrackerService.registerNodeManager(registerReq); - Assert.assertEquals("Action should be normal on valid Node Attributes", - NodeAction.NORMAL, response.getNodeAction()); - Assert.assertTrue(NodeLabelUtil.isNodeAttributesEquals( + assertEquals(NodeAction.NORMAL, response.getNodeAction(), + "Action should be normal on valid Node Attributes"); + assertTrue(NodeLabelUtil.isNodeAttributesEquals( rm.getRMContext().getNodeAttributesManager() .getAttributesForNode(nodeId.getHost()).keySet(), registerReq.getNodeAttributes())); - Assert.assertTrue("Valid Node Attributes were not accepted by RM", - response.getAreNodeAttributesAcceptedByRM()); + assertTrue(response.getAreNodeAttributesAcceptedByRM(), + "Valid Node Attributes were not accepted by RM"); if (rm != null) { rm.stop(); @@ -922,28 +915,28 @@ public void testNodeRegistrationWithInvalidAttributes() throws Exception { toSet(validNodeAttribute, invalidPrefixNodeAttribute)); RegisterNodeManagerResponse response = resourceTrackerService.registerNodeManager(req); - Assert.assertEquals(0, rm.getRMContext().getNodeAttributesManager() + assertEquals(0, rm.getRMContext().getNodeAttributesManager() .getAttributesForNode(nodeId.getHost()).size()); assertRegisterResponseForInvalidAttributes(response); - Assert.assertTrue(response.getDiagnosticsMessage() + assertTrue(response.getDiagnosticsMessage() .endsWith("attributes in HB must have prefix nm.yarn.io")); // check invalid name req.setNodeAttributes(toSet(validNodeAttribute, invalidNameNodeAttribute)); response = resourceTrackerService.registerNodeManager(req); - Assert.assertEquals(0, rm.getRMContext().getNodeAttributesManager() + assertEquals(0, rm.getRMContext().getNodeAttributesManager() .getAttributesForNode(nodeId.getHost()).size()); assertRegisterResponseForInvalidAttributes(response); - Assert.assertTrue(response.getDiagnosticsMessage() + assertTrue(response.getDiagnosticsMessage() .startsWith("attribute name should only contains")); // check invalid value req.setNodeAttributes(toSet(validNodeAttribute, invalidValueNodeAttribute)); response = resourceTrackerService.registerNodeManager(req); - Assert.assertEquals(0, rm.getRMContext().getNodeAttributesManager() + assertEquals(0, rm.getRMContext().getNodeAttributesManager() .getAttributesForNode(nodeId.getHost()).size()); assertRegisterResponseForInvalidAttributes(response); - Assert.assertTrue(response.getDiagnosticsMessage() + assertTrue(response.getDiagnosticsMessage() .startsWith("attribute value should only contains")); if (rm != null) { @@ -953,12 +946,11 @@ public void testNodeRegistrationWithInvalidAttributes() throws Exception { private void assertRegisterResponseForInvalidAttributes( RegisterNodeManagerResponse response) { - Assert.assertEquals( - "On Invalid Node Labels action is expected to be normal", - NodeAction.NORMAL, response.getNodeAction()); - Assert.assertNotNull(response.getDiagnosticsMessage()); - Assert.assertFalse("Node Labels should not accepted by RM If Invalid", - response.getAreNodeLabelsAcceptedByRM()); + assertEquals(NodeAction.NORMAL, response.getNodeAction(), + "On Invalid Node Labels action is expected to be normal"); + assertNotNull(response.getDiagnosticsMessage()); + assertFalse(response.getAreNodeLabelsAcceptedByRM(), + "Node Labels should not accepted by RM If Invalid"); } private NodeStatus getNodeStatusObject(NodeId nodeId) { @@ -992,7 +984,7 @@ protected RMNodeLabelsManager createNodeLabelManager() { try { nodeLabelsMgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("A", "B", "C")); } catch (IOException e) { - Assert.fail("Caught Exception while initializing"); + fail("Caught Exception while initializing"); e.printStackTrace(); } @@ -1024,12 +1016,12 @@ protected RMNodeLabelsManager createNodeLabelManager() { NodeHeartbeatResponse nodeHeartbeatResponse = resourceTrackerService.nodeHeartbeat(heartbeatReq); - Assert.assertEquals("InValid Node Labels were not accepted by RM", - NodeAction.NORMAL, nodeHeartbeatResponse.getNodeAction()); + assertEquals(NodeAction.NORMAL, nodeHeartbeatResponse.getNodeAction(), + "InValid Node Labels were not accepted by RM"); assertCollectionEquals(nodeLabelsMgr.getNodeLabels().get(nodeId), NodeLabelsUtils.convertToStringSet(heartbeatReq.getNodeLabels())); - Assert.assertTrue("Valid Node Labels were not accepted by RM", - nodeHeartbeatResponse.getAreNodeLabelsAcceptedByRM()); + assertTrue(nodeHeartbeatResponse.getAreNodeLabelsAcceptedByRM(), + "Valid Node Labels were not accepted by RM"); // After modification of labels next heartbeat sends null informing no update Set oldLabels = nodeLabelsMgr.getNodeLabels().get(nodeId); @@ -1046,12 +1038,12 @@ protected RMNodeLabelsManager createNodeLabelManager() { .getContainerTokenMasterKey()); nodeHeartbeatResponse = resourceTrackerService.nodeHeartbeat(heartbeatReq); - Assert.assertEquals("InValid Node Labels were not accepted by RM", - NodeAction.NORMAL, nodeHeartbeatResponse.getNodeAction()); + assertEquals(NodeAction.NORMAL, nodeHeartbeatResponse.getNodeAction(), + "InValid Node Labels were not accepted by RM"); assertCollectionEquals(nodeLabelsMgr.getNodeLabels().get(nodeId), oldLabels); - Assert.assertFalse("Node Labels should not accepted by RM", - nodeHeartbeatResponse.getAreNodeLabelsAcceptedByRM()); + assertFalse(nodeHeartbeatResponse.getAreNodeLabelsAcceptedByRM(), + "Node Labels should not accepted by RM"); rm.stop(); } @@ -1105,11 +1097,11 @@ public void testNodeHeartbeatWithNodeAttributes() throws Exception { rm.getRMContext().getNodeAttributesManager(); Map attrs = attributeManager .getAttributesForNode(nodeId.getHost()); - Assert.assertEquals(1, attrs.size()); + assertEquals(1, attrs.size()); NodeAttribute na = attrs.keySet().iterator().next(); - Assert.assertEquals("host", na.getAttributeKey().getAttributeName()); - Assert.assertEquals("host2", na.getAttributeValue()); - Assert.assertEquals(NodeAttributeType.STRING, na.getAttributeType()); + assertEquals("host", na.getAttributeKey().getAttributeName()); + assertEquals("host2", na.getAttributeValue()); + assertEquals(NodeAttributeType.STRING, na.getAttributeType()); // Send another HB to RM with updated node atrribute @@ -1125,11 +1117,11 @@ public void testNodeHeartbeatWithNodeAttributes() throws Exception { // Make sure RM gets the updated attribute attrs = attributeManager.getAttributesForNode(nodeId.getHost()); - Assert.assertEquals(1, attrs.size()); + assertEquals(1, attrs.size()); na = attrs.keySet().iterator().next(); - Assert.assertEquals("host", na.getAttributeKey().getAttributeName()); - Assert.assertEquals("host3", na.getAttributeValue()); - Assert.assertEquals(NodeAttributeType.STRING, na.getAttributeType()); + assertEquals("host", na.getAttributeKey().getAttributeName()); + assertEquals("host3", na.getAttributeValue()); + assertEquals(NodeAttributeType.STRING, na.getAttributeType()); } @Test @@ -1189,10 +1181,10 @@ public void testNodeHeartbeatWithInvalidNodeAttributes() throws Exception { toSet(validNodeAttribute, invalidPrefixNodeAttribute)); NodeHeartbeatResponse response = resourceTrackerService.nodeHeartbeat(heartbeatReq); - Assert.assertEquals(0, rm.getRMContext().getNodeAttributesManager() + assertEquals(0, rm.getRMContext().getNodeAttributesManager() .getAttributesForNode(nodeId.getHost()).size()); assertNodeHeartbeatResponseForInvalidAttributes(response); - Assert.assertTrue(response.getDiagnosticsMessage() + assertTrue(response.getDiagnosticsMessage() .endsWith("attributes in HB must have prefix nm.yarn.io")); // Send another HB to RM with invalid name node attributes @@ -1200,10 +1192,10 @@ public void testNodeHeartbeatWithInvalidNodeAttributes() throws Exception { heartbeatReq .setNodeAttributes(toSet(validNodeAttribute, invalidNameNodeAttribute)); response = resourceTrackerService.nodeHeartbeat(heartbeatReq); - Assert.assertEquals(0, rm.getRMContext().getNodeAttributesManager() + assertEquals(0, rm.getRMContext().getNodeAttributesManager() .getAttributesForNode(nodeId.getHost()).size()); assertNodeHeartbeatResponseForInvalidAttributes(response); - Assert.assertTrue(response.getDiagnosticsMessage() + assertTrue(response.getDiagnosticsMessage() .startsWith("attribute name should only contains")); // Send another HB to RM with invalid value node attributes @@ -1211,10 +1203,10 @@ public void testNodeHeartbeatWithInvalidNodeAttributes() throws Exception { heartbeatReq.setNodeAttributes( toSet(validNodeAttribute, invalidValueNodeAttribute)); response = resourceTrackerService.nodeHeartbeat(heartbeatReq); - Assert.assertEquals(0, rm.getRMContext().getNodeAttributesManager() + assertEquals(0, rm.getRMContext().getNodeAttributesManager() .getAttributesForNode(nodeId.getHost()).size()); assertNodeHeartbeatResponseForInvalidAttributes(response); - Assert.assertTrue(response.getDiagnosticsMessage() + assertTrue(response.getDiagnosticsMessage() .startsWith("attribute value should only contains")); // Send another HB to RM with updated node attribute @@ -1230,21 +1222,20 @@ public void testNodeHeartbeatWithInvalidNodeAttributes() throws Exception { rm.getRMContext().getNodeAttributesManager(); Map attrs = attributeManager.getAttributesForNode(nodeId.getHost()); - Assert.assertEquals(1, attrs.size()); + assertEquals(1, attrs.size()); NodeAttribute na = attrs.keySet().iterator().next(); - Assert.assertEquals("host", na.getAttributeKey().getAttributeName()); - Assert.assertEquals("host3", na.getAttributeValue()); - Assert.assertEquals(NodeAttributeType.STRING, na.getAttributeType()); + assertEquals("host", na.getAttributeKey().getAttributeName()); + assertEquals("host3", na.getAttributeValue()); + assertEquals(NodeAttributeType.STRING, na.getAttributeType()); } private void assertNodeHeartbeatResponseForInvalidAttributes( NodeHeartbeatResponse response) { - Assert.assertEquals( - "On Invalid Node Labels action is expected to be normal", - NodeAction.NORMAL, response.getNodeAction()); - Assert.assertNotNull(response.getDiagnosticsMessage()); - Assert.assertFalse("Node Labels should not accepted by RM If Invalid", - response.getAreNodeLabelsAcceptedByRM()); + assertEquals(NodeAction.NORMAL, response.getNodeAction(), + "On Invalid Node Labels action is expected to be normal"); + assertNotNull(response.getDiagnosticsMessage()); + assertFalse(response.getAreNodeLabelsAcceptedByRM(), + "Node Labels should not accepted by RM If Invalid"); } @Test @@ -1315,12 +1306,12 @@ public Object answer(InvocationOnMock invocation) throws Exception { Map attrs = spyAttributeManager .getAttributesForNode(nodeId.getHost()); spyAttributeManager.getNodesToAttributes(ImmutableSet.of(nodeId.getHost())); - Assert.assertEquals(1, attrs.size()); + assertEquals(1, attrs.size()); NodeAttribute na = attrs.keySet().iterator().next(); - Assert.assertEquals("host", na.getAttributeKey().getAttributeName()); - Assert.assertEquals("host2", na.getAttributeValue()); - Assert.assertEquals(NodeAttributeType.STRING, na.getAttributeType()); - Assert.assertEquals(1, count.get()); + assertEquals("host", na.getAttributeKey().getAttributeName()); + assertEquals("host2", na.getAttributeValue()); + assertEquals(NodeAttributeType.STRING, na.getAttributeType()); + assertEquals(1, count.get()); // Send HBs to RM with the same node attributes nodeStatusObject.setResponseId(++responseId); @@ -1332,7 +1323,7 @@ public Object answer(InvocationOnMock invocation) throws Exception { resourceTrackerService.nodeHeartbeat(heartbeatReq); // Make sure RM updated node attributes once - Assert.assertEquals(1, count.get()); + assertEquals(1, count.get()); // Send another HB to RM with updated node attributes nodeAttributes.clear(); @@ -1346,14 +1337,14 @@ public Object answer(InvocationOnMock invocation) throws Exception { // Make sure RM gets the updated attribute attrs = spyAttributeManager.getAttributesForNode(nodeId.getHost()); - Assert.assertEquals(1, attrs.size()); + assertEquals(1, attrs.size()); na = attrs.keySet().iterator().next(); - Assert.assertEquals("host", na.getAttributeKey().getAttributeName()); - Assert.assertEquals("host3", na.getAttributeValue()); - Assert.assertEquals(NodeAttributeType.STRING, na.getAttributeType()); + assertEquals("host", na.getAttributeKey().getAttributeName()); + assertEquals("host3", na.getAttributeValue()); + assertEquals(NodeAttributeType.STRING, na.getAttributeType()); // Make sure RM updated node attributes twice - Assert.assertEquals(2, count.get()); + assertEquals(2, count.get()); // Add centralized attributes Map> nodeAttributeMapping = ImmutableMap @@ -1364,7 +1355,7 @@ public Object answer(InvocationOnMock invocation) throws Exception { nodeAttributeMapping); // Make sure RM updated node attributes three times - Assert.assertEquals(3, count.get()); + assertEquals(3, count.get()); // Send another HB to RM with non-updated node attributes nodeAttributes.clear(); @@ -1377,7 +1368,7 @@ public Object answer(InvocationOnMock invocation) throws Exception { resourceTrackerService.nodeHeartbeat(heartbeatReq); // Make sure RM still updated node attributes three times - Assert.assertEquals(3, count.get()); + assertEquals(3, count.get()); // Send another HB to RM with updated node attributes nodeAttributes.clear(); @@ -1391,20 +1382,20 @@ public Object answer(InvocationOnMock invocation) throws Exception { // Make sure RM gets the updated attribute attrs = spyAttributeManager.getAttributesForNode(nodeId.getHost()); - Assert.assertEquals(2, attrs.size()); + assertEquals(2, attrs.size()); attrs.keySet().stream().forEach(e -> { - Assert.assertEquals(NodeAttributeType.STRING, e.getAttributeType()); + assertEquals(NodeAttributeType.STRING, e.getAttributeType()); if (e.getAttributeKey().getAttributePrefix() == NodeAttribute.PREFIX_DISTRIBUTED) { - Assert.assertEquals("host", e.getAttributeKey().getAttributeName()); - Assert.assertEquals("host4", e.getAttributeValue()); + assertEquals("host", e.getAttributeKey().getAttributeName()); + assertEquals("host4", e.getAttributeValue()); } else if (e.getAttributeKey().getAttributePrefix() == NodeAttribute.PREFIX_CENTRALIZED) { - Assert.assertEquals("centAttr", e.getAttributeKey().getAttributeName()); - Assert.assertEquals("x", e.getAttributeValue()); + assertEquals("centAttr", e.getAttributeKey().getAttributeName()); + assertEquals("x", e.getAttributeValue()); } }); // Make sure RM updated node attributes four times - Assert.assertEquals(4, count.get()); + assertEquals(4, count.get()); if (rm != null) { rm.stop(); @@ -1433,7 +1424,7 @@ protected RMNodeLabelsManager createNodeLabelManager() { try { nodeLabelsMgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("A", "B", "C")); } catch (IOException e) { - Assert.fail("Caught Exception while initializing"); + fail("Caught Exception while initializing"); e.printStackTrace(); } @@ -1463,11 +1454,11 @@ protected RMNodeLabelsManager createNodeLabelManager() { resourceTrackerService.nodeHeartbeat(heartbeatReq); // response should be NORMAL when RM heartbeat labels are rejected - Assert.assertEquals("Response should be NORMAL when RM heartbeat labels" - + " are rejected", NodeAction.NORMAL, - nodeHeartbeatResponse.getNodeAction()); - Assert.assertFalse(nodeHeartbeatResponse.getAreNodeLabelsAcceptedByRM()); - Assert.assertNotNull(nodeHeartbeatResponse.getDiagnosticsMessage()); + assertEquals(NodeAction.NORMAL, nodeHeartbeatResponse.getNodeAction(), + "Response should be NORMAL when RM heartbeat labels" + + " are rejected"); + assertFalse(nodeHeartbeatResponse.getAreNodeLabelsAcceptedByRM()); + assertNotNull(nodeHeartbeatResponse.getDiagnosticsMessage()); rm.stop(); } @@ -1516,13 +1507,13 @@ protected RMNodeLabelsManager createNodeLabelManager() { resourceTrackerService.nodeHeartbeat(heartbeatReq); // response should be ok but the RMacceptNodeLabelsUpdate should be false - Assert.assertEquals(NodeAction.NORMAL, + assertEquals(NodeAction.NORMAL, nodeHeartbeatResponse.getNodeAction()); // no change in the labels, - Assert.assertNull(nodeLabelsMgr.getNodeLabels().get(nodeId)); + assertNull(nodeLabelsMgr.getNodeLabels().get(nodeId)); // heartbeat labels rejected - Assert.assertFalse("Invalid Node Labels should not accepted by RM", - nodeHeartbeatResponse.getAreNodeLabelsAcceptedByRM()); + assertFalse(nodeHeartbeatResponse.getAreNodeLabelsAcceptedByRM(), + "Invalid Node Labels should not accepted by RM"); if (rm != null) { rm.stop(); } @@ -1550,11 +1541,11 @@ public void testNodeRegistrationVersionLessThanRM() throws Exception { req.setNMVersion(nmVersion); // trying to register a invalid node. RegisterNodeManagerResponse response = resourceTrackerService.registerNodeManager(req); - Assert.assertEquals(NodeAction.SHUTDOWN,response.getNodeAction()); - Assert.assertTrue("Diagnostic message did not contain: 'Disallowed NodeManager " + - "Version "+ nmVersion + ", is less than the minimum version'", - response.getDiagnosticsMessage().contains("Disallowed NodeManager Version " + - nmVersion + ", is less than the minimum version ")); + assertEquals(NodeAction.SHUTDOWN, response.getNodeAction()); + assertTrue(response.getDiagnosticsMessage().contains("Disallowed NodeManager Version " + + nmVersion + ", is less than the minimum version "), + "Diagnostic message did not contain: 'Disallowed NodeManager " + + "Version "+ nmVersion + ", is less than the minimum version'"); } @@ -1575,9 +1566,8 @@ public void testNodeRegistrationFailure() throws Exception { req.setHttpPort(1234); // trying to register a invalid node. RegisterNodeManagerResponse response = resourceTrackerService.registerNodeManager(req); - Assert.assertEquals(NodeAction.SHUTDOWN,response.getNodeAction()); - Assert - .assertEquals( + assertEquals(NodeAction.SHUTDOWN, response.getNodeAction()); + assertEquals( "Disallowed NodeManager from host2, Sending SHUTDOWN signal to the NodeManager.", response.getDiagnosticsMessage()); } @@ -1593,7 +1583,7 @@ public void testSetRMIdentifierInRegistration() throws Exception { RegisterNodeManagerResponse response = nm.registerNode(); // Verify the RMIdentifier is correctly set in RegisterNodeManagerResponse - Assert.assertEquals(ResourceManager.getClusterTimeStamp(), + assertEquals(ResourceManager.getClusterTimeStamp(), response.getRMIdentifier()); } @@ -1616,28 +1606,28 @@ public void testNodeRegistrationWithMinimumAllocations() throws Exception { req.setResource(capability); RegisterNodeManagerResponse response1 = resourceTrackerService.registerNodeManager(req); - Assert.assertEquals(NodeAction.SHUTDOWN,response1.getNodeAction()); + assertEquals(NodeAction.SHUTDOWN, response1.getNodeAction()); capability.setMemorySize(2048); capability.setVirtualCores(1); req.setResource(capability); RegisterNodeManagerResponse response2 = resourceTrackerService.registerNodeManager(req); - Assert.assertEquals(NodeAction.SHUTDOWN,response2.getNodeAction()); + assertEquals(NodeAction.SHUTDOWN, response2.getNodeAction()); capability.setMemorySize(1024); capability.setVirtualCores(4); req.setResource(capability); RegisterNodeManagerResponse response3 = resourceTrackerService.registerNodeManager(req); - Assert.assertEquals(NodeAction.SHUTDOWN,response3.getNodeAction()); + assertEquals(NodeAction.SHUTDOWN, response3.getNodeAction()); capability.setMemorySize(2048); capability.setVirtualCores(4); req.setResource(capability); RegisterNodeManagerResponse response4 = resourceTrackerService.registerNodeManager(req); - Assert.assertEquals(NodeAction.NORMAL,response4.getNodeAction()); + assertEquals(NodeAction.NORMAL, response4.getNodeAction()); } @Test @@ -1651,12 +1641,12 @@ public void testReboot() throws Exception { int initialMetricCount = ClusterMetrics.getMetrics().getNumRebootedNMs(); NodeHeartbeatResponse nodeHeartbeat = nm1.nodeHeartbeat(true); - Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); nodeHeartbeat = nm2.nodeHeartbeat( new HashMap>(), true, -100); - Assert.assertTrue(NodeAction.RESYNC.equals(nodeHeartbeat.getNodeAction())); - Assert.assertEquals("Too far behind rm response id:0 nm response id:-100", + assertTrue(NodeAction.RESYNC.equals(nodeHeartbeat.getNodeAction())); + assertEquals("Too far behind rm response id:0 nm response id:-100", nodeHeartbeat.getDiagnosticsMessage()); checkRebootedNMCount(rm, ++initialMetricCount); } @@ -1723,8 +1713,8 @@ public void testNodeHeartbeatForAppCollectorsMap() throws Exception { statusList, null, nodeHealth, null, null, null); node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus)); - Assert.assertEquals(1, node1.getRunningApps().size()); - Assert.assertEquals(app1.getApplicationId(), node1.getRunningApps().get(0)); + assertEquals(1, node1.getRunningApps().size()); + assertEquals(app1.getApplicationId(), node1.getRunningApps().get(0)); // Create a running container for app2 running on nm2 ContainerId runningContainerId2 = BuilderUtils.newContainerId( @@ -1738,21 +1728,21 @@ public void testNodeHeartbeatForAppCollectorsMap() throws Exception { nodeStatus = NodeStatus.newInstance(nm1.getNodeId(), 0, statusList, null, nodeHealth, null, null, null); node2.handle(new RMNodeStatusEvent(nm2.getNodeId(), nodeStatus)); - Assert.assertEquals(1, node2.getRunningApps().size()); - Assert.assertEquals(app2.getApplicationId(), node2.getRunningApps().get(0)); + assertEquals(1, node2.getRunningApps().size()); + assertEquals(app2.getApplicationId(), node2.getRunningApps().get(0)); nodeHeartbeat1 = nm1.nodeHeartbeat(true); Map map1 = nodeHeartbeat1.getAppCollectors(); - Assert.assertEquals(1, map1.size()); - Assert.assertEquals(collectorAddr1, + assertEquals(1, map1.size()); + assertEquals(collectorAddr1, map1.get(app1.getApplicationId()).getCollectorAddr()); nodeHeartbeat2 = nm2.nodeHeartbeat(true); Map map2 = nodeHeartbeat2.getAppCollectors(); - Assert.assertEquals(1, map2.size()); - Assert.assertEquals(collectorAddr4, + assertEquals(1, map2.size()); + assertEquals(collectorAddr4, map2.get(app2.getApplicationId()).getCollectorAddr()); } @@ -1766,8 +1756,8 @@ private void checkRebootedNMCount(MockRM rm2, int count) wait(100); } } - Assert.assertEquals("The rebooted metrics are not updated", count, - ClusterMetrics.getMetrics().getNumRebootedNMs()); + assertEquals(count, ClusterMetrics.getMetrics().getNumRebootedNMs(), + "The rebooted metrics are not updated"); } @Test @@ -1780,7 +1770,7 @@ public void testUnhealthyNodeStatus() throws Exception { rm.start(); MockNM nm1 = rm.registerNode("host1:1234", 5120); - Assert.assertEquals(0, ClusterMetrics.getMetrics().getUnhealthyNMs()); + assertEquals(0, ClusterMetrics.getMetrics().getUnhealthyNMs()); // node healthy nm1.nodeHeartbeat(true); @@ -1804,10 +1794,10 @@ private void checkUnhealthyNMCount(MockRM rm, MockNM nm1, boolean health, wait(100); } } - Assert.assertFalse((rm.getRMContext().getRMNodes().get(nm1.getNodeId()) + assertFalse((rm.getRMContext().getRMNodes().get(nm1.getNodeId()) .getState() != NodeState.UNHEALTHY) == health); - Assert.assertEquals("Unhealthy metrics not incremented", count, - ClusterMetrics.getMetrics().getUnhealthyNMs()); + assertEquals(count, ClusterMetrics.getMetrics().getUnhealthyNMs(), + "Unhealthy metrics not incremented"); } @SuppressWarnings({ "unchecked", "rawtypes" }) @@ -1904,22 +1894,22 @@ public void handle(SchedulerEvent event) { final int expectedNMs = ClusterMetrics.getMetrics().getNumActiveNMs(); QueueMetrics metrics = rm.getResourceScheduler().getRootQueueMetrics(); // TODO Metrics incorrect in case of the FifoScheduler - Assert.assertEquals(5120, metrics.getAvailableMB()); + assertEquals(5120, metrics.getAvailableMB()); // reconnect of healthy node nm1 = rm.registerNode("host1:1234", 5120); NodeHeartbeatResponse response = nm1.nodeHeartbeat(true); - Assert.assertTrue(NodeAction.NORMAL.equals(response.getNodeAction())); + assertTrue(NodeAction.NORMAL.equals(response.getNodeAction())); rm.drainEvents(); - Assert.assertEquals(expectedNMs, ClusterMetrics.getMetrics().getNumActiveNMs()); + assertEquals(expectedNMs, ClusterMetrics.getMetrics().getNumActiveNMs()); checkUnhealthyNMCount(rm, nm2, true, 1); // reconnect of unhealthy node nm2 = rm.registerNode("host2:5678", 5120); response = nm2.nodeHeartbeat(false); - Assert.assertTrue(NodeAction.NORMAL.equals(response.getNodeAction())); + assertTrue(NodeAction.NORMAL.equals(response.getNodeAction())); rm.drainEvents(); - Assert.assertEquals(expectedNMs, ClusterMetrics.getMetrics().getNumActiveNMs()); + assertEquals(expectedNMs, ClusterMetrics.getMetrics().getNumActiveNMs()); checkUnhealthyNMCount(rm, nm2, true, 1); // unhealthy node changed back to healthy @@ -1927,14 +1917,14 @@ public void handle(SchedulerEvent event) { response = nm2.nodeHeartbeat(true); response = nm2.nodeHeartbeat(true); rm.drainEvents(); - Assert.assertEquals(5120 + 5120, metrics.getAvailableMB()); + assertEquals(5120 + 5120, metrics.getAvailableMB()); // reconnect of node with changed capability nm1 = rm.registerNode("host2:5678", 10240); response = nm1.nodeHeartbeat(true); rm.drainEvents(); - Assert.assertTrue(NodeAction.NORMAL.equals(response.getNodeAction())); - Assert.assertEquals(5120 + 10240, metrics.getAvailableMB()); + assertTrue(NodeAction.NORMAL.equals(response.getNodeAction())); + assertEquals(5120 + 10240, metrics.getAvailableMB()); // reconnect of node with changed capability and running applications List runningApps = new ArrayList(); @@ -1942,8 +1932,8 @@ public void handle(SchedulerEvent event) { nm1 = rm.registerNode("host2:5678", 15360, 2, runningApps); response = nm1.nodeHeartbeat(true); rm.drainEvents(); - Assert.assertTrue(NodeAction.NORMAL.equals(response.getNodeAction())); - Assert.assertEquals(5120 + 15360, metrics.getAvailableMB()); + assertTrue(NodeAction.NORMAL.equals(response.getNodeAction())); + assertEquals(5120 + 15360, metrics.getAvailableMB()); // reconnect healthy node changing http port nm1 = new MockNM("host1:1234", 5120, rm.getResourceTrackerService()); @@ -1953,9 +1943,9 @@ public void handle(SchedulerEvent event) { response = nm1.nodeHeartbeat(true); rm.drainEvents(); RMNode rmNode = rm.getRMContext().getRMNodes().get(nm1.getNodeId()); - Assert.assertEquals(3, rmNode.getHttpPort()); - Assert.assertEquals(5120, rmNode.getTotalCapability().getMemorySize()); - Assert.assertEquals(5120 + 15360, metrics.getAvailableMB()); + assertEquals(3, rmNode.getHttpPort()); + assertEquals(5120, rmNode.getTotalCapability().getMemorySize()); + assertEquals(5120 + 15360, metrics.getAvailableMB()); } @@ -1972,7 +1962,7 @@ public void testNMUnregistration() throws Exception { int shutdownNMsCount = ClusterMetrics.getMetrics() .getNumShutdownNMs(); NodeHeartbeatResponse nodeHeartbeat = nm1.nodeHeartbeat(true); - Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); UnRegisterNodeManagerRequest request = Records .newRecord(UnRegisterNodeManagerRequest.class); @@ -1983,7 +1973,7 @@ public void testNMUnregistration() throws Exception { // The RM should remove the node after unregistration, hence send a reboot // command. nodeHeartbeat = nm1.nodeHeartbeat(true); - Assert.assertTrue(NodeAction.RESYNC.equals(nodeHeartbeat.getNodeAction())); + assertTrue(NodeAction.RESYNC.equals(nodeHeartbeat.getNodeAction())); } @Test @@ -1995,7 +1985,7 @@ public void testUnhealthyNMUnregistration() throws Exception { ResourceTrackerService resourceTrackerService = rm .getResourceTrackerService(); MockNM nm1 = rm.registerNode("host1:1234", 5120); - Assert.assertEquals(0, ClusterMetrics.getMetrics().getUnhealthyNMs()); + assertEquals(0, ClusterMetrics.getMetrics().getUnhealthyNMs()); // node healthy nm1.nodeHeartbeat(true); int shutdownNMsCount = ClusterMetrics.getMetrics().getNumShutdownNMs(); @@ -2034,14 +2024,14 @@ public void testInvalidNMUnregistration() throws Exception { // 4. Unregister the Node Manager MockNM nm1 = new MockNM("host1:1234", 5120, resourceTrackerService); RegisterNodeManagerResponse response = nm1.registerNode(); - Assert.assertEquals(NodeAction.NORMAL, response.getNodeAction()); + assertEquals(NodeAction.NORMAL, response.getNodeAction()); int shutdownNMsCount = ClusterMetrics.getMetrics().getNumShutdownNMs(); writeToHostsFile("host2"); conf.set(YarnConfiguration.RM_NODES_INCLUDE_FILE_PATH, hostFile.getAbsolutePath()); rm.getNodesListManager().refreshNodes(conf); NodeHeartbeatResponse heartbeatResponse = nm1.nodeHeartbeat(true); - Assert.assertEquals(NodeAction.SHUTDOWN, heartbeatResponse.getNodeAction()); + assertEquals(NodeAction.SHUTDOWN, heartbeatResponse.getNodeAction()); checkDecommissionedNMCount(rm, decommisionedNMsCount); request.setNodeId(nm1.getNodeId()); resourceTrackerService.unRegisterNodeManager(request); @@ -2053,7 +2043,7 @@ public void testInvalidNMUnregistration() throws Exception { // 3. Unregister the Node Manager MockNM nm2 = new MockNM("host2:1234", 5120, resourceTrackerService); RegisterNodeManagerResponse response2 = nm2.registerNode(); - Assert.assertEquals(NodeAction.NORMAL, response2.getNodeAction()); + assertEquals(NodeAction.NORMAL, response2.getNodeAction()); writeToHostsFile("host1"); conf.set(YarnConfiguration.RM_NODES_INCLUDE_FILE_PATH, hostFile.getAbsolutePath()); @@ -2065,7 +2055,8 @@ public void testInvalidNMUnregistration() throws Exception { rm.stop(); } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testInitDecommMetric() throws Exception { testInitDecommMetricHelper(true); testInitDecommMetricHelper(false); @@ -2101,11 +2092,11 @@ public void testInitDecommMetricHelper(boolean hasIncludeList) nm1.nodeHeartbeat(true); nm2.nodeHeartbeat(true); rm1.drainEvents(); - Assert.assertEquals("Number of Decommissioned nodes should be 1", - 1, ClusterMetrics.getMetrics().getNumDecommisionedNMs()); - Assert.assertEquals("The inactiveRMNodes should contain an entry for the" + - "decommissioned node", - 1, rm1.getRMContext().getInactiveRMNodes().size()); + assertEquals(1, ClusterMetrics.getMetrics().getNumDecommisionedNMs(), + "Number of Decommissioned nodes should be 1"); + assertEquals(1, rm1.getRMContext().getInactiveRMNodes().size(), + "The inactiveRMNodes should contain an entry for the" + + "decommissioned node"); writeToHostsFile(excludeHostFile, ""); conf.set(YarnConfiguration.RM_NODES_EXCLUDE_FILE_PATH, excludeHostFile.getAbsolutePath()); @@ -2114,18 +2105,19 @@ public void testInitDecommMetricHelper(boolean hasIncludeList) nm1.nodeHeartbeat(true); nm2.nodeHeartbeat(true); rm1.drainEvents(); - Assert.assertEquals("The decommissioned nodes metric should have " + - "decremented to 0", - 0, ClusterMetrics.getMetrics().getNumDecommisionedNMs()); - Assert.assertEquals("The active nodes metric should be 2", - 2, ClusterMetrics.getMetrics().getNumActiveNMs()); - Assert.assertEquals("The inactive RMNodes entry should have been removed", - 0, rm1.getRMContext().getInactiveRMNodes().size()); + assertEquals(0, ClusterMetrics.getMetrics().getNumDecommisionedNMs(), + "The decommissioned nodes metric should have " + + "decremented to 0"); + assertEquals(2, ClusterMetrics.getMetrics().getNumActiveNMs(), + "The active nodes metric should be 2"); + assertEquals(0, rm1.getRMContext().getInactiveRMNodes().size(), + "The inactive RMNodes entry should have been removed"); rm1.drainEvents(); rm1.stop(); } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testInitDecommMetricNoRegistration() throws Exception { Configuration conf = new Configuration(); rm = new MockRM(conf); @@ -2143,16 +2135,16 @@ public void testInitDecommMetricNoRegistration() throws Exception { hostFile.getAbsolutePath()); rm.getNodesListManager().refreshNodes(conf); rm.drainEvents(); - Assert.assertEquals("The decommissioned nodes metric should be 1 ", - 1, ClusterMetrics.getMetrics().getNumDecommisionedNMs()); + assertEquals(1, ClusterMetrics.getMetrics().getNumDecommisionedNMs(), + "The decommissioned nodes metric should be 1 "); rm.stop(); MockRM rm1 = new MockRM(conf); rm1.start(); rm1.getNodesListManager().refreshNodes(conf); rm1.drainEvents(); - Assert.assertEquals("The decommissioned nodes metric should be 2 ", - 2, ClusterMetrics.getMetrics().getNumDecommisionedNMs()); + assertEquals(2, ClusterMetrics.getMetrics().getNumDecommisionedNMs(), + "The decommissioned nodes metric should be 2 "); rm1.stop(); } @@ -2175,17 +2167,17 @@ public void testIncorrectRecommission() throws Exception { rm.drainEvents(); nm1.nodeHeartbeat(true); rm.drainEvents(); - Assert.assertTrue("Node " + nm1.getNodeId().getHost() + - " should be Decommissioned", rm.getRMContext() + assertTrue(rm.getRMContext() .getInactiveRMNodes().get(nm1.getNodeId()).getState() == NodeState - .DECOMMISSIONED); + .DECOMMISSIONED, "Node " + nm1.getNodeId().getHost() + + " should be Decommissioned"); writeToHostsFile(excludeHostFile, ""); rm.getNodesListManager().refreshNodesGracefully(conf, null); rm.drainEvents(); - Assert.assertTrue("Node " + nm1.getNodeId().getHost() + - " should be Decommissioned", rm.getRMContext() + assertTrue(rm.getRMContext() .getInactiveRMNodes().get(nm1.getNodeId()).getState() == NodeState - .DECOMMISSIONED); + .DECOMMISSIONED, "Node " + nm1.getNodeId().getHost() + + " should be Decommissioned"); rm.stop(); } @@ -2238,14 +2230,14 @@ public void testNodeRemovalUtil(boolean doGraceful) throws Exception { //check all 3 nodes joined in as NORMAL NodeHeartbeatResponse nodeHeartbeat = nm1.nodeHeartbeat(true); - Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); nodeHeartbeat = nm2.nodeHeartbeat(true); - Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); nodeHeartbeat = nm3.nodeHeartbeat(true); - Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); rm.drainEvents(); - Assert.assertEquals("All 3 nodes should be active", - metrics.getNumActiveNMs(), 3); + assertEquals(metrics.getNumActiveNMs(), 3, + "All 3 nodes should be active"); //Remove nm2 from include list, should now be shutdown with timer test String ip = NetUtils.normalizeHostName("localhost"); @@ -2259,17 +2251,16 @@ public void testNodeRemovalUtil(boolean doGraceful) throws Exception { nm1.nodeHeartbeat(true); nm2.nodeHeartbeat(true); rm.drainEvents(); - Assert.assertTrue("Node should not be in active node list", - !rmContext.getRMNodes().containsKey(nm2.getNodeId())); + assertTrue(!rmContext.getRMNodes().containsKey(nm2.getNodeId()), + "Node should not be in active node list"); RMNode rmNode = rmContext.getInactiveRMNodes().get(nm2.getNodeId()); - Assert.assertEquals("Node should be in inactive node list", - rmNode.getState(), - doGraceful? NodeState.DECOMMISSIONED : NodeState.SHUTDOWN); - Assert.assertEquals("Active nodes should be 2", - metrics.getNumActiveNMs(), 2); - Assert.assertEquals("Shutdown nodes should be expected", - metrics.getNumShutdownNMs(), doGraceful? 0 : 1); + assertEquals(rmNode.getState(), + doGraceful? NodeState.DECOMMISSIONED : NodeState.SHUTDOWN, + "Node should be in inactive node list"); + assertEquals(metrics.getNumActiveNMs(), 2, "Active nodes should be 2"); + assertEquals(metrics.getNumShutdownNMs(), doGraceful? 0 : 1, + "Shutdown nodes should be expected"); int nodeRemovalTimeout = conf.getInt( @@ -2282,10 +2273,8 @@ public void testNodeRemovalUtil(boolean doGraceful) throws Exception { latch.await(maxThreadSleeptime, TimeUnit.MILLISECONDS); rmNode = rmContext.getInactiveRMNodes().get(nm2.getNodeId()); - Assert.assertEquals("Node should have been forgotten!", - rmNode, null); - Assert.assertEquals("Shutdown nodes should be 0 now", - metrics.getNumShutdownNMs(), 0); + assertEquals(rmNode, null, "Node should have been forgotten!"); + assertEquals(metrics.getNumShutdownNMs(), 0, "Shutdown nodes should be 0 now"); //Check node removal and re-addition before timer expires writeToHostsFile("host1", ip, "host2"); @@ -2299,13 +2288,11 @@ public void testNodeRemovalUtil(boolean doGraceful) throws Exception { nm2.nodeHeartbeat(true); rm.drainEvents(); rmNode = rmContext.getInactiveRMNodes().get(nm2.getNodeId()); - Assert.assertEquals("Node should be shutdown", - rmNode.getState(), - doGraceful? NodeState.DECOMMISSIONED : NodeState.SHUTDOWN); - Assert.assertEquals("Active nodes should be 2", - metrics.getNumActiveNMs(), 2); - Assert.assertEquals("Shutdown nodes should be expected", - metrics.getNumShutdownNMs(), doGraceful? 0 : 1); + assertEquals(rmNode.getState(), doGraceful? NodeState.DECOMMISSIONED : + NodeState.SHUTDOWN, "Node should be shutdown"); + assertEquals(metrics.getNumActiveNMs(), 2, "Active nodes should be 2"); + assertEquals(metrics.getNumShutdownNMs(), doGraceful? 0 : 1, + "Shutdown nodes should be expected"); //add back the node before timer expires latch.await(maxThreadSleeptime - 2000, TimeUnit.MILLISECONDS); @@ -2314,11 +2301,11 @@ public void testNodeRemovalUtil(boolean doGraceful) throws Exception { nm2 = rm.registerNode("host2:5678", 10240); nodeHeartbeat = nm2.nodeHeartbeat(true); rm.drainEvents(); - Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); - Assert.assertEquals("Shutdown nodes should be 0 now", - metrics.getNumShutdownNMs(), 0); - Assert.assertEquals("All 3 nodes should be active", - metrics.getNumActiveNMs(), 3); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); + assertEquals(metrics.getNumShutdownNMs(), 0, + "Shutdown nodes should be 0 now"); + assertEquals(metrics.getNumActiveNMs(), 3, + "All 3 nodes should be active"); //Decommission this node, check timer doesn't remove it writeToHostsFile("host1", "host2", ip); @@ -2329,23 +2316,23 @@ public void testNodeRemovalUtil(boolean doGraceful) throws Exception { rm.drainEvents(); rmNode = doGraceful ? rmContext.getRMNodes().get(nm2.getNodeId()) : rmContext.getInactiveRMNodes().get(nm2.getNodeId()); - Assert.assertTrue("Node should be DECOMMISSIONED or DECOMMISSIONING", - (rmNode.getState() == NodeState.DECOMMISSIONED) || - (rmNode.getState() == NodeState.DECOMMISSIONING)); + assertTrue((rmNode.getState() == NodeState.DECOMMISSIONED) || + (rmNode.getState() == NodeState.DECOMMISSIONING), + "Node should be DECOMMISSIONED or DECOMMISSIONING"); if (rmNode.getState() == NodeState.DECOMMISSIONED) { - Assert.assertEquals("Decommissioned/ing nodes should be 1 now", - metrics.getNumDecommisionedNMs(), 1); + assertEquals(metrics.getNumDecommisionedNMs(), 1, + "Decommissioned/ing nodes should be 1 now"); } latch.await(maxThreadSleeptime, TimeUnit.MILLISECONDS); rmNode = doGraceful ? rmContext.getRMNodes().get(nm2.getNodeId()) : rmContext.getInactiveRMNodes().get(nm2.getNodeId()); - Assert.assertTrue("Node should be DECOMMISSIONED or DECOMMISSIONING", - (rmNode.getState() == NodeState.DECOMMISSIONED) || - (rmNode.getState() == NodeState.DECOMMISSIONING)); + assertTrue((rmNode.getState() == NodeState.DECOMMISSIONED) || + (rmNode.getState() == NodeState.DECOMMISSIONING), + "Node should be DECOMMISSIONED or DECOMMISSIONING"); if (rmNode.getState() == NodeState.DECOMMISSIONED) { - Assert.assertEquals("Decommissioned/ing nodes should be 1 now", - metrics.getNumDecommisionedNMs(), 1); + assertEquals(metrics.getNumDecommisionedNMs(), 1, + "Decommissioned/ing nodes should be 1 now"); } //Test decommed/ing node that transitions to untracked,timer should remove @@ -2421,14 +2408,13 @@ private void testNodeRemovalUtilLost(boolean doGraceful) throws Exception { rm.drainEvents(); //check all 3 nodes joined in as NORMAL NodeHeartbeatResponse nodeHeartbeat = nm1.nodeHeartbeat(true); - Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); nodeHeartbeat = nm2.nodeHeartbeat(true); - Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); nodeHeartbeat = nm3.nodeHeartbeat(true); - Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); rm.drainEvents(); - Assert.assertEquals("All 3 nodes should be active", - metrics.getNumActiveNMs(), 3); + assertEquals(metrics.getNumActiveNMs(), 3, "All 3 nodes should be active"); int waitCount = 0; while(waitCount++ < 20){ synchronized (this) { @@ -2437,15 +2423,12 @@ private void testNodeRemovalUtilLost(boolean doGraceful) throws Exception { nm3.nodeHeartbeat(true); nm1.nodeHeartbeat(true); } - Assert.assertNotEquals("host2 should be a lost NM!", - rmContext.getInactiveRMNodes().get(nm2.getNodeId()), null); - Assert.assertEquals("host2 should be a lost NM!", - rmContext.getInactiveRMNodes().get(nm2.getNodeId()).getState(), - NodeState.LOST); - Assert.assertEquals("There should be 1 Lost NM!", - clusterMetrics.getNumLostNMs(), 1); - Assert.assertEquals("There should be 2 Active NM!", - clusterMetrics.getNumActiveNMs(), 2); + assertNotEquals(rmContext.getInactiveRMNodes().get(nm2.getNodeId()), + null, "host2 should be a lost NM!"); + assertEquals(rmContext.getInactiveRMNodes().get(nm2.getNodeId()).getState(), + NodeState.LOST, "host2 should be a lost NM!"); + assertEquals(clusterMetrics.getNumLostNMs(), 1, "There should be 1 Lost NM!"); + assertEquals(clusterMetrics.getNumActiveNMs(), 2, "There should be 2 Active NM!"); int nodeRemovalTimeout = conf.getInt( YarnConfiguration.RM_NODEMANAGER_UNTRACKED_REMOVAL_TIMEOUT_MSEC, @@ -2469,12 +2452,12 @@ private void testNodeRemovalUtilLost(boolean doGraceful) throws Exception { nm2.nodeHeartbeat(true); } } - Assert.assertEquals("host2 should have been forgotten!", - rmContext.getInactiveRMNodes().get(nm2.getNodeId()), null); - Assert.assertEquals("There should be no Lost NMs!", - clusterMetrics.getNumLostNMs(), 0); - Assert.assertEquals("There should be 2 Active NM!", - clusterMetrics.getNumActiveNMs(), 2); + assertEquals(rmContext.getInactiveRMNodes().get(nm2.getNodeId()), null, + "host2 should have been forgotten!"); + assertEquals(clusterMetrics.getNumLostNMs(), 0, + "There should be no Lost NMs!"); + assertEquals(clusterMetrics.getNumActiveNMs(), 2, + "There should be 2 Active NM!"); rm.stop(); } @@ -2507,15 +2490,14 @@ private void testNodeRemovalUtilRebooted(boolean doGraceful) rm.drainEvents(); rm.drainEvents(); - Assert.assertNotEquals("host2 should be a rebooted NM!", - rmContext.getInactiveRMNodes().get(nm2.getNodeId()), null); - Assert.assertEquals("host2 should be a rebooted NM!", - rmContext.getInactiveRMNodes().get(nm2.getNodeId()).getState(), - NodeState.REBOOTED); - Assert.assertEquals("There should be 1 Rebooted NM!", - clusterMetrics.getNumRebootedNMs(), 1); - Assert.assertEquals("There should be 2 Active NM!", - clusterMetrics.getNumActiveNMs(), 2); + assertNotEquals(rmContext.getInactiveRMNodes().get(nm2.getNodeId()), null, + "host2 should be a rebooted NM!"); + assertEquals(rmContext.getInactiveRMNodes().get(nm2.getNodeId()).getState(), + NodeState.REBOOTED, "host2 should be a rebooted NM!"); + assertEquals(clusterMetrics.getNumRebootedNMs(), 1, + "There should be 1 Rebooted NM!"); + assertEquals(clusterMetrics.getNumActiveNMs(), 2, + "There should be 2 Active NM!"); int nodeRemovalTimeout = conf.getInt( @@ -2539,12 +2521,12 @@ private void testNodeRemovalUtilRebooted(boolean doGraceful) wait(maxThreadSleeptime); } } - Assert.assertEquals("host2 should have been forgotten!", - rmContext.getInactiveRMNodes().get(nm2.getNodeId()), null); - Assert.assertEquals("There should be no Rebooted NMs!", - clusterMetrics.getNumRebootedNMs(), 0); - Assert.assertEquals("There should be 2 Active NM!", - clusterMetrics.getNumActiveNMs(), 2); + assertEquals(rmContext.getInactiveRMNodes().get(nm2.getNodeId()), null, + "host2 should have been forgotten!"); + assertEquals(clusterMetrics.getNumRebootedNMs(), 0, + "There should be no Rebooted NMs!"); + assertEquals(clusterMetrics.getNumActiveNMs(), 2, + "There should be 2 Active NM!"); rm.stop(); } @@ -2575,14 +2557,13 @@ private void testNodeRemovalUtilUnhealthy(boolean doGraceful) rm.drainEvents(); //check all 3 nodes joined in as NORMAL NodeHeartbeatResponse nodeHeartbeat = nm1.nodeHeartbeat(true); - Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); nodeHeartbeat = nm2.nodeHeartbeat(true); - Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); nodeHeartbeat = nm3.nodeHeartbeat(true); - Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); + assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction())); rm.drainEvents(); - Assert.assertEquals("All 3 nodes should be active", - metrics.getNumActiveNMs(), 3); + assertEquals(metrics.getNumActiveNMs(), 3, "All 3 nodes should be active"); // node healthy nm1.nodeHeartbeat(true); nm2.nodeHeartbeat(false); @@ -2596,20 +2577,19 @@ private void testNodeRemovalUtilUnhealthy(boolean doGraceful) nm3.nodeHeartbeat(true); rm.drainEvents(); if (!doGraceful) { - Assert.assertNotEquals("host2 should be a shutdown NM!", - rmContext.getInactiveRMNodes().get(nm2.getNodeId()), null); - Assert.assertEquals("host2 should be a shutdown NM!", - rmContext.getInactiveRMNodes().get(nm2.getNodeId()).getState(), - NodeState.SHUTDOWN); + assertNotEquals(rmContext.getInactiveRMNodes().get(nm2.getNodeId()), null, + "host2 should be a shutdown NM!"); + assertEquals(rmContext.getInactiveRMNodes().get(nm2.getNodeId()).getState(), + NodeState.SHUTDOWN, "host2 should be a shutdown NM!"); } - Assert.assertEquals("There should be 2 Active NM!", - clusterMetrics.getNumActiveNMs(), 2); + assertEquals(clusterMetrics.getNumActiveNMs(), 2, + "There should be 2 Active NM!"); if (!doGraceful) { - Assert.assertEquals("There should be 1 Shutdown NM!", - clusterMetrics.getNumShutdownNMs(), 1); + assertEquals(clusterMetrics.getNumShutdownNMs(), 1, + "There should be 1 Shutdown NM!"); } - Assert.assertEquals("There should be 0 Unhealthy NM!", - clusterMetrics.getUnhealthyNMs(), 0); + assertEquals(clusterMetrics.getUnhealthyNMs(), 0, + "There should be 0 Unhealthy NM!"); int nodeRemovalTimeout = conf.getInt( YarnConfiguration.RM_NODEMANAGER_UNTRACKED_REMOVAL_TIMEOUT_MSEC, @@ -2625,12 +2605,11 @@ private void testNodeRemovalUtilUnhealthy(boolean doGraceful) wait(maxThreadSleeptime); } } - Assert.assertEquals("host2 should have been forgotten!", - rmContext.getInactiveRMNodes().get(nm2.getNodeId()), null); - Assert.assertEquals("There should be no Shutdown NMs!", - clusterMetrics.getNumRebootedNMs(), 0); - Assert.assertEquals("There should be 2 Active NM!", - clusterMetrics.getNumActiveNMs(), 2); + assertEquals(rmContext.getInactiveRMNodes().get(nm2.getNodeId()), null, + "host2 should have been forgotten!"); + assertEquals(clusterMetrics.getNumRebootedNMs(), 0, + "There should be no Shutdown NMs!"); + assertEquals(clusterMetrics.getNumActiveNMs(), 2, "There should be 2 Active NM!"); rm.stop(); } @@ -2699,10 +2678,11 @@ private void checkDecommissionedNMCount(MockRM rm, int count) wait(100); } } - Assert.assertEquals(count, ClusterMetrics.getMetrics() + assertEquals(count, ClusterMetrics.getMetrics() .getNumDecommisionedNMs()); - Assert.assertEquals("The decommisioned metrics are not updated", count, - ClusterMetrics.getMetrics().getNumDecommisionedNMs()); + assertEquals(count, + ClusterMetrics.getMetrics().getNumDecommisionedNMs(), + "The decommisioned metrics are not updated"); } private void checkShutdownNMCount(MockRM rm, int count) @@ -2714,11 +2694,12 @@ private void checkShutdownNMCount(MockRM rm, int count) wait(100); } } - Assert.assertEquals("The shutdown metrics are not updated", count, - ClusterMetrics.getMetrics().getNumShutdownNMs()); + assertEquals(count, + ClusterMetrics.getMetrics().getNumShutdownNMs(), + "The shutdown metrics are not updated"); } - @After + @AfterEach public void tearDown() { if (hostFile != null && hostFile.exists()) { hostFile.delete(); @@ -2770,10 +2751,10 @@ protected Dispatcher createDispatcher() { } Resource currentConsumption = applicationAttempt.getCurrentConsumption(); - Assert.assertEquals(Resource.newInstance(0, 0), currentConsumption); + assertEquals(Resource.newInstance(0, 0), currentConsumption); Resource allocResources = applicationAttempt.getQueue().getMetrics().getAllocatedResources(); - Assert.assertEquals(Resource.newInstance(0, 0), allocResources); + assertEquals(Resource.newInstance(0, 0), allocResources); RegisterNodeManagerRequest req = Records.newRecord( RegisterNodeManagerRequest.class); @@ -2812,15 +2793,15 @@ protected Dispatcher createDispatcher() { dispatcher.await(); Thread.sleep(2000); dispatcher.await(); - Assert.assertEquals(NodeAction.NORMAL, response.getNodeAction()); + assertEquals(NodeAction.NORMAL, response.getNodeAction()); Collection liveContainers = applicationAttempt .getLiveContainers(); - Assert.assertEquals(3, liveContainers.size()); + assertEquals(3, liveContainers.size()); Iterator iter = liveContainers.iterator(); while (iter.hasNext()) { RMContainer rc = iter.next(); - Assert.assertEquals( + assertEquals( rc.getContainerId().equals(c3) ? ExecutionType.GUARANTEED : ExecutionType.OPPORTUNISTIC, rc.getExecutionType()); @@ -2828,19 +2809,20 @@ protected Dispatcher createDispatcher() { // Should only include GUARANTEED resources currentConsumption = applicationAttempt.getCurrentConsumption(); - Assert.assertEquals(Resource.newInstance(2048, 1), currentConsumption); + assertEquals(Resource.newInstance(2048, 1), currentConsumption); allocResources = applicationAttempt.getQueue().getMetrics().getAllocatedResources(); - Assert.assertEquals(Resource.newInstance(2048, 1), allocResources); + assertEquals(Resource.newInstance(2048, 1), allocResources); SchedulerNode schedulerNode = rm.getRMContext().getScheduler().getSchedulerNode(nodeId); - Assert.assertNotNull(schedulerNode); + assertNotNull(schedulerNode); Resource nodeResources = schedulerNode.getAllocatedResource(); - Assert.assertEquals(Resource.newInstance(2048, 1), nodeResources); + assertEquals(Resource.newInstance(2048, 1), nodeResources); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testNodeHeartBeatResponseForUnknownContainerCleanUp() throws Exception { Configuration conf = new Configuration(); @@ -2902,7 +2884,7 @@ public void testResponseIdOverflow() throws Exception { MockNM nm1 = rm.registerNode("host1:1234", 5120); NodeHeartbeatResponse nodeHeartbeat = nm1.nodeHeartbeat(true); - Assert.assertEquals(NodeAction.NORMAL, nodeHeartbeat.getNodeAction()); + assertEquals(NodeAction.NORMAL, nodeHeartbeat.getNodeAction()); // prepare the responseId that's about to overflow RMNode node = rm.getRMContext().getRMNodes().get(nm1.getNodeId()); @@ -2912,12 +2894,12 @@ public void testResponseIdOverflow() throws Exception { // heartbeat twice and check responseId nodeHeartbeat = nm1.nodeHeartbeat(true); - Assert.assertEquals(NodeAction.NORMAL, nodeHeartbeat.getNodeAction()); - Assert.assertEquals(0, nodeHeartbeat.getResponseId()); + assertEquals(NodeAction.NORMAL, nodeHeartbeat.getNodeAction()); + assertEquals(0, nodeHeartbeat.getResponseId()); nodeHeartbeat = nm1.nodeHeartbeat(true); - Assert.assertEquals(NodeAction.NORMAL, nodeHeartbeat.getNodeAction()); - Assert.assertEquals(1, nodeHeartbeat.getResponseId()); + assertEquals(NodeAction.NORMAL, nodeHeartbeat.getNodeAction()); + assertEquals(1, nodeHeartbeat.getResponseId()); } @Test @@ -2945,11 +2927,11 @@ protected ResourceTrackerService createResourceTrackerService() { NodeId.newInstance("host1" + System.currentTimeMillis(), 1234), 1236, Resource.newInstance(10000, 10), "2", new ArrayList<>(), new ArrayList<>())); - Assert - .assertEquals("Shutdown signal should be received", NodeAction.SHUTDOWN, - response.getNodeAction()); - Assert.assertTrue("Diagnostic Message", response.getDiagnosticsMessage() - .contains("hostname cannot be resolved ")); + + assertEquals(NodeAction.SHUTDOWN, + response.getNodeAction(), "Shutdown signal should be received"); + assertTrue(response.getDiagnosticsMessage() + .contains("hostname cannot be resolved "), "Diagnostic Message"); // Test success rmTracker = ServerRMProxy.createRMProxy(mockRM.getConfig(), ResourceTracker.class); @@ -2957,8 +2939,8 @@ protected ResourceTrackerService createResourceTrackerService() { .newInstance(NodeId.newInstance("localhost", 1234), 1236, Resource.newInstance(10000, 10), "2", new ArrayList<>(), new ArrayList<>())); - Assert.assertEquals("Successfull registration", NodeAction.NORMAL, - response.getNodeAction()); + assertEquals(NodeAction.NORMAL, + response.getNodeAction(), "Successfull registration"); mockRM.stop(); } @@ -3004,7 +2986,8 @@ public void close() { } } - @Test(timeout = 5000) + @Test + @Timeout(value = 5) public void testSystemCredentialsAfterTokenSequenceNoChange() throws Exception { @@ -3340,20 +3323,20 @@ public void testMarkUnregisteredNodesAsLost() throws Exception { TimeUnit.MILLISECONDS.sleep(50); // Allow some time for event processing // Step 5: Validate that test_host3 is marked as a LOST node - Assert.assertNotNull(clusterMetrics); // Ensure metrics are not null - assertEquals("test_host3 should be a lost NM!", - NodeState.LOST, + assertNotNull(clusterMetrics); // Ensure metrics are not null + assertEquals(NodeState.LOST, rmContext.getInactiveRMNodes().get( - rm.getNodesListManager().createLostNodeId("test_host3")).getState()); + rm.getNodesListManager().createLostNodeId("test_host3")).getState(), + "test_host3 should be a lost NM!"); // Step 6: Validate node metrics for lost, active, and decommissioned nodes // Two nodes are lost - assertEquals("There should be 2 Lost NM!", 2, clusterMetrics.getNumLostNMs()); + assertEquals(2, clusterMetrics.getNumLostNMs(), "There should be 2 Lost NM!"); // One node is active - assertEquals("There should be 1 Active NM!", 1, clusterMetrics.getNumActiveNMs()); + assertEquals(1, clusterMetrics.getNumActiveNMs(), "There should be 1 Active NM!"); // One node is decommissioned - assertEquals("There should be 1 Decommissioned NM!", 1, - clusterMetrics.getNumDecommisionedNMs()); + assertEquals(1, clusterMetrics.getNumDecommisionedNMs(), + "There should be 1 Decommissioned NM!"); // Step 7: Register and simulate node activity for "test_host3" MockNM nm3 = rm.registerNode("test_host3:5678", 10240); // Register test_host3 with 10240MB @@ -3361,9 +3344,9 @@ public void testMarkUnregisteredNodesAsLost() throws Exception { TimeUnit.MILLISECONDS.sleep(50); // Allow some time for event dispatch and processing // Step 8: Validate updated node metrics after registering test_host3 - assertEquals("There should be 1 Lost NM!", 1, - clusterMetrics.getNumLostNMs()); // Only one node is lost now - assertEquals("There should be 2 Active NM!", 2, - clusterMetrics.getNumActiveNMs()); // Two nodes are now active + assertEquals(1, clusterMetrics.getNumLostNMs(), + "There should be 1 Lost NM!"); // Only one node is lost now + assertEquals(2, clusterMetrics.getNumActiveNMs(), + "There should be 2 Active NM!"); // Two nodes are now active } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestSignalContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestSignalContainer.java index 9f03a7226f971..842c6096d729d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestSignalContainer.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestSignalContainer.java @@ -18,12 +18,13 @@ package org.apache.hadoop.yarn.server.resourcemanager; +import static org.junit.jupiter.api.Assertions.assertEquals; + import java.util.ArrayList; import java.util.List; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; -import org.junit.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,7 +39,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestSignalContainer { @@ -85,7 +86,7 @@ public void testSignalRequestDeliveryToNM() throws Exception { nm1.nodeHeartbeat(true); } } - Assert.assertEquals(request, conts.size()); + assertEquals(request, conts.size()); for(Container container : conts) { rm.signalToContainer(container.getId(), @@ -107,7 +108,7 @@ public void testSignalRequestDeliveryToNM() throws Exception { } // Verify NM receives the expected number of signal container requests. - Assert.assertEquals(request, signaledConts); + assertEquals(request, signaledConts); am.unregisterAppAttempt(); nm1.nodeHeartbeat(attempt.getAppAttemptId(), 1, ContainerState.COMPLETE); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestTokenClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestTokenClientRMService.java index 3e1ede9cd4681..78c3957523cbc 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestTokenClientRMService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestTokenClientRMService.java @@ -18,6 +18,8 @@ package org.apache.hadoop.yarn.server.resourcemanager; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -39,10 +41,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager; import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.Records; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; public class TestTokenClientRMService { @@ -69,7 +70,7 @@ public class TestTokenClientRMService { private static final UserGroupInformation otherKerb = UserGroupInformation .createRemoteUser(otherPrincipal, AuthMethod.KERBEROS); - @BeforeClass + @BeforeAll public static void setupSecretManager() throws IOException { ResourceManager rm = mock(ResourceManager.class); RMContext rmContext = mock(RMContext.class); @@ -88,7 +89,7 @@ public static void setupSecretManager() throws IOException { .setAuthenticationMethod(AuthenticationMethod.KERBEROS); } - @AfterClass + @AfterAll public static void teardownSecretManager() { if (dtsm != null) { dtsm.stopThreads(); @@ -128,9 +129,9 @@ public Void run() throws Exception { checkTokenRenewal(owner, other); return null; } catch (YarnException ex) { - Assert.assertTrue(ex.getMessage().contains( + assertTrue(ex.getMessage().contains( owner.getUserName() + " tries to renew a token")); - Assert.assertTrue(ex.getMessage().contains( + assertTrue(ex.getMessage().contains( "with non-matching renewer " + other.getUserName())); throw ex; } @@ -139,7 +140,7 @@ public Void run() throws Exception { } catch (Exception e) { return; } - Assert.fail("renew should have failed"); + fail("renew should have failed"); } @Test @@ -219,12 +220,12 @@ public void testTokenCancellationByWrongUser() { public Void run() throws Exception { try { checkTokenCancellation(rmService, tokOwner, tokRenewer); - Assert.fail("We should not reach here; token owner = " + fail("We should not reach here; token owner = " + tokOwner.getUserName() + ", renewer = " + tokRenewer.getUserName()); return null; } catch (YarnException e) { - Assert.assertTrue(e.getMessage().contains( + assertTrue(e.getMessage().contains( testerKerb.getUserName() + " is not authorized to cancel the token")); return null; @@ -232,7 +233,7 @@ public Void run() throws Exception { } }); } catch (Exception e) { - Assert.fail("Unexpected exception; " + e.getMessage()); + fail("Unexpected exception; " + e.getMessage()); } } } @@ -249,12 +250,12 @@ public Void run() throws Exception { public Void run() throws Exception { try { checkTokenCancellation(tokOwner, tokRenewer); - Assert.fail("We should not reach here; token owner = " + fail("We should not reach here; token owner = " + tokOwner.getUserName() + ", renewer = " + tokRenewer.getUserName()); return null; } catch (YarnException ex) { - Assert.assertTrue(ex.getMessage().contains( + assertTrue(ex.getMessage().contains( tester.getUserName() + " is not authorized to cancel the token")); return null; @@ -262,7 +263,7 @@ public Void run() throws Exception { } }); } catch (Exception e) { - Assert.fail("Unexpected exception; " + e.getMessage()); + fail("Unexpected exception; " + e.getMessage()); } } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithCSRFFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithCSRFFilter.java index 0f827a7e8899a..ef17057e89ea4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithCSRFFilter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithCSRFFilter.java @@ -36,8 +36,8 @@ import org.glassfish.jersey.jettison.JettisonFeature; import org.glassfish.jersey.server.ResourceConfig; import org.glassfish.jersey.test.TestProperties; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.w3c.dom.Document; import org.w3c.dom.Element; import org.w3c.dom.NodeList; @@ -55,8 +55,8 @@ import java.util.HashMap; import java.util.Map; -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 static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -108,7 +108,7 @@ protected void configure() { } } - @Before + @BeforeEach @Override public void setUp() throws Exception { super.setUp(); @@ -137,8 +137,8 @@ public void testIncludeCustomHeaderFromBrowser() throws Exception { .header(RestCsrfPreventionFilter.HEADER_USER_AGENT,"Mozilla/5.0") .header("X-XSRF-HEADER", "") .get(Response.class); - assertTrue("Should have been accepted", response.getStatus() == - Response.Status.OK.getStatusCode()); + assertTrue(response.getStatus() == + Response.Status.OK.getStatusCode(), "Should have been accepted"); assertEquals(MediaType.APPLICATION_XML_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); String xml = response.readEntity(String.class); @@ -152,8 +152,8 @@ public void testAllowedMethod() throws Exception { .path("info").request("application/xml") .header(RestCsrfPreventionFilter.HEADER_USER_AGENT,"Mozilla/5.0") .head(); - assertTrue("Should have been allowed", response.getStatus() == - Response.Status.OK.getStatusCode()); + assertTrue(response.getStatus() == + Response.Status.OK.getStatusCode(), "Should have been allowed"); } @Test @@ -162,8 +162,8 @@ public void testAllowNonBrowserInteractionWithoutHeader() throws Exception { Response response = r.path("ws").path("v1").path("cluster") .path("info").request("application/xml") .get(Response.class); - assertTrue("Should have been accepted", response.getStatus() == - Response.Status.OK.getStatusCode()); + assertTrue(response.getStatus() == + Response.Status.OK.getStatusCode(), "Should have been accepted"); assertEquals(MediaType.APPLICATION_XML_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); String xml = response.readEntity(String.class); @@ -177,7 +177,7 @@ public void verifyClusterInfoXML(String xml) throws Exception { is.setCharacterStream(new StringReader(xml)); Document dom = db.parse(is); NodeList nodes = dom.getElementsByTagName("clusterInfo"); - assertEquals("incorrect number of elements", 1, nodes.getLength()); + assertEquals(1, nodes.getLength(), "incorrect number of elements"); for (int i = 0; i < nodes.getLength(); i++) { Element element = (Element) nodes.item(i); @@ -209,14 +209,14 @@ public void verifyClusterGeneric(long clusterid, long startedon, String resourceManagerBuildVersion, String resourceManagerVersion) { - assertEquals("clusterId doesn't match: ", - ResourceManager.getClusterTimeStamp(), clusterid); - assertEquals("startedOn doesn't match: ", - ResourceManager.getClusterTimeStamp(), startedon); - assertTrue("stated doesn't match: " + state, - state.matches(STATE.INITED.toString())); - assertTrue("HA state doesn't match: " + haState, - haState.matches("INITIALIZING")); + assertEquals(ResourceManager.getClusterTimeStamp(), clusterid, + "clusterId doesn't match: "); + assertEquals(ResourceManager.getClusterTimeStamp(), startedon, + "startedOn doesn't match: "); + assertTrue(state.matches(STATE.INITED.toString()), + "stated doesn't match: " + state); + assertTrue(haState.matches("INITIALIZING"), + "HA state doesn't match: " + haState); WebServicesTestUtils.checkStringMatch("hadoopVersionBuiltOn", VersionInfo.getDate(), hadoopVersionBuiltOn); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithXFSFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithXFSFilter.java index 2054754760d71..4a671f2376fb4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithXFSFilter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithXFSFilter.java @@ -18,15 +18,18 @@ package org.apache.hadoop.yarn.webapp; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.http.HttpServer2; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.MockRM; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.net.HttpURLConnection; @@ -62,7 +65,7 @@ public void testXFrameOptionsDefaultBehaviour() throws Exception { URL url = new URL("http://localhost:8088/ws/v1/cluster/info"); HttpURLConnection conn = (HttpURLConnection) url.openConnection(); String xfoHeader = conn.getHeaderField("X-FRAME-OPTIONS"); - Assert.assertTrue(xfoHeader.endsWith(HttpServer2.XFrameOption + assertTrue(xfoHeader.endsWith(HttpServer2.XFrameOption .SAMEORIGIN.toString())); } @@ -74,7 +77,7 @@ public void testXFrameOptionsExplicitlyEnabled() throws Exception { URL url = new URL("http://localhost:8088/ws/v1/cluster/info"); HttpURLConnection conn = (HttpURLConnection) url.openConnection(); String xfoHeader = conn.getHeaderField("X-FRAME-OPTIONS"); - Assert.assertTrue(xfoHeader.endsWith(HttpServer2.XFrameOption + assertTrue(xfoHeader.endsWith(HttpServer2.XFrameOption .SAMEORIGIN.toString())); } @@ -86,7 +89,7 @@ public void testXFrameOptionsEnabledDefaultApps() throws Exception { URL url = new URL("http://localhost:8088/logs"); HttpURLConnection conn = (HttpURLConnection) url.openConnection(); String xfoHeader = conn.getHeaderField("X-FRAME-OPTIONS"); - Assert.assertTrue(xfoHeader.endsWith(HttpServer2.XFrameOption + assertTrue(xfoHeader.endsWith(HttpServer2.XFrameOption .SAMEORIGIN.toString())); } @@ -97,17 +100,17 @@ public void testXFrameOptionsDisabled() throws Exception { URL url = new URL("http://localhost:8088/ws/v1/cluster/info"); HttpURLConnection conn = (HttpURLConnection) url.openConnection(); String xfoHeader = conn.getHeaderField("X-FRAME-OPTIONS"); - Assert.assertNull("Unexpected X-FRAME-OPTION in header", xfoHeader); + assertNull(xfoHeader, "Unexpected X-FRAME-OPTION in header"); } @Test public void testXFrameOptionsIllegalOption() { - IllegalArgumentException e = Assert.assertThrows( + IllegalArgumentException e = assertThrows( IllegalArgumentException.class, () -> createMockRm(true, "otherValue")); } - @After + @AfterEach public void tearDown() throws IOException { rm.close(); }