diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/RMHATestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/RMHATestBase.java index c4a6e9dad64f8..6db700178c7b5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/RMHATestBase.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/RMHATestBase.java @@ -40,9 +40,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; + +import static org.junit.jupiter.api.Assertions.assertTrue; public abstract class RMHATestBase extends ClientBaseWithFixes{ @@ -57,7 +58,7 @@ public abstract class RMHATestBase extends ClientBaseWithFixes{ Configuration confForRM1; Configuration confForRM2; - @Before + @BeforeEach public void setup() throws Exception { configuration.setBoolean(YarnConfiguration.RM_HA_ENABLED, true); configuration.set(YarnConfiguration.RM_HA_IDS, "rm1,rm2"); @@ -83,7 +84,7 @@ public void setup() throws Exception { confForRM2.set(YarnConfiguration.RM_HA_ID, "rm2"); } - @After + @AfterEach public void teardown() { if (rm1 != null) { rm1.stop(); @@ -197,9 +198,9 @@ protected boolean isFinalState(RMAppState state) { protected void explicitFailover() throws IOException { 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); } @@ -207,16 +208,16 @@ protected void startRMs(MockRM rm1, Configuration confForRM1, MockRM rm2, Configuration confForRM2) throws IOException { rm1.init(confForRM1); rm1.start(); - Assert.assertTrue(rm1.getRMContext().getHAServiceState() + assertTrue(rm1.getRMContext().getHAServiceState() == HAServiceState.STANDBY); rm2.init(confForRM2); 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); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestKillApplicationWithRMHA.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestKillApplicationWithRMHA.java index e0f9bf57cc0d9..2983da5400a6e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestKillApplicationWithRMHA.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestKillApplicationWithRMHA.java @@ -19,7 +19,9 @@ package org.apache.hadoop.yarn.server.resourcemanager; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import java.io.IOException; @@ -41,8 +43,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager; import org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; public class TestKillApplicationWithRMHA extends RMHATestBase{ @@ -50,7 +52,8 @@ public class TestKillApplicationWithRMHA extends RMHATestBase{ public static final Logger LOG = LoggerFactory .getLogger(TestKillApplicationWithRMHA.class); - @Test (timeout = 20000) + @Test + @Timeout(value = 20) public void testKillAppWhenFailoverHappensAtNewState() throws Exception { // create a customized RMAppManager @@ -92,12 +95,13 @@ public void testKillAppWhenFailoverHappensAtNewState() failOverAndKillApp(app0.getApplicationId(), RMAppState.NEW); fail("Should get an exception here"); } catch (ApplicationNotFoundException ex) { - Assert.assertTrue(ex.getMessage().contains( + assertTrue(ex.getMessage().contains( "Trying to kill an absent application " + app0.getApplicationId())); } } - @Test (timeout = 20000) + @Test + @Timeout(value = 20) public void testKillAppWhenFailoverHappensAtRunningState() throws Exception { startRMs(); @@ -120,7 +124,8 @@ public void testKillAppWhenFailoverHappensAtRunningState() RMAppAttemptState.RUNNING, RMAppState.ACCEPTED); } - @Test (timeout = 20000) + @Test + @Timeout(value = 20) public void testKillAppWhenFailoverHappensAtFinalState() throws Exception { startRMs(); @@ -148,7 +153,8 @@ public void testKillAppWhenFailoverHappensAtFinalState() RMAppAttemptState.KILLED, RMAppState.KILLED); } - @Test (timeout = 20000) + @Test + @Timeout(value = 20) public void testKillAppWhenFailOverHappensDuringApplicationKill() throws Exception { // create a customized ClientRMService @@ -182,22 +188,22 @@ private void failOverAndKillApp(ApplicationId appId, ApplicationAttemptId appAttemptId, RMAppState initialRMAppState, RMAppAttemptState initialRMAppAttemptState, RMAppState expectedAppStateBeforeKillApp) throws Exception { - Assert.assertEquals(initialRMAppState, + assertEquals(initialRMAppState, rm1.getRMContext().getRMApps().get(appId).getState()); - Assert.assertEquals(initialRMAppAttemptState, rm1.getRMContext() + assertEquals(initialRMAppAttemptState, rm1.getRMContext() .getRMApps().get(appId).getAppAttempts().get(appAttemptId).getState()); explicitFailover(); - Assert.assertEquals(expectedAppStateBeforeKillApp, + assertEquals(expectedAppStateBeforeKillApp, rm2.getRMContext().getRMApps().get(appId).getState()); killApplication(rm2, appId, appAttemptId, initialRMAppState); } private void failOverAndKillApp(ApplicationId appId, RMAppState initialRMAppState) throws Exception { - Assert.assertEquals(initialRMAppState, + assertEquals(initialRMAppState, rm1.getRMContext().getRMApps().get(appId).getState()); explicitFailover(); - Assert.assertTrue(rm2.getRMContext().getRMApps().get(appId) == null); + assertTrue(rm2.getRMContext().getRMApps().get(appId) == null); killApplication(rm2, appId, null, initialRMAppState); } @@ -259,8 +265,7 @@ private void killApplication(MockRM rm, ApplicationId appId, ApplicationAttemptId appAttemptId, RMAppState rmAppState) throws Exception { KillApplicationResponse response = rm.killApp(appId); - Assert - .assertTrue(response.getIsKillCompleted() == isFinalState(rmAppState)); + assertTrue(response.getIsKillCompleted() == isFinalState(rmAppState)); RMApp loadedApp0 = rm.getRMContext().getRMApps().get(appId); rm.waitForState(appId, RMAppState.KILLED); @@ -268,6 +273,6 @@ private void killApplication(MockRM rm, ApplicationId appId, rm.waitForState(appAttemptId, RMAppAttemptState.KILLED); } // no new attempt is created. - Assert.assertEquals(1, loadedApp0.getAppAttempts().size()); + assertEquals(1, loadedApp0.getAppAttempts().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/TestRMEmbeddedElector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMEmbeddedElector.java index 1781fed69a091..9b4fca78b08d0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMEmbeddedElector.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMEmbeddedElector.java @@ -25,16 +25,17 @@ import org.apache.hadoop.service.ServiceStateException; import org.apache.hadoop.test.GenericTestUtils; 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.junit.jupiter.api.Timeout; import java.io.IOException; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import static org.junit.Assert.fail; +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.doAnswer; import static org.mockito.Mockito.mock; @@ -65,7 +66,7 @@ private enum SyncTestType { STANDBY_TIMING } - @Before + @BeforeEach public void setup() throws IOException { conf = new YarnConfiguration(); conf.setBoolean(YarnConfiguration.RM_HA_ENABLED, true); @@ -94,7 +95,8 @@ public void setup() throws IOException { * * The test times out if there is a deadlock. */ - @Test (timeout = 10000) + @Test + @Timeout(value = 10) public void testDeadlockShutdownBecomeActive() throws InterruptedException { MockRM rm = new MockRMWithElector(conf, 1000); rm.start(); @@ -320,9 +322,9 @@ public void testFailureToConnectToZookeeper() throws Exception { new ActiveStandbyElectorBasedElectorService(rm); try { ees.init(myConf); - Assert.fail("expect failure to connect to Zookeeper"); + fail("expect failure to connect to Zookeeper"); } catch (ServiceStateException sse) { - Assert.assertTrue(sse.getMessage().contains("ConnectionLoss")); + assertTrue(sse.getMessage().contains("ConnectionLoss")); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHAForAsyncScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHAForAsyncScheduler.java index 1da9a61d1dcdf..4895987be4c5a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHAForAsyncScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHAForAsyncScheduler.java @@ -32,18 +32,22 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestCapacitySchedulerAsyncScheduling; import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; -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.Arrays; import java.util.List; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + public class TestRMHAForAsyncScheduler extends RMHATestBase { private TestCapacitySchedulerAsyncScheduling.NMHeartbeatThread nmHeartbeatThread = null; - @Before + @BeforeEach @Override public void setup() throws Exception { super.setup(); @@ -82,7 +86,8 @@ private void pauseNMHeartbeat() { } } - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testAsyncScheduleThreadStateAfterRMHATransit() throws Exception { // start two RMs, and transit rm1 to active, rm2 to standby startRMs(); @@ -114,9 +119,9 @@ public void testAsyncScheduleThreadStateAfterRMHATransit() throws Exception { HAServiceProtocol.RequestSource.REQUEST_BY_USER); rm2.adminService.transitionToStandby(requestInfo); rm1.adminService.transitionToActive(requestInfo); - Assert.assertTrue(rm2.getRMContext().getHAServiceState() + assertTrue(rm2.getRMContext().getHAServiceState() == HAServiceProtocol.HAServiceState.STANDBY); - Assert.assertTrue(rm1.getRMContext().getHAServiceState() + assertTrue(rm1.getRMContext().getHAServiceState() == HAServiceProtocol.HAServiceState.ACTIVE); // check async schedule threads checkAsyncSchedulerThreads(Thread.currentThread()); @@ -136,7 +141,8 @@ public void testAsyncScheduleThreadStateAfterRMHATransit() throws Exception { rm2.stop(); } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testAsyncScheduleThreadExit() throws Exception { // start two RMs, and transit rm1 to active, rm2 to standby startRMs(); @@ -241,10 +247,10 @@ private void checkAsyncSchedulerThreads(Thread currentThread){ } } } - Assert.assertEquals(1, numResourceCommitterService); - Assert.assertEquals(1, numAsyncScheduleThread); - Assert.assertNotNull(asyncScheduleThread); - Assert.assertNotNull(resourceCommitterService); + assertEquals(1, numResourceCommitterService); + assertEquals(1, numAsyncScheduleThread); + assertNotNull(asyncScheduleThread); + assertNotNull(resourceCommitterService); } } \ 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/TestReservationSystemWithRMHA.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestReservationSystemWithRMHA.java index 239fbf3832f21..075e064e8c9aa 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestReservationSystemWithRMHA.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestReservationSystemWithRMHA.java @@ -39,13 +39,19 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; import org.apache.hadoop.yarn.util.Clock; import org.apache.hadoop.yarn.util.UTCClock; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.util.Map; +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.fail; + public class TestReservationSystemWithRMHA extends RMHATestBase { + @BeforeEach @Override public void setup() throws Exception { CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); @@ -75,10 +81,10 @@ public void testSubmitReservationAndCheckAfterFailover() throws Exception { try { response = clientService.submitReservation(request); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } - Assert.assertNotNull(response); - Assert.assertNotNull(reservationID); + assertNotNull(response); + assertNotNull(reservationID); LOG.info("Submit reservation response: " + reservationID); // Do the failover @@ -89,8 +95,8 @@ public void testSubmitReservationAndCheckAfterFailover() throws Exception { RMState state = rm2.getRMContext().getStateStore().loadState(); Map reservationStateMap = state.getReservationState().get(ReservationSystemTestUtil.reservationQ); - Assert.assertNotNull(reservationStateMap); - Assert.assertNotNull(reservationStateMap.get(reservationID)); + assertNotNull(reservationStateMap); + assertNotNull(reservationStateMap.get(reservationID)); } @Test @@ -111,10 +117,10 @@ public void testUpdateReservationAndCheckAfterFailover() throws Exception { try { response = clientService.submitReservation(request); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } - Assert.assertNotNull(response); - Assert.assertNotNull(reservationID); + assertNotNull(response); + assertNotNull(reservationID); LOG.info("Submit reservation response: " + reservationID); ReservationDefinition reservationDefinition = request.getReservationDefinition(); @@ -135,10 +141,10 @@ public void testUpdateReservationAndCheckAfterFailover() throws Exception { RMState state = rm2.getRMContext().getStateStore().loadState(); Map reservationStateMap = state.getReservationState().get(ReservationSystemTestUtil.reservationQ); - Assert.assertNotNull(reservationStateMap); + assertNotNull(reservationStateMap); ReservationAllocationStateProto reservationState = reservationStateMap.get(reservationID); - Assert.assertEquals(newDeadline, + assertEquals(newDeadline, reservationState.getReservationDefinition().getDeadline()); } @@ -160,10 +166,10 @@ public void testDeleteReservationAndCheckAfterFailover() throws Exception { try { response = clientService.submitReservation(request); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } - Assert.assertNotNull(response); - Assert.assertNotNull(reservationID); + assertNotNull(response); + assertNotNull(reservationID); // Delete the reservation ReservationDeleteRequest deleteRequest = @@ -176,7 +182,7 @@ public void testDeleteReservationAndCheckAfterFailover() throws Exception { rm2.registerNode("127.0.0.1:1", 102400, 100); RMState state = rm2.getRMContext().getStateStore().loadState(); - Assert.assertNull(state.getReservationState() + assertNull(state.getReservationState() .get(ReservationSystemTestUtil.reservationQ)); } @@ -197,12 +203,12 @@ private void addNodeCapacityToPlan(MockRM rm, int memory, int vCores) { Thread.sleep(1000); } while (attempts-- > 0); if (attempts <= 0) { - Assert.fail("Exhausted attempts in checking if node capacity was " + fail("Exhausted attempts in checking if node capacity was " + "added to the plan"); } } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } } @@ -219,8 +225,8 @@ private ReservationSubmissionRequest createReservationSubmissionRequest( private void validateReservation(Plan plan, ReservationId resId, ReservationDefinition rDef) { ReservationAllocation reservation = plan.getReservationById(resId); - Assert.assertNotNull(reservation); - Assert.assertEquals(rDef.getDeadline(), + assertNotNull(reservation); + assertEquals(rDef.getDeadline(), reservation.getReservationDefinition().getDeadline()); } @@ -242,10 +248,10 @@ public void testSubmitReservationFailoverAndDelete() throws Exception { try { response = clientService.submitReservation(request); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } - Assert.assertNotNull(response); - Assert.assertNotNull(reservationID); + assertNotNull(response); + assertNotNull(reservationID); LOG.info("Submit reservation response: " + reservationID); ReservationDefinition reservationDefinition = request.getReservationDefinition(); @@ -268,10 +274,10 @@ public void testSubmitReservationFailoverAndDelete() throws Exception { try { deleteResponse = clientService.deleteReservation(deleteRequest); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } - Assert.assertNotNull(deleteResponse); - Assert.assertNull(plan.getReservationById(reservationID)); + assertNotNull(deleteResponse); + assertNull(plan.getReservationById(reservationID)); } @Test @@ -296,10 +302,10 @@ public void testFailoverAndSubmitReservation() throws Exception { try { response = clientService.submitReservation(request); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } - Assert.assertNotNull(response); - Assert.assertNotNull(reservationID); + assertNotNull(response); + assertNotNull(reservationID); LOG.info("Submit reservation response: " + reservationID); ReservationDefinition reservationDefinition = request.getReservationDefinition(); @@ -328,10 +334,10 @@ public void testSubmitReservationFailoverAndUpdate() throws Exception { try { response = clientService.submitReservation(request); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } - Assert.assertNotNull(response); - Assert.assertNotNull(reservationID); + assertNotNull(response); + assertNotNull(reservationID); LOG.info("Submit reservation response: " + reservationID); ReservationDefinition reservationDefinition = request.getReservationDefinition(); @@ -356,9 +362,9 @@ public void testSubmitReservationFailoverAndUpdate() throws Exception { try { updateResponse = clientService.updateReservation(updateRequest); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } - Assert.assertNotNull(updateResponse); + assertNotNull(updateResponse); validateReservation(plan, reservationID, reservationDefinition); } @@ -380,10 +386,10 @@ public void testSubmitUpdateReservationFailoverAndDelete() throws Exception { try { response = clientService.submitReservation(request); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } - Assert.assertNotNull(response); - Assert.assertNotNull(reservationID); + assertNotNull(response); + assertNotNull(reservationID); LOG.info("Submit reservation response: " + reservationID); ReservationDefinition reservationDefinition = request.getReservationDefinition(); @@ -402,9 +408,9 @@ public void testSubmitUpdateReservationFailoverAndDelete() throws Exception { try { updateResponse = clientService.updateReservation(updateRequest); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } - Assert.assertNotNull(updateResponse); + assertNotNull(updateResponse); validateReservation(plan, reservationID, reservationDefinition); // Do the failover @@ -425,10 +431,10 @@ public void testSubmitUpdateReservationFailoverAndDelete() throws Exception { try { deleteResponse = clientService.deleteReservation(deleteRequest); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } - Assert.assertNotNull(deleteResponse); - Assert.assertNull(plan.getReservationById(reservationID)); + assertNotNull(deleteResponse); + assertNull(plan.getReservationById(reservationID)); } @Test @@ -451,10 +457,10 @@ public void testReservationResizeAfterFailover() throws Exception { try { response = clientService.submitReservation(request); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } - Assert.assertNotNull(response); - Assert.assertNotNull(resID1); + assertNotNull(response); + assertNotNull(resID1); LOG.info("Submit reservation response: " + resID1); ReservationId resID2 = getNewReservation(clientService) @@ -463,10 +469,10 @@ public void testReservationResizeAfterFailover() throws Exception { try { response = clientService.submitReservation(request); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } - Assert.assertNotNull(response); - Assert.assertNotNull(resID2); + assertNotNull(response); + assertNotNull(resID2); LOG.info("Submit reservation response: " + resID2); ReservationId resID3 = getNewReservation(clientService) @@ -475,10 +481,10 @@ public void testReservationResizeAfterFailover() throws Exception { try { response = clientService.submitReservation(request); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } - Assert.assertNotNull(response); - Assert.assertNotNull(resID3); + assertNotNull(response); + assertNotNull(resID3); LOG.info("Submit reservation response: " + resID3); // allow the reservations to become active @@ -493,11 +499,11 @@ public void testReservationResizeAfterFailover() throws Exception { validateReservation(plan, resID3, reservationDefinition); ResourceScheduler scheduler = rm1.getResourceScheduler(); QueueInfo resQ1 = scheduler.getQueueInfo(resID1.toString(), false, false); - Assert.assertEquals(0.05, resQ1.getCapacity(), 0.001f); + assertEquals(0.05, resQ1.getCapacity(), 0.001f); QueueInfo resQ2 = scheduler.getQueueInfo(resID2.toString(), false, false); - Assert.assertEquals(0.05, resQ2.getCapacity(), 0.001f); + assertEquals(0.05, resQ2.getCapacity(), 0.001f); QueueInfo resQ3 = scheduler.getQueueInfo(resID3.toString(), false, false); - Assert.assertEquals(0.05, resQ3.getCapacity(), 0.001f); + assertEquals(0.05, resQ3.getCapacity(), 0.001f); // Do the failover explicitFailover(); @@ -513,11 +519,11 @@ public void testReservationResizeAfterFailover() throws Exception { // verify if the reservations have been resized scheduler = rm2.getResourceScheduler(); resQ1 = scheduler.getQueueInfo(resID1.toString(), false, false); - Assert.assertEquals(1f / 3f, resQ1.getCapacity(), 0.001f); + assertEquals(1f / 3f, resQ1.getCapacity(), 0.001f); resQ2 = scheduler.getQueueInfo(resID2.toString(), false, false); - Assert.assertEquals(1f / 3f, resQ2.getCapacity(), 0.001f); + assertEquals(1f / 3f, resQ2.getCapacity(), 0.001f); resQ3 = scheduler.getQueueInfo(resID3.toString(), false, false); - Assert.assertEquals(1f / 3f, resQ3.getCapacity(), 0.001f); + assertEquals(1f / 3f, resQ3.getCapacity(), 0.001f); } private void waitForReservationActivation(MockRM rm, @@ -536,11 +542,10 @@ private void waitForReservationActivation(MockRM rm, Thread.sleep(100); } while (attempts-- > 0); if (attempts <= 0) { - Assert - .fail("Exceeded attempts in waiting for reservation to be active"); + fail("Exceeded attempts in waiting for reservation to be active"); } } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } } @@ -553,7 +558,7 @@ private GetNewReservationResponse getNewReservation(ClientRMService getNewReservationResponse = clientRMService.getNewReservation( newReservationRequest); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } return getNewReservationResponse; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestSubmitApplicationWithRMHA.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestSubmitApplicationWithRMHA.java index cb599ca3f416b..d7e4d02f7d611 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestSubmitApplicationWithRMHA.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestSubmitApplicationWithRMHA.java @@ -18,7 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -29,7 +29,8 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; public class TestSubmitApplicationWithRMHA extends RMHATestBase{ @@ -91,9 +92,9 @@ private void verifySubmitApp(MockRM rm, RMApp app, YarnApplicationState state = rm.getApplicationReport(app.getApplicationId()) .getYarnApplicationState(); - Assert.assertTrue(state == YarnApplicationState.ACCEPTED + Assertions.assertTrue(state == YarnApplicationState.ACCEPTED || state == YarnApplicationState.SUBMITTED); - Assert.assertEquals(expectedAppId, app.getApplicationId()); + Assertions.assertEquals(expectedAppId, app.getApplicationId()); } // There are two scenarios when RM failover happens @@ -124,7 +125,7 @@ private void verifySubmitApp(MockRM rm, RMApp app, rm2.getApplicationReport(app0.getApplicationId()); // verify previous submission is successful. - Assert.assertTrue(appReport.getYarnApplicationState() + Assertions.assertTrue(appReport.getYarnApplicationState() == YarnApplicationState.ACCEPTED || appReport.getYarnApplicationState() == YarnApplicationState.SUBMITTED); @@ -171,7 +172,7 @@ private void verifySubmitApp(MockRM rm, RMApp app, // Expect ApplicationNotFoundException by calling getApplicationReport(). try { rm2.getApplicationReport(app0.getApplicationId()); - Assert.fail("Should get ApplicationNotFoundException here"); + Assertions.fail("Should get ApplicationNotFoundException here"); } catch (ApplicationNotFoundException ex) { // expected ApplicationNotFoundException } @@ -217,7 +218,7 @@ public void testGetApplicationReportIdempotent() throws Exception{ ApplicationReport appReport1 = rm1.getApplicationReport(app.getApplicationId()); - Assert.assertTrue(appReport1.getYarnApplicationState() == + Assertions.assertTrue(appReport1.getYarnApplicationState() == YarnApplicationState.ACCEPTED || appReport1.getYarnApplicationState() == YarnApplicationState.SUBMITTED); @@ -225,9 +226,9 @@ public void testGetApplicationReportIdempotent() throws Exception{ // call getApplicationReport again ApplicationReport appReport2 = rm1.getApplicationReport(app.getApplicationId()); - Assert.assertEquals(appReport1.getApplicationId(), + Assertions.assertEquals(appReport1.getApplicationId(), appReport2.getApplicationId()); - Assert.assertEquals(appReport1.getYarnApplicationState(), + Assertions.assertEquals(appReport1.getYarnApplicationState(), appReport2.getYarnApplicationState()); // Do the failover @@ -236,17 +237,17 @@ public void testGetApplicationReportIdempotent() throws Exception{ // call getApplicationReport ApplicationReport appReport3 = rm2.getApplicationReport(app.getApplicationId()); - Assert.assertEquals(appReport1.getApplicationId(), + Assertions.assertEquals(appReport1.getApplicationId(), appReport3.getApplicationId()); - Assert.assertEquals(appReport1.getYarnApplicationState(), + Assertions.assertEquals(appReport1.getYarnApplicationState(), appReport3.getYarnApplicationState()); // call getApplicationReport again ApplicationReport appReport4 = rm2.getApplicationReport(app.getApplicationId()); - Assert.assertEquals(appReport3.getApplicationId(), + Assertions.assertEquals(appReport3.getApplicationId(), appReport4.getApplicationId()); - Assert.assertEquals(appReport3.getYarnApplicationState(), + Assertions.assertEquals(appReport3.getYarnApplicationState(), appReport4.getYarnApplicationState()); } @@ -254,7 +255,8 @@ public void testGetApplicationReportIdempotent() throws Exception{ // during SubmitApplication Call: // 1) RMStateStore already saved the ApplicationState when failover happens // 2) RMStateStore did not save the ApplicationState when failover happens - @Test (timeout = 50000) + @Test + @Timeout(value = 50) public void testHandleRMHADuringSubmitApplicationCallWithSavedApplicationState() throws Exception { @@ -274,7 +276,7 @@ public void testGetApplicationReportIdempotent() throws Exception{ // before failover happens, the current active rm can load the previous // applicationState. // This RMApp should exist in the RMContext of current active RM - Assert.assertTrue(rm2.getRMContext().getRMApps() + Assertions.assertTrue(rm2.getRMContext().getRMApps() .containsKey(app0.getApplicationId())); // When we re-submit the application with same applicationId, it will @@ -298,10 +300,11 @@ public void testGetApplicationReportIdempotent() throws Exception{ .withApplicationId(app0.getApplicationId()) .build()); - Assert.assertEquals(app1.getApplicationId(), app0.getApplicationId()); + Assertions.assertEquals(app1.getApplicationId(), app0.getApplicationId()); } - @Test (timeout = 50000) + @Test + @Timeout(value = 50) public void testHandleRMHADuringSubmitApplicationCallWithoutSavedApplicationState() throws Exception { @@ -339,7 +342,7 @@ public void testGetApplicationReportIdempotent() throws Exception{ // When failover happens, the RMStateStore has not saved applicationState. // The applicationState of this RMApp is lost. // We should not find the RMApp in the RMContext of current active rm. - Assert.assertFalse(rm2.getRMContext().getRMApps() + Assertions.assertFalse(rm2.getRMContext().getRMApps() .containsKey(app0.getApplicationId())); // Submit the application with previous ApplicationId to current active RM @@ -366,7 +369,7 @@ public void testGetApplicationReportIdempotent() throws Exception{ .build()); verifySubmitApp(rm2, app1, app0.getApplicationId()); - Assert.assertTrue(rm2.getRMContext().getRMApps() + Assertions.assertTrue(rm2.getRMContext().getRMApps() .containsKey(app0.getApplicationId())); } }