From 8a2485d1658dab203fec01d429756549caaea0fd Mon Sep 17 00:00:00 2001 From: slfan1989 Date: Thu, 6 Mar 2025 13:13:56 +0800 Subject: [PATCH 1/6] YARN-11262. Upgrade JUnit from 4 to 5 in hadoop-yarn-server-resourcemanager Part1. --- .../ahs/TestRMApplicationHistoryWriter.java | 80 +++---- .../TestAMLaunchFailure.java | 14 +- .../TestAMRMRPCNodeUpdates.java | 84 ++++---- .../TestAMRMRPCResponseId.java | 24 +-- .../applicationsmanager/TestAMRestart.java | 180 ++++++++-------- .../TestASMStateMachine.java | 34 +-- .../TestApplicationMasterExpiry.java | 12 +- .../blacklist/TestBlacklistManager.java | 68 +++--- .../TestFederationRMStateStoreService.java | 108 +++++----- ...estFederationStateStoreServiceMetrics.java | 4 +- .../TestRMAppLogAggregationStatus.java | 114 +++++----- .../TestCombinedSystemMetricsPublisher.java | 74 ++++--- .../metrics/TestSystemMetricsPublisher.java | 166 +++++++------- .../TestSystemMetricsPublisherForV2.java | 71 +++--- .../monitor/TestSchedulingMonitor.java | 16 +- .../TestPreemptionForQueueWithPriorities.java | 6 +- ...tProportionalCapacityPreemptionPolicy.java | 32 +-- ...cityPreemptionPolicyForNodePartitions.java | 6 +- ...PreemptionPolicyForReservedContainers.java | 6 +- ...cityPreemptionPolicyInterQueueWithDRF.java | 6 +- ...nalCapacityPreemptionPolicyIntraQueue.java | 6 +- ...reemptionPolicyIntraQueueFairOrdering.java | 6 +- ...tyPreemptionPolicyIntraQueueUserLimit.java | 6 +- ...cityPreemptionPolicyIntraQueueWithDRF.java | 6 +- ...CapacityPreemptionPolicyMockFramework.java | 46 ++-- ...acityPreemptionPolicyPreemptToBalance.java | 4 +- ...CapacityPreemptionPolicyMockFramework.java | 24 +-- .../TestMetricsInvariantChecker.java | 12 +- .../TestFileSystemNodeAttributeStore.java | 74 ++++--- .../nodelabels/TestNodeAttributesManager.java | 80 +++---- .../TestNodeLabelFileReplication.java | 10 +- .../TestRMDelegatedNodeLabelsUpdater.java | 14 +- .../nodelabels/TestRMNodeLabelsManager.java | 204 +++++++++--------- 33 files changed, 825 insertions(+), 772 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/TestRMApplicationHistoryWriter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/TestRMApplicationHistoryWriter.java index a77df99fb25b6..193dbd064b7a7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/TestRMApplicationHistoryWriter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/TestRMApplicationHistoryWriter.java @@ -28,7 +28,7 @@ import java.util.Random; import org.apache.hadoop.yarn.server.resourcemanager.MockRMAppSubmitter; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; @@ -64,9 +64,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedulerConfiguration; -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; public class TestRMApplicationHistoryWriter { @@ -77,7 +77,7 @@ public class TestRMApplicationHistoryWriter { private List dispatchers = new ArrayList(); - @Before + @BeforeEach public void setup() { store = new MemoryApplicationHistoryStore(); Configuration conf = new Configuration(); @@ -124,7 +124,7 @@ protected AsyncDispatcher createDispatcher() { writer.start(); } - @After + @AfterEach public void tearDown() { writer.stop(); } @@ -188,8 +188,8 @@ public void testDefaultStoreSetup() throws Exception { writer.init(conf); writer.start(); try { - Assert.assertFalse(writer.historyServiceEnabled); - Assert.assertNull(writer.writer); + Assertions.assertFalse(writer.historyServiceEnabled); + Assertions.assertNull(writer.writer); } finally { writer.stop(); writer.close(); @@ -210,13 +210,13 @@ public void testWriteApplication() throws Exception { Thread.sleep(100); } } - Assert.assertNotNull(appHD); - Assert.assertEquals("test app", appHD.getApplicationName()); - Assert.assertEquals("test app type", appHD.getApplicationType()); - Assert.assertEquals("test user", appHD.getUser()); - Assert.assertEquals("test queue", appHD.getQueue()); - Assert.assertEquals(0L, appHD.getSubmitTime()); - Assert.assertEquals(1L, appHD.getStartTime()); + Assertions.assertNotNull(appHD); + Assertions.assertEquals("test app", appHD.getApplicationName()); + Assertions.assertEquals("test app type", appHD.getApplicationType()); + Assertions.assertEquals("test user", appHD.getUser()); + Assertions.assertEquals("test queue", appHD.getQueue()); + Assertions.assertEquals(0L, appHD.getSubmitTime()); + Assertions.assertEquals(1L, appHD.getStartTime()); writer.applicationFinished(app, RMAppState.FINISHED); for (int i = 0; i < MAX_RETRIES; ++i) { @@ -227,11 +227,11 @@ public void testWriteApplication() throws Exception { Thread.sleep(100); } } - Assert.assertEquals(2L, appHD.getFinishTime()); - Assert.assertEquals("test diagnostics info", appHD.getDiagnosticsInfo()); - Assert.assertEquals(FinalApplicationStatus.UNDEFINED, + Assertions.assertEquals(2L, appHD.getFinishTime()); + Assertions.assertEquals("test diagnostics info", appHD.getDiagnosticsInfo()); + Assertions.assertEquals(FinalApplicationStatus.UNDEFINED, appHD.getFinalApplicationStatus()); - Assert.assertEquals(YarnApplicationState.FINISHED, + Assertions.assertEquals(YarnApplicationState.FINISHED, appHD.getYarnApplicationState()); } @@ -252,10 +252,10 @@ public void testWriteApplicationAttempt() throws Exception { Thread.sleep(100); } } - Assert.assertNotNull(appAttemptHD); - Assert.assertEquals("test host", appAttemptHD.getHost()); - Assert.assertEquals(-100, appAttemptHD.getRPCPort()); - Assert.assertEquals(ContainerId.newContainerId( + Assertions.assertNotNull(appAttemptHD); + Assertions.assertEquals("test host", appAttemptHD.getHost()); + Assertions.assertEquals(-100, appAttemptHD.getRPCPort()); + Assertions.assertEquals(ContainerId.newContainerId( ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 1), 1), 1), appAttemptHD.getMasterContainerId()); @@ -270,12 +270,12 @@ public void testWriteApplicationAttempt() throws Exception { Thread.sleep(100); } } - Assert.assertEquals("test diagnostics info", + Assertions.assertEquals("test diagnostics info", appAttemptHD.getDiagnosticsInfo()); - Assert.assertEquals("test url", appAttemptHD.getTrackingURL()); - Assert.assertEquals(FinalApplicationStatus.UNDEFINED, + Assertions.assertEquals("test url", appAttemptHD.getTrackingURL()); + Assertions.assertEquals(FinalApplicationStatus.UNDEFINED, appAttemptHD.getFinalApplicationStatus()); - Assert.assertEquals(YarnApplicationAttemptState.FINISHED, + Assertions.assertEquals(YarnApplicationAttemptState.FINISHED, appAttemptHD.getYarnApplicationAttemptState()); } @@ -297,13 +297,13 @@ public void testWriteContainer() throws Exception { Thread.sleep(100); } } - Assert.assertNotNull(containerHD); - Assert.assertEquals(NodeId.newInstance("test host", -100), + Assertions.assertNotNull(containerHD); + Assertions.assertEquals(NodeId.newInstance("test host", -100), containerHD.getAssignedNode()); - Assert.assertEquals(Resource.newInstance(-1, -1), + Assertions.assertEquals(Resource.newInstance(-1, -1), containerHD.getAllocatedResource()); - Assert.assertEquals(Priority.UNDEFINED, containerHD.getPriority()); - Assert.assertEquals(0L, container.getCreationTime()); + Assertions.assertEquals(Priority.UNDEFINED, containerHD.getPriority()); + Assertions.assertEquals(0L, container.getCreationTime()); writer.containerFinished(container); for (int i = 0; i < MAX_RETRIES; ++i) { @@ -316,10 +316,10 @@ public void testWriteContainer() throws Exception { Thread.sleep(100); } } - Assert.assertEquals("test diagnostics info", + Assertions.assertEquals("test diagnostics info", containerHD.getDiagnosticsInfo()); - Assert.assertEquals(-1, containerHD.getContainerExitStatus()); - Assert.assertEquals(ContainerState.COMPLETE, + Assertions.assertEquals(-1, containerHD.getContainerExitStatus()); + Assertions.assertEquals(ContainerState.COMPLETE, containerHD.getContainerState()); } @@ -355,10 +355,10 @@ public void testParallelWrite() throws Exception { Thread.sleep(500); } } - Assert.assertTrue(allEventsHandled(20 * 10 * 10 + 20 * 10 + 20)); + Assertions.assertTrue(allEventsHandled(20 * 10 * 10 + 20 * 10 + 20)); // Validate all events of one application are handled by one dispatcher for (ApplicationId appId : appIds) { - Assert.assertTrue(handledByOne(appId)); + Assertions.assertTrue(handledByOne(appId)); } } @@ -438,7 +438,7 @@ public void containerFinished(RMContainer container) { long elapsedTime2 = finishTime2 - startTime2; // No more than 10% additional workload // Should be much less, but computation time is fluctuated - Assert.assertTrue(elapsedTime2 - elapsedTime1 < elapsedTime1 / 10); + Assertions.assertTrue(elapsedTime2 - elapsedTime1 < elapsedTime1 / 10); } private void testRMWritingMassiveHistory(MockRM rm) throws Exception { @@ -470,7 +470,7 @@ private void testRMWritingMassiveHistory(MockRM rm) throws Exception { allocatedSize += allocated.size(); nm.nodeHeartbeat(true); } - Assert.assertEquals(request, allocatedSize); + Assertions.assertEquals(request, allocatedSize); am.unregisterAppAttempt(); rm.waitForState(am.getApplicationAttemptId(), RMAppAttemptState.FINISHING); @@ -487,7 +487,7 @@ private void testRMWritingMassiveHistory(MockRM rm) throws Exception { cleaned = resp.getContainersToCleanup(); cleanedSize += cleaned.size(); } - Assert.assertEquals(allocatedSize, cleanedSize); + Assertions.assertEquals(allocatedSize, cleanedSize); rm.waitForState(app.getApplicationId(), RMAppState.FINISHED); 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/applicationsmanager/TestAMLaunchFailure.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMLaunchFailure.java index 06e20098ed375..6c8cfce2e8b39 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMLaunchFailure.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMLaunchFailure.java @@ -24,7 +24,7 @@ import java.util.List; import java.util.concurrent.atomic.AtomicInteger; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,9 +53,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler; import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager; import org.apache.hadoop.yarn.util.resource.Resources; -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; */ /* a test case that tests the launch failure of a AM */ @@ -179,7 +179,7 @@ public class TestAMLaunchFailure { // } // // -// @Before +// @BeforeEach // public void setUp() { // context = new RMContextImpl(new MemStore()); // Configuration conf = new Configuration(); @@ -202,7 +202,7 @@ public class TestAMLaunchFailure { // asmImpl.start(); // } // -// @After +// @AfterEach // public void tearDown() { // asmImpl.stop(); // } @@ -229,6 +229,6 @@ public class TestAMLaunchFailure { // Thread.sleep(200); // application = context.getApplications().get(appID); // } -// Assert.assertEquals(ApplicationState.FAILED, application.getState()); +// Assertions.assertEquals(ApplicationState.FAILED, application.getState()); // } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRMRPCNodeUpdates.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRMRPCNodeUpdates.java index c4291b6040bf7..f9ff438fcf0dc 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRMRPCNodeUpdates.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRMRPCNodeUpdates.java @@ -23,7 +23,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRMAppSubmitter; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; @@ -42,15 +42,15 @@ 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.scheduler.capacity.CapacitySchedulerConfiguration; -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; public class TestAMRMRPCNodeUpdates { private MockRM rm; private ApplicationMasterService amService; - @Before + @BeforeEach public void setUp() { this.rm = new MockRM() { @Override @@ -66,7 +66,7 @@ public void init(Configuration conf) { amService = rm.getApplicationMasterService(); } - @After + @AfterEach public void tearDown() { if (rm != null) { this.rm.stop(); @@ -139,11 +139,11 @@ public void testAMRMDecommissioningNodes() throws Exception { AllocateResponse response1 = allocate(attempt1.getAppAttemptId(), allocateRequest1); List updatedNodes = response1.getUpdatedNodes(); - Assert.assertEquals(1, updatedNodes.size()); + Assertions.assertEquals(1, updatedNodes.size()); NodeReport nr = updatedNodes.iterator().next(); - Assert.assertEquals( + Assertions.assertEquals( decommissioningTimeout, nr.getDecommissioningTimeout()); - Assert.assertEquals( + Assertions.assertEquals( NodeUpdateType.NODE_DECOMMISSIONING, nr.getNodeUpdateType()); } @@ -173,11 +173,11 @@ public void testAMRMRecommissioningNodes() throws Exception { AllocateResponse response1 = allocate(attempt1.getAppAttemptId(), allocateRequest1); List updatedNodes = response1.getUpdatedNodes(); - Assert.assertEquals(1, updatedNodes.size()); + Assertions.assertEquals(1, updatedNodes.size()); NodeReport nr = updatedNodes.iterator().next(); - Assert.assertEquals( + Assertions.assertEquals( decommissioningTimeout, nr.getDecommissioningTimeout()); - Assert.assertEquals( + Assertions.assertEquals( NodeUpdateType.NODE_DECOMMISSIONING, nr.getNodeUpdateType()); // Wait for nm2 to RECOMMISSION @@ -188,9 +188,9 @@ public void testAMRMRecommissioningNodes() throws Exception { AllocateResponse response2 = allocate(attempt1.getAppAttemptId(), allocateRequest2); List updatedNodes2 = response2.getUpdatedNodes(); - Assert.assertEquals(1, updatedNodes2.size()); + Assertions.assertEquals(1, updatedNodes2.size()); NodeReport nr2 = updatedNodes2.iterator().next(); - Assert.assertEquals( + Assertions.assertEquals( NodeUpdateType.NODE_USABLE, nr2.getNodeUpdateType()); } @@ -220,7 +220,7 @@ public void testAMRMUnusableNodes() throws Exception { AllocateResponse response1 = allocate(attempt1.getAppAttemptId(), allocateRequest1); List updatedNodes = response1.getUpdatedNodes(); - Assert.assertEquals(0, updatedNodes.size()); + Assertions.assertEquals(0, updatedNodes.size()); syncNodeHeartbeat(nm4, false); @@ -230,22 +230,22 @@ public void testAMRMUnusableNodes() throws Exception { null); response1 = allocate(attempt1.getAppAttemptId(), allocateRequest1); updatedNodes = response1.getUpdatedNodes(); - Assert.assertEquals(1, updatedNodes.size()); + Assertions.assertEquals(1, updatedNodes.size()); NodeReport nr = updatedNodes.iterator().next(); - Assert.assertEquals(nm4.getNodeId(), nr.getNodeId()); - Assert.assertEquals(NodeState.UNHEALTHY, nr.getNodeState()); - Assert.assertNull(nr.getDecommissioningTimeout()); - Assert.assertEquals(NodeUpdateType.NODE_UNUSABLE, nr.getNodeUpdateType()); + Assertions.assertEquals(nm4.getNodeId(), nr.getNodeId()); + Assertions.assertEquals(NodeState.UNHEALTHY, nr.getNodeState()); + Assertions.assertNull(nr.getDecommissioningTimeout()); + Assertions.assertEquals(NodeUpdateType.NODE_UNUSABLE, nr.getNodeUpdateType()); // resending the allocate request returns the same result response1 = allocate(attempt1.getAppAttemptId(), allocateRequest1); updatedNodes = response1.getUpdatedNodes(); - Assert.assertEquals(1, updatedNodes.size()); + Assertions.assertEquals(1, updatedNodes.size()); nr = updatedNodes.iterator().next(); - Assert.assertEquals(nm4.getNodeId(), nr.getNodeId()); - Assert.assertEquals(NodeState.UNHEALTHY, nr.getNodeState()); - Assert.assertNull(nr.getDecommissioningTimeout()); - Assert.assertEquals(NodeUpdateType.NODE_UNUSABLE, nr.getNodeUpdateType()); + Assertions.assertEquals(nm4.getNodeId(), nr.getNodeId()); + Assertions.assertEquals(NodeState.UNHEALTHY, nr.getNodeState()); + Assertions.assertNull(nr.getDecommissioningTimeout()); + Assertions.assertEquals(NodeUpdateType.NODE_UNUSABLE, nr.getNodeUpdateType()); syncNodeLost(nm3); @@ -255,12 +255,12 @@ public void testAMRMUnusableNodes() throws Exception { null); response1 = allocate(attempt1.getAppAttemptId(), allocateRequest1); updatedNodes = response1.getUpdatedNodes(); - Assert.assertEquals(1, updatedNodes.size()); + Assertions.assertEquals(1, updatedNodes.size()); nr = updatedNodes.iterator().next(); - Assert.assertEquals(nm3.getNodeId(), nr.getNodeId()); - Assert.assertEquals(NodeState.LOST, nr.getNodeState()); - Assert.assertNull(nr.getDecommissioningTimeout()); - Assert.assertEquals(NodeUpdateType.NODE_UNUSABLE, nr.getNodeUpdateType()); + Assertions.assertEquals(nm3.getNodeId(), nr.getNodeId()); + Assertions.assertEquals(NodeState.LOST, nr.getNodeState()); + Assertions.assertNull(nr.getDecommissioningTimeout()); + Assertions.assertEquals(NodeUpdateType.NODE_UNUSABLE, nr.getNodeUpdateType()); // registering another AM gives it the complete failed list RMApp app2 = MockRMAppSubmitter.submitWithMemory(2000, rm); @@ -278,7 +278,7 @@ public void testAMRMUnusableNodes() throws Exception { AllocateResponse response2 = allocate(attempt2.getAppAttemptId(), allocateRequest2); updatedNodes = response2.getUpdatedNodes(); - Assert.assertEquals(0, updatedNodes.size()); + Assertions.assertEquals(0, updatedNodes.size()); syncNodeHeartbeat(nm4, true); @@ -288,24 +288,24 @@ public void testAMRMUnusableNodes() throws Exception { null); response1 = allocate(attempt1.getAppAttemptId(), allocateRequest1); updatedNodes = response1.getUpdatedNodes(); - Assert.assertEquals(1, updatedNodes.size()); + Assertions.assertEquals(1, updatedNodes.size()); nr = updatedNodes.iterator().next(); - Assert.assertEquals(nm4.getNodeId(), nr.getNodeId()); - Assert.assertEquals(NodeState.RUNNING, nr.getNodeState()); - Assert.assertNull(nr.getDecommissioningTimeout()); - Assert.assertEquals(NodeUpdateType.NODE_USABLE, nr.getNodeUpdateType()); + Assertions.assertEquals(nm4.getNodeId(), nr.getNodeId()); + Assertions.assertEquals(NodeState.RUNNING, nr.getNodeState()); + Assertions.assertNull(nr.getDecommissioningTimeout()); + Assertions.assertEquals(NodeUpdateType.NODE_USABLE, nr.getNodeUpdateType()); allocateRequest2 = AllocateRequest.newInstance(response2.getResponseId(), 0F, null, null, null); response2 = allocate(attempt2.getAppAttemptId(), allocateRequest2); updatedNodes = response2.getUpdatedNodes(); - Assert.assertEquals(1, updatedNodes.size()); + Assertions.assertEquals(1, updatedNodes.size()); nr = updatedNodes.iterator().next(); - Assert.assertEquals(nm4.getNodeId(), nr.getNodeId()); - Assert.assertEquals(NodeState.RUNNING, nr.getNodeState()); - Assert.assertNull(nr.getDecommissioningTimeout()); - Assert.assertEquals(NodeUpdateType.NODE_USABLE, nr.getNodeUpdateType()); + Assertions.assertEquals(nm4.getNodeId(), nr.getNodeId()); + Assertions.assertEquals(NodeState.RUNNING, nr.getNodeState()); + Assertions.assertNull(nr.getDecommissioningTimeout()); + Assertions.assertEquals(NodeUpdateType.NODE_USABLE, nr.getNodeUpdateType()); // subsequent allocate calls should return no updated nodes allocateRequest2 = @@ -313,7 +313,7 @@ public void testAMRMUnusableNodes() throws Exception { null); response2 = allocate(attempt2.getAppAttemptId(), allocateRequest2); updatedNodes = response2.getUpdatedNodes(); - Assert.assertEquals(0, updatedNodes.size()); + Assertions.assertEquals(0, updatedNodes.size()); // how to do the above for LOST node diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRMRPCResponseId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRMRPCResponseId.java index aac46f7caa24b..0476abc1d1887 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRMRPCResponseId.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRMRPCResponseId.java @@ -33,24 +33,24 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRMAppSubmitter; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestAMRMRPCResponseId { private MockRM rm; ApplicationMasterService amService = null; - @Before + @BeforeEach public void setUp() { this.rm = new MockRM(); rm.start(); amService = rm.getApplicationMasterService(); } - @After + @AfterEach public void tearDown() { if (rm != null) { this.rm.stop(); @@ -93,26 +93,26 @@ public void testARRMResponseId() throws Exception { AllocateResponse response = allocate(attempt.getAppAttemptId(), allocateRequest); - Assert.assertEquals(1, response.getResponseId()); - Assert.assertTrue(response.getAMCommand() == null); + Assertions.assertEquals(1, response.getResponseId()); + Assertions.assertTrue(response.getAMCommand() == null); allocateRequest = AllocateRequest.newInstance(response.getResponseId(), 0F, null, null, null); response = allocate(attempt.getAppAttemptId(), allocateRequest); - Assert.assertEquals(2, response.getResponseId()); + Assertions.assertEquals(2, response.getResponseId()); /* try resending */ response = allocate(attempt.getAppAttemptId(), allocateRequest); - Assert.assertEquals(2, response.getResponseId()); + Assertions.assertEquals(2, response.getResponseId()); /** try sending old request again **/ allocateRequest = AllocateRequest.newInstance(0, 0F, null, null, null); try { allocate(attempt.getAppAttemptId(), allocateRequest); - Assert.fail(); + Assertions.fail(); } catch (Exception e) { - Assert.assertTrue(e.getCause() instanceof InvalidApplicationMasterRequestException); + Assertions.assertTrue(e.getCause() instanceof InvalidApplicationMasterRequestException); } } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java index 82c534e6aad79..630a6200d8645 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java @@ -72,8 +72,9 @@ import org.apache.hadoop.thirdparty.com.google.common.base.Throwables; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; /** * Test AM restart functions. @@ -84,7 +85,8 @@ public TestAMRestart(SchedulerType type) throws IOException { super(type); } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testAMRestartWithExistingContainers() throws Exception { getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2); @@ -158,7 +160,7 @@ public void testAMRestartWithExistingContainers() throws Exception { Thread.sleep(200); } // assert containerId6 is reserved. - Assert.assertEquals(containerId6, schedulerAttempt.getReservedContainers() + Assertions.assertEquals(containerId6, schedulerAttempt.getReservedContainers() .get(0).getContainerId()); // fail the AM by sending CONTAINER_FINISHED event without registering. @@ -170,15 +172,15 @@ public void testAMRestartWithExistingContainers() throws Exception { Thread.sleep(3000); rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING); // acquired/allocated containers are cleaned up. - Assert.assertNull(rm1.getResourceScheduler().getRMContainer(containerId4)); - Assert.assertNull(rm1.getResourceScheduler().getRMContainer(containerId5)); + Assertions.assertNull(rm1.getResourceScheduler().getRMContainer(containerId4)); + Assertions.assertNull(rm1.getResourceScheduler().getRMContainer(containerId5)); // wait for app to start a new attempt. rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); // assert this is a new AM. ApplicationAttemptId newAttemptId = app1.getCurrentAppAttempt().getAppAttemptId(); - Assert.assertFalse(newAttemptId.equals(am1.getApplicationAttemptId())); + Assertions.assertFalse(newAttemptId.equals(am1.getApplicationAttemptId())); // launch the new AM MockAM am2 = rm1.launchAM(app1, rm1, nm1); @@ -187,7 +189,7 @@ public void testAMRestartWithExistingContainers() throws Exception { // Assert two containers are running: container2 and container3; - Assert.assertEquals(2, registerResponse.getContainersFromPreviousAttempts() + Assertions.assertEquals(2, registerResponse.getContainersFromPreviousAttempts() .size()); boolean containerId2Exists = false, containerId3Exists = false; for (Container container : registerResponse @@ -199,7 +201,7 @@ public void testAMRestartWithExistingContainers() throws Exception { containerId3Exists = true; } } - Assert.assertTrue(containerId2Exists && containerId3Exists); + Assertions.assertTrue(containerId2Exists && containerId3Exists); rm1.waitForState(app1.getApplicationId(), RMAppState.RUNNING); // complete container by sending the container complete event which has earlier @@ -236,7 +238,7 @@ public void testAMRestartWithExistingContainers() throws Exception { container6Exists = true; } } - Assert.assertTrue(container3Exists && container4Exists && container5Exists + Assertions.assertTrue(container3Exists && container4Exists && container5Exists && container6Exists); // New SchedulerApplicationAttempt also has the containers info. @@ -251,7 +253,7 @@ public void testAMRestartWithExistingContainers() throws Exception { // the 2nd attempt released the 1st attempt's running container, when the // 2nd attempt finishes. - Assert.assertFalse(schedulerNewAttempt.getLiveContainers().contains( + Assertions.assertFalse(schedulerNewAttempt.getLiveContainers().contains( containerId2)); // all 4 normal containers finished. System.out.println("New attempt's just finished containers: " @@ -278,8 +280,8 @@ public static List allocateContainers(MockNM nm1, MockAM am1, Thread.sleep(200); } - Assert.assertEquals("Did not get all containers allocated", - NUM_CONTAINERS, containers.size()); + Assertions.assertEquals( + NUM_CONTAINERS, containers.size(), "Did not get all containers allocated"); return containers; } @@ -293,7 +295,8 @@ private void waitForContainersToFinish(int expectedNum, RMAppAttempt attempt) } } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testNMTokensRebindOnAMRestart() throws Exception { getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 3); // To prevent test from blacklisting nm1 for AM, we sit threshold to half @@ -369,10 +372,10 @@ public void testNMTokensRebindOnAMRestart() throws Exception { rm1.waitForState(am2.getApplicationAttemptId(), RMAppAttemptState.RUNNING); // check am2 get the nm token from am1. - Assert.assertEquals(expectedNMTokens.size(), + Assertions.assertEquals(expectedNMTokens.size(), registerResponse.getNMTokensFromPreviousAttempts().size()); for (int i = 0; i < expectedNMTokens.size(); i++) { - Assert.assertTrue(expectedNMTokens.get(i) + Assertions.assertTrue(expectedNMTokens.get(i) .equals(registerResponse.getNMTokensFromPreviousAttempts().get(i))); } @@ -408,8 +411,8 @@ public void testNMTokensRebindOnAMRestart() throws Exception { // check am3 get the NM token from both am1 and am2; List transferredTokens = registerResponse.getNMTokensFromPreviousAttempts(); - Assert.assertEquals(2, transferredTokens.size()); - Assert.assertTrue(transferredTokens.containsAll(expectedNMTokens)); + Assertions.assertEquals(2, transferredTokens.size()); + Assertions.assertTrue(transferredTokens.containsAll(expectedNMTokens)); rm1.stop(); } @@ -417,7 +420,8 @@ public void testNMTokensRebindOnAMRestart() throws Exception { * AM container preempted, nm disk failure * should not be counted towards AM max retry count. */ - @Test(timeout = 100000) + @Test + @Timeout(value = 100) public void testShouldNotCountFailureToMaxAttemptRetry() throws Exception { getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2); getConf().setBoolean(YarnConfiguration.RECOVERY_ENABLED, true); @@ -442,7 +446,7 @@ public void testShouldNotCountFailureToMaxAttemptRetry() throws Exception { TestSchedulerUtils.waitSchedulerApplicationAttemptStopped(scheduler, am1.getApplicationAttemptId()); - Assert.assertFalse(attempt1.shouldCountTowardsMaxAttemptRetry()); + Assertions.assertFalse(attempt1.shouldCountTowardsMaxAttemptRetry()); rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); ApplicationStateData appState = ((MemoryRMStateStore) rm1.getRMStateStore()).getState() @@ -462,7 +466,7 @@ public void testShouldNotCountFailureToMaxAttemptRetry() throws Exception { TestSchedulerUtils.waitSchedulerApplicationAttemptStopped(scheduler, am2.getApplicationAttemptId()); - Assert.assertFalse(attempt2.shouldCountTowardsMaxAttemptRetry()); + Assertions.assertFalse(attempt2.shouldCountTowardsMaxAttemptRetry()); rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); MockAM am3 = rm1.waitForNewAMToLaunchAndRegister(app1.getApplicationId(), 3, nm1); @@ -484,8 +488,8 @@ public void testShouldNotCountFailureToMaxAttemptRetry() throws Exception { TestSchedulerUtils.waitSchedulerApplicationAttemptStopped(scheduler, am3.getApplicationAttemptId()); - Assert.assertFalse(attempt3.shouldCountTowardsMaxAttemptRetry()); - Assert.assertEquals(ContainerExitStatus.DISKS_FAILED, + Assertions.assertFalse(attempt3.shouldCountTowardsMaxAttemptRetry()); + Assertions.assertEquals(ContainerExitStatus.DISKS_FAILED, appState.getAttempt(am3.getApplicationAttemptId()) .getAMContainerExitStatus()); @@ -505,8 +509,8 @@ public void testShouldNotCountFailureToMaxAttemptRetry() throws Exception { TestSchedulerUtils.waitSchedulerApplicationAttemptStopped(scheduler, am4.getApplicationAttemptId()); - Assert.assertFalse(attempt4.shouldCountTowardsMaxAttemptRetry()); - Assert.assertEquals(ContainerExitStatus.ABORTED, + Assertions.assertFalse(attempt4.shouldCountTowardsMaxAttemptRetry()); + Assertions.assertEquals(ContainerExitStatus.ABORTED, appState.getAttempt(am4.getApplicationAttemptId()) .getAMContainerExitStatus()); // launch next AM in nm2 @@ -520,7 +524,7 @@ public void testShouldNotCountFailureToMaxAttemptRetry() throws Exception { TestSchedulerUtils.waitSchedulerApplicationAttemptStopped(scheduler, am5.getApplicationAttemptId()); - Assert.assertTrue(attempt5.shouldCountTowardsMaxAttemptRetry()); + Assertions.assertTrue(attempt5.shouldCountTowardsMaxAttemptRetry()); // launch next AM in nm2 MockAM am6 = @@ -534,15 +538,16 @@ public void testShouldNotCountFailureToMaxAttemptRetry() throws Exception { TestSchedulerUtils.waitSchedulerApplicationAttemptStopped(scheduler, am6.getApplicationAttemptId()); - Assert.assertTrue(attempt6.shouldCountTowardsMaxAttemptRetry()); + Assertions.assertTrue(attempt6.shouldCountTowardsMaxAttemptRetry()); // AM should not be restarted. rm1.waitForState(app1.getApplicationId(), RMAppState.FAILED); - Assert.assertEquals(6, app1.getAppAttempts().size()); + Assertions.assertEquals(6, app1.getAppAttempts().size()); rm1.stop(); } - @Test(timeout = 100000) + @Test + @Timeout(value = 100) public void testMaxAttemptOneMeansOne() throws Exception { getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1); getConf().setBoolean(YarnConfiguration.RECOVERY_ENABLED, true); @@ -569,7 +574,7 @@ public void testMaxAttemptOneMeansOne() throws Exception { // AM should not be restarted. rm1.waitForState(app1.getApplicationId(), RMAppState.FAILED); - Assert.assertEquals(1, app1.getAppAttempts().size()); + Assertions.assertEquals(1, app1.getAppAttempts().size()); rm1.stop(); } @@ -578,7 +583,8 @@ public void testMaxAttemptOneMeansOne() throws Exception { * AM preemption failure towards the max-retry-account and should be able to * re-launch the AM. */ - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testPreemptedAMRestartOnRMRestart() throws Exception { getConf().setBoolean(YarnConfiguration.RECOVERY_ENABLED, true); getConf().setBoolean( @@ -608,7 +614,7 @@ public void testPreemptedAMRestartOnRMRestart() throws Exception { rm1.waitForState(am1.getApplicationAttemptId(), RMAppAttemptState.FAILED); TestSchedulerUtils.waitSchedulerApplicationAttemptStopped(scheduler, am1.getApplicationAttemptId()); - Assert.assertTrue(attempt1.shouldCountTowardsMaxAttemptRetry()); + Assertions.assertTrue(attempt1.shouldCountTowardsMaxAttemptRetry()); // wait for the next AM to start rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); @@ -621,21 +627,21 @@ public void testPreemptedAMRestartOnRMRestart() throws Exception { scheduler.killContainer(scheduler.getRMContainer(amContainer)); rm1.waitForState(am2.getApplicationAttemptId(), RMAppAttemptState.FAILED); - Assert.assertFalse(attempt2.shouldCountTowardsMaxAttemptRetry()); + Assertions.assertFalse(attempt2.shouldCountTowardsMaxAttemptRetry()); rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); // state store has 2 attempts stored. ApplicationStateData appState = memStore.getState().getApplicationState().get(app1.getApplicationId()); - Assert.assertEquals(2, appState.getAttemptCount()); + Assertions.assertEquals(2, appState.getAttemptCount()); if (getSchedulerType().equals(SchedulerType.FAIR)) { // attempt stored has the preempted container exit status. - Assert.assertEquals(ContainerExitStatus.KILLED_BY_RESOURCEMANAGER, + Assertions.assertEquals(ContainerExitStatus.KILLED_BY_RESOURCEMANAGER, appState.getAttempt(am2.getApplicationAttemptId()) .getAMContainerExitStatus()); } else { // attempt stored has the preempted container exit status. - Assert.assertEquals(ContainerExitStatus.PREEMPTED, + Assertions.assertEquals(ContainerExitStatus.PREEMPTED, appState.getAttempt(am2.getApplicationAttemptId()) .getAMContainerExitStatus()); } @@ -652,8 +658,8 @@ public void testPreemptedAMRestartOnRMRestart() throws Exception { RMAppAttempt attempt3 = rm2.getRMContext().getRMApps().get(app1.getApplicationId()) .getCurrentAppAttempt(); - Assert.assertTrue(attempt3.shouldCountTowardsMaxAttemptRetry()); - Assert.assertEquals(ContainerExitStatus.INVALID, + Assertions.assertTrue(attempt3.shouldCountTowardsMaxAttemptRetry()); + Assertions.assertEquals(ContainerExitStatus.INVALID, appState.getAttempt(am3.getApplicationAttemptId()) .getAMContainerExitStatus()); rm1.stop(); @@ -665,7 +671,8 @@ public void testPreemptedAMRestartOnRMRestart() throws Exception { * AM failure towards the max-retry-account and should be able to * re-launch the AM. */ - @Test(timeout = 50000) + @Test + @Timeout(value = 50) public void testRMRestartOrFailoverNotCountedForAMFailures() throws Exception { getConf().setBoolean(YarnConfiguration.RECOVERY_ENABLED, true); @@ -696,7 +703,7 @@ public void testRMRestartOrFailoverNotCountedForAMFailures() rm1.waitForState(am1.getApplicationAttemptId(), RMAppAttemptState.FAILED); TestSchedulerUtils.waitSchedulerApplicationAttemptStopped(scheduler, am1.getApplicationAttemptId()); - Assert.assertTrue(attempt1.shouldCountTowardsMaxAttemptRetry()); + Assertions.assertTrue(attempt1.shouldCountTowardsMaxAttemptRetry()); // wait for the next AM to start rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); @@ -720,7 +727,7 @@ public void testRMRestartOrFailoverNotCountedForAMFailures() nm1.registerNode(Collections.singletonList(status), null); rm2.waitForState(attempt2.getAppAttemptId(), RMAppAttemptState.FAILED); - Assert.assertEquals(ContainerExitStatus.KILLED_BY_RESOURCEMANAGER, + Assertions.assertEquals(ContainerExitStatus.KILLED_BY_RESOURCEMANAGER, appState.getAttempt(am2.getApplicationAttemptId()) .getAMContainerExitStatus()); // Will automatically start a new AppAttempt in rm2 @@ -731,8 +738,8 @@ public void testRMRestartOrFailoverNotCountedForAMFailures() RMAppAttempt attempt3 = rm2.getRMContext().getRMApps().get(app1.getApplicationId()) .getCurrentAppAttempt(); - Assert.assertTrue(attempt3.shouldCountTowardsMaxAttemptRetry()); - Assert.assertEquals(ContainerExitStatus.INVALID, + Assertions.assertTrue(attempt3.shouldCountTowardsMaxAttemptRetry()); + Assertions.assertEquals(ContainerExitStatus.INVALID, appState.getAttempt(am3.getApplicationAttemptId()) .getAMContainerExitStatus()); @@ -740,7 +747,8 @@ public void testRMRestartOrFailoverNotCountedForAMFailures() rm2.stop(); } - @Test (timeout = 120000) + @Test + @Timeout(value = 120) public void testRMAppAttemptFailuresValidityInterval() throws Exception { getConf().setBoolean(YarnConfiguration.RECOVERY_ENABLED, true); getConf().setBoolean( @@ -778,7 +786,7 @@ public void testRMAppAttemptFailuresValidityInterval() throws Exception { rm1.waitForState(am.getApplicationAttemptId(), RMAppAttemptState.FAILED); // launch the second attempt rm1.waitForState(app.getApplicationId(), RMAppState.ACCEPTED); - Assert.assertEquals(2, app.getAppAttempts().size()); + Assertions.assertEquals(2, app.getAppAttempts().size()); MockAM am_2 = MockRM.launchAndRegisterAM(app, rm1, nm1); rm1.waitForState(am_2.getApplicationAttemptId(), RMAppAttemptState.RUNNING); @@ -809,7 +817,7 @@ public void testRMAppAttemptFailuresValidityInterval() throws Exception { // launch the second attempt rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); - Assert.assertEquals(2, app1.getAppAttempts().size()); + Assertions.assertEquals(2, app1.getAppAttempts().size()); RMAppAttempt attempt2 = app1.getCurrentAppAttempt(); MockAM am2 = MockRM.launchAndRegisterAM(app1, rm1, nm1); @@ -824,7 +832,7 @@ public void testRMAppAttemptFailuresValidityInterval() throws Exception { // can launch the third attempt successfully rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); - Assert.assertEquals(3, app1.getAppAttempts().size()); + Assertions.assertEquals(3, app1.getAppAttempts().size()); RMAppAttempt attempt3 = app1.getCurrentAppAttempt(); clock.reset(); MockAM am3 = MockRM.launchAndRegisterAM(app1, rm1, nm1); @@ -840,7 +848,7 @@ public void testRMAppAttemptFailuresValidityInterval() throws Exception { ApplicationStateData app1State = memStore1.getState().getApplicationState(). get(app1.getApplicationId()); - Assert.assertEquals(1, app1State.getFirstAttemptId()); + Assertions.assertEquals(1, app1State.getFirstAttemptId()); // re-register the NM nm1.setResourceTrackerService(rm2.getResourceTrackerService()); @@ -856,7 +864,7 @@ public void testRMAppAttemptFailuresValidityInterval() throws Exception { //Wait to make sure attempt3 be removed in State Store //TODO explore a better way than sleeping for a while (YARN-4929) Thread.sleep(15 * 1000); - Assert.assertEquals(2, app1State.getAttemptCount()); + Assertions.assertEquals(2, app1State.getAttemptCount()); rm2.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); @@ -870,7 +878,7 @@ public void testRMAppAttemptFailuresValidityInterval() throws Exception { nm1 .nodeHeartbeat(am4.getApplicationAttemptId(), 1, ContainerState.COMPLETE); rm2.waitForState(am4.getApplicationAttemptId(), RMAppAttemptState.FAILED); - Assert.assertEquals(2, app1State.getAttemptCount()); + Assertions.assertEquals(2, app1State.getAttemptCount()); // can launch the 5th attempt successfully rm2.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); @@ -884,7 +892,7 @@ public void testRMAppAttemptFailuresValidityInterval() throws Exception { nm1 .nodeHeartbeat(am5.getApplicationAttemptId(), 1, ContainerState.COMPLETE); rm2.waitForState(am5.getApplicationAttemptId(), RMAppAttemptState.FAILED); - Assert.assertEquals(2, app1State.getAttemptCount()); + Assertions.assertEquals(2, app1State.getAttemptCount()); rm2.waitForState(app1.getApplicationId(), RMAppState.FAILED); rm1.stop(); @@ -901,7 +909,8 @@ private boolean isContainerIdInContainerStatus( return false; } - @Test(timeout = 40000) + @Test + @Timeout(value = 40) public void testAMRestartNotLostContainerCompleteMsg() throws Exception { getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2); @@ -958,7 +967,7 @@ public void testAMRestartNotLostContainerCompleteMsg() throws Exception { app1.getCurrentAppAttempt().getJustFinishedContainers(); if (isContainerIdInContainerStatus(containerStatuses, containerId2)) { - Assert.fail(); + Assertions.fail(); } break; } @@ -973,7 +982,7 @@ public void testAMRestartNotLostContainerCompleteMsg() throws Exception { // assert this is a new AM. ApplicationAttemptId newAttemptId = app1.getCurrentAppAttempt().getAppAttemptId(); - Assert.assertFalse(newAttemptId.equals(am1.getApplicationAttemptId())); + Assertions.assertFalse(newAttemptId.equals(am1.getApplicationAttemptId())); // launch the new AM RMAppAttempt attempt2 = app1.getCurrentAppAttempt(); @@ -986,11 +995,11 @@ public void testAMRestartNotLostContainerCompleteMsg() throws Exception { allocateResponse.getCompletedContainersStatuses(); if (isContainerIdInContainerStatus(containerStatuses, containerId2) == false) { - Assert.fail(); + Assertions.fail(); } containerStatuses = attempt2.getJustFinishedContainers(); if (isContainerIdInContainerStatus(containerStatuses, containerId2)) { - Assert.fail(); + Assertions.fail(); } // the second allocate should not get container complete msg @@ -999,7 +1008,7 @@ public void testAMRestartNotLostContainerCompleteMsg() throws Exception { containerStatuses = allocateResponse.getCompletedContainersStatuses(); if (isContainerIdInContainerStatus(containerStatuses, containerId2)) { - Assert.fail(); + Assertions.fail(); } rm1.stop(); @@ -1010,7 +1019,8 @@ public void testAMRestartNotLostContainerCompleteMsg() throws Exception { * after AM reset window, even if AM who was the last is failed, * all containers are launched by previous AM should be kept. */ - @Test (timeout = 20000) + @Test + @Timeout(value = 20) public void testAMRestartNotLostContainerAfterAttemptFailuresValidityInterval() throws Exception { // explicitly set max-am-retry count as 2. @@ -1047,7 +1057,7 @@ public void testAMRestartNotLostContainerAfterAttemptFailuresValidityInterval() // launch the second attempt rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); - Assert.assertEquals(2, app1.getAppAttempts().size()); + Assertions.assertEquals(2, app1.getAppAttempts().size()); // It will be the last attempt. RMAppAttempt attempt2 = app1.getCurrentAppAttempt(); @@ -1064,20 +1074,20 @@ public void testAMRestartNotLostContainerAfterAttemptFailuresValidityInterval() // can launch the third attempt successfully rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); - Assert.assertEquals(3, app1.getAppAttempts().size()); + Assertions.assertEquals(3, app1.getAppAttempts().size()); MockAM am3 = rm1.launchAM(app1, rm1, nm1); RegisterApplicationMasterResponse registerResponse = am3.registerAppAttempt(); // keepContainers is applied, even if attempt2 was the last attempt. - Assert.assertEquals(1, registerResponse.getContainersFromPreviousAttempts() + Assertions.assertEquals(1, registerResponse.getContainersFromPreviousAttempts() .size()); boolean containerId2Exists = false; Container container = registerResponse.getContainersFromPreviousAttempts().get(0); if (container.getId().equals(containerId2)) { containerId2Exists = true; } - Assert.assertTrue(containerId2Exists); + Assertions.assertTrue(containerId2Exists); rm1.waitForState(app1.getApplicationId(), RMAppState.RUNNING); rm1.stop(); @@ -1102,7 +1112,8 @@ public void testAMRestartNotLostContainerAfterAttemptFailuresValidityInterval() * 8. Verify that the app master receives container 3 in the RM response to * its heartbeat. */ - @Test(timeout = 200000) + @Test + @Timeout(value = 200) public void testContainersFromPreviousAttemptsWithRMRestart() throws Exception { getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2); @@ -1182,7 +1193,7 @@ public void testContainersFromPreviousAttemptsWithRMRestart() nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 2, ContainerState.RUNNING); rm2.waitForState(nm1, containerId2, RMContainerState.RUNNING); - Assert.assertNotNull(rm2.getResourceScheduler() + Assertions.assertNotNull(rm2.getResourceScheduler() .getRMContainer(containerId2)); // wait for app to start a new attempt. @@ -1190,7 +1201,7 @@ public void testContainersFromPreviousAttemptsWithRMRestart() // assert this is a new AM. ApplicationAttemptId newAttemptId = app1.getCurrentAppAttempt().getAppAttemptId(); - Assert.assertFalse(newAttemptId.equals(am1.getApplicationAttemptId())); + Assertions.assertFalse(newAttemptId.equals(am1.getApplicationAttemptId())); // launch the new AM MockAM am2 = MockRM.launchAMWhenAsyncSchedulingEnabled(app1, rm2); @@ -1198,15 +1209,15 @@ public void testContainersFromPreviousAttemptsWithRMRestart() am2.registerAppAttempt(); // container2 is recovered from previous attempt - Assert.assertEquals(1, + Assertions.assertEquals(1, registerResponse.getContainersFromPreviousAttempts().size()); - Assert.assertEquals("container 2", containerId2, - registerResponse.getContainersFromPreviousAttempts().get(0).getId()); + Assertions.assertEquals(containerId2 +, registerResponse.getContainersFromPreviousAttempts().get(0).getId(), "container 2"); List prevNMTokens = registerResponse .getNMTokensFromPreviousAttempts(); - Assert.assertEquals(1, prevNMTokens.size()); + Assertions.assertEquals(1, prevNMTokens.size()); // container 2 is running on node 1 - Assert.assertEquals(nm1Address, prevNMTokens.get(0).getNodeId().toString()); + Assertions.assertEquals(nm1Address, prevNMTokens.get(0).getNodeId().toString()); rm2.waitForState(app1.getApplicationId(), RMAppState.RUNNING); @@ -1220,7 +1231,7 @@ public void testContainersFromPreviousAttemptsWithRMRestart() nm2.nodeHeartbeat(am1.getApplicationAttemptId(), 3, ContainerState.RUNNING); rm2.waitForState(nm2, containerId3, RMContainerState.RUNNING); - Assert.assertNotNull(rm2.getResourceScheduler() + Assertions.assertNotNull(rm2.getResourceScheduler() .getRMContainer(containerId3)); List containersFromPreviousAttempts = new ArrayList<>(); @@ -1230,12 +1241,12 @@ public void testContainersFromPreviousAttemptsWithRMRestart() if (allocateResponse.getContainersFromPreviousAttempts().size() > 0){ containersFromPreviousAttempts.addAll( allocateResponse.getContainersFromPreviousAttempts()); - Assert.assertEquals("new containers should not be allocated", - 0, allocateResponse.getAllocatedContainers().size()); + Assertions.assertEquals( + 0, allocateResponse.getAllocatedContainers().size(), "new containers should not be allocated"); List nmTokens = allocateResponse.getNMTokens(); - Assert.assertEquals(1, nmTokens.size()); + Assertions.assertEquals(1, nmTokens.size()); // container 3 is running on node 2 - Assert.assertEquals(nm2Address, + Assertions.assertEquals(nm2Address, nmTokens.get(0).getNodeId().toString()); return true; } @@ -1245,8 +1256,8 @@ public void testContainersFromPreviousAttemptsWithRMRestart() } return false; }, 2000, 200000); - Assert.assertEquals("container 3", containerId3, - containersFromPreviousAttempts.get(0).getId()); + Assertions.assertEquals(containerId3 +, containersFromPreviousAttempts.get(0).getId(), "container 3"); rm2.stop(); rm1.stop(); } @@ -1262,7 +1273,8 @@ public void testContainersFromPreviousAttemptsWithRMRestart() * 4. Verify that the used resource of queue should be cleaned up normally * after app fail. */ - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testQueueResourceDoesNotLeak() throws Exception { getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1); getConf().setBoolean(YarnConfiguration.RECOVERY_ENABLED, true); @@ -1301,7 +1313,7 @@ public void testQueueResourceDoesNotLeak() throws Exception { TestSchedulerUtils.waitSchedulerApplicationAttemptStopped(scheduler, am1.getApplicationAttemptId()); - Assert.assertFalse(attempt1.shouldCountTowardsMaxAttemptRetry()); + Assertions.assertFalse(attempt1.shouldCountTowardsMaxAttemptRetry()); // AM should not be restarted. rm1.waitForState(app1.getApplicationId(), RMAppState.FAILED); @@ -1311,20 +1323,20 @@ public void testQueueResourceDoesNotLeak() throws Exception { if (getSchedulerType() == SchedulerType.CAPACITY) { LeafQueue queue = (LeafQueue) ((CapacityScheduler) scheduler).getQueue("default"); - Assert.assertEquals(0, + Assertions.assertEquals(0, queue.getQueueResourceUsage().getUsed().getMemorySize()); - Assert.assertEquals(0, + Assertions.assertEquals(0, queue.getQueueResourceUsage().getUsed().getVirtualCores()); } else if (getSchedulerType() == SchedulerType.FAIR) { // The default queue is not auto created after YARN-7769 so // user-as-default-queue option is used Collection queues = ((FairScheduler) scheduler) .getQueueManager().getLeafQueues(); - Assert.assertEquals(1, queues.size()); + Assertions.assertEquals(1, queues.size()); FSLeafQueue queue = queues.iterator().next(); - Assert.assertEquals(0, queue.getResourceUsage().getMemorySize()); - Assert.assertEquals(0, queue.getResourceUsage().getVirtualCores()); + Assertions.assertEquals(0, queue.getResourceUsage().getMemorySize()); + Assertions.assertEquals(0, queue.getResourceUsage().getVirtualCores()); } rm1.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/applicationsmanager/TestASMStateMachine.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestASMStateMachine.java index ef3b65cc1a6c7..8ba565ffeb750 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestASMStateMachine.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestASMStateMachine.java @@ -21,7 +21,7 @@ /* import java.util.concurrent.atomic.AtomicInteger; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,9 +37,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType; import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStoreFactory; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor; -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; */ public class TestASMStateMachine { @@ -56,7 +56,7 @@ public class TestASMStateMachine { // private boolean launchCleanupCalled = false; // private AtomicInteger waitForState = new AtomicInteger(); // private Configuration conf = new Configuration(); -// @Before +// @BeforeEach // public void setUp() { // context.getDispatcher().init(conf); // context.getDispatcher().start(); @@ -67,7 +67,7 @@ public class TestASMStateMachine { // new MockAppplicationMasterInfo(); // } // -// @After +// @AfterEach // public void tearDown() { // // } @@ -160,7 +160,7 @@ public class TestASMStateMachine { // Thread.sleep(500); // count++; // } -// Assert.assertEquals(finalState, masterInfo.getState()); +// Assertions.assertEquals(finalState, masterInfo.getState()); // } // // /* Test the state machine. @@ -183,13 +183,13 @@ public class TestASMStateMachine { // ApplicationEventType.ALLOCATE, submissioncontext.getApplicationId())); // // waitForState(ApplicationState.LAUNCHED, masterInfo); -// Assert.assertTrue(snAllocateReceived); -// Assert.assertTrue(launchCalled); -// Assert.assertTrue(addedApplication); +// Assertions.assertTrue(snAllocateReceived); +// Assertions.assertTrue(launchCalled); +// Assertions.assertTrue(addedApplication); // handler // .handle(new AMRegistrationEvent(masterInfo.getMaster())); // waitForState(ApplicationState.RUNNING, masterInfo); -// Assert.assertEquals(ApplicationState.RUNNING, masterInfo.getState()); +// Assertions.assertEquals(ApplicationState.RUNNING, masterInfo.getState()); // // ApplicationStatus status = recordFactory // .newRecordInstance(ApplicationStatus.class); @@ -198,20 +198,20 @@ public class TestASMStateMachine { // // /* check if the state is still RUNNING */ // -// Assert.assertEquals(ApplicationState.RUNNING, masterInfo.getState()); +// Assertions.assertEquals(ApplicationState.RUNNING, masterInfo.getState()); // // handler.handle(new AMFinishEvent(masterInfo.getApplicationID(), // ApplicationState.COMPLETED, "", "")); // waitForState(ApplicationState.COMPLETED, masterInfo); -// Assert.assertEquals(ApplicationState.COMPLETED, masterInfo.getState()); +// Assertions.assertEquals(ApplicationState.COMPLETED, masterInfo.getState()); // /* check if clean up is called for everyone */ -// Assert.assertTrue(launchCleanupCalled); -// Assert.assertTrue(snreceivedCleanUp); -// Assert.assertTrue(removedApplication); +// Assertions.assertTrue(launchCleanupCalled); +// Assertions.assertTrue(snreceivedCleanUp); +// Assertions.assertTrue(removedApplication); // // /* check if expiry doesnt make it failed */ // handler.handle(new ApplicationEvent(ApplicationEventType.EXPIRE, // masterInfo.getApplicationID())); -// Assert.assertEquals(ApplicationState.COMPLETED, masterInfo.getState()); +// Assertions.assertEquals(ApplicationState.COMPLETED, masterInfo.getState()); // } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestApplicationMasterExpiry.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestApplicationMasterExpiry.java index beaec79fa38bb..ce496d0b21711 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestApplicationMasterExpiry.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestApplicationMasterExpiry.java @@ -21,7 +21,7 @@ /* import java.util.concurrent.atomic.AtomicInteger; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,8 +38,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; */ /** @@ -54,7 +54,7 @@ public class TestApplicationMasterExpiry { // private final RMContext context = new RMContextImpl(new MemStore()); // private AMLivelinessMonitor amLivelinessMonitor; // -// @Before +// @BeforeEach // public void setUp() { // new DummyApplicationTracker(); // new DummySN(); @@ -127,7 +127,7 @@ public class TestApplicationMasterExpiry { // Thread.sleep(500); // count++; // } -// Assert.assertEquals(finalState, application.getState()); +// Assertions.assertEquals(finalState, application.getState()); // } // // @Test @@ -169,6 +169,6 @@ public class TestApplicationMasterExpiry { // expiry.wait(1000); // } // } -// Assert.assertTrue(expired); +// Assertions.assertTrue(expired); // } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/TestBlacklistManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/TestBlacklistManager.java index 3102c6fbf8d79..87b80188f00ab 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/TestBlacklistManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/TestBlacklistManager.java @@ -23,8 +23,8 @@ import java.util.List; import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; public class TestBlacklistManager { @@ -45,14 +45,14 @@ public void testSimpleBlacklistBelowFailureThreshold() { Collections.sort(blacklistAdditions); List blacklistRemovals = blacklist.getBlacklistRemovals(); String[] expectedBlacklistAdditions = new String[]{anyNode2, anyNode}; - Assert.assertArrayEquals( - "Blacklist additions was not as expected", - expectedBlacklistAdditions, - blacklistAdditions.toArray()); - Assert.assertTrue( - "Blacklist removals should be empty but was " + - blacklistRemovals, - blacklistRemovals.isEmpty()); + Assertions.assertArrayEquals( + + expectedBlacklistAdditions +, blacklistAdditions.toArray(), "Blacklist additions was not as expected"); + Assertions.assertTrue( + + blacklistRemovals.isEmpty(), "Blacklist removals should be empty but was " + + blacklistRemovals); } @Test @@ -69,14 +69,14 @@ public void testSimpleBlacklistAboveFailureThreshold() { Collections.sort(blacklistAdditions); List blacklistRemovals = blacklist.getBlacklistRemovals(); String[] expectedBlacklistAdditions = new String[]{anyNode}; - Assert.assertArrayEquals( - "Blacklist additions was not as expected", - expectedBlacklistAdditions, - blacklistAdditions.toArray()); - Assert.assertTrue( - "Blacklist removals should be empty but was " + - blacklistRemovals, - blacklistRemovals.isEmpty()); + Assertions.assertArrayEquals( + + expectedBlacklistAdditions +, blacklistAdditions.toArray(), "Blacklist additions was not as expected"); + Assertions.assertTrue( + + blacklistRemovals.isEmpty(), "Blacklist removals should be empty but was " + + blacklistRemovals); manager.addNode(anyNode2); @@ -87,14 +87,14 @@ public void testSimpleBlacklistAboveFailureThreshold() { blacklistRemovals = blacklist.getBlacklistRemovals(); Collections.sort(blacklistRemovals); String[] expectedBlacklistRemovals = new String[] {anyNode2, anyNode}; - Assert.assertTrue( - "Blacklist additions should be empty but was " + - blacklistAdditions, - blacklistAdditions.isEmpty()); - Assert.assertArrayEquals( - "Blacklist removals was not as expected", - expectedBlacklistRemovals, - blacklistRemovals.toArray()); + Assertions.assertTrue( + + blacklistAdditions.isEmpty(), "Blacklist additions should be empty but was " + + blacklistAdditions); + Assertions.assertArrayEquals( + + expectedBlacklistRemovals +, blacklistRemovals.toArray(), "Blacklist removals was not as expected"); } @Test @@ -107,13 +107,13 @@ public void testDisabledBlacklist() { List blacklistAdditions = blacklist.getBlacklistAdditions(); List blacklistRemovals = blacklist.getBlacklistRemovals(); - Assert.assertTrue( - "Blacklist additions should be empty but was " + - blacklistAdditions, - blacklistAdditions.isEmpty()); - Assert.assertTrue( - "Blacklist removals should be empty but was " + - blacklistRemovals, - blacklistRemovals.isEmpty()); + Assertions.assertTrue( + + blacklistAdditions.isEmpty(), "Blacklist additions should be empty but was " + + blacklistAdditions); + Assertions.assertTrue( + + blacklistRemovals.isEmpty(), "Blacklist removals should be empty but was " + + blacklistRemovals); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/federation/TestFederationRMStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/federation/TestFederationRMStateStoreService.java index 722a0e329828c..e08a4206963fb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/federation/TestFederationRMStateStoreService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/federation/TestFederationRMStateStoreService.java @@ -74,10 +74,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterMetricsInfo; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.glassfish.jersey.jettison.JettisonJaxbContext; import org.glassfish.jersey.jettison.JettisonUnmarshaller; @@ -103,7 +103,7 @@ public class TestFederationRMStateStoreService { private JettisonUnmarshaller jsonUnmarshaller; private MockRM mockRM; - @Before + @BeforeEach public void setUp() throws IOException, YarnException, JAXBException { conf = new YarnConfiguration(); this.jettisonJaxbContext = new JettisonJaxbContext(ClusterMetricsInfo.class); @@ -118,7 +118,7 @@ public void setUp() throws IOException, YarnException, JAXBException { mockRM.start(); } - @After + @AfterEach public void tearDown() throws Exception { jettisonJaxbContext = null; jsonUnmarshaller = null; @@ -136,12 +136,12 @@ public void testFederationStateStoreService() throws Exception { rm.init(conf); stateStore = rm.getFederationStateStoreService().getStateStoreClient(); GetSubClusterInfoResponse response = stateStore.getSubCluster(request); - Assert.assertNull(response); + Assertions.assertNull(response); // Validate if sub-cluster is registered rm.start(); String capability = checkSubClusterInfo(SubClusterState.SC_NEW); - Assert.assertTrue(capability.isEmpty()); + Assertions.assertTrue(capability.isEmpty()); // Heartbeat to see if sub-cluster transitions to running FederationStateStoreHeartbeat storeHeartbeat = @@ -166,7 +166,7 @@ public void testFederationStateStoreService() throws Exception { explicitFailover(rm); capability = checkSubClusterInfo(SubClusterState.SC_NEW); - Assert.assertTrue(capability.isEmpty()); + Assertions.assertTrue(capability.isEmpty()); // Heartbeat to see if sub-cluster transitions to running storeHeartbeat = @@ -186,10 +186,10 @@ public void testFederationStateStoreService() throws Exception { private void explicitFailover(MockRM rm) throws IOException { rm.getAdminService().transitionToStandby(requestInfo); - Assert.assertTrue(rm.getRMContext() + Assertions.assertTrue(rm.getRMContext() .getHAServiceState() == HAServiceProtocol.HAServiceState.STANDBY); rm.getAdminService().transitionToActive(requestInfo); - Assert.assertTrue(rm.getRMContext() + Assertions.assertTrue(rm.getRMContext() .getHAServiceState() == HAServiceProtocol.HAServiceState.ACTIVE); lastHearbeatTS = 0; stateStore = rm.getFederationStateStoreService().getStateStoreClient(); @@ -199,23 +199,23 @@ private void checkClusterMetricsInfo(String capability, int numNodes) throws JAXBException { ClusterMetricsInfo clusterMetricsInfo = jsonUnmarshaller.unmarshalFromJSON( new StringReader(capability), ClusterMetricsInfo.class); - Assert.assertEquals(numNodes, clusterMetricsInfo.getTotalNodes()); + Assertions.assertEquals(numNodes, clusterMetricsInfo.getTotalNodes()); } private String checkSubClusterInfo(SubClusterState state) throws YarnException, UnknownHostException { - Assert.assertNotNull(stateStore.getSubCluster(request)); + Assertions.assertNotNull(stateStore.getSubCluster(request)); SubClusterInfo response = stateStore.getSubCluster(request).getSubClusterInfo(); - Assert.assertEquals(state, response.getState()); - Assert.assertTrue(response.getLastHeartBeat() >= lastHearbeatTS); + Assertions.assertEquals(state, response.getState()); + Assertions.assertTrue(response.getLastHeartBeat() >= lastHearbeatTS); String expectedAddress = (response.getClientRMServiceAddress().split(":"))[0]; - Assert.assertEquals(expectedAddress, + Assertions.assertEquals(expectedAddress, (response.getAMRMServiceAddress().split(":"))[0]); - Assert.assertEquals(expectedAddress, + Assertions.assertEquals(expectedAddress, (response.getRMAdminServiceAddress().split(":"))[0]); - Assert.assertEquals(expectedAddress, + Assertions.assertEquals(expectedAddress, (response.getRMWebServiceAddress().split(":"))[0]); lastHearbeatTS = response.getLastHeartBeat(); return response.getCapability(); @@ -236,12 +236,12 @@ public void testFederationStateStoreServiceInitialHeartbeatDelay() throws Except rm.init(conf); stateStore = rm.getFederationStateStoreService().getStateStoreClient(); GetSubClusterInfoResponse response = stateStore.getSubCluster(request); - Assert.assertNull(response); + Assertions.assertNull(response); // Validate if sub-cluster is registered rm.start(); String capability = checkSubClusterInfo(SubClusterState.SC_NEW); - Assert.assertTrue(capability.isEmpty()); + Assertions.assertTrue(capability.isEmpty()); // Heartbeat to see if sub-cluster transitions to running FederationStateStoreHeartbeat storeHeartbeat = @@ -250,7 +250,7 @@ public void testFederationStateStoreServiceInitialHeartbeatDelay() throws Except capability = checkSubClusterInfo(SubClusterState.SC_RUNNING); checkClusterMetricsInfo(capability, 0); - Assert.assertTrue(logCapture.getOutput().contains( + Assertions.assertTrue(logCapture.getOutput().contains( "Started federation membership heartbeat with interval: 300 and initial delay: 10")); rm.stop(); } @@ -285,16 +285,16 @@ public void testCleanUpApplication() throws Exception { GetApplicationHomeSubClusterRequest.newInstance(appId); GetApplicationHomeSubClusterResponse response = stateStore.getApplicationHomeSubCluster(appRequest); - Assert.assertNotNull(response); + Assertions.assertNotNull(response); ApplicationHomeSubCluster appHomeSubCluster = response.getApplicationHomeSubCluster(); - Assert.assertNotNull(appHomeSubCluster); - Assert.assertNotNull(appHomeSubCluster.getApplicationId()); - Assert.assertEquals(appId, appHomeSubCluster.getApplicationId()); + Assertions.assertNotNull(appHomeSubCluster); + Assertions.assertNotNull(appHomeSubCluster.getApplicationId()); + Assertions.assertEquals(appId, appHomeSubCluster.getApplicationId()); // clean up the app. boolean cleanUpResult = stateStoreService.cleanUpFinishApplicationsWithRetries(appId, true); - Assert.assertTrue(cleanUpResult); + Assertions.assertTrue(cleanUpResult); // after clean, the app can no longer be queried from the stateStore. LambdaTestUtils.intercept(FederationStateStoreException.class, @@ -344,10 +344,10 @@ public void testCleanUpApplicationWhenRMStart() throws Exception { GetApplicationsHomeSubClusterRequest.newInstance(subClusterId); GetApplicationsHomeSubClusterResponse allResponse = stateStore.getApplicationsHomeSubCluster(allRequest); - Assert.assertNotNull(allResponse); + Assertions.assertNotNull(allResponse); List appHomeSCLists = allResponse.getAppsHomeSubClusters(); - Assert.assertNotNull(appHomeSCLists); - Assert.assertEquals(3, appHomeSCLists.size()); + Assertions.assertNotNull(appHomeSCLists); + Assertions.assertEquals(3, appHomeSCLists.size()); // app04 exists in both RM memory and stateStore. ApplicationId appId04 = ApplicationId.newInstance(Time.now(), 4); @@ -363,7 +363,7 @@ public void testCleanUpApplicationWhenRMStart() throws Exception { try { List subClusters = getApplicationsFromStateStore(); - Assert.assertNotNull(subClusters); + Assertions.assertNotNull(subClusters); appsSize = subClusters.size(); } catch (YarnException e) { e.printStackTrace(); @@ -446,11 +446,11 @@ public void testCleanUpApplicationWhenRMCompleteOneApp() throws Exception { // app03 should remain in statestore List appHomeScList = getApplicationsFromStateStore(); - Assert.assertNotNull(appHomeScList); - Assert.assertEquals(1, appHomeScList.size()); + Assertions.assertNotNull(appHomeScList); + Assertions.assertEquals(1, appHomeScList.size()); ApplicationHomeSubCluster homeSubCluster = appHomeScList.get(0); - Assert.assertNotNull(homeSubCluster); - Assert.assertEquals(appId03, homeSubCluster.getApplicationId()); + Assertions.assertNotNull(homeSubCluster); + Assertions.assertEquals(appId03, homeSubCluster.getApplicationId()); } private void addApplication2StateStore(ApplicationId appId, @@ -468,9 +468,9 @@ private List getApplicationsFromStateStore() throws Y GetApplicationsHomeSubClusterRequest.newInstance(subClusterId); GetApplicationsHomeSubClusterResponse allResponse = stateStore.getApplicationsHomeSubCluster(allRequest); - Assert.assertNotNull(allResponse); + Assertions.assertNotNull(allResponse); List appHomeSCLists = allResponse.getAppsHomeSubClusters(); - Assert.assertNotNull(appHomeSCLists); + Assertions.assertNotNull(appHomeSCLists); return appHomeSCLists; } @@ -533,25 +533,25 @@ public void testPolicyConfigurationMethod() throws YarnException { GetSubClusterPolicyConfigurationRequest.newInstance(queue1); GetSubClusterPolicyConfigurationResponse response = stateStoreService.getPolicyConfiguration(request1); - Assert.assertNotNull(response); + Assertions.assertNotNull(response); SubClusterPolicyConfiguration responsePolicyConf = response.getPolicyConfiguration(); - Assert.assertNotNull(responsePolicyConf); - Assert.assertEquals(requestPolicyConf1, responsePolicyConf); + Assertions.assertNotNull(responsePolicyConf); + Assertions.assertEquals(requestPolicyConf1, responsePolicyConf); // get policy configurations GetSubClusterPoliciesConfigurationsRequest policiesRequest1 = GetSubClusterPoliciesConfigurationsRequest.newInstance(); GetSubClusterPoliciesConfigurationsResponse policiesResponse1 = stateStoreService.getPoliciesConfigurations(policiesRequest1); - Assert.assertNotNull(policiesResponse1); + Assertions.assertNotNull(policiesResponse1); List policiesConfigs = policiesResponse1.getPoliciesConfigs(); - Assert.assertNotNull(policiesConfigs); - Assert.assertEquals(2, policiesConfigs.size()); - Assert.assertTrue(policiesConfigs.contains(requestPolicyConf1)); - Assert.assertTrue(policiesConfigs.contains(requestPolicyConf2)); + Assertions.assertNotNull(policiesConfigs); + Assertions.assertEquals(2, policiesConfigs.size()); + Assertions.assertTrue(policiesConfigs.contains(requestPolicyConf1)); + Assertions.assertTrue(policiesConfigs.contains(requestPolicyConf2)); } public SubClusterPolicyConfiguration getUniformPolicy(String queue) @@ -596,12 +596,12 @@ public void testSubClusterMethod() throws YarnException { GetSubClusterInfoRequest.newInstance(subClusterId1); GetSubClusterInfoResponse subClusterResponse = stateStoreService.getSubCluster(subClusterRequest); - Assert.assertNotNull(subClusterResponse); + Assertions.assertNotNull(subClusterResponse); // We query subCluster1, we want to get SubClusterInfo of subCluster1 SubClusterInfo subClusterInfo1Resp = subClusterResponse.getSubClusterInfo(); - Assert.assertNotNull(subClusterInfo1Resp); - Assert.assertEquals(subClusterInfo1, subClusterInfo1Resp); + Assertions.assertNotNull(subClusterInfo1Resp); + Assertions.assertEquals(subClusterInfo1, subClusterInfo1Resp); // We call the getSubClusters method and filter the Active SubCluster // subCluster1 and subCluster2 are just registered, they are in NEW state, @@ -610,10 +610,10 @@ public void testSubClusterMethod() throws YarnException { GetSubClustersInfoRequest.newInstance(true); GetSubClustersInfoResponse subClustersInfoResp = stateStoreService.getSubClusters(subClustersInfoRequest); - Assert.assertNotNull(subClustersInfoResp); + Assertions.assertNotNull(subClustersInfoResp); List subClusterInfos = subClustersInfoResp.getSubClusters(); - Assert.assertNotNull(subClusterInfos); - Assert.assertEquals(0, subClusterInfos.size()); + Assertions.assertNotNull(subClusterInfos); + Assertions.assertEquals(0, subClusterInfos.size()); // We let subCluster1 heartbeat and set subCluster1 to Running state SubClusterHeartbeatRequest heartbeatRequest = @@ -621,7 +621,7 @@ public void testSubClusterMethod() throws YarnException { "capability"); SubClusterHeartbeatResponse heartbeatResponse = stateStoreService.subClusterHeartbeat(heartbeatRequest); - Assert.assertNotNull(heartbeatResponse); + Assertions.assertNotNull(heartbeatResponse); // We call the getSubClusters method again and filter the Active SubCluster // We want to get 1 active SubCluster @@ -629,10 +629,10 @@ public void testSubClusterMethod() throws YarnException { GetSubClustersInfoRequest.newInstance(true); GetSubClustersInfoResponse subClustersInfoResp1 = stateStoreService.getSubClusters(subClustersInfoRequest1); - Assert.assertNotNull(subClustersInfoResp1); + Assertions.assertNotNull(subClustersInfoResp1); List subClusterInfos1 = subClustersInfoResp1.getSubClusters(); - Assert.assertNotNull(subClusterInfos1); - Assert.assertEquals(1, subClusterInfos1.size()); + Assertions.assertNotNull(subClusterInfos1); + Assertions.assertEquals(1, subClusterInfos1.size()); } private SubClusterInfo createSubClusterInfo(SubClusterId clusterId) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/federation/TestFederationStateStoreServiceMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/federation/TestFederationStateStoreServiceMetrics.java index e7a79b843e938..f80aff10e12d6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/federation/TestFederationStateStoreServiceMetrics.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/federation/TestFederationStateStoreServiceMetrics.java @@ -17,11 +17,11 @@ package org.apache.hadoop.yarn.server.resourcemanager.federation; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; /** * Unit tests for TestFederationStateStoreServiceMetrics. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java index 6836288ed1cd1..beddbee6bdd62 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java @@ -60,10 +60,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType; import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -85,7 +86,7 @@ public void handle(SchedulerEvent event) { } } - @Before + @BeforeEach public void setUp() throws Exception { InlineDispatcher rmDispatcher = new InlineDispatcher(); @@ -120,7 +121,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { appId = ApplicationId.newInstance(System.currentTimeMillis(), 1); } - @After + @AfterEach public void tearDown() throws Exception { } @@ -155,12 +156,12 @@ public void testLogAggregationStatus() throws Exception { // should be NOT_STARTED Map logAggregationStatus = rmApp.getLogAggregationReportsForApp(); - Assert.assertEquals(2, logAggregationStatus.size()); - Assert.assertTrue(logAggregationStatus.containsKey(nodeId1)); - Assert.assertTrue(logAggregationStatus.containsKey(nodeId2)); + Assertions.assertEquals(2, logAggregationStatus.size()); + Assertions.assertTrue(logAggregationStatus.containsKey(nodeId1)); + Assertions.assertTrue(logAggregationStatus.containsKey(nodeId2)); for (Entry report : logAggregationStatus .entrySet()) { - Assert.assertEquals(LogAggregationStatus.NOT_START, report.getValue() + Assertions.assertEquals(LogAggregationStatus.NOT_START, report.getValue() .getLogAggregationStatus()); } @@ -195,24 +196,24 @@ public void testLogAggregationStatus() throws Exception { // verify that the log aggregation status for node1, node2 // has been changed logAggregationStatus = rmApp.getLogAggregationReportsForApp(); - Assert.assertEquals(2, logAggregationStatus.size()); - Assert.assertTrue(logAggregationStatus.containsKey(nodeId1)); - Assert.assertTrue(logAggregationStatus.containsKey(nodeId2)); + Assertions.assertEquals(2, logAggregationStatus.size()); + Assertions.assertTrue(logAggregationStatus.containsKey(nodeId1)); + Assertions.assertTrue(logAggregationStatus.containsKey(nodeId2)); for (Entry report : logAggregationStatus .entrySet()) { if (report.getKey().equals(node1.getNodeID())) { - Assert.assertEquals(LogAggregationStatus.RUNNING, report.getValue() + Assertions.assertEquals(LogAggregationStatus.RUNNING, report.getValue() .getLogAggregationStatus()); - Assert.assertEquals(messageForNode1_1, report.getValue() + Assertions.assertEquals(messageForNode1_1, report.getValue() .getDiagnosticMessage()); } else if (report.getKey().equals(node2.getNodeID())) { - Assert.assertEquals(LogAggregationStatus.RUNNING, report.getValue() + Assertions.assertEquals(LogAggregationStatus.RUNNING, report.getValue() .getLogAggregationStatus()); - Assert.assertEquals(messageForNode2_1, report.getValue() + Assertions.assertEquals(messageForNode2_1, report.getValue() .getDiagnosticMessage()); } else { // should not contain log aggregation report for other nodes - Assert + Assertions .fail("should not contain log aggregation report for other nodes"); } } @@ -234,25 +235,25 @@ public void testLogAggregationStatus() throws Exception { // verify that the log aggregation status for node2 // does not change logAggregationStatus = rmApp.getLogAggregationReportsForApp(); - Assert.assertEquals(2, logAggregationStatus.size()); - Assert.assertTrue(logAggregationStatus.containsKey(nodeId1)); - Assert.assertTrue(logAggregationStatus.containsKey(nodeId2)); + Assertions.assertEquals(2, logAggregationStatus.size()); + Assertions.assertTrue(logAggregationStatus.containsKey(nodeId1)); + Assertions.assertTrue(logAggregationStatus.containsKey(nodeId2)); for (Entry report : logAggregationStatus .entrySet()) { if (report.getKey().equals(node1.getNodeID())) { - Assert.assertEquals(LogAggregationStatus.RUNNING, report.getValue() + Assertions.assertEquals(LogAggregationStatus.RUNNING, report.getValue() .getLogAggregationStatus()); - Assert.assertEquals( + Assertions.assertEquals( messageForNode1_1 + "\n" + messageForNode1_2, report .getValue().getDiagnosticMessage()); } else if (report.getKey().equals(node2.getNodeID())) { - Assert.assertEquals(LogAggregationStatus.RUNNING, report.getValue() + Assertions.assertEquals(LogAggregationStatus.RUNNING, report.getValue() .getLogAggregationStatus()); - Assert.assertEquals(messageForNode2_1, report.getValue() + Assertions.assertEquals(messageForNode2_1, report.getValue() .getDiagnosticMessage()); } else { // should not contain log aggregation report for other nodes - Assert + Assertions .fail("should not contain log aggregation report for other nodes"); } } @@ -261,7 +262,7 @@ public void testLogAggregationStatus() throws Exception { rmApp.handle(new RMAppEvent(appId, RMAppEventType.KILL)); rmApp.handle(new RMAppEvent(appId, RMAppEventType.ATTEMPT_KILLED)); rmApp.handle(new RMAppEvent(appId, RMAppEventType.APP_UPDATE_SAVED)); - Assert.assertEquals(RMAppState.KILLED, rmApp.getState()); + Assertions.assertEquals(RMAppState.KILLED, rmApp.getState()); // wait for 1500 ms Thread.sleep(1500); @@ -269,12 +270,12 @@ public void testLogAggregationStatus() throws Exception { // the log aggregation status for both nodes should be changed // to TIME_OUT logAggregationStatus = rmApp.getLogAggregationReportsForApp(); - Assert.assertEquals(2, logAggregationStatus.size()); - Assert.assertTrue(logAggregationStatus.containsKey(nodeId1)); - Assert.assertTrue(logAggregationStatus.containsKey(nodeId2)); + Assertions.assertEquals(2, logAggregationStatus.size()); + Assertions.assertTrue(logAggregationStatus.containsKey(nodeId1)); + Assertions.assertTrue(logAggregationStatus.containsKey(nodeId2)); for (Entry report : logAggregationStatus .entrySet()) { - Assert.assertEquals(LogAggregationStatus.TIME_OUT, report.getValue() + Assertions.assertEquals(LogAggregationStatus.TIME_OUT, report.getValue() .getLogAggregationStatus()); } @@ -298,13 +299,13 @@ public void testLogAggregationStatus() throws Exception { node1ReportForApp3)); logAggregationStatus = rmApp.getLogAggregationReportsForApp(); - Assert.assertEquals(2, logAggregationStatus.size()); - Assert.assertTrue(logAggregationStatus.containsKey(nodeId1)); - Assert.assertTrue(logAggregationStatus.containsKey(nodeId2)); + Assertions.assertEquals(2, logAggregationStatus.size()); + Assertions.assertTrue(logAggregationStatus.containsKey(nodeId1)); + Assertions.assertTrue(logAggregationStatus.containsKey(nodeId2)); for (Entry report : logAggregationStatus .entrySet()) { if (report.getKey().equals(node1.getNodeID())) { - Assert.assertEquals(LogAggregationStatus.SUCCEEDED, report.getValue() + Assertions.assertEquals(LogAggregationStatus.SUCCEEDED, report.getValue() .getLogAggregationStatus()); StringBuilder builder = new StringBuilder(); for (int i = 0; i < 9; i ++) { @@ -312,14 +313,14 @@ public void testLogAggregationStatus() throws Exception { builder.append("\n"); } builder.append("test_message_" + 9); - Assert.assertEquals(builder.toString(), report.getValue() + Assertions.assertEquals(builder.toString(), report.getValue() .getDiagnosticMessage()); } else if (report.getKey().equals(node2.getNodeID())) { - Assert.assertEquals(LogAggregationStatus.TIME_OUT, report.getValue() + Assertions.assertEquals(LogAggregationStatus.TIME_OUT, report.getValue() .getLogAggregationStatus()); } else { // should not contain log aggregation report for other nodes - Assert + Assertions .fail("should not contain log aggregation report for other nodes"); } } @@ -340,17 +341,18 @@ public void testLogAggregationStatus() throws Exception { node2ReportForApp2.add(report2_3); node2.handle(new RMNodeStatusEvent(node2.getNodeID(), nodeStatus2, node2ReportForApp2)); - Assert.assertEquals(LogAggregationStatus.FAILED, + Assertions.assertEquals(LogAggregationStatus.FAILED, rmApp.getLogAggregationStatusForAppReport()); logAggregationStatus = rmApp.getLogAggregationReportsForApp(); - Assert.assertTrue(logAggregationStatus.size() == 1); - Assert.assertTrue(logAggregationStatus.containsKey(node2.getNodeID())); - Assert.assertTrue(!logAggregationStatus.containsKey(node1.getNodeID())); - Assert.assertEquals("Fail_Message", + Assertions.assertTrue(logAggregationStatus.size() == 1); + Assertions.assertTrue(logAggregationStatus.containsKey(node2.getNodeID())); + Assertions.assertTrue(!logAggregationStatus.containsKey(node1.getNodeID())); + Assertions.assertEquals("Fail_Message", ((RMAppImpl)rmApp).getLogAggregationFailureMessagesForNM(nodeId2)); } - @Test (timeout = 10000) + @Test + @Timeout(value = 10) public void testGetLogAggregationStatusForAppReport() { YarnConfiguration conf = new YarnConfiguration(); @@ -358,7 +360,7 @@ public void testGetLogAggregationStatusForAppReport() { conf.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, false); RMAppImpl rmApp = (RMAppImpl)createRMApp(conf); // The log aggregation status should be DISABLED. - Assert.assertEquals(LogAggregationStatus.DISABLED, + Assertions.assertEquals(LogAggregationStatus.DISABLED, rmApp.getLogAggregationStatusForAppReport()); // Enable the log aggregation @@ -367,7 +369,7 @@ public void testGetLogAggregationStatusForAppReport() { // If we do not know any NodeManagers for this application , and // the log aggregation is enabled, the log aggregation status will // return NOT_START - Assert.assertEquals(LogAggregationStatus.NOT_START, + Assertions.assertEquals(LogAggregationStatus.NOT_START, rmApp.getLogAggregationStatusForAppReport()); NodeId nodeId1 = NodeId.newInstance("localhost", 1111); @@ -385,7 +387,7 @@ public void testGetLogAggregationStatusForAppReport() { rmApp.getApplicationId(), LogAggregationStatus.NOT_START, "")); rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance( rmApp.getApplicationId(), LogAggregationStatus.NOT_START, "")); - Assert.assertEquals(LogAggregationStatus.NOT_START, + Assertions.assertEquals(LogAggregationStatus.NOT_START, rmApp.getLogAggregationStatusForAppReport()); rmApp.aggregateLogReport(nodeId1, LogAggregationReport.newInstance( @@ -396,11 +398,11 @@ public void testGetLogAggregationStatusForAppReport() { rmApp.getApplicationId(), LogAggregationStatus.SUCCEEDED, "")); rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance( rmApp.getApplicationId(), LogAggregationStatus.SUCCEEDED, "")); - Assert.assertEquals(LogAggregationStatus.RUNNING, + Assertions.assertEquals(LogAggregationStatus.RUNNING, rmApp.getLogAggregationStatusForAppReport()); rmApp.handle(new RMAppEvent(rmApp.getApplicationId(), RMAppEventType.KILL)); - Assert.assertTrue(RMAppImpl.isAppInFinalState(rmApp)); + Assertions.assertTrue(RMAppImpl.isAppInFinalState(rmApp)); // If at least of one log aggregation status for one NM is TIME_OUT, // others are SUCCEEDED, the log aggregation status for this app will @@ -413,7 +415,7 @@ public void testGetLogAggregationStatusForAppReport() { rmApp.getApplicationId(), LogAggregationStatus.SUCCEEDED, "")); rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance( rmApp.getApplicationId(), LogAggregationStatus.SUCCEEDED, "")); - Assert.assertEquals(LogAggregationStatus.TIME_OUT, + Assertions.assertEquals(LogAggregationStatus.TIME_OUT, rmApp.getLogAggregationStatusForAppReport()); rmApp = (RMAppImpl)createRMApp(conf); @@ -429,7 +431,7 @@ public void testGetLogAggregationStatusForAppReport() { rmApp.getApplicationId(), LogAggregationStatus.SUCCEEDED, "")); rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance( rmApp.getApplicationId(), LogAggregationStatus.SUCCEEDED, "")); - Assert.assertEquals(LogAggregationStatus.SUCCEEDED, + Assertions.assertEquals(LogAggregationStatus.SUCCEEDED, rmApp.getLogAggregationStatusForAppReport()); rmApp = (RMAppImpl)createRMApp(conf); @@ -443,7 +445,7 @@ public void testGetLogAggregationStatusForAppReport() { rmApp.getApplicationId(), LogAggregationStatus.NOT_START, "")); rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance( rmApp.getApplicationId(), LogAggregationStatus.NOT_START, "")); - Assert.assertEquals(LogAggregationStatus.RUNNING, + Assertions.assertEquals(LogAggregationStatus.RUNNING, rmApp.getLogAggregationStatusForAppReport()); // If the log aggregation status for at least one of NMs @@ -458,7 +460,7 @@ public void testGetLogAggregationStatusForAppReport() { rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance( rmApp.getApplicationId(), LogAggregationStatus.RUNNING_WITH_FAILURE, "")); - Assert.assertEquals(LogAggregationStatus.RUNNING_WITH_FAILURE, + Assertions.assertEquals(LogAggregationStatus.RUNNING_WITH_FAILURE, rmApp.getLogAggregationStatusForAppReport()); // For node4, the previous log aggregation status is RUNNING_WITH_FAILURE, @@ -472,11 +474,11 @@ public void testGetLogAggregationStatusForAppReport() { rmApp.getApplicationId(), LogAggregationStatus.NOT_START, "")); rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance( rmApp.getApplicationId(), LogAggregationStatus.RUNNING, "")); - Assert.assertEquals(LogAggregationStatus.RUNNING_WITH_FAILURE, + Assertions.assertEquals(LogAggregationStatus.RUNNING_WITH_FAILURE, rmApp.getLogAggregationStatusForAppReport()); rmApp.handle(new RMAppEvent(rmApp.getApplicationId(), RMAppEventType.KILL)); - Assert.assertTrue(RMAppImpl.isAppInFinalState(rmApp)); + Assertions.assertTrue(RMAppImpl.isAppInFinalState(rmApp)); // If at least of one log aggregation status for one NM is FAILED, // others are either SUCCEEDED or TIME_OUT, and this application is // at the final state, the log aggregation status for this app @@ -489,7 +491,7 @@ public void testGetLogAggregationStatusForAppReport() { rmApp.getApplicationId(), LogAggregationStatus.FAILED, "")); rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance( rmApp.getApplicationId(), LogAggregationStatus.FAILED, "")); - Assert.assertEquals(LogAggregationStatus.FAILED, + Assertions.assertEquals(LogAggregationStatus.FAILED, rmApp.getLogAggregationStatusForAppReport()); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestCombinedSystemMetricsPublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestCombinedSystemMetricsPublisher.java index 63f007b45b923..c657ca845af72 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestCombinedSystemMetricsPublisher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestCombinedSystemMetricsPublisher.java @@ -18,8 +18,8 @@ package org.apache.hadoop.yarn.server.resourcemanager.metrics; -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; @@ -62,8 +62,9 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter; import org.apache.hadoop.yarn.util.TimelineServiceHelper; import org.apache.hadoop.yarn.util.timeline.TimelineUtils; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; /** * Tests that a CombinedSystemMetricsPublisher publishes metrics for timeline @@ -132,18 +133,18 @@ private void testSetup(boolean enableV1, boolean enableV2) throws Exception { new ArrayList(); if (YarnConfiguration.timelineServiceV1Enabled(conf)) { - Assert.assertTrue(enableV1); + Assertions.assertTrue(enableV1); publisherV1 = new TimelineServiceV1Publisher(); publishers.add(publisherV1); publisherV1.init(conf); publisherV1.start(); } else { - Assert.assertFalse(enableV1); + Assertions.assertFalse(enableV1); publisherV1 = null; } if (YarnConfiguration.timelineServiceV2Enabled(conf)) { - Assert.assertTrue(enableV2); + Assertions.assertTrue(enableV2); publisherV2 = new TimelineServiceV2Publisher( rmTimelineCollectorManager) { @Override @@ -155,7 +156,7 @@ protected Dispatcher getDispatcher() { publisherV2.init(conf); publisherV2.start(); } else { - Assert.assertFalse(enableV2); + Assertions.assertFalse(enableV2); publisherV2 = null; } @@ -220,7 +221,7 @@ private static YarnConfiguration getConf(boolean v1Enabled, testRootDir.getCanonicalPath()); } catch (IOException e) { e.printStackTrace(); - Assert.fail("Exception while setting the " + + Assertions.fail("Exception while setting the " + "TIMELINE_SERVICE_STORAGE_DIR_ROOT "); } } @@ -246,28 +247,33 @@ private void runTest(boolean v1Enabled, boolean v2Enabled) throws Exception { testCleanup(); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testTimelineServiceEventPublishingV1V2Enabled() throws Exception { runTest(true, true); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testTimelineServiceEventPublishingV1Enabled() throws Exception { runTest(true, false); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testTimelineServiceEventPublishingV2Enabled() throws Exception { runTest(false, true); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testTimelineServiceEventPublishingNoService() throws Exception { runTest(false, false); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testTimelineServiceConfiguration() throws Exception { Configuration config = new Configuration(false); @@ -275,21 +281,21 @@ public void testTimelineServiceConfiguration() config.set(YarnConfiguration.TIMELINE_SERVICE_VERSIONS, "2.0,1.5"); config.set(YarnConfiguration.TIMELINE_SERVICE_VERSION, "2.0"); - Assert.assertTrue(YarnConfiguration.timelineServiceV2Enabled(config)); - Assert.assertTrue(YarnConfiguration.timelineServiceV15Enabled(config)); - Assert.assertTrue(YarnConfiguration.timelineServiceV1Enabled(config)); + Assertions.assertTrue(YarnConfiguration.timelineServiceV2Enabled(config)); + Assertions.assertTrue(YarnConfiguration.timelineServiceV15Enabled(config)); + Assertions.assertTrue(YarnConfiguration.timelineServiceV1Enabled(config)); config.set(YarnConfiguration.TIMELINE_SERVICE_VERSIONS, "2.0,1"); config.set(YarnConfiguration.TIMELINE_SERVICE_VERSION, "1.5"); - Assert.assertTrue(YarnConfiguration.timelineServiceV2Enabled(config)); - Assert.assertFalse(YarnConfiguration.timelineServiceV15Enabled(config)); - Assert.assertTrue(YarnConfiguration.timelineServiceV1Enabled(config)); + Assertions.assertTrue(YarnConfiguration.timelineServiceV2Enabled(config)); + Assertions.assertFalse(YarnConfiguration.timelineServiceV15Enabled(config)); + Assertions.assertTrue(YarnConfiguration.timelineServiceV1Enabled(config)); config.set(YarnConfiguration.TIMELINE_SERVICE_VERSIONS, "2.0"); config.set(YarnConfiguration.TIMELINE_SERVICE_VERSION, "1.5"); - Assert.assertTrue(YarnConfiguration.timelineServiceV2Enabled(config)); - Assert.assertFalse(YarnConfiguration.timelineServiceV15Enabled(config)); - Assert.assertFalse(YarnConfiguration.timelineServiceV1Enabled(config)); + Assertions.assertTrue(YarnConfiguration.timelineServiceV2Enabled(config)); + Assertions.assertFalse(YarnConfiguration.timelineServiceV15Enabled(config)); + Assertions.assertFalse(YarnConfiguration.timelineServiceV1Enabled(config)); } private void publishEvents(boolean v1Enabled, boolean v2Enabled) { @@ -326,7 +332,7 @@ private void validateV1(boolean v1Enabled) throws Exception { store.getEntity(appAttemptId.toString(), AppAttemptMetricsConstants.ENTITY_TYPE, EnumSet.allOf(Field.class)); - Assert.assertNull(entity); + Assertions.assertNull(entity); return; } @@ -349,19 +355,19 @@ private void validateV1(boolean v1Enabled) throws Exception { } else if (event.getEventType().equals( AppAttemptMetricsConstants.FINISHED_EVENT_TYPE)) { hasFinishedEvent = true; - Assert.assertEquals( + Assertions.assertEquals( FinalApplicationStatus.UNDEFINED.toString(), event.getEventInfo().get( AppAttemptMetricsConstants.FINAL_STATUS_INFO)); - Assert.assertEquals( + Assertions.assertEquals( YarnApplicationAttemptState.FINISHED.toString(), event.getEventInfo().get( AppAttemptMetricsConstants.STATE_INFO)); } - Assert + Assertions .assertEquals(appAttemptId.toString(), entity.getEntityId()); } - Assert.assertTrue(hasRegisteredEvent && hasFinishedEvent); + Assertions.assertTrue(hasRegisteredEvent && hasFinishedEvent); } private void validateV2(boolean v2Enabled) throws Exception { @@ -370,13 +376,13 @@ private void validateV2(boolean v2Enabled) throws Exception { + TimelineEntityType.YARN_APPLICATION_ATTEMPT + "/"; File entityFolder = new File(outputDirApp); - Assert.assertEquals(v2Enabled, entityFolder.isDirectory()); + Assertions.assertEquals(v2Enabled, entityFolder.isDirectory()); if (v2Enabled) { String timelineServiceFileName = appAttemptId.toString() + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION; File entityFile = new File(outputDirApp, timelineServiceFileName); - Assert.assertTrue(entityFile.exists()); + Assertions.assertTrue(entityFile.exists()); long idPrefix = TimelineServiceHelper .invertLong(appAttemptId.getAttemptId()); verifyEntity(entityFile, 2, @@ -414,10 +420,10 @@ private void verifyEntity(File entityFile, long expectedEvents, } finally { reader.close(); } - assertEquals("Expected " + expectedEvents + " events to be published", - expectedEvents, count); - assertEquals("Expected " + expectedMetrics + " metrics is incorrect", - expectedMetrics, metricsCount); + assertEquals( + expectedEvents, count, "Expected " + expectedEvents + " events to be published"); + assertEquals( + expectedMetrics, metricsCount, "Expected " + expectedMetrics + " metrics is incorrect"); } private String getTimelineEntityDir() { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java index 146a931e5acef..135332eec1041 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java @@ -27,10 +27,11 @@ import java.util.Map; import java.util.Set; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; @@ -96,7 +97,7 @@ public TestSystemMetricsPublisher(boolean rmTimelineServerV1PublisherBatchEnable this.rmTimelineServerV1PublisherInterval = rmTimelineServerV1PublisherInterval; } - @Before + @BeforeEach public void setup() throws Exception { YarnConfiguration conf = new YarnConfiguration(); conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); @@ -123,7 +124,7 @@ public void setup() throws Exception { metricsPublisher.start(); } - @After + @AfterEach public void tearDown() throws Exception { if (metricsPublisher != null) { metricsPublisher.stop(); @@ -133,7 +134,8 @@ public void tearDown() throws Exception { } } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testPublishApplicationMetrics() throws Exception { long stateUpdateTimeStamp = System.currentTimeMillis(); for (int i = 1; i <= 2; ++i) { @@ -178,92 +180,92 @@ public void testPublishApplicationMetrics() throws Exception { // ensure Five events are both published before leaving the loop } while (entity == null || entity.getEvents().size() < 6); // verify all the fields - Assert.assertEquals(ApplicationMetricsConstants.ENTITY_TYPE, + Assertions.assertEquals(ApplicationMetricsConstants.ENTITY_TYPE, entity.getEntityType()); - Assert + Assertions .assertEquals(app.getApplicationId().toString(), entity.getEntityId()); - Assert + Assertions .assertEquals( app.getName(), entity.getOtherInfo().get( ApplicationMetricsConstants.NAME_ENTITY_INFO)); if (i != 1) { - Assert.assertEquals( + Assertions.assertEquals( app.getQueue(), entity.getOtherInfo().get( ApplicationMetricsConstants.QUEUE_ENTITY_INFO)); } - Assert.assertEquals( + Assertions.assertEquals( app.getApplicationSubmissionContext().getUnmanagedAM(), entity.getOtherInfo().get( ApplicationMetricsConstants.UNMANAGED_APPLICATION_ENTITY_INFO)); if (i != 1) { - Assert.assertEquals( + Assertions.assertEquals( app.getApplicationSubmissionContext().getPriority().getPriority(), entity.getOtherInfo().get( ApplicationMetricsConstants.APPLICATION_PRIORITY_INFO)); } - Assert.assertEquals(app.getAmNodeLabelExpression(), entity.getOtherInfo() + Assertions.assertEquals(app.getAmNodeLabelExpression(), entity.getOtherInfo() .get(ApplicationMetricsConstants.AM_NODE_LABEL_EXPRESSION)); - Assert.assertEquals( + Assertions.assertEquals( app.getApplicationSubmissionContext().getNodeLabelExpression(), entity.getOtherInfo() .get(ApplicationMetricsConstants.APP_NODE_LABEL_EXPRESSION)); - Assert + Assertions .assertEquals( app.getUser(), entity.getOtherInfo().get( ApplicationMetricsConstants.USER_ENTITY_INFO)); - Assert + Assertions .assertEquals( app.getApplicationType(), entity.getOtherInfo().get( ApplicationMetricsConstants.TYPE_ENTITY_INFO)); - Assert.assertEquals(app.getSubmitTime(), + Assertions.assertEquals(app.getSubmitTime(), entity.getOtherInfo().get( ApplicationMetricsConstants.SUBMITTED_TIME_ENTITY_INFO)); - Assert.assertTrue(verifyAppTags(app.getApplicationTags(), + Assertions.assertTrue(verifyAppTags(app.getApplicationTags(), entity.getOtherInfo())); if (i == 1) { - Assert.assertEquals("uers1,user2", + Assertions.assertEquals("uers1,user2", entity.getOtherInfo().get( ApplicationMetricsConstants.APP_VIEW_ACLS_ENTITY_INFO)); - Assert.assertEquals( + Assertions.assertEquals( app.getApplicationSubmissionContext().getAMContainerSpec() .getCommands(), entity.getOtherInfo() .get(ApplicationMetricsConstants.AM_CONTAINER_LAUNCH_COMMAND)); } else { - Assert.assertEquals( + Assertions.assertEquals( "", entity.getOtherInfo().get( ApplicationMetricsConstants.APP_VIEW_ACLS_ENTITY_INFO)); - Assert.assertEquals( + Assertions.assertEquals( app.getRMAppMetrics().getMemorySeconds(), Long.parseLong(entity.getOtherInfo() .get(ApplicationMetricsConstants.APP_MEM_METRICS).toString())); - Assert.assertEquals( + Assertions.assertEquals( app.getRMAppMetrics().getVcoreSeconds(), Long.parseLong(entity.getOtherInfo() .get(ApplicationMetricsConstants.APP_CPU_METRICS).toString())); - Assert.assertEquals( + Assertions.assertEquals( app.getRMAppMetrics().getPreemptedMemorySeconds(), Long.parseLong(entity.getOtherInfo() .get(ApplicationMetricsConstants.APP_MEM_PREEMPT_METRICS) .toString())); - Assert.assertEquals( + Assertions.assertEquals( app.getRMAppMetrics().getPreemptedVcoreSeconds(), Long.parseLong(entity.getOtherInfo() .get(ApplicationMetricsConstants.APP_CPU_PREEMPT_METRICS) .toString())); } - Assert.assertEquals("context", entity.getOtherInfo() + Assertions.assertEquals("context", entity.getOtherInfo() .get(ApplicationMetricsConstants.YARN_APP_CALLER_CONTEXT)); boolean hasCreatedEvent = false; boolean hasLaunchedEvent = false; @@ -275,35 +277,35 @@ public void testPublishApplicationMetrics() throws Exception { if (event.getEventType().equals( ApplicationMetricsConstants.CREATED_EVENT_TYPE)) { hasCreatedEvent = true; - Assert.assertEquals(app.getStartTime(), event.getTimestamp()); + Assertions.assertEquals(app.getStartTime(), event.getTimestamp()); } else if (event.getEventType().equals( ApplicationMetricsConstants.LAUNCHED_EVENT_TYPE)) { hasLaunchedEvent = true; - Assert.assertEquals(app.getLaunchTime(), event.getTimestamp()); + Assertions.assertEquals(app.getLaunchTime(), event.getTimestamp()); } else if (event.getEventType().equals( ApplicationMetricsConstants.FINISHED_EVENT_TYPE)) { hasFinishedEvent = true; - Assert.assertEquals(app.getFinishTime(), event.getTimestamp()); - Assert.assertEquals( + Assertions.assertEquals(app.getFinishTime(), event.getTimestamp()); + Assertions.assertEquals( app.getDiagnostics().toString(), event.getEventInfo().get( ApplicationMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO)); - Assert.assertEquals( + Assertions.assertEquals( app.getFinalApplicationStatus().toString(), event.getEventInfo().get( ApplicationMetricsConstants.FINAL_STATUS_EVENT_INFO)); - Assert.assertEquals(YarnApplicationState.FINISHED.toString(), event + Assertions.assertEquals(YarnApplicationState.FINISHED.toString(), event .getEventInfo().get(ApplicationMetricsConstants.STATE_EVENT_INFO)); } else if (event.getEventType().equals( ApplicationMetricsConstants.UPDATED_EVENT_TYPE)) { hasUpdatedEvent = true; - Assert.assertEquals(4L, event.getTimestamp()); + Assertions.assertEquals(4L, event.getTimestamp()); if (1 == i) { - Assert.assertEquals( + Assertions.assertEquals( 1, event.getEventInfo().get( ApplicationMetricsConstants.APPLICATION_PRIORITY_INFO)); - Assert.assertEquals( + Assertions.assertEquals( "new test queue", event.getEventInfo().get( ApplicationMetricsConstants.QUEUE_ENTITY_INFO)); @@ -311,27 +313,28 @@ public void testPublishApplicationMetrics() throws Exception { } else if (event.getEventType().equals( ApplicationMetricsConstants.ACLS_UPDATED_EVENT_TYPE)) { hasACLsUpdatedEvent = true; - Assert.assertEquals(4L, event.getTimestamp()); + Assertions.assertEquals(4L, event.getTimestamp()); } else if (event.getEventType().equals( ApplicationMetricsConstants.STATE_UPDATED_EVENT_TYPE)) { hasStateUpdateEvent = true; assertThat(event.getTimestamp()).isEqualTo(stateUpdateTimeStamp); - Assert.assertEquals(YarnApplicationState.RUNNING.toString(), event + Assertions.assertEquals(YarnApplicationState.RUNNING.toString(), event .getEventInfo().get( ApplicationMetricsConstants.STATE_EVENT_INFO)); } } // Do assertTrue verification separately for easier debug - Assert.assertTrue(hasCreatedEvent); - Assert.assertTrue(hasLaunchedEvent); - Assert.assertTrue(hasFinishedEvent); - Assert.assertTrue(hasACLsUpdatedEvent); - Assert.assertTrue(hasUpdatedEvent); - Assert.assertTrue(hasStateUpdateEvent); + Assertions.assertTrue(hasCreatedEvent); + Assertions.assertTrue(hasLaunchedEvent); + Assertions.assertTrue(hasFinishedEvent); + Assertions.assertTrue(hasACLsUpdatedEvent); + Assertions.assertTrue(hasUpdatedEvent); + Assertions.assertTrue(hasStateUpdateEvent); } } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testPublishAppAttemptMetricsForUnmanagedAM() throws Exception { ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 1), 1); @@ -351,7 +354,8 @@ public void testPublishAppAttemptMetricsForUnmanagedAM() throws Exception { } while (entity == null || entity.getEvents().size() < 2); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testPublishAppAttemptMetrics() throws Exception { ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 1), 1); @@ -370,10 +374,10 @@ public void testPublishAppAttemptMetrics() throws Exception { // ensure two events are both published before leaving the loop } while (entity == null || entity.getEvents().size() < 2); // verify all the fields - Assert.assertEquals(AppAttemptMetricsConstants.ENTITY_TYPE, + Assertions.assertEquals(AppAttemptMetricsConstants.ENTITY_TYPE, entity.getEntityType()); - Assert.assertEquals(appAttemptId.toString(), entity.getEntityId()); - Assert.assertEquals( + Assertions.assertEquals(appAttemptId.toString(), entity.getEntityId()); + Assertions.assertEquals( appAttemptId.getApplicationId().toString(), entity.getPrimaryFilters() .get(AppAttemptMetricsConstants.PARENT_PRIMARY_FILTER).iterator() @@ -384,42 +388,43 @@ public void testPublishAppAttemptMetrics() throws Exception { if (event.getEventType().equals( AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE)) { hasRegisteredEvent = true; - Assert.assertEquals(appAttempt.getHost(), + Assertions.assertEquals(appAttempt.getHost(), event.getEventInfo() .get(AppAttemptMetricsConstants.HOST_INFO)); - Assert + Assertions .assertEquals(appAttempt.getRpcPort(), event.getEventInfo().get( AppAttemptMetricsConstants.RPC_PORT_INFO)); - Assert.assertEquals( + Assertions.assertEquals( appAttempt.getMasterContainer().getId().toString(), event.getEventInfo().get( AppAttemptMetricsConstants.MASTER_CONTAINER_INFO)); } else if (event.getEventType().equals( AppAttemptMetricsConstants.FINISHED_EVENT_TYPE)) { hasFinishedEvent = true; - Assert.assertEquals(appAttempt.getDiagnostics(), event.getEventInfo() + Assertions.assertEquals(appAttempt.getDiagnostics(), event.getEventInfo() .get(AppAttemptMetricsConstants.DIAGNOSTICS_INFO)); - Assert.assertEquals(appAttempt.getTrackingUrl(), event.getEventInfo() + Assertions.assertEquals(appAttempt.getTrackingUrl(), event.getEventInfo() .get(AppAttemptMetricsConstants.TRACKING_URL_INFO)); - Assert.assertEquals( + Assertions.assertEquals( appAttempt.getOriginalTrackingUrl(), event.getEventInfo().get( AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_INFO)); - Assert.assertEquals( + Assertions.assertEquals( FinalApplicationStatus.UNDEFINED.toString(), event.getEventInfo().get( AppAttemptMetricsConstants.FINAL_STATUS_INFO)); - Assert.assertEquals( + Assertions.assertEquals( YarnApplicationAttemptState.FINISHED.toString(), event.getEventInfo().get( AppAttemptMetricsConstants.STATE_INFO)); } } - Assert.assertTrue(hasRegisteredEvent && hasFinishedEvent); + Assertions.assertTrue(hasRegisteredEvent && hasFinishedEvent); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testPublishHostPortInfoOnContainerFinished() throws Exception { ContainerId containerId = ContainerId.newContainerId(ApplicationAttemptId.newInstance( @@ -433,23 +438,24 @@ public void testPublishHostPortInfoOnContainerFinished() throws Exception { ContainerMetricsConstants.ENTITY_TYPE, EnumSet.allOf(Field.class)); } while (entity == null || entity.getEvents().size() < 1); - Assert.assertNotNull(entity.getOtherInfo()); - Assert.assertEquals(2, entity.getOtherInfo().size()); - Assert.assertNotNull(entity.getOtherInfo().get( + Assertions.assertNotNull(entity.getOtherInfo()); + Assertions.assertEquals(2, entity.getOtherInfo().size()); + Assertions.assertNotNull(entity.getOtherInfo().get( ContainerMetricsConstants.ALLOCATED_HOST_INFO)); - Assert.assertNotNull(entity.getOtherInfo().get( + Assertions.assertNotNull(entity.getOtherInfo().get( ContainerMetricsConstants.ALLOCATED_PORT_INFO)); - Assert.assertEquals( + Assertions.assertEquals( container.getAllocatedNode().getHost(), entity.getOtherInfo().get( ContainerMetricsConstants.ALLOCATED_HOST_INFO)); - Assert.assertEquals( + Assertions.assertEquals( container.getAllocatedNode().getPort(), entity.getOtherInfo().get( ContainerMetricsConstants.ALLOCATED_PORT_INFO)); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testPublishContainerMetrics() throws Exception { ContainerId containerId = ContainerId.newContainerId(ApplicationAttemptId.newInstance( @@ -466,33 +472,33 @@ public void testPublishContainerMetrics() throws Exception { // ensure two events are both published before leaving the loop } while (entity == null || entity.getEvents().size() < 2); // verify all the fields - Assert.assertEquals(ContainerMetricsConstants.ENTITY_TYPE, + Assertions.assertEquals(ContainerMetricsConstants.ENTITY_TYPE, entity.getEntityType()); - Assert.assertEquals(containerId.toString(), entity.getEntityId()); - Assert.assertEquals( + Assertions.assertEquals(containerId.toString(), entity.getEntityId()); + Assertions.assertEquals( containerId.getApplicationAttemptId().toString(), entity.getPrimaryFilters() .get(ContainerMetricsConstants.PARENT_PRIMARIY_FILTER).iterator() .next()); - Assert.assertEquals( + Assertions.assertEquals( container.getAllocatedNode().getHost(), entity.getOtherInfo().get( ContainerMetricsConstants.ALLOCATED_HOST_INFO)); - Assert.assertEquals( + Assertions.assertEquals( container.getAllocatedNode().getPort(), entity.getOtherInfo().get( ContainerMetricsConstants.ALLOCATED_PORT_INFO)); - Assert.assertEquals(container.getAllocatedResource().getMemorySize(), + Assertions.assertEquals(container.getAllocatedResource().getMemorySize(), // KeyValueBasedTimelineStore could cast long to integer, need make sure // variables for compare have same type. ((Integer) entity.getOtherInfo().get( ContainerMetricsConstants.ALLOCATED_MEMORY_INFO)) .longValue()); - Assert.assertEquals( + Assertions.assertEquals( container.getAllocatedResource().getVirtualCores(), entity.getOtherInfo().get( ContainerMetricsConstants.ALLOCATED_VCORE_INFO)); - Assert.assertEquals( + Assertions.assertEquals( container.getAllocatedPriority().getPriority(), entity.getOtherInfo().get( ContainerMetricsConstants.ALLOCATED_PRIORITY_INFO)); @@ -502,24 +508,24 @@ public void testPublishContainerMetrics() throws Exception { if (event.getEventType().equals( ContainerMetricsConstants.CREATED_EVENT_TYPE)) { hasCreatedEvent = true; - Assert.assertEquals(container.getCreationTime(), event.getTimestamp()); + Assertions.assertEquals(container.getCreationTime(), event.getTimestamp()); } else if (event.getEventType().equals( ContainerMetricsConstants.FINISHED_EVENT_TYPE)) { hasFinishedEvent = true; - Assert.assertEquals(container.getFinishTime(), event.getTimestamp()); - Assert.assertEquals( + Assertions.assertEquals(container.getFinishTime(), event.getTimestamp()); + Assertions.assertEquals( container.getDiagnosticsInfo(), event.getEventInfo().get( ContainerMetricsConstants.DIAGNOSTICS_INFO)); - Assert.assertEquals( + Assertions.assertEquals( container.getContainerExitStatus(), event.getEventInfo().get( ContainerMetricsConstants.EXIT_STATUS_INFO)); - Assert.assertEquals(container.getContainerState().toString(), event + Assertions.assertEquals(container.getContainerState().toString(), event .getEventInfo().get(ContainerMetricsConstants.STATE_INFO)); } } - Assert.assertTrue(hasCreatedEvent && hasFinishedEvent); + Assertions.assertTrue(hasCreatedEvent && hasFinishedEvent); } private static RMApp createRMApp(ApplicationId 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/metrics/TestSystemMetricsPublisherForV2.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java index a1989d5c0c2cc..fe566c611d1c5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java @@ -18,9 +18,9 @@ package org.apache.hadoop.yarn.server.resourcemanager.metrics; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -79,10 +79,11 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter; import org.apache.hadoop.yarn.util.TimelineServiceHelper; import org.apache.hadoop.yarn.util.timeline.TimelineUtils; -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.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; public class TestSystemMetricsPublisherForV2 { @@ -100,7 +101,7 @@ public class TestSystemMetricsPublisherForV2 { private static RMTimelineCollectorManager rmTimelineCollectorManager; - @BeforeClass + @BeforeAll public static void setup() throws Exception { if (testRootDir.exists()) { //cleanup before hand @@ -136,7 +137,7 @@ protected Dispatcher getDispatcher() { metricsPublisher.start(); } - @AfterClass + @AfterAll public static void tearDown() throws Exception { if (testRootDir.exists()) { FileContext.getLocalFSFileContext().delete( @@ -164,7 +165,7 @@ private static Configuration getTimelineV2Conf() { testRootDir.getCanonicalPath()); } catch (IOException e) { e.printStackTrace(); - Assert + Assertions .fail("Exception while setting the " + "TIMELINE_SERVICE_STORAGE_DIR_ROOT "); } @@ -182,8 +183,8 @@ public void testSystemMetricPublisherInitialization() { YarnConfiguration.DEFAULT_RM_PUBLISH_CONTAINER_EVENTS_ENABLED); publisher.init(conf); assertFalse( - "Default configuration should not publish container events from RM", - publisher.isPublishContainerEvents()); + + publisher.isPublishContainerEvents(), "Default configuration should not publish container events from RM"); publisher.stop(); @@ -191,18 +192,19 @@ public void testSystemMetricPublisherInitialization() { mock(RMTimelineCollectorManager.class)); conf = getTimelineV2Conf(); publisher.init(conf); - assertTrue("Expected to have registered event handlers and set ready to " - + "publish events after init", - publisher.isPublishContainerEvents()); + assertTrue( + publisher.isPublishContainerEvents(), "Expected to have registered event handlers and set ready to " + + "publish events after init"); publisher.start(); - assertTrue("Expected to publish container events from RM", - publisher.isPublishContainerEvents()); + assertTrue( + publisher.isPublishContainerEvents(), "Expected to publish container events from RM"); } finally { publisher.stop(); } } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testPublishApplicationMetrics() throws Exception { ApplicationId appId = ApplicationId.newInstance(0, 1); RMApp app = createAppAndRegister(appId); @@ -218,19 +220,20 @@ public void testPublishApplicationMetrics() throws Exception { + "/"; File entityFolder = new File(outputDirApp); - Assert.assertTrue(entityFolder.isDirectory()); + Assertions.assertTrue(entityFolder.isDirectory()); // file name is .thist String timelineServiceFileName = appId.toString() + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION; File appFile = new File(outputDirApp, timelineServiceFileName); - Assert.assertTrue(appFile.exists()); + Assertions.assertTrue(appFile.exists()); verifyEntity( appFile, 4, ApplicationMetricsConstants.CREATED_EVENT_TYPE, 8, 0); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testPublishAppAttemptMetrics() throws Exception { ApplicationId appId = ApplicationId.newInstance(0, 1); RMApp app = rmAppsMapInContext.get(appId); @@ -253,19 +256,20 @@ public void testPublishAppAttemptMetrics() throws Exception { + TimelineEntityType.YARN_APPLICATION_ATTEMPT + "/"; File entityFolder = new File(outputDirApp); - Assert.assertTrue(entityFolder.isDirectory()); + Assertions.assertTrue(entityFolder.isDirectory()); // file name is .thist String timelineServiceFileName = appAttemptId.toString() + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION; File appFile = new File(outputDirApp, timelineServiceFileName); - Assert.assertTrue(appFile.exists()); + Assertions.assertTrue(appFile.exists()); verifyEntity(appFile, 2, AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE, 0, TimelineServiceHelper.invertLong(appAttemptId.getAttemptId())); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testPublishContainerMetrics() throws Exception { ApplicationId appId = ApplicationId.newInstance(0, 1); RMApp app = rmAppsMapInContext.get(appId); @@ -285,20 +289,21 @@ public void testPublishContainerMetrics() throws Exception { + TimelineEntityType.YARN_CONTAINER + "/"; File entityFolder = new File(outputDirApp); - Assert.assertTrue(entityFolder.isDirectory()); + Assertions.assertTrue(entityFolder.isDirectory()); // file name is .thist String timelineServiceFileName = containerId.toString() + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION; File appFile = new File(outputDirApp, timelineServiceFileName); - Assert.assertTrue(appFile.exists()); + Assertions.assertTrue(appFile.exists()); verifyEntity(appFile, 2, ContainerMetricsConstants.CREATED_IN_RM_EVENT_TYPE, 0, TimelineServiceHelper.invertLong(containerId.getContainerId())); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testPutEntityWhenNoCollector() throws Exception { // Validating the logs as DrainDispatcher won't throw exception class TestAppender extends AppenderSkeleton { @@ -332,8 +337,8 @@ public List getLog() { metricsPublisher.appCreated(app, app.getStartTime()); dispatcher.await(); for (LoggingEvent event : appender.getLog()) { - assertFalse("Dispatcher Crashed", - event.getRenderedMessage().contains("Error in dispatcher thread")); + assertFalse( + event.getRenderedMessage().contains("Error in dispatcher thread"), "Dispatcher Crashed"); } } finally { logger.removeAppender(appender); @@ -377,10 +382,10 @@ private static void verifyEntity(File entityFile, long expectedEvents, } finally { reader.close(); } - assertEquals("Expected " + expectedEvents + " events to be published", - expectedEvents, count); - assertEquals("Expected " + expectedMetrics + " metrics is incorrect", - expectedMetrics, metricsCount); + assertEquals( + expectedEvents, count, "Expected " + expectedEvents + " events to be published"); + assertEquals( + expectedMetrics, metricsCount, "Expected " + expectedMetrics + " metrics is incorrect"); } private String getTimelineEntityDir(RMApp app) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/TestSchedulingMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/TestSchedulingMonitor.java index f04081e48600d..55e295d979c5c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/TestSchedulingMonitor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/TestSchedulingMonitor.java @@ -27,12 +27,13 @@ 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.AutoCreatedQueueDeletionPolicy; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.HashSet; import java.util.Set; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.timeout; import static org.mockito.Mockito.verify; @@ -40,7 +41,8 @@ public class TestSchedulingMonitor { - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testRMStarts() throws Exception { Configuration conf = new YarnConfiguration(); conf.setBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS, true); @@ -60,7 +62,8 @@ public void testRMStarts() throws Exception { rm.close(); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testRMUpdateSchedulingEditPolicy() throws Exception { CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, @@ -95,7 +98,8 @@ public void testRMUpdateSchedulingEditPolicy() throws Exception { rm.close(); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testRMUpdateAutoCreatedQueueDeletionPolicy() throws Exception { CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.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/monitor/capacity/TestPreemptionForQueueWithPriorities.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestPreemptionForQueueWithPriorities.java index 1f82617543fd0..9f56909102ea1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestPreemptionForQueueWithPriorities.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestPreemptionForQueueWithPriorities.java @@ -25,8 +25,8 @@ import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceUtils; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.IOException; @@ -38,7 +38,7 @@ public class TestPreemptionForQueueWithPriorities extends ProportionalCapacityPreemptionPolicyMockFramework { - @Before + @BeforeEach public void setup() { resourceCalculator = new DefaultResourceCalculator(); 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/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java index 1bac22743b0ff..ec23b4beb5bfa 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java @@ -58,9 +58,9 @@ import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.Before; +import org.junit.jupiter.api.BeforeEach; import org.junit.Rule; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.junit.rules.TestName; import org.mockito.ArgumentCaptor; import org.mockito.ArgumentMatcher; @@ -81,11 +81,11 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType.MARK_CONTAINER_FOR_PREEMPTION; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.argThat; @@ -152,7 +152,7 @@ public int getValue() { @Rule public TestName name = new TestName(); - @Before + @BeforeEach @SuppressWarnings("unchecked") public void setup() { conf = new CapacitySchedulerConfiguration(new Configuration(false)); @@ -400,8 +400,8 @@ public void testPerQueueDisablePreemptionHierarchical() { ApplicationAttemptId expectedAttemptOnQueueB = ApplicationAttemptId.newInstance( appA.getApplicationId(), appA.getAttemptId()); - assertTrue("appA should be running on queueB", - mCS.getAppsInQueue("queueB").contains(expectedAttemptOnQueueB)); + assertTrue( + mCS.getAppsInQueue("queueB").contains(expectedAttemptOnQueueB), "appA should be running on queueB"); verify(mDisp, times(10)).handle(argThat(new IsPreemptionRequestFor(appA))); // Need to call setup() again to reset mDisp @@ -418,10 +418,10 @@ public void testPerQueueDisablePreemptionHierarchical() { appC.getApplicationId(), appC.getAttemptId()); // Now, all of queueB's (appA) over capacity is not preemptable, so neither // is queueA's. Verify that capacity is taken from queueE (appC). - assertTrue("appB should be running on queueC", - mCS.getAppsInQueue("queueC").contains(expectedAttemptOnQueueC)); - assertTrue("appC should be running on queueE", - mCS.getAppsInQueue("queueE").contains(expectedAttemptOnQueueE)); + assertTrue( + mCS.getAppsInQueue("queueC").contains(expectedAttemptOnQueueC), "appB should be running on queueC"); + assertTrue( + mCS.getAppsInQueue("queueE").contains(expectedAttemptOnQueueE), "appC should be running on queueE"); // Resources should have come from queueE (appC) and neither of queueA's // children. verify(mDisp, never()).handle(argThat(new IsPreemptionRequestFor(appA))); @@ -1118,8 +1118,8 @@ public void testLeafQueueNameExtraction() throws Exception { policy.editSchedule(); - assertFalse("dynamicParent should not be a LeafQueue " + - "candidate", policy.getLeafQueueNames().contains("root.dynamicParent")); + assertFalse(policy.getLeafQueueNames().contains("root.dynamicParent"), "dynamicParent should not be a LeafQueue " + + "candidate"); } static class IsPreemptionRequestFor diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java index 2ddb0d8ae1c51..375f9308e88d4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForNodePartitions.java @@ -25,8 +25,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.mockframework.ProportionalCapacityPreemptionPolicyMockFramework; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.util.resource.ResourceUtils; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.HashMap; @@ -39,7 +39,7 @@ public class TestProportionalCapacityPreemptionPolicyForNodePartitions extends ProportionalCapacityPreemptionPolicyMockFramework { - @Before + @BeforeEach public void setup() { super.setup(); policy = new ProportionalCapacityPreemptionPolicy(rmContext, cs, mClock); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForReservedContainers.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForReservedContainers.java index 6c723493ced1d..1830fb45bd73b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForReservedContainers.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyForReservedContainers.java @@ -21,8 +21,8 @@ import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.mockframework.ProportionalCapacityPreemptionPolicyMockFramework; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.IOException; @@ -32,7 +32,7 @@ public class TestProportionalCapacityPreemptionPolicyForReservedContainers extends ProportionalCapacityPreemptionPolicyMockFramework { - @Before + @BeforeEach public void setup() { super.setup(); conf.setBoolean( diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyInterQueueWithDRF.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyInterQueueWithDRF.java index efd2ffc991cbd..24f5e56fe0f1e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyInterQueueWithDRF.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyInterQueueWithDRF.java @@ -20,8 +20,8 @@ import java.io.IOException; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes; import org.apache.hadoop.yarn.api.records.Resource; @@ -43,7 +43,7 @@ public class TestProportionalCapacityPreemptionPolicyInterQueueWithDRF extends ProportionalCapacityPreemptionPolicyMockFramework { - @Before + @BeforeEach public void setup() { super.setup(); resourceCalculator = new DominantResourceCalculator(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueue.java index a11153a0482f5..c0e2acb16943c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueue.java @@ -21,8 +21,8 @@ import java.io.IOException; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor; import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.mockframework.ProportionalCapacityPreemptionPolicyMockFramework; @@ -39,7 +39,7 @@ public class TestProportionalCapacityPreemptionPolicyIntraQueue extends ProportionalCapacityPreemptionPolicyMockFramework { - @Before + @BeforeEach public void setup() { super.setup(); conf.setBoolean( diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueFairOrdering.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueFairOrdering.java index def4554ec5b24..8877d34495f70 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueFairOrdering.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueFairOrdering.java @@ -26,8 +26,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.mockframework.ProportionalCapacityPreemptionPolicyMockFramework; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /* * Test class for testing intra-queue preemption when the fair ordering policy @@ -35,7 +35,7 @@ */ public class TestProportionalCapacityPreemptionPolicyIntraQueueFairOrdering extends ProportionalCapacityPreemptionPolicyMockFramework { - @Before + @BeforeEach public void setup() { super.setup(); conf.setBoolean( diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueUserLimit.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueUserLimit.java index 2a0838b83f453..7106aa3b92c59 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueUserLimit.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueUserLimit.java @@ -20,8 +20,8 @@ import java.io.IOException; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.mockframework.ProportionalCapacityPreemptionPolicyMockFramework; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; @@ -36,7 +36,7 @@ public class TestProportionalCapacityPreemptionPolicyIntraQueueUserLimit extends ProportionalCapacityPreemptionPolicyMockFramework { - @Before + @BeforeEach public void setup() { super.setup(); conf.setBoolean( diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueWithDRF.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueWithDRF.java index b3ad5ada0b322..b2bb069427b20 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueWithDRF.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueWithDRF.java @@ -26,8 +26,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue; import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceUtils; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.IOException; @@ -44,7 +44,7 @@ public class TestProportionalCapacityPreemptionPolicyIntraQueueWithDRF extends ProportionalCapacityPreemptionPolicyMockFramework { - @Before + @BeforeEach public void setup() { super.setup(); conf.setBoolean( diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyMockFramework.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyMockFramework.java index d59c8548e4a7b..2fbf9d3f14df9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyMockFramework.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyMockFramework.java @@ -23,8 +23,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; public class TestProportionalCapacityPreemptionPolicyMockFramework extends ProportionalCapacityPreemptionPolicyMockFramework { @@ -104,7 +104,7 @@ public void testBuilder() throws Exception { checkAbsCapacities(cs.getQueue("a2"), "blue", 0.5f, 1f, 0f); checkPendingResource(cs.getQueue("a2"), "blue", 200); checkPriority(cs.getQueue("a2"), 2); - Assert.assertFalse(cs.getQueue("a2").getPreemptionDisabled()); + Assertions.assertFalse(cs.getQueue("a2").getPreemptionDisabled()); // b checkAbsCapacities(cs.getQueue("b"), "", 0.5f, 1f, 0f); @@ -114,16 +114,16 @@ public void testBuilder() throws Exception { checkAbsCapacities(cs.getQueue("b"), "blue", 0f, 0f, 0f); checkPendingResource(cs.getQueue("b"), "blue", 0); checkPriority(cs.getQueue("b"), 1); - Assert.assertTrue(cs.getQueue("b").getPreemptionDisabled()); + Assertions.assertTrue(cs.getQueue("b").getPreemptionDisabled()); // Check ignored partitioned containers in queue - Assert.assertEquals(100, ((LeafQueue) cs.getQueue("a1")) + Assertions.assertEquals(100, ((LeafQueue) cs.getQueue("a1")) .getIgnoreExclusivityRMContainers().get("blue").size()); // Check applications - Assert.assertEquals(2, ((LeafQueue)cs.getQueue("a1")).getApplications().size()); - Assert.assertEquals(1, ((LeafQueue)cs.getQueue("a2")).getApplications().size()); - Assert.assertEquals(1, ((LeafQueue)cs.getQueue("b")).getApplications().size()); + Assertions.assertEquals(2, ((LeafQueue)cs.getQueue("a1")).getApplications().size()); + Assertions.assertEquals(1, ((LeafQueue)cs.getQueue("a2")).getApplications().size()); + Assertions.assertEquals(1, ((LeafQueue)cs.getQueue("b")).getApplications().size()); // Check #containers FiCaSchedulerApp app1 = getApp("a1", 1); @@ -131,17 +131,17 @@ public void testBuilder() throws Exception { FiCaSchedulerApp app3 = getApp("a2", 3); FiCaSchedulerApp app4 = getApp("b", 4); - Assert.assertEquals(50, app1.getLiveContainers().size()); + Assertions.assertEquals(50, app1.getLiveContainers().size()); checkContainerNodesInApp(app1, 50, "n3"); - Assert.assertEquals(50, app2.getLiveContainers().size()); - Assert.assertEquals(150, app2.getReservedContainers().size()); + Assertions.assertEquals(50, app2.getLiveContainers().size()); + Assertions.assertEquals(150, app2.getReservedContainers().size()); checkContainerNodesInApp(app2, 200, "n2"); - Assert.assertEquals(50, app3.getLiveContainers().size()); + Assertions.assertEquals(50, app3.getLiveContainers().size()); checkContainerNodesInApp(app3, 50, "n3"); - Assert.assertEquals(100, app4.getLiveContainers().size()); + Assertions.assertEquals(100, app4.getLiveContainers().size()); checkContainerNodesInApp(app4, 100, "n1"); } @@ -236,20 +236,20 @@ public void testBuilderWithSpecifiedNodeResources() throws Exception { buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig); // Check host resources - Assert.assertEquals(3, this.cs.getAllNodes().size()); + Assertions.assertEquals(3, this.cs.getAllNodes().size()); SchedulerNode node1 = cs.getSchedulerNode(NodeId.newInstance("n1", 1)); - Assert.assertEquals(100, node1.getTotalResource().getMemorySize()); - Assert.assertEquals(100, node1.getCopiedListOfRunningContainers().size()); - Assert.assertNull(node1.getReservedContainer()); + Assertions.assertEquals(100, node1.getTotalResource().getMemorySize()); + Assertions.assertEquals(100, node1.getCopiedListOfRunningContainers().size()); + Assertions.assertNull(node1.getReservedContainer()); SchedulerNode node2 = cs.getSchedulerNode(NodeId.newInstance("n2", 1)); - Assert.assertEquals(0, node2.getTotalResource().getMemorySize()); - Assert.assertEquals(50, node2.getCopiedListOfRunningContainers().size()); - Assert.assertNotNull(node2.getReservedContainer()); + Assertions.assertEquals(0, node2.getTotalResource().getMemorySize()); + Assertions.assertEquals(50, node2.getCopiedListOfRunningContainers().size()); + Assertions.assertNotNull(node2.getReservedContainer()); SchedulerNode node3 = cs.getSchedulerNode(NodeId.newInstance("n3", 1)); - Assert.assertEquals(30, node3.getTotalResource().getMemorySize()); - Assert.assertEquals(100, node3.getCopiedListOfRunningContainers().size()); - Assert.assertNull(node3.getReservedContainer()); + Assertions.assertEquals(30, node3.getTotalResource().getMemorySize()); + Assertions.assertEquals(100, node3.getCopiedListOfRunningContainers().size()); + Assertions.assertNull(node3.getReservedContainer()); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyPreemptToBalance.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyPreemptToBalance.java index 2e7b01ed50d9a..91a409a333e34 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyPreemptToBalance.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyPreemptToBalance.java @@ -24,12 +24,12 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; import org.apache.log4j.Level; import org.apache.log4j.Logger; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.Map; import java.util.Set; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.ArgumentMatchers.argThat; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/mockframework/ProportionalCapacityPreemptionPolicyMockFramework.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/mockframework/ProportionalCapacityPreemptionPolicyMockFramework.java index 024ec86f7d70e..ca4eec164884b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/mockframework/ProportionalCapacityPreemptionPolicyMockFramework.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/mockframework/ProportionalCapacityPreemptionPolicyMockFramework.java @@ -49,9 +49,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.Before; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; import org.mockito.ArgumentMatcher; import java.io.IOException; @@ -106,7 +106,7 @@ private void resetResourceInformationMap() { } @SuppressWarnings("unchecked") - @Before + @BeforeEach public void setup() { resetResourceInformationMap(); @@ -147,7 +147,7 @@ public void setup() { clusterResource = Resource.newInstance(0, 0); } - @After + @AfterEach public void cleanup() { resetResourceInformationMap(); } @@ -252,7 +252,7 @@ protected void checkContainerNodesInApp(FiCaSchedulerApp app, num++; } } - Assert.assertEquals(expectedContainersNumber, num); + Assertions.assertEquals(expectedContainersNumber, num); } public FiCaSchedulerApp getApp(String queueName, int appId) { @@ -268,28 +268,28 @@ public FiCaSchedulerApp getApp(String queueName, int appId) { protected void checkAbsCapacities(CSQueue queue, String partition, float guaranteed, float max, float used) { QueueCapacities qc = queue.getQueueCapacities(); - Assert.assertEquals(guaranteed, qc.getAbsoluteCapacity(partition), + Assertions.assertEquals(guaranteed, qc.getAbsoluteCapacity(partition), ALLOWED_CAPACITY_DELTA); - Assert.assertEquals(max, qc.getAbsoluteMaximumCapacity(partition), + Assertions.assertEquals(max, qc.getAbsoluteMaximumCapacity(partition), ALLOWED_CAPACITY_DELTA); - Assert.assertEquals(used, qc.getAbsoluteUsedCapacity(partition), + Assertions.assertEquals(used, qc.getAbsoluteUsedCapacity(partition), ALLOWED_CAPACITY_DELTA); } protected void checkPendingResource(CSQueue queue, String partition, int pending) { ResourceUsage ru = queue.getQueueResourceUsage(); - Assert.assertEquals(pending, ru.getPending(partition).getMemorySize()); + Assertions.assertEquals(pending, ru.getPending(partition).getMemorySize()); } protected void checkPriority(CSQueue queue, int expectedPriority) { - Assert.assertEquals(expectedPriority, queue.getPriority().getPriority()); + Assertions.assertEquals(expectedPriority, queue.getPriority().getPriority()); } protected void checkReservedResource(CSQueue queue, String partition, int reserved) { ResourceUsage ru = queue.getQueueResourceUsage(); - Assert.assertEquals(reserved, ru.getReserved(partition).getMemorySize()); + Assertions.assertEquals(reserved, ru.getReserved(partition).getMemorySize()); } public static class IsPreemptionRequestForQueueAndNode diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/TestMetricsInvariantChecker.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/TestMetricsInvariantChecker.java index 6aaa15f3e18a1..903b1d842d9b9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/TestMetricsInvariantChecker.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/TestMetricsInvariantChecker.java @@ -26,10 +26,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.log4j.Logger; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; -import static junit.framework.TestCase.fail; +import static org.junit.jupiter.api.Assertions.fail; /** * This class tests the {@code MetricsInvariantChecker} by running it multiple @@ -44,7 +45,7 @@ public class TestMetricsInvariantChecker { private MetricsInvariantChecker ic; private Configuration conf; - @Before + @BeforeEach public void setup() { this.metricsSystem = DefaultMetricsSystem.instance(); JvmMetrics.initSingleton("ResourceManager", null); @@ -61,7 +62,8 @@ public void setup() { ic.init(conf, null, null); } - @Test(timeout = 5000) + @Test + @Timeout(value = 5) public void testManyRuns() { QueueMetrics qm = diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java index 840ff2aec284d..e084db778ed50 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java @@ -32,10 +32,11 @@ import org.apache.hadoop.yarn.nodelabels.AttributeValue; import org.apache.hadoop.yarn.nodelabels.NodeAttributeStore; import org.apache.hadoop.yarn.server.resourcemanager.NodeAttributeTestUtils; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.io.IOException; import java.util.HashMap; @@ -68,7 +69,7 @@ protected void stopDispatcher() { } } - @Before + @BeforeEach public void before() throws IOException { mgr = new MockNodeAttrbuteManager(); conf = new Configuration(); @@ -79,7 +80,7 @@ public void before() throws IOException { mgr.start(); } - @After + @AfterEach public void after() throws IOException { FileSystemNodeAttributeStore fsStore = ((FileSystemNodeAttributeStore) mgr.store); @@ -87,7 +88,8 @@ public void after() throws IOException { mgr.stop(); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testEmptyRecoverSkipInternalUdpate() throws Exception { // Stop manager mgr.stop(); @@ -101,7 +103,8 @@ public void testEmptyRecoverSkipInternalUdpate() throws Exception { .internalUpdateAttributesOnNodes(any(), any(), any(), any()); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testRecoverWithMirror() throws Exception { //------host0---- @@ -126,8 +129,8 @@ public void testRecoverWithMirror() throws Exception { // Add node attribute mgr.addNodeAttributes(toAddAttributes); - Assert.assertEquals("host0 size", 2, - mgr.getAttributesForNode("host0").size()); + Assertions.assertEquals(2 +, mgr.getAttributesForNode("host0").size(), "host0 size"); // Add test to remove toAddAttributes.clear(); toAddAttributes.put("host0", ImmutableSet.of(gpu)); @@ -150,10 +153,10 @@ public void testRecoverWithMirror() throws Exception { mgr.start(); mgr.getAttributesForNode("host0"); - Assert.assertEquals("host0 size", 1, - mgr.getAttributesForNode("host0").size()); - Assert.assertEquals("host1 size", 1, - mgr.getAttributesForNode("host1").size()); + Assertions.assertEquals(1 +, mgr.getAttributesForNode("host0").size(), "host0 size"); + Assertions.assertEquals(1 +, mgr.getAttributesForNode("host1").size(), "host1 size"); attrs = mgr.getAttributesForNode("host0"); assertThat(attrs).hasSize(1); assertThat(attrs.keySet().toArray()[0]).isEqualTo(docker); @@ -177,19 +180,20 @@ public void testRecoverWithMirror() throws Exception { mgr = new MockNodeAttrbuteManager(); mgr.init(conf); mgr.start(); - Assert.assertEquals("host0 size", 1, - mgr.getAttributesForNode("host0").size()); - Assert.assertEquals("host1 size", 2, - mgr.getAttributesForNode("host1").size()); + Assertions.assertEquals(1 +, mgr.getAttributesForNode("host0").size(), "host0 size"); + Assertions.assertEquals(2 +, mgr.getAttributesForNode("host1").size(), "host1 size"); attrs = mgr.getAttributesForNode("host0"); assertThat(attrs).hasSize(1); assertThat(attrs.keySet().toArray()[0]).isEqualTo(gpu); attrs = mgr.getAttributesForNode("host1"); - Assert.assertTrue(attrs.keySet().contains(docker)); - Assert.assertTrue(attrs.keySet().contains(gpu)); + Assertions.assertTrue(attrs.keySet().contains(docker)); + Assertions.assertTrue(attrs.keySet().contains(gpu)); } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testRecoverFromEditLog() throws Exception { NodeAttribute docker = NodeAttribute .newInstance(NodeAttribute.PREFIX_CENTRALIZED, "DOCKER", @@ -208,8 +212,8 @@ public void testRecoverFromEditLog() throws Exception { // Add node attribute mgr.addNodeAttributes(toAddAttributes); - Assert.assertEquals("host0 size", 2, - mgr.getAttributesForNode("host0").size()); + Assertions.assertEquals(2 +, mgr.getAttributesForNode("host0").size(), "host0 size"); // Increase editlog operation for (int i = 0; i < 5; i++) { @@ -241,10 +245,10 @@ public void testRecoverFromEditLog() throws Exception { mgr.init(conf); mgr.start(); - Assert.assertEquals("host0 size", 1, - mgr.getAttributesForNode("host0").size()); - Assert.assertEquals("host1 size", 2, - mgr.getAttributesForNode("host1").size()); + Assertions.assertEquals(1 +, mgr.getAttributesForNode("host0").size(), "host0 size"); + Assertions.assertEquals(2 +, mgr.getAttributesForNode("host1").size(), "host1 size"); toAddAttributes.clear(); NodeAttribute replaced = @@ -262,19 +266,19 @@ public void testRecoverFromEditLog() throws Exception { Map.Entry entry = valueMap.entrySet().iterator().next(); NodeAttribute attribute = entry.getKey(); - Assert.assertEquals("host0 size", 1, - mgr.getAttributesForNode("host0").size()); - Assert.assertEquals("host1 size", 2, - mgr.getAttributesForNode("host1").size()); + Assertions.assertEquals(1 +, mgr.getAttributesForNode("host0").size(), "host0 size"); + Assertions.assertEquals(2 +, mgr.getAttributesForNode("host1").size(), "host1 size"); checkNodeAttributeEqual(replaced, attribute); } public void checkNodeAttributeEqual(NodeAttribute atr1, NodeAttribute atr2) { - Assert.assertEquals(atr1.getAttributeType(), atr2.getAttributeType()); - Assert.assertEquals(atr1.getAttributeKey().getAttributeName(), + Assertions.assertEquals(atr1.getAttributeType(), atr2.getAttributeType()); + Assertions.assertEquals(atr1.getAttributeKey().getAttributeName(), atr2.getAttributeKey().getAttributeName()); - Assert.assertEquals(atr1.getAttributeKey().getAttributePrefix(), + Assertions.assertEquals(atr1.getAttributeKey().getAttributePrefix(), atr2.getAttributeKey().getAttributePrefix()); - Assert.assertEquals(atr1.getAttributeValue(), atr2.getAttributeValue()); + Assertions.assertEquals(atr1.getAttributeValue(), atr2.getAttributeValue()); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java index 43df13e262ea9..bbe2f981efe45 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java @@ -29,10 +29,10 @@ import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager; import org.apache.hadoop.yarn.nodelabels.NodeLabelUtil; import org.apache.hadoop.yarn.server.resourcemanager.NodeAttributeTestUtils; -import org.junit.Test; -import org.junit.Before; -import org.junit.After; -import org.junit.Assert; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; import java.io.IOException; import java.util.HashMap; @@ -51,7 +51,7 @@ public class TestNodeAttributesManager { private final static String[] HOSTNAMES = new String[] {"host1", "host2", "host3"}; - @Before + @BeforeEach public void init() throws IOException { Configuration conf = new Configuration(); attributesManager = new NodeAttributesManagerImpl(); @@ -62,7 +62,7 @@ public void init() throws IOException { attributesManager.start(); } - @After + @AfterEach public void cleanUp() { if (attributesManager != null) { attributesManager.stop(); @@ -102,8 +102,8 @@ public void testAddNodeAttributes() throws IOException { attributesManager.addNodeAttributes(toAddAttributes); nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]); - Assert.assertEquals(3, nodeAttributes.size()); - Assert.assertTrue(sameAttributeSet(toAddAttributes.get(HOSTNAMES[0]), + Assertions.assertEquals(3, nodeAttributes.size()); + Assertions.assertTrue(sameAttributeSet(toAddAttributes.get(HOSTNAMES[0]), nodeAttributes.keySet())); // Add 2 attributes to host2 @@ -116,12 +116,12 @@ public void testAddNodeAttributes() throws IOException { // Verify host1 attributes are still valid. nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]); - Assert.assertEquals(3, nodeAttributes.size()); + Assertions.assertEquals(3, nodeAttributes.size()); // Verify new added host2 attributes are correctly updated. nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[1]); - Assert.assertEquals(2, nodeAttributes.size()); - Assert.assertTrue(sameAttributeSet(toAddAttributes.get(HOSTNAMES[1]), + Assertions.assertEquals(2, nodeAttributes.size()); + Assertions.assertTrue(sameAttributeSet(toAddAttributes.get(HOSTNAMES[1]), nodeAttributes.keySet())); // Cluster wide, it only has 3 attributes. @@ -130,17 +130,17 @@ public void testAddNodeAttributes() throws IOException { // yarn.test1.io/A3 Set clusterAttributes = attributesManager .getClusterNodeAttributes(Sets.newHashSet(PREFIXES[0])); - Assert.assertEquals(3, clusterAttributes.size()); + Assertions.assertEquals(3, clusterAttributes.size()); // Query for attributes under a non-exist prefix, // ensure it returns an empty set. clusterAttributes = attributesManager .getClusterNodeAttributes(Sets.newHashSet("non_exist_prefix")); - Assert.assertEquals(0, clusterAttributes.size()); + Assertions.assertEquals(0, clusterAttributes.size()); // Not provide any prefix, ensure it returns all attributes. clusterAttributes = attributesManager.getClusterNodeAttributes(null); - Assert.assertEquals(3, clusterAttributes.size()); + Assertions.assertEquals(3, clusterAttributes.size()); // Add some other attributes with different prefixes on host1 and host2. toAddAttributes.clear(); @@ -160,10 +160,10 @@ public void testAddNodeAttributes() throws IOException { attributesManager.addNodeAttributes(toAddAttributes); nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]); - Assert.assertEquals(13, nodeAttributes.size()); + Assertions.assertEquals(13, nodeAttributes.size()); nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[1]); - Assert.assertEquals(22, nodeAttributes.size()); + Assertions.assertEquals(22, nodeAttributes.size()); } @Test @@ -210,20 +210,20 @@ public void testRemoveNodeAttributes() throws IOException { attributesManager.addNodeAttributes(toAddAttributes); nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]); - Assert.assertEquals(8, nodeAttributes.size()); + Assertions.assertEquals(8, nodeAttributes.size()); nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[1]); - Assert.assertEquals(4, nodeAttributes.size()); + Assertions.assertEquals(4, nodeAttributes.size()); allAttributesPerPrefix = attributesManager .getClusterNodeAttributes(Sets.newHashSet(PREFIXES[0])); - Assert.assertEquals(3, allAttributesPerPrefix.size()); + Assertions.assertEquals(3, allAttributesPerPrefix.size()); allAttributesPerPrefix = attributesManager .getClusterNodeAttributes(Sets.newHashSet(PREFIXES[1])); - Assert.assertEquals(5, allAttributesPerPrefix.size()); + Assertions.assertEquals(5, allAttributesPerPrefix.size()); allAttributesPerPrefix = attributesManager .getClusterNodeAttributes(Sets.newHashSet(PREFIXES[2])); - Assert.assertEquals(2, allAttributesPerPrefix.size()); + Assertions.assertEquals(2, allAttributesPerPrefix.size()); // Remove "yarn.test1.io/A_2" from host1 Set attributes2rm1 = new HashSet<>(); @@ -233,7 +233,7 @@ public void testRemoveNodeAttributes() throws IOException { attributesManager.removeNodeAttributes(toRemoveAttributes); nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]); - Assert.assertEquals(7, nodeAttributes.size()); + Assertions.assertEquals(7, nodeAttributes.size()); // Remove again, but give a non-exist attribute name attributes2rm1.clear(); @@ -244,7 +244,7 @@ public void testRemoveNodeAttributes() throws IOException { attributesManager.removeNodeAttributes(toRemoveAttributes); nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]); - Assert.assertEquals(7, nodeAttributes.size()); + Assertions.assertEquals(7, nodeAttributes.size()); // Remove "yarn.test1.io/A_2" from host2 too, // by then there will be no such attribute exist in the cluster. @@ -261,7 +261,7 @@ public void testRemoveNodeAttributes() throws IOException { // us A_1 and A_3. allAttributesPerPrefix = attributesManager .getClusterNodeAttributes(Sets.newHashSet(PREFIXES[0])); - Assert.assertEquals(2, allAttributesPerPrefix.size()); + Assertions.assertEquals(2, allAttributesPerPrefix.size()); } @Test @@ -281,7 +281,7 @@ public void testReplaceNodeAttributes() throws IOException { attributesManager.addNodeAttributes(toAddAttributes); nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]); - Assert.assertEquals(3, nodeAttributes.size()); + Assertions.assertEquals(3, nodeAttributes.size()); // Add 10 distributed node attributes to host1 // nn.yarn.io/dist-node-attribute1=dist_v1_1 @@ -294,10 +294,10 @@ public void testReplaceNodeAttributes() throws IOException { 10, "dist-node-attribute", "dist_v1")); attributesManager.addNodeAttributes(toAddAttributes); nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]); - Assert.assertEquals(13, nodeAttributes.size()); + Assertions.assertEquals(13, nodeAttributes.size()); clusterAttributes = attributesManager.getClusterNodeAttributes( Sets.newHashSet(NodeAttribute.PREFIX_DISTRIBUTED, PREFIXES[0])); - Assert.assertEquals(13, clusterAttributes.size()); + Assertions.assertEquals(13, clusterAttributes.size()); // Replace by prefix // Same distributed attributes names, but different values. @@ -308,24 +308,24 @@ public void testReplaceNodeAttributes() throws IOException { attributesManager.replaceNodeAttributes(NodeAttribute.PREFIX_DISTRIBUTED, ImmutableMap.of(HOSTNAMES[0], toReplaceAttributes)); nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]); - Assert.assertEquals(8, nodeAttributes.size()); + Assertions.assertEquals(8, nodeAttributes.size()); clusterAttributes = attributesManager.getClusterNodeAttributes( Sets.newHashSet(NodeAttribute.PREFIX_DISTRIBUTED, PREFIXES[0])); - Assert.assertEquals(8, clusterAttributes.size()); + Assertions.assertEquals(8, clusterAttributes.size()); // Now we have 5 distributed attributes filteredAttributes = NodeLabelUtil.filterAttributesByPrefix( nodeAttributes.keySet(), NodeAttribute.PREFIX_DISTRIBUTED); - Assert.assertEquals(5, filteredAttributes.size()); + Assertions.assertEquals(5, filteredAttributes.size()); // Values are updated to have prefix dist_v2 - Assert.assertTrue(filteredAttributes.stream().allMatch( + Assertions.assertTrue(filteredAttributes.stream().allMatch( nodeAttribute -> nodeAttribute.getAttributeValue().startsWith("dist_v2"))); // We still have 3 yarn.test1.io attributes filteredAttributes = NodeLabelUtil.filterAttributesByPrefix( nodeAttributes.keySet(), PREFIXES[0]); - Assert.assertEquals(3, filteredAttributes.size()); + Assertions.assertEquals(3, filteredAttributes.size()); // Replace with prefix // Different attribute names @@ -335,16 +335,16 @@ public void testReplaceNodeAttributes() throws IOException { attributesManager.replaceNodeAttributes(NodeAttribute.PREFIX_DISTRIBUTED, ImmutableMap.of(HOSTNAMES[0], toReplaceAttributes)); nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]); - Assert.assertEquals(4, nodeAttributes.size()); + Assertions.assertEquals(4, nodeAttributes.size()); clusterAttributes = attributesManager.getClusterNodeAttributes( Sets.newHashSet(NodeAttribute.PREFIX_DISTRIBUTED)); - Assert.assertEquals(1, clusterAttributes.size()); + Assertions.assertEquals(1, clusterAttributes.size()); NodeAttribute attr = clusterAttributes.iterator().next(); - Assert.assertEquals("dist-node-attribute-v2_0", + Assertions.assertEquals("dist-node-attribute-v2_0", attr.getAttributeKey().getAttributeName()); - Assert.assertEquals(NodeAttribute.PREFIX_DISTRIBUTED, + Assertions.assertEquals(NodeAttribute.PREFIX_DISTRIBUTED, attr.getAttributeKey().getAttributePrefix()); - Assert.assertEquals("dist_v3_0", attr.getAttributeValue()); + Assertions.assertEquals("dist_v3_0", attr.getAttributeValue()); // Replace all attributes toReplaceMap.put(HOSTNAMES[0], @@ -352,13 +352,13 @@ public void testReplaceNodeAttributes() throws IOException { attributesManager.replaceNodeAttributes(null, toReplaceMap); nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]); - Assert.assertEquals(2, nodeAttributes.size()); + Assertions.assertEquals(2, nodeAttributes.size()); clusterAttributes = attributesManager .getClusterNodeAttributes(Sets.newHashSet(PREFIXES[1])); - Assert.assertEquals(2, clusterAttributes.size()); + Assertions.assertEquals(2, clusterAttributes.size()); clusterAttributes = attributesManager .getClusterNodeAttributes(Sets.newHashSet( NodeAttribute.PREFIX_DISTRIBUTED)); - Assert.assertEquals(0, clusterAttributes.size()); + Assertions.assertEquals(0, clusterAttributes.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/nodelabels/TestNodeLabelFileReplication.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeLabelFileReplication.java index b1cdc93a8757e..b510b03207e71 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeLabelFileReplication.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeLabelFileReplication.java @@ -26,8 +26,8 @@ import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; public class TestNodeLabelFileReplication { @@ -51,9 +51,9 @@ public void testNodeLabelFileReplication() throws IOException { int fileReplication = fs .getFileStatus(new Path(nodeLabelDir, "nodelabel.mirror")) .getReplication(); - Assert.assertEquals( - "Node label file replication should be " + expectedReplication, - expectedReplication, fileReplication); + Assertions.assertEquals( + + expectedReplication, fileReplication, "Node label file replication should be " + expectedReplication); manager.close(); } finally { if (cluster != 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/nodelabels/TestRMDelegatedNodeLabelsUpdater.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMDelegatedNodeLabelsUpdater.java index def531f9c59b7..d6e6b19928104 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMDelegatedNodeLabelsUpdater.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMDelegatedNodeLabelsUpdater.java @@ -18,7 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.nodelabels; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import java.io.IOException; import java.util.Map; @@ -37,9 +37,9 @@ import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.YarnVersionInfo; 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.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; import org.apache.hadoop.thirdparty.com.google.common.collect.Maps; @@ -48,7 +48,7 @@ public class TestRMDelegatedNodeLabelsUpdater extends NodeLabelTestBase { private YarnConfiguration conf; private static Map> nodeLabelsMap = Maps.newHashMap(); - @Before + @BeforeEach public void setup() { conf = new YarnConfiguration(); conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true); @@ -66,10 +66,10 @@ public void testRMNodeLabelsMappingProviderConfiguration() { MockRM rm = new MockRM(conf); rm.init(conf); rm.start(); - Assert.fail("Expected an exception"); + Assertions.fail("Expected an exception"); } catch (Exception e) { // expected an exception - Assert.assertTrue(e.getMessage().contains( + Assertions.assertTrue(e.getMessage().contains( "RMNodeLabelsMappingProvider should be configured")); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java index affd984e5334a..1335dd1c9fc35 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java @@ -19,9 +19,9 @@ package org.apache.hadoop.yarn.server.resourcemanager.nodelabels; 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.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -55,10 +55,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType; 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.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; @@ -71,7 +72,7 @@ public class TestRMNodeLabelsManager extends NodeLabelTestBase { NullRMNodeLabelsManager mgr = null; RMNodeLabelsManager lmgr = null; boolean checkQueueCall = false; - @Before + @BeforeEach public void before() { mgr = new NullRMNodeLabelsManager(); Configuration conf = new Configuration(); @@ -80,12 +81,13 @@ public void before() { mgr.start(); } - @After + @AfterEach public void after() { mgr.stop(); } - @Test(timeout = 5000) + @Test + @Timeout(value = 5) public void testGetLabelResourceWhenNodeActiveDeactive() throws Exception { mgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("p1", "p2", "p3")); mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1"), toSet("p1"), @@ -108,7 +110,7 @@ public void testGetLabelResourceWhenNodeActiveDeactive() throws Exception { mgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("p1", "p4")); assertThat(mgr.getResourceByLabel("p1", null)).isEqualTo( Resources.add(SMALL_RESOURCE, LARGE_NODE)); - Assert.assertEquals(mgr.getResourceByLabel("p4", null), EMPTY_RESOURCE); + Assertions.assertEquals(mgr.getResourceByLabel("p4", null), EMPTY_RESOURCE); // change the large NM to small, check if resource updated mgr.updateNodeResource(NodeId.newInstance("n1", 2), SMALL_RESOURCE); @@ -133,7 +135,8 @@ public void testGetLabelResourceWhenNodeActiveDeactive() throws Exception { isEqualTo(Resources.add(SMALL_RESOURCE, LARGE_NODE)); } - @Test(timeout = 5000) + @Test + @Timeout(value = 5) public void testActivateNodeManagerWithZeroPort() throws Exception { // active two NM, one is zero port , another is non-zero port. no exception // should be raised @@ -142,7 +145,8 @@ public void testActivateNodeManagerWithZeroPort() throws Exception { } @SuppressWarnings({ "unchecked", "rawtypes" }) - @Test(timeout = 5000) + @Test + @Timeout(value = 5) public void testGetLabelResource() throws Exception { mgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("p1", "p2", "p3")); mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1"), toSet("p1"), @@ -229,7 +233,8 @@ public void testGetLabelResource() throws Exception { Resources.multiply(SMALL_RESOURCE, 2)); } - @Test(timeout=5000) + @Test + @Timeout(value = 5) public void testGetQueueResource() throws Exception { Resource clusterResource = Resource.newInstance(9999, 1); @@ -268,15 +273,15 @@ public void testGetQueueResource() throws Exception { mgr.reinitializeQueueLabels(queueToLabels); // check resource - Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), + Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), mgr.getQueueResource("Q1", q1Label, clusterResource)); - Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), + Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), mgr.getQueueResource("Q2", q2Label, clusterResource)); - Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 2), + Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 2), mgr.getQueueResource("Q3", q3Label, clusterResource)); - Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 1), + Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 1), mgr.getQueueResource("Q4", q4Label, clusterResource)); - Assert.assertEquals(clusterResource, + Assertions.assertEquals(clusterResource, mgr.getQueueResource("Q5", q5Label, clusterResource)); mgr.removeLabelsFromNode(ImmutableMap.of(toNodeId("host2"), toSet("blue"))); @@ -290,15 +295,15 @@ public void testGetQueueResource() throws Exception { */ // check resource - Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), + Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), mgr.getQueueResource("Q1", q1Label, clusterResource)); - Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), + Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), mgr.getQueueResource("Q2", q2Label, clusterResource)); - Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), + Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), mgr.getQueueResource("Q3", q3Label, clusterResource)); - Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 2), + Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 2), mgr.getQueueResource("Q4", q4Label, clusterResource)); - Assert.assertEquals(clusterResource, + Assertions.assertEquals(clusterResource, mgr.getQueueResource("Q5", q5Label, clusterResource)); /* @@ -314,15 +319,15 @@ public void testGetQueueResource() throws Exception { mgr.activateNode(NodeId.newInstance("host3", 1), SMALL_RESOURCE); // check resource - Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 2), + Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 2), mgr.getQueueResource("Q1", q1Label, clusterResource)); - Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), + Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), mgr.getQueueResource("Q2", q2Label, clusterResource)); - Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), + Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), mgr.getQueueResource("Q3", q3Label, clusterResource)); - Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 2), + Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 2), mgr.getQueueResource("Q4", q4Label, clusterResource)); - Assert.assertEquals(clusterResource, + Assertions.assertEquals(clusterResource, mgr.getQueueResource("Q5", q5Label, clusterResource)); /* @@ -349,15 +354,15 @@ public void testGetQueueResource() throws Exception { mgr.reinitializeQueueLabels(queueToLabels); // check resource - Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 2), + Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 2), mgr.getQueueResource("Q1", q1Label, clusterResource)); - Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 2), + Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 2), mgr.getQueueResource("Q2", q2Label, clusterResource)); - Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 2), + Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 2), mgr.getQueueResource("Q3", q3Label, clusterResource)); - Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 2), + Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 2), mgr.getQueueResource("Q4", q4Label, clusterResource)); - Assert.assertEquals(clusterResource, + Assertions.assertEquals(clusterResource, mgr.getQueueResource("Q5", q5Label, clusterResource)); /* @@ -372,15 +377,15 @@ public void testGetQueueResource() throws Exception { mgr.activateNode(NodeId.newInstance("host4", 2), SMALL_RESOURCE); // check resource - Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), + Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), mgr.getQueueResource("Q1", q1Label, clusterResource)); - Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), + Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), mgr.getQueueResource("Q2", q2Label, clusterResource)); - Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), + Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), mgr.getQueueResource("Q3", q3Label, clusterResource)); - Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), + Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), mgr.getQueueResource("Q4", q4Label, clusterResource)); - Assert.assertEquals(clusterResource, + Assertions.assertEquals(clusterResource, mgr.getQueueResource("Q5", q5Label, clusterResource)); /* @@ -395,19 +400,20 @@ public void testGetQueueResource() throws Exception { mgr.deactivateNode(NodeId.newInstance("host4", 1)); // check resource - Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 1), + Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 1), mgr.getQueueResource("Q1", q1Label, clusterResource)); - Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 1), + Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 1), mgr.getQueueResource("Q2", q2Label, clusterResource)); - Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 1), + Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 1), mgr.getQueueResource("Q3", q3Label, clusterResource)); - Assert.assertEquals(Resources.multiply(SMALL_RESOURCE, 1), + Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 1), mgr.getQueueResource("Q4", q4Label, clusterResource)); - Assert.assertEquals(clusterResource, + Assertions.assertEquals(clusterResource, mgr.getQueueResource("Q5", q5Label, clusterResource)); } - @Test(timeout=5000) + @Test + @Timeout(value = 5) public void testGetLabelResourceWhenMultipleNMsExistingInSameHost() throws IOException { // active two NM to n1, one large and one small mgr.activateNode(NodeId.newInstance("n1", 1), SMALL_RESOURCE); @@ -434,7 +440,8 @@ public void testGetLabelResourceWhenMultipleNMsExistingInSameHost() throws IOExc Resources.multiply(SMALL_RESOURCE, 2)); } - @Test(timeout = 5000) + @Test + @Timeout(value = 5) public void testRemoveLabelsFromNode() throws Exception { mgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("p1", "p2", "p3")); mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1"), toSet("p1"), @@ -443,14 +450,14 @@ public void testRemoveLabelsFromNode() throws Exception { mgr.activateNode(NodeId.newInstance("n1", 1), SMALL_RESOURCE); try { mgr.removeLabelsFromNode(ImmutableMap.of(toNodeId("n1:1"), toSet("p1"))); - Assert.fail("removeLabelsFromNode should trigger IOException"); + Assertions.fail("removeLabelsFromNode should trigger IOException"); } catch (IOException e) { } mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1:1"), toSet("p1"))); try { mgr.removeLabelsFromNode(ImmutableMap.of(toNodeId("n1:1"), toSet("p1"))); } catch (IOException e) { - Assert.fail("IOException from removeLabelsFromNode " + e); + Assertions.fail("IOException from removeLabelsFromNode " + e); } } @@ -489,53 +496,54 @@ public void testReplaceLabelsFromNode() throws Exception { mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1:1"), toSet("p1"), toNodeId("n2:1"), toSet("p2"), toNodeId("n3"), toSet("p3"))); - assertTrue("Event should be sent when there is change in labels", - schedEventsHandler.receivedEvent); - assertEquals("3 node label mapping modified", 3, - schedEventsHandler.updatedNodeToLabels.size()); + assertTrue( + schedEventsHandler.receivedEvent, "Event should be sent when there is change in labels"); + assertEquals(3 +, schedEventsHandler.updatedNodeToLabels.size(), "3 node label mapping modified"); ImmutableMap> modifiedMap = ImmutableMap.of(toNodeId("n1:1"), toSet("p1"), toNodeId("n2:1"), toSet("p2"), toNodeId("n3:1"), toSet("p3")); - assertEquals("Node label mapping is not matching", modifiedMap, - schedEventsHandler.updatedNodeToLabels); + assertEquals(modifiedMap +, schedEventsHandler.updatedNodeToLabels, "Node label mapping is not matching"); schedEventsHandler.receivedEvent = false; mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1:1"), toSet("p1"))); - assertFalse("No event should be sent when there is no change in labels", - schedEventsHandler.receivedEvent); + assertFalse( + schedEventsHandler.receivedEvent, "No event should be sent when there is no change in labels"); schedEventsHandler.receivedEvent = false; mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n2:1"), toSet("p1"), toNodeId("n3"), toSet("p3"))); - assertTrue("Event should be sent when there is change in labels", - schedEventsHandler.receivedEvent); - assertEquals("Single node label mapping modified", 1, - schedEventsHandler.updatedNodeToLabels.size()); + assertTrue( + schedEventsHandler.receivedEvent, "Event should be sent when there is change in labels"); + assertEquals(1 +, schedEventsHandler.updatedNodeToLabels.size(), "Single node label mapping modified"); assertCollectionEquals(toSet("p1"), schedEventsHandler.updatedNodeToLabels.get(toNodeId("n2:1"))); schedEventsHandler.receivedEvent = false; mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n3"), toSet("p2"))); - assertTrue("Event should be sent when there is change in labels @ HOST", - schedEventsHandler.receivedEvent); - assertEquals("Single node label mapping modified", 1, - schedEventsHandler.updatedNodeToLabels.size()); + assertTrue( + schedEventsHandler.receivedEvent, "Event should be sent when there is change in labels @ HOST"); + assertEquals(1 +, schedEventsHandler.updatedNodeToLabels.size(), "Single node label mapping modified"); assertCollectionEquals(toSet("p2"), schedEventsHandler.updatedNodeToLabels.get(toNodeId("n3:1"))); schedEventsHandler.receivedEvent = false; mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1"), toSet("p2"))); assertTrue( - "Event should be sent when labels are modified at host though labels were set @ NM level", - schedEventsHandler.receivedEvent); - assertEquals("Single node label mapping modified", 1, - schedEventsHandler.updatedNodeToLabels.size()); + + schedEventsHandler.receivedEvent, "Event should be sent when labels are modified at host though labels were set @ NM level"); + assertEquals(1 +, schedEventsHandler.updatedNodeToLabels.size(), "Single node label mapping modified"); assertCollectionEquals(toSet("p2"), schedEventsHandler.updatedNodeToLabels.get(toNodeId("n1:1"))); schedEventsHandler.receivedEvent = false; } - @Test(timeout = 5000) + @Test + @Timeout(value = 5) public void testGetLabelsOnNodesWhenNodeActiveDeactive() throws Exception { mgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("p1", "p2", "p3")); mgr.replaceLabelsOnNode(ImmutableMap.of( @@ -560,7 +568,7 @@ public void testGetLabelsOnNodesWhenNodeActiveDeactive() throws Exception { assertCollectionEquals(toSet("p2"), mgr.getNodeLabels().get(toNodeId("n1:2"))); mgr.deactivateNode(toNodeId("n1:2")); - Assert.assertNull(mgr.getNodeLabels().get(toNodeId("n1:2"))); + Assertions.assertNull(mgr.getNodeLabels().get(toNodeId("n1:2"))); // Host will not affected too assertCollectionEquals(toSet("p2"), mgr.getNodeLabels().get(toNodeId("n1"))); @@ -574,7 +582,7 @@ public void testGetLabelsOnNodesWhenNodeActiveDeactive() throws Exception { assertCollectionEquals(toSet("p3"), mgr.getNodeLabels().get(toNodeId("n1:2"))); mgr.deactivateNode(toNodeId("n1:2")); - Assert.assertNull(mgr.getNodeLabels().get(toNodeId("n1:2"))); + Assertions.assertNull(mgr.getNodeLabels().get(toNodeId("n1:2"))); // Host will not affected too assertCollectionEquals(toSet("p3"), mgr.getNodeLabels().get(toNodeId("n1"))); @@ -583,15 +591,16 @@ public void testGetLabelsOnNodesWhenNodeActiveDeactive() throws Exception { private void checkNodeLabelInfo(List infos, String labelName, int activeNMs, int memory) { for (RMNodeLabel info : infos) { if (info.getLabelName().equals(labelName)) { - Assert.assertEquals(activeNMs, info.getNumActiveNMs()); - Assert.assertEquals(memory, info.getResource().getMemorySize()); + Assertions.assertEquals(activeNMs, info.getNumActiveNMs()); + Assertions.assertEquals(memory, info.getResource().getMemorySize()); return; } } - Assert.fail("Failed to find info has label=" + labelName); + Assertions.fail("Failed to find info has label=" + labelName); } - @Test(timeout = 5000) + @Test + @Timeout(value = 5) public void testPullRMNodeLabelsInfo() throws IOException { mgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("x", "y", "z")); mgr.activateNode(NodeId.newInstance("n1", 1), Resource.newInstance(10, 0)); @@ -604,14 +613,15 @@ public void testPullRMNodeLabelsInfo() throws IOException { // x, y, z and "" List infos = mgr.pullRMNodeLabelsInfo(); - Assert.assertEquals(4, infos.size()); + Assertions.assertEquals(4, infos.size()); checkNodeLabelInfo(infos, RMNodeLabelsManager.NO_LABEL, 2, 20); checkNodeLabelInfo(infos, "x", 2, 20); checkNodeLabelInfo(infos, "y", 1, 10); checkNodeLabelInfo(infos, "z", 0, 0); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testcheckRemoveFromClusterNodeLabelsOfQueue() throws Exception { lmgr = new RMNodeLabelsManager(); Configuration conf = new Configuration(); @@ -640,14 +650,14 @@ protected void checkRemoveFromClusterNodeLabelsOfQueue( } lmgr = new TestRMLabelManger(); MockRM rm2 = initRM(withQueueLabels); - Assert.assertFalse( - "checkRemoveFromClusterNodeLabelsOfQueue should not be called" - + "on recovery", - checkQueueCall); + Assertions.assertFalse( + + checkQueueCall, "checkRemoveFromClusterNodeLabelsOfQueue should not be called" + + "on recovery"); lmgr.removeFromClusterNodeLabels(Arrays.asList(new String[] { "x" })); - Assert - .assertTrue("checkRemoveFromClusterNodeLabelsOfQueue should be called " - + "since its not recovery", checkQueueCall); + Assertions + .assertTrue(checkQueueCall, "checkRemoveFromClusterNodeLabelsOfQueue should be called " + + "since its not recovery"); rm2.stop(); } @@ -660,7 +670,7 @@ public RMNodeLabelsManager createNodeLabelManager() { }; rm.getRMContext().setNodeLabelManager(lmgr); rm.start(); - Assert.assertEquals(Service.STATE.STARTED, rm.getServiceState()); + Assertions.assertEquals(Service.STATE.STARTED, rm.getServiceState()); return rm; } @@ -679,11 +689,12 @@ private Configuration getConfigurationWithQueueLabels(Configuration config) { return conf; } - @Test(timeout = 5000) + @Test + @Timeout(value = 5) public void testLabelsToNodesOnNodeActiveDeactive() throws Exception { // Activate a node without assigning any labels mgr.activateNode(NodeId.newInstance("n1", 1), Resource.newInstance(10, 0)); - Assert.assertTrue(mgr.getLabelsToNodes().isEmpty()); + Assertions.assertTrue(mgr.getLabelsToNodes().isEmpty()); assertLabelsToNodesEquals( mgr.getLabelsToNodes(), transposeNodeToLabels(mgr.getNodeLabels())); @@ -691,26 +702,27 @@ public void testLabelsToNodesOnNodeActiveDeactive() throws Exception { mgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("p1")); mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1"), toSet("p1"))); // p1 -> n1, n1:1 - Assert.assertEquals(2, mgr.getLabelsToNodes().get("p1").size()); + Assertions.assertEquals(2, mgr.getLabelsToNodes().get("p1").size()); assertLabelsToNodesEquals( mgr.getLabelsToNodes(), transposeNodeToLabels(mgr.getNodeLabels())); // Activate a node for which host to label mapping exists mgr.activateNode(NodeId.newInstance("n1", 2), Resource.newInstance(10, 0)); // p1 -> n1, n1:1, n1:2 - Assert.assertEquals(3, mgr.getLabelsToNodes().get("p1").size()); + Assertions.assertEquals(3, mgr.getLabelsToNodes().get("p1").size()); assertLabelsToNodesEquals( mgr.getLabelsToNodes(), transposeNodeToLabels(mgr.getNodeLabels())); // Deactivate a node. n1:1 will be removed from the map mgr.deactivateNode(NodeId.newInstance("n1", 1)); // p1 -> n1, n1:2 - Assert.assertEquals(2, mgr.getLabelsToNodes().get("p1").size()); + Assertions.assertEquals(2, mgr.getLabelsToNodes().get("p1").size()); assertLabelsToNodesEquals( mgr.getLabelsToNodes(), transposeNodeToLabels(mgr.getNodeLabels())); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testBackwardsCompatableMirror() throws Exception { lmgr = new RMNodeLabelsManager(); Configuration conf = new Configuration(); @@ -747,12 +759,12 @@ public void testBackwardsCompatableMirror() throws Exception { Set labelNames = lmgr.getClusterNodeLabelNames(); Map> labeledNodes = lmgr.getLabelsToNodes(); - Assert.assertTrue(labelNames.contains("a")); - Assert.assertTrue(labelNames.contains("b")); - Assert.assertTrue(labelNames.contains("c")); - Assert.assertTrue(labeledNodes.get("a") + Assertions.assertTrue(labelNames.contains("a")); + Assertions.assertTrue(labelNames.contains("b")); + Assertions.assertTrue(labelNames.contains("c")); + Assertions.assertTrue(labeledNodes.get("a") .contains(NodeId.newInstance("host1", 0))); - Assert.assertTrue(labeledNodes.get("b") + Assertions.assertTrue(labeledNodes.get("b") .contains(NodeId.newInstance("host2", 0))); rm.stop(); From 54697d4c3dfaf801761773ebb5020878746508f8 Mon Sep 17 00:00:00 2001 From: slfan1989 Date: Thu, 6 Mar 2025 17:55:04 +0800 Subject: [PATCH 2/6] YARN-11262. Fix CheckStyle Issue & Junit Test. --- .../ahs/TestRMApplicationHistoryWriter.java | 74 +++++---- .../TestAMRMRPCNodeUpdates.java | 76 ++++----- .../applicationsmanager/TestAMRestart.java | 142 ++++++++-------- .../blacklist/TestBlacklistManager.java | 51 +++--- .../TestFederationRMStateStoreService.java | 102 ++++++------ .../TestRMAppLogAggregationStatus.java | 115 +++++++------ .../TestCombinedSystemMetricsPublisher.java | 47 +++--- .../metrics/TestSystemMetricsPublisher.java | 152 +++++++++--------- 8 files changed, 380 insertions(+), 379 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/TestRMApplicationHistoryWriter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/TestRMApplicationHistoryWriter.java index 193dbd064b7a7..a172b6ebd1e88 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/TestRMApplicationHistoryWriter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/TestRMApplicationHistoryWriter.java @@ -18,6 +18,11 @@ package org.apache.hadoop.yarn.server.resourcemanager.ahs; +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.Mockito.mock; import static org.mockito.Mockito.when; @@ -28,7 +33,6 @@ import java.util.Random; import org.apache.hadoop.yarn.server.resourcemanager.MockRMAppSubmitter; -import org.junit.jupiter.api.Assertions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; @@ -188,8 +192,8 @@ public void testDefaultStoreSetup() throws Exception { writer.init(conf); writer.start(); try { - Assertions.assertFalse(writer.historyServiceEnabled); - Assertions.assertNull(writer.writer); + assertFalse(writer.historyServiceEnabled); + assertNull(writer.writer); } finally { writer.stop(); writer.close(); @@ -210,13 +214,13 @@ public void testWriteApplication() throws Exception { Thread.sleep(100); } } - Assertions.assertNotNull(appHD); - Assertions.assertEquals("test app", appHD.getApplicationName()); - Assertions.assertEquals("test app type", appHD.getApplicationType()); - Assertions.assertEquals("test user", appHD.getUser()); - Assertions.assertEquals("test queue", appHD.getQueue()); - Assertions.assertEquals(0L, appHD.getSubmitTime()); - Assertions.assertEquals(1L, appHD.getStartTime()); + assertNotNull(appHD); + assertEquals("test app", appHD.getApplicationName()); + assertEquals("test app type", appHD.getApplicationType()); + assertEquals("test user", appHD.getUser()); + assertEquals("test queue", appHD.getQueue()); + assertEquals(0L, appHD.getSubmitTime()); + assertEquals(1L, appHD.getStartTime()); writer.applicationFinished(app, RMAppState.FINISHED); for (int i = 0; i < MAX_RETRIES; ++i) { @@ -227,11 +231,11 @@ public void testWriteApplication() throws Exception { Thread.sleep(100); } } - Assertions.assertEquals(2L, appHD.getFinishTime()); - Assertions.assertEquals("test diagnostics info", appHD.getDiagnosticsInfo()); - Assertions.assertEquals(FinalApplicationStatus.UNDEFINED, + assertEquals(2L, appHD.getFinishTime()); + assertEquals("test diagnostics info", appHD.getDiagnosticsInfo()); + assertEquals(FinalApplicationStatus.UNDEFINED, appHD.getFinalApplicationStatus()); - Assertions.assertEquals(YarnApplicationState.FINISHED, + assertEquals(YarnApplicationState.FINISHED, appHD.getYarnApplicationState()); } @@ -252,10 +256,10 @@ public void testWriteApplicationAttempt() throws Exception { Thread.sleep(100); } } - Assertions.assertNotNull(appAttemptHD); - Assertions.assertEquals("test host", appAttemptHD.getHost()); - Assertions.assertEquals(-100, appAttemptHD.getRPCPort()); - Assertions.assertEquals(ContainerId.newContainerId( + assertNotNull(appAttemptHD); + assertEquals("test host", appAttemptHD.getHost()); + assertEquals(-100, appAttemptHD.getRPCPort()); + assertEquals(ContainerId.newContainerId( ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 1), 1), 1), appAttemptHD.getMasterContainerId()); @@ -270,12 +274,12 @@ public void testWriteApplicationAttempt() throws Exception { Thread.sleep(100); } } - Assertions.assertEquals("test diagnostics info", + assertEquals("test diagnostics info", appAttemptHD.getDiagnosticsInfo()); - Assertions.assertEquals("test url", appAttemptHD.getTrackingURL()); - Assertions.assertEquals(FinalApplicationStatus.UNDEFINED, + assertEquals("test url", appAttemptHD.getTrackingURL()); + assertEquals(FinalApplicationStatus.UNDEFINED, appAttemptHD.getFinalApplicationStatus()); - Assertions.assertEquals(YarnApplicationAttemptState.FINISHED, + assertEquals(YarnApplicationAttemptState.FINISHED, appAttemptHD.getYarnApplicationAttemptState()); } @@ -297,13 +301,13 @@ public void testWriteContainer() throws Exception { Thread.sleep(100); } } - Assertions.assertNotNull(containerHD); - Assertions.assertEquals(NodeId.newInstance("test host", -100), + assertNotNull(containerHD); + assertEquals(NodeId.newInstance("test host", -100), containerHD.getAssignedNode()); - Assertions.assertEquals(Resource.newInstance(-1, -1), + assertEquals(Resource.newInstance(-1, -1), containerHD.getAllocatedResource()); - Assertions.assertEquals(Priority.UNDEFINED, containerHD.getPriority()); - Assertions.assertEquals(0L, container.getCreationTime()); + assertEquals(Priority.UNDEFINED, containerHD.getPriority()); + assertEquals(0L, container.getCreationTime()); writer.containerFinished(container); for (int i = 0; i < MAX_RETRIES; ++i) { @@ -316,10 +320,10 @@ public void testWriteContainer() throws Exception { Thread.sleep(100); } } - Assertions.assertEquals("test diagnostics info", + assertEquals("test diagnostics info", containerHD.getDiagnosticsInfo()); - Assertions.assertEquals(-1, containerHD.getContainerExitStatus()); - Assertions.assertEquals(ContainerState.COMPLETE, + assertEquals(-1, containerHD.getContainerExitStatus()); + assertEquals(ContainerState.COMPLETE, containerHD.getContainerState()); } @@ -355,10 +359,10 @@ public void testParallelWrite() throws Exception { Thread.sleep(500); } } - Assertions.assertTrue(allEventsHandled(20 * 10 * 10 + 20 * 10 + 20)); + assertTrue(allEventsHandled(20 * 10 * 10 + 20 * 10 + 20)); // Validate all events of one application are handled by one dispatcher for (ApplicationId appId : appIds) { - Assertions.assertTrue(handledByOne(appId)); + assertTrue(handledByOne(appId)); } } @@ -438,7 +442,7 @@ public void containerFinished(RMContainer container) { long elapsedTime2 = finishTime2 - startTime2; // No more than 10% additional workload // Should be much less, but computation time is fluctuated - Assertions.assertTrue(elapsedTime2 - elapsedTime1 < elapsedTime1 / 10); + assertTrue(elapsedTime2 - elapsedTime1 < elapsedTime1 / 10); } private void testRMWritingMassiveHistory(MockRM rm) throws Exception { @@ -470,7 +474,7 @@ private void testRMWritingMassiveHistory(MockRM rm) throws Exception { allocatedSize += allocated.size(); nm.nodeHeartbeat(true); } - Assertions.assertEquals(request, allocatedSize); + assertEquals(request, allocatedSize); am.unregisterAppAttempt(); rm.waitForState(am.getApplicationAttemptId(), RMAppAttemptState.FINISHING); @@ -487,7 +491,7 @@ private void testRMWritingMassiveHistory(MockRM rm) throws Exception { cleaned = resp.getContainersToCleanup(); cleanedSize += cleaned.size(); } - Assertions.assertEquals(allocatedSize, cleanedSize); + assertEquals(allocatedSize, cleanedSize); rm.waitForState(app.getApplicationId(), RMAppState.FINISHED); 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/applicationsmanager/TestAMRMRPCNodeUpdates.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRMRPCNodeUpdates.java index f9ff438fcf0dc..417e5256941f7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRMRPCNodeUpdates.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRMRPCNodeUpdates.java @@ -18,12 +18,14 @@ package org.apache.hadoop.yarn.server.resourcemanager.applicationsmanager; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; + import java.security.PrivilegedExceptionAction; import java.util.List; import org.apache.hadoop.yarn.server.resourcemanager.MockRMAppSubmitter; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType; -import org.junit.jupiter.api.Assertions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; @@ -139,11 +141,11 @@ public void testAMRMDecommissioningNodes() throws Exception { AllocateResponse response1 = allocate(attempt1.getAppAttemptId(), allocateRequest1); List updatedNodes = response1.getUpdatedNodes(); - Assertions.assertEquals(1, updatedNodes.size()); + assertEquals(1, updatedNodes.size()); NodeReport nr = updatedNodes.iterator().next(); - Assertions.assertEquals( + assertEquals( decommissioningTimeout, nr.getDecommissioningTimeout()); - Assertions.assertEquals( + assertEquals( NodeUpdateType.NODE_DECOMMISSIONING, nr.getNodeUpdateType()); } @@ -173,11 +175,11 @@ public void testAMRMRecommissioningNodes() throws Exception { AllocateResponse response1 = allocate(attempt1.getAppAttemptId(), allocateRequest1); List updatedNodes = response1.getUpdatedNodes(); - Assertions.assertEquals(1, updatedNodes.size()); + assertEquals(1, updatedNodes.size()); NodeReport nr = updatedNodes.iterator().next(); - Assertions.assertEquals( + assertEquals( decommissioningTimeout, nr.getDecommissioningTimeout()); - Assertions.assertEquals( + assertEquals( NodeUpdateType.NODE_DECOMMISSIONING, nr.getNodeUpdateType()); // Wait for nm2 to RECOMMISSION @@ -188,9 +190,9 @@ public void testAMRMRecommissioningNodes() throws Exception { AllocateResponse response2 = allocate(attempt1.getAppAttemptId(), allocateRequest2); List updatedNodes2 = response2.getUpdatedNodes(); - Assertions.assertEquals(1, updatedNodes2.size()); + assertEquals(1, updatedNodes2.size()); NodeReport nr2 = updatedNodes2.iterator().next(); - Assertions.assertEquals( + assertEquals( NodeUpdateType.NODE_USABLE, nr2.getNodeUpdateType()); } @@ -220,7 +222,7 @@ public void testAMRMUnusableNodes() throws Exception { AllocateResponse response1 = allocate(attempt1.getAppAttemptId(), allocateRequest1); List updatedNodes = response1.getUpdatedNodes(); - Assertions.assertEquals(0, updatedNodes.size()); + assertEquals(0, updatedNodes.size()); syncNodeHeartbeat(nm4, false); @@ -230,22 +232,22 @@ public void testAMRMUnusableNodes() throws Exception { null); response1 = allocate(attempt1.getAppAttemptId(), allocateRequest1); updatedNodes = response1.getUpdatedNodes(); - Assertions.assertEquals(1, updatedNodes.size()); + assertEquals(1, updatedNodes.size()); NodeReport nr = updatedNodes.iterator().next(); - Assertions.assertEquals(nm4.getNodeId(), nr.getNodeId()); - Assertions.assertEquals(NodeState.UNHEALTHY, nr.getNodeState()); - Assertions.assertNull(nr.getDecommissioningTimeout()); - Assertions.assertEquals(NodeUpdateType.NODE_UNUSABLE, nr.getNodeUpdateType()); + assertEquals(nm4.getNodeId(), nr.getNodeId()); + assertEquals(NodeState.UNHEALTHY, nr.getNodeState()); + assertNull(nr.getDecommissioningTimeout()); + assertEquals(NodeUpdateType.NODE_UNUSABLE, nr.getNodeUpdateType()); // resending the allocate request returns the same result response1 = allocate(attempt1.getAppAttemptId(), allocateRequest1); updatedNodes = response1.getUpdatedNodes(); - Assertions.assertEquals(1, updatedNodes.size()); + assertEquals(1, updatedNodes.size()); nr = updatedNodes.iterator().next(); - Assertions.assertEquals(nm4.getNodeId(), nr.getNodeId()); - Assertions.assertEquals(NodeState.UNHEALTHY, nr.getNodeState()); - Assertions.assertNull(nr.getDecommissioningTimeout()); - Assertions.assertEquals(NodeUpdateType.NODE_UNUSABLE, nr.getNodeUpdateType()); + assertEquals(nm4.getNodeId(), nr.getNodeId()); + assertEquals(NodeState.UNHEALTHY, nr.getNodeState()); + assertNull(nr.getDecommissioningTimeout()); + assertEquals(NodeUpdateType.NODE_UNUSABLE, nr.getNodeUpdateType()); syncNodeLost(nm3); @@ -255,12 +257,12 @@ public void testAMRMUnusableNodes() throws Exception { null); response1 = allocate(attempt1.getAppAttemptId(), allocateRequest1); updatedNodes = response1.getUpdatedNodes(); - Assertions.assertEquals(1, updatedNodes.size()); + assertEquals(1, updatedNodes.size()); nr = updatedNodes.iterator().next(); - Assertions.assertEquals(nm3.getNodeId(), nr.getNodeId()); - Assertions.assertEquals(NodeState.LOST, nr.getNodeState()); - Assertions.assertNull(nr.getDecommissioningTimeout()); - Assertions.assertEquals(NodeUpdateType.NODE_UNUSABLE, nr.getNodeUpdateType()); + assertEquals(nm3.getNodeId(), nr.getNodeId()); + assertEquals(NodeState.LOST, nr.getNodeState()); + assertNull(nr.getDecommissioningTimeout()); + assertEquals(NodeUpdateType.NODE_UNUSABLE, nr.getNodeUpdateType()); // registering another AM gives it the complete failed list RMApp app2 = MockRMAppSubmitter.submitWithMemory(2000, rm); @@ -278,7 +280,7 @@ public void testAMRMUnusableNodes() throws Exception { AllocateResponse response2 = allocate(attempt2.getAppAttemptId(), allocateRequest2); updatedNodes = response2.getUpdatedNodes(); - Assertions.assertEquals(0, updatedNodes.size()); + assertEquals(0, updatedNodes.size()); syncNodeHeartbeat(nm4, true); @@ -288,24 +290,24 @@ public void testAMRMUnusableNodes() throws Exception { null); response1 = allocate(attempt1.getAppAttemptId(), allocateRequest1); updatedNodes = response1.getUpdatedNodes(); - Assertions.assertEquals(1, updatedNodes.size()); + assertEquals(1, updatedNodes.size()); nr = updatedNodes.iterator().next(); - Assertions.assertEquals(nm4.getNodeId(), nr.getNodeId()); - Assertions.assertEquals(NodeState.RUNNING, nr.getNodeState()); - Assertions.assertNull(nr.getDecommissioningTimeout()); - Assertions.assertEquals(NodeUpdateType.NODE_USABLE, nr.getNodeUpdateType()); + assertEquals(nm4.getNodeId(), nr.getNodeId()); + assertEquals(NodeState.RUNNING, nr.getNodeState()); + assertNull(nr.getDecommissioningTimeout()); + assertEquals(NodeUpdateType.NODE_USABLE, nr.getNodeUpdateType()); allocateRequest2 = AllocateRequest.newInstance(response2.getResponseId(), 0F, null, null, null); response2 = allocate(attempt2.getAppAttemptId(), allocateRequest2); updatedNodes = response2.getUpdatedNodes(); - Assertions.assertEquals(1, updatedNodes.size()); + assertEquals(1, updatedNodes.size()); nr = updatedNodes.iterator().next(); - Assertions.assertEquals(nm4.getNodeId(), nr.getNodeId()); - Assertions.assertEquals(NodeState.RUNNING, nr.getNodeState()); - Assertions.assertNull(nr.getDecommissioningTimeout()); - Assertions.assertEquals(NodeUpdateType.NODE_USABLE, nr.getNodeUpdateType()); + assertEquals(nm4.getNodeId(), nr.getNodeId()); + assertEquals(NodeState.RUNNING, nr.getNodeState()); + assertNull(nr.getDecommissioningTimeout()); + assertEquals(NodeUpdateType.NODE_USABLE, nr.getNodeUpdateType()); // subsequent allocate calls should return no updated nodes allocateRequest2 = @@ -313,7 +315,7 @@ public void testAMRMUnusableNodes() throws Exception { null); response2 = allocate(attempt2.getAppAttemptId(), allocateRequest2); updatedNodes = response2.getUpdatedNodes(); - Assertions.assertEquals(0, updatedNodes.size()); + assertEquals(0, updatedNodes.size()); // how to do the above for LOST node diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java index 630a6200d8645..9006c6b57d782 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java @@ -18,6 +18,13 @@ package org.apache.hadoop.yarn.server.resourcemanager.applicationsmanager; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; + import java.io.IOException; import java.util.ArrayList; import java.util.Collection; @@ -72,7 +79,6 @@ import org.apache.hadoop.thirdparty.com.google.common.base.Throwables; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -160,7 +166,7 @@ public void testAMRestartWithExistingContainers() throws Exception { Thread.sleep(200); } // assert containerId6 is reserved. - Assertions.assertEquals(containerId6, schedulerAttempt.getReservedContainers() + assertEquals(containerId6, schedulerAttempt.getReservedContainers() .get(0).getContainerId()); // fail the AM by sending CONTAINER_FINISHED event without registering. @@ -172,15 +178,15 @@ public void testAMRestartWithExistingContainers() throws Exception { Thread.sleep(3000); rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING); // acquired/allocated containers are cleaned up. - Assertions.assertNull(rm1.getResourceScheduler().getRMContainer(containerId4)); - Assertions.assertNull(rm1.getResourceScheduler().getRMContainer(containerId5)); + assertNull(rm1.getResourceScheduler().getRMContainer(containerId4)); + assertNull(rm1.getResourceScheduler().getRMContainer(containerId5)); // wait for app to start a new attempt. rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); // assert this is a new AM. ApplicationAttemptId newAttemptId = app1.getCurrentAppAttempt().getAppAttemptId(); - Assertions.assertFalse(newAttemptId.equals(am1.getApplicationAttemptId())); + assertFalse(newAttemptId.equals(am1.getApplicationAttemptId())); // launch the new AM MockAM am2 = rm1.launchAM(app1, rm1, nm1); @@ -189,7 +195,7 @@ public void testAMRestartWithExistingContainers() throws Exception { // Assert two containers are running: container2 and container3; - Assertions.assertEquals(2, registerResponse.getContainersFromPreviousAttempts() + assertEquals(2, registerResponse.getContainersFromPreviousAttempts() .size()); boolean containerId2Exists = false, containerId3Exists = false; for (Container container : registerResponse @@ -201,7 +207,7 @@ public void testAMRestartWithExistingContainers() throws Exception { containerId3Exists = true; } } - Assertions.assertTrue(containerId2Exists && containerId3Exists); + assertTrue(containerId2Exists && containerId3Exists); rm1.waitForState(app1.getApplicationId(), RMAppState.RUNNING); // complete container by sending the container complete event which has earlier @@ -238,7 +244,7 @@ public void testAMRestartWithExistingContainers() throws Exception { container6Exists = true; } } - Assertions.assertTrue(container3Exists && container4Exists && container5Exists + assertTrue(container3Exists && container4Exists && container5Exists && container6Exists); // New SchedulerApplicationAttempt also has the containers info. @@ -253,7 +259,7 @@ public void testAMRestartWithExistingContainers() throws Exception { // the 2nd attempt released the 1st attempt's running container, when the // 2nd attempt finishes. - Assertions.assertFalse(schedulerNewAttempt.getLiveContainers().contains( + assertFalse(schedulerNewAttempt.getLiveContainers().contains( containerId2)); // all 4 normal containers finished. System.out.println("New attempt's just finished containers: " @@ -280,7 +286,7 @@ public static List allocateContainers(MockNM nm1, MockAM am1, Thread.sleep(200); } - Assertions.assertEquals( + assertEquals( NUM_CONTAINERS, containers.size(), "Did not get all containers allocated"); return containers; } @@ -372,10 +378,10 @@ public void testNMTokensRebindOnAMRestart() throws Exception { rm1.waitForState(am2.getApplicationAttemptId(), RMAppAttemptState.RUNNING); // check am2 get the nm token from am1. - Assertions.assertEquals(expectedNMTokens.size(), + assertEquals(expectedNMTokens.size(), registerResponse.getNMTokensFromPreviousAttempts().size()); for (int i = 0; i < expectedNMTokens.size(); i++) { - Assertions.assertTrue(expectedNMTokens.get(i) + assertTrue(expectedNMTokens.get(i) .equals(registerResponse.getNMTokensFromPreviousAttempts().get(i))); } @@ -411,8 +417,8 @@ public void testNMTokensRebindOnAMRestart() throws Exception { // check am3 get the NM token from both am1 and am2; List transferredTokens = registerResponse.getNMTokensFromPreviousAttempts(); - Assertions.assertEquals(2, transferredTokens.size()); - Assertions.assertTrue(transferredTokens.containsAll(expectedNMTokens)); + assertEquals(2, transferredTokens.size()); + assertTrue(transferredTokens.containsAll(expectedNMTokens)); rm1.stop(); } @@ -446,7 +452,7 @@ public void testShouldNotCountFailureToMaxAttemptRetry() throws Exception { TestSchedulerUtils.waitSchedulerApplicationAttemptStopped(scheduler, am1.getApplicationAttemptId()); - Assertions.assertFalse(attempt1.shouldCountTowardsMaxAttemptRetry()); + assertFalse(attempt1.shouldCountTowardsMaxAttemptRetry()); rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); ApplicationStateData appState = ((MemoryRMStateStore) rm1.getRMStateStore()).getState() @@ -466,7 +472,7 @@ public void testShouldNotCountFailureToMaxAttemptRetry() throws Exception { TestSchedulerUtils.waitSchedulerApplicationAttemptStopped(scheduler, am2.getApplicationAttemptId()); - Assertions.assertFalse(attempt2.shouldCountTowardsMaxAttemptRetry()); + assertFalse(attempt2.shouldCountTowardsMaxAttemptRetry()); rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); MockAM am3 = rm1.waitForNewAMToLaunchAndRegister(app1.getApplicationId(), 3, nm1); @@ -488,8 +494,8 @@ public void testShouldNotCountFailureToMaxAttemptRetry() throws Exception { TestSchedulerUtils.waitSchedulerApplicationAttemptStopped(scheduler, am3.getApplicationAttemptId()); - Assertions.assertFalse(attempt3.shouldCountTowardsMaxAttemptRetry()); - Assertions.assertEquals(ContainerExitStatus.DISKS_FAILED, + assertFalse(attempt3.shouldCountTowardsMaxAttemptRetry()); + assertEquals(ContainerExitStatus.DISKS_FAILED, appState.getAttempt(am3.getApplicationAttemptId()) .getAMContainerExitStatus()); @@ -509,8 +515,8 @@ public void testShouldNotCountFailureToMaxAttemptRetry() throws Exception { TestSchedulerUtils.waitSchedulerApplicationAttemptStopped(scheduler, am4.getApplicationAttemptId()); - Assertions.assertFalse(attempt4.shouldCountTowardsMaxAttemptRetry()); - Assertions.assertEquals(ContainerExitStatus.ABORTED, + assertFalse(attempt4.shouldCountTowardsMaxAttemptRetry()); + assertEquals(ContainerExitStatus.ABORTED, appState.getAttempt(am4.getApplicationAttemptId()) .getAMContainerExitStatus()); // launch next AM in nm2 @@ -524,7 +530,7 @@ public void testShouldNotCountFailureToMaxAttemptRetry() throws Exception { TestSchedulerUtils.waitSchedulerApplicationAttemptStopped(scheduler, am5.getApplicationAttemptId()); - Assertions.assertTrue(attempt5.shouldCountTowardsMaxAttemptRetry()); + assertTrue(attempt5.shouldCountTowardsMaxAttemptRetry()); // launch next AM in nm2 MockAM am6 = @@ -538,11 +544,11 @@ public void testShouldNotCountFailureToMaxAttemptRetry() throws Exception { TestSchedulerUtils.waitSchedulerApplicationAttemptStopped(scheduler, am6.getApplicationAttemptId()); - Assertions.assertTrue(attempt6.shouldCountTowardsMaxAttemptRetry()); + assertTrue(attempt6.shouldCountTowardsMaxAttemptRetry()); // AM should not be restarted. rm1.waitForState(app1.getApplicationId(), RMAppState.FAILED); - Assertions.assertEquals(6, app1.getAppAttempts().size()); + assertEquals(6, app1.getAppAttempts().size()); rm1.stop(); } @@ -574,7 +580,7 @@ public void testMaxAttemptOneMeansOne() throws Exception { // AM should not be restarted. rm1.waitForState(app1.getApplicationId(), RMAppState.FAILED); - Assertions.assertEquals(1, app1.getAppAttempts().size()); + assertEquals(1, app1.getAppAttempts().size()); rm1.stop(); } @@ -614,7 +620,7 @@ public void testPreemptedAMRestartOnRMRestart() throws Exception { rm1.waitForState(am1.getApplicationAttemptId(), RMAppAttemptState.FAILED); TestSchedulerUtils.waitSchedulerApplicationAttemptStopped(scheduler, am1.getApplicationAttemptId()); - Assertions.assertTrue(attempt1.shouldCountTowardsMaxAttemptRetry()); + assertTrue(attempt1.shouldCountTowardsMaxAttemptRetry()); // wait for the next AM to start rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); @@ -627,21 +633,21 @@ public void testPreemptedAMRestartOnRMRestart() throws Exception { scheduler.killContainer(scheduler.getRMContainer(amContainer)); rm1.waitForState(am2.getApplicationAttemptId(), RMAppAttemptState.FAILED); - Assertions.assertFalse(attempt2.shouldCountTowardsMaxAttemptRetry()); + assertFalse(attempt2.shouldCountTowardsMaxAttemptRetry()); rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); // state store has 2 attempts stored. ApplicationStateData appState = memStore.getState().getApplicationState().get(app1.getApplicationId()); - Assertions.assertEquals(2, appState.getAttemptCount()); + assertEquals(2, appState.getAttemptCount()); if (getSchedulerType().equals(SchedulerType.FAIR)) { // attempt stored has the preempted container exit status. - Assertions.assertEquals(ContainerExitStatus.KILLED_BY_RESOURCEMANAGER, + assertEquals(ContainerExitStatus.KILLED_BY_RESOURCEMANAGER, appState.getAttempt(am2.getApplicationAttemptId()) .getAMContainerExitStatus()); } else { // attempt stored has the preempted container exit status. - Assertions.assertEquals(ContainerExitStatus.PREEMPTED, + assertEquals(ContainerExitStatus.PREEMPTED, appState.getAttempt(am2.getApplicationAttemptId()) .getAMContainerExitStatus()); } @@ -658,8 +664,8 @@ public void testPreemptedAMRestartOnRMRestart() throws Exception { RMAppAttempt attempt3 = rm2.getRMContext().getRMApps().get(app1.getApplicationId()) .getCurrentAppAttempt(); - Assertions.assertTrue(attempt3.shouldCountTowardsMaxAttemptRetry()); - Assertions.assertEquals(ContainerExitStatus.INVALID, + assertTrue(attempt3.shouldCountTowardsMaxAttemptRetry()); + assertEquals(ContainerExitStatus.INVALID, appState.getAttempt(am3.getApplicationAttemptId()) .getAMContainerExitStatus()); rm1.stop(); @@ -703,7 +709,7 @@ public void testRMRestartOrFailoverNotCountedForAMFailures() rm1.waitForState(am1.getApplicationAttemptId(), RMAppAttemptState.FAILED); TestSchedulerUtils.waitSchedulerApplicationAttemptStopped(scheduler, am1.getApplicationAttemptId()); - Assertions.assertTrue(attempt1.shouldCountTowardsMaxAttemptRetry()); + assertTrue(attempt1.shouldCountTowardsMaxAttemptRetry()); // wait for the next AM to start rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); @@ -727,7 +733,7 @@ public void testRMRestartOrFailoverNotCountedForAMFailures() nm1.registerNode(Collections.singletonList(status), null); rm2.waitForState(attempt2.getAppAttemptId(), RMAppAttemptState.FAILED); - Assertions.assertEquals(ContainerExitStatus.KILLED_BY_RESOURCEMANAGER, + assertEquals(ContainerExitStatus.KILLED_BY_RESOURCEMANAGER, appState.getAttempt(am2.getApplicationAttemptId()) .getAMContainerExitStatus()); // Will automatically start a new AppAttempt in rm2 @@ -738,8 +744,8 @@ public void testRMRestartOrFailoverNotCountedForAMFailures() RMAppAttempt attempt3 = rm2.getRMContext().getRMApps().get(app1.getApplicationId()) .getCurrentAppAttempt(); - Assertions.assertTrue(attempt3.shouldCountTowardsMaxAttemptRetry()); - Assertions.assertEquals(ContainerExitStatus.INVALID, + assertTrue(attempt3.shouldCountTowardsMaxAttemptRetry()); + assertEquals(ContainerExitStatus.INVALID, appState.getAttempt(am3.getApplicationAttemptId()) .getAMContainerExitStatus()); @@ -786,7 +792,7 @@ public void testRMAppAttemptFailuresValidityInterval() throws Exception { rm1.waitForState(am.getApplicationAttemptId(), RMAppAttemptState.FAILED); // launch the second attempt rm1.waitForState(app.getApplicationId(), RMAppState.ACCEPTED); - Assertions.assertEquals(2, app.getAppAttempts().size()); + assertEquals(2, app.getAppAttempts().size()); MockAM am_2 = MockRM.launchAndRegisterAM(app, rm1, nm1); rm1.waitForState(am_2.getApplicationAttemptId(), RMAppAttemptState.RUNNING); @@ -817,7 +823,7 @@ public void testRMAppAttemptFailuresValidityInterval() throws Exception { // launch the second attempt rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); - Assertions.assertEquals(2, app1.getAppAttempts().size()); + assertEquals(2, app1.getAppAttempts().size()); RMAppAttempt attempt2 = app1.getCurrentAppAttempt(); MockAM am2 = MockRM.launchAndRegisterAM(app1, rm1, nm1); @@ -832,7 +838,7 @@ public void testRMAppAttemptFailuresValidityInterval() throws Exception { // can launch the third attempt successfully rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); - Assertions.assertEquals(3, app1.getAppAttempts().size()); + assertEquals(3, app1.getAppAttempts().size()); RMAppAttempt attempt3 = app1.getCurrentAppAttempt(); clock.reset(); MockAM am3 = MockRM.launchAndRegisterAM(app1, rm1, nm1); @@ -848,7 +854,7 @@ public void testRMAppAttemptFailuresValidityInterval() throws Exception { ApplicationStateData app1State = memStore1.getState().getApplicationState(). get(app1.getApplicationId()); - Assertions.assertEquals(1, app1State.getFirstAttemptId()); + assertEquals(1, app1State.getFirstAttemptId()); // re-register the NM nm1.setResourceTrackerService(rm2.getResourceTrackerService()); @@ -864,7 +870,7 @@ public void testRMAppAttemptFailuresValidityInterval() throws Exception { //Wait to make sure attempt3 be removed in State Store //TODO explore a better way than sleeping for a while (YARN-4929) Thread.sleep(15 * 1000); - Assertions.assertEquals(2, app1State.getAttemptCount()); + assertEquals(2, app1State.getAttemptCount()); rm2.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); @@ -878,7 +884,7 @@ public void testRMAppAttemptFailuresValidityInterval() throws Exception { nm1 .nodeHeartbeat(am4.getApplicationAttemptId(), 1, ContainerState.COMPLETE); rm2.waitForState(am4.getApplicationAttemptId(), RMAppAttemptState.FAILED); - Assertions.assertEquals(2, app1State.getAttemptCount()); + assertEquals(2, app1State.getAttemptCount()); // can launch the 5th attempt successfully rm2.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); @@ -892,7 +898,7 @@ public void testRMAppAttemptFailuresValidityInterval() throws Exception { nm1 .nodeHeartbeat(am5.getApplicationAttemptId(), 1, ContainerState.COMPLETE); rm2.waitForState(am5.getApplicationAttemptId(), RMAppAttemptState.FAILED); - Assertions.assertEquals(2, app1State.getAttemptCount()); + assertEquals(2, app1State.getAttemptCount()); rm2.waitForState(app1.getApplicationId(), RMAppState.FAILED); rm1.stop(); @@ -967,7 +973,7 @@ public void testAMRestartNotLostContainerCompleteMsg() throws Exception { app1.getCurrentAppAttempt().getJustFinishedContainers(); if (isContainerIdInContainerStatus(containerStatuses, containerId2)) { - Assertions.fail(); + fail(); } break; } @@ -982,7 +988,7 @@ public void testAMRestartNotLostContainerCompleteMsg() throws Exception { // assert this is a new AM. ApplicationAttemptId newAttemptId = app1.getCurrentAppAttempt().getAppAttemptId(); - Assertions.assertFalse(newAttemptId.equals(am1.getApplicationAttemptId())); + assertFalse(newAttemptId.equals(am1.getApplicationAttemptId())); // launch the new AM RMAppAttempt attempt2 = app1.getCurrentAppAttempt(); @@ -995,11 +1001,11 @@ public void testAMRestartNotLostContainerCompleteMsg() throws Exception { allocateResponse.getCompletedContainersStatuses(); if (isContainerIdInContainerStatus(containerStatuses, containerId2) == false) { - Assertions.fail(); + fail(); } containerStatuses = attempt2.getJustFinishedContainers(); if (isContainerIdInContainerStatus(containerStatuses, containerId2)) { - Assertions.fail(); + fail(); } // the second allocate should not get container complete msg @@ -1008,7 +1014,7 @@ public void testAMRestartNotLostContainerCompleteMsg() throws Exception { containerStatuses = allocateResponse.getCompletedContainersStatuses(); if (isContainerIdInContainerStatus(containerStatuses, containerId2)) { - Assertions.fail(); + fail(); } rm1.stop(); @@ -1057,7 +1063,7 @@ public void testAMRestartNotLostContainerAfterAttemptFailuresValidityInterval() // launch the second attempt rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); - Assertions.assertEquals(2, app1.getAppAttempts().size()); + assertEquals(2, app1.getAppAttempts().size()); // It will be the last attempt. RMAppAttempt attempt2 = app1.getCurrentAppAttempt(); @@ -1074,20 +1080,20 @@ public void testAMRestartNotLostContainerAfterAttemptFailuresValidityInterval() // can launch the third attempt successfully rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); - Assertions.assertEquals(3, app1.getAppAttempts().size()); + assertEquals(3, app1.getAppAttempts().size()); MockAM am3 = rm1.launchAM(app1, rm1, nm1); RegisterApplicationMasterResponse registerResponse = am3.registerAppAttempt(); // keepContainers is applied, even if attempt2 was the last attempt. - Assertions.assertEquals(1, registerResponse.getContainersFromPreviousAttempts() + assertEquals(1, registerResponse.getContainersFromPreviousAttempts() .size()); boolean containerId2Exists = false; Container container = registerResponse.getContainersFromPreviousAttempts().get(0); if (container.getId().equals(containerId2)) { containerId2Exists = true; } - Assertions.assertTrue(containerId2Exists); + assertTrue(containerId2Exists); rm1.waitForState(app1.getApplicationId(), RMAppState.RUNNING); rm1.stop(); @@ -1193,7 +1199,7 @@ public void testContainersFromPreviousAttemptsWithRMRestart() nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 2, ContainerState.RUNNING); rm2.waitForState(nm1, containerId2, RMContainerState.RUNNING); - Assertions.assertNotNull(rm2.getResourceScheduler() + assertNotNull(rm2.getResourceScheduler() .getRMContainer(containerId2)); // wait for app to start a new attempt. @@ -1201,7 +1207,7 @@ public void testContainersFromPreviousAttemptsWithRMRestart() // assert this is a new AM. ApplicationAttemptId newAttemptId = app1.getCurrentAppAttempt().getAppAttemptId(); - Assertions.assertFalse(newAttemptId.equals(am1.getApplicationAttemptId())); + assertFalse(newAttemptId.equals(am1.getApplicationAttemptId())); // launch the new AM MockAM am2 = MockRM.launchAMWhenAsyncSchedulingEnabled(app1, rm2); @@ -1209,15 +1215,15 @@ public void testContainersFromPreviousAttemptsWithRMRestart() am2.registerAppAttempt(); // container2 is recovered from previous attempt - Assertions.assertEquals(1, + assertEquals(1, registerResponse.getContainersFromPreviousAttempts().size()); - Assertions.assertEquals(containerId2 + assertEquals(containerId2 , registerResponse.getContainersFromPreviousAttempts().get(0).getId(), "container 2"); List prevNMTokens = registerResponse .getNMTokensFromPreviousAttempts(); - Assertions.assertEquals(1, prevNMTokens.size()); + assertEquals(1, prevNMTokens.size()); // container 2 is running on node 1 - Assertions.assertEquals(nm1Address, prevNMTokens.get(0).getNodeId().toString()); + assertEquals(nm1Address, prevNMTokens.get(0).getNodeId().toString()); rm2.waitForState(app1.getApplicationId(), RMAppState.RUNNING); @@ -1231,7 +1237,7 @@ public void testContainersFromPreviousAttemptsWithRMRestart() nm2.nodeHeartbeat(am1.getApplicationAttemptId(), 3, ContainerState.RUNNING); rm2.waitForState(nm2, containerId3, RMContainerState.RUNNING); - Assertions.assertNotNull(rm2.getResourceScheduler() + assertNotNull(rm2.getResourceScheduler() .getRMContainer(containerId3)); List containersFromPreviousAttempts = new ArrayList<>(); @@ -1241,12 +1247,12 @@ public void testContainersFromPreviousAttemptsWithRMRestart() if (allocateResponse.getContainersFromPreviousAttempts().size() > 0){ containersFromPreviousAttempts.addAll( allocateResponse.getContainersFromPreviousAttempts()); - Assertions.assertEquals( + assertEquals( 0, allocateResponse.getAllocatedContainers().size(), "new containers should not be allocated"); List nmTokens = allocateResponse.getNMTokens(); - Assertions.assertEquals(1, nmTokens.size()); + assertEquals(1, nmTokens.size()); // container 3 is running on node 2 - Assertions.assertEquals(nm2Address, + assertEquals(nm2Address, nmTokens.get(0).getNodeId().toString()); return true; } @@ -1256,7 +1262,7 @@ public void testContainersFromPreviousAttemptsWithRMRestart() } return false; }, 2000, 200000); - Assertions.assertEquals(containerId3 + assertEquals(containerId3 , containersFromPreviousAttempts.get(0).getId(), "container 3"); rm2.stop(); rm1.stop(); @@ -1313,7 +1319,7 @@ public void testQueueResourceDoesNotLeak() throws Exception { TestSchedulerUtils.waitSchedulerApplicationAttemptStopped(scheduler, am1.getApplicationAttemptId()); - Assertions.assertFalse(attempt1.shouldCountTowardsMaxAttemptRetry()); + assertFalse(attempt1.shouldCountTowardsMaxAttemptRetry()); // AM should not be restarted. rm1.waitForState(app1.getApplicationId(), RMAppState.FAILED); @@ -1323,20 +1329,20 @@ public void testQueueResourceDoesNotLeak() throws Exception { if (getSchedulerType() == SchedulerType.CAPACITY) { LeafQueue queue = (LeafQueue) ((CapacityScheduler) scheduler).getQueue("default"); - Assertions.assertEquals(0, + assertEquals(0, queue.getQueueResourceUsage().getUsed().getMemorySize()); - Assertions.assertEquals(0, + assertEquals(0, queue.getQueueResourceUsage().getUsed().getVirtualCores()); } else if (getSchedulerType() == SchedulerType.FAIR) { // The default queue is not auto created after YARN-7769 so // user-as-default-queue option is used Collection queues = ((FairScheduler) scheduler) .getQueueManager().getLeafQueues(); - Assertions.assertEquals(1, queues.size()); + assertEquals(1, queues.size()); FSLeafQueue queue = queues.iterator().next(); - Assertions.assertEquals(0, queue.getResourceUsage().getMemorySize()); - Assertions.assertEquals(0, queue.getResourceUsage().getVirtualCores()); + assertEquals(0, queue.getResourceUsage().getMemorySize()); + assertEquals(0, queue.getResourceUsage().getVirtualCores()); } rm1.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/blacklist/TestBlacklistManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/TestBlacklistManager.java index 87b80188f00ab..a9f2038d0636e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/TestBlacklistManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/TestBlacklistManager.java @@ -18,12 +18,13 @@ package org.apache.hadoop.yarn.server.resourcemanager.blacklist; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.Collections; import java.util.List; import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; public class TestBlacklistManager { @@ -45,14 +46,10 @@ public void testSimpleBlacklistBelowFailureThreshold() { Collections.sort(blacklistAdditions); List blacklistRemovals = blacklist.getBlacklistRemovals(); String[] expectedBlacklistAdditions = new String[]{anyNode2, anyNode}; - Assertions.assertArrayEquals( - - expectedBlacklistAdditions -, blacklistAdditions.toArray(), "Blacklist additions was not as expected"); - Assertions.assertTrue( - - blacklistRemovals.isEmpty(), "Blacklist removals should be empty but was " + - blacklistRemovals); + assertArrayEquals(expectedBlacklistAdditions, + blacklistAdditions.toArray(), "Blacklist additions was not as expected"); + assertTrue(blacklistRemovals.isEmpty(), + "Blacklist removals should be empty but was " + blacklistRemovals); } @Test @@ -69,14 +66,10 @@ public void testSimpleBlacklistAboveFailureThreshold() { Collections.sort(blacklistAdditions); List blacklistRemovals = blacklist.getBlacklistRemovals(); String[] expectedBlacklistAdditions = new String[]{anyNode}; - Assertions.assertArrayEquals( - - expectedBlacklistAdditions -, blacklistAdditions.toArray(), "Blacklist additions was not as expected"); - Assertions.assertTrue( - - blacklistRemovals.isEmpty(), "Blacklist removals should be empty but was " + - blacklistRemovals); + assertArrayEquals(expectedBlacklistAdditions, + blacklistAdditions.toArray(), "Blacklist additions was not as expected"); + assertTrue(blacklistRemovals.isEmpty(), + "Blacklist removals should be empty but was " + blacklistRemovals); manager.addNode(anyNode2); @@ -87,14 +80,10 @@ public void testSimpleBlacklistAboveFailureThreshold() { blacklistRemovals = blacklist.getBlacklistRemovals(); Collections.sort(blacklistRemovals); String[] expectedBlacklistRemovals = new String[] {anyNode2, anyNode}; - Assertions.assertTrue( - - blacklistAdditions.isEmpty(), "Blacklist additions should be empty but was " + - blacklistAdditions); - Assertions.assertArrayEquals( - - expectedBlacklistRemovals -, blacklistRemovals.toArray(), "Blacklist removals was not as expected"); + assertTrue(blacklistAdditions.isEmpty(), + "Blacklist additions should be empty but was " + blacklistAdditions); + assertArrayEquals(expectedBlacklistRemovals, + blacklistRemovals.toArray(), "Blacklist removals was not as expected"); } @Test @@ -107,13 +96,9 @@ public void testDisabledBlacklist() { List blacklistAdditions = blacklist.getBlacklistAdditions(); List blacklistRemovals = blacklist.getBlacklistRemovals(); - Assertions.assertTrue( - - blacklistAdditions.isEmpty(), "Blacklist additions should be empty but was " + - blacklistAdditions); - Assertions.assertTrue( - - blacklistRemovals.isEmpty(), "Blacklist removals should be empty but was " + - blacklistRemovals); + assertTrue(blacklistAdditions.isEmpty(), + "Blacklist additions should be empty but was " + blacklistAdditions); + assertTrue(blacklistRemovals.isEmpty(), + "Blacklist removals should be empty but was " + blacklistRemovals); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/federation/TestFederationRMStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/federation/TestFederationRMStateStoreService.java index e08a4206963fb..92c337b0773cd 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/federation/TestFederationRMStateStoreService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/federation/TestFederationRMStateStoreService.java @@ -17,6 +17,11 @@ package org.apache.hadoop.yarn.server.resourcemanager.federation; +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 java.io.IOException; import java.io.StringReader; import java.net.UnknownHostException; @@ -75,7 +80,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterMetricsInfo; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -136,12 +140,12 @@ public void testFederationStateStoreService() throws Exception { rm.init(conf); stateStore = rm.getFederationStateStoreService().getStateStoreClient(); GetSubClusterInfoResponse response = stateStore.getSubCluster(request); - Assertions.assertNull(response); + assertNull(response); // Validate if sub-cluster is registered rm.start(); String capability = checkSubClusterInfo(SubClusterState.SC_NEW); - Assertions.assertTrue(capability.isEmpty()); + assertTrue(capability.isEmpty()); // Heartbeat to see if sub-cluster transitions to running FederationStateStoreHeartbeat storeHeartbeat = @@ -166,7 +170,7 @@ public void testFederationStateStoreService() throws Exception { explicitFailover(rm); capability = checkSubClusterInfo(SubClusterState.SC_NEW); - Assertions.assertTrue(capability.isEmpty()); + assertTrue(capability.isEmpty()); // Heartbeat to see if sub-cluster transitions to running storeHeartbeat = @@ -186,10 +190,10 @@ public void testFederationStateStoreService() throws Exception { private void explicitFailover(MockRM rm) throws IOException { rm.getAdminService().transitionToStandby(requestInfo); - Assertions.assertTrue(rm.getRMContext() + assertTrue(rm.getRMContext() .getHAServiceState() == HAServiceProtocol.HAServiceState.STANDBY); rm.getAdminService().transitionToActive(requestInfo); - Assertions.assertTrue(rm.getRMContext() + assertTrue(rm.getRMContext() .getHAServiceState() == HAServiceProtocol.HAServiceState.ACTIVE); lastHearbeatTS = 0; stateStore = rm.getFederationStateStoreService().getStateStoreClient(); @@ -199,23 +203,23 @@ private void checkClusterMetricsInfo(String capability, int numNodes) throws JAXBException { ClusterMetricsInfo clusterMetricsInfo = jsonUnmarshaller.unmarshalFromJSON( new StringReader(capability), ClusterMetricsInfo.class); - Assertions.assertEquals(numNodes, clusterMetricsInfo.getTotalNodes()); + assertEquals(numNodes, clusterMetricsInfo.getTotalNodes()); } private String checkSubClusterInfo(SubClusterState state) throws YarnException, UnknownHostException { - Assertions.assertNotNull(stateStore.getSubCluster(request)); + assertNotNull(stateStore.getSubCluster(request)); SubClusterInfo response = stateStore.getSubCluster(request).getSubClusterInfo(); - Assertions.assertEquals(state, response.getState()); - Assertions.assertTrue(response.getLastHeartBeat() >= lastHearbeatTS); + assertEquals(state, response.getState()); + assertTrue(response.getLastHeartBeat() >= lastHearbeatTS); String expectedAddress = (response.getClientRMServiceAddress().split(":"))[0]; - Assertions.assertEquals(expectedAddress, + assertEquals(expectedAddress, (response.getAMRMServiceAddress().split(":"))[0]); - Assertions.assertEquals(expectedAddress, + assertEquals(expectedAddress, (response.getRMAdminServiceAddress().split(":"))[0]); - Assertions.assertEquals(expectedAddress, + assertEquals(expectedAddress, (response.getRMWebServiceAddress().split(":"))[0]); lastHearbeatTS = response.getLastHeartBeat(); return response.getCapability(); @@ -236,12 +240,12 @@ public void testFederationStateStoreServiceInitialHeartbeatDelay() throws Except rm.init(conf); stateStore = rm.getFederationStateStoreService().getStateStoreClient(); GetSubClusterInfoResponse response = stateStore.getSubCluster(request); - Assertions.assertNull(response); + assertNull(response); // Validate if sub-cluster is registered rm.start(); String capability = checkSubClusterInfo(SubClusterState.SC_NEW); - Assertions.assertTrue(capability.isEmpty()); + assertTrue(capability.isEmpty()); // Heartbeat to see if sub-cluster transitions to running FederationStateStoreHeartbeat storeHeartbeat = @@ -250,7 +254,7 @@ public void testFederationStateStoreServiceInitialHeartbeatDelay() throws Except capability = checkSubClusterInfo(SubClusterState.SC_RUNNING); checkClusterMetricsInfo(capability, 0); - Assertions.assertTrue(logCapture.getOutput().contains( + assertTrue(logCapture.getOutput().contains( "Started federation membership heartbeat with interval: 300 and initial delay: 10")); rm.stop(); } @@ -285,16 +289,16 @@ public void testCleanUpApplication() throws Exception { GetApplicationHomeSubClusterRequest.newInstance(appId); GetApplicationHomeSubClusterResponse response = stateStore.getApplicationHomeSubCluster(appRequest); - Assertions.assertNotNull(response); + assertNotNull(response); ApplicationHomeSubCluster appHomeSubCluster = response.getApplicationHomeSubCluster(); - Assertions.assertNotNull(appHomeSubCluster); - Assertions.assertNotNull(appHomeSubCluster.getApplicationId()); - Assertions.assertEquals(appId, appHomeSubCluster.getApplicationId()); + assertNotNull(appHomeSubCluster); + assertNotNull(appHomeSubCluster.getApplicationId()); + assertEquals(appId, appHomeSubCluster.getApplicationId()); // clean up the app. boolean cleanUpResult = stateStoreService.cleanUpFinishApplicationsWithRetries(appId, true); - Assertions.assertTrue(cleanUpResult); + assertTrue(cleanUpResult); // after clean, the app can no longer be queried from the stateStore. LambdaTestUtils.intercept(FederationStateStoreException.class, @@ -344,10 +348,10 @@ public void testCleanUpApplicationWhenRMStart() throws Exception { GetApplicationsHomeSubClusterRequest.newInstance(subClusterId); GetApplicationsHomeSubClusterResponse allResponse = stateStore.getApplicationsHomeSubCluster(allRequest); - Assertions.assertNotNull(allResponse); + assertNotNull(allResponse); List appHomeSCLists = allResponse.getAppsHomeSubClusters(); - Assertions.assertNotNull(appHomeSCLists); - Assertions.assertEquals(3, appHomeSCLists.size()); + assertNotNull(appHomeSCLists); + assertEquals(3, appHomeSCLists.size()); // app04 exists in both RM memory and stateStore. ApplicationId appId04 = ApplicationId.newInstance(Time.now(), 4); @@ -363,7 +367,7 @@ public void testCleanUpApplicationWhenRMStart() throws Exception { try { List subClusters = getApplicationsFromStateStore(); - Assertions.assertNotNull(subClusters); + assertNotNull(subClusters); appsSize = subClusters.size(); } catch (YarnException e) { e.printStackTrace(); @@ -446,11 +450,11 @@ public void testCleanUpApplicationWhenRMCompleteOneApp() throws Exception { // app03 should remain in statestore List appHomeScList = getApplicationsFromStateStore(); - Assertions.assertNotNull(appHomeScList); - Assertions.assertEquals(1, appHomeScList.size()); + assertNotNull(appHomeScList); + assertEquals(1, appHomeScList.size()); ApplicationHomeSubCluster homeSubCluster = appHomeScList.get(0); - Assertions.assertNotNull(homeSubCluster); - Assertions.assertEquals(appId03, homeSubCluster.getApplicationId()); + assertNotNull(homeSubCluster); + assertEquals(appId03, homeSubCluster.getApplicationId()); } private void addApplication2StateStore(ApplicationId appId, @@ -468,9 +472,9 @@ private List getApplicationsFromStateStore() throws Y GetApplicationsHomeSubClusterRequest.newInstance(subClusterId); GetApplicationsHomeSubClusterResponse allResponse = stateStore.getApplicationsHomeSubCluster(allRequest); - Assertions.assertNotNull(allResponse); + assertNotNull(allResponse); List appHomeSCLists = allResponse.getAppsHomeSubClusters(); - Assertions.assertNotNull(appHomeSCLists); + assertNotNull(appHomeSCLists); return appHomeSCLists; } @@ -533,25 +537,25 @@ public void testPolicyConfigurationMethod() throws YarnException { GetSubClusterPolicyConfigurationRequest.newInstance(queue1); GetSubClusterPolicyConfigurationResponse response = stateStoreService.getPolicyConfiguration(request1); - Assertions.assertNotNull(response); + assertNotNull(response); SubClusterPolicyConfiguration responsePolicyConf = response.getPolicyConfiguration(); - Assertions.assertNotNull(responsePolicyConf); - Assertions.assertEquals(requestPolicyConf1, responsePolicyConf); + assertNotNull(responsePolicyConf); + assertEquals(requestPolicyConf1, responsePolicyConf); // get policy configurations GetSubClusterPoliciesConfigurationsRequest policiesRequest1 = GetSubClusterPoliciesConfigurationsRequest.newInstance(); GetSubClusterPoliciesConfigurationsResponse policiesResponse1 = stateStoreService.getPoliciesConfigurations(policiesRequest1); - Assertions.assertNotNull(policiesResponse1); + assertNotNull(policiesResponse1); List policiesConfigs = policiesResponse1.getPoliciesConfigs(); - Assertions.assertNotNull(policiesConfigs); - Assertions.assertEquals(2, policiesConfigs.size()); - Assertions.assertTrue(policiesConfigs.contains(requestPolicyConf1)); - Assertions.assertTrue(policiesConfigs.contains(requestPolicyConf2)); + assertNotNull(policiesConfigs); + assertEquals(2, policiesConfigs.size()); + assertTrue(policiesConfigs.contains(requestPolicyConf1)); + assertTrue(policiesConfigs.contains(requestPolicyConf2)); } public SubClusterPolicyConfiguration getUniformPolicy(String queue) @@ -596,12 +600,12 @@ public void testSubClusterMethod() throws YarnException { GetSubClusterInfoRequest.newInstance(subClusterId1); GetSubClusterInfoResponse subClusterResponse = stateStoreService.getSubCluster(subClusterRequest); - Assertions.assertNotNull(subClusterResponse); + assertNotNull(subClusterResponse); // We query subCluster1, we want to get SubClusterInfo of subCluster1 SubClusterInfo subClusterInfo1Resp = subClusterResponse.getSubClusterInfo(); - Assertions.assertNotNull(subClusterInfo1Resp); - Assertions.assertEquals(subClusterInfo1, subClusterInfo1Resp); + assertNotNull(subClusterInfo1Resp); + assertEquals(subClusterInfo1, subClusterInfo1Resp); // We call the getSubClusters method and filter the Active SubCluster // subCluster1 and subCluster2 are just registered, they are in NEW state, @@ -610,10 +614,10 @@ public void testSubClusterMethod() throws YarnException { GetSubClustersInfoRequest.newInstance(true); GetSubClustersInfoResponse subClustersInfoResp = stateStoreService.getSubClusters(subClustersInfoRequest); - Assertions.assertNotNull(subClustersInfoResp); + assertNotNull(subClustersInfoResp); List subClusterInfos = subClustersInfoResp.getSubClusters(); - Assertions.assertNotNull(subClusterInfos); - Assertions.assertEquals(0, subClusterInfos.size()); + assertNotNull(subClusterInfos); + assertEquals(0, subClusterInfos.size()); // We let subCluster1 heartbeat and set subCluster1 to Running state SubClusterHeartbeatRequest heartbeatRequest = @@ -621,7 +625,7 @@ public void testSubClusterMethod() throws YarnException { "capability"); SubClusterHeartbeatResponse heartbeatResponse = stateStoreService.subClusterHeartbeat(heartbeatRequest); - Assertions.assertNotNull(heartbeatResponse); + assertNotNull(heartbeatResponse); // We call the getSubClusters method again and filter the Active SubCluster // We want to get 1 active SubCluster @@ -629,10 +633,10 @@ public void testSubClusterMethod() throws YarnException { GetSubClustersInfoRequest.newInstance(true); GetSubClustersInfoResponse subClustersInfoResp1 = stateStoreService.getSubClusters(subClustersInfoRequest1); - Assertions.assertNotNull(subClustersInfoResp1); + assertNotNull(subClustersInfoResp1); List subClusterInfos1 = subClustersInfoResp1.getSubClusters(); - Assertions.assertNotNull(subClusterInfos1); - Assertions.assertEquals(1, subClusterInfos1.size()); + assertNotNull(subClusterInfos1); + assertEquals(1, subClusterInfos1.size()); } private SubClusterInfo createSubClusterInfo(SubClusterId clusterId) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java index beddbee6bdd62..fca8211fda36e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java @@ -19,6 +19,9 @@ package org.apache.hadoop.yarn.server.resourcemanager.logaggregationstatus; 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.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; @@ -61,7 +64,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType; import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -156,13 +158,13 @@ public void testLogAggregationStatus() throws Exception { // should be NOT_STARTED Map logAggregationStatus = rmApp.getLogAggregationReportsForApp(); - Assertions.assertEquals(2, logAggregationStatus.size()); - Assertions.assertTrue(logAggregationStatus.containsKey(nodeId1)); - Assertions.assertTrue(logAggregationStatus.containsKey(nodeId2)); + assertEquals(2, logAggregationStatus.size()); + assertTrue(logAggregationStatus.containsKey(nodeId1)); + assertTrue(logAggregationStatus.containsKey(nodeId2)); for (Entry report : logAggregationStatus .entrySet()) { - Assertions.assertEquals(LogAggregationStatus.NOT_START, report.getValue() - .getLogAggregationStatus()); + assertEquals(LogAggregationStatus.NOT_START, report.getValue() + .getLogAggregationStatus()); } List node1ReportForApp = @@ -196,25 +198,24 @@ public void testLogAggregationStatus() throws Exception { // verify that the log aggregation status for node1, node2 // has been changed logAggregationStatus = rmApp.getLogAggregationReportsForApp(); - Assertions.assertEquals(2, logAggregationStatus.size()); - Assertions.assertTrue(logAggregationStatus.containsKey(nodeId1)); - Assertions.assertTrue(logAggregationStatus.containsKey(nodeId2)); + assertEquals(2, logAggregationStatus.size()); + assertTrue(logAggregationStatus.containsKey(nodeId1)); + assertTrue(logAggregationStatus.containsKey(nodeId2)); for (Entry report : logAggregationStatus .entrySet()) { if (report.getKey().equals(node1.getNodeID())) { - Assertions.assertEquals(LogAggregationStatus.RUNNING, report.getValue() - .getLogAggregationStatus()); - Assertions.assertEquals(messageForNode1_1, report.getValue() - .getDiagnosticMessage()); + assertEquals(LogAggregationStatus.RUNNING, report.getValue() + .getLogAggregationStatus()); + assertEquals(messageForNode1_1, report.getValue() + .getDiagnosticMessage()); } else if (report.getKey().equals(node2.getNodeID())) { - Assertions.assertEquals(LogAggregationStatus.RUNNING, report.getValue() - .getLogAggregationStatus()); - Assertions.assertEquals(messageForNode2_1, report.getValue() - .getDiagnosticMessage()); + assertEquals(LogAggregationStatus.RUNNING, report.getValue() + .getLogAggregationStatus()); + assertEquals(messageForNode2_1, report.getValue() + .getDiagnosticMessage()); } else { // should not contain log aggregation report for other nodes - Assertions - .fail("should not contain log aggregation report for other nodes"); + fail("should not contain log aggregation report for other nodes"); } } @@ -235,26 +236,25 @@ public void testLogAggregationStatus() throws Exception { // verify that the log aggregation status for node2 // does not change logAggregationStatus = rmApp.getLogAggregationReportsForApp(); - Assertions.assertEquals(2, logAggregationStatus.size()); - Assertions.assertTrue(logAggregationStatus.containsKey(nodeId1)); - Assertions.assertTrue(logAggregationStatus.containsKey(nodeId2)); + assertEquals(2, logAggregationStatus.size()); + assertTrue(logAggregationStatus.containsKey(nodeId1)); + assertTrue(logAggregationStatus.containsKey(nodeId2)); for (Entry report : logAggregationStatus .entrySet()) { if (report.getKey().equals(node1.getNodeID())) { - Assertions.assertEquals(LogAggregationStatus.RUNNING, report.getValue() + assertEquals(LogAggregationStatus.RUNNING, report.getValue() .getLogAggregationStatus()); - Assertions.assertEquals( + assertEquals( messageForNode1_1 + "\n" + messageForNode1_2, report .getValue().getDiagnosticMessage()); } else if (report.getKey().equals(node2.getNodeID())) { - Assertions.assertEquals(LogAggregationStatus.RUNNING, report.getValue() + assertEquals(LogAggregationStatus.RUNNING, report.getValue() .getLogAggregationStatus()); - Assertions.assertEquals(messageForNode2_1, report.getValue() + assertEquals(messageForNode2_1, report.getValue() .getDiagnosticMessage()); } else { // should not contain log aggregation report for other nodes - Assertions - .fail("should not contain log aggregation report for other nodes"); + fail("should not contain log aggregation report for other nodes"); } } @@ -262,7 +262,7 @@ public void testLogAggregationStatus() throws Exception { rmApp.handle(new RMAppEvent(appId, RMAppEventType.KILL)); rmApp.handle(new RMAppEvent(appId, RMAppEventType.ATTEMPT_KILLED)); rmApp.handle(new RMAppEvent(appId, RMAppEventType.APP_UPDATE_SAVED)); - Assertions.assertEquals(RMAppState.KILLED, rmApp.getState()); + assertEquals(RMAppState.KILLED, rmApp.getState()); // wait for 1500 ms Thread.sleep(1500); @@ -270,12 +270,12 @@ public void testLogAggregationStatus() throws Exception { // the log aggregation status for both nodes should be changed // to TIME_OUT logAggregationStatus = rmApp.getLogAggregationReportsForApp(); - Assertions.assertEquals(2, logAggregationStatus.size()); - Assertions.assertTrue(logAggregationStatus.containsKey(nodeId1)); - Assertions.assertTrue(logAggregationStatus.containsKey(nodeId2)); + assertEquals(2, logAggregationStatus.size()); + assertTrue(logAggregationStatus.containsKey(nodeId1)); + assertTrue(logAggregationStatus.containsKey(nodeId2)); for (Entry report : logAggregationStatus .entrySet()) { - Assertions.assertEquals(LogAggregationStatus.TIME_OUT, report.getValue() + assertEquals(LogAggregationStatus.TIME_OUT, report.getValue() .getLogAggregationStatus()); } @@ -299,13 +299,13 @@ public void testLogAggregationStatus() throws Exception { node1ReportForApp3)); logAggregationStatus = rmApp.getLogAggregationReportsForApp(); - Assertions.assertEquals(2, logAggregationStatus.size()); - Assertions.assertTrue(logAggregationStatus.containsKey(nodeId1)); - Assertions.assertTrue(logAggregationStatus.containsKey(nodeId2)); + assertEquals(2, logAggregationStatus.size()); + assertTrue(logAggregationStatus.containsKey(nodeId1)); + assertTrue(logAggregationStatus.containsKey(nodeId2)); for (Entry report : logAggregationStatus .entrySet()) { if (report.getKey().equals(node1.getNodeID())) { - Assertions.assertEquals(LogAggregationStatus.SUCCEEDED, report.getValue() + assertEquals(LogAggregationStatus.SUCCEEDED, report.getValue() .getLogAggregationStatus()); StringBuilder builder = new StringBuilder(); for (int i = 0; i < 9; i ++) { @@ -313,15 +313,14 @@ public void testLogAggregationStatus() throws Exception { builder.append("\n"); } builder.append("test_message_" + 9); - Assertions.assertEquals(builder.toString(), report.getValue() + assertEquals(builder.toString(), report.getValue() .getDiagnosticMessage()); } else if (report.getKey().equals(node2.getNodeID())) { - Assertions.assertEquals(LogAggregationStatus.TIME_OUT, report.getValue() + assertEquals(LogAggregationStatus.TIME_OUT, report.getValue() .getLogAggregationStatus()); } else { // should not contain log aggregation report for other nodes - Assertions - .fail("should not contain log aggregation report for other nodes"); + fail("should not contain log aggregation report for other nodes"); } } @@ -341,13 +340,13 @@ public void testLogAggregationStatus() throws Exception { node2ReportForApp2.add(report2_3); node2.handle(new RMNodeStatusEvent(node2.getNodeID(), nodeStatus2, node2ReportForApp2)); - Assertions.assertEquals(LogAggregationStatus.FAILED, + assertEquals(LogAggregationStatus.FAILED, rmApp.getLogAggregationStatusForAppReport()); logAggregationStatus = rmApp.getLogAggregationReportsForApp(); - Assertions.assertTrue(logAggregationStatus.size() == 1); - Assertions.assertTrue(logAggregationStatus.containsKey(node2.getNodeID())); - Assertions.assertTrue(!logAggregationStatus.containsKey(node1.getNodeID())); - Assertions.assertEquals("Fail_Message", + assertTrue(logAggregationStatus.size() == 1); + assertTrue(logAggregationStatus.containsKey(node2.getNodeID())); + assertTrue(!logAggregationStatus.containsKey(node1.getNodeID())); + assertEquals("Fail_Message", ((RMAppImpl)rmApp).getLogAggregationFailureMessagesForNM(nodeId2)); } @@ -360,7 +359,7 @@ public void testGetLogAggregationStatusForAppReport() { conf.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, false); RMAppImpl rmApp = (RMAppImpl)createRMApp(conf); // The log aggregation status should be DISABLED. - Assertions.assertEquals(LogAggregationStatus.DISABLED, + assertEquals(LogAggregationStatus.DISABLED, rmApp.getLogAggregationStatusForAppReport()); // Enable the log aggregation @@ -369,7 +368,7 @@ public void testGetLogAggregationStatusForAppReport() { // If we do not know any NodeManagers for this application , and // the log aggregation is enabled, the log aggregation status will // return NOT_START - Assertions.assertEquals(LogAggregationStatus.NOT_START, + assertEquals(LogAggregationStatus.NOT_START, rmApp.getLogAggregationStatusForAppReport()); NodeId nodeId1 = NodeId.newInstance("localhost", 1111); @@ -387,7 +386,7 @@ public void testGetLogAggregationStatusForAppReport() { rmApp.getApplicationId(), LogAggregationStatus.NOT_START, "")); rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance( rmApp.getApplicationId(), LogAggregationStatus.NOT_START, "")); - Assertions.assertEquals(LogAggregationStatus.NOT_START, + assertEquals(LogAggregationStatus.NOT_START, rmApp.getLogAggregationStatusForAppReport()); rmApp.aggregateLogReport(nodeId1, LogAggregationReport.newInstance( @@ -398,11 +397,11 @@ public void testGetLogAggregationStatusForAppReport() { rmApp.getApplicationId(), LogAggregationStatus.SUCCEEDED, "")); rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance( rmApp.getApplicationId(), LogAggregationStatus.SUCCEEDED, "")); - Assertions.assertEquals(LogAggregationStatus.RUNNING, + assertEquals(LogAggregationStatus.RUNNING, rmApp.getLogAggregationStatusForAppReport()); rmApp.handle(new RMAppEvent(rmApp.getApplicationId(), RMAppEventType.KILL)); - Assertions.assertTrue(RMAppImpl.isAppInFinalState(rmApp)); + assertTrue(RMAppImpl.isAppInFinalState(rmApp)); // If at least of one log aggregation status for one NM is TIME_OUT, // others are SUCCEEDED, the log aggregation status for this app will @@ -415,7 +414,7 @@ public void testGetLogAggregationStatusForAppReport() { rmApp.getApplicationId(), LogAggregationStatus.SUCCEEDED, "")); rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance( rmApp.getApplicationId(), LogAggregationStatus.SUCCEEDED, "")); - Assertions.assertEquals(LogAggregationStatus.TIME_OUT, + assertEquals(LogAggregationStatus.TIME_OUT, rmApp.getLogAggregationStatusForAppReport()); rmApp = (RMAppImpl)createRMApp(conf); @@ -431,7 +430,7 @@ public void testGetLogAggregationStatusForAppReport() { rmApp.getApplicationId(), LogAggregationStatus.SUCCEEDED, "")); rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance( rmApp.getApplicationId(), LogAggregationStatus.SUCCEEDED, "")); - Assertions.assertEquals(LogAggregationStatus.SUCCEEDED, + assertEquals(LogAggregationStatus.SUCCEEDED, rmApp.getLogAggregationStatusForAppReport()); rmApp = (RMAppImpl)createRMApp(conf); @@ -445,7 +444,7 @@ public void testGetLogAggregationStatusForAppReport() { rmApp.getApplicationId(), LogAggregationStatus.NOT_START, "")); rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance( rmApp.getApplicationId(), LogAggregationStatus.NOT_START, "")); - Assertions.assertEquals(LogAggregationStatus.RUNNING, + assertEquals(LogAggregationStatus.RUNNING, rmApp.getLogAggregationStatusForAppReport()); // If the log aggregation status for at least one of NMs @@ -460,7 +459,7 @@ public void testGetLogAggregationStatusForAppReport() { rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance( rmApp.getApplicationId(), LogAggregationStatus.RUNNING_WITH_FAILURE, "")); - Assertions.assertEquals(LogAggregationStatus.RUNNING_WITH_FAILURE, + assertEquals(LogAggregationStatus.RUNNING_WITH_FAILURE, rmApp.getLogAggregationStatusForAppReport()); // For node4, the previous log aggregation status is RUNNING_WITH_FAILURE, @@ -474,11 +473,11 @@ public void testGetLogAggregationStatusForAppReport() { rmApp.getApplicationId(), LogAggregationStatus.NOT_START, "")); rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance( rmApp.getApplicationId(), LogAggregationStatus.RUNNING, "")); - Assertions.assertEquals(LogAggregationStatus.RUNNING_WITH_FAILURE, + assertEquals(LogAggregationStatus.RUNNING_WITH_FAILURE, rmApp.getLogAggregationStatusForAppReport()); rmApp.handle(new RMAppEvent(rmApp.getApplicationId(), RMAppEventType.KILL)); - Assertions.assertTrue(RMAppImpl.isAppInFinalState(rmApp)); + assertTrue(RMAppImpl.isAppInFinalState(rmApp)); // If at least of one log aggregation status for one NM is FAILED, // others are either SUCCEEDED or TIME_OUT, and this application is // at the final state, the log aggregation status for this app @@ -491,7 +490,7 @@ public void testGetLogAggregationStatusForAppReport() { rmApp.getApplicationId(), LogAggregationStatus.FAILED, "")); rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance( rmApp.getApplicationId(), LogAggregationStatus.FAILED, "")); - Assertions.assertEquals(LogAggregationStatus.FAILED, + assertEquals(LogAggregationStatus.FAILED, rmApp.getLogAggregationStatusForAppReport()); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestCombinedSystemMetricsPublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestCombinedSystemMetricsPublisher.java index c657ca845af72..092310270a9b3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestCombinedSystemMetricsPublisher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestCombinedSystemMetricsPublisher.java @@ -19,7 +19,10 @@ package org.apache.hadoop.yarn.server.resourcemanager.metrics; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -62,7 +65,6 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter; import org.apache.hadoop.yarn.util.TimelineServiceHelper; import org.apache.hadoop.yarn.util.timeline.TimelineUtils; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -133,18 +135,18 @@ private void testSetup(boolean enableV1, boolean enableV2) throws Exception { new ArrayList(); if (YarnConfiguration.timelineServiceV1Enabled(conf)) { - Assertions.assertTrue(enableV1); + assertTrue(enableV1); publisherV1 = new TimelineServiceV1Publisher(); publishers.add(publisherV1); publisherV1.init(conf); publisherV1.start(); } else { - Assertions.assertFalse(enableV1); + assertFalse(enableV1); publisherV1 = null; } if (YarnConfiguration.timelineServiceV2Enabled(conf)) { - Assertions.assertTrue(enableV2); + assertTrue(enableV2); publisherV2 = new TimelineServiceV2Publisher( rmTimelineCollectorManager) { @Override @@ -156,7 +158,7 @@ protected Dispatcher getDispatcher() { publisherV2.init(conf); publisherV2.start(); } else { - Assertions.assertFalse(enableV2); + assertFalse(enableV2); publisherV2 = null; } @@ -221,7 +223,7 @@ private static YarnConfiguration getConf(boolean v1Enabled, testRootDir.getCanonicalPath()); } catch (IOException e) { e.printStackTrace(); - Assertions.fail("Exception while setting the " + + fail("Exception while setting the " + "TIMELINE_SERVICE_STORAGE_DIR_ROOT "); } } @@ -281,21 +283,21 @@ public void testTimelineServiceConfiguration() config.set(YarnConfiguration.TIMELINE_SERVICE_VERSIONS, "2.0,1.5"); config.set(YarnConfiguration.TIMELINE_SERVICE_VERSION, "2.0"); - Assertions.assertTrue(YarnConfiguration.timelineServiceV2Enabled(config)); - Assertions.assertTrue(YarnConfiguration.timelineServiceV15Enabled(config)); - Assertions.assertTrue(YarnConfiguration.timelineServiceV1Enabled(config)); + assertTrue(YarnConfiguration.timelineServiceV2Enabled(config)); + assertTrue(YarnConfiguration.timelineServiceV15Enabled(config)); + assertTrue(YarnConfiguration.timelineServiceV1Enabled(config)); config.set(YarnConfiguration.TIMELINE_SERVICE_VERSIONS, "2.0,1"); config.set(YarnConfiguration.TIMELINE_SERVICE_VERSION, "1.5"); - Assertions.assertTrue(YarnConfiguration.timelineServiceV2Enabled(config)); - Assertions.assertFalse(YarnConfiguration.timelineServiceV15Enabled(config)); - Assertions.assertTrue(YarnConfiguration.timelineServiceV1Enabled(config)); + assertTrue(YarnConfiguration.timelineServiceV2Enabled(config)); + assertFalse(YarnConfiguration.timelineServiceV15Enabled(config)); + assertTrue(YarnConfiguration.timelineServiceV1Enabled(config)); config.set(YarnConfiguration.TIMELINE_SERVICE_VERSIONS, "2.0"); config.set(YarnConfiguration.TIMELINE_SERVICE_VERSION, "1.5"); - Assertions.assertTrue(YarnConfiguration.timelineServiceV2Enabled(config)); - Assertions.assertFalse(YarnConfiguration.timelineServiceV15Enabled(config)); - Assertions.assertFalse(YarnConfiguration.timelineServiceV1Enabled(config)); + assertTrue(YarnConfiguration.timelineServiceV2Enabled(config)); + assertFalse(YarnConfiguration.timelineServiceV15Enabled(config)); + assertFalse(YarnConfiguration.timelineServiceV1Enabled(config)); } private void publishEvents(boolean v1Enabled, boolean v2Enabled) { @@ -332,7 +334,7 @@ private void validateV1(boolean v1Enabled) throws Exception { store.getEntity(appAttemptId.toString(), AppAttemptMetricsConstants.ENTITY_TYPE, EnumSet.allOf(Field.class)); - Assertions.assertNull(entity); + assertNull(entity); return; } @@ -355,19 +357,18 @@ private void validateV1(boolean v1Enabled) throws Exception { } else if (event.getEventType().equals( AppAttemptMetricsConstants.FINISHED_EVENT_TYPE)) { hasFinishedEvent = true; - Assertions.assertEquals( + assertEquals( FinalApplicationStatus.UNDEFINED.toString(), event.getEventInfo().get( AppAttemptMetricsConstants.FINAL_STATUS_INFO)); - Assertions.assertEquals( + assertEquals( YarnApplicationAttemptState.FINISHED.toString(), event.getEventInfo().get( AppAttemptMetricsConstants.STATE_INFO)); } - Assertions - .assertEquals(appAttemptId.toString(), entity.getEntityId()); + assertEquals(appAttemptId.toString(), entity.getEntityId()); } - Assertions.assertTrue(hasRegisteredEvent && hasFinishedEvent); + assertTrue(hasRegisteredEvent && hasFinishedEvent); } private void validateV2(boolean v2Enabled) throws Exception { @@ -376,13 +377,13 @@ private void validateV2(boolean v2Enabled) throws Exception { + TimelineEntityType.YARN_APPLICATION_ATTEMPT + "/"; File entityFolder = new File(outputDirApp); - Assertions.assertEquals(v2Enabled, entityFolder.isDirectory()); + assertEquals(v2Enabled, entityFolder.isDirectory()); if (v2Enabled) { String timelineServiceFileName = appAttemptId.toString() + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION; File entityFile = new File(outputDirApp, timelineServiceFileName); - Assertions.assertTrue(entityFile.exists()); + assertTrue(entityFile.exists()); long idPrefix = TimelineServiceHelper .invertLong(appAttemptId.getAttemptId()); verifyEntity(entityFile, 2, diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java index 135332eec1041..81be02eec7c52 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java @@ -18,6 +18,11 @@ package org.apache.hadoop.yarn.server.resourcemanager.metrics; +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.assertTrue; + import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -28,7 +33,6 @@ import java.util.Set; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -72,7 +76,6 @@ import org.apache.hadoop.yarn.server.timeline.recovery.MemoryTimelineStateStore; import org.apache.hadoop.yarn.server.timeline.recovery.TimelineStateStore; -import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -180,92 +183,90 @@ public void testPublishApplicationMetrics() throws Exception { // ensure Five events are both published before leaving the loop } while (entity == null || entity.getEvents().size() < 6); // verify all the fields - Assertions.assertEquals(ApplicationMetricsConstants.ENTITY_TYPE, + assertEquals(ApplicationMetricsConstants.ENTITY_TYPE, entity.getEntityType()); - Assertions - .assertEquals(app.getApplicationId().toString(), entity.getEntityId()); - Assertions - .assertEquals( + assertEquals(app.getApplicationId().toString(), entity.getEntityId()); + assertEquals( app.getName(), entity.getOtherInfo().get( ApplicationMetricsConstants.NAME_ENTITY_INFO)); if (i != 1) { - Assertions.assertEquals( + assertEquals( app.getQueue(), entity.getOtherInfo().get( ApplicationMetricsConstants.QUEUE_ENTITY_INFO)); } - Assertions.assertEquals( + assertEquals( app.getApplicationSubmissionContext().getUnmanagedAM(), entity.getOtherInfo().get( ApplicationMetricsConstants.UNMANAGED_APPLICATION_ENTITY_INFO)); if (i != 1) { - Assertions.assertEquals( + assertEquals( app.getApplicationSubmissionContext().getPriority().getPriority(), entity.getOtherInfo().get( ApplicationMetricsConstants.APPLICATION_PRIORITY_INFO)); } - Assertions.assertEquals(app.getAmNodeLabelExpression(), entity.getOtherInfo() + assertEquals(app.getAmNodeLabelExpression(), entity.getOtherInfo() .get(ApplicationMetricsConstants.AM_NODE_LABEL_EXPRESSION)); - Assertions.assertEquals( + assertEquals( app.getApplicationSubmissionContext().getNodeLabelExpression(), entity.getOtherInfo() .get(ApplicationMetricsConstants.APP_NODE_LABEL_EXPRESSION)); - Assertions - .assertEquals( + + assertEquals( app.getUser(), entity.getOtherInfo().get( ApplicationMetricsConstants.USER_ENTITY_INFO)); - Assertions - .assertEquals( + + assertEquals( app.getApplicationType(), entity.getOtherInfo().get( ApplicationMetricsConstants.TYPE_ENTITY_INFO)); - Assertions.assertEquals(app.getSubmitTime(), + assertEquals(app.getSubmitTime(), entity.getOtherInfo().get( ApplicationMetricsConstants.SUBMITTED_TIME_ENTITY_INFO)); - Assertions.assertTrue(verifyAppTags(app.getApplicationTags(), + assertTrue(verifyAppTags(app.getApplicationTags(), entity.getOtherInfo())); if (i == 1) { - Assertions.assertEquals("uers1,user2", + assertEquals("uers1,user2", entity.getOtherInfo().get( ApplicationMetricsConstants.APP_VIEW_ACLS_ENTITY_INFO)); - Assertions.assertEquals( + assertEquals( app.getApplicationSubmissionContext().getAMContainerSpec() .getCommands(), entity.getOtherInfo() .get(ApplicationMetricsConstants.AM_CONTAINER_LAUNCH_COMMAND)); } else { - Assertions.assertEquals( + assertEquals( "", entity.getOtherInfo().get( ApplicationMetricsConstants.APP_VIEW_ACLS_ENTITY_INFO)); - Assertions.assertEquals( + assertEquals( app.getRMAppMetrics().getMemorySeconds(), Long.parseLong(entity.getOtherInfo() .get(ApplicationMetricsConstants.APP_MEM_METRICS).toString())); - Assertions.assertEquals( + assertEquals( app.getRMAppMetrics().getVcoreSeconds(), Long.parseLong(entity.getOtherInfo() .get(ApplicationMetricsConstants.APP_CPU_METRICS).toString())); - Assertions.assertEquals( + assertEquals( app.getRMAppMetrics().getPreemptedMemorySeconds(), Long.parseLong(entity.getOtherInfo() .get(ApplicationMetricsConstants.APP_MEM_PREEMPT_METRICS) .toString())); - Assertions.assertEquals( + assertEquals( app.getRMAppMetrics().getPreemptedVcoreSeconds(), Long.parseLong(entity.getOtherInfo() .get(ApplicationMetricsConstants.APP_CPU_PREEMPT_METRICS) .toString())); } - Assertions.assertEquals("context", entity.getOtherInfo() + assertEquals("context", entity.getOtherInfo() .get(ApplicationMetricsConstants.YARN_APP_CALLER_CONTEXT)); boolean hasCreatedEvent = false; boolean hasLaunchedEvent = false; @@ -277,35 +278,35 @@ public void testPublishApplicationMetrics() throws Exception { if (event.getEventType().equals( ApplicationMetricsConstants.CREATED_EVENT_TYPE)) { hasCreatedEvent = true; - Assertions.assertEquals(app.getStartTime(), event.getTimestamp()); + assertEquals(app.getStartTime(), event.getTimestamp()); } else if (event.getEventType().equals( ApplicationMetricsConstants.LAUNCHED_EVENT_TYPE)) { hasLaunchedEvent = true; - Assertions.assertEquals(app.getLaunchTime(), event.getTimestamp()); + assertEquals(app.getLaunchTime(), event.getTimestamp()); } else if (event.getEventType().equals( ApplicationMetricsConstants.FINISHED_EVENT_TYPE)) { hasFinishedEvent = true; - Assertions.assertEquals(app.getFinishTime(), event.getTimestamp()); - Assertions.assertEquals( + assertEquals(app.getFinishTime(), event.getTimestamp()); + assertEquals( app.getDiagnostics().toString(), event.getEventInfo().get( ApplicationMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO)); - Assertions.assertEquals( + assertEquals( app.getFinalApplicationStatus().toString(), event.getEventInfo().get( ApplicationMetricsConstants.FINAL_STATUS_EVENT_INFO)); - Assertions.assertEquals(YarnApplicationState.FINISHED.toString(), event + assertEquals(YarnApplicationState.FINISHED.toString(), event .getEventInfo().get(ApplicationMetricsConstants.STATE_EVENT_INFO)); } else if (event.getEventType().equals( ApplicationMetricsConstants.UPDATED_EVENT_TYPE)) { hasUpdatedEvent = true; - Assertions.assertEquals(4L, event.getTimestamp()); + assertEquals(4L, event.getTimestamp()); if (1 == i) { - Assertions.assertEquals( + assertEquals( 1, event.getEventInfo().get( ApplicationMetricsConstants.APPLICATION_PRIORITY_INFO)); - Assertions.assertEquals( + assertEquals( "new test queue", event.getEventInfo().get( ApplicationMetricsConstants.QUEUE_ENTITY_INFO)); @@ -313,23 +314,23 @@ public void testPublishApplicationMetrics() throws Exception { } else if (event.getEventType().equals( ApplicationMetricsConstants.ACLS_UPDATED_EVENT_TYPE)) { hasACLsUpdatedEvent = true; - Assertions.assertEquals(4L, event.getTimestamp()); + assertEquals(4L, event.getTimestamp()); } else if (event.getEventType().equals( ApplicationMetricsConstants.STATE_UPDATED_EVENT_TYPE)) { hasStateUpdateEvent = true; assertThat(event.getTimestamp()).isEqualTo(stateUpdateTimeStamp); - Assertions.assertEquals(YarnApplicationState.RUNNING.toString(), event + assertEquals(YarnApplicationState.RUNNING.toString(), event .getEventInfo().get( ApplicationMetricsConstants.STATE_EVENT_INFO)); } } // Do assertTrue verification separately for easier debug - Assertions.assertTrue(hasCreatedEvent); - Assertions.assertTrue(hasLaunchedEvent); - Assertions.assertTrue(hasFinishedEvent); - Assertions.assertTrue(hasACLsUpdatedEvent); - Assertions.assertTrue(hasUpdatedEvent); - Assertions.assertTrue(hasStateUpdateEvent); + assertTrue(hasCreatedEvent); + assertTrue(hasLaunchedEvent); + assertTrue(hasFinishedEvent); + assertTrue(hasACLsUpdatedEvent); + assertTrue(hasUpdatedEvent); + assertTrue(hasStateUpdateEvent); } } @@ -374,10 +375,10 @@ public void testPublishAppAttemptMetrics() throws Exception { // ensure two events are both published before leaving the loop } while (entity == null || entity.getEvents().size() < 2); // verify all the fields - Assertions.assertEquals(AppAttemptMetricsConstants.ENTITY_TYPE, + assertEquals(AppAttemptMetricsConstants.ENTITY_TYPE, entity.getEntityType()); - Assertions.assertEquals(appAttemptId.toString(), entity.getEntityId()); - Assertions.assertEquals( + assertEquals(appAttemptId.toString(), entity.getEntityId()); + assertEquals( appAttemptId.getApplicationId().toString(), entity.getPrimaryFilters() .get(AppAttemptMetricsConstants.PARENT_PRIMARY_FILTER).iterator() @@ -388,39 +389,38 @@ public void testPublishAppAttemptMetrics() throws Exception { if (event.getEventType().equals( AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE)) { hasRegisteredEvent = true; - Assertions.assertEquals(appAttempt.getHost(), + assertEquals(appAttempt.getHost(), event.getEventInfo() .get(AppAttemptMetricsConstants.HOST_INFO)); - Assertions - .assertEquals(appAttempt.getRpcPort(), + assertEquals(appAttempt.getRpcPort(), event.getEventInfo().get( AppAttemptMetricsConstants.RPC_PORT_INFO)); - Assertions.assertEquals( + assertEquals( appAttempt.getMasterContainer().getId().toString(), event.getEventInfo().get( AppAttemptMetricsConstants.MASTER_CONTAINER_INFO)); } else if (event.getEventType().equals( AppAttemptMetricsConstants.FINISHED_EVENT_TYPE)) { hasFinishedEvent = true; - Assertions.assertEquals(appAttempt.getDiagnostics(), event.getEventInfo() + assertEquals(appAttempt.getDiagnostics(), event.getEventInfo() .get(AppAttemptMetricsConstants.DIAGNOSTICS_INFO)); - Assertions.assertEquals(appAttempt.getTrackingUrl(), event.getEventInfo() + assertEquals(appAttempt.getTrackingUrl(), event.getEventInfo() .get(AppAttemptMetricsConstants.TRACKING_URL_INFO)); - Assertions.assertEquals( + assertEquals( appAttempt.getOriginalTrackingUrl(), event.getEventInfo().get( AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_INFO)); - Assertions.assertEquals( + assertEquals( FinalApplicationStatus.UNDEFINED.toString(), event.getEventInfo().get( AppAttemptMetricsConstants.FINAL_STATUS_INFO)); - Assertions.assertEquals( + assertEquals( YarnApplicationAttemptState.FINISHED.toString(), event.getEventInfo().get( AppAttemptMetricsConstants.STATE_INFO)); } } - Assertions.assertTrue(hasRegisteredEvent && hasFinishedEvent); + assertTrue(hasRegisteredEvent && hasFinishedEvent); } @Test @@ -438,17 +438,17 @@ public void testPublishHostPortInfoOnContainerFinished() throws Exception { ContainerMetricsConstants.ENTITY_TYPE, EnumSet.allOf(Field.class)); } while (entity == null || entity.getEvents().size() < 1); - Assertions.assertNotNull(entity.getOtherInfo()); - Assertions.assertEquals(2, entity.getOtherInfo().size()); - Assertions.assertNotNull(entity.getOtherInfo().get( + assertNotNull(entity.getOtherInfo()); + assertEquals(2, entity.getOtherInfo().size()); + assertNotNull(entity.getOtherInfo().get( ContainerMetricsConstants.ALLOCATED_HOST_INFO)); - Assertions.assertNotNull(entity.getOtherInfo().get( + assertNotNull(entity.getOtherInfo().get( ContainerMetricsConstants.ALLOCATED_PORT_INFO)); - Assertions.assertEquals( + assertEquals( container.getAllocatedNode().getHost(), entity.getOtherInfo().get( ContainerMetricsConstants.ALLOCATED_HOST_INFO)); - Assertions.assertEquals( + assertEquals( container.getAllocatedNode().getPort(), entity.getOtherInfo().get( ContainerMetricsConstants.ALLOCATED_PORT_INFO)); @@ -472,33 +472,33 @@ public void testPublishContainerMetrics() throws Exception { // ensure two events are both published before leaving the loop } while (entity == null || entity.getEvents().size() < 2); // verify all the fields - Assertions.assertEquals(ContainerMetricsConstants.ENTITY_TYPE, + assertEquals(ContainerMetricsConstants.ENTITY_TYPE, entity.getEntityType()); - Assertions.assertEquals(containerId.toString(), entity.getEntityId()); - Assertions.assertEquals( + assertEquals(containerId.toString(), entity.getEntityId()); + assertEquals( containerId.getApplicationAttemptId().toString(), entity.getPrimaryFilters() .get(ContainerMetricsConstants.PARENT_PRIMARIY_FILTER).iterator() .next()); - Assertions.assertEquals( + assertEquals( container.getAllocatedNode().getHost(), entity.getOtherInfo().get( ContainerMetricsConstants.ALLOCATED_HOST_INFO)); - Assertions.assertEquals( + assertEquals( container.getAllocatedNode().getPort(), entity.getOtherInfo().get( ContainerMetricsConstants.ALLOCATED_PORT_INFO)); - Assertions.assertEquals(container.getAllocatedResource().getMemorySize(), + assertEquals(container.getAllocatedResource().getMemorySize(), // KeyValueBasedTimelineStore could cast long to integer, need make sure // variables for compare have same type. ((Integer) entity.getOtherInfo().get( ContainerMetricsConstants.ALLOCATED_MEMORY_INFO)) .longValue()); - Assertions.assertEquals( + assertEquals( container.getAllocatedResource().getVirtualCores(), entity.getOtherInfo().get( ContainerMetricsConstants.ALLOCATED_VCORE_INFO)); - Assertions.assertEquals( + assertEquals( container.getAllocatedPriority().getPriority(), entity.getOtherInfo().get( ContainerMetricsConstants.ALLOCATED_PRIORITY_INFO)); @@ -508,24 +508,24 @@ public void testPublishContainerMetrics() throws Exception { if (event.getEventType().equals( ContainerMetricsConstants.CREATED_EVENT_TYPE)) { hasCreatedEvent = true; - Assertions.assertEquals(container.getCreationTime(), event.getTimestamp()); + assertEquals(container.getCreationTime(), event.getTimestamp()); } else if (event.getEventType().equals( ContainerMetricsConstants.FINISHED_EVENT_TYPE)) { hasFinishedEvent = true; - Assertions.assertEquals(container.getFinishTime(), event.getTimestamp()); - Assertions.assertEquals( + assertEquals(container.getFinishTime(), event.getTimestamp()); + assertEquals( container.getDiagnosticsInfo(), event.getEventInfo().get( ContainerMetricsConstants.DIAGNOSTICS_INFO)); - Assertions.assertEquals( + assertEquals( container.getContainerExitStatus(), event.getEventInfo().get( ContainerMetricsConstants.EXIT_STATUS_INFO)); - Assertions.assertEquals(container.getContainerState().toString(), event + assertEquals(container.getContainerState().toString(), event .getEventInfo().get(ContainerMetricsConstants.STATE_INFO)); } } - Assertions.assertTrue(hasCreatedEvent && hasFinishedEvent); + assertTrue(hasCreatedEvent && hasFinishedEvent); } private static RMApp createRMApp(ApplicationId appId) { From c5a0286569e6587dba667b13c4fbeda276378c59 Mon Sep 17 00:00:00 2001 From: slfan1989 Date: Sat, 8 Mar 2025 14:20:34 +0800 Subject: [PATCH 3/6] YARN-11262. Fix CheckStyle & Junit Test. --- .../yarn/client/api/impl/TestYarnClient.java | 317 ++++---- .../client/api/impl/TestYarnClientImpl.java | 73 +- .../ParameterizedSchedulerTestBase.java | 14 +- .../resourcemanager/TestApplicationACLs.java | 111 ++- .../yarn/server/resourcemanager/TestRM.java | 174 +++-- .../server/resourcemanager/TestRMRestart.java | 705 ++++++++++-------- .../TestWorkPreservingRMRestart.java | 275 ++++--- .../TestWorkPreservingUnmanagedAM.java | 67 +- .../TestAMLaunchFailure.java | 14 +- .../applicationsmanager/TestAMRestart.java | 89 ++- .../TestASMStateMachine.java | 34 +- .../TestApplicationMasterExpiry.java | 12 +- .../metrics/TestSystemMetricsPublisher.java | 70 +- .../TestSystemMetricsPublisherForV2.java | 43 +- ...tProportionalCapacityPreemptionPolicy.java | 25 +- ...CapacityPreemptionPolicyMockFramework.java | 49 +- ...CapacityPreemptionPolicyMockFramework.java | 16 +- .../reservation/TestReservationSystem.java | 49 +- .../resourcetracker/TestNMReconnect.java | 70 +- .../scheduler/TestAbstractYarnScheduler.java | 255 ++++--- ...TestSchedulingWithAllocationRequestId.java | 54 +- .../security/TestClientToAMTokens.java | 72 +- 22 files changed, 1504 insertions(+), 1084 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java index 08437c0a2c042..0f25073777ceb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java @@ -77,9 +77,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.util.Records; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import org.mockito.ArgumentCaptor; import org.slf4j.event.Level; @@ -97,6 +97,12 @@ import java.util.Set; 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.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; @@ -109,20 +115,22 @@ */ public class TestYarnClient extends ParameterizedSchedulerTestBase { - public TestYarnClient(SchedulerType type) throws IOException { - super(type); + public void initTestYarnClient(SchedulerType type) throws IOException { + initParameterizedSchedulerTestBase(type); + setup(); } protected void configureFairScheduler(YarnConfiguration conf) {} - @Before public void setup() { QueueMetrics.clearQueueMetrics(); DefaultMetricsSystem.setMiniClusterMode(true); } - @Test - public void testClientStop() { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testClientStop(SchedulerType type) throws IOException { + initTestYarnClient(type); Configuration conf = getConf(); ResourceManager rm = new ResourceManager(); rm.init(conf); @@ -135,9 +143,11 @@ public void testClientStop() { rm.stop(); } - @Test - public void testStartTimelineClientWithErrors() + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testStartTimelineClientWithErrors(SchedulerType type) throws Exception { + initTestYarnClient(type); // If timeline client failed to init with a NoClassDefFoundError // it should be wrapped with an informative error message testCreateTimelineClientWithError( @@ -148,10 +158,9 @@ public void testStartTimelineClientWithErrors() new CreateTimelineClientErrorVerifier(1) { @Override public void verifyError(Throwable e) { - Assert.assertTrue(e instanceof NoClassDefFoundError); - Assert.assertTrue(e.getMessage() != null && - e.getMessage().contains( - YarnConfiguration.TIMELINE_SERVICE_ENABLED)); + assertTrue(e instanceof NoClassDefFoundError); + assertTrue(e.getMessage() != null && + e.getMessage().contains(YarnConfiguration.TIMELINE_SERVICE_ENABLED)); } }); @@ -164,7 +173,7 @@ public void verifyError(Throwable e) { new NoClassDefFoundError("Mock a NoClassDefFoundError"), new CreateTimelineClientErrorVerifier(0) { @Override public void verifyError(Throwable e) { - Assert.fail("NoClassDefFoundError while creating timeline client" + fail("NoClassDefFoundError while creating timeline client" + "should be tolerated when timeline service is disabled."); } } @@ -178,10 +187,9 @@ public void verifyError(Throwable e) { new NoClassDefFoundError("Mock a NoClassDefFoundError"), new CreateTimelineClientErrorVerifier(1) { @Override public void verifyError(Throwable e) { - Assert.assertTrue(e instanceof NoClassDefFoundError); - Assert.assertTrue(e.getMessage() != null && - e.getMessage().contains( - YarnConfiguration.TIMELINE_SERVICE_ENABLED)); + assertTrue(e instanceof NoClassDefFoundError); + assertTrue(e.getMessage() != null && + e.getMessage().contains(YarnConfiguration.TIMELINE_SERVICE_ENABLED)); } } ); @@ -196,7 +204,7 @@ public void verifyError(Throwable e) { new CreateTimelineClientErrorVerifier(1) { @Override public void verifyError(Throwable e) { - Assert.assertTrue(e instanceof IOException); + assertTrue(e instanceof IOException); } } ); @@ -211,7 +219,7 @@ public void verifyError(Throwable e) { new CreateTimelineClientErrorVerifier(0) { @Override public void verifyError(Throwable e) { - Assert.fail("IOException while creating timeline client" + fail("IOException while creating timeline client" + "should be tolerated when best effort is true"); } } @@ -219,8 +227,11 @@ public void verifyError(Throwable e) { } @SuppressWarnings("deprecation") - @Test (timeout = 30000) - public void testSubmitApplication() throws Exception { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(value = 30) + public void testSubmitApplication(SchedulerType type) throws Exception { + initTestYarnClient(type); Configuration conf = getConf(); conf.setLong(YarnConfiguration.YARN_CLIENT_APP_SUBMISSION_POLL_INTERVAL_MS, 100); // speed up tests @@ -241,10 +252,10 @@ public void testSubmitApplication() throws Exception { mock(ApplicationSubmissionContext.class); try { client.submitApplication(contextWithoutApplicationId); - Assert.fail("Should throw the ApplicationIdNotProvidedException"); + fail("Should throw the ApplicationIdNotProvidedException"); } catch (YarnException e) { - Assert.assertTrue(e instanceof ApplicationIdNotProvidedException); - Assert.assertTrue(e.getMessage().contains( + assertTrue(e instanceof ApplicationIdNotProvidedException); + assertTrue(e.getMessage().contains( "ApplicationId is not provided in ApplicationSubmissionContext")); } @@ -266,8 +277,11 @@ public void testSubmitApplication() throws Exception { } @SuppressWarnings("deprecation") - @Test (timeout = 20000) - public void testSubmitApplicationInterrupted() throws IOException { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(value = 20) + public void testSubmitApplicationInterrupted(SchedulerType type) throws IOException { + initTestYarnClient(type); Configuration conf = getConf(); int pollIntervalMs = 1000; conf.setLong(YarnConfiguration.YARN_CLIENT_APP_SUBMISSION_POLL_INTERVAL_MS, @@ -311,13 +325,16 @@ public void run() { appSubmitThread.join(); } catch (InterruptedException e) { } - Assert.assertTrue("Expected an InterruptedException wrapped inside a " + - "YarnException", appSubmitThread.isInterrupted); + assertTrue(appSubmitThread.isInterrupted, + "Expected an InterruptedException wrapped inside a YarnException"); } } - @Test (timeout = 30000) - public void testSubmitIncorrectQueueToCapacityScheduler() throws IOException { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(value = 30) + public void testSubmitIncorrectQueueToCapacityScheduler(SchedulerType type) throws IOException { + initTestYarnClient(type); MiniYARNCluster cluster = new MiniYARNCluster("testMRAMTokens", 1, 1, 1); YarnClient rmClient = null; try { @@ -356,9 +373,9 @@ public void testSubmitIncorrectQueueToCapacityScheduler() throws IOException { // Submit the application to the applications manager rmClient.submitApplication(appContext); - Assert.fail("Job submission should have thrown an exception"); + fail("Job submission should have thrown an exception"); } catch (YarnException e) { - Assert.assertTrue(e.getMessage().contains("Failed to submit")); + assertTrue(e.getMessage().contains("Failed to submit")); } finally { if (rmClient != null) { rmClient.stop(); @@ -366,9 +383,12 @@ public void testSubmitIncorrectQueueToCapacityScheduler() throws IOException { cluster.stop(); } } - - @Test - public void testKillApplication() throws Exception { + + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(value = 30) + public void testKillApplication(SchedulerType type) throws Exception { + initTestYarnClient(type); MockRM rm = new MockRM(); rm.start(); RMApp app = MockRMAppSubmitter.submitWithMemory(2000, rm); @@ -383,8 +403,11 @@ public void testKillApplication() throws Exception { .forceKillApplication(any(KillApplicationRequest.class)); } - @Test(timeout = 30000) - public void testApplicationType() throws Exception { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(value = 30) + public void testApplicationType(SchedulerType type) throws Exception { + initTestYarnClient(type); GenericTestUtils.setRootLogLevel(Level.DEBUG); MockRM rm = new MockRM(); rm.start(); @@ -401,13 +424,16 @@ public void testApplicationType() throws Exception { .withCredentials(null) .withAppType("MAPREDUCE") .build()); - Assert.assertEquals("YARN", app.getApplicationType()); - Assert.assertEquals("MAPREDUCE", app1.getApplicationType()); + assertEquals("YARN", app.getApplicationType()); + assertEquals("MAPREDUCE", app1.getApplicationType()); rm.stop(); } - @Test(timeout = 30000) - public void testApplicationTypeLimit() throws Exception { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(value = 30) + public void testApplicationTypeLimit(SchedulerType type) throws Exception { + initTestYarnClient(type); GenericTestUtils.setRootLogLevel(Level.DEBUG); MockRM rm = new MockRM(); rm.start(); @@ -423,12 +449,15 @@ public void testApplicationTypeLimit() throws Exception { .withCredentials(null) .withAppType("MAPREDUCE-LENGTH-IS-20") .build()); - Assert.assertEquals("MAPREDUCE-LENGTH-IS-", app1.getApplicationType()); + assertEquals("MAPREDUCE-LENGTH-IS-", app1.getApplicationType()); rm.stop(); } - @Test (timeout = 10000) - public void testGetApplications() throws YarnException, IOException { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(value = 10) + public void testGetApplications(SchedulerType type) throws YarnException, IOException { + initTestYarnClient(type); final YarnClient client = new MockYarnClient(); client.init(getConf()); client.start(); @@ -445,13 +474,12 @@ public void testGetApplications() throws YarnException, IOException { reports = client.getApplications(appTypes, null); assertThat(reports).hasSize(2); - Assert - .assertTrue((reports.get(0).getApplicationType().equals("YARN") && reports - .get(1).getApplicationType().equals("NON-YARN")) - || (reports.get(1).getApplicationType().equals("YARN") && reports - .get(0).getApplicationType().equals("NON-YARN"))); + assertTrue((reports.get(0).getApplicationType().equals("YARN") && reports + .get(1).getApplicationType().equals("NON-YARN")) + || (reports.get(1).getApplicationType().equals("YARN") && reports + .get(0).getApplicationType().equals("NON-YARN"))); for(ApplicationReport report : reports) { - Assert.assertTrue(expectedReports.contains(report)); + assertTrue(expectedReports.contains(report)); } EnumSet appStates = @@ -460,27 +488,29 @@ public void testGetApplications() throws YarnException, IOException { appStates.add(YarnApplicationState.FAILED); reports = client.getApplications(null, appStates); assertThat(reports).hasSize(2); - Assert - .assertTrue((reports.get(0).getApplicationType().equals("NON-YARN") && reports + assertTrue((reports.get(0).getApplicationType().equals("NON-YARN") && reports .get(1).getApplicationType().equals("NON-MAPREDUCE")) || (reports.get(1).getApplicationType().equals("NON-YARN") && reports - .get(0).getApplicationType().equals("NON-MAPREDUCE"))); + .get(0).getApplicationType().equals("NON-MAPREDUCE"))); for (ApplicationReport report : reports) { - Assert.assertTrue(expectedReports.contains(report)); + assertTrue(expectedReports.contains(report)); } reports = client.getApplications(appTypes, appStates); - Assert.assertEquals(1, reports.size()); - Assert.assertEquals("NON-YARN", reports.get(0).getApplicationType()); + assertEquals(1, reports.size()); + assertEquals("NON-YARN", reports.get(0).getApplicationType()); for (ApplicationReport report : reports) { - Assert.assertTrue(expectedReports.contains(report)); + assertTrue(expectedReports.contains(report)); } client.stop(); } - @Test(timeout = 10000) - public void testGetApplicationAttempts() throws YarnException, IOException { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(value = 10) + public void testGetApplicationAttempts(SchedulerType type) throws YarnException, IOException { + initTestYarnClient(type); final YarnClient client = new MockYarnClient(); client.init(getConf()); client.start(); @@ -488,7 +518,7 @@ public void testGetApplicationAttempts() throws YarnException, IOException { ApplicationId applicationId = ApplicationId.newInstance(1234, 5); List reports = client .getApplicationAttempts(applicationId); - Assert.assertNotNull(reports); + assertNotNull(reports); assertThat(reports.get(0).getApplicationAttemptId()).isEqualTo( ApplicationAttemptId.newInstance(applicationId, 1)); assertThat(reports.get(1).getApplicationAttemptId()).isEqualTo( @@ -496,8 +526,11 @@ public void testGetApplicationAttempts() throws YarnException, IOException { client.stop(); } - @Test(timeout = 10000) - public void testGetApplicationAttempt() throws YarnException, IOException { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(value = 10) + public void testGetApplicationAttempt(SchedulerType type) throws YarnException, IOException { + initTestYarnClient(type); Configuration conf = new Configuration(); final YarnClient client = new MockYarnClient(); client.init(conf); @@ -511,14 +544,17 @@ public void testGetApplicationAttempt() throws YarnException, IOException { applicationId, 1); ApplicationAttemptReport report = client .getApplicationAttemptReport(appAttemptId); - Assert.assertNotNull(report); + assertNotNull(report); assertThat(report.getApplicationAttemptId().toString()).isEqualTo( expectedReports.get(0).getCurrentApplicationAttemptId().toString()); client.stop(); } - @Test(timeout = 10000) - public void testGetContainers() throws YarnException, IOException { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(value = 10) + public void testGetContainers(SchedulerType type) throws YarnException, IOException { + initTestYarnClient(type); Configuration conf = getConf(); conf.setBoolean(YarnConfiguration.APPLICATION_HISTORY_ENABLED, true); @@ -531,7 +567,7 @@ public void testGetContainers() throws YarnException, IOException { ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance( applicationId, 1); List reports = client.getContainers(appAttemptId); - Assert.assertNotNull(reports); + assertNotNull(reports); assertThat(reports.get(0).getContainerId()).isEqualTo( (ContainerId.newContainerId(appAttemptId, 1))); assertThat(reports.get(1).getContainerId()).isEqualTo( @@ -541,17 +577,20 @@ public void testGetContainers() throws YarnException, IOException { //First2 containers should come from RM with updated state information and // 3rd container is not there in RM and should - Assert.assertEquals(ContainerState.RUNNING, + assertEquals(ContainerState.RUNNING, (reports.get(0).getContainerState())); - Assert.assertEquals(ContainerState.RUNNING, + assertEquals(ContainerState.RUNNING, (reports.get(1).getContainerState())); - Assert.assertEquals(ContainerState.COMPLETE, + assertEquals(ContainerState.COMPLETE, (reports.get(2).getContainerState())); client.stop(); } - @Test(timeout = 10000) - public void testGetContainersOnAHSFail() throws YarnException, IOException { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(value = 10) + public void testGetContainersOnAHSFail(SchedulerType type) throws YarnException, IOException { + initTestYarnClient(type); Configuration conf = getConf(); conf.setBoolean(YarnConfiguration.APPLICATION_HISTORY_ENABLED, true); @@ -572,24 +611,27 @@ public List getContainers( ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance( applicationId, 1); List reports = client.getContainers(appAttemptId); - Assert.assertNotNull(reports); - Assert.assertTrue(reports.size() == 2); + assertNotNull(reports); + assertTrue(reports.size() == 2); assertThat(reports.get(0).getContainerId()).isEqualTo( (ContainerId.newContainerId(appAttemptId, 1))); assertThat(reports.get(1).getContainerId()).isEqualTo( (ContainerId.newContainerId(appAttemptId, 2))); //Only 2 running containers from RM are present when AHS throws exception - Assert.assertEquals(ContainerState.RUNNING, + assertEquals(ContainerState.RUNNING, (reports.get(0).getContainerState())); - Assert.assertEquals(ContainerState.RUNNING, + assertEquals(ContainerState.RUNNING, (reports.get(1).getContainerState())); client.stop(); } - @Test(timeout = 10000) - public void testGetContainerReport() throws YarnException, IOException { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(value = 10) + public void testGetContainerReport(SchedulerType type) throws YarnException, IOException { + initTestYarnClient(type); Configuration conf = getConf(); conf.setBoolean(YarnConfiguration.APPLICATION_HISTORY_ENABLED, true); @@ -605,22 +647,25 @@ public void testGetContainerReport() throws YarnException, IOException { applicationId, 1); ContainerId containerId = ContainerId.newContainerId(appAttemptId, 1); ContainerReport report = client.getContainerReport(containerId); - Assert.assertNotNull(report); + assertNotNull(report); assertThat(report.getContainerId().toString()).isEqualTo( (ContainerId.newContainerId(expectedReports.get(0) .getCurrentApplicationAttemptId(), 1)).toString()); containerId = ContainerId.newContainerId(appAttemptId, 3); report = client.getContainerReport(containerId); - Assert.assertNotNull(report); + assertNotNull(report); assertThat(report.getContainerId().toString()).isEqualTo( (ContainerId.newContainerId(expectedReports.get(0) .getCurrentApplicationAttemptId(), 3)).toString()); - Assert.assertNotNull(report.getExecutionType()); + assertNotNull(report.getExecutionType()); client.stop(); } - @Test (timeout = 10000) - public void testGetLabelsToNodes() throws YarnException, IOException { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(value = 10) + public void testGetLabelsToNodes(SchedulerType type) throws YarnException, IOException { + initTestYarnClient(type); final YarnClient client = new MockYarnClient(); client.init(getConf()); client.start(); @@ -644,8 +689,11 @@ public void testGetLabelsToNodes() throws YarnException, IOException { client.close(); } - @Test (timeout = 10000) - public void testGetNodesToLabels() throws YarnException, IOException { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(value = 10) + public void testGetNodesToLabels(SchedulerType type) throws YarnException, IOException { + initTestYarnClient(type); final YarnClient client = new MockYarnClient(); client.init(getConf()); client.start(); @@ -731,7 +779,7 @@ public void start() { historyClient = mock(AHSClient.class); } catch (Exception e) { - Assert.fail("Unexpected exception caught: " + e); + fail("Unexpected exception caught: " + e); } when(mockResponse.getApplicationReport()).thenReturn(mockReport); @@ -1048,8 +1096,11 @@ private ContainerReport getContainer( } } - @Test(timeout = 30000) - public void testAMMRTokens() throws Exception { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(value = 30) + public void testAMMRTokens(SchedulerType type) throws Exception { + initTestYarnClient(type); MiniYARNCluster cluster = new MiniYARNCluster("testMRAMTokens", 1, 1, 1); YarnClient rmClient = null; try { @@ -1063,19 +1114,19 @@ public void testAMMRTokens() throws Exception { ApplicationId appId = createApp(rmClient, false); waitTillAccepted(rmClient, appId, false); //managed AMs don't return AMRM token - Assert.assertNull(rmClient.getAMRMToken(appId)); + assertNull(rmClient.getAMRMToken(appId)); appId = createApp(rmClient, true); waitTillAccepted(rmClient, appId, true); long start = System.currentTimeMillis(); while (rmClient.getAMRMToken(appId) == null) { if (System.currentTimeMillis() - start > 20 * 1000) { - Assert.fail("AMRM token is null"); + fail("AMRM token is null"); } Thread.sleep(100); } //unmanaged AMs do return AMRM token - Assert.assertNotNull(rmClient.getAMRMToken(appId)); + assertNotNull(rmClient.getAMRMToken(appId)); UserGroupInformation other = UserGroupInformation.createUserForTesting("foo", new String[]{}); @@ -1091,17 +1142,17 @@ public ApplicationId run() throws Exception { long start = System.currentTimeMillis(); while (rmClient.getAMRMToken(appId) == null) { if (System.currentTimeMillis() - start > 20 * 1000) { - Assert.fail("AMRM token is null"); + fail("AMRM token is null"); } Thread.sleep(100); } //unmanaged AMs do return AMRM token - Assert.assertNotNull(rmClient.getAMRMToken(appId)); + assertNotNull(rmClient.getAMRMToken(appId)); return appId; } }); //other users don't get AMRM token - Assert.assertNull(rmClient.getAMRMToken(appId)); + assertNull(rmClient.getAMRMToken(appId)); } finally { if (rmClient != null) { rmClient.stop(); @@ -1160,43 +1211,51 @@ private void waitTillAccepted(YarnClient rmClient, ApplicationId appId, Thread.sleep(200); report = rmClient.getApplicationReport(appId); } - Assert.assertEquals(unmanagedApplication, report.isUnmanagedApp()); + assertEquals(unmanagedApplication, report.isUnmanagedApp()); } - @Test(timeout = 30000, expected = ApplicationNotFoundException.class) - public void testShouldNotRetryForeverForNonNetworkExceptions() throws Exception { - YarnConfiguration conf = getConf(); - conf.setInt(YarnConfiguration.RESOURCEMANAGER_CONNECT_MAX_WAIT_MS, -1); - - ResourceManager rm = null; - YarnClient yarnClient = null; - try { - // start rm - rm = new ResourceManager(); - rm.init(conf); - rm.start(); - - yarnClient = YarnClient.createYarnClient(); - yarnClient.init(conf); - yarnClient.start(); - - // create invalid application id - ApplicationId appId = ApplicationId.newInstance(1430126768L, 10645); - - // RM should throw ApplicationNotFoundException exception - yarnClient.getApplicationReport(appId); - } finally { - if (yarnClient != null) { - yarnClient.stop(); - } - if (rm != null) { - rm.stop(); + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(value = 30) + public void testShouldNotRetryForeverForNonNetworkExceptions(SchedulerType type) throws Exception { + assertThrows(ApplicationNotFoundException.class, () -> { + initTestYarnClient(type); + YarnConfiguration conf = getConf(); + conf.setInt(YarnConfiguration.RESOURCEMANAGER_CONNECT_MAX_WAIT_MS, -1); + + ResourceManager rm = null; + YarnClient yarnClient = null; + try { + // start rm + rm = new ResourceManager(); + rm.init(conf); + rm.start(); + + yarnClient = YarnClient.createYarnClient(); + yarnClient.init(conf); + yarnClient.start(); + + // create invalid application id + ApplicationId appId = ApplicationId.newInstance(1430126768L, 10645); + + // RM should throw ApplicationNotFoundException exception + yarnClient.getApplicationReport(appId); + } finally { + if (yarnClient != null) { + yarnClient.stop(); + } + if (rm != null) { + rm.stop(); + } } - } + }); } - @Test - public void testSignalContainer() throws Exception { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(value = 30) + public void testSignalContainer(SchedulerType type) throws Exception { + initTestYarnClient(type); @SuppressWarnings("resource") final YarnClient client = new MockYarnClient(); client.init(getConf()); @@ -1212,8 +1271,8 @@ public void testSignalContainer() throws Exception { verify(((MockYarnClient) client).getRMClient()) .signalToContainer(signalReqCaptor.capture()); SignalContainerRequest request = signalReqCaptor.getValue(); - Assert.assertEquals(containerId, request.getContainerId()); - Assert.assertEquals(command, request.getCommand()); + assertEquals(containerId, request.getContainerId()); + assertEquals(command, request.getCommand()); } private void testCreateTimelineClientWithError( diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClientImpl.java index 947fa9985e245..11938f0439ed1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClientImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClientImpl.java @@ -50,9 +50,8 @@ import org.apache.hadoop.yarn.server.resourcemanager .ParameterizedSchedulerTestBase; 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.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -60,6 +59,8 @@ import java.nio.ByteBuffer; import java.util.Collection; +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.Mockito.doAnswer; import static org.mockito.Mockito.mock; @@ -75,18 +76,20 @@ public class TestYarnClientImpl extends ParameterizedSchedulerTestBase { protected static final String YARN_RM = "yarn-rm@EXAMPLE.COM"; - public TestYarnClientImpl(SchedulerType type) throws IOException { - super(type); + public void initTestYarnClientImpl(SchedulerType type) throws IOException { + initParameterizedSchedulerTestBase(type); + setup(); } - @Before public void setup() { QueueMetrics.clearQueueMetrics(); DefaultMetricsSystem.setMiniClusterMode(true); } - @Test - public void testStartWithTimelineV15() { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testStartWithTimelineV15(SchedulerType type) throws IOException { + initTestYarnClientImpl(type); Configuration conf = getConf(); conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 1.5f); @@ -96,8 +99,10 @@ public void testStartWithTimelineV15() { client.stop(); } - @Test - public void testAsyncAPIPollTimeout() { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testAsyncAPIPollTimeout(SchedulerType type) throws IOException { + initTestYarnClientImpl(type); testAsyncAPIPollTimeoutHelper(null, false); testAsyncAPIPollTimeoutHelper(0L, true); testAsyncAPIPollTimeoutHelper(1L, true); @@ -116,16 +121,17 @@ private void testAsyncAPIPollTimeoutHelper(Long valueForTimeout, client.init(conf); - Assert.assertEquals( - expectedTimeoutEnforcement, client.enforceAsyncAPITimeout()); + assertEquals(expectedTimeoutEnforcement, client.enforceAsyncAPITimeout()); } finally { IOUtils.closeStream(client); } } - @Test - public void testBestEffortTimelineDelegationToken() - throws Exception { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testBestEffortTimelineDelegationToken(SchedulerType type) + throws Exception { + initTestYarnClientImpl(type); Configuration conf = getConf(); conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); SecurityUtil.setAuthenticationMethod(UserGroupInformation.AuthenticationMethod.KERBEROS, conf); @@ -152,7 +158,7 @@ TimelineClient createTimelineClient() throws IOException, YarnException { conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_CLIENT_BEST_EFFORT, false); client.serviceInit(conf); client.getTimelineDelegationToken(); - Assert.fail("Get delegation token should have thrown an exception"); + fail("Get delegation token should have thrown an exception"); } catch (IOException e) { // Success } @@ -161,9 +167,11 @@ TimelineClient createTimelineClient() throws IOException, YarnException { // Validates if YarnClientImpl automatically adds HDFS Delegation // token for Log Aggregation Path in a cluster setup with fs.DefaultFS // set to LocalFileSystem and Log Aggregation Path set to HDFS. - @Test - public void testAutomaitcLogAggregationDelegationToken() + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testAutomaitcLogAggregationDelegationToken(SchedulerType type) throws Exception { + initTestYarnClientImpl(type); Configuration conf = getConf(); SecurityUtil.setAuthenticationMethod( UserGroupInformation.AuthenticationMethod.KERBEROS, conf); @@ -262,10 +270,10 @@ public boolean isSecurityEnabled() { } Collection> dTokens = credentials.getAllTokens(); - Assert.assertEquals("Failed to place token for Log Aggregation Path", - 1, dTokens.size()); - Assert.assertEquals("Wrong Token for Log Aggregation", - hdfsDT.getKind(), dTokens.iterator().next().getKind()); + assertEquals(1, dTokens.size(), + "Failed to place token for Log Aggregation Path"); + assertEquals(hdfsDT.getKind(), dTokens.iterator().next().getKind(), + "Wrong Token for Log Aggregation"); } finally { if (hdfsCluster != null) { @@ -274,9 +282,11 @@ public boolean isSecurityEnabled() { } } - @Test - public void testAutomaticTimelineDelegationTokenLoading() - throws Exception { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testAutomaticTimelineDelegationTokenLoading(SchedulerType type) + throws Exception { + initTestYarnClientImpl(type); Configuration conf = getConf(); conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); SecurityUtil.setAuthenticationMethod(UserGroupInformation.AuthenticationMethod.KERBEROS, conf); @@ -356,16 +366,19 @@ public boolean isSecurityEnabled() { } Collection> dTokens = credentials.getAllTokens(); - Assert.assertEquals(1, dTokens.size()); - Assert.assertEquals(dToken, dTokens.iterator().next()); + assertEquals(1, dTokens.size()); + assertEquals(dToken, dTokens.iterator().next()); } } finally { client.stop(); } } - @Test - public void testParseTimelineDelegationTokenRenewer() { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testParseTimelineDelegationTokenRenewer(SchedulerType type) + throws IOException { + initTestYarnClientImpl(type); // Client side YarnClientImpl client = (YarnClientImpl) YarnClient.createYarnClient(); Configuration conf = getConf(); @@ -376,7 +389,7 @@ public void testParseTimelineDelegationTokenRenewer() { try { client.init(conf); client.start(); - Assert.assertEquals("rm/localhost@EXAMPLE.COM", client.timelineDTRenewer); + assertEquals("rm/localhost@EXAMPLE.COM", client.timelineDTRenewer); } finally { client.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/ParameterizedSchedulerTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ParameterizedSchedulerTestBase.java index 262168573d0ce..552988c61e98b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ParameterizedSchedulerTestBase.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ParameterizedSchedulerTestBase.java @@ -26,15 +26,11 @@ 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.server.resourcemanager.scheduler.fair .allocationfile.AllocationFileQueue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair .allocationfile.AllocationFileWriter; -import org.junit.After; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.AfterEach; import java.io.File; import java.io.IOException; @@ -42,7 +38,6 @@ import java.util.Collection; import java.util.stream.Collectors; -@RunWith(Parameterized.class) public abstract class ParameterizedSchedulerTestBase { protected final static String TEST_DIR = new File(System.getProperty("test.build.data", "/tmp")).getAbsolutePath(); @@ -53,7 +48,6 @@ public enum SchedulerType { CAPACITY, FAIR } - @Parameterized.Parameters(name = "{0}") public static Collection getParameters() { return Arrays.stream(SchedulerType.values()).map( type -> new Object[]{type}).collect(Collectors.toList()); @@ -68,7 +62,7 @@ public YarnConfiguration getConf() { } // Due to parameterization, this gets called before each test method - public ParameterizedSchedulerTestBase(SchedulerType type) + public void initParameterizedSchedulerTestBase(SchedulerType type) throws IOException { conf = new YarnConfiguration(); @@ -113,7 +107,7 @@ protected void configureFairScheduler(YarnConfiguration configuration) { configuration.setLong(FairSchedulerConfiguration.UPDATE_INTERVAL_MS, 10); } - @After + @AfterEach public void tearDown() { if (schedulerType == SchedulerType.FAIR) { (new File(FS_ALLOC_FILE)).delete(); @@ -128,7 +122,7 @@ public SchedulerType getSchedulerType() { * Return a scheduler configured by {@code YarnConfiguration.RM_SCHEDULER} * *

The scheduler is configured by - * {@link #ParameterizedSchedulerTestBase(SchedulerType)}. + * {@link ParameterizedSchedulerTestBase}. * Client test code can obtain the scheduler with this getter method. * Schedulers supported by this class are {@link FairScheduler} or * {@link CapacityScheduler}.

diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationACLs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationACLs.java index 7e8cf7e6f4ff2..7d7330ed9cf7f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationACLs.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationACLs.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.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import static org.mockito.ArgumentMatchers.any; @@ -43,8 +46,9 @@ .allocationfile.AllocationFileQueuePlacementRule; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair .allocationfile.AllocationFileWriter; -import org.junit.After; -import org.junit.Assert; +import org.junit.jupiter.api.AfterEach; +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; @@ -76,8 +80,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.Before; -import org.junit.Test; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -103,11 +105,12 @@ public class TestApplicationACLs extends ParameterizedSchedulerTestBase { private RecordFactory recordFactory; private boolean isQueueUser; - public TestApplicationACLs(SchedulerType type) throws IOException { - super(type); + public void initTestApplicationACLs(SchedulerType type) + throws IOException, InterruptedException { + initParameterizedSchedulerTestBase(type); + setup(); } - @Before public void setup() throws InterruptedException, IOException { conf = getConf(); rpc = YarnRPC.create(conf); @@ -181,7 +184,7 @@ public ApplicationClientProtocol run() throws Exception { }); } - @After + @AfterEach public void tearDown() { if(resourceManager != null) { resourceManager.stop(); @@ -210,8 +213,11 @@ protected void configureFairScheduler(YarnConfiguration configuration) { configuration.set(FairSchedulerConfiguration.ALLOCATION_FILE, allocFile); } - @Test - public void testApplicationACLs() throws Exception { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testApplicationACLs(SchedulerType type) throws Exception { + + initTestApplicationACLs(type); verifyOwnerAccess(); @@ -294,10 +300,9 @@ private void verifyOwnerAccess() throws Exception { rmClient.getApplicationReport(appReportRequest); // List apps as owner - Assert.assertEquals("App view by owner should list the apps!!", 1, - rmClient.getApplications( - recordFactory.newRecordInstance(GetApplicationsRequest.class)) - .getApplicationList().size()); + assertEquals(1, rmClient.getApplications( + recordFactory.newRecordInstance(GetApplicationsRequest.class)) + .getApplicationList().size(), "App view by owner should list the apps!!"); // Kill app as owner rmClient.forceKillApplication(finishAppRequest); @@ -325,10 +330,10 @@ private void verifySuperUserAccess() throws Exception { superUserClient.getApplicationReport(appReportRequest); // List apps as superUser - Assert.assertEquals("App view by super-user should list the apps!!", 2, + assertEquals(2, superUserClient.getApplications( - recordFactory.newRecordInstance(GetApplicationsRequest.class)) - .getApplicationList().size()); + recordFactory.newRecordInstance(GetApplicationsRequest.class)) + .getApplicationList().size(), "App view by super-user should list the apps!!"); // Kill app as the superUser superUserClient.forceKillApplication(finishAppRequest); @@ -356,10 +361,9 @@ private void verifyFriendAccess() throws Exception { friendClient.getApplicationReport(appReportRequest); // List apps as friend - Assert.assertEquals("App view by a friend should list the apps!!", 3, - friendClient.getApplications( - recordFactory.newRecordInstance(GetApplicationsRequest.class)) - .getApplicationList().size()); + assertEquals(3, friendClient.getApplications( + recordFactory.newRecordInstance(GetApplicationsRequest.class)) + .getApplicationList().size(), "App view by a friend should list the apps!!"); // Kill app as the friend friendClient.forceKillApplication(finishAppRequest); @@ -393,8 +397,8 @@ private void verifyEnemyAccess() throws Exception { .getApplications(recordFactory .newRecordInstance(GetApplicationsRequest.class)) .getApplicationList(); - Assert.assertEquals("App view by enemy should list the apps!!", 4, - appReports.size()); + assertEquals(4, appReports.size(), + "App view by enemy should list the apps!!"); for (ApplicationReport report : appReports) { verifyEnemyAppReport(report); } @@ -402,43 +406,39 @@ private void verifyEnemyAccess() throws Exception { // Kill app as the enemy try { enemyRmClient.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 enemy cannot perform operation MODIFY_APP on " - + applicationId)); + assertTrue(e.getMessage().contains( + "User enemy cannot perform operation MODIFY_APP on " + applicationId)); } rmClient.forceKillApplication(finishAppRequest); } private void verifyEnemyAppReport(ApplicationReport appReport) { - Assert.assertEquals("Enemy should not see app host!", - UNAVAILABLE, appReport.getHost()); - Assert.assertEquals("Enemy should not see app rpc port!", - -1, appReport.getRpcPort()); - Assert.assertEquals("Enemy should not see app client token!", - null, appReport.getClientToAMToken()); - Assert.assertEquals("Enemy should not see app diagnostics!", - UNAVAILABLE, appReport.getDiagnostics()); - Assert.assertEquals("Enemy should not see app tracking url!", - UNAVAILABLE, appReport.getTrackingUrl()); - Assert.assertEquals("Enemy should not see app original tracking url!", - UNAVAILABLE, appReport.getOriginalTrackingUrl()); + assertEquals(UNAVAILABLE, appReport.getHost(), "Enemy should not see app host!"); + assertEquals(-1, appReport.getRpcPort(), "Enemy should not see app rpc port!"); + assertEquals(null, appReport.getClientToAMToken(), + "Enemy should not see app client token!"); + assertEquals(UNAVAILABLE, appReport.getDiagnostics(), + "Enemy should not see app diagnostics!"); + assertEquals(UNAVAILABLE, appReport.getTrackingUrl(), + "Enemy should not see app tracking url!"); + assertEquals(UNAVAILABLE, appReport.getOriginalTrackingUrl(), + "Enemy should not see app original tracking url!"); ApplicationResourceUsageReport usageReport = appReport.getApplicationResourceUsageReport(); - Assert.assertEquals("Enemy should not see app used containers", - -1, usageReport.getNumUsedContainers()); - Assert.assertEquals("Enemy should not see app reserved containers", - -1, usageReport.getNumReservedContainers()); - Assert.assertEquals("Enemy should not see app used resources", - -1, usageReport.getUsedResources().getMemorySize()); - Assert.assertEquals("Enemy should not see app reserved resources", - -1, usageReport.getReservedResources().getMemorySize()); - Assert.assertEquals("Enemy should not see app needed resources", - -1, usageReport.getNeededResources().getMemorySize()); + assertEquals(-1, usageReport.getNumUsedContainers(), + "Enemy should not see app used containers"); + assertEquals(-1, usageReport.getNumReservedContainers(), + "Enemy should not see app reserved containers"); + assertEquals(-1, usageReport.getUsedResources().getMemorySize(), + "Enemy should not see app used resources"); + assertEquals(-1, usageReport.getReservedResources().getMemorySize(), + "Enemy should not see app reserved resources"); + assertEquals(-1, usageReport.getNeededResources().getMemorySize(), + "Enemy should not see app needed resources"); } private void verifyInvalidQueueWithAcl() throws Exception { @@ -472,10 +472,10 @@ private void verifyInvalidQueueWithAcl() throws Exception { ApplicationReport appReport = applicationReport.getApplicationReport(); if (conf.get(YarnConfiguration.RM_SCHEDULER) .equals(FairScheduler.class.getName())) { - Assert.assertTrue(appReport.getDiagnostics() + assertTrue(appReport.getDiagnostics() .contains("user owner application rejected by placement rules.")); } else { - Assert.assertTrue(appReport.getDiagnostics() + assertTrue(appReport.getDiagnostics() .contains("submitted by user owner to unknown queue: InvalidQueue")); } } @@ -502,10 +502,9 @@ private void verifyAdministerQueueUserAccess() throws Exception { administerQueueUserRmClient.getApplicationReport(appReportRequest); // List apps as administerQueueUserRmClient - Assert.assertEquals("App view by queue-admin-user should list the apps!!", - 5, administerQueueUserRmClient.getApplications( - recordFactory.newRecordInstance(GetApplicationsRequest.class)) - .getApplicationList().size()); + assertEquals(5, administerQueueUserRmClient.getApplications( + recordFactory.newRecordInstance(GetApplicationsRequest.class)) + .getApplicationList().size(), "App view by queue-admin-user should list the apps!!"); // Kill app as the administerQueueUserRmClient administerQueueUserRmClient.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/TestRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRM.java index a3bdf3945ad06..c1f3c33e658ea 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRM.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRM.java @@ -22,7 +22,10 @@ import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.yarn.event.DrainDispatcher; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler; -import org.junit.Before; +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.argThat; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.spy; @@ -36,9 +39,11 @@ import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; -import org.junit.After; -import org.junit.Assert; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.event.Level; @@ -76,7 +81,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM; -import org.junit.Test; import org.mockito.ArgumentMatcher; @SuppressWarnings({"unchecked", "rawtypes"}) @@ -89,24 +93,26 @@ public class TestRM extends ParameterizedSchedulerTestBase { private YarnConfiguration conf; - public TestRM(SchedulerType type) throws IOException { - super(type); + public void initTestRM(SchedulerType type) throws IOException { + initParameterizedSchedulerTestBase(type); + setup(); } - @Before public void setup() { conf = getConf(); } - @After + @AfterEach public void tearDown() { ClusterMetrics.destroy(); QueueMetrics.clearQueueMetrics(); DefaultMetricsSystem.shutdown(); } - @Test - public void testGetNewAppId() throws Exception { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testGetNewAppId(SchedulerType type) throws Exception { + initTestRM(type); GenericTestUtils.setRootLogLevel(Level.DEBUG); MockRM rm = new MockRM(conf); rm.start(); @@ -116,9 +122,12 @@ public void testGetNewAppId() throws Exception { assert (resp.getMaximumResourceCapability().getMemorySize() > 0); rm.stop(); } - - @Test (timeout = 30000) - public void testAppWithNoContainers() throws Exception { + + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(30) + public void testAppWithNoContainers(SchedulerType type) throws Exception { + initTestRM(type); GenericTestUtils.setRootLogLevel(Level.DEBUG); MockRM rm = new MockRM(conf); rm.start(); @@ -138,8 +147,11 @@ public void testAppWithNoContainers() throws Exception { rm.stop(); } - @Test (timeout = 30000) - public void testAppOnMultiNode() throws Exception { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(30) + public void testAppOnMultiNode(SchedulerType type) throws Exception { + initTestRM(type); GenericTestUtils.setRootLogLevel(Level.DEBUG); conf.set(CapacitySchedulerConfiguration.NODE_LOCALITY_DELAY, "-1"); MockRM rm = new MockRM(conf); @@ -172,7 +184,7 @@ public void testAppOnMultiNode() throws Exception { LOG.info("Got " + contReceived + " containers. Waiting to get " + 3); Thread.sleep(WAIT_SLEEP_MS); } - Assert.assertEquals(3, conts.size()); + assertEquals(3, conts.size()); //send node2 heartbeat conts = am.allocate(new ArrayList(), @@ -186,7 +198,7 @@ public void testAppOnMultiNode() throws Exception { LOG.info("Got " + contReceived + " containers. Waiting to get " + 10); Thread.sleep(WAIT_SLEEP_MS); } - Assert.assertEquals(10, conts.size()); + assertEquals(10, conts.size()); am.unregisterAppAttempt(); nm1.nodeHeartbeat(attempt.getAppAttemptId(), 1, ContainerState.COMPLETE); @@ -198,8 +210,11 @@ public void testAppOnMultiNode() throws Exception { // Test even if AM container is allocated with containerId not equal to 1, the // following allocate requests from AM should be able to retrieve the // corresponding NM Token. - @Test (timeout = 20000) - public void testNMTokenSentForNormalContainer() throws Exception { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(20) + public void testNMTokenSentForNormalContainer(SchedulerType type) throws Exception { + initTestRM(type); MockRM rm = new MockRM(conf); rm.start(); MockNM nm1 = rm.registerNode("h1:1234", 5120); @@ -215,10 +230,10 @@ public void testNMTokenSentForNormalContainer() throws Exception { MockAM am = MockRM.launchAM(app, rm, nm1); // am container Id not equal to 1. - Assert.assertTrue( + assertTrue( attempt.getMasterContainer().getId().getContainerId() != 1); // NMSecretManager doesn't record the node on which the am is allocated. - Assert.assertFalse(rm.getRMContext().getNMTokenSecretManager() + assertFalse(rm.getRMContext().getNMTokenSecretManager() .isApplicationAttemptNMTokenPresent(attempt.getAppAttemptId(), nm1.getNodeId())); am.registerAppAttempt(); @@ -245,11 +260,14 @@ public void testNMTokenSentForNormalContainer() throws Exception { } NodeId nodeId = expectedNMTokens.get(0).getNodeId(); // NMToken is sent for the allocated container. - Assert.assertEquals(nm1.getNodeId(), nodeId); + assertEquals(nm1.getNodeId(), nodeId); } - @Test (timeout = 40000) - public void testNMToken() throws Exception { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(40) + public void testNMToken(SchedulerType type) throws Exception { + initTestRM(type); MockRM rm = new MockRM(conf); try { rm.start(); @@ -270,7 +288,7 @@ public void testNMToken() throws Exception { MockAM am = rm.sendAMLaunched(attempt.getAppAttemptId()); - Assert.assertTrue(nmTokenSecretManager + assertTrue(nmTokenSecretManager .isApplicationAttemptRegistered(attempt.getAppAttemptId())); // This will register application master. @@ -285,18 +303,18 @@ public void testNMToken() throws Exception { // initially requesting 2 containers. AllocateResponse response = am.allocate("h1", 1000, 2, releaseContainerList); - Assert.assertEquals(0, response.getAllocatedContainers().size()); + assertEquals(0, response.getAllocatedContainers().size()); allocateContainersAndValidateNMTokens(am, containersReceivedForNM1, 2, nmTokens, nm1); - Assert.assertEquals(1, nmTokens.size()); + assertEquals(1, nmTokens.size()); // requesting 2 more containers. response = am.allocate("h1", 1000, 2, releaseContainerList); - Assert.assertEquals(0, response.getAllocatedContainers().size()); + assertEquals(0, response.getAllocatedContainers().size()); allocateContainersAndValidateNMTokens(am, containersReceivedForNM1, 4, nmTokens, nm1); - Assert.assertEquals(1, nmTokens.size()); + assertEquals(1, nmTokens.size()); // We will be simulating NM restart so restarting newly added h2:1234 @@ -307,10 +325,10 @@ public void testNMToken() throws Exception { new ArrayList(); response = am.allocate("h2", 1000, 2, releaseContainerList); - Assert.assertEquals(0, response.getAllocatedContainers().size()); + assertEquals(0, response.getAllocatedContainers().size()); allocateContainersAndValidateNMTokens(am, containersReceivedForNM2, 2, nmTokens, nm2); - Assert.assertEquals(2, nmTokens.size()); + assertEquals(2, nmTokens.size()); // Simulating NM-2 restart. nm2 = rm.registerNode("h2:1234", 10000); @@ -329,57 +347,57 @@ public void testNMToken() throws Exception { LOG.info("waiting for nmToken to be cleared for : " + nm2.getNodeId()); Thread.sleep(WAIT_SLEEP_MS); } - Assert.assertTrue(nmTokenSecretManager + assertTrue(nmTokenSecretManager .isApplicationAttemptRegistered(attempt.getAppAttemptId())); // removing NMToken for h2:1234 nmTokens.remove(nm2.getNodeId().toString()); - Assert.assertEquals(1, nmTokens.size()); + assertEquals(1, nmTokens.size()); // We should again receive the NMToken. response = am.allocate("h2", 1000, 2, releaseContainerList); - Assert.assertEquals(0, response.getAllocatedContainers().size()); + assertEquals(0, response.getAllocatedContainers().size()); allocateContainersAndValidateNMTokens(am, containersReceivedForNM2, 4, nmTokens, nm2); - Assert.assertEquals(2, nmTokens.size()); + assertEquals(2, nmTokens.size()); // Now rolling over NMToken masterKey. it should resend the NMToken in // next allocate call. - Assert.assertTrue(nmTokenSecretManager + assertTrue(nmTokenSecretManager .isApplicationAttemptNMTokenPresent(attempt.getAppAttemptId(), nm1.getNodeId())); - Assert.assertTrue(nmTokenSecretManager + assertTrue(nmTokenSecretManager .isApplicationAttemptNMTokenPresent(attempt.getAppAttemptId(), nm2.getNodeId())); nmTokenSecretManager.rollMasterKey(); nmTokenSecretManager.activateNextMasterKey(); - Assert.assertFalse(nmTokenSecretManager + assertFalse(nmTokenSecretManager .isApplicationAttemptNMTokenPresent(attempt.getAppAttemptId(), nm1.getNodeId())); - Assert.assertFalse(nmTokenSecretManager + assertFalse(nmTokenSecretManager .isApplicationAttemptNMTokenPresent(attempt.getAppAttemptId(), nm2.getNodeId())); // It should not remove application attempt entry. - Assert.assertTrue(nmTokenSecretManager + assertTrue(nmTokenSecretManager .isApplicationAttemptRegistered(attempt.getAppAttemptId())); nmTokens.clear(); - Assert.assertEquals(0, nmTokens.size()); + assertEquals(0, nmTokens.size()); // We should again receive the NMToken. response = am.allocate("h2", 1000, 1, releaseContainerList); - Assert.assertEquals(0, response.getAllocatedContainers().size()); + assertEquals(0, response.getAllocatedContainers().size()); allocateContainersAndValidateNMTokens(am, containersReceivedForNM2, 5, nmTokens, nm2); - Assert.assertEquals(1, nmTokens.size()); - Assert.assertTrue(nmTokenSecretManager + assertEquals(1, nmTokens.size()); + assertTrue(nmTokenSecretManager .isApplicationAttemptNMTokenPresent(attempt.getAppAttemptId(), nm2.getNodeId())); // After AM is finished making sure that nmtoken entry for app - Assert.assertTrue(nmTokenSecretManager + assertTrue(nmTokenSecretManager .isApplicationAttemptRegistered(attempt.getAppAttemptId())); am.unregisterAppAttempt(); // marking all the containers as finished. @@ -396,7 +414,7 @@ public void testNMToken() throws Exception { nm1.nodeHeartbeat(am.getApplicationAttemptId(), 1, ContainerState.COMPLETE); rm.waitForState(am.getApplicationAttemptId(), RMAppAttemptState.FINISHED); - Assert.assertFalse(nmTokenSecretManager + assertFalse(nmTokenSecretManager .isApplicationAttemptRegistered(attempt.getAppAttemptId())); } finally { rm.stop(); @@ -421,7 +439,7 @@ protected void allocateContainersAndValidateNMTokens(MockAM am, for (NMToken nmToken : response.getNMTokens()) { String nodeId = nmToken.getNodeId().toString(); if (nmTokens.containsKey(nodeId)) { - Assert.fail("Duplicate NMToken received for : " + nodeId); + fail("Duplicate NMToken received for : " + nodeId); } nmTokens.put(nodeId, nmToken.getToken()); } @@ -432,8 +450,11 @@ protected void allocateContainersAndValidateNMTokens(MockAM am, } } - @Test (timeout = 300000) - public void testActivatingApplicationAfterAddingNM() throws Exception { + @Timeout(300) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testActivatingApplicationAfterAddingNM(SchedulerType type) throws Exception { + initTestRM(type); MockRM rm1 = new MockRM(conf); // start like normal because state is empty @@ -477,8 +498,11 @@ public void testActivatingApplicationAfterAddingNM() throws Exception { // This is to test AM Host and rpc port are invalidated after the am attempt // is killed or failed, so that client doesn't get the wrong information. - @Test (timeout = 80000) - public void testInvalidateAMHostPortWhenAMFailedOrKilled() throws Exception { + @Timeout(80) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testInvalidateAMHostPortWhenAMFailedOrKilled(SchedulerType type) throws Exception { + initTestRM(type); conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1); MockRM rm1 = new MockRM(conf); rm1.start(); @@ -513,24 +537,27 @@ public void testInvalidateAMHostPortWhenAMFailedOrKilled() throws Exception { rm1.getClientRMService().getApplications(request1); List appList1 = response1.getApplicationList(); - Assert.assertEquals(3, appList1.size()); + assertEquals(3, appList1.size()); for (ApplicationReport report : appList1) { // killed/failed apps host and rpc port are invalidated. if (report.getApplicationId().equals(app2.getApplicationId()) || report.getApplicationId().equals(app3.getApplicationId())) { - Assert.assertEquals("N/A", report.getHost()); - Assert.assertEquals(-1, report.getRpcPort()); + assertEquals("N/A", report.getHost()); + assertEquals(-1, report.getRpcPort()); } // succeeded app's host and rpc port is not invalidated if (report.getApplicationId().equals(app1.getApplicationId())) { - Assert.assertFalse(report.getHost().equals("N/A")); - Assert.assertTrue(report.getRpcPort() != -1); + assertFalse(report.getHost().equals("N/A")); + assertTrue(report.getRpcPort() != -1); } } } - @Test (timeout = 60000) - public void testInvalidatedAMHostPortOnAMRestart() throws Exception { + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testInvalidatedAMHostPortOnAMRestart(SchedulerType type) throws Exception { + initTestRM(type); MockRM rm1 = new MockRM(conf); rm1.start(); MockNM nm1 = @@ -552,17 +579,19 @@ public void testInvalidatedAMHostPortOnAMRestart() throws Exception { ApplicationReport report1 = rm1.getClientRMService().getApplicationReport(request1) .getApplicationReport(); - Assert.assertEquals("N/A", report1.getHost()); - Assert.assertEquals(-1, report1.getRpcPort()); + assertEquals("N/A", report1.getHost()); + assertEquals(-1, report1.getRpcPort()); } /** * Validate killing an application when it is at accepted state. * @throws Exception exception */ - @Test (timeout = 60000) - public void testApplicationKillAtAcceptedState() throws Exception { - + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testApplicationKillAtAcceptedState(SchedulerType type) throws Exception { + initTestRM(type); final Dispatcher dispatcher = new DrainDispatcher() { @Override public EventHandler getEventHandler() { @@ -641,13 +670,16 @@ public Boolean get() { && appsSubmitted + 1 == metrics.getAppsSubmitted(); } }, 100, 10000); - Assert.assertEquals(appsKilled + 1, metrics.getAppsKilled()); - Assert.assertEquals(appsSubmitted + 1, metrics.getAppsSubmitted()); + assertEquals(appsKilled + 1, metrics.getAppsKilled()); + assertEquals(appsSubmitted + 1, metrics.getAppsSubmitted()); } // Test Kill an app while the app is finishing in the meanwhile. - @Test (timeout = 30000) - public void testKillFinishingApp() throws Exception{ + @Timeout(30) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testKillFinishingApp(SchedulerType type) throws Exception { + initTestRM(type); // this dispatcher ignores RMAppAttemptEventType.KILL event final Dispatcher dispatcher = new DrainDispatcher() { @@ -700,9 +732,11 @@ protected Dispatcher createDispatcher() { } // Test Kill an app while the app is failing - @Test (timeout = 30000) - public void testKillFailingApp() throws Exception{ - + @Timeout(30) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testKillFailingApp(SchedulerType type) throws Exception{ + initTestRM(type); // this dispatcher ignores RMAppAttemptEventType.KILL event final Dispatcher dispatcher = new DrainDispatcher() { @Override diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java index c5c15e2f3dbae..4d1cf857044c4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java @@ -19,11 +19,19 @@ package org.apache.hadoop.yarn.server.resourcemanager; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.isA; +import static org.mockito.Mockito.anyLong; +import static org.mockito.Mockito.isA; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.timeout; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -126,14 +134,13 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter; import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.ConverterUtils; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.mockito.Mockito; import java.util.function.Supplier; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.event.Level; @@ -150,11 +157,11 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase { private static InetSocketAddress rmAddr; private List rms = new ArrayList(); - public TestRMRestart(SchedulerType type) throws IOException { - super(type); + public void initTestRMRestart(SchedulerType type) throws IOException { + initParameterizedSchedulerTestBase(type); + setup(); } - @Before public void setup() throws IOException { conf = getConf(); GenericTestUtils.setRootLogLevel(Level.DEBUG); @@ -165,10 +172,10 @@ public void setup() throws IOException { conf.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS, FileSystemTimelineWriterImpl.class, TimelineWriter.class); rmAddr = new InetSocketAddress("localhost", 8032); - Assert.assertTrue(YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS > 1); + assertTrue(YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS > 1); } - @After + @AfterEach public void tearDown() { for (MockRM rm : rms) { rm.stop(); @@ -225,8 +232,11 @@ private RMApp submitApp(MockRM rm, List amResourceRequests, return MockRMAppSubmitter.submit(rm, data); } - @Test (timeout=180000) - public void testRMRestart() throws Exception { + @Timeout(180) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testRMRestart(SchedulerType type) throws Exception { + initTestRMRestart(type); conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS); @@ -251,7 +261,7 @@ public void testRMRestart() throws Exception { RMApp app0 = MockRMAppSubmitter.submitWithMemory(200, rm1); RMAppAttempt attempt0 = app0.getCurrentAppAttempt(); // spot check that app is saved - Assert.assertEquals(1, rmAppState.size()); + assertEquals(1, rmAppState.size()); nm1.nodeHeartbeat(true); MockAM am0 = rm1.sendAMLaunched(attempt0.getAppAttemptId()); am0.registerAppAttempt(); @@ -261,9 +271,9 @@ public void testRMRestart() throws Exception { RMApp app1 = MockRMAppSubmitter.submitWithMemory(200, rm1); // assert app1 info is saved ApplicationStateData appState = rmAppState.get(app1.getApplicationId()); - Assert.assertNotNull(appState); - Assert.assertEquals(0, appState.getAttemptCount()); - Assert.assertEquals(appState.getApplicationSubmissionContext() + assertNotNull(appState); + assertEquals(0, appState.getAttemptCount()); + assertEquals(appState.getApplicationSubmissionContext() .getApplicationId(), app1.getApplicationSubmissionContext() .getApplicationId()); @@ -274,11 +284,11 @@ public void testRMRestart() throws Exception { RMAppAttempt attempt1 = app1.getCurrentAppAttempt(); ApplicationAttemptId attemptId1 = attempt1.getAppAttemptId(); rm1.waitForState(attemptId1, RMAppAttemptState.ALLOCATED); - Assert.assertEquals(1, appState.getAttemptCount()); + assertEquals(1, appState.getAttemptCount()); ApplicationAttemptStateData attemptState = appState.getAttempt(attemptId1); - Assert.assertNotNull(attemptState); - Assert.assertEquals(BuilderUtils.newContainerId(attemptId1, 1), + assertNotNull(attemptState); + assertEquals(BuilderUtils.newContainerId(attemptId1, 1), attemptState.getMasterContainer().getId()); // launch the AM @@ -303,9 +313,9 @@ public void testRMRestart() throws Exception { // assert app2 info is saved appState = rmAppState.get(app2.getApplicationId()); - Assert.assertNotNull(appState); - Assert.assertEquals(0, appState.getAttemptCount()); - Assert.assertEquals(appState.getApplicationSubmissionContext() + assertNotNull(appState); + assertEquals(0, appState.getAttemptCount()); + assertEquals(appState.getApplicationSubmissionContext() .getApplicationId(), app2.getApplicationSubmissionContext() .getApplicationId()); @@ -327,13 +337,13 @@ public void testRMRestart() throws Exception { // assert appUnmanaged info is saved ApplicationId unmanagedAppId = appUnmanaged.getApplicationId(); appState = rmAppState.get(unmanagedAppId); - Assert.assertNotNull(appState); + assertNotNull(appState); // wait for attempt to reach LAUNCHED state rm1.waitForState(unmanagedAttemptId, RMAppAttemptState.LAUNCHED); rm1.waitForState(unmanagedAppId, RMAppState.ACCEPTED); // assert unmanaged attempt info is saved - Assert.assertEquals(1, appState.getAttemptCount()); - Assert.assertEquals(appState.getApplicationSubmissionContext() + assertEquals(1, appState.getAttemptCount()); + assertEquals(appState.getApplicationSubmissionContext() .getApplicationId(), appUnmanaged.getApplicationSubmissionContext() .getApplicationId()); @@ -355,7 +365,7 @@ public void testRMRestart() throws Exception { // Unmanaged app state is still loaded back but it cannot be restarted by // the RM. this will change with work preserving RM restart in which AMs/NMs // are not rebooted. - Assert.assertEquals(4, rm2.getRMContext().getRMApps().size()); + assertEquals(4, rm2.getRMContext().getRMApps().size()); // check that earlier finished app and attempt is also loaded back and move // to finished state. rm2.waitForState(app0.getApplicationId(), RMAppState.FINISHED); @@ -363,16 +373,16 @@ public void testRMRestart() throws Exception { // verify correct number of attempts and other data RMApp loadedApp1 = rm2.getRMContext().getRMApps().get(app1.getApplicationId()); - Assert.assertNotNull(loadedApp1); - Assert.assertEquals(1, loadedApp1.getAppAttempts().size()); - Assert.assertEquals(app1.getApplicationSubmissionContext() + assertNotNull(loadedApp1); + assertEquals(1, loadedApp1.getAppAttempts().size()); + assertEquals(app1.getApplicationSubmissionContext() .getApplicationId(), loadedApp1.getApplicationSubmissionContext() .getApplicationId()); RMApp loadedApp2 = rm2.getRMContext().getRMApps().get(app2.getApplicationId()); - Assert.assertNotNull(loadedApp2); - //Assert.assertEquals(0, loadedApp2.getAppAttempts().size()); - Assert.assertEquals(app2.getApplicationSubmissionContext() + assertNotNull(loadedApp2); + //assertEquals(0, loadedApp2.getAppAttempts().size()); + assertEquals(app2.getApplicationSubmissionContext() .getApplicationId(), loadedApp2.getApplicationSubmissionContext() .getApplicationId()); @@ -384,8 +394,8 @@ public void testRMRestart() throws Exception { // The app for which AM was started will wait for previous am // container finish event to arrive. However for an application for which // no am container was running will start new application attempt. - Assert.assertEquals(1, loadedApp1.getAppAttempts().size()); - Assert.assertEquals(1, loadedApp2.getAppAttempts().size()); + assertEquals(1, loadedApp1.getAppAttempts().size()); + assertEquals(1, loadedApp2.getAppAttempts().size()); // verify old AM is not accepted // change running AM to talk to new RM @@ -393,16 +403,16 @@ public void testRMRestart() throws Exception { try { am1.allocate(new ArrayList(), new ArrayList()); - Assert.fail(); + fail(); } catch (ApplicationAttemptNotFoundException e) { - Assert.assertTrue(e instanceof ApplicationAttemptNotFoundException); + assertTrue(e instanceof ApplicationAttemptNotFoundException); } // NM should be rebooted on heartbeat, even first heartbeat for nm2 NodeHeartbeatResponse hbResponse = nm1.nodeHeartbeat(true); - Assert.assertEquals(NodeAction.RESYNC, hbResponse.getNodeAction()); + assertEquals(NodeAction.RESYNC, hbResponse.getNodeAction()); hbResponse = nm2.nodeHeartbeat(true); - Assert.assertEquals(NodeAction.RESYNC, hbResponse.getNodeAction()); + assertEquals(NodeAction.RESYNC, hbResponse.getNodeAction()); // new NM to represent NM re-register nm1 = new MockNM("127.0.0.1:1234", 15120, rm2.getResourceTrackerService()); @@ -424,9 +434,9 @@ public void testRMRestart() throws Exception { // verify no more reboot response sent hbResponse = nm1.nodeHeartbeat(true); - Assert.assertTrue(NodeAction.RESYNC != hbResponse.getNodeAction()); + assertTrue(NodeAction.RESYNC != hbResponse.getNodeAction()); hbResponse = nm2.nodeHeartbeat(true); - Assert.assertTrue(NodeAction.RESYNC != hbResponse.getNodeAction()); + assertTrue(NodeAction.RESYNC != hbResponse.getNodeAction()); // assert app1 attempt is saved attempt1 = loadedApp1.getCurrentAppAttempt(); @@ -435,8 +445,8 @@ public void testRMRestart() throws Exception { rm2.waitForState(attemptId1, RMAppAttemptState.ALLOCATED); appState = rmAppState.get(loadedApp1.getApplicationId()); attemptState = appState.getAttempt(attemptId1); - Assert.assertNotNull(attemptState); - Assert.assertEquals(BuilderUtils.newContainerId(attemptId1, 1), + assertNotNull(attemptState); + assertEquals(BuilderUtils.newContainerId(attemptId1, 1), attemptState.getMasterContainer().getId()); // Nodes on which the AM's run @@ -452,8 +462,8 @@ public void testRMRestart() throws Exception { rm2.waitForState(attemptId2, RMAppAttemptState.ALLOCATED); appState = rmAppState.get(loadedApp2.getApplicationId()); attemptState = appState.getAttempt(attemptId2); - Assert.assertNotNull(attemptState); - Assert.assertEquals(BuilderUtils.newContainerId(attemptId2, 1), + assertNotNull(attemptState); + assertEquals(BuilderUtils.newContainerId(attemptId2, 1), attemptState.getMasterContainer().getId()); MockNM am2Node = nm1; @@ -495,11 +505,14 @@ public void testRMRestart() throws Exception { // completed apps are not removed immediately after app finish // And finished app is also loaded back. - Assert.assertEquals(4, rmAppState.size()); + assertEquals(4, rmAppState.size()); } - @Test(timeout = 60000) - public void testAppReportNodeLabelRMRestart() throws Exception { + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testAppReportNodeLabelRMRestart(SchedulerType type) throws Exception { + initTestRMRestart(type); if (getSchedulerType() != SchedulerType.CAPACITY) { return; } @@ -529,20 +542,22 @@ public void testAppReportNodeLabelRMRestart() throws Exception { // start new RM MockRM rm2 = createMockRM(conf, memStore); rm2.start(); - Assert.assertEquals(1, rm2.getRMContext().getRMApps().size()); + assertEquals(1, rm2.getRMContext().getRMApps().size()); ApplicationReport appReport = rm2.getClientRMService().getApplicationReport( GetApplicationReportRequest.newInstance(app0.getApplicationId())) .getApplicationReport(); - Assert - .assertEquals(amLabel.getName(), appReport.getAmNodeLabelExpression()); - Assert.assertEquals(appLabel.getName(), + assertEquals(amLabel.getName(), appReport.getAmNodeLabelExpression()); + assertEquals(appLabel.getName(), appReport.getAppNodeLabelExpression()); rm1.stop(); rm2.stop(); } - @Test(timeout = 60000) - public void testUnManagedRMRestart() throws Exception { + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testUnManagedRMRestart(SchedulerType type) throws Exception { + initTestRMRestart(type); // Create RM MockRM rm1 = createMockRM(conf); MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore(); @@ -562,17 +577,20 @@ public void testUnManagedRMRestart() throws Exception { // start new RM MockRM rm2 = createMockRM(conf, memStore); rm2.start(); - Assert.assertEquals(1, rm2.getRMContext().getRMApps().size()); + assertEquals(1, rm2.getRMContext().getRMApps().size()); ApplicationReport appReport = rm2.getClientRMService().getApplicationReport( GetApplicationReportRequest.newInstance(app0.getApplicationId())) .getApplicationReport(); - Assert.assertEquals(true, appReport.isUnmanagedApp()); + assertEquals(true, appReport.isUnmanagedApp()); rm1.stop(); rm2.stop(); } - @Test (timeout = 60000) - public void testRMRestartAppRunningAMFailed() throws Exception { + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testRMRestartAppRunningAMFailed(SchedulerType type) throws Exception { + initTestRMRestart(type); conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS); @@ -610,11 +628,11 @@ public void testRMRestartAppRunningAMFailed() throws Exception { ApplicationStateData appState = rmAppState.get(app0.getApplicationId()); // assert the AM failed state is saved. - Assert.assertEquals(RMAppAttemptState.FAILED, + assertEquals(RMAppAttemptState.FAILED, appState.getAttempt(am0.getApplicationAttemptId()).getState()); // assert app state has not been saved. - Assert.assertNull(rmAppState.get(app0.getApplicationId()).getState()); + assertNull(rmAppState.get(app0.getApplicationId()).getState()); // new AM started but not registered, app still stays at ACCECPTED state. rm1.waitForState(app0.getApplicationId(), RMAppState.ACCEPTED); @@ -627,8 +645,11 @@ public void testRMRestartAppRunningAMFailed() throws Exception { rm2.waitForState(am0.getApplicationAttemptId(), RMAppAttemptState.FAILED); } - @Test (timeout = 60000) - public void testRMRestartWaitForPreviousAMToFinish() throws Exception { + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testRMRestartWaitForPreviousAMToFinish(SchedulerType type) throws Exception { + initTestRMRestart(type); // testing 3 cases // After RM restarts // 1) New application attempt is not started until previous AM container @@ -667,9 +688,9 @@ public void testRMRestartWaitForPreviousAMToFinish() throws Exception { // launch another AM. MockAM am2 = launchAM(app1, rm1, nm1); - Assert.assertEquals(1, rmAppState.size()); + assertEquals(1, rmAppState.size()); assertThat(app1.getState()).isEqualTo(RMAppState.RUNNING); - Assert.assertEquals(app1.getAppAttempts() + assertEquals(app1.getAppAttempts() .get(app1.getCurrentAppAttempt().getAppAttemptId()) .getAppAttemptState(), RMAppAttemptState.RUNNING); @@ -679,23 +700,23 @@ public void testRMRestartWaitForPreviousAMToFinish() throws Exception { nm1.setResourceTrackerService(rm2.getResourceTrackerService()); NodeHeartbeatResponse res = nm1.nodeHeartbeat(true); - Assert.assertEquals(NodeAction.RESYNC, res.getNodeAction()); + assertEquals(NodeAction.RESYNC, res.getNodeAction()); RMApp rmApp = rm2.getRMContext().getRMApps().get(app1.getApplicationId()); // application should be in ACCEPTED state rm2.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); - Assert.assertEquals(RMAppState.ACCEPTED, rmApp.getState()); + assertEquals(RMAppState.ACCEPTED, rmApp.getState()); // new attempt should not be started - Assert.assertEquals(2, rmApp.getAppAttempts().size()); + assertEquals(2, rmApp.getAppAttempts().size()); // am1 attempt should be in FAILED state where as am2 attempt should be in // LAUNCHED state rm2.waitForState(am1.getApplicationAttemptId(), RMAppAttemptState.FAILED); rm2.waitForState(am2.getApplicationAttemptId(), RMAppAttemptState.LAUNCHED); - Assert.assertEquals(RMAppAttemptState.FAILED, + assertEquals(RMAppAttemptState.FAILED, rmApp.getAppAttempts().get(am1.getApplicationAttemptId()) .getAppAttemptState()); - Assert.assertEquals(RMAppAttemptState.LAUNCHED, + assertEquals(RMAppAttemptState.LAUNCHED, rmApp.getAppAttempts().get(am2.getApplicationAttemptId()) .getAppAttemptState()); @@ -709,7 +730,7 @@ public void testRMRestartWaitForPreviousAMToFinish() throws Exception { am2.getApplicationAttemptId()); launchAM(rmApp, rm2, nm1); - Assert.assertEquals(3, rmApp.getAppAttempts().size()); + assertEquals(3, rmApp.getAppAttempts().size()); rm2.waitForState(rmApp.getCurrentAppAttempt().getAppAttemptId(), RMAppAttemptState.RUNNING); // Now restart RM ... @@ -726,7 +747,7 @@ public void testRMRestartWaitForPreviousAMToFinish() throws Exception { rm3.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); assertThat(rmApp.getState()).isEqualTo(RMAppState.ACCEPTED); // new attempt should not be started - Assert.assertEquals(3, rmApp.getAppAttempts().size()); + assertEquals(3, rmApp.getAppAttempts().size()); // am1 and am2 attempts should be in FAILED state where as am3 should be // in LAUNCHED state rm3.waitForState(am1.getApplicationAttemptId(), RMAppAttemptState.FAILED); @@ -734,13 +755,13 @@ public void testRMRestartWaitForPreviousAMToFinish() throws Exception { ApplicationAttemptId latestAppAttemptId = rmApp.getCurrentAppAttempt().getAppAttemptId(); rm3.waitForState(latestAppAttemptId, RMAppAttemptState.LAUNCHED); - Assert.assertEquals(RMAppAttemptState.FAILED, + assertEquals(RMAppAttemptState.FAILED, rmApp.getAppAttempts().get(am1.getApplicationAttemptId()) .getAppAttemptState()); - Assert.assertEquals(RMAppAttemptState.FAILED, + assertEquals(RMAppAttemptState.FAILED, rmApp.getAppAttempts().get(am2.getApplicationAttemptId()) .getAppAttemptState()); - Assert.assertEquals(RMAppAttemptState.LAUNCHED,rmApp.getAppAttempts() + assertEquals(RMAppAttemptState.LAUNCHED,rmApp.getAppAttempts() .get(latestAppAttemptId).getAppAttemptState()); rm3.waitForState(latestAppAttemptId, RMAppAttemptState.FAILED); @@ -755,7 +776,7 @@ public Boolean get() { } }, 100, maxRetry * 100); - Assert.assertEquals(RMAppAttemptState.FAILED, + assertEquals(RMAppAttemptState.FAILED, rmApp.getAppAttempts().get(latestAppAttemptId).getAppAttemptState()); latestAppAttemptId = rmApp.getCurrentAppAttempt().getAppAttemptId(); @@ -766,8 +787,8 @@ public Boolean get() { // submitting app but not starting AM for it. RMApp app2 = MockRMAppSubmitter.submitWithMemory(200, rm3); rm3.waitForState(app2.getApplicationId(), RMAppState.ACCEPTED); - Assert.assertEquals(1, app2.getAppAttempts().size()); - Assert.assertEquals(0, + assertEquals(1, app2.getAppAttempts().size()); + assertEquals(0, memStore.getState().getApplicationState().get(app2.getApplicationId()) .getAttemptCount()); @@ -781,21 +802,21 @@ public Boolean get() { while (rmApp.getAppAttempts().size() != 2 && timeoutSecs++ < 40) { Thread.sleep(200); } - Assert.assertEquals(4, rmApp.getAppAttempts().size()); - Assert.assertEquals(RMAppState.ACCEPTED, rmApp.getState()); + assertEquals(4, rmApp.getAppAttempts().size()); + assertEquals(RMAppState.ACCEPTED, rmApp.getState()); rm4.waitForState(latestAppAttemptId, RMAppAttemptState.SCHEDULED); - Assert.assertEquals(RMAppAttemptState.SCHEDULED, rmApp.getAppAttempts() + assertEquals(RMAppAttemptState.SCHEDULED, rmApp.getAppAttempts() .get(latestAppAttemptId).getAppAttemptState()); // The initial application for which an AM was not started should be in // ACCEPTED state with one application attempt started. app2 = rm4.getRMContext().getRMApps().get(app2.getApplicationId()); rm4.waitForState(app2.getApplicationId(), RMAppState.ACCEPTED); - Assert.assertEquals(RMAppState.ACCEPTED, app2.getState()); - Assert.assertEquals(1, app2.getAppAttempts().size()); + assertEquals(RMAppState.ACCEPTED, app2.getState()); + assertEquals(1, app2.getAppAttempts().size()); rm4.waitForState(app2.getCurrentAppAttempt().getAppAttemptId(), RMAppAttemptState.SCHEDULED); - Assert.assertEquals(RMAppAttemptState.SCHEDULED, app2 + assertEquals(RMAppAttemptState.SCHEDULED, app2 .getCurrentAppAttempt().getAppAttemptState()); } @@ -803,8 +824,11 @@ public Boolean get() { // store but before the RMAppAttempt notifies RMApp that it has succeeded. On // recovery, RMAppAttempt should send the AttemptFinished event to RMApp so // that RMApp can recover its state. - @Test (timeout = 60000) - public void testRMRestartWaitForPreviousSucceededAttempt() throws Exception { + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testRMRestartWaitForPreviousSucceededAttempt(SchedulerType type) throws Exception { + initTestRMRestart(type); conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2); MemoryRMStateStore memStore = new MockMemoryRMStateStore() { int count = 0; @@ -841,7 +865,7 @@ public void updateApplicationStateInternal(ApplicationId appId, // app final state is not saved. This guarantees that RMApp cannot be // recovered via its own saved state, but only via the event notification // from the RMAppAttempt on recovery. - Assert.assertNull(rmAppState.get(app0.getApplicationId()).getState()); + assertNull(rmAppState.get(app0.getApplicationId()).getState()); // start RM MockRM rm2 = createMockRM(conf, memStore); @@ -852,12 +876,15 @@ public void updateApplicationStateInternal(ApplicationId appId, RMAppAttemptState.FINISHED); rm2.waitForState(app0.getApplicationId(), RMAppState.FINISHED); // app final state is saved via the finish event from attempt. - Assert.assertEquals(RMAppState.FINISHED, + assertEquals(RMAppState.FINISHED, rmAppState.get(app0.getApplicationId()).getState()); } - @Test (timeout = 60000) - public void testRMRestartFailedApp() throws Exception { + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testRMRestartFailedApp(SchedulerType type) throws Exception { + initTestRMRestart(type); conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1); // create RM MockRM rm1 = createMockRM(conf); @@ -882,8 +909,8 @@ public void testRMRestartFailedApp() throws Exception { // assert the app/attempt failed state is saved. ApplicationStateData appState = rmAppState.get(app0.getApplicationId()); - Assert.assertEquals(RMAppState.FAILED, appState.getState()); - Assert.assertEquals(RMAppAttemptState.FAILED, + assertEquals(RMAppState.FAILED, appState.getState()); + assertEquals(RMAppAttemptState.FAILED, appState.getAttempt(am0.getApplicationAttemptId()).getState()); // start new RM @@ -892,19 +919,22 @@ public void testRMRestartFailedApp() throws Exception { RMApp loadedApp0 = rm2.getRMContext().getRMApps().get(app0.getApplicationId()); rm2.waitForState(app0.getApplicationId(), RMAppState.FAILED); rm2.waitForState(am0.getApplicationAttemptId(), RMAppAttemptState.FAILED); - Assert.assertEquals(app0.getUser(), loadedApp0.getUser()); + assertEquals(app0.getUser(), loadedApp0.getUser()); // no new attempt is created. - Assert.assertEquals(1, loadedApp0.getAppAttempts().size()); + assertEquals(1, loadedApp0.getAppAttempts().size()); verifyAppReportAfterRMRestart(app0, rm2); - Assert.assertTrue(app0.getDiagnostics().toString() + assertTrue(app0.getDiagnostics().toString() .contains("Failing the application.")); // failed diagnostics from attempt is lost because the diagnostics from // attempt is not yet available by the time app is saving the app state. } - @Test (timeout = 60000) - public void testRMRestartKilledApp() throws Exception{ + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testRMRestartKilledApp(SchedulerType type) throws Exception { + initTestRMRestart(type); conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS); // create RM @@ -930,11 +960,11 @@ public void testRMRestartKilledApp() throws Exception{ // killed state is saved. ApplicationStateData appState = rmAppState.get(app0.getApplicationId()); - Assert.assertEquals(RMAppState.KILLED, appState.getState()); - Assert.assertEquals(RMAppAttemptState.KILLED, + assertEquals(RMAppState.KILLED, appState.getState()); + assertEquals(RMAppAttemptState.KILLED, appState.getAttempt(am0.getApplicationAttemptId()).getState()); String trackingUrl = app0.getCurrentAppAttempt().getOriginalTrackingUrl(); - Assert.assertNotNull(trackingUrl); + assertNotNull(trackingUrl); // restart rm MockRM rm2 = createMockRM(conf, memStore); @@ -943,17 +973,20 @@ public void testRMRestartKilledApp() throws Exception{ rm2.waitForState(app0.getApplicationId(), RMAppState.KILLED); rm2.waitForState(am0.getApplicationAttemptId(), RMAppAttemptState.KILLED); // no new attempt is created. - Assert.assertEquals(1, loadedApp0.getAppAttempts().size()); + assertEquals(1, loadedApp0.getAppAttempts().size()); ApplicationReport appReport = verifyAppReportAfterRMRestart(app0, rm2); - Assert.assertEquals(app0.getDiagnostics().toString(), + assertEquals(app0.getDiagnostics().toString(), appReport.getDiagnostics()); - Assert.assertEquals(trackingUrl, loadedApp0.getCurrentAppAttempt() + assertEquals(trackingUrl, loadedApp0.getCurrentAppAttempt() .getOriginalTrackingUrl()); } - @Test (timeout = 60000) - public void testRMRestartKilledAppWithNoAttempts() throws Exception { + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testRMRestartKilledAppWithNoAttempts(SchedulerType type) throws Exception { + initTestRMRestart(type); MockMemoryRMStateStore memStore = new MockMemoryRMStateStore() { @Override public synchronized void storeApplicationAttemptStateInternal( @@ -998,11 +1031,14 @@ public synchronized void updateApplicationAttemptStateInternal( RMApp loadedApp0 = rm2.getRMContext().getRMApps().get(app0.getApplicationId()); rm2.waitForState(loadedApp0.getApplicationId(), RMAppState.KILLED); - Assert.assertTrue(loadedApp0.getAppAttempts().size() == 0); + assertTrue(loadedApp0.getAppAttempts().size() == 0); } - @Test (timeout = 60000) - public void testRMRestartSucceededApp() throws Exception { + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testRMRestartSucceededApp(SchedulerType type) throws Exception { + initTestRMRestart(type); conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS); // PHASE 1: create RM and get state @@ -1032,11 +1068,11 @@ public void testRMRestartSucceededApp() throws Exception { ApplicationStateData appState = rmAppState.get(app0.getApplicationId()); ApplicationAttemptStateData attemptState0 = appState.getAttempt(am0.getApplicationAttemptId()); - Assert.assertEquals("diagnostics", attemptState0.getDiagnostics()); - Assert.assertEquals(FinalApplicationStatus.SUCCEEDED, + assertEquals("diagnostics", attemptState0.getDiagnostics()); + assertEquals(FinalApplicationStatus.SUCCEEDED, attemptState0.getFinalApplicationStatus()); - Assert.assertEquals("trackingUrl", attemptState0.getFinalTrackingUrl()); - Assert.assertEquals(app0.getFinishTime(), appState.getFinishTime()); + assertEquals("trackingUrl", attemptState0.getFinalTrackingUrl()); + assertEquals(app0.getFinishTime(), appState.getFinishTime()); // restart rm MockRM rm2 = createMockRM(conf, memStore); @@ -1045,13 +1081,16 @@ public void testRMRestartSucceededApp() throws Exception { // verify application report returns the same app info as the app info // before RM restarts. ApplicationReport appReport = verifyAppReportAfterRMRestart(app0, rm2); - Assert.assertEquals(FinalApplicationStatus.SUCCEEDED, + assertEquals(FinalApplicationStatus.SUCCEEDED, appReport.getFinalApplicationStatus()); - Assert.assertEquals("trackingUrl", appReport.getOriginalTrackingUrl()); + assertEquals("trackingUrl", appReport.getOriginalTrackingUrl()); } - @Test (timeout = 60000) - public void testRMRestartGetApplicationList() throws Exception { + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testRMRestartGetApplicationList(SchedulerType type) throws Exception { + initTestRMRestart(type); conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1); // start RM MockRM rm1 = new MockRM(conf) { @@ -1119,7 +1158,7 @@ protected SystemMetricsPublisher createSystemMetricsPublisher() { rm1.waitForState(app2.getApplicationId(), RMAppState.KILLED); rm1.waitForState(am2.getApplicationAttemptId(), RMAppAttemptState.KILLED); - verify(rm1.getRMContext().getSystemMetricsPublisher(),Mockito.times(3)) + verify(rm1.getRMContext().getSystemMetricsPublisher(),times(3)) .appCreated(any(RMApp.class), anyLong()); // restart rm @@ -1137,7 +1176,7 @@ protected SystemMetricsPublisher createSystemMetricsPublisher() { rms.add(rm2); rm2.start(); - verify(rm2.getRMContext().getSystemMetricsPublisher(),Mockito.times(3)) + verify(rm2.getRMContext().getSystemMetricsPublisher(),times(3)) .appCreated(any(RMApp.class), anyLong()); GetApplicationsRequest request1 = @@ -1153,22 +1192,22 @@ protected SystemMetricsPublisher createSystemMetricsPublisher() { boolean forApp0 = false, forApp1 = false, forApp2 = false; for (ApplicationReport report : appList1) { if (report.getApplicationId().equals(app0.getApplicationId())) { - Assert.assertEquals(YarnApplicationState.FINISHED, + assertEquals(YarnApplicationState.FINISHED, report.getYarnApplicationState()); forApp0 = true; } if (report.getApplicationId().equals(app1.getApplicationId())) { - Assert.assertEquals(YarnApplicationState.FAILED, + assertEquals(YarnApplicationState.FAILED, report.getYarnApplicationState()); forApp1 = true; } if (report.getApplicationId().equals(app2.getApplicationId())) { - Assert.assertEquals(YarnApplicationState.KILLED, + assertEquals(YarnApplicationState.KILLED, report.getYarnApplicationState()); forApp2 = true; } } - Assert.assertTrue(forApp0 && forApp1 && forApp2); + assertTrue(forApp0 && forApp1 && forApp2); // assert all applications exist according to application type after RM // restarts. @@ -1179,7 +1218,7 @@ protected SystemMetricsPublisher createSystemMetricsPublisher() { GetApplicationsResponse response2 = rm2.getClientRMService().getApplications(request2); List appList2 = response2.getApplicationList(); - Assert.assertTrue(3 == appList2.size()); + assertTrue(3 == appList2.size()); // check application summary is logged for the completed apps with timeout // to make sure APP_COMPLETED events are processed, after RM restart. @@ -1204,11 +1243,11 @@ private ApplicationReport verifyAppReportAfterRMRestart(RMApp app, MockRM rm) GetApplicationReportResponse response = rm.getClientRMService().getApplicationReport(reportRequest); ApplicationReport report = response.getApplicationReport(); - Assert.assertEquals(app.getStartTime(), report.getStartTime()); - Assert.assertEquals(app.getFinishTime(), report.getFinishTime()); - Assert.assertEquals(app.createApplicationState(), + assertEquals(app.getStartTime(), report.getStartTime()); + assertEquals(app.getFinishTime(), report.getFinishTime()); + assertEquals(app.createApplicationState(), report.getYarnApplicationState()); - Assert.assertTrue(1 == report.getProgress()); + assertTrue(1 == report.getProgress()); return response.getApplicationReport(); } @@ -1233,14 +1272,16 @@ private void finishApplicationMaster(RMApp rmApp, MockRM rm, MockNM nm, rm.waitForState(rmApp.getApplicationId(), RMAppState.FINISHED); // check that app/attempt is saved with the final state ApplicationStateData appState = rmAppState.get(rmApp.getApplicationId()); - Assert - .assertEquals(RMAppState.FINISHED, appState.getState()); - Assert.assertEquals(RMAppAttemptState.FINISHED, + assertEquals(RMAppState.FINISHED, appState.getState()); + assertEquals(RMAppAttemptState.FINISHED, appState.getAttempt(am.getApplicationAttemptId()).getState()); } - @Test (timeout = 60000) - public void testRMRestartOnMaxAppAttempts() throws Exception { + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testRMRestartOnMaxAppAttempts(SchedulerType type) throws Exception { + initTestRMRestart(type); conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS); @@ -1280,9 +1321,9 @@ public void testRMRestartOnMaxAppAttempts() throws Exception { // assert app1 info is saved ApplicationStateData appState = rmAppState.get(app1.getApplicationId()); - Assert.assertNotNull(appState); - Assert.assertEquals(0, appState.getAttemptCount()); - Assert.assertEquals(appState.getApplicationSubmissionContext() + assertNotNull(appState); + assertEquals(0, appState.getAttemptCount()); + assertEquals(appState.getApplicationSubmissionContext() .getApplicationId(), app1.getApplicationSubmissionContext() .getApplicationId()); @@ -1291,11 +1332,11 @@ public void testRMRestartOnMaxAppAttempts() throws Exception { RMAppAttempt attempt = app1.getCurrentAppAttempt(); ApplicationAttemptId attemptId1 = attempt.getAppAttemptId(); rm1.waitForState(attemptId1, RMAppAttemptState.ALLOCATED); - Assert.assertEquals(1, appState.getAttemptCount()); + assertEquals(1, appState.getAttemptCount()); ApplicationAttemptStateData attemptState = appState.getAttempt(attemptId1); - Assert.assertNotNull(attemptState); - Assert.assertEquals(BuilderUtils.newContainerId(attemptId1, 1), + assertNotNull(attemptState); + assertEquals(BuilderUtils.newContainerId(attemptId1, 1), attemptState.getMasterContainer().getId()); // Setting AMLivelinessMonitor interval to be 3 Secs. @@ -1305,25 +1346,28 @@ public void testRMRestartOnMaxAppAttempts() throws Exception { rm2.start(); // verify that maxAppAttempts is set to global value - Assert.assertEquals(2, + assertEquals(2, rm2.getRMContext().getRMApps().get(app2.getApplicationId()) .getMaxAppAttempts()); // app1 and app2 are loaded back, but app1 failed because it's // hitting max-retry. - Assert.assertEquals(2, rm2.getRMContext().getRMApps().size()); + assertEquals(2, rm2.getRMContext().getRMApps().size()); rm2.waitForState(app1.getApplicationId(), RMAppState.FAILED); rm2.waitForState(app2.getApplicationId(), RMAppState.ACCEPTED); // app1 failed state is saved in state store. app2 final saved state is not // determined yet. - Assert.assertEquals(RMAppState.FAILED, + assertEquals(RMAppState.FAILED, rmAppState.get(app1.getApplicationId()).getState()); - Assert.assertNull(rmAppState.get(app2.getApplicationId()).getState()); + assertNull(rmAppState.get(app2.getApplicationId()).getState()); } - @Test (timeout = 60000) - public void testRMRestartTimelineCollectorContext() throws Exception { + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testRMRestartTimelineCollectorContext(SchedulerType type) throws Exception { + initTestRMRestart(type); conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f); @@ -1352,9 +1396,9 @@ public void testRMRestartTimelineCollectorContext() throws Exception { .build()); // Check if app info has been saved. ApplicationStateData appState = rmAppState.get(app.getApplicationId()); - Assert.assertNotNull(appState); - Assert.assertEquals(0, appState.getAttemptCount()); - Assert.assertEquals(appState.getApplicationSubmissionContext() + assertNotNull(appState); + assertEquals(0, appState.getAttemptCount()); + assertEquals(appState.getApplicationSubmissionContext() .getApplicationId(), app.getApplicationSubmissionContext() .getApplicationId()); @@ -1372,13 +1416,13 @@ public void testRMRestartTimelineCollectorContext() throws Exception { // Restart RM. rm2 = createMockRM(conf, memStore); rm2.start(); - Assert.assertEquals(1, rm2.getRMContext().getRMApps().size()); + assertEquals(1, rm2.getRMContext().getRMApps().size()); rm2.waitForState(app.getApplicationId(), RMAppState.ACCEPTED); TimelineCollectorContext contextAfterRestart = rm2.getRMContext().getRMTimelineCollectorManager().get(appId). getTimelineEntityContext(); - Assert.assertEquals("Collector contexts for an app should be same " + - "across restarts", contextBeforeRestart, contextAfterRestart); + assertEquals(contextBeforeRestart, contextAfterRestart, + "Collector contexts for an app should be same across restarts"); } finally { conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, false); if (rm1 != null) { @@ -1390,9 +1434,12 @@ public void testRMRestartTimelineCollectorContext() throws Exception { } } - @Test (timeout = 60000) - public void testDelegationTokenRestoredInDelegationTokenRenewer() + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testDelegationTokenRestoredInDelegationTokenRenewer(SchedulerType type) throws Exception { + initTestRMRestart(type); conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2); conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos"); @@ -1449,10 +1496,10 @@ public void testDelegationTokenRestoredInDelegationTokenRenewer() // assert app info is saved ApplicationStateData appState = rmAppState.get(app.getApplicationId()); - Assert.assertNotNull(appState); + assertNotNull(appState); // assert delegation tokens exist in rm1 DelegationTokenRenewr - Assert.assertEquals(tokenSet, rm1.getRMContext() + assertEquals(tokenSet, rm1.getRMContext() .getDelegationTokenRenewer().getDelegationTokens()); // assert delegation tokens are saved @@ -1461,7 +1508,7 @@ public void testDelegationTokenRestoredInDelegationTokenRenewer() ByteBuffer securityTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength()); securityTokens.rewind(); - Assert.assertEquals(securityTokens, appState + assertEquals(securityTokens, appState .getApplicationSubmissionContext().getAMContainerSpec() .getTokens()); @@ -1474,7 +1521,7 @@ public void testDelegationTokenRestoredInDelegationTokenRenewer() waitForTokensToBeRenewed(rm2, tokenSet); // verify tokens are properly populated back to rm2 DelegationTokenRenewer - Assert.assertEquals(tokenSet, rm2.getRMContext() + assertEquals(tokenSet, rm2.getRMContext() .getDelegationTokenRenewer().getDelegationTokens()); } @@ -1493,8 +1540,11 @@ private void waitForTokensToBeRenewed(MockRM rm2, } } - @Test (timeout = 60000) - public void testAppAttemptTokensRestoredOnRMRestart() throws Exception { + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testAppAttemptTokensRestoredOnRMRestart(SchedulerType type) throws Exception { + initTestRMRestart(type); conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2); conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos"); @@ -1525,7 +1575,7 @@ public void testAppAttemptTokensRestoredOnRMRestart() throws Exception { // assert app info is saved ApplicationStateData appState = rmAppState.get(app1.getApplicationId()); - Assert.assertNotNull(appState); + assertNotNull(appState); // Allocate the AM nm1.nodeHeartbeat(true); @@ -1535,8 +1585,8 @@ public void testAppAttemptTokensRestoredOnRMRestart() throws Exception { // assert attempt info is saved ApplicationAttemptStateData attemptState = appState.getAttempt(attemptId1); - Assert.assertNotNull(attemptState); - Assert.assertEquals(BuilderUtils.newContainerId(attemptId1, 1), + assertNotNull(attemptState); + assertEquals(BuilderUtils.newContainerId(attemptId1, 1), attemptState.getMasterContainer().getId()); // the clientTokenMasterKey that are generated when @@ -1546,9 +1596,9 @@ public void testAppAttemptTokensRestoredOnRMRestart() throws Exception { // assert application credentials are saved Credentials savedCredentials = attemptState.getAppAttemptTokens(); - Assert.assertArrayEquals("client token master key not saved", - clientTokenMasterKey, savedCredentials.getSecretKey( - RMStateStore.AM_CLIENT_TOKEN_MASTER_KEY_NAME)); + assertArrayEquals(clientTokenMasterKey, savedCredentials.getSecretKey( + RMStateStore.AM_CLIENT_TOKEN_MASTER_KEY_NAME), + "client token master key not saved"); // start new RM MockRM rm2 = new TestSecurityMockRM(conf, memStore); @@ -1559,28 +1609,31 @@ public void testAppAttemptTokensRestoredOnRMRestart() throws Exception { RMAppAttempt loadedAttempt1 = loadedApp1.getRMAppAttempt(attemptId1); // assert loaded attempt recovered - Assert.assertNotNull(loadedAttempt1); + assertNotNull(loadedAttempt1); // assert client token master key is recovered back to api-versioned // client token master key - Assert.assertEquals("client token master key not restored", - attempt1.getClientTokenMasterKey(), - loadedAttempt1.getClientTokenMasterKey()); + assertEquals(attempt1.getClientTokenMasterKey(), + loadedAttempt1.getClientTokenMasterKey(), + "client token master key not restored"); // assert ClientTokenSecretManager also knows about the key - Assert.assertArrayEquals(clientTokenMasterKey, + assertArrayEquals(clientTokenMasterKey, rm2.getClientToAMTokenSecretManager().getMasterKey(attemptId1) .getEncoded()); // assert AMRMTokenSecretManager also knows about the AMRMToken password Token amrmToken = loadedAttempt1.getAMRMToken(); - Assert.assertArrayEquals(amrmToken.getPassword(), + assertArrayEquals(amrmToken.getPassword(), rm2.getRMContext().getAMRMTokenSecretManager().retrievePassword( amrmToken.decodeIdentifier())); } - @Test (timeout = 60000) - public void testRMDelegationTokenRestoredOnRMRestart() throws Exception { + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testRMDelegationTokenRestoredOnRMRestart(SchedulerType type) throws Exception { + initTestRMRestart(type); conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2); conf.set( CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, @@ -1635,21 +1688,21 @@ public void testRMDelegationTokenRestoredOnRMRestart() throws Exception { // assert app info is saved ApplicationStateData appState = rmAppState.get(app.getApplicationId()); - Assert.assertNotNull(appState); + assertNotNull(appState); // assert all master keys are saved Set allKeysRM1 = rm1.getRMContext() .getRMDelegationTokenSecretManager().getAllMasterKeys(); - Assert.assertEquals(allKeysRM1, rmDTMasterKeyState); + assertEquals(allKeysRM1, rmDTMasterKeyState); // assert all tokens are saved Map allTokensRM1 = rm1.getRMContext().getRMDelegationTokenSecretManager().getAllTokens(); - Assert.assertEquals(tokenIdentSet, allTokensRM1.keySet()); - Assert.assertEquals(allTokensRM1, rmDTState); + assertEquals(tokenIdentSet, allTokensRM1.keySet()); + assertEquals(allTokensRM1, rmDTState); // assert sequence number is saved - Assert.assertEquals(rm1.getRMContext().getRMDelegationTokenSecretManager() + assertEquals(rm1.getRMContext().getRMDelegationTokenSecretManager() .getLatestDTSequenceNumber(), rmState.getRMDTSecretManagerState() .getDTSequenceNumber()); @@ -1669,13 +1722,13 @@ public void testRMDelegationTokenRestoredOnRMRestart() throws Exception { rm1.getRMContext().getRMDelegationTokenSecretManager().cancelToken(token2, UserGroupInformation.getCurrentUser().getUserName()); } catch(Exception e) { - Assert.fail(); + fail(); } // Assert the token which has the latest delegationTokenSequenceNumber is removed - Assert.assertEquals(rm1.getRMContext().getRMDelegationTokenSecretManager() + assertEquals(rm1.getRMContext().getRMDelegationTokenSecretManager() .getLatestDTSequenceNumber(), dtId2.getSequenceNumber()); - Assert.assertFalse(rmDTState.containsKey(dtId2)); + assertFalse(rmDTState.containsKey(dtId2)); // start new RM MockRM rm2 = new TestSecurityMockRM(conf, memStore); @@ -1684,14 +1737,14 @@ public void testRMDelegationTokenRestoredOnRMRestart() throws Exception { // assert master keys and tokens are populated back to DTSecretManager Map allTokensRM2 = rm2.getRMContext().getRMDelegationTokenSecretManager().getAllTokens(); - Assert.assertEquals(allTokensRM2.keySet(), allTokensRM1.keySet()); + assertEquals(allTokensRM2.keySet(), allTokensRM1.keySet()); // rm2 has its own master keys when it starts, we use containsAll here - Assert.assertTrue(rm2.getRMContext().getRMDelegationTokenSecretManager() + assertTrue(rm2.getRMContext().getRMDelegationTokenSecretManager() .getAllMasterKeys().containsAll(allKeysRM1)); // assert sequenceNumber is properly recovered, // even though the token which has max sequenceNumber is not stored - Assert.assertEquals(rm1.getRMContext().getRMDelegationTokenSecretManager() + assertEquals(rm1.getRMContext().getRMDelegationTokenSecretManager() .getLatestDTSequenceNumber(), rm2.getRMContext() .getRMDelegationTokenSecretManager().getLatestDTSequenceNumber()); @@ -1704,39 +1757,42 @@ public void testRMDelegationTokenRestoredOnRMRestart() throws Exception { rm2.getRMContext().getRMDelegationTokenSecretManager().renewToken( token1, "renewer1"); } catch(Exception e) { - Assert.fail(); + fail(); } allTokensRM2 = rm2.getRMContext().getRMDelegationTokenSecretManager() .getAllTokens(); Long renewDateAfterRenew = allTokensRM2.get(dtId1); // assert token is renewed - Assert.assertTrue(renewDateAfterRenew > renewDateBeforeRenew); + assertTrue(renewDateAfterRenew > renewDateBeforeRenew); // assert new token is added into state store - Assert.assertTrue(rmDTState.containsValue(renewDateAfterRenew)); + assertTrue(rmDTState.containsValue(renewDateAfterRenew)); // assert old token is removed from state store - Assert.assertFalse(rmDTState.containsValue(renewDateBeforeRenew)); + assertFalse(rmDTState.containsValue(renewDateBeforeRenew)); try{ rm2.getRMContext().getRMDelegationTokenSecretManager().cancelToken(token1, UserGroupInformation.getCurrentUser().getUserName()); } catch(Exception e) { - Assert.fail(); + fail(); } // assert token is removed from state after its cancelled allTokensRM2 = rm2.getRMContext().getRMDelegationTokenSecretManager() .getAllTokens(); - Assert.assertFalse(allTokensRM2.containsKey(dtId1)); - Assert.assertFalse(rmDTState.containsKey(dtId1)); + assertFalse(allTokensRM2.containsKey(dtId1)); + assertFalse(rmDTState.containsKey(dtId1)); } // This is to test submit an application to the new RM with the old delegation // token got from previous RM. - @Test (timeout = 60000) - public void testAppSubmissionWithOldDelegationTokenAfterRMRestart() + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testAppSubmissionWithOldDelegationTokenAfterRMRestart(SchedulerType type) throws Exception { + initTestRMRestart(type); conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2); conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos"); @@ -1775,8 +1831,11 @@ public void testAppSubmissionWithOldDelegationTokenAfterRMRestart() rm2.waitForState(app.getApplicationId(), RMAppState.ACCEPTED); } - @Test (timeout = 60000) - public void testRMStateStoreDispatcherDrainedOnRMStop() throws Exception { + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testRMStateStoreDispatcherDrainedOnRMStop(SchedulerType type) throws Exception { + initTestRMRestart(type); MemoryRMStateStore memStore = new MemoryRMStateStore() { volatile boolean wait = true; @Override @@ -1832,7 +1891,7 @@ protected void handleStoreEvent(RMStateStoreEvent event) { // Nothing exist in state store before stop is called. Map rmAppState = memStore.getState().getApplicationState(); - Assert.assertTrue(rmAppState.size() == 0); + assertTrue(rmAppState.size() == 0); // stop rm rm1.stop(); @@ -1841,17 +1900,20 @@ protected void handleStoreEvent(RMStateStoreEvent event) { // request on dispatcher. for (RMApp app : appList) { ApplicationStateData appState = rmAppState.get(app.getApplicationId()); - Assert.assertNotNull(appState); - Assert.assertEquals(0, appState.getAttemptCount()); - Assert.assertEquals(appState.getApplicationSubmissionContext() + assertNotNull(appState); + assertEquals(0, appState.getAttemptCount()); + assertEquals(appState.getApplicationSubmissionContext() .getApplicationId(), app.getApplicationSubmissionContext() .getApplicationId()); } - Assert.assertTrue(rmAppState.size() == NUM_APPS); + assertTrue(rmAppState.size() == NUM_APPS); } - @Test (timeout = 60000) - public void testFinishedAppRemovalAfterRMRestart() throws Exception { + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testFinishedAppRemovalAfterRMRestart(SchedulerType type) throws Exception { + initTestRMRestart(type); conf.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, 1); // start RM @@ -1878,7 +1940,7 @@ public void testFinishedAppRemovalAfterRMRestart() throws Exception { rmState.getApplicationState(); // app0 exits in both state store and rmContext - Assert.assertEquals(RMAppState.FINISHED, + assertEquals(RMAppState.FINISHED, rmAppState.get(app0.getApplicationId()).getState()); rm2.waitForState(app0.getApplicationId(), RMAppState.FINISHED); @@ -1889,14 +1951,17 @@ public void testFinishedAppRemovalAfterRMRestart() throws Exception { rm2.drainEvents(); // the first app0 get kicked out from both rmContext and state store - Assert.assertNull(rm2.getRMContext().getRMApps() + assertNull(rm2.getRMContext().getRMApps() .get(app0.getApplicationId())); - Assert.assertNull(rmAppState.get(app0.getApplicationId())); + assertNull(rmAppState.get(app0.getApplicationId())); } // This is to test RM does not get hang on shutdown. - @Test (timeout = 10000) - public void testRMShutdown() throws Exception { + @Timeout(10) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testRMShutdown(SchedulerType type) throws Exception { + initTestRMRestart(type); MemoryRMStateStore memStore = new MockMemoryRMStateStore() { @Override public synchronized void checkVersion() @@ -1910,18 +1975,21 @@ public synchronized void checkVersion() try { rm1 = createMockRM(conf, memStore); rm1.start(); - Assert.fail(); + fail(); } catch (Exception e) { - Assert.assertTrue(e.getMessage().contains("Invalid version.")); + assertTrue(e.getMessage().contains("Invalid version.")); } - Assert.assertTrue(rm1.getServiceState() == STATE.STOPPED); + assertTrue(rm1.getServiceState() == STATE.STOPPED); } // This is to test Killing application should be able to wait until app // reaches killed state and also check that attempt state is saved before app // state is saved. - @Test (timeout = 60000) - public void testClientRetryOnKillingApplication() throws Exception { + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testClientRetryOnKillingApplication(SchedulerType type) throws Exception { + initTestRMRestart(type); MemoryRMStateStore memStore = new TestMemoryRMStateStore(); memStore.init(conf); @@ -1958,20 +2026,23 @@ public void testClientRetryOnKillingApplication() throws Exception { } // we expect at least 2 calls for killApp as the first killApp always return // false. - Assert.assertTrue(count >= 1); + assertTrue(count >= 1); rm1.waitForState(am1.getApplicationAttemptId(), RMAppAttemptState.KILLED); rm1.waitForState(app1.getApplicationId(), RMAppState.KILLED); // count = 1 on storing RMApp launchTime // count = 2 on storing attempt state on kill // count = 3 on storing app state on kill - Assert.assertEquals(2, ((TestMemoryRMStateStore) memStore).updateAttempt); - Assert.assertEquals(3, ((TestMemoryRMStateStore) memStore).updateApp); + assertEquals(2, ((TestMemoryRMStateStore) memStore).updateAttempt); + assertEquals(3, ((TestMemoryRMStateStore) memStore).updateApp); } // Test Application that fails on submission is saved in state store. - @Test (timeout = 20000) - public void testAppFailedOnSubmissionSavedInStateStore() throws Exception { + @Timeout(20) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testAppFailedOnSubmissionSavedInStateStore(SchedulerType type) throws Exception { + initTestRMRestart(type); conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos"); UserGroupInformation.setConfiguration(conf); @@ -2004,14 +2075,14 @@ protected DelegationTokenRenewer createDelegationTokenRenewer() { .withAppType("MAPREDUCE") .withWaitForAppAcceptedState(false) .build()); - Assert.fail(); + fail(); } catch (Exception e) { } app1 = rm1.getRMContext().getRMApps().values().iterator().next(); rm1.waitForState(app1.getApplicationId(), RMAppState.FAILED); // Check app staet is saved in state store. - Assert.assertEquals(RMAppState.FAILED, memStore.getState() + assertEquals(RMAppState.FAILED, memStore.getState() .getApplicationState().get(app1.getApplicationId()).getState()); MockRM rm2 = new TestSecurityMockRM(conf, memStore); @@ -2020,8 +2091,11 @@ protected DelegationTokenRenewer createDelegationTokenRenewer() { rm2.waitForState(app1.getApplicationId(), RMAppState.FAILED); } - @Test (timeout = 20000) - public void testAppRecoveredInOrderOnRMRestart() throws Exception { + @Timeout(20) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testAppRecoveredInOrderOnRMRestart(SchedulerType type) throws Exception { + initTestRMRestart(type); MemoryRMStateStore memStore = new MemoryRMStateStore(); memStore.init(conf); @@ -2057,8 +2131,8 @@ public TestRMAppManager(RMContext context, YarnScheduler scheduler, protected void recoverApplication(ApplicationStateData appState, RMState rmState) throws Exception { // check application is recovered in order. - Assert.assertTrue(rmState.getApplicationState().size() > 0); - Assert.assertTrue(appState.getApplicationSubmissionContext() + assertTrue(rmState.getApplicationState().size() > 0); + assertTrue(appState.getApplicationSubmissionContext() .getApplicationId().compareTo(prevId) > 0); prevId = appState.getApplicationSubmissionContext().getApplicationId(); @@ -2073,8 +2147,11 @@ protected void recoverApplication(ApplicationStateData appState, } @SuppressWarnings("resource") - @Test (timeout = 60000) - public void testQueueMetricsOnRMRestart() throws Exception { + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testQueueMetricsOnRMRestart(SchedulerType type) throws Exception { + initTestRMRestart(type); conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS); // start RM @@ -2185,18 +2262,21 @@ private void resetQueueMetrics(QueueMetrics qm) { private void assertQueueMetrics(QueueMetrics qm, int appsSubmitted, int appsPending, int appsRunning, int appsCompleted) { - Assert.assertEquals(appsSubmitted + appsSubmittedCarryOn, + assertEquals(appsSubmitted + appsSubmittedCarryOn, qm.getAppsSubmitted()); - Assert.assertEquals(appsPending + appsPendingCarryOn, + assertEquals(appsPending + appsPendingCarryOn, qm.getAppsPending()); - Assert.assertEquals(appsRunning + appsRunningCarryOn, + assertEquals(appsRunning + appsRunningCarryOn, qm.getAppsRunning()); - Assert.assertEquals(appsCompleted + appsCompletedCarryOn, + assertEquals(appsCompleted + appsCompletedCarryOn, qm.getAppsCompleted()); } - @Test (timeout = 60000) - public void testDecommissionedNMsMetricsOnRMRestart() throws Exception { + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testDecommissionedNMsMetricsOnRMRestart(SchedulerType type) throws Exception { + initTestRMRestart(type); conf.set(YarnConfiguration.RM_NODES_EXCLUDE_FILE_PATH, hostFile.getAbsolutePath()); writeToHostsFile(""); @@ -2209,9 +2289,8 @@ public void testDecommissionedNMsMetricsOnRMRestart() throws Exception { Resource expectedCapability = Resource.newInstance(nm1.getMemory(), nm1.getvCores()); String expectedVersion = nm1.getVersion(); - Assert - .assertEquals(0, - ClusterMetrics.getMetrics().getNumDecommisionedNMs()); + assertEquals(0, + ClusterMetrics.getMetrics().getNumDecommisionedNMs()); String ip = NetUtils.normalizeHostName("localhost"); // Add 2 hosts to exclude list. writeToHostsFile("host2", ip); @@ -2219,31 +2298,26 @@ public void testDecommissionedNMsMetricsOnRMRestart() throws Exception { // refresh nodes rm1.getNodesListManager().refreshNodes(conf); NodeHeartbeatResponse nodeHeartbeat = nm1.nodeHeartbeat(true); - Assert - .assertTrue( - NodeAction.SHUTDOWN.equals(nodeHeartbeat.getNodeAction())); + assertTrue(NodeAction.SHUTDOWN.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"); rm1.drainEvents(); - Assert - .assertEquals(2, - ClusterMetrics.getMetrics().getNumDecommisionedNMs()); + assertEquals(2, + ClusterMetrics.getMetrics().getNumDecommisionedNMs()); verifyNodesAfterDecom(rm1, 2, expectedCapability, expectedVersion); rm1.stop(); rm1 = null; - Assert - .assertEquals(0, - ClusterMetrics.getMetrics().getNumDecommisionedNMs()); + assertEquals(0, + ClusterMetrics.getMetrics().getNumDecommisionedNMs()); // restart RM. rm2 = new MockRM(conf); rm2.start(); rm2.drainEvents(); - Assert - .assertEquals(2, - ClusterMetrics.getMetrics().getNumDecommisionedNMs()); + assertEquals(2, + ClusterMetrics.getMetrics().getNumDecommisionedNMs()); verifyNodesAfterDecom(rm2, 2, Resource.newInstance(0, 0), "unknown"); } finally { if (rm1 != null) { @@ -2260,18 +2334,21 @@ private void verifyNodesAfterDecom(MockRM rm, int numNodes, String expectedVersion) { ConcurrentMap inactiveRMNodes = rm.getRMContext().getInactiveRMNodes(); - Assert.assertEquals(numNodes, inactiveRMNodes.size()); + assertEquals(numNodes, inactiveRMNodes.size()); for (RMNode rmNode : inactiveRMNodes.values()) { - Assert.assertEquals(expectedCapability, rmNode.getTotalCapability()); - Assert.assertEquals(expectedVersion, rmNode.getNodeManagerVersion()); + assertEquals(expectedCapability, rmNode.getTotalCapability()); + assertEquals(expectedVersion, rmNode.getNodeManagerVersion()); } } // Test Delegation token is renewed synchronously so that recover events // can be processed before any other external incoming events, specifically // the ContainerFinished event on NM re-registraton. - @Test (timeout = 20000) - public void testSynchronouslyRenewDTOnRecovery() throws Exception { + @Timeout(20) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testSynchronouslyRenewDTOnRecovery(SchedulerType type) throws Exception { + initTestRMRestart(type); conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2); conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos"); @@ -2433,8 +2510,11 @@ protected void doSecureLogin() throws IOException { // 2. Add/remove/replace labels to cluster and node lable and verify // 3. Start RM2 with store patch /tmp only // 4. Get cluster and node lobel, it should be present by recovering it - @Test(timeout = 20000) - public void testRMRestartRecoveringNodeLabelManager() throws Exception { + @Timeout(20) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testRMRestartRecoveringNodeLabelManager(SchedulerType type) throws Exception { + initTestRMRestart(type); // Initial FS node label store root dir to a random tmp dir File nodeLabelFsStoreDir = new File("target", this.getClass().getSimpleName() @@ -2492,12 +2572,12 @@ protected RMNodeLabelsManager createNodeLabelManager() { Thread.sleep(500); } - Assert.assertEquals(clusterNodeLabels.size(), nodeLabelManager + assertEquals(clusterNodeLabels.size(), nodeLabelManager .getClusterNodeLabelNames().size()); Map> nodeLabels = nodeLabelManager.getNodeLabels(); - Assert.assertEquals(1, nodeLabelManager.getNodeLabels().size()); - Assert.assertTrue(nodeLabels.get(n1).equals(toSet("y"))); + assertEquals(1, nodeLabelManager.getNodeLabels().size()); + assertTrue(nodeLabels.get(n1).equals(toSet("y"))); MockRM rm2 = new MockRM(conf, rm1.getRMStateStore()) { @Override @@ -2511,18 +2591,21 @@ protected RMNodeLabelsManager createNodeLabelManager() { rm2.start(); nodeLabelManager = rm2.getRMContext().getNodeLabelManager(); - Assert.assertEquals(clusterNodeLabels.size(), + assertEquals(clusterNodeLabels.size(), nodeLabelManager.getClusterNodeLabelNames().size()); nodeLabels = nodeLabelManager.getNodeLabels(); - Assert.assertEquals(1, nodeLabelManager.getNodeLabels().size()); - Assert.assertTrue(nodeLabels.get(n1).equals(toSet("y"))); + assertEquals(1, nodeLabelManager.getNodeLabels().size()); + assertTrue(nodeLabels.get(n1).equals(toSet("y"))); rm1.stop(); rm2.stop(); } - @Test(timeout = 60000) - public void testRMRestartFailAppAttempt() throws Exception { + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testRMRestartFailAppAttempt(SchedulerType type) throws Exception { + initTestRMRestart(type); conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS); int maxAttempt = @@ -2546,7 +2629,7 @@ public void testRMRestartFailAppAttempt() throws Exception { ApplicationId applicationId = app0.getApplicationId(); ApplicationAttemptId appAttemptId1 = app0.getCurrentAppAttempt().getAppAttemptId(); - Assert.assertEquals(1, appAttemptId1.getAttemptId()); + assertEquals(1, appAttemptId1.getAttemptId()); // fail the 1st app attempt. rm1.failApplicationAttempt(appAttemptId1); @@ -2556,7 +2639,7 @@ public void testRMRestartFailAppAttempt() throws Exception { ApplicationAttemptId appAttemptId2 = app0.getCurrentAppAttempt().getAppAttemptId(); - Assert.assertEquals(2, appAttemptId2.getAttemptId()); + assertEquals(2, appAttemptId2.getAttemptId()); rm1.waitForState(appAttemptId2, RMAppAttemptState.SCHEDULED); // restart rm @@ -2569,18 +2652,18 @@ public void testRMRestartFailAppAttempt() throws Exception { //Wait to make sure the loadedApp0 has the right number of attempts //TODO explore a better way than sleeping for a while (YARN-4929) Thread.sleep(1000); - Assert.assertEquals(2, loadedApp0.getAppAttempts().size()); + assertEquals(2, loadedApp0.getAppAttempts().size()); rm2.waitForState(appAttemptId2, RMAppAttemptState.SCHEDULED); appAttemptId2 = loadedApp0.getCurrentAppAttempt().getAppAttemptId(); - Assert.assertEquals(2, appAttemptId2.getAttemptId()); + assertEquals(2, appAttemptId2.getAttemptId()); // fail 2nd attempt rm2.failApplicationAttempt(appAttemptId2); rm2.waitForState(appAttemptId2, RMAppAttemptState.FAILED); rm2.waitForState(applicationId, RMAppState.FAILED); - Assert.assertEquals(maxAttempt, loadedApp0.getAppAttempts().size()); + assertEquals(maxAttempt, loadedApp0.getAppAttempts().size()); } private Set toSet(E... elements) { @@ -2588,8 +2671,11 @@ private Set toSet(E... elements) { return set; } - @Test(timeout = 20000) - public void testRMRestartNodeMapping() throws Exception { + @Timeout(20) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testRMRestartNodeMapping(SchedulerType type) throws Exception { + initTestRMRestart(type); // Initial FS node label store root dir to a random tmp dir File nodeLabelFsStoreDir = new File("target", this.getClass().getSimpleName() + "-testRMRestartNodeMapping"); @@ -2644,15 +2730,18 @@ protected RMNodeLabelsManager createNodeLabelManager() { nodeLabelManager = rm2.getRMContext().getNodeLabelManager(); Map> labelsToNodes = nodeLabelManager.getLabelsToNodes(toSet("x")); - Assert.assertEquals(1, + assertEquals(1, null == labelsToNodes.get("x") ? 0 : labelsToNodes.get("x").size()); } rm1.stop(); rm2.stop(); } - @Test(timeout = 120000) - public void testRMRestartAfterPreemption() throws Exception { + @Timeout(120) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testRMRestartAfterPreemption(SchedulerType type) throws Exception { + initTestRMRestart(type); conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2); if (!getSchedulerType().equals(SchedulerType.CAPACITY)) { return; @@ -2701,17 +2790,20 @@ public void testRMRestartAfterPreemption() throws Exception { try { rm2 = new MockRM(conf, memStore); rm2.start(); - Assert.assertTrue("RM start successfully", true); + assertTrue(true, "RM start successfully"); } catch (Exception e) { LOG.debug("Exception on start", e); - Assert.fail("RM should start with out any issue"); + fail("RM should start with out any issue"); } finally { rm1.stop(); } } - @Test(timeout = 60000) - public void testRMRestartOnMissingAttempts() throws Exception { + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testRMRestartOnMissingAttempts(SchedulerType type) throws Exception { + initTestRMRestart(type); conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 5); // create RM MockRM rm1 = createMockRM(conf); @@ -2744,22 +2836,22 @@ public void testRMRestartOnMissingAttempts() throws Exception { MockRM rm2 = createMockRM(conf, memStore); rm2.start(); - Assert.assertEquals(1, rm2.getRMContext().getRMApps().size()); + assertEquals(1, rm2.getRMContext().getRMApps().size()); RMApp recoveredApp0 = rm2.getRMContext().getRMApps().values() .iterator().next(); Map recoveredAppAttempts = recoveredApp0.getAppAttempts(); - Assert.assertEquals(3, recoveredAppAttempts.size()); - Assert.assertEquals(RMAppAttemptState.FAILED, + assertEquals(3, recoveredAppAttempts.size()); + assertEquals(RMAppAttemptState.FAILED, recoveredAppAttempts.get( am0.getApplicationAttemptId()).getAppAttemptState()); - Assert.assertEquals(RMAppAttemptState.FAILED, + assertEquals(RMAppAttemptState.FAILED, recoveredAppAttempts.get( am2.getApplicationAttemptId()).getAppAttemptState()); - Assert.assertEquals(RMAppAttemptState.LAUNCHED, + assertEquals(RMAppAttemptState.LAUNCHED, recoveredAppAttempts.get( am3.getApplicationAttemptId()).getAppAttemptState()); - Assert.assertEquals(5, ((RMAppImpl)app0).getNextAttemptId()); + assertEquals(5, ((RMAppImpl)app0).getNextAttemptId()); } private MockAM launchAndFailAM(RMApp app, MockRM rm, MockNM nm) @@ -2770,8 +2862,11 @@ private MockAM launchAndFailAM(RMApp app, MockRM rm, MockNM nm) return am; } - @Test(timeout = 60000) - public void testRMRestartAfterNodeLabelDisabled() throws Exception { + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testRMRestartAfterNodeLabelDisabled(SchedulerType type) throws Exception { + initTestRMRestart(type); if (getSchedulerType() != SchedulerType.CAPACITY) { return; } @@ -2828,7 +2923,7 @@ protected RMNodeLabelsManager createNodeLabelManager() { MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1); ContainerId amContainerId1 = ContainerId.newContainerId(am1.getApplicationAttemptId(), 1); - Assert.assertEquals("x", rm1.getRMContext().getScheduler(). + assertEquals("x", rm1.getRMContext().getScheduler(). getRMContainer(amContainerId1).getNodeLabelExpression()); finishApplicationMaster(app1, rm1, nm1, am1); @@ -2851,20 +2946,23 @@ protected RMNodeLabelsManager createNodeLabelManager() { try { rm2.start(); - Assert.assertTrue("RM start successfully", true); - Assert.assertEquals(1, rm2.getRMContext().getRMApps().size()); + assertTrue(true, "RM start successfully"); + assertEquals(1, rm2.getRMContext().getRMApps().size()); } catch (Exception e) { LOG.debug("Exception on start", e); - Assert.fail("RM should start without any issue"); + fail("RM should start without any issue"); } finally { rm1.stop(); rm2.stop(); } } - @Test(timeout = 20000) - public void testRMRestartAfterPriorityChangesInAllocatedResponse() + @Timeout(20) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testRMRestartAfterPriorityChangesInAllocatedResponse(SchedulerType type) throws Exception { + initTestRMRestart(type); conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos"); UserGroupInformation.setConfiguration(conf); @@ -2909,7 +3007,7 @@ public void testRMRestartAfterPriorityChangesInAllocatedResponse() 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); @@ -2919,7 +3017,7 @@ public void testRMRestartAfterPriorityChangesInAllocatedResponse() app1.getApplicationId(), appPriority2); AllocateResponse response2 = am1.allocate(allocateRequest); - Assert.assertEquals(appPriority2, response2.getApplicationPriority()); + assertEquals(appPriority2, response2.getApplicationPriority()); /* * Ensure tokensConf has been retained even after UPDATE_APP event in @@ -2930,7 +3028,7 @@ public void testRMRestartAfterPriorityChangesInAllocatedResponse() memStore.getState().getApplicationState(); ApplicationStateData appState = rmAppState.get(app1.getApplicationId()); - Assert.assertEquals(getTokensConf(), + assertEquals(getTokensConf(), appState.getApplicationSubmissionContext(). getAMContainerSpec().getTokensConf()); @@ -2939,7 +3037,7 @@ public void testRMRestartAfterPriorityChangesInAllocatedResponse() rm2.start(); AllocateResponse response3 = am1.allocate(allocateRequest); - Assert.assertEquals(appPriority2, response3.getApplicationPriority()); + assertEquals(appPriority2, response3.getApplicationPriority()); /* * Ensure tokensConf has been retained even after RECOVER event in @@ -2949,7 +3047,7 @@ public void testRMRestartAfterPriorityChangesInAllocatedResponse() memStore.getState().getApplicationState(); ApplicationStateData appStateNew = rmAppStateNew.get(app1.getApplicationId()); - Assert.assertEquals(getTokensConf(), + assertEquals(getTokensConf(), appStateNew.getApplicationSubmissionContext(). getAMContainerSpec().getTokensConf()); @@ -2957,8 +3055,11 @@ public void testRMRestartAfterPriorityChangesInAllocatedResponse() rm2.stop(); } - @Test(timeout = 20000) - public void testRMRestartAfterUpdateTrackingUrl() throws Exception { + @Timeout(20) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testRMRestartAfterUpdateTrackingUrl(SchedulerType type) throws Exception { + initTestRMRestart(type); MockRM rm = new MockRM(conf); rm.start(); @@ -2980,21 +3081,21 @@ public void testRMRestartAfterUpdateTrackingUrl() throws Exception { am1.allocate(allocateRequest); // Check in-memory and stored tracking url - Assert.assertEquals(newTrackingUrl, rm.getRMContext().getRMApps().get( + assertEquals(newTrackingUrl, rm.getRMContext().getRMApps().get( app1.getApplicationId()).getOriginalTrackingUrl()); - Assert.assertEquals(newTrackingUrl, rm.getRMContext().getRMApps().get( + assertEquals(newTrackingUrl, rm.getRMContext().getRMApps().get( app1.getApplicationId()).getCurrentAppAttempt() .getOriginalTrackingUrl()); - Assert.assertEquals(newTrackingUrl, memStore.getState() + assertEquals(newTrackingUrl, memStore.getState() .getApplicationState().get(app1.getApplicationId()) .getAttempt(attempt1.getAppAttemptId()).getFinalTrackingUrl()); // Start new RM, should recover updated tracking url MockRM rm2 = new MockRM(conf, memStore); rm2.start(); - Assert.assertEquals(newTrackingUrl, rm.getRMContext().getRMApps().get( + assertEquals(newTrackingUrl, rm.getRMContext().getRMApps().get( app1.getApplicationId()).getOriginalTrackingUrl()); - Assert.assertEquals(newTrackingUrl, rm.getRMContext().getRMApps().get( + assertEquals(newTrackingUrl, rm.getRMContext().getRMApps().get( app1.getApplicationId()).getCurrentAppAttempt() .getOriginalTrackingUrl()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java index 76cf0db9a02b5..beb6b1915cbee 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java @@ -86,11 +86,11 @@ import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import org.slf4j.event.Level; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; import java.io.File; import java.io.IOException; @@ -110,7 +110,13 @@ .capacity.TestCapacitySchedulerAutoCreatedQueueBase.USER1; import static org.apache.hadoop.yarn.server.resourcemanager.webapp .RMWebServices.DEFAULT_QUEUE; -import static org.junit.Assert.*; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; @@ -123,11 +129,11 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase MockRM rm1 = null; MockRM rm2 = null; - public TestWorkPreservingRMRestart(SchedulerType type) throws IOException { - super(type); + public void initTestWorkPreservingRMRestart(SchedulerType type) throws IOException { + initParameterizedSchedulerTestBase(type); + setup(); } - @Before public void setup() throws UnknownHostException { GenericTestUtils.setRootLogLevel(Level.DEBUG); conf = getConf(); @@ -139,7 +145,7 @@ public void setup() throws UnknownHostException { DefaultMetricsSystem.setMiniClusterMode(true); } - @After + @AfterEach public void tearDown() { if (rm1 != null) { rm1.stop(); @@ -158,8 +164,11 @@ public void tearDown() { // Test Strategy: send 3 container recovery reports(AMContainer, running // container, completed container) on NM re-registration, check the states of // SchedulerAttempt, SchedulerNode etc. are updated accordingly. - @Test(timeout = 20000) - public void testSchedulerRecovery() throws Exception { + @Timeout(20) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testSchedulerRecovery(SchedulerType type) throws Exception { + initTestWorkPreservingRMRestart(type); conf.setBoolean(CapacitySchedulerConfiguration.ENABLE_USER_METRICS, true); conf.set(CapacitySchedulerConfiguration.RESOURCE_CALCULATOR_CLASS, DominantResourceCalculator.class.getName()); @@ -218,14 +227,13 @@ public void testSchedulerRecovery() throws Exception { AbstractYarnScheduler scheduler = (AbstractYarnScheduler) rm2.getResourceScheduler(); SchedulerNode schedulerNode1 = scheduler.getSchedulerNode(nm1.getNodeId()); + assertTrue(schedulerNode1 + .toString().contains(schedulerNode1.getUnallocatedResource().toString()), + "SchedulerNode#toString is not in expected format"); assertTrue( - "SchedulerNode#toString is not in expected format", - schedulerNode1 - .toString().contains(schedulerNode1.getUnallocatedResource().toString())); - assertTrue( - "SchedulerNode#toString is not in expected format", schedulerNode1 - .toString().contains(schedulerNode1.getAllocatedResource().toString())); + .toString().contains(schedulerNode1.getAllocatedResource().toString()), + "SchedulerNode#toString is not in expected format"); // ********* check scheduler node state.******* // 2 running containers. @@ -322,8 +330,11 @@ private Configuration getSchedulerDynamicConfiguration() throws IOException { // 5. Check if all running containers are recovered, // 6. Verify the scheduler state like attempt info, // 7. Verify the queue/user metrics for the dynamic reservable queue. - @Test(timeout = 30000) - public void testDynamicQueueRecovery() throws Exception { + @Timeout(30) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testDynamicQueueRecovery(SchedulerType type) throws Exception { + initTestWorkPreservingRMRestart(type); conf.setBoolean(CapacitySchedulerConfiguration.ENABLE_USER_METRICS, true); conf.set(CapacitySchedulerConfiguration.RESOURCE_CALCULATOR_CLASS, DominantResourceCalculator.class.getName()); @@ -512,7 +523,7 @@ private void checkFSQueue(ResourceManager rm, } retry++; if (retry > 30) { - Assert.fail("Apps are not scheduled within assumed timeout"); + fail("Apps are not scheduled within assumed timeout"); } } @@ -622,8 +633,11 @@ private void setupQueueConfigurationChildOfB(CapacitySchedulerConfiguration conf // 1. submit an app to default queue and let it finish // 2. restart rm with no default queue // 3. getApplicationReport call should succeed (with no NPE) - @Test (timeout = 30000) - public void testRMRestartWithRemovedQueue() throws Exception{ + @Timeout(30) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testRMRestartWithRemovedQueue(SchedulerType type) throws Exception { + initTestWorkPreservingRMRestart(type); conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true); conf.set(YarnConfiguration.YARN_ADMIN_ACL, ""); rm1 = new MockRM(conf); @@ -659,7 +673,7 @@ public ApplicationReport run() throws Exception { return rm2.getApplicationReport(app1.getApplicationId()); } }); - Assert.assertNotNull(report); + assertNotNull(report); } // Test CS recovery with multi-level queues and multi-users: @@ -673,8 +687,11 @@ public ApplicationReport run() throws Exception { // 8. nm2 re-syncs back containers belong to user2. // 9. Assert the parent queue and 2 leaf queues state and the metrics. // 10. Assert each user's consumption inside the queue. - @Test (timeout = 30000) - public void testCapacitySchedulerRecovery() throws Exception { + @Timeout(30) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testCapacitySchedulerRecovery(SchedulerType type) throws Exception { + initTestWorkPreservingRMRestart(type); if (getSchedulerType() != SchedulerType.CAPACITY) { return; } @@ -852,7 +869,7 @@ private void verifyAppRecoveryWithWrongQueueConfig( MockRM rm = new MockRM(csConf, memStore2); try { rm.start(); - Assert.fail("QueueException must have been thrown"); + fail("QueueException must have been thrown"); } catch (QueueInvalidException e) { } finally { rm.close(); @@ -866,8 +883,11 @@ private void verifyAppRecoveryWithWrongQueueConfig( // fail fast config as false and once with fail fast as true. //3. Verify that app was killed if fail fast is false. //4. Verify that QueueException was thrown if fail fast is true. - @Test (timeout = 30000) - public void testCapacityLeafQueueBecomesParentOnRecovery() throws Exception { + @Timeout(30) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testCapacityLeafQueueBecomesParentOnRecovery(SchedulerType type) throws Exception { + initTestWorkPreservingRMRestart(type); if (getSchedulerType() != SchedulerType.CAPACITY) { return; } @@ -922,8 +942,11 @@ public void testCapacityLeafQueueBecomesParentOnRecovery() throws Exception { // false and once with fail fast as true. //3. Verify that app was killed if fail fast is false. //4. Verify that QueueException was thrown if fail fast is true. - @Test (timeout = 30000) - public void testCapacitySchedulerQueueRemovedRecovery() throws Exception { + @Timeout(30) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testCapacitySchedulerQueueRemovedRecovery(SchedulerType type) throws Exception { + initTestWorkPreservingRMRestart(type); if (getSchedulerType() != SchedulerType.CAPACITY) { return; } @@ -1016,8 +1039,11 @@ private void checkParentQueue(ParentQueue parentQueue, int numContainers, // Test RM shuts down, in the meanwhile, AM fails. Restarted RM scheduler // should not recover the containers that belong to the failed AM. - @Test(timeout = 20000) - public void testAMfailedBetweenRMRestart() throws Exception { + @Timeout(20) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testAMfailedBetweenRMRestart(SchedulerType type) throws Exception { + initTestWorkPreservingRMRestart(type); conf.setLong(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_SCHEDULING_WAIT_MS, 0); rm1 = new MockRM(conf); rm1.start(); @@ -1068,8 +1094,11 @@ public void testAMfailedBetweenRMRestart() throws Exception { // Apps already completed before RM restart. Restarted RM scheduler should not // recover containers for completed apps. - @Test(timeout = 20000) - public void testContainersNotRecoveredForCompletedApps() throws Exception { + @Timeout(20) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testContainersNotRecoveredForCompletedApps(SchedulerType type) throws Exception { + initTestWorkPreservingRMRestart(type); rm1 = new MockRM(conf); rm1.start(); MockMemoryRMStateStore memStore = @@ -1105,8 +1134,11 @@ public void testContainersNotRecoveredForCompletedApps() throws Exception { assertNull(scheduler.getRMContainer(completedContainer.getContainerId())); } - @Test (timeout = 600000) - public void testAppReregisterOnRMWorkPreservingRestart() throws Exception { + @Timeout(600) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testAppReregisterOnRMWorkPreservingRestart(SchedulerType type) throws Exception { + initTestWorkPreservingRMRestart(type); conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1); // start RM @@ -1136,9 +1168,12 @@ public void testAppReregisterOnRMWorkPreservingRestart() throws Exception { rm2.waitForState(app0.getApplicationId(), RMAppState.RUNNING); rm2.waitForState(am0.getApplicationAttemptId(), RMAppAttemptState.RUNNING); } - - @Test (timeout = 30000) - public void testAMContainerStatusWithRMRestart() throws Exception { + + @Timeout(30) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testAMContainerStatusWithRMRestart(SchedulerType type) throws Exception { + initTestWorkPreservingRMRestart(type); rm1 = new MockRM(conf); rm1.start(); MockNM nm1 = @@ -1150,7 +1185,7 @@ public void testAMContainerStatusWithRMRestart() throws Exception { RMAppAttempt attempt0 = app1_1.getCurrentAppAttempt(); YarnScheduler scheduler = rm1.getResourceScheduler(); - Assert.assertTrue(scheduler.getRMContainer( + assertTrue(scheduler.getRMContainer( attempt0.getMasterContainer().getId()).isAMContainer()); // Re-start RM @@ -1166,12 +1201,15 @@ public void testAMContainerStatusWithRMRestart() throws Exception { waitForNumContainersToRecover(2, rm2, am1_1.getApplicationAttemptId()); scheduler = rm2.getResourceScheduler(); - Assert.assertTrue(scheduler.getRMContainer( + assertTrue(scheduler.getRMContainer( attempt0.getMasterContainer().getId()).isAMContainer()); } - @Test (timeout = 20000) - public void testRecoverSchedulerAppAndAttemptSynchronously() throws Exception { + @Timeout(20) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testRecoverSchedulerAppAndAttemptSynchronously(SchedulerType type) throws Exception { + initTestWorkPreservingRMRestart(type); // start RM rm1 = new MockRM(conf); rm1.start(); @@ -1187,7 +1225,7 @@ public void testRecoverSchedulerAppAndAttemptSynchronously() throws Exception { rm2.start(); nm1.setResourceTrackerService(rm2.getResourceTrackerService()); // scheduler app/attempt is immediately available after RM is re-started. - Assert.assertNotNull(rm2.getResourceScheduler().getSchedulerAppInfo( + assertNotNull(rm2.getResourceScheduler().getSchedulerAppInfo( am0.getApplicationAttemptId())); // getTransferredContainers should not throw NPE. @@ -1202,8 +1240,11 @@ public void testRecoverSchedulerAppAndAttemptSynchronously() throws Exception { // Test if RM on recovery receives the container release request from AM // before it receives the container status reported by NM for recovery. this // container should not be recovered. - @Test (timeout = 50000) - public void testReleasedContainerNotRecovered() throws Exception { + @Timeout(50) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testReleasedContainerNotRecovered(SchedulerType type) throws Exception { + initTestWorkPreservingRMRestart(type); rm1 = new MockRM(conf); MockNM nm1 = new MockNM("h1:1234", 15120, rm1.getResourceTrackerService()); nm1.registerNode(); @@ -1295,9 +1336,12 @@ public static void waitForNumContainersToRecover(int num, MockRM rm, } } - @Test (timeout = 20000) - public void testNewContainersNotAllocatedDuringSchedulerRecovery() + @Timeout(20) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testNewContainersNotAllocatedDuringSchedulerRecovery(SchedulerType type) throws Exception { + initTestWorkPreservingRMRestart(type); conf.setLong( YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_SCHEDULING_WAIT_MS, 4000); rm1 = new MockRM(conf); @@ -1332,7 +1376,7 @@ public void testNewContainersNotAllocatedDuringSchedulerRecovery() containers.addAll(am1.allocate(new ArrayList(), new ArrayList()).getAllocatedContainers()); // container is not allocated during scheduling recovery. - Assert.assertTrue(containers.isEmpty()); + assertTrue(containers.isEmpty()); clock.setTime(startTime + 8000); nm1.nodeHeartbeat(true); @@ -1348,9 +1392,12 @@ public void testNewContainersNotAllocatedDuringSchedulerRecovery() * Testing to confirm that retried finishApplicationMaster() doesn't throw * InvalidApplicationMasterRequest before and after RM restart. */ - @Test (timeout = 20000) - public void testRetriedFinishApplicationMasterRequest() + @Timeout(20) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testRetriedFinishApplicationMasterRequest(SchedulerType type) throws Exception { + initTestWorkPreservingRMRestart(type); conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1); // start RM rm1 = new MockRM(conf); @@ -1383,8 +1430,11 @@ public void testRetriedFinishApplicationMasterRequest() am0.unregisterAppAttempt(false); } - @Test (timeout = 30000) - public void testAppFailedToRenewTokenOnRecovery() throws Exception { + @Timeout(30) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testAppFailedToRenewTokenOnRecovery(SchedulerType type) throws Exception { + initTestWorkPreservingRMRestart(type); conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos"); conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1); @@ -1432,8 +1482,11 @@ public void addApplicationSync(ApplicationId applicationId, * Test validateAndCreateResourceRequest fails on recovery, app should ignore * this Exception and continue */ - @Test (timeout = 30000) - public void testAppFailToValidateResourceRequestOnRecovery() throws Exception{ + @Timeout(30) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testAppFailToValidateResourceRequestOnRecovery(SchedulerType type) throws Exception { + initTestWorkPreservingRMRestart(type); rm1 = new MockRM(conf); rm1.start(); MockNM nm1 = @@ -1452,8 +1505,12 @@ public void testAppFailToValidateResourceRequestOnRecovery() throws Exception{ rm2.start(); } - @Test(timeout = 20000) - public void testContainerCompleteMsgNotLostAfterAMFailedAndRMRestart() throws Exception { + @Timeout(20) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testContainerCompleteMsgNotLostAfterAMFailedAndRMRestart(SchedulerType type) + throws Exception { + initTestWorkPreservingRMRestart(type); rm1 = new MockRM(conf); rm1.start(); @@ -1516,8 +1573,11 @@ public void testContainerCompleteMsgNotLostAfterAMFailedAndRMRestart() throws Ex // Test that if application state was saved, but attempt state was not saved. // RM should start correctly. - @Test (timeout = 20000) - public void testAppStateSavedButAttemptStateNotSaved() throws Exception { + @Timeout(20) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testAppStateSavedButAttemptStateNotSaved(SchedulerType type) throws Exception { + initTestWorkPreservingRMRestart(type); MockMemoryRMStateStore memStore = new MockMemoryRMStateStore() { @Override public synchronized void updateApplicationAttemptStateInternal( ApplicationAttemptId appAttemptId, @@ -1555,8 +1615,11 @@ public void testAppStateSavedButAttemptStateNotSaved() throws Exception { assertEquals(RMAppAttemptState.FINISHED, recoveredApp1.getCurrentAppAttempt().getState()); } - @Test(timeout = 600000) - public void testUAMRecoveryOnRMWorkPreservingRestart() throws Exception { + @Timeout(600) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testUAMRecoveryOnRMWorkPreservingRestart(SchedulerType type) throws Exception { + initTestWorkPreservingRMRestart(type); conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1); // start RM @@ -1635,18 +1698,18 @@ public void testUAMRecoveryOnRMWorkPreservingRestart() throws Exception { schedulerApps.get(recoveredApp.getApplicationId()); SchedulerApplicationAttempt schedulerAttempt = schedulerApp.getCurrentAppAttempt(); - Assert.assertEquals(numContainers, + assertEquals(numContainers, schedulerAttempt.getLiveContainers().size()); // Check if UAM is able to heart beat - Assert.assertNotNull(am0.doHeartbeat()); + assertNotNull(am0.doHeartbeat()); assertUnmanagedAMQueueMetrics(qm2, 1, 0, 1, 0); // Complete the UAM am0.unregisterAppAttempt(false); rm2.waitForState(am0.getApplicationAttemptId(), RMAppAttemptState.FINISHED); rm2.waitForState(app0.getApplicationId(), RMAppState.FINISHED); - Assert.assertEquals(FinalApplicationStatus.SUCCEEDED, + assertEquals(FinalApplicationStatus.SUCCEEDED, recoveredApp.getFinalApplicationStatus()); assertUnmanagedAMQueueMetrics(qm2, 1, 0, 0, 1); @@ -1655,7 +1718,7 @@ public void testUAMRecoveryOnRMWorkPreservingRestart() throws Exception { rm3.start(); recoveredApp = rm3.getRMContext().getRMApps().get(app0.getApplicationId()); QueueMetrics qm3 = rm3.getResourceScheduler().getRootQueueMetrics(); - Assert.assertEquals(RMAppState.FINISHED, recoveredApp.getState()); + assertEquals(RMAppState.FINISHED, recoveredApp.getState()); assertUnmanagedAMQueueMetrics(qm2, 1, 0, 0, 1); } @@ -1666,8 +1729,11 @@ public void testUAMRecoveryOnRMWorkPreservingRestart() throws Exception { //2. While the applications is running, restart the rm and // check whether the app submitted to the queue it was submitted initially. //3. Verify that application running successfully. - @Test(timeout = 60000) - public void testQueueRecoveryOnRMWorkPreservingRestart() throws Exception { + @Timeout(60) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testQueueRecoveryOnRMWorkPreservingRestart(SchedulerType type) throws Exception { + initTestWorkPreservingRMRestart(type); if (getSchedulerType() != SchedulerType.CAPACITY) { return; } @@ -1742,21 +1808,23 @@ ApplicationPlacementContext placeApplication( am.registerAppAttempt(true); rm2.waitForState(recoveredApp0.getApplicationId(), RMAppState.RUNNING); - Assert.assertEquals("root.joe.test", recoveredApp0.getQueue()); + assertEquals("root.joe.test", recoveredApp0.getQueue()); } private void assertUnmanagedAMQueueMetrics(QueueMetrics qm, int appsSubmitted, int appsPending, int appsRunning, int appsCompleted) { - Assert.assertEquals(appsSubmitted, qm.getUnmanagedAppsSubmitted()); - Assert.assertEquals(appsPending, qm.getUnmanagedAppsPending()); - Assert.assertEquals(appsRunning, qm.getUnmanagedAppsRunning()); - Assert.assertEquals(appsCompleted, qm.getUnmanagedAppsCompleted()); + assertEquals(appsSubmitted, qm.getUnmanagedAppsSubmitted()); + assertEquals(appsPending, qm.getUnmanagedAppsPending()); + assertEquals(appsRunning, qm.getUnmanagedAppsRunning()); + assertEquals(appsCompleted, qm.getUnmanagedAppsCompleted()); } - @Test(timeout = 30000) - public void testUnknownUserOnRecovery() throws Exception { - + @Timeout(30) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testUnknownUserOnRecovery(SchedulerType type) throws Exception { + initTestWorkPreservingRMRestart(type); MockRM rm1 = new MockRM(conf); rm1.start(); MockMemoryRMStateStore memStore = @@ -1797,32 +1865,44 @@ ApplicationPlacementContext placeApplication( rm2.start(); RMApp recoveredApp = rm2.getRMContext().getRMApps().get(app0.getApplicationId()); - Assert.assertEquals(RMAppState.KILLED, recoveredApp.getState()); + assertEquals(RMAppState.KILLED, recoveredApp.getState()); } - @Test(timeout = 30000) - public void testDynamicFlexibleAutoCreatedQueueRecoveryWithDefaultQueue() - throws Exception { + @Timeout(30) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testDynamicFlexibleAutoCreatedQueueRecoveryWithDefaultQueue( + SchedulerType type) throws Exception { + initTestWorkPreservingRMRestart(type); //if queue name is not specified, it should submit to 'default' queue testDynamicAutoCreatedQueueRecovery(USER1, null, true); } - @Test(timeout = 30000) - public void testDynamicAutoCreatedQueueRecoveryWithDefaultQueue() - throws Exception { + @Timeout(30) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testDynamicAutoCreatedQueueRecoveryWithDefaultQueue( + SchedulerType type) throws Exception { + initTestWorkPreservingRMRestart(type); //if queue name is not specified, it should submit to 'default' queue testDynamicAutoCreatedQueueRecovery(USER1, null, false); } - @Test(timeout = 30000) - public void testDynamicFlexibleAutoCreatedQueueRecoveryWithOverrideQueueMappingFlag() - throws Exception { + @Timeout(30) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testDynamicFlexibleAutoCreatedQueueRecoveryWithOverrideQueueMappingFlag( + SchedulerType type) throws Exception { + initTestWorkPreservingRMRestart(type); testDynamicAutoCreatedQueueRecovery(USER1, USER1, true); } - @Test(timeout = 30000) - public void testDynamicAutoCreatedQueueRecoveryWithOverrideQueueMappingFlag() - throws Exception { + @Timeout(30) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testDynamicAutoCreatedQueueRecoveryWithOverrideQueueMappingFlag( + SchedulerType type) throws Exception { + initTestWorkPreservingRMRestart(type); testDynamicAutoCreatedQueueRecovery(USER1, USER1, false); } @@ -1962,8 +2042,11 @@ public void testDynamicAutoCreatedQueueRecovery( // Apps already completed before RM restart. Make sure we restore the queue // correctly - @Test(timeout = 20000) - public void testFairSchedulerCompletedAppsQueue() throws Exception { + @Timeout(20) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testFairSchedulerCompletedAppsQueue(SchedulerType type) throws Exception { + initTestWorkPreservingRMRestart(type); if (getSchedulerType() != SchedulerType.FAIR) { return; } @@ -1979,10 +2062,10 @@ public void testFairSchedulerCompletedAppsQueue() throws Exception { String fsQueueContext = app.getApplicationSubmissionContext().getQueue(); String fsQueueApp = app.getQueue(); - assertEquals("Queue in app not equal to submission context", fsQueueApp, - fsQueueContext); + assertEquals(fsQueueApp, + fsQueueContext, "Queue in app not equal to submission context"); RMAppAttempt rmAttempt = app.getCurrentAppAttempt(); - assertNotNull("No AppAttempt found", rmAttempt); + assertNotNull(rmAttempt, "No AppAttempt found"); rm2 = new MockRM(conf, rm1.getRMStateStore()); rm2.start(); @@ -1990,12 +2073,12 @@ public void testFairSchedulerCompletedAppsQueue() throws Exception { RMApp recoveredApp = rm2.getRMContext().getRMApps().get(app.getApplicationId()); RMAppAttempt rmAttemptRecovered = recoveredApp.getCurrentAppAttempt(); - assertNotNull("No AppAttempt found after recovery", rmAttemptRecovered); + assertNotNull(rmAttemptRecovered, "No AppAttempt found after recovery"); String fsQueueContextRecovered = recoveredApp.getApplicationSubmissionContext().getQueue(); String fsQueueAppRecovered = recoveredApp.getQueue(); assertEquals(RMAppState.FINISHED, recoveredApp.getState()); - assertEquals("Recovered app queue is not the same as context queue", - fsQueueAppRecovered, fsQueueContextRecovered); + assertEquals(fsQueueAppRecovered, fsQueueContextRecovered, + "Recovered app queue is not the same as context queue"); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingUnmanagedAM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingUnmanagedAM.java index 3c57dfc19e61c..3ed7d0036455f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingUnmanagedAM.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingUnmanagedAM.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.assertTrue; + import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -38,10 +41,10 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.InvalidApplicationMasterRequestException; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import org.slf4j.event.Level; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; /** * Test UAM handling in RM. @@ -51,11 +54,11 @@ public class TestWorkPreservingUnmanagedAM private YarnConfiguration conf; - public TestWorkPreservingUnmanagedAM(SchedulerType type) throws IOException { - super(type); + public void initTestWorkPreservingUnmanagedAM(SchedulerType type) throws IOException { + initParameterizedSchedulerTestBase(type); + setup(); } - @Before public void setup() { GenericTestUtils.setRootLogLevel(Level.DEBUG); conf = getConf(); @@ -142,15 +145,15 @@ protected void testUAMRestart(boolean keepContainers) throws Exception { response.getNMTokensFromPreviousAttempts() .forEach(token -> tokenCacheClientSide.add(token.getNodeId())); } catch (InvalidApplicationMasterRequestException e) { - Assert.assertEquals(false, keepContainers); + assertEquals(false, keepContainers); return; } - Assert.assertEquals("RM should not allow second register" - + " for UAM without keep container flag ", true, keepContainers); + assertEquals(true, keepContainers, "RM should not allow second register" + + " for UAM without keep container flag "); // Expecting the two running containers previously - Assert.assertEquals(2, response.getContainersFromPreviousAttempts().size()); - Assert.assertEquals(1, response.getNMTokensFromPreviousAttempts().size()); + assertEquals(2, response.getContainersFromPreviousAttempts().size()); + assertEquals(1, response.getNMTokensFromPreviousAttempts().size()); // Allocate one more containers to UAM, just to be safe numContainers = 1; @@ -248,15 +251,15 @@ protected void testUAMRestartWithoutTransferContainer(boolean keepContainers) th response.getNMTokensFromPreviousAttempts() .forEach(token -> tokenCacheClientSide.add(token.getNodeId())); } catch (InvalidApplicationMasterRequestException e) { - Assert.assertEquals(false, keepContainers); + assertEquals(false, keepContainers); return; } - Assert.assertEquals("RM should not allow second register" - + " for UAM without keep container flag ", true, keepContainers); + assertEquals(true, keepContainers, "RM should not allow second register" + + " for UAM without keep container flag "); // Expecting the zero running containers previously - Assert.assertEquals(0, response.getContainersFromPreviousAttempts().size()); - Assert.assertEquals(0, response.getNMTokensFromPreviousAttempts().size()); + assertEquals(0, response.getContainersFromPreviousAttempts().size()); + assertEquals(0, response.getNMTokensFromPreviousAttempts().size()); // Allocate one more containers to UAM, just to be safe numContainers = 1; @@ -278,29 +281,43 @@ protected void testUAMRestartWithoutTransferContainer(boolean keepContainers) th rm.stop(); } - @Test(timeout = 600000) - public void testUAMRestartKeepContainers() throws Exception { + @Timeout(value = 600) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testUAMRestartKeepContainers(SchedulerType type) throws Exception { + initTestWorkPreservingUnmanagedAM(type); testUAMRestart(true); } - @Test(timeout = 600000) - public void testUAMRestartNoKeepContainers() throws Exception { + @Timeout(value = 600) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testUAMRestartNoKeepContainers(SchedulerType type) throws Exception { + initTestWorkPreservingUnmanagedAM(type); testUAMRestart(false); } - @Test(timeout = 600000) - public void testUAMRestartKeepContainersWithoutTransferContainer() throws Exception { + @Timeout(value = 600) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testUAMRestartKeepContainersWithoutTransferContainer( + SchedulerType type) throws Exception { + initTestWorkPreservingUnmanagedAM(type); testUAMRestartWithoutTransferContainer(true); } - @Test(timeout = 600000) - public void testUAMRestartNoKeepContainersWithoutTransferContainer() throws Exception { + @Timeout(value = 600) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testUAMRestartNoKeepContainersWithoutTransferContainer( + SchedulerType type) throws Exception { + initTestWorkPreservingUnmanagedAM(type); testUAMRestartWithoutTransferContainer(false); } private void checkNMTokenForContainer(Set cacheToken, List containers) { for (Container container : containers) { - Assert.assertTrue(cacheToken.contains(container.getNodeId())); + assertTrue(cacheToken.contains(container.getNodeId())); } } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMLaunchFailure.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMLaunchFailure.java index 6c8cfce2e8b39..06e20098ed375 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMLaunchFailure.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMLaunchFailure.java @@ -24,7 +24,7 @@ import java.util.List; import java.util.concurrent.atomic.AtomicInteger; -import org.junit.jupiter.api.Assertions; +import org.junit.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,9 +53,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler; import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; */ /* a test case that tests the launch failure of a AM */ @@ -179,7 +179,7 @@ public class TestAMLaunchFailure { // } // // -// @BeforeEach +// @Before // public void setUp() { // context = new RMContextImpl(new MemStore()); // Configuration conf = new Configuration(); @@ -202,7 +202,7 @@ public class TestAMLaunchFailure { // asmImpl.start(); // } // -// @AfterEach +// @After // public void tearDown() { // asmImpl.stop(); // } @@ -229,6 +229,6 @@ public class TestAMLaunchFailure { // Thread.sleep(200); // application = context.getApplications().get(appID); // } -// Assertions.assertEquals(ApplicationState.FAILED, application.getState()); +// Assert.assertEquals(ApplicationState.FAILED, application.getState()); // } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java index 9006c6b57d782..864e50c4e3182 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java @@ -79,21 +79,24 @@ import org.apache.hadoop.thirdparty.com.google.common.base.Throwables; -import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; /** * Test AM restart functions. */ public class TestAMRestart extends ParameterizedSchedulerTestBase { - public TestAMRestart(SchedulerType type) throws IOException { - super(type); + public void initTestAMRestart(SchedulerType type) throws IOException { + initParameterizedSchedulerTestBase(type); } - @Test + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") @Timeout(value = 30) - public void testAMRestartWithExistingContainers() throws Exception { + public void testAMRestartWithExistingContainers(SchedulerType type) throws Exception { + initTestAMRestart(type); getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2); MockRM rm1 = new MockRM(getConf()); @@ -286,8 +289,8 @@ public static List allocateContainers(MockNM nm1, MockAM am1, Thread.sleep(200); } - assertEquals( - NUM_CONTAINERS, containers.size(), "Did not get all containers allocated"); + assertEquals(NUM_CONTAINERS, containers.size(), + "Did not get all containers allocated"); return containers; } @@ -301,9 +304,11 @@ private void waitForContainersToFinish(int expectedNum, RMAppAttempt attempt) } } - @Test + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") @Timeout(value = 30) - public void testNMTokensRebindOnAMRestart() throws Exception { + public void testNMTokensRebindOnAMRestart(SchedulerType type) throws Exception { + initTestAMRestart(type); getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 3); // To prevent test from blacklisting nm1 for AM, we sit threshold to half // of 2 nodes which is 1 @@ -426,9 +431,11 @@ public void testNMTokensRebindOnAMRestart() throws Exception { * AM container preempted, nm disk failure * should not be counted towards AM max retry count. */ - @Test + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") @Timeout(value = 100) - public void testShouldNotCountFailureToMaxAttemptRetry() throws Exception { + public void testShouldNotCountFailureToMaxAttemptRetry(SchedulerType type) throws Exception { + initTestAMRestart(type); getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2); getConf().setBoolean(YarnConfiguration.RECOVERY_ENABLED, true); getConf().set( @@ -552,9 +559,11 @@ public void testShouldNotCountFailureToMaxAttemptRetry() throws Exception { rm1.stop(); } - @Test + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") @Timeout(value = 100) - public void testMaxAttemptOneMeansOne() throws Exception { + public void testMaxAttemptOneMeansOne(SchedulerType type) throws Exception { + initTestAMRestart(type); getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1); getConf().setBoolean(YarnConfiguration.RECOVERY_ENABLED, true); getConf().set( @@ -589,9 +598,11 @@ public void testMaxAttemptOneMeansOne() throws Exception { * AM preemption failure towards the max-retry-account and should be able to * re-launch the AM. */ - @Test + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") @Timeout(value = 60) - public void testPreemptedAMRestartOnRMRestart() throws Exception { + public void testPreemptedAMRestartOnRMRestart(SchedulerType type) throws Exception { + initTestAMRestart(type); getConf().setBoolean(YarnConfiguration.RECOVERY_ENABLED, true); getConf().setBoolean( YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, false); @@ -677,10 +688,12 @@ public void testPreemptedAMRestartOnRMRestart() throws Exception { * AM failure towards the max-retry-account and should be able to * re-launch the AM. */ - @Test + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") @Timeout(value = 50) - public void testRMRestartOrFailoverNotCountedForAMFailures() + public void testRMRestartOrFailoverNotCountedForAMFailures(SchedulerType type) throws Exception { + initTestAMRestart(type); getConf().setBoolean(YarnConfiguration.RECOVERY_ENABLED, true); getConf().setBoolean( YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, false); @@ -753,9 +766,11 @@ public void testRMRestartOrFailoverNotCountedForAMFailures() rm2.stop(); } - @Test + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") @Timeout(value = 120) - public void testRMAppAttemptFailuresValidityInterval() throws Exception { + public void testRMAppAttemptFailuresValidityInterval(SchedulerType type) throws Exception { + initTestAMRestart(type); getConf().setBoolean(YarnConfiguration.RECOVERY_ENABLED, true); getConf().setBoolean( YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, false); @@ -915,9 +930,11 @@ private boolean isContainerIdInContainerStatus( return false; } - @Test + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") @Timeout(value = 40) - public void testAMRestartNotLostContainerCompleteMsg() throws Exception { + public void testAMRestartNotLostContainerCompleteMsg(SchedulerType type) throws Exception { + initTestAMRestart(type); getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2); MockRM rm1 = new MockRM(getConf()); @@ -1025,10 +1042,12 @@ public void testAMRestartNotLostContainerCompleteMsg() throws Exception { * after AM reset window, even if AM who was the last is failed, * all containers are launched by previous AM should be kept. */ - @Test + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") @Timeout(value = 20) - public void testAMRestartNotLostContainerAfterAttemptFailuresValidityInterval() + public void testAMRestartNotLostContainerAfterAttemptFailuresValidityInterval(SchedulerType type) throws Exception { + initTestAMRestart(type); // explicitly set max-am-retry count as 2. getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2); @@ -1118,10 +1137,12 @@ public void testAMRestartNotLostContainerAfterAttemptFailuresValidityInterval() * 8. Verify that the app master receives container 3 in the RM response to * its heartbeat. */ - @Test + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") @Timeout(value = 200) - public void testContainersFromPreviousAttemptsWithRMRestart() + public void testContainersFromPreviousAttemptsWithRMRestart(SchedulerType type) throws Exception { + initTestAMRestart(type); getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2); getConf().setBoolean(YarnConfiguration.RECOVERY_ENABLED, true); getConf().setBoolean( @@ -1217,8 +1238,8 @@ public void testContainersFromPreviousAttemptsWithRMRestart() // container2 is recovered from previous attempt assertEquals(1, registerResponse.getContainersFromPreviousAttempts().size()); - assertEquals(containerId2 -, registerResponse.getContainersFromPreviousAttempts().get(0).getId(), "container 2"); + assertEquals(containerId2, registerResponse.getContainersFromPreviousAttempts().get(0).getId(), + "container 2"); List prevNMTokens = registerResponse .getNMTokensFromPreviousAttempts(); assertEquals(1, prevNMTokens.size()); @@ -1247,8 +1268,8 @@ public void testContainersFromPreviousAttemptsWithRMRestart() if (allocateResponse.getContainersFromPreviousAttempts().size() > 0){ containersFromPreviousAttempts.addAll( allocateResponse.getContainersFromPreviousAttempts()); - assertEquals( - 0, allocateResponse.getAllocatedContainers().size(), "new containers should not be allocated"); + assertEquals(0, allocateResponse.getAllocatedContainers().size(), + "new containers should not be allocated"); List nmTokens = allocateResponse.getNMTokens(); assertEquals(1, nmTokens.size()); // container 3 is running on node 2 @@ -1262,8 +1283,8 @@ public void testContainersFromPreviousAttemptsWithRMRestart() } return false; }, 2000, 200000); - assertEquals(containerId3 -, containersFromPreviousAttempts.get(0).getId(), "container 3"); + assertEquals(containerId3, + containersFromPreviousAttempts.get(0).getId(), "container 3"); rm2.stop(); rm1.stop(); } @@ -1279,9 +1300,11 @@ public void testContainersFromPreviousAttemptsWithRMRestart() * 4. Verify that the used resource of queue should be cleaned up normally * after app fail. */ - @Test + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") @Timeout(value = 30) - public void testQueueResourceDoesNotLeak() throws Exception { + public void testQueueResourceDoesNotLeak(SchedulerType type) throws Exception { + initTestAMRestart(type); getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1); getConf().setBoolean(YarnConfiguration.RECOVERY_ENABLED, true); getConf() diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestASMStateMachine.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestASMStateMachine.java index 8ba565ffeb750..ef3b65cc1a6c7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestASMStateMachine.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestASMStateMachine.java @@ -21,7 +21,7 @@ /* import java.util.concurrent.atomic.AtomicInteger; -import org.junit.jupiter.api.Assertions; +import org.junit.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,9 +37,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType; import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStoreFactory; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; */ public class TestASMStateMachine { @@ -56,7 +56,7 @@ public class TestASMStateMachine { // private boolean launchCleanupCalled = false; // private AtomicInteger waitForState = new AtomicInteger(); // private Configuration conf = new Configuration(); -// @BeforeEach +// @Before // public void setUp() { // context.getDispatcher().init(conf); // context.getDispatcher().start(); @@ -67,7 +67,7 @@ public class TestASMStateMachine { // new MockAppplicationMasterInfo(); // } // -// @AfterEach +// @After // public void tearDown() { // // } @@ -160,7 +160,7 @@ public class TestASMStateMachine { // Thread.sleep(500); // count++; // } -// Assertions.assertEquals(finalState, masterInfo.getState()); +// Assert.assertEquals(finalState, masterInfo.getState()); // } // // /* Test the state machine. @@ -183,13 +183,13 @@ public class TestASMStateMachine { // ApplicationEventType.ALLOCATE, submissioncontext.getApplicationId())); // // waitForState(ApplicationState.LAUNCHED, masterInfo); -// Assertions.assertTrue(snAllocateReceived); -// Assertions.assertTrue(launchCalled); -// Assertions.assertTrue(addedApplication); +// Assert.assertTrue(snAllocateReceived); +// Assert.assertTrue(launchCalled); +// Assert.assertTrue(addedApplication); // handler // .handle(new AMRegistrationEvent(masterInfo.getMaster())); // waitForState(ApplicationState.RUNNING, masterInfo); -// Assertions.assertEquals(ApplicationState.RUNNING, masterInfo.getState()); +// Assert.assertEquals(ApplicationState.RUNNING, masterInfo.getState()); // // ApplicationStatus status = recordFactory // .newRecordInstance(ApplicationStatus.class); @@ -198,20 +198,20 @@ public class TestASMStateMachine { // // /* check if the state is still RUNNING */ // -// Assertions.assertEquals(ApplicationState.RUNNING, masterInfo.getState()); +// Assert.assertEquals(ApplicationState.RUNNING, masterInfo.getState()); // // handler.handle(new AMFinishEvent(masterInfo.getApplicationID(), // ApplicationState.COMPLETED, "", "")); // waitForState(ApplicationState.COMPLETED, masterInfo); -// Assertions.assertEquals(ApplicationState.COMPLETED, masterInfo.getState()); +// Assert.assertEquals(ApplicationState.COMPLETED, masterInfo.getState()); // /* check if clean up is called for everyone */ -// Assertions.assertTrue(launchCleanupCalled); -// Assertions.assertTrue(snreceivedCleanUp); -// Assertions.assertTrue(removedApplication); +// Assert.assertTrue(launchCleanupCalled); +// Assert.assertTrue(snreceivedCleanUp); +// Assert.assertTrue(removedApplication); // // /* check if expiry doesnt make it failed */ // handler.handle(new ApplicationEvent(ApplicationEventType.EXPIRE, // masterInfo.getApplicationID())); -// Assertions.assertEquals(ApplicationState.COMPLETED, masterInfo.getState()); +// Assert.assertEquals(ApplicationState.COMPLETED, masterInfo.getState()); // } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestApplicationMasterExpiry.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestApplicationMasterExpiry.java index ce496d0b21711..beaec79fa38bb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestApplicationMasterExpiry.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestApplicationMasterExpiry.java @@ -21,7 +21,7 @@ /* import java.util.concurrent.atomic.AtomicInteger; -import org.junit.jupiter.api.Assertions; +import org.junit.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,8 +38,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; +import org.junit.Before; +import org.junit.Test; */ /** @@ -54,7 +54,7 @@ public class TestApplicationMasterExpiry { // private final RMContext context = new RMContextImpl(new MemStore()); // private AMLivelinessMonitor amLivelinessMonitor; // -// @BeforeEach +// @Before // public void setUp() { // new DummyApplicationTracker(); // new DummySN(); @@ -127,7 +127,7 @@ public class TestApplicationMasterExpiry { // Thread.sleep(500); // count++; // } -// Assertions.assertEquals(finalState, application.getState()); +// Assert.assertEquals(finalState, application.getState()); // } // // @Test @@ -169,6 +169,6 @@ public class TestApplicationMasterExpiry { // expiry.wait(1000); // } // } -// Assertions.assertTrue(expired); +// Assert.assertTrue(expired); // } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java index 81be02eec7c52..211b7d3edfa95 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java @@ -18,10 +18,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.metrics; -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.assertTrue; import java.util.Arrays; import java.util.Collection; @@ -33,12 +29,9 @@ import java.util.Set; 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.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import org.apache.hadoop.ipc.CallerContext; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; @@ -76,15 +69,17 @@ import org.apache.hadoop.yarn.server.timeline.recovery.MemoryTimelineStateStore; import org.apache.hadoop.yarn.server.timeline.recovery.TimelineStateStore; +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.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -@RunWith(Parameterized.class) public class TestSystemMetricsPublisher { - @Parameters public static Collection data() { - return Arrays.asList(new Object[][] {{false, 0}, {true, 1}}); + return Arrays.asList(new Object[][]{{false, 0}, {true, 1}}); } private static ApplicationHistoryServer timelineServer; @@ -94,13 +89,13 @@ public static Collection data() { private boolean rmTimelineServerV1PublisherBatchEnabled; private int rmTimelineServerV1PublisherInterval; - public TestSystemMetricsPublisher(boolean rmTimelineServerV1PublisherBatchEnabled, - int rmTimelineServerV1PublisherInterval) { - this.rmTimelineServerV1PublisherBatchEnabled = rmTimelineServerV1PublisherBatchEnabled; - this.rmTimelineServerV1PublisherInterval = rmTimelineServerV1PublisherInterval; + private void initTestSystemMetricsPublisher(boolean pRmTimelineServerV1PublisherBatchEnabled, + int pRmTimelineServerV1PublisherInterval) throws Exception { + this.rmTimelineServerV1PublisherBatchEnabled = pRmTimelineServerV1PublisherBatchEnabled; + this.rmTimelineServerV1PublisherInterval = pRmTimelineServerV1PublisherInterval; + setup(); } - @BeforeEach public void setup() throws Exception { YarnConfiguration conf = new YarnConfiguration(); conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true); @@ -137,9 +132,13 @@ public void tearDown() throws Exception { } } - @Test + @ParameterizedTest + @MethodSource("data") @Timeout(value = 10) - public void testPublishApplicationMetrics() throws Exception { + public void testPublishApplicationMetrics(boolean pRmTimelineServerV1PublisherBatchEnabled, + int pRmTimelineServerV1PublisherInterval) throws Exception { + initTestSystemMetricsPublisher(pRmTimelineServerV1PublisherBatchEnabled, + pRmTimelineServerV1PublisherInterval); long stateUpdateTimeStamp = System.currentTimeMillis(); for (int i = 1; i <= 2; ++i) { ApplicationId appId = ApplicationId.newInstance(0, i); @@ -334,9 +333,13 @@ public void testPublishApplicationMetrics() throws Exception { } } - @Test + @ParameterizedTest + @MethodSource("data") @Timeout(value = 10) - public void testPublishAppAttemptMetricsForUnmanagedAM() throws Exception { + public void testPublishAppAttemptMetricsForUnmanagedAM(boolean pRmTimelineServerV1PublisherBatchEnabled, + int pRmTimelineServerV1PublisherInterval) throws Exception { + initTestSystemMetricsPublisher(pRmTimelineServerV1PublisherBatchEnabled, + pRmTimelineServerV1PublisherInterval); ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 1), 1); RMAppAttempt appAttempt = createRMAppAttempt(appAttemptId,true); @@ -355,9 +358,13 @@ public void testPublishAppAttemptMetricsForUnmanagedAM() throws Exception { } while (entity == null || entity.getEvents().size() < 2); } - @Test + @ParameterizedTest + @MethodSource("data") @Timeout(value = 10) - public void testPublishAppAttemptMetrics() throws Exception { + public void testPublishAppAttemptMetrics(boolean pRmTimelineServerV1PublisherBatchEnabled, + int pRmTimelineServerV1PublisherInterval) throws Exception { + initTestSystemMetricsPublisher(pRmTimelineServerV1PublisherBatchEnabled, + pRmTimelineServerV1PublisherInterval); ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 1), 1); RMAppAttempt appAttempt = createRMAppAttempt(appAttemptId, false); @@ -423,9 +430,14 @@ public void testPublishAppAttemptMetrics() throws Exception { assertTrue(hasRegisteredEvent && hasFinishedEvent); } - @Test + @ParameterizedTest + @MethodSource("data") @Timeout(value = 10) - public void testPublishHostPortInfoOnContainerFinished() throws Exception { + public void testPublishHostPortInfoOnContainerFinished( + boolean pRmTimelineServerV1PublisherBatchEnabled, + int pRmTimelineServerV1PublisherInterval) throws Exception { + initTestSystemMetricsPublisher(pRmTimelineServerV1PublisherBatchEnabled, + pRmTimelineServerV1PublisherInterval); ContainerId containerId = ContainerId.newContainerId(ApplicationAttemptId.newInstance( ApplicationId.newInstance(0, 1), 1), 1); @@ -454,9 +466,13 @@ public void testPublishHostPortInfoOnContainerFinished() throws Exception { ContainerMetricsConstants.ALLOCATED_PORT_INFO)); } - @Test + @ParameterizedTest + @MethodSource("data") @Timeout(value = 10) - public void testPublishContainerMetrics() throws Exception { + public void testPublishContainerMetrics(boolean pRmTimelineServerV1PublisherBatchEnabled, + int pRmTimelineServerV1PublisherInterval) throws Exception { + initTestSystemMetricsPublisher(pRmTimelineServerV1PublisherBatchEnabled, + pRmTimelineServerV1PublisherInterval); ContainerId containerId = ContainerId.newContainerId(ApplicationAttemptId.newInstance( ApplicationId.newInstance(0, 1), 1), 1); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java index fe566c611d1c5..92143d9aee3e3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java @@ -21,6 +21,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -80,7 +81,6 @@ import org.apache.hadoop.yarn.util.TimelineServiceHelper; import org.apache.hadoop.yarn.util.timeline.TimelineUtils; import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -165,9 +165,7 @@ private static Configuration getTimelineV2Conf() { testRootDir.getCanonicalPath()); } catch (IOException e) { e.printStackTrace(); - Assertions - .fail("Exception while setting the " + - "TIMELINE_SERVICE_STORAGE_DIR_ROOT "); + fail("Exception while setting the TIMELINE_SERVICE_STORAGE_DIR_ROOT "); } return conf; } @@ -182,9 +180,8 @@ public void testSystemMetricPublisherInitialization() { conf.setBoolean(YarnConfiguration.RM_PUBLISH_CONTAINER_EVENTS_ENABLED, YarnConfiguration.DEFAULT_RM_PUBLISH_CONTAINER_EVENTS_ENABLED); publisher.init(conf); - assertFalse( - - publisher.isPublishContainerEvents(), "Default configuration should not publish container events from RM"); + assertFalse(publisher.isPublishContainerEvents(), + "Default configuration should not publish container events from RM"); publisher.stop(); @@ -192,12 +189,12 @@ public void testSystemMetricPublisherInitialization() { mock(RMTimelineCollectorManager.class)); conf = getTimelineV2Conf(); publisher.init(conf); - assertTrue( - publisher.isPublishContainerEvents(), "Expected to have registered event handlers and set ready to " + assertTrue(publisher.isPublishContainerEvents(), + "Expected to have registered event handlers and set ready to " + "publish events after init"); publisher.start(); - assertTrue( - publisher.isPublishContainerEvents(), "Expected to publish container events from RM"); + assertTrue(publisher.isPublishContainerEvents(), + "Expected to publish container events from RM"); } finally { publisher.stop(); } @@ -220,14 +217,14 @@ public void testPublishApplicationMetrics() throws Exception { + "/"; File entityFolder = new File(outputDirApp); - Assertions.assertTrue(entityFolder.isDirectory()); + assertTrue(entityFolder.isDirectory()); // file name is .thist String timelineServiceFileName = appId.toString() + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION; File appFile = new File(outputDirApp, timelineServiceFileName); - Assertions.assertTrue(appFile.exists()); + assertTrue(appFile.exists()); verifyEntity( appFile, 4, ApplicationMetricsConstants.CREATED_EVENT_TYPE, 8, 0); } @@ -256,14 +253,14 @@ public void testPublishAppAttemptMetrics() throws Exception { + TimelineEntityType.YARN_APPLICATION_ATTEMPT + "/"; File entityFolder = new File(outputDirApp); - Assertions.assertTrue(entityFolder.isDirectory()); + assertTrue(entityFolder.isDirectory()); // file name is .thist String timelineServiceFileName = appAttemptId.toString() + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION; File appFile = new File(outputDirApp, timelineServiceFileName); - Assertions.assertTrue(appFile.exists()); + assertTrue(appFile.exists()); verifyEntity(appFile, 2, AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE, 0, TimelineServiceHelper.invertLong(appAttemptId.getAttemptId())); } @@ -289,14 +286,14 @@ public void testPublishContainerMetrics() throws Exception { + TimelineEntityType.YARN_CONTAINER + "/"; File entityFolder = new File(outputDirApp); - Assertions.assertTrue(entityFolder.isDirectory()); + assertTrue(entityFolder.isDirectory()); // file name is .thist String timelineServiceFileName = containerId.toString() + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION; File appFile = new File(outputDirApp, timelineServiceFileName); - Assertions.assertTrue(appFile.exists()); + assertTrue(appFile.exists()); verifyEntity(appFile, 2, ContainerMetricsConstants.CREATED_IN_RM_EVENT_TYPE, 0, TimelineServiceHelper.invertLong(containerId.getContainerId())); @@ -337,8 +334,8 @@ public List getLog() { metricsPublisher.appCreated(app, app.getStartTime()); dispatcher.await(); for (LoggingEvent event : appender.getLog()) { - assertFalse( - event.getRenderedMessage().contains("Error in dispatcher thread"), "Dispatcher Crashed"); + assertFalse(event.getRenderedMessage(). + contains("Error in dispatcher thread"), "Dispatcher Crashed"); } } finally { logger.removeAppender(appender); @@ -382,10 +379,10 @@ private static void verifyEntity(File entityFile, long expectedEvents, } finally { reader.close(); } - assertEquals( - expectedEvents, count, "Expected " + expectedEvents + " events to be published"); - assertEquals( - expectedMetrics, metricsCount, "Expected " + expectedMetrics + " metrics is incorrect"); + assertEquals(expectedEvents, count, + "Expected " + expectedEvents + " events to be published"); + assertEquals(expectedMetrics, metricsCount, + "Expected " + expectedMetrics + " metrics is incorrect"); } private String getTimelineEntityDir(RMApp app) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java index ec23b4beb5bfa..75b5b909ae5cd 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java @@ -19,6 +19,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.Service; +import org.apache.hadoop.test.TestName; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.Container; @@ -59,9 +60,8 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; import org.junit.jupiter.api.BeforeEach; -import org.junit.Rule; import org.junit.jupiter.api.Test; -import org.junit.rules.TestName; +import org.junit.jupiter.api.extension.RegisterExtension; import org.mockito.ArgumentCaptor; import org.mockito.ArgumentMatcher; import org.mockito.invocation.InvocationOnMock; @@ -148,9 +148,10 @@ public enum priority { public int getValue() { return this.value; } - }; + }; - @Rule public TestName name = new TestName(); + @RegisterExtension + public TestName name = new TestName(); @BeforeEach @SuppressWarnings("unchecked") @@ -400,8 +401,8 @@ public void testPerQueueDisablePreemptionHierarchical() { ApplicationAttemptId expectedAttemptOnQueueB = ApplicationAttemptId.newInstance( appA.getApplicationId(), appA.getAttemptId()); - assertTrue( - mCS.getAppsInQueue("queueB").contains(expectedAttemptOnQueueB), "appA should be running on queueB"); + assertTrue(mCS.getAppsInQueue("queueB").contains(expectedAttemptOnQueueB), + "appA should be running on queueB"); verify(mDisp, times(10)).handle(argThat(new IsPreemptionRequestFor(appA))); // Need to call setup() again to reset mDisp @@ -418,10 +419,10 @@ public void testPerQueueDisablePreemptionHierarchical() { appC.getApplicationId(), appC.getAttemptId()); // Now, all of queueB's (appA) over capacity is not preemptable, so neither // is queueA's. Verify that capacity is taken from queueE (appC). - assertTrue( - mCS.getAppsInQueue("queueC").contains(expectedAttemptOnQueueC), "appB should be running on queueC"); - assertTrue( - mCS.getAppsInQueue("queueE").contains(expectedAttemptOnQueueE), "appC should be running on queueE"); + assertTrue(mCS.getAppsInQueue("queueC").contains(expectedAttemptOnQueueC), + "appB should be running on queueC"); + assertTrue(mCS.getAppsInQueue("queueE").contains(expectedAttemptOnQueueE), + "appC should be running on queueE"); // Resources should have come from queueE (appC) and neither of queueA's // children. verify(mDisp, never()).handle(argThat(new IsPreemptionRequestFor(appA))); @@ -1118,8 +1119,8 @@ public void testLeafQueueNameExtraction() throws Exception { policy.editSchedule(); - assertFalse(policy.getLeafQueueNames().contains("root.dynamicParent"), "dynamicParent should not be a LeafQueue " + - "candidate"); + assertFalse(policy.getLeafQueueNames().contains("root.dynamicParent"), + "dynamicParent should not be a LeafQueue candidate"); } static class IsPreemptionRequestFor diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyMockFramework.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyMockFramework.java index 2fbf9d3f14df9..28bc159b9a14c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyMockFramework.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyMockFramework.java @@ -23,9 +23,14 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +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; + public class TestProportionalCapacityPreemptionPolicyMockFramework extends ProportionalCapacityPreemptionPolicyMockFramework { @@ -104,7 +109,7 @@ public void testBuilder() throws Exception { checkAbsCapacities(cs.getQueue("a2"), "blue", 0.5f, 1f, 0f); checkPendingResource(cs.getQueue("a2"), "blue", 200); checkPriority(cs.getQueue("a2"), 2); - Assertions.assertFalse(cs.getQueue("a2").getPreemptionDisabled()); + assertFalse(cs.getQueue("a2").getPreemptionDisabled()); // b checkAbsCapacities(cs.getQueue("b"), "", 0.5f, 1f, 0f); @@ -114,16 +119,16 @@ public void testBuilder() throws Exception { checkAbsCapacities(cs.getQueue("b"), "blue", 0f, 0f, 0f); checkPendingResource(cs.getQueue("b"), "blue", 0); checkPriority(cs.getQueue("b"), 1); - Assertions.assertTrue(cs.getQueue("b").getPreemptionDisabled()); + assertTrue(cs.getQueue("b").getPreemptionDisabled()); // Check ignored partitioned containers in queue - Assertions.assertEquals(100, ((LeafQueue) cs.getQueue("a1")) + assertEquals(100, ((LeafQueue) cs.getQueue("a1")) .getIgnoreExclusivityRMContainers().get("blue").size()); // Check applications - Assertions.assertEquals(2, ((LeafQueue)cs.getQueue("a1")).getApplications().size()); - Assertions.assertEquals(1, ((LeafQueue)cs.getQueue("a2")).getApplications().size()); - Assertions.assertEquals(1, ((LeafQueue)cs.getQueue("b")).getApplications().size()); + assertEquals(2, ((LeafQueue)cs.getQueue("a1")).getApplications().size()); + assertEquals(1, ((LeafQueue)cs.getQueue("a2")).getApplications().size()); + assertEquals(1, ((LeafQueue)cs.getQueue("b")).getApplications().size()); // Check #containers FiCaSchedulerApp app1 = getApp("a1", 1); @@ -131,17 +136,17 @@ public void testBuilder() throws Exception { FiCaSchedulerApp app3 = getApp("a2", 3); FiCaSchedulerApp app4 = getApp("b", 4); - Assertions.assertEquals(50, app1.getLiveContainers().size()); + assertEquals(50, app1.getLiveContainers().size()); checkContainerNodesInApp(app1, 50, "n3"); - Assertions.assertEquals(50, app2.getLiveContainers().size()); - Assertions.assertEquals(150, app2.getReservedContainers().size()); + assertEquals(50, app2.getLiveContainers().size()); + assertEquals(150, app2.getReservedContainers().size()); checkContainerNodesInApp(app2, 200, "n2"); - Assertions.assertEquals(50, app3.getLiveContainers().size()); + assertEquals(50, app3.getLiveContainers().size()); checkContainerNodesInApp(app3, 50, "n3"); - Assertions.assertEquals(100, app4.getLiveContainers().size()); + assertEquals(100, app4.getLiveContainers().size()); checkContainerNodesInApp(app4, 100, "n1"); } @@ -236,20 +241,20 @@ public void testBuilderWithSpecifiedNodeResources() throws Exception { buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig); // Check host resources - Assertions.assertEquals(3, this.cs.getAllNodes().size()); + assertEquals(3, this.cs.getAllNodes().size()); SchedulerNode node1 = cs.getSchedulerNode(NodeId.newInstance("n1", 1)); - Assertions.assertEquals(100, node1.getTotalResource().getMemorySize()); - Assertions.assertEquals(100, node1.getCopiedListOfRunningContainers().size()); - Assertions.assertNull(node1.getReservedContainer()); + assertEquals(100, node1.getTotalResource().getMemorySize()); + assertEquals(100, node1.getCopiedListOfRunningContainers().size()); + assertNull(node1.getReservedContainer()); SchedulerNode node2 = cs.getSchedulerNode(NodeId.newInstance("n2", 1)); - Assertions.assertEquals(0, node2.getTotalResource().getMemorySize()); - Assertions.assertEquals(50, node2.getCopiedListOfRunningContainers().size()); - Assertions.assertNotNull(node2.getReservedContainer()); + assertEquals(0, node2.getTotalResource().getMemorySize()); + assertEquals(50, node2.getCopiedListOfRunningContainers().size()); + assertNotNull(node2.getReservedContainer()); SchedulerNode node3 = cs.getSchedulerNode(NodeId.newInstance("n3", 1)); - Assertions.assertEquals(30, node3.getTotalResource().getMemorySize()); - Assertions.assertEquals(100, node3.getCopiedListOfRunningContainers().size()); - Assertions.assertNull(node3.getReservedContainer()); + assertEquals(30, node3.getTotalResource().getMemorySize()); + assertEquals(100, node3.getCopiedListOfRunningContainers().size()); + assertNull(node3.getReservedContainer()); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/mockframework/ProportionalCapacityPreemptionPolicyMockFramework.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/mockframework/ProportionalCapacityPreemptionPolicyMockFramework.java index ca4eec164884b..35cfc9700e3bf 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/mockframework/ProportionalCapacityPreemptionPolicyMockFramework.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/mockframework/ProportionalCapacityPreemptionPolicyMockFramework.java @@ -50,7 +50,6 @@ import org.apache.hadoop.yarn.util.resource.ResourceUtils; import org.apache.hadoop.yarn.util.resource.Resources; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.mockito.ArgumentMatcher; @@ -62,6 +61,7 @@ import org.apache.hadoop.yarn.event.Event; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -252,7 +252,7 @@ protected void checkContainerNodesInApp(FiCaSchedulerApp app, num++; } } - Assertions.assertEquals(expectedContainersNumber, num); + assertEquals(expectedContainersNumber, num); } public FiCaSchedulerApp getApp(String queueName, int appId) { @@ -268,28 +268,28 @@ public FiCaSchedulerApp getApp(String queueName, int appId) { protected void checkAbsCapacities(CSQueue queue, String partition, float guaranteed, float max, float used) { QueueCapacities qc = queue.getQueueCapacities(); - Assertions.assertEquals(guaranteed, qc.getAbsoluteCapacity(partition), + assertEquals(guaranteed, qc.getAbsoluteCapacity(partition), ALLOWED_CAPACITY_DELTA); - Assertions.assertEquals(max, qc.getAbsoluteMaximumCapacity(partition), + assertEquals(max, qc.getAbsoluteMaximumCapacity(partition), ALLOWED_CAPACITY_DELTA); - Assertions.assertEquals(used, qc.getAbsoluteUsedCapacity(partition), + assertEquals(used, qc.getAbsoluteUsedCapacity(partition), ALLOWED_CAPACITY_DELTA); } protected void checkPendingResource(CSQueue queue, String partition, int pending) { ResourceUsage ru = queue.getQueueResourceUsage(); - Assertions.assertEquals(pending, ru.getPending(partition).getMemorySize()); + assertEquals(pending, ru.getPending(partition).getMemorySize()); } protected void checkPriority(CSQueue queue, int expectedPriority) { - Assertions.assertEquals(expectedPriority, queue.getPriority().getPriority()); + assertEquals(expectedPriority, queue.getPriority().getPriority()); } protected void checkReservedResource(CSQueue queue, String partition, int reserved) { ResourceUsage ru = queue.getQueueResourceUsage(); - Assertions.assertEquals(reserved, ru.getReserved(partition).getMemorySize()); + assertEquals(reserved, ru.getReserved(partition).getMemorySize()); } public static class IsPreemptionRequestForQueueAndNode diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationSystem.java index 389dd62e4afbd..d83b97edcfb59 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationSystem.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationSystem.java @@ -17,6 +17,11 @@ *******************************************************************************/ package org.apache.hadoop.yarn.server.resourcemanager.reservation; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.fail; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.yarn.conf.YarnConfiguration; @@ -32,11 +37,9 @@ 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.server.resourcemanager.scheduler.fair.FairSchedulerTestBase; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.mockito.Mockito; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import java.io.File; import java.io.IOException; @@ -54,11 +57,11 @@ public class TestReservationSystem extends private Configuration conf; private RMContext mockRMContext; - public TestReservationSystem(SchedulerType type) throws IOException { - super(type); + public void initTestReservationSystem(SchedulerType type) throws IOException { + initParameterizedSchedulerTestBase(type); + setUp(); } - @Before public void setUp() throws IOException { scheduler = initializeScheduler(); rmContext = getRMContext(); @@ -67,7 +70,7 @@ public void setUp() throws IOException { DefaultMetricsSystem.setMiniClusterMode(true); } - @After + @AfterEach public void tearDown() { conf = null; reservationSystem = null; @@ -77,12 +80,14 @@ public void tearDown() { QueueMetrics.clearQueueMetrics(); } - @Test - public void testInitialize() throws IOException { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testInitialize(SchedulerType type) throws IOException { + initTestReservationSystem(type); try { reservationSystem.reinitialize(scheduler.getConfig(), rmContext); } catch (YarnException e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } if (getSchedulerType().equals(SchedulerType.CAPACITY)) { ReservationSystemTestUtil.validateReservationQueue(reservationSystem, @@ -94,13 +99,15 @@ public void testInitialize() throws IOException { } - @Test - public void testReinitialize() throws IOException { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testReinitialize(SchedulerType type) throws IOException { + initTestReservationSystem(type); conf = scheduler.getConfig(); try { reservationSystem.reinitialize(conf, rmContext); } catch (YarnException e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } if (getSchedulerType().equals(SchedulerType.CAPACITY)) { ReservationSystemTestUtil.validateReservationQueue(reservationSystem, @@ -112,17 +119,17 @@ public void testReinitialize() throws IOException { // Dynamically add a plan String newQ = "reservation"; - Assert.assertNull(reservationSystem.getPlan(newQ)); + assertNull(reservationSystem.getPlan(newQ)); updateSchedulerConf(conf, newQ); try { scheduler.reinitialize(conf, rmContext); } catch (IOException e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } try { reservationSystem.reinitialize(conf, rmContext); } catch (YarnException e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } ReservationSystemTestUtil.validateReservationQueue( reservationSystem, @@ -174,10 +181,10 @@ private CapacityScheduler initializeCapacityScheduler() { CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); ReservationSystemTestUtil.setupQueueConfiguration(conf); - CapacityScheduler cs = Mockito.spy(new CapacityScheduler()); + CapacityScheduler cs = spy(new CapacityScheduler()); cs.setConf(conf); CSMaxRunningAppsEnforcer enforcer = - Mockito.mock(CSMaxRunningAppsEnforcer.class); + mock(CSMaxRunningAppsEnforcer.class); cs.setMaxRunningAppsEnforcer(enforcer); mockRMContext = ReservationSystemTestUtil.createRMContext(conf); @@ -186,7 +193,7 @@ private CapacityScheduler initializeCapacityScheduler() { try { cs.serviceInit(conf); } catch (Exception e) { - Assert.fail(e.getMessage()); + fail(e.getMessage()); } ReservationSystemTestUtil.initializeRMContext(10, cs, mockRMContext); return cs; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMReconnect.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMReconnect.java index f5a3bf85bcc04..9ceb8af3b2cc6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMReconnect.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMReconnect.java @@ -19,6 +19,9 @@ package org.apache.hadoop.yarn.server.resourcemanager.resourcetracker; 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 java.io.IOException; import java.util.ArrayList; @@ -56,10 +59,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM; import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager; 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.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; /** * TestNMReconnect run tests against the scheduler set by @@ -74,8 +77,9 @@ public class TestNMReconnect extends ParameterizedSchedulerTestBase { private Dispatcher dispatcher; private RMContextImpl context; - public TestNMReconnect(SchedulerType type) throws IOException { - super(type); + public void initTestNMReconnect(SchedulerType type) throws IOException { + initParameterizedSchedulerTestBase(type); + setUp(); } private class TestRMNodeEventDispatcher implements @@ -90,7 +94,6 @@ public void handle(RMNodeEvent event) { ResourceTrackerService resourceTrackerService; - @Before public void setUp() { Configuration conf = new Configuration(); // Dispatcher that processes events inline @@ -125,13 +128,15 @@ public void setUp() { resourceTrackerService.start(); } - @After + @AfterEach public void tearDown() { resourceTrackerService.stop(); } - @Test - public void testReconnect() throws Exception { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testReconnect(SchedulerType type) throws Exception { + initTestNMReconnect(type); String hostname1 = "localhost1"; Resource capability = Resources.createResource(1024); @@ -143,23 +148,25 @@ public void testReconnect() throws Exception { request1.setResource(capability); resourceTrackerService.registerNodeManager(request1); - Assert.assertEquals(RMNodeEventType.STARTED, rmNodeEvents.get(0).getType()); + assertEquals(RMNodeEventType.STARTED, rmNodeEvents.get(0).getType()); rmNodeEvents.clear(); resourceTrackerService.registerNodeManager(request1); - Assert.assertEquals(RMNodeEventType.RECONNECTED, + assertEquals(RMNodeEventType.RECONNECTED, rmNodeEvents.get(0).getType()); rmNodeEvents.clear(); resourceTrackerService.registerNodeManager(request1); capability = Resources.createResource(1024, 2); request1.setResource(capability); - Assert.assertEquals(RMNodeEventType.RECONNECTED, + assertEquals(RMNodeEventType.RECONNECTED, rmNodeEvents.get(0).getType()); } - @Test - public void testCompareRMNodeAfterReconnect() throws Exception { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testCompareRMNodeAfterReconnect(SchedulerType type) throws Exception { + initTestNMReconnect(type); AbstractYarnScheduler scheduler = getScheduler(); Configuration yarnConf = new YarnConfiguration(); ConfigurationProvider configurationProvider = @@ -189,31 +196,34 @@ public void testCompareRMNodeAfterReconnect() throws Exception { request1.setResource(capability); request1.setNodeStatus(mockNodeStatus); resourceTrackerService.registerNodeManager(request1); - Assert.assertNotNull(context.getRMNodes().get(nodeId1)); + assertNotNull(context.getRMNodes().get(nodeId1)); // verify Scheduler and RMContext use same RMNode reference. - Assert.assertTrue(scheduler.getSchedulerNode(nodeId1).getRMNode() == + assertTrue(scheduler.getSchedulerNode(nodeId1).getRMNode() == context.getRMNodes().get(nodeId1)); - Assert.assertEquals(context.getRMNodes().get(nodeId1). + assertEquals(context.getRMNodes().get(nodeId1). getTotalCapability(), capability); Resource capability1 = Resources.createResource(2048, 2); request1.setResource(capability1); resourceTrackerService.registerNodeManager(request1); - Assert.assertNotNull(context.getRMNodes().get(nodeId1)); + assertNotNull(context.getRMNodes().get(nodeId1)); // verify Scheduler and RMContext use same RMNode reference // after reconnect. - Assert.assertTrue(scheduler.getSchedulerNode(nodeId1).getRMNode() == + assertTrue(scheduler.getSchedulerNode(nodeId1).getRMNode() == context.getRMNodes().get(nodeId1)); // verify RMNode's capability is changed. - Assert.assertEquals(context.getRMNodes().get(nodeId1). + assertEquals(context.getRMNodes().get(nodeId1). getTotalCapability(), capability1); nlm.stop(); scheduler.stop(); } @SuppressWarnings("unchecked") - @Test(timeout = 10000) - public void testDecommissioningNodeReconnect() - throws Exception { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(10) + public void testDecommissioningNodeReconnect(SchedulerType type) throws Exception { + initTestNMReconnect(type); + MockRM rm = new MockRM(); rm.start(); MockNM nm1 = @@ -230,12 +240,15 @@ public void testDecommissioningNodeReconnect() new MockNM("127.0.0.1:1234", 15120, rm.getResourceTrackerService()); RegisterNodeManagerResponse response = nm2.registerNode(); // not SHUTDOWN - Assert.assertTrue(response.getNodeAction().equals(NodeAction.NORMAL)); + assertTrue(response.getNodeAction().equals(NodeAction.NORMAL)); rm.stop(); } - @Test(timeout = 10000) - public void testRMNodeStatusAfterReconnect() throws Exception { + @Timeout(10) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testRMNodeStatusAfterReconnect(SchedulerType type) throws Exception { + initTestNMReconnect(type); // The node(127.0.0.1:1234) reconnected with RM. When it registered with // RM, RM set its lastNodeHeartbeatResponse's id to 0 asynchronously. But // the node's heartbeat come before RM succeeded setting the id to 0. @@ -258,8 +271,7 @@ public void testRMNodeStatusAfterReconnect() throws Exception { RMNode rmNode = rm.getRMContext().getRMNodes().get(nm2.getNodeId()); nm2.nodeHeartbeat(true); rm.drainEvents(); - Assert.assertEquals("Node is Not in Running state.", NodeState.RUNNING, - rmNode.getState()); + assertEquals(NodeState.RUNNING, rmNode.getState(), "Node is Not in Running state."); 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/scheduler/TestAbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAbstractYarnScheduler.java index e71029b5a1b04..d6baacebd47ab 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAbstractYarnScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAbstractYarnScheduler.java @@ -20,7 +20,12 @@ import static org.apache.hadoop.yarn.server.resourcemanager.MockNM.createMockNodeStatus; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils.createResourceRequest; -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.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.Mockito.mock; import static org.mockito.Mockito.when; @@ -94,19 +99,21 @@ import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.Assert; -import org.junit.Test; -import org.mockito.Mockito; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; @SuppressWarnings("unchecked") public class TestAbstractYarnScheduler extends ParameterizedSchedulerTestBase { - public TestAbstractYarnScheduler(SchedulerType type) throws IOException { - super(type); + public void initTestAbstractYarnScheduler(SchedulerType type) throws IOException { + initParameterizedSchedulerTestBase(type); } - @Test - public void testMaximimumAllocationMemory() throws Exception { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testMaximimumAllocationMemory(SchedulerType type) throws Exception { + initTestAbstractYarnScheduler(type); final int node1MaxMemory = 15 * 1024; final int node2MaxMemory = 5 * 1024; final int node3MaxMemory = 6 * 1024; @@ -150,49 +157,51 @@ private void testMaximumAllocationMemoryHelper( final int node1MaxMemory, final int node2MaxMemory, final int node3MaxMemory, final int... expectedMaxMemory) throws Exception { - Assert.assertEquals(6, expectedMaxMemory.length); + assertEquals(6, expectedMaxMemory.length); - Assert.assertEquals(0, scheduler.getNumClusterNodes()); + assertEquals(0, scheduler.getNumClusterNodes()); long maxMemory = scheduler.getMaximumResourceCapability().getMemorySize(); - Assert.assertEquals(expectedMaxMemory[0], maxMemory); + assertEquals(expectedMaxMemory[0], maxMemory); RMNode node1 = MockNodes.newNodeInfo( 0, Resources.createResource(node1MaxMemory), 1, "127.0.0.2"); scheduler.handle(new NodeAddedSchedulerEvent(node1)); - Assert.assertEquals(1, scheduler.getNumClusterNodes()); + assertEquals(1, scheduler.getNumClusterNodes()); maxMemory = scheduler.getMaximumResourceCapability().getMemorySize(); - Assert.assertEquals(expectedMaxMemory[1], maxMemory); + assertEquals(expectedMaxMemory[1], maxMemory); scheduler.handle(new NodeRemovedSchedulerEvent(node1)); - Assert.assertEquals(0, scheduler.getNumClusterNodes()); + assertEquals(0, scheduler.getNumClusterNodes()); maxMemory = scheduler.getMaximumResourceCapability().getMemorySize(); - Assert.assertEquals(expectedMaxMemory[2], maxMemory); + assertEquals(expectedMaxMemory[2], maxMemory); RMNode node2 = MockNodes.newNodeInfo( 0, Resources.createResource(node2MaxMemory), 2, "127.0.0.3"); scheduler.handle(new NodeAddedSchedulerEvent(node2)); - Assert.assertEquals(1, scheduler.getNumClusterNodes()); + assertEquals(1, scheduler.getNumClusterNodes()); maxMemory = scheduler.getMaximumResourceCapability().getMemorySize(); - Assert.assertEquals(expectedMaxMemory[3], maxMemory); + assertEquals(expectedMaxMemory[3], maxMemory); RMNode node3 = MockNodes.newNodeInfo( 0, Resources.createResource(node3MaxMemory), 3, "127.0.0.4"); scheduler.handle(new NodeAddedSchedulerEvent(node3)); - Assert.assertEquals(2, scheduler.getNumClusterNodes()); + assertEquals(2, scheduler.getNumClusterNodes()); maxMemory = scheduler.getMaximumResourceCapability().getMemorySize(); - Assert.assertEquals(expectedMaxMemory[4], maxMemory); + assertEquals(expectedMaxMemory[4], maxMemory); scheduler.handle(new NodeRemovedSchedulerEvent(node3)); - Assert.assertEquals(1, scheduler.getNumClusterNodes()); + assertEquals(1, scheduler.getNumClusterNodes()); maxMemory = scheduler.getMaximumResourceCapability().getMemorySize(); - Assert.assertEquals(expectedMaxMemory[5], maxMemory); + assertEquals(expectedMaxMemory[5], maxMemory); scheduler.handle(new NodeRemovedSchedulerEvent(node2)); - Assert.assertEquals(0, scheduler.getNumClusterNodes()); + assertEquals(0, scheduler.getNumClusterNodes()); } - @Test - public void testMaximimumAllocationVCores() throws Exception { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testMaximimumAllocationVCores(SchedulerType type) throws Exception { + initTestAbstractYarnScheduler(type); final int node1MaxVCores = 15; final int node2MaxVCores = 5; final int node3MaxVCores = 6; @@ -236,52 +245,54 @@ private void testMaximumAllocationVCoresHelper( final int node1MaxVCores, final int node2MaxVCores, final int node3MaxVCores, final int... expectedMaxVCores) throws Exception { - Assert.assertEquals(6, expectedMaxVCores.length); + assertEquals(6, expectedMaxVCores.length); - Assert.assertEquals(0, scheduler.getNumClusterNodes()); + assertEquals(0, scheduler.getNumClusterNodes()); int maxVCores = scheduler.getMaximumResourceCapability().getVirtualCores(); - Assert.assertEquals(expectedMaxVCores[0], maxVCores); + assertEquals(expectedMaxVCores[0], maxVCores); RMNode node1 = MockNodes.newNodeInfo( 0, Resources.createResource(1024, node1MaxVCores), 1, "127.0.0.2"); scheduler.handle(new NodeAddedSchedulerEvent(node1)); - Assert.assertEquals(1, scheduler.getNumClusterNodes()); + assertEquals(1, scheduler.getNumClusterNodes()); maxVCores = scheduler.getMaximumResourceCapability().getVirtualCores(); - Assert.assertEquals(expectedMaxVCores[1], maxVCores); + assertEquals(expectedMaxVCores[1], maxVCores); scheduler.handle(new NodeRemovedSchedulerEvent(node1)); - Assert.assertEquals(0, scheduler.getNumClusterNodes()); + assertEquals(0, scheduler.getNumClusterNodes()); maxVCores = scheduler.getMaximumResourceCapability().getVirtualCores(); - Assert.assertEquals(expectedMaxVCores[2], maxVCores); + assertEquals(expectedMaxVCores[2], maxVCores); RMNode node2 = MockNodes.newNodeInfo( 0, Resources.createResource(1024, node2MaxVCores), 2, "127.0.0.3"); scheduler.handle(new NodeAddedSchedulerEvent(node2)); - Assert.assertEquals(1, scheduler.getNumClusterNodes()); + assertEquals(1, scheduler.getNumClusterNodes()); maxVCores = scheduler.getMaximumResourceCapability().getVirtualCores(); - Assert.assertEquals(expectedMaxVCores[3], maxVCores); + assertEquals(expectedMaxVCores[3], maxVCores); RMNode node3 = MockNodes.newNodeInfo( 0, Resources.createResource(1024, node3MaxVCores), 3, "127.0.0.4"); scheduler.handle(new NodeAddedSchedulerEvent(node3)); - Assert.assertEquals(2, scheduler.getNumClusterNodes()); + assertEquals(2, scheduler.getNumClusterNodes()); maxVCores = scheduler.getMaximumResourceCapability().getVirtualCores(); - Assert.assertEquals(expectedMaxVCores[4], maxVCores); + assertEquals(expectedMaxVCores[4], maxVCores); scheduler.handle(new NodeRemovedSchedulerEvent(node3)); - Assert.assertEquals(1, scheduler.getNumClusterNodes()); + assertEquals(1, scheduler.getNumClusterNodes()); maxVCores = scheduler.getMaximumResourceCapability().getVirtualCores(); - Assert.assertEquals(expectedMaxVCores[5], maxVCores); + assertEquals(expectedMaxVCores[5], maxVCores); scheduler.handle(new NodeRemovedSchedulerEvent(node2)); - Assert.assertEquals(0, scheduler.getNumClusterNodes()); + assertEquals(0, scheduler.getNumClusterNodes()); } /** * Test for testing autocorrect container allocation feature. */ - @Test - public void testAutoCorrectContainerAllocation() { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testAutoCorrectContainerAllocation(SchedulerType type) throws IOException { + initTestAbstractYarnScheduler(type); Configuration conf = new Configuration(getConf()); conf.setBoolean(YarnConfiguration.RM_SCHEDULER_AUTOCORRECT_CONTAINER_ALLOCATION, true); conf.setBoolean("yarn.scheduler.capacity.root.auto-create-child-queue.enabled", @@ -623,8 +634,10 @@ private void testContainerAskFourAndNewlyAllocatedContainerSix(AbstractYarnSched assertEquals(4, application.pullNewlyAllocatedContainers().size()); } - @Test - public void testUpdateMaxAllocationUsesTotal() throws IOException { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testUpdateMaxAllocationUsesTotal(SchedulerType type) throws IOException { + initTestAbstractYarnScheduler(type); final int configuredMaxVCores = 20; final int configuredMaxMemory = 10 * 1024; Resource configuredMaximumResource = Resource.newInstance @@ -677,8 +690,10 @@ public void testUpdateMaxAllocationUsesTotal() throws IOException { } } - @Test - public void testMaxAllocationAfterUpdateNodeResource() throws IOException { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testMaxAllocationAfterUpdateNodeResource(SchedulerType type) throws IOException { + initTestAbstractYarnScheduler(type); final int configuredMaxVCores = 20; final int configuredMaxMemory = 10 * 1024; Resource configuredMaximumResource = Resource.newInstance @@ -743,8 +758,11 @@ public void testMaxAllocationAfterUpdateNodeResource() throws IOException { * null (no attempt). */ @SuppressWarnings({ "rawtypes" }) - @Test(timeout = 10000) - public void testReleasedContainerIfAppAttemptisNull() throws Exception { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(10) + public void testReleasedContainerIfAppAttemptisNull(SchedulerType type) throws Exception { + initTestAbstractYarnScheduler(type); YarnConfiguration conf=getConf(); MockRM rm1 = new MockRM(conf); try { @@ -777,9 +795,9 @@ public void testReleasedContainerIfAppAttemptisNull() throws Exception { scheduler.clearPendingContainerCache(); - Assert.assertEquals("Pending containers are not released " - + "when one of the application attempt is null !", schedulerApp - .getCurrentAppAttempt().getPendingRelease().size(), 0); + assertEquals(schedulerApp.getCurrentAppAttempt().getPendingRelease().size(), + 0, "Pending containers are not released " + + "when one of the application attempt is null !"); } finally { if (rm1 != null) { rm1.stop(); @@ -787,8 +805,11 @@ public void testReleasedContainerIfAppAttemptisNull() throws Exception { } } - @Test(timeout = 30000l) - public void testContainerReleaseWithAllocationTags() throws Exception { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(30) + public void testContainerReleaseWithAllocationTags(SchedulerType type) throws Exception { + initTestAbstractYarnScheduler(type); // Currently only can be tested against capacity scheduler. if (getSchedulerType().equals(SchedulerType.CAPACITY)) { final String testTag1 = "some-tag"; @@ -847,13 +868,13 @@ public void testContainerReleaseWithAllocationTags() throws Exception { Thread.sleep(1000); } - Assert.assertEquals(4, allocated.size()); + assertEquals(4, allocated.size()); Set containers = allocated.stream() .filter(container -> container.getAllocationRequestId() == 1l) .collect(Collectors.toSet()); - Assert.assertNotNull(containers); - Assert.assertEquals(1, containers.size()); + assertNotNull(containers); + assertEquals(1, containers.size()); ContainerId cid = containers.iterator().next().getId(); // mock container start @@ -864,8 +885,8 @@ public void testContainerReleaseWithAllocationTags() throws Exception { Map nodeTags = rm1.getRMContext() .getAllocationTagsManager() .getAllocationTagsWithCount(nm1.getNodeId()); - Assert.assertNotNull(nodeTags.get(testTag1)); - Assert.assertEquals(1, nodeTags.get(testTag1).intValue()); + assertNotNull(nodeTags.get(testTag1)); + assertEquals(1, nodeTags.get(testTag1).intValue()); // release a container am1.allocate(new ArrayList<>(), Lists.newArrayList(cid)); @@ -873,9 +894,9 @@ public void testContainerReleaseWithAllocationTags() throws Exception { // before NM confirms, the tag should still exist nodeTags = rm1.getRMContext().getAllocationTagsManager() .getAllocationTagsWithCount(nm1.getNodeId()); - Assert.assertNotNull(nodeTags); - Assert.assertNotNull(nodeTags.get(testTag1)); - Assert.assertEquals(1, nodeTags.get(testTag1).intValue()); + assertNotNull(nodeTags); + assertNotNull(nodeTags.get(testTag1)); + assertEquals(1, nodeTags.get(testTag1).intValue()); // NM reports back that container is released // RM should cleanup the tag @@ -897,8 +918,11 @@ public void testContainerReleaseWithAllocationTags() throws Exception { } - @Test(timeout = 30000L) - public void testNodeRemovedWithAllocationTags() throws Exception { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(30) + public void testNodeRemovedWithAllocationTags(SchedulerType type) throws Exception { + initTestAbstractYarnScheduler(type); // Currently only can be tested against capacity scheduler. if (getSchedulerType().equals(SchedulerType.CAPACITY)) { final String testTag1 = "some-tag"; @@ -948,13 +972,13 @@ public void testNodeRemovedWithAllocationTags() throws Exception { Thread.sleep(1000); } - Assert.assertEquals(1, allocated.size()); + assertEquals(1, allocated.size()); Set containers = allocated.stream() .filter(container -> container.getAllocationRequestId() == 1L) .collect(Collectors.toSet()); - Assert.assertNotNull(containers); - Assert.assertEquals(1, containers.size()); + assertNotNull(containers); + assertEquals(1, containers.size()); ContainerId cid = containers.iterator().next().getId(); // mock container start @@ -965,8 +989,8 @@ public void testNodeRemovedWithAllocationTags() throws Exception { Map nodeTags = rm1.getRMContext() .getAllocationTagsManager() .getAllocationTagsWithCount(nm1.getNodeId()); - Assert.assertNotNull(nodeTags.get(testTag1)); - Assert.assertEquals(1, nodeTags.get(testTag1).intValue()); + assertNotNull(nodeTags.get(testTag1)); + assertEquals(1, nodeTags.get(testTag1).intValue()); // remove the node RMNode node1 = MockNodes.newNodeInfo( @@ -977,13 +1001,15 @@ public void testNodeRemovedWithAllocationTags() throws Exception { // Once the node is removed, the tag should be removed immediately nodeTags = rm1.getRMContext().getAllocationTagsManager() .getAllocationTagsWithCount(nm1.getNodeId()); - Assert.assertNull(nodeTags); + assertNull(nodeTags); } } - - @Test(timeout=60000) - public void testContainerReleasedByNode() throws Exception { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(60) + public void testContainerReleasedByNode(SchedulerType type) throws Exception { + initTestAbstractYarnScheduler(type); System.out.println("Starting testContainerReleasedByNode"); YarnConfiguration conf = getConf(); MockRM rm1 = new MockRM(conf); @@ -1075,15 +1101,15 @@ public void testContainerReleasedByNode() throws Exception { containers = am1.allocate("127.0.0.1", 8192, 1, new ArrayList()).getAllocatedContainers(); nm1.nodeHeartbeat(true); - Assert.assertTrue("new container allocated before node freed old", - containers.isEmpty()); + assertTrue(containers.isEmpty(), + "new container allocated before node freed old"); for (int i = 0; i < 10; ++i) { Thread.sleep(10); containers = am1.allocate(new ArrayList(), new ArrayList()).getAllocatedContainers(); nm1.nodeHeartbeat(true); - Assert.assertTrue("new container allocated before node freed old", - containers.isEmpty()); + assertTrue(containers.isEmpty(), + "new container allocated before node freed old"); } // free the old container from the node @@ -1105,9 +1131,12 @@ public void testContainerReleasedByNode() throws Exception { } } - @Test(timeout = 60000) - public void testResourceRequestRestoreWhenRMContainerIsAtAllocated() + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(60) + public void testResourceRequestRestoreWhenRMContainerIsAtAllocated(SchedulerType type) throws Exception { + initTestAbstractYarnScheduler(type); YarnConfiguration conf = getConf(); MockRM rm1 = new MockRM(conf); try { @@ -1203,10 +1232,11 @@ public void testResourceRequestRestoreWhenRMContainerIsAtAllocated() * * @throws Exception */ - @Test - public void testResourceRequestRecoveryToTheRightAppAttempt() + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testResourceRequestRecoveryToTheRightAppAttempt(SchedulerType type) throws Exception { - + initTestAbstractYarnScheduler(type); YarnConfiguration conf = getConf(); MockRM rm = new MockRM(conf); try { @@ -1280,11 +1310,11 @@ public void testResourceRequestRecoveryToTheRightAppAttempt() RMAppAttempt rmAppAttempt2 = MockRM.waitForAttemptScheduled(rmApp, rm); ApplicationAttemptId applicationAttemptTwoID = rmAppAttempt2.getAppAttemptId(); - Assert.assertEquals(2, applicationAttemptTwoID.getAttemptId()); + assertEquals(2, applicationAttemptTwoID.getAttemptId()); // All outstanding allocated containers will be killed (irrespective of // keep-alive of container across app-attempts) - Assert.assertEquals(RMContainerState.KILLED, + assertEquals(RMContainerState.KILLED, allocatedContainer.getState()); // The core part of this test @@ -1292,11 +1322,11 @@ public void testResourceRequestRecoveryToTheRightAppAttempt() // original app-attempt, not the new one for (SchedulerRequestKey key : firstSchedulerAppAttempt.getSchedulerKeys()) { if (key.getPriority().getPriority() == 0) { - Assert.assertEquals(0, + assertEquals(0, firstSchedulerAppAttempt.getOutstandingAsksCount(key)); } else if (key.getPriority().getPriority() == ALLOCATED_CONTAINER_PRIORITY) { - Assert.assertEquals(1, + assertEquals(1, firstSchedulerAppAttempt.getOutstandingAsksCount(key)); } } @@ -1307,8 +1337,8 @@ public void testResourceRequestRecoveryToTheRightAppAttempt() List transferredContainers = rm.getResourceScheduler().getTransferredContainers( applicationAttemptTwoID); - Assert.assertEquals(1, transferredContainers.size()); - Assert.assertEquals(runningContainerID, transferredContainers.get(0) + assertEquals(1, transferredContainers.size()); + assertEquals(runningContainerID, transferredContainers.get(0) .getId()); } finally { @@ -1321,9 +1351,9 @@ private void verifyMaximumResourceCapability( final Resource schedulerMaximumResourceCapability = scheduler .getMaximumResourceCapability(); - Assert.assertEquals(expectedMaximumResource.getMemorySize(), + assertEquals(expectedMaximumResource.getMemorySize(), schedulerMaximumResourceCapability.getMemorySize()); - Assert.assertEquals(expectedMaximumResource.getVirtualCores(), + assertEquals(expectedMaximumResource.getVirtualCores(), schedulerMaximumResourceCapability.getVirtualCores()); } @@ -1349,7 +1379,7 @@ private ResourceTrackerService getPrivateResourceTrackerService( RMContext privateContext = new RMContextImpl(privateDispatcher, null, null, null, null, null, null, null, null, null); - privateContext.setNodeLabelManager(Mockito.mock(RMNodeLabelsManager.class)); + privateContext.setNodeLabelManager(mock(RMNodeLabelsManager.class)); privateDispatcher.register(SchedulerEventType.class, sleepHandler); privateDispatcher.register(SchedulerEventType.class, @@ -1386,8 +1416,11 @@ private ResourceTrackerService getPrivateResourceTrackerService( * that might occur due to the use of the RMNode object. * @throws Exception */ - @Test(timeout = 60000) - public void testNodemanagerReconnect() throws Exception { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(60) + public void testNodemanagerReconnect(SchedulerType type) throws Exception { + initTestAbstractYarnScheduler(type); Configuration conf = getConf(); MockRM rm = new MockRM(conf); try { @@ -1418,8 +1451,8 @@ public void testNodemanagerReconnect() throws Exception { privateDispatcher.await(); Resource clusterResource = rm.getResourceScheduler().getClusterResource(); - Assert.assertEquals("Initial cluster resources don't match", capability, - clusterResource); + assertEquals(capability, + clusterResource, "Initial cluster resources don't match"); Resource newCapability = Resources.createResource(1024); RegisterNodeManagerRequest request2 = @@ -1431,16 +1464,19 @@ public void testNodemanagerReconnect() throws Exception { sleepHandler.sleepFlag = true; privateResourceTrackerService.registerNodeManager(request2); privateDispatcher.await(); - Assert.assertEquals("Cluster resources don't match", newCapability, - rm.getResourceScheduler().getClusterResource()); + assertEquals(newCapability, rm.getResourceScheduler().getClusterResource(), + "Cluster resources don't match"); privateResourceTrackerService.stop(); } finally { rm.stop(); } } - @Test(timeout = 10000) - public void testUpdateThreadLifeCycle() throws Exception { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(10) + public void testUpdateThreadLifeCycle(SchedulerType type) throws Exception { + initTestAbstractYarnScheduler(type); MockRM rm = new MockRM(getConf()); try { rm.start(); @@ -1449,7 +1485,7 @@ public void testUpdateThreadLifeCycle() throws Exception { if (getSchedulerType().equals(SchedulerType.FAIR)) { Thread updateThread = scheduler.updateThread; - Assert.assertTrue(updateThread.isAlive()); + assertTrue(updateThread.isAlive()); scheduler.stop(); int numRetries = 100; @@ -1457,12 +1493,12 @@ public void testUpdateThreadLifeCycle() throws Exception { Thread.sleep(50); } - Assert.assertNotEquals("The Update thread is still alive", 0, numRetries); + assertNotEquals(0, numRetries, "The Update thread is still alive"); } else if (getSchedulerType().equals(SchedulerType.CAPACITY)) { - Assert.assertNull("updateThread shouldn't have been created", - scheduler.updateThread); + assertNull(scheduler.updateThread, + "updateThread shouldn't have been created"); } else { - Assert.fail("Unhandled SchedulerType, " + getSchedulerType() + + fail("Unhandled SchedulerType, " + getSchedulerType() + ", please update this unit test."); } } finally { @@ -1470,8 +1506,11 @@ public void testUpdateThreadLifeCycle() throws Exception { } } - @Test(timeout=60000) - public void testContainerRecoveredByNode() throws Exception { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(60) + public void testContainerRecoveredByNode(SchedulerType type) throws Exception { + initTestAbstractYarnScheduler(type); System.out.println("Starting testContainerRecoveredByNode"); final int maxMemory = 10 * 1024; YarnConfiguration conf = getConf(); @@ -1521,11 +1560,11 @@ public void testContainerRecoveredByNode() throws Exception { //verify queue name when rmContainer is recovered if (scheduler instanceof CapacityScheduler) { - Assert.assertEquals( + assertEquals( app1.getQueue(), rmContainer.getQueueName()); } else { - Assert.assertEquals(app1.getQueue(), rmContainer.getQueueName()); + assertEquals(app1.getQueue(), rmContainer.getQueueName()); } } finally { @@ -1538,8 +1577,10 @@ public void testContainerRecoveredByNode() throws Exception { * Test the order we get the containers to kill. It should respect the order * described in {@link SchedulerNode#getContainersToKill()}. */ - @Test - public void testGetRunningContainersToKill() { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testGetRunningContainersToKill(SchedulerType type) throws IOException { + initTestAbstractYarnScheduler(type); final SchedulerNode node = new MockSchedulerNode(); assertEquals(Collections.emptyList(), node.getContainersToKill()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulingWithAllocationRequestId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulingWithAllocationRequestId.java index db2a576eb7513..f8b74e16bf52f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulingWithAllocationRequestId.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulingWithAllocationRequestId.java @@ -18,6 +18,8 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler; +import static org.junit.jupiter.api.Assertions.assertEquals; + import java.io.IOException; import java.util.List; @@ -34,8 +36,9 @@ 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.scheduler.fair.FairSchedulerConfiguration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,8 +53,8 @@ public class TestSchedulingWithAllocationRequestId LoggerFactory.getLogger(TestSchedulingWithAllocationRequestId.class); private static final int GB = 1024; - public TestSchedulingWithAllocationRequestId(SchedulerType type) throws IOException { - super(type); + public void initTestSchedulingWithAllocationRequestId(SchedulerType type) throws IOException { + initParameterizedSchedulerTestBase(type); } @Override @@ -65,8 +68,11 @@ public YarnConfiguration getConf() { return conf; } - @Test (timeout = 10000) - public void testMultipleAllocationRequestIds() throws Exception { + @Timeout(10) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testMultipleAllocationRequestIds(SchedulerType type) throws Exception { + initParameterizedSchedulerTestBase(type); YarnConfiguration conf = getConf(); MockRM rm = new MockRM(conf); try { @@ -90,13 +96,13 @@ public void testMultipleAllocationRequestIds() throws Exception { // check if request id 10 is satisfied AllocateResponse allocResponse = waitForAllocResponse(rm, am1, nm1, 1); List allocated = allocResponse.getAllocatedContainers(); - Assert.assertEquals(1, allocated.size()); + assertEquals(1, allocated.size()); checkAllocatedContainer(allocated.get(0), 2 * GB, nm1.getNodeId(), 10); // check now if request id 20 is satisfied allocResponse = waitForAllocResponse(rm, am1, nm2, 2); allocated = allocResponse.getAllocatedContainers(); - Assert.assertEquals(2, allocated.size()); + assertEquals(2, allocated.size()); for (Container container : allocated) { checkAllocatedContainer(container, 2 * GB, nm2.getNodeId(), 20); } @@ -107,8 +113,11 @@ public void testMultipleAllocationRequestIds() throws Exception { } } - @Test (timeout = 10000) - public void testMultipleAllocationRequestDiffPriority() throws Exception { + @Timeout(10) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testMultipleAllocationRequestDiffPriority(SchedulerType type) throws Exception { + initTestSchedulingWithAllocationRequestId(type); YarnConfiguration conf = getConf(); MockRM rm = new MockRM(conf); try { @@ -132,7 +141,7 @@ public void testMultipleAllocationRequestDiffPriority() throws Exception { // check if request id 20 is satisfied first AllocateResponse allocResponse = waitForAllocResponse(rm, am1, nm2, 2); List allocated = allocResponse.getAllocatedContainers(); - Assert.assertEquals(2, allocated.size()); + assertEquals(2, allocated.size()); for (Container container : allocated) { checkAllocatedContainer(container, 2 * GB, nm2.getNodeId(), 20); } @@ -140,7 +149,7 @@ public void testMultipleAllocationRequestDiffPriority() throws Exception { // check now if request id 10 is satisfied allocResponse = waitForAllocResponse(rm, am1, nm1, 1); allocated = allocResponse.getAllocatedContainers(); - Assert.assertEquals(1, allocated.size()); + assertEquals(1, allocated.size()); checkAllocatedContainer(allocated.get(0), 2 * GB, nm1.getNodeId(), 10); } finally { if (rm != null) { @@ -151,14 +160,17 @@ public void testMultipleAllocationRequestDiffPriority() throws Exception { private void checkAllocatedContainer(Container allocated, int memory, NodeId nodeId, long allocationRequestId) { - Assert.assertEquals(memory, allocated.getResource().getMemorySize()); - Assert.assertEquals(nodeId, allocated.getNodeId()); - Assert.assertEquals(allocationRequestId, + assertEquals(memory, allocated.getResource().getMemorySize()); + assertEquals(nodeId, allocated.getNodeId()); + assertEquals(allocationRequestId, allocated.getAllocationRequestId()); } - @Test (timeout = 10000) - public void testMultipleAppsWithAllocationReqId() throws Exception { + @Timeout(10) + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testMultipleAppsWithAllocationReqId(SchedulerType type) throws Exception { + initTestSchedulingWithAllocationRequestId(type); YarnConfiguration conf = getConf(); MockRM rm = new MockRM(conf); try { @@ -193,7 +205,7 @@ public void testMultipleAppsWithAllocationReqId() throws Exception { // wait for container to be allocated. AllocateResponse allocResponse = waitForAllocResponse(rm, am1, nm1, 1); List allocated = allocResponse.getAllocatedContainers(); - Assert.assertEquals(1, allocated.size()); + assertEquals(1, allocated.size()); checkAllocatedContainer(allocated.get(0), 1 * GB, nm1.getNodeId(), 5L); // Submit another application @@ -212,7 +224,7 @@ public void testMultipleAppsWithAllocationReqId() throws Exception { // wait for container to be allocated. allocResponse = waitForAllocResponse(rm, am2, nm2, 1); allocated = allocResponse.getAllocatedContainers(); - Assert.assertEquals(1, allocated.size()); + assertEquals(1, allocated.size()); checkAllocatedContainer(allocated.get(0), 2 * GB, nm2.getNodeId(), 5L); // Now submit app2 RR with allocationReqId = 10 @@ -222,7 +234,7 @@ public void testMultipleAppsWithAllocationReqId() throws Exception { // wait for container to be allocated. allocResponse = waitForAllocResponse(rm, am2, nm1, 1); allocated = allocResponse.getAllocatedContainers(); - Assert.assertEquals(1, allocated.size()); + assertEquals(1, allocated.size()); checkAllocatedContainer(allocated.get(0), 3 * GB, nm1.getNodeId(), 10L); // Now submit app1 RR with allocationReqId = 10 @@ -232,7 +244,7 @@ public void testMultipleAppsWithAllocationReqId() throws Exception { // wait for container to be allocated. allocResponse = waitForAllocResponse(rm, am1, nm2, 1); allocated = allocResponse.getAllocatedContainers(); - Assert.assertEquals(1, allocated.size()); + assertEquals(1, allocated.size()); checkAllocatedContainer(allocated.get(0), 4 * GB, nm2.getNodeId(), 10L); } 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/security/TestClientToAMTokens.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientToAMTokens.java index 03b68944b481f..1f9b5bb03da8b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientToAMTokens.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientToAMTokens.java @@ -65,9 +65,9 @@ import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.util.Records; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import javax.security.sasl.SaslException; import java.io.IOException; @@ -79,7 +79,11 @@ import java.util.Timer; import java.util.TimerTask; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -87,11 +91,11 @@ public class TestClientToAMTokens extends ParameterizedSchedulerTestBase { private YarnConfiguration conf; - public TestClientToAMTokens(SchedulerType type) throws IOException { - super(type); + public void initTestClientToAMTokens(SchedulerType type) throws IOException { + initParameterizedSchedulerTestBase(type); + setup(); } - - @Before + public void setup() { conf = getConf(); } @@ -189,8 +193,10 @@ public void setClientSecretKey(byte[] key) { } } - @Test - public void testClientToAMTokens() throws Exception { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + public void testClientToAMTokens(SchedulerType type) throws Exception { + initTestClientToAMTokens(type); conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos"); // Set RPC engine to protobuf RPC engine @@ -242,7 +248,7 @@ public RegisterApplicationMasterResponse run() { try { response = mockAM.registerAppAttempt(); } catch (Exception e) { - Assert.fail("Exception was not expected"); + fail("Exception was not expected"); } return response; } @@ -260,14 +266,13 @@ public RegisterApplicationMasterResponse run() { // ClientToAMToken master key should have been received on register // application master response. - Assert.assertNotNull(response.getClientToAMTokenMasterKey()); - Assert - .assertTrue(response.getClientToAMTokenMasterKey().array().length > 0); + assertNotNull(response.getClientToAMTokenMasterKey()); + assertTrue(response.getClientToAMTokenMasterKey().array().length > 0); // Start the AM with the correct shared-secret. ApplicationAttemptId appAttemptId = app.getAppAttempts().keySet().iterator().next(); - Assert.assertNotNull(appAttemptId); + assertNotNull(appAttemptId); final CustomAM am = new CustomAM(appAttemptId, response.getClientToAMTokenMasterKey() .array()); @@ -285,7 +290,7 @@ public RegisterApplicationMasterResponse run() { client.ping(null, TestRpcBase.newEmptyRequest()); fail("Access by unauthenticated user should fail!!"); } catch (Exception e) { - Assert.assertFalse(am.pinged); + assertFalse(am.pinged); } Token token = @@ -360,18 +365,15 @@ public Void run() throws Exception { } }); } catch (Exception e) { - Assert.assertEquals(RemoteException.class.getName(), e.getClass() + assertEquals(RemoteException.class.getName(), e.getClass() .getName()); e = ((RemoteException)e).unwrapRemoteException(); - Assert - .assertEquals(SaslException.class + assertEquals(SaslException.class .getCanonicalName(), e.getClass().getCanonicalName()); - Assert.assertTrue(e - .getMessage() - .contains( - "DIGEST-MD5: digest response format violation. " - + "Mismatched response.")); - Assert.assertFalse(am.pinged); + assertTrue(e + .getMessage() + .contains("DIGEST-MD5: digest response format violation. " + "Mismatched response.")); + assertFalse(am.pinged); } } @@ -395,7 +397,7 @@ public Void run() throws Exception { CustomProtocol client = RPC.getProxy(CustomProtocol.class, 1L, am.address, conf); client.ping(null, TestRpcBase.newEmptyRequest()); - Assert.assertTrue(am.pinged); + assertTrue(am.pinged); return null; } }); @@ -414,14 +416,18 @@ public Void run() throws Exception { CustomProtocol client = RPC.getProxy(CustomProtocol.class, 1L, am.address, conf); client.ping(null, TestRpcBase.newEmptyRequest()); - Assert.assertTrue(am.pinged); + assertTrue(am.pinged); return null; } }); } - @Test(timeout=20000) - public void testClientTokenRace() throws Exception { + @ParameterizedTest(name = "{0}") + @MethodSource("getParameters") + @Timeout(20) + public void testClientTokenRace(SchedulerType type) throws Exception { + + initTestClientToAMTokens(type); conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos"); @@ -472,7 +478,7 @@ public RegisterApplicationMasterResponse run() { try { response = mockAM.registerAppAttempt(); } catch (Exception e) { - Assert.fail("Exception was not expected"); + fail("Exception was not expected"); } return response; } @@ -491,13 +497,13 @@ public RegisterApplicationMasterResponse run() { // ClientToAMToken master key should have been received on register // application master response. final ByteBuffer clientMasterKey = response.getClientToAMTokenMasterKey(); - Assert.assertNotNull(clientMasterKey); - Assert.assertTrue(clientMasterKey.array().length > 0); + assertNotNull(clientMasterKey); + assertTrue(clientMasterKey.array().length > 0); // Start the AM with the correct shared-secret. ApplicationAttemptId appAttemptId = app.getAppAttempts().keySet().iterator().next(); - Assert.assertNotNull(appAttemptId); + assertNotNull(appAttemptId); final CustomAM am = new CustomAM(appAttemptId, null); am.init(conf); am.start(); From 9a7aea2866bc5fbaa78d9ca902ae5220442fcb44 Mon Sep 17 00:00:00 2001 From: slfan1989 Date: Sat, 8 Mar 2025 14:39:32 +0800 Subject: [PATCH 4/6] YARN-11262. Fix CheckStyle & Junit Test. --- .../TestWorkPreservingRMRestart.java | 4 +- .../TestWorkPreservingUnmanagedAM.java | 2 +- .../TestAMRMRPCResponseId.java | 17 +- .../TestFileSystemNodeAttributeStore.java | 45 +++--- .../nodelabels/TestNodeAttributesManager.java | 72 ++++----- .../TestNodeLabelFileReplication.java | 8 +- .../TestRMDelegatedNodeLabelsUpdater.java | 7 +- .../nodelabels/TestRMNodeLabelsManager.java | 148 +++++++++--------- 8 files changed, 148 insertions(+), 155 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java index beb6b1915cbee..2711e86f244da 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java @@ -86,12 +86,11 @@ import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; +import org.slf4j.event.Level; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; -import org.slf4j.event.Level; - import java.io.File; import java.io.IOException; import java.net.UnknownHostException; @@ -110,7 +109,6 @@ .capacity.TestCapacitySchedulerAutoCreatedQueueBase.USER1; import static org.apache.hadoop.yarn.server.resourcemanager.webapp .RMWebServices.DEFAULT_QUEUE; - import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingUnmanagedAM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingUnmanagedAM.java index 3ed7d0036455f..387b744e4c3f8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingUnmanagedAM.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingUnmanagedAM.java @@ -41,10 +41,10 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.InvalidApplicationMasterRequestException; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.slf4j.event.Level; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; -import org.slf4j.event.Level; /** * Test UAM handling in RM. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRMRPCResponseId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRMRPCResponseId.java index 0476abc1d1887..c4227333f36ba 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRMRPCResponseId.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRMRPCResponseId.java @@ -18,6 +18,10 @@ package org.apache.hadoop.yarn.server.resourcemanager.applicationsmanager; +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.security.PrivilegedExceptionAction; import org.apache.hadoop.security.UserGroupInformation; @@ -34,7 +38,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -93,26 +96,26 @@ public void testARRMResponseId() throws Exception { AllocateResponse response = allocate(attempt.getAppAttemptId(), allocateRequest); - Assertions.assertEquals(1, response.getResponseId()); - Assertions.assertTrue(response.getAMCommand() == null); + assertEquals(1, response.getResponseId()); + assertTrue(response.getAMCommand() == null); allocateRequest = AllocateRequest.newInstance(response.getResponseId(), 0F, null, null, null); response = allocate(attempt.getAppAttemptId(), allocateRequest); - Assertions.assertEquals(2, response.getResponseId()); + assertEquals(2, response.getResponseId()); /* try resending */ response = allocate(attempt.getAppAttemptId(), allocateRequest); - Assertions.assertEquals(2, response.getResponseId()); + assertEquals(2, response.getResponseId()); /** try sending old request again **/ allocateRequest = AllocateRequest.newInstance(0, 0F, null, null, null); try { allocate(attempt.getAppAttemptId(), allocateRequest); - Assertions.fail(); + fail(); } catch (Exception e) { - Assertions.assertTrue(e.getCause() instanceof InvalidApplicationMasterRequestException); + assertTrue(e.getCause() instanceof InvalidApplicationMasterRequestException); } } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java index e084db778ed50..c2138fe19f215 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java @@ -18,6 +18,8 @@ package org.apache.hadoop.yarn.server.resourcemanager.nodelabels; 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.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -33,7 +35,6 @@ import org.apache.hadoop.yarn.nodelabels.NodeAttributeStore; import org.apache.hadoop.yarn.server.resourcemanager.NodeAttributeTestUtils; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -129,8 +130,7 @@ public void testRecoverWithMirror() throws Exception { // Add node attribute mgr.addNodeAttributes(toAddAttributes); - Assertions.assertEquals(2 -, mgr.getAttributesForNode("host0").size(), "host0 size"); + assertEquals(2, mgr.getAttributesForNode("host0").size(), "host0 size"); // Add test to remove toAddAttributes.clear(); toAddAttributes.put("host0", ImmutableSet.of(gpu)); @@ -153,10 +153,8 @@ public void testRecoverWithMirror() throws Exception { mgr.start(); mgr.getAttributesForNode("host0"); - Assertions.assertEquals(1 -, mgr.getAttributesForNode("host0").size(), "host0 size"); - Assertions.assertEquals(1 -, mgr.getAttributesForNode("host1").size(), "host1 size"); + assertEquals(1, mgr.getAttributesForNode("host0").size(), "host0 size"); + assertEquals(1, mgr.getAttributesForNode("host1").size(), "host1 size"); attrs = mgr.getAttributesForNode("host0"); assertThat(attrs).hasSize(1); assertThat(attrs.keySet().toArray()[0]).isEqualTo(docker); @@ -180,16 +178,14 @@ public void testRecoverWithMirror() throws Exception { mgr = new MockNodeAttrbuteManager(); mgr.init(conf); mgr.start(); - Assertions.assertEquals(1 -, mgr.getAttributesForNode("host0").size(), "host0 size"); - Assertions.assertEquals(2 -, mgr.getAttributesForNode("host1").size(), "host1 size"); + assertEquals(1, mgr.getAttributesForNode("host0").size(), "host0 size"); + assertEquals(2, mgr.getAttributesForNode("host1").size(), "host1 size"); attrs = mgr.getAttributesForNode("host0"); assertThat(attrs).hasSize(1); assertThat(attrs.keySet().toArray()[0]).isEqualTo(gpu); attrs = mgr.getAttributesForNode("host1"); - Assertions.assertTrue(attrs.keySet().contains(docker)); - Assertions.assertTrue(attrs.keySet().contains(gpu)); + assertTrue(attrs.keySet().contains(docker)); + assertTrue(attrs.keySet().contains(gpu)); } @Test @@ -212,8 +208,7 @@ public void testRecoverFromEditLog() throws Exception { // Add node attribute mgr.addNodeAttributes(toAddAttributes); - Assertions.assertEquals(2 -, mgr.getAttributesForNode("host0").size(), "host0 size"); + assertEquals(2, mgr.getAttributesForNode("host0").size(), "host0 size"); // Increase editlog operation for (int i = 0; i < 5; i++) { @@ -245,10 +240,8 @@ public void testRecoverFromEditLog() throws Exception { mgr.init(conf); mgr.start(); - Assertions.assertEquals(1 -, mgr.getAttributesForNode("host0").size(), "host0 size"); - Assertions.assertEquals(2 -, mgr.getAttributesForNode("host1").size(), "host1 size"); + assertEquals(1, mgr.getAttributesForNode("host0").size(), "host0 size"); + assertEquals(2, mgr.getAttributesForNode("host1").size(), "host1 size"); toAddAttributes.clear(); NodeAttribute replaced = @@ -266,19 +259,17 @@ public void testRecoverFromEditLog() throws Exception { Map.Entry entry = valueMap.entrySet().iterator().next(); NodeAttribute attribute = entry.getKey(); - Assertions.assertEquals(1 -, mgr.getAttributesForNode("host0").size(), "host0 size"); - Assertions.assertEquals(2 -, mgr.getAttributesForNode("host1").size(), "host1 size"); + assertEquals(1, mgr.getAttributesForNode("host0").size(), "host0 size"); + assertEquals(2, mgr.getAttributesForNode("host1").size(), "host1 size"); checkNodeAttributeEqual(replaced, attribute); } public void checkNodeAttributeEqual(NodeAttribute atr1, NodeAttribute atr2) { - Assertions.assertEquals(atr1.getAttributeType(), atr2.getAttributeType()); - Assertions.assertEquals(atr1.getAttributeKey().getAttributeName(), + assertEquals(atr1.getAttributeType(), atr2.getAttributeType()); + assertEquals(atr1.getAttributeKey().getAttributeName(), atr2.getAttributeKey().getAttributeName()); - Assertions.assertEquals(atr1.getAttributeKey().getAttributePrefix(), + assertEquals(atr1.getAttributeKey().getAttributePrefix(), atr2.getAttributeKey().getAttributePrefix()); - Assertions.assertEquals(atr1.getAttributeValue(), atr2.getAttributeValue()); + assertEquals(atr1.getAttributeValue(), atr2.getAttributeValue()); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java index bbe2f981efe45..d4f6249c3bd27 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java @@ -18,6 +18,9 @@ package org.apache.hadoop.yarn.server.resourcemanager.nodelabels; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.util.Sets; @@ -32,7 +35,6 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; import java.io.IOException; import java.util.HashMap; @@ -102,8 +104,8 @@ public void testAddNodeAttributes() throws IOException { attributesManager.addNodeAttributes(toAddAttributes); nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]); - Assertions.assertEquals(3, nodeAttributes.size()); - Assertions.assertTrue(sameAttributeSet(toAddAttributes.get(HOSTNAMES[0]), + assertEquals(3, nodeAttributes.size()); + assertTrue(sameAttributeSet(toAddAttributes.get(HOSTNAMES[0]), nodeAttributes.keySet())); // Add 2 attributes to host2 @@ -116,12 +118,12 @@ public void testAddNodeAttributes() throws IOException { // Verify host1 attributes are still valid. nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]); - Assertions.assertEquals(3, nodeAttributes.size()); + assertEquals(3, nodeAttributes.size()); // Verify new added host2 attributes are correctly updated. nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[1]); - Assertions.assertEquals(2, nodeAttributes.size()); - Assertions.assertTrue(sameAttributeSet(toAddAttributes.get(HOSTNAMES[1]), + assertEquals(2, nodeAttributes.size()); + assertTrue(sameAttributeSet(toAddAttributes.get(HOSTNAMES[1]), nodeAttributes.keySet())); // Cluster wide, it only has 3 attributes. @@ -130,17 +132,17 @@ public void testAddNodeAttributes() throws IOException { // yarn.test1.io/A3 Set clusterAttributes = attributesManager .getClusterNodeAttributes(Sets.newHashSet(PREFIXES[0])); - Assertions.assertEquals(3, clusterAttributes.size()); + assertEquals(3, clusterAttributes.size()); // Query for attributes under a non-exist prefix, // ensure it returns an empty set. clusterAttributes = attributesManager .getClusterNodeAttributes(Sets.newHashSet("non_exist_prefix")); - Assertions.assertEquals(0, clusterAttributes.size()); + assertEquals(0, clusterAttributes.size()); // Not provide any prefix, ensure it returns all attributes. clusterAttributes = attributesManager.getClusterNodeAttributes(null); - Assertions.assertEquals(3, clusterAttributes.size()); + assertEquals(3, clusterAttributes.size()); // Add some other attributes with different prefixes on host1 and host2. toAddAttributes.clear(); @@ -160,10 +162,10 @@ public void testAddNodeAttributes() throws IOException { attributesManager.addNodeAttributes(toAddAttributes); nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]); - Assertions.assertEquals(13, nodeAttributes.size()); + assertEquals(13, nodeAttributes.size()); nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[1]); - Assertions.assertEquals(22, nodeAttributes.size()); + assertEquals(22, nodeAttributes.size()); } @Test @@ -210,20 +212,20 @@ public void testRemoveNodeAttributes() throws IOException { attributesManager.addNodeAttributes(toAddAttributes); nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]); - Assertions.assertEquals(8, nodeAttributes.size()); + assertEquals(8, nodeAttributes.size()); nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[1]); - Assertions.assertEquals(4, nodeAttributes.size()); + assertEquals(4, nodeAttributes.size()); allAttributesPerPrefix = attributesManager .getClusterNodeAttributes(Sets.newHashSet(PREFIXES[0])); - Assertions.assertEquals(3, allAttributesPerPrefix.size()); + assertEquals(3, allAttributesPerPrefix.size()); allAttributesPerPrefix = attributesManager .getClusterNodeAttributes(Sets.newHashSet(PREFIXES[1])); - Assertions.assertEquals(5, allAttributesPerPrefix.size()); + assertEquals(5, allAttributesPerPrefix.size()); allAttributesPerPrefix = attributesManager .getClusterNodeAttributes(Sets.newHashSet(PREFIXES[2])); - Assertions.assertEquals(2, allAttributesPerPrefix.size()); + assertEquals(2, allAttributesPerPrefix.size()); // Remove "yarn.test1.io/A_2" from host1 Set attributes2rm1 = new HashSet<>(); @@ -233,7 +235,7 @@ public void testRemoveNodeAttributes() throws IOException { attributesManager.removeNodeAttributes(toRemoveAttributes); nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]); - Assertions.assertEquals(7, nodeAttributes.size()); + assertEquals(7, nodeAttributes.size()); // Remove again, but give a non-exist attribute name attributes2rm1.clear(); @@ -244,7 +246,7 @@ public void testRemoveNodeAttributes() throws IOException { attributesManager.removeNodeAttributes(toRemoveAttributes); nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]); - Assertions.assertEquals(7, nodeAttributes.size()); + assertEquals(7, nodeAttributes.size()); // Remove "yarn.test1.io/A_2" from host2 too, // by then there will be no such attribute exist in the cluster. @@ -261,7 +263,7 @@ public void testRemoveNodeAttributes() throws IOException { // us A_1 and A_3. allAttributesPerPrefix = attributesManager .getClusterNodeAttributes(Sets.newHashSet(PREFIXES[0])); - Assertions.assertEquals(2, allAttributesPerPrefix.size()); + assertEquals(2, allAttributesPerPrefix.size()); } @Test @@ -281,7 +283,7 @@ public void testReplaceNodeAttributes() throws IOException { attributesManager.addNodeAttributes(toAddAttributes); nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]); - Assertions.assertEquals(3, nodeAttributes.size()); + assertEquals(3, nodeAttributes.size()); // Add 10 distributed node attributes to host1 // nn.yarn.io/dist-node-attribute1=dist_v1_1 @@ -294,10 +296,10 @@ public void testReplaceNodeAttributes() throws IOException { 10, "dist-node-attribute", "dist_v1")); attributesManager.addNodeAttributes(toAddAttributes); nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]); - Assertions.assertEquals(13, nodeAttributes.size()); + assertEquals(13, nodeAttributes.size()); clusterAttributes = attributesManager.getClusterNodeAttributes( Sets.newHashSet(NodeAttribute.PREFIX_DISTRIBUTED, PREFIXES[0])); - Assertions.assertEquals(13, clusterAttributes.size()); + assertEquals(13, clusterAttributes.size()); // Replace by prefix // Same distributed attributes names, but different values. @@ -308,24 +310,24 @@ public void testReplaceNodeAttributes() throws IOException { attributesManager.replaceNodeAttributes(NodeAttribute.PREFIX_DISTRIBUTED, ImmutableMap.of(HOSTNAMES[0], toReplaceAttributes)); nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]); - Assertions.assertEquals(8, nodeAttributes.size()); + assertEquals(8, nodeAttributes.size()); clusterAttributes = attributesManager.getClusterNodeAttributes( Sets.newHashSet(NodeAttribute.PREFIX_DISTRIBUTED, PREFIXES[0])); - Assertions.assertEquals(8, clusterAttributes.size()); + assertEquals(8, clusterAttributes.size()); // Now we have 5 distributed attributes filteredAttributes = NodeLabelUtil.filterAttributesByPrefix( nodeAttributes.keySet(), NodeAttribute.PREFIX_DISTRIBUTED); - Assertions.assertEquals(5, filteredAttributes.size()); + assertEquals(5, filteredAttributes.size()); // Values are updated to have prefix dist_v2 - Assertions.assertTrue(filteredAttributes.stream().allMatch( + assertTrue(filteredAttributes.stream().allMatch( nodeAttribute -> nodeAttribute.getAttributeValue().startsWith("dist_v2"))); // We still have 3 yarn.test1.io attributes filteredAttributes = NodeLabelUtil.filterAttributesByPrefix( nodeAttributes.keySet(), PREFIXES[0]); - Assertions.assertEquals(3, filteredAttributes.size()); + assertEquals(3, filteredAttributes.size()); // Replace with prefix // Different attribute names @@ -335,16 +337,16 @@ public void testReplaceNodeAttributes() throws IOException { attributesManager.replaceNodeAttributes(NodeAttribute.PREFIX_DISTRIBUTED, ImmutableMap.of(HOSTNAMES[0], toReplaceAttributes)); nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]); - Assertions.assertEquals(4, nodeAttributes.size()); + assertEquals(4, nodeAttributes.size()); clusterAttributes = attributesManager.getClusterNodeAttributes( Sets.newHashSet(NodeAttribute.PREFIX_DISTRIBUTED)); - Assertions.assertEquals(1, clusterAttributes.size()); + assertEquals(1, clusterAttributes.size()); NodeAttribute attr = clusterAttributes.iterator().next(); - Assertions.assertEquals("dist-node-attribute-v2_0", + assertEquals("dist-node-attribute-v2_0", attr.getAttributeKey().getAttributeName()); - Assertions.assertEquals(NodeAttribute.PREFIX_DISTRIBUTED, + assertEquals(NodeAttribute.PREFIX_DISTRIBUTED, attr.getAttributeKey().getAttributePrefix()); - Assertions.assertEquals("dist_v3_0", attr.getAttributeValue()); + assertEquals("dist_v3_0", attr.getAttributeValue()); // Replace all attributes toReplaceMap.put(HOSTNAMES[0], @@ -352,13 +354,13 @@ public void testReplaceNodeAttributes() throws IOException { attributesManager.replaceNodeAttributes(null, toReplaceMap); nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]); - Assertions.assertEquals(2, nodeAttributes.size()); + assertEquals(2, nodeAttributes.size()); clusterAttributes = attributesManager .getClusterNodeAttributes(Sets.newHashSet(PREFIXES[1])); - Assertions.assertEquals(2, clusterAttributes.size()); + assertEquals(2, clusterAttributes.size()); clusterAttributes = attributesManager .getClusterNodeAttributes(Sets.newHashSet( NodeAttribute.PREFIX_DISTRIBUTED)); - Assertions.assertEquals(0, clusterAttributes.size()); + assertEquals(0, clusterAttributes.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/nodelabels/TestNodeLabelFileReplication.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeLabelFileReplication.java index b510b03207e71..1722d5f363ed6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeLabelFileReplication.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeLabelFileReplication.java @@ -18,6 +18,8 @@ package org.apache.hadoop.yarn.server.resourcemanager.nodelabels; +import static org.junit.jupiter.api.Assertions.assertEquals; + import java.io.IOException; import org.apache.hadoop.conf.Configuration; @@ -26,7 +28,6 @@ import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; public class TestNodeLabelFileReplication { @@ -51,9 +52,8 @@ public void testNodeLabelFileReplication() throws IOException { int fileReplication = fs .getFileStatus(new Path(nodeLabelDir, "nodelabel.mirror")) .getReplication(); - Assertions.assertEquals( - - expectedReplication, fileReplication, "Node label file replication should be " + expectedReplication); + assertEquals(expectedReplication, fileReplication, + "Node label file replication should be " + expectedReplication); manager.close(); } finally { if (cluster != 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/nodelabels/TestRMDelegatedNodeLabelsUpdater.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMDelegatedNodeLabelsUpdater.java index d6e6b19928104..4a570730067b5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMDelegatedNodeLabelsUpdater.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMDelegatedNodeLabelsUpdater.java @@ -19,6 +19,8 @@ package org.apache.hadoop.yarn.server.resourcemanager.nodelabels; 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.Map; @@ -37,7 +39,6 @@ import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.YarnVersionInfo; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -66,10 +67,10 @@ public void testRMNodeLabelsMappingProviderConfiguration() { MockRM rm = new MockRM(conf); rm.init(conf); rm.start(); - Assertions.fail("Expected an exception"); + fail("Expected an exception"); } catch (Exception e) { // expected an exception - Assertions.assertTrue(e.getMessage().contains( + assertTrue(e.getMessage().contains( "RMNodeLabelsMappingProvider should be configured")); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java index 1335dd1c9fc35..57171a7762673 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java @@ -21,7 +21,9 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; 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; @@ -56,7 +58,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType; import org.apache.hadoop.yarn.util.resource.Resources; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -110,7 +111,7 @@ public void testGetLabelResourceWhenNodeActiveDeactive() throws Exception { mgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("p1", "p4")); assertThat(mgr.getResourceByLabel("p1", null)).isEqualTo( Resources.add(SMALL_RESOURCE, LARGE_NODE)); - Assertions.assertEquals(mgr.getResourceByLabel("p4", null), EMPTY_RESOURCE); + assertEquals(mgr.getResourceByLabel("p4", null), EMPTY_RESOURCE); // change the large NM to small, check if resource updated mgr.updateNodeResource(NodeId.newInstance("n1", 2), SMALL_RESOURCE); @@ -273,15 +274,15 @@ public void testGetQueueResource() throws Exception { mgr.reinitializeQueueLabels(queueToLabels); // check resource - Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), + assertEquals(Resources.multiply(SMALL_RESOURCE, 3), mgr.getQueueResource("Q1", q1Label, clusterResource)); - Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), + assertEquals(Resources.multiply(SMALL_RESOURCE, 3), mgr.getQueueResource("Q2", q2Label, clusterResource)); - Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 2), + assertEquals(Resources.multiply(SMALL_RESOURCE, 2), mgr.getQueueResource("Q3", q3Label, clusterResource)); - Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 1), + assertEquals(Resources.multiply(SMALL_RESOURCE, 1), mgr.getQueueResource("Q4", q4Label, clusterResource)); - Assertions.assertEquals(clusterResource, + assertEquals(clusterResource, mgr.getQueueResource("Q5", q5Label, clusterResource)); mgr.removeLabelsFromNode(ImmutableMap.of(toNodeId("host2"), toSet("blue"))); @@ -295,15 +296,15 @@ public void testGetQueueResource() throws Exception { */ // check resource - Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), + assertEquals(Resources.multiply(SMALL_RESOURCE, 3), mgr.getQueueResource("Q1", q1Label, clusterResource)); - Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), + assertEquals(Resources.multiply(SMALL_RESOURCE, 3), mgr.getQueueResource("Q2", q2Label, clusterResource)); - Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), + assertEquals(Resources.multiply(SMALL_RESOURCE, 3), mgr.getQueueResource("Q3", q3Label, clusterResource)); - Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 2), + assertEquals(Resources.multiply(SMALL_RESOURCE, 2), mgr.getQueueResource("Q4", q4Label, clusterResource)); - Assertions.assertEquals(clusterResource, + assertEquals(clusterResource, mgr.getQueueResource("Q5", q5Label, clusterResource)); /* @@ -319,15 +320,15 @@ public void testGetQueueResource() throws Exception { mgr.activateNode(NodeId.newInstance("host3", 1), SMALL_RESOURCE); // check resource - Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 2), + assertEquals(Resources.multiply(SMALL_RESOURCE, 2), mgr.getQueueResource("Q1", q1Label, clusterResource)); - Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), + assertEquals(Resources.multiply(SMALL_RESOURCE, 3), mgr.getQueueResource("Q2", q2Label, clusterResource)); - Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), + assertEquals(Resources.multiply(SMALL_RESOURCE, 3), mgr.getQueueResource("Q3", q3Label, clusterResource)); - Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 2), + assertEquals(Resources.multiply(SMALL_RESOURCE, 2), mgr.getQueueResource("Q4", q4Label, clusterResource)); - Assertions.assertEquals(clusterResource, + assertEquals(clusterResource, mgr.getQueueResource("Q5", q5Label, clusterResource)); /* @@ -354,15 +355,15 @@ public void testGetQueueResource() throws Exception { mgr.reinitializeQueueLabels(queueToLabels); // check resource - Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 2), + assertEquals(Resources.multiply(SMALL_RESOURCE, 2), mgr.getQueueResource("Q1", q1Label, clusterResource)); - Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 2), + assertEquals(Resources.multiply(SMALL_RESOURCE, 2), mgr.getQueueResource("Q2", q2Label, clusterResource)); - Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 2), + assertEquals(Resources.multiply(SMALL_RESOURCE, 2), mgr.getQueueResource("Q3", q3Label, clusterResource)); - Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 2), + assertEquals(Resources.multiply(SMALL_RESOURCE, 2), mgr.getQueueResource("Q4", q4Label, clusterResource)); - Assertions.assertEquals(clusterResource, + assertEquals(clusterResource, mgr.getQueueResource("Q5", q5Label, clusterResource)); /* @@ -377,15 +378,15 @@ public void testGetQueueResource() throws Exception { mgr.activateNode(NodeId.newInstance("host4", 2), SMALL_RESOURCE); // check resource - Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), + assertEquals(Resources.multiply(SMALL_RESOURCE, 3), mgr.getQueueResource("Q1", q1Label, clusterResource)); - Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), + assertEquals(Resources.multiply(SMALL_RESOURCE, 3), mgr.getQueueResource("Q2", q2Label, clusterResource)); - Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), + assertEquals(Resources.multiply(SMALL_RESOURCE, 3), mgr.getQueueResource("Q3", q3Label, clusterResource)); - Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 3), + assertEquals(Resources.multiply(SMALL_RESOURCE, 3), mgr.getQueueResource("Q4", q4Label, clusterResource)); - Assertions.assertEquals(clusterResource, + assertEquals(clusterResource, mgr.getQueueResource("Q5", q5Label, clusterResource)); /* @@ -400,15 +401,15 @@ public void testGetQueueResource() throws Exception { mgr.deactivateNode(NodeId.newInstance("host4", 1)); // check resource - Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 1), + assertEquals(Resources.multiply(SMALL_RESOURCE, 1), mgr.getQueueResource("Q1", q1Label, clusterResource)); - Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 1), + assertEquals(Resources.multiply(SMALL_RESOURCE, 1), mgr.getQueueResource("Q2", q2Label, clusterResource)); - Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 1), + assertEquals(Resources.multiply(SMALL_RESOURCE, 1), mgr.getQueueResource("Q3", q3Label, clusterResource)); - Assertions.assertEquals(Resources.multiply(SMALL_RESOURCE, 1), + assertEquals(Resources.multiply(SMALL_RESOURCE, 1), mgr.getQueueResource("Q4", q4Label, clusterResource)); - Assertions.assertEquals(clusterResource, + assertEquals(clusterResource, mgr.getQueueResource("Q5", q5Label, clusterResource)); } @@ -450,14 +451,14 @@ public void testRemoveLabelsFromNode() throws Exception { mgr.activateNode(NodeId.newInstance("n1", 1), SMALL_RESOURCE); try { mgr.removeLabelsFromNode(ImmutableMap.of(toNodeId("n1:1"), toSet("p1"))); - Assertions.fail("removeLabelsFromNode should trigger IOException"); + fail("removeLabelsFromNode should trigger IOException"); } catch (IOException e) { } mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1:1"), toSet("p1"))); try { mgr.removeLabelsFromNode(ImmutableMap.of(toNodeId("n1:1"), toSet("p1"))); } catch (IOException e) { - Assertions.fail("IOException from removeLabelsFromNode " + e); + fail("IOException from removeLabelsFromNode " + e); } } @@ -496,37 +497,37 @@ public void testReplaceLabelsFromNode() throws Exception { mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1:1"), toSet("p1"), toNodeId("n2:1"), toSet("p2"), toNodeId("n3"), toSet("p3"))); - assertTrue( - schedEventsHandler.receivedEvent, "Event should be sent when there is change in labels"); - assertEquals(3 -, schedEventsHandler.updatedNodeToLabels.size(), "3 node label mapping modified"); + assertTrue(schedEventsHandler.receivedEvent, + "Event should be sent when there is change in labels"); + assertEquals(3, schedEventsHandler.updatedNodeToLabels.size(), + "3 node label mapping modified"); ImmutableMap> modifiedMap = ImmutableMap.of(toNodeId("n1:1"), toSet("p1"), toNodeId("n2:1"), toSet("p2"), toNodeId("n3:1"), toSet("p3")); - assertEquals(modifiedMap -, schedEventsHandler.updatedNodeToLabels, "Node label mapping is not matching"); + assertEquals(modifiedMap, schedEventsHandler.updatedNodeToLabels, + "Node label mapping is not matching"); schedEventsHandler.receivedEvent = false; mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1:1"), toSet("p1"))); - assertFalse( - schedEventsHandler.receivedEvent, "No event should be sent when there is no change in labels"); + assertFalse(schedEventsHandler.receivedEvent, + "No event should be sent when there is no change in labels"); schedEventsHandler.receivedEvent = false; mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n2:1"), toSet("p1"), toNodeId("n3"), toSet("p3"))); - assertTrue( - schedEventsHandler.receivedEvent, "Event should be sent when there is change in labels"); - assertEquals(1 -, schedEventsHandler.updatedNodeToLabels.size(), "Single node label mapping modified"); + assertTrue(schedEventsHandler.receivedEvent, + "Event should be sent when there is change in labels"); + assertEquals(1, schedEventsHandler.updatedNodeToLabels.size(), + "Single node label mapping modified"); assertCollectionEquals(toSet("p1"), schedEventsHandler.updatedNodeToLabels.get(toNodeId("n2:1"))); schedEventsHandler.receivedEvent = false; mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n3"), toSet("p2"))); - assertTrue( - schedEventsHandler.receivedEvent, "Event should be sent when there is change in labels @ HOST"); - assertEquals(1 -, schedEventsHandler.updatedNodeToLabels.size(), "Single node label mapping modified"); + assertTrue(schedEventsHandler.receivedEvent, + "Event should be sent when there is change in labels @ HOST"); + assertEquals(1, schedEventsHandler.updatedNodeToLabels.size(), + "Single node label mapping modified"); assertCollectionEquals(toSet("p2"), schedEventsHandler.updatedNodeToLabels.get(toNodeId("n3:1"))); schedEventsHandler.receivedEvent = false; @@ -535,8 +536,8 @@ public void testReplaceLabelsFromNode() throws Exception { assertTrue( schedEventsHandler.receivedEvent, "Event should be sent when labels are modified at host though labels were set @ NM level"); - assertEquals(1 -, schedEventsHandler.updatedNodeToLabels.size(), "Single node label mapping modified"); + assertEquals(1, schedEventsHandler.updatedNodeToLabels.size(), + "Single node label mapping modified"); assertCollectionEquals(toSet("p2"), schedEventsHandler.updatedNodeToLabels.get(toNodeId("n1:1"))); schedEventsHandler.receivedEvent = false; @@ -568,7 +569,7 @@ public void testGetLabelsOnNodesWhenNodeActiveDeactive() throws Exception { assertCollectionEquals(toSet("p2"), mgr.getNodeLabels().get(toNodeId("n1:2"))); mgr.deactivateNode(toNodeId("n1:2")); - Assertions.assertNull(mgr.getNodeLabels().get(toNodeId("n1:2"))); + assertNull(mgr.getNodeLabels().get(toNodeId("n1:2"))); // Host will not affected too assertCollectionEquals(toSet("p2"), mgr.getNodeLabels().get(toNodeId("n1"))); @@ -582,7 +583,7 @@ public void testGetLabelsOnNodesWhenNodeActiveDeactive() throws Exception { assertCollectionEquals(toSet("p3"), mgr.getNodeLabels().get(toNodeId("n1:2"))); mgr.deactivateNode(toNodeId("n1:2")); - Assertions.assertNull(mgr.getNodeLabels().get(toNodeId("n1:2"))); + assertNull(mgr.getNodeLabels().get(toNodeId("n1:2"))); // Host will not affected too assertCollectionEquals(toSet("p3"), mgr.getNodeLabels().get(toNodeId("n1"))); @@ -591,12 +592,12 @@ public void testGetLabelsOnNodesWhenNodeActiveDeactive() throws Exception { private void checkNodeLabelInfo(List infos, String labelName, int activeNMs, int memory) { for (RMNodeLabel info : infos) { if (info.getLabelName().equals(labelName)) { - Assertions.assertEquals(activeNMs, info.getNumActiveNMs()); - Assertions.assertEquals(memory, info.getResource().getMemorySize()); + assertEquals(activeNMs, info.getNumActiveNMs()); + assertEquals(memory, info.getResource().getMemorySize()); return; } } - Assertions.fail("Failed to find info has label=" + labelName); + fail("Failed to find info has label=" + labelName); } @Test @@ -613,7 +614,7 @@ public void testPullRMNodeLabelsInfo() throws IOException { // x, y, z and "" List infos = mgr.pullRMNodeLabelsInfo(); - Assertions.assertEquals(4, infos.size()); + assertEquals(4, infos.size()); checkNodeLabelInfo(infos, RMNodeLabelsManager.NO_LABEL, 2, 20); checkNodeLabelInfo(infos, "x", 2, 20); checkNodeLabelInfo(infos, "y", 1, 10); @@ -650,14 +651,11 @@ protected void checkRemoveFromClusterNodeLabelsOfQueue( } lmgr = new TestRMLabelManger(); MockRM rm2 = initRM(withQueueLabels); - Assertions.assertFalse( - - checkQueueCall, "checkRemoveFromClusterNodeLabelsOfQueue should not be called" - + "on recovery"); + assertFalse(checkQueueCall, + "checkRemoveFromClusterNodeLabelsOfQueue should not be called on recovery"); lmgr.removeFromClusterNodeLabels(Arrays.asList(new String[] { "x" })); - Assertions - .assertTrue(checkQueueCall, "checkRemoveFromClusterNodeLabelsOfQueue should be called " - + "since its not recovery"); + assertTrue(checkQueueCall, "checkRemoveFromClusterNodeLabelsOfQueue should be called " + + "since its not recovery"); rm2.stop(); } @@ -670,7 +668,7 @@ public RMNodeLabelsManager createNodeLabelManager() { }; rm.getRMContext().setNodeLabelManager(lmgr); rm.start(); - Assertions.assertEquals(Service.STATE.STARTED, rm.getServiceState()); + assertEquals(Service.STATE.STARTED, rm.getServiceState()); return rm; } @@ -694,7 +692,7 @@ private Configuration getConfigurationWithQueueLabels(Configuration config) { public void testLabelsToNodesOnNodeActiveDeactive() throws Exception { // Activate a node without assigning any labels mgr.activateNode(NodeId.newInstance("n1", 1), Resource.newInstance(10, 0)); - Assertions.assertTrue(mgr.getLabelsToNodes().isEmpty()); + assertTrue(mgr.getLabelsToNodes().isEmpty()); assertLabelsToNodesEquals( mgr.getLabelsToNodes(), transposeNodeToLabels(mgr.getNodeLabels())); @@ -702,21 +700,21 @@ public void testLabelsToNodesOnNodeActiveDeactive() throws Exception { mgr.addToCluserNodeLabelsWithDefaultExclusivity(toSet("p1")); mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1"), toSet("p1"))); // p1 -> n1, n1:1 - Assertions.assertEquals(2, mgr.getLabelsToNodes().get("p1").size()); + assertEquals(2, mgr.getLabelsToNodes().get("p1").size()); assertLabelsToNodesEquals( mgr.getLabelsToNodes(), transposeNodeToLabels(mgr.getNodeLabels())); // Activate a node for which host to label mapping exists mgr.activateNode(NodeId.newInstance("n1", 2), Resource.newInstance(10, 0)); // p1 -> n1, n1:1, n1:2 - Assertions.assertEquals(3, mgr.getLabelsToNodes().get("p1").size()); + assertEquals(3, mgr.getLabelsToNodes().get("p1").size()); assertLabelsToNodesEquals( mgr.getLabelsToNodes(), transposeNodeToLabels(mgr.getNodeLabels())); // Deactivate a node. n1:1 will be removed from the map mgr.deactivateNode(NodeId.newInstance("n1", 1)); // p1 -> n1, n1:2 - Assertions.assertEquals(2, mgr.getLabelsToNodes().get("p1").size()); + assertEquals(2, mgr.getLabelsToNodes().get("p1").size()); assertLabelsToNodesEquals( mgr.getLabelsToNodes(), transposeNodeToLabels(mgr.getNodeLabels())); } @@ -759,12 +757,12 @@ public void testBackwardsCompatableMirror() throws Exception { Set labelNames = lmgr.getClusterNodeLabelNames(); Map> labeledNodes = lmgr.getLabelsToNodes(); - Assertions.assertTrue(labelNames.contains("a")); - Assertions.assertTrue(labelNames.contains("b")); - Assertions.assertTrue(labelNames.contains("c")); - Assertions.assertTrue(labeledNodes.get("a") + assertTrue(labelNames.contains("a")); + assertTrue(labelNames.contains("b")); + assertTrue(labelNames.contains("c")); + assertTrue(labeledNodes.get("a") .contains(NodeId.newInstance("host1", 0))); - Assertions.assertTrue(labeledNodes.get("b") + assertTrue(labeledNodes.get("b") .contains(NodeId.newInstance("host2", 0))); rm.stop(); From 927f0ee0dcf583dbde0eb28334056a0cc45ed47f Mon Sep 17 00:00:00 2001 From: slfan1989 Date: Sun, 9 Mar 2025 07:44:08 +0800 Subject: [PATCH 5/6] YARN-11262. Fix CheckStyle & Junit Test. --- .../yarn/client/api/impl/TestYarnClient.java | 3 ++- .../yarn/server/resourcemanager/TestRM.java | 1 + .../server/resourcemanager/TestRMRestart.java | 24 ++++++++++--------- .../applicationsmanager/TestAMRestart.java | 1 + .../TestRMAppLogAggregationStatus.java | 2 +- .../TestCombinedSystemMetricsPublisher.java | 8 +++---- .../metrics/TestSystemMetricsPublisher.java | 4 +++- ...tProportionalCapacityPreemptionPolicy.java | 2 +- .../nodelabels/TestRMNodeLabelsManager.java | 7 +++--- .../security/TestClientToAMTokens.java | 2 +- 10 files changed, 31 insertions(+), 23 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java index 0f25073777ceb..f0da771332124 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java @@ -1217,7 +1217,8 @@ private void waitTillAccepted(YarnClient rmClient, ApplicationId appId, @ParameterizedTest(name = "{0}") @MethodSource("getParameters") @Timeout(value = 30) - public void testShouldNotRetryForeverForNonNetworkExceptions(SchedulerType type) throws Exception { + public void testShouldNotRetryForeverForNonNetworkExceptions(SchedulerType type) + throws Exception { assertThrows(ApplicationNotFoundException.class, () -> { initTestYarnClient(type); YarnConfiguration conf = getConf(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRM.java index c1f3c33e658ea..4e46f8930a5f0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRM.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRM.java @@ -266,6 +266,7 @@ public void testNMTokenSentForNormalContainer(SchedulerType type) throws Excepti @ParameterizedTest(name = "{0}") @MethodSource("getParameters") @Timeout(40) + @SuppressWarnings("checkstyle:MethodLength") public void testNMToken(SchedulerType type) throws Exception { initTestRM(type); MockRM rm = new MockRM(conf); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java index 4d1cf857044c4..3ddaf3d7afdd6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java @@ -235,6 +235,7 @@ private RMApp submitApp(MockRM rm, List amResourceRequests, @Timeout(180) @ParameterizedTest(name = "{0}") @MethodSource("getParameters") + @SuppressWarnings("checkstyle:MethodLength") public void testRMRestart(SchedulerType type) throws Exception { initTestRMRestart(type); conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, @@ -288,7 +289,7 @@ public void testRMRestart(SchedulerType type) throws Exception { ApplicationAttemptStateData attemptState = appState.getAttempt(attemptId1); assertNotNull(attemptState); - assertEquals(BuilderUtils.newContainerId(attemptId1, 1), + assertEquals(BuilderUtils.newContainerId(attemptId1, 1), attemptState.getMasterContainer().getId()); // launch the AM @@ -446,8 +447,8 @@ public void testRMRestart(SchedulerType type) throws Exception { appState = rmAppState.get(loadedApp1.getApplicationId()); attemptState = appState.getAttempt(attemptId1); assertNotNull(attemptState); - assertEquals(BuilderUtils.newContainerId(attemptId1, 1), - attemptState.getMasterContainer().getId()); + assertEquals(BuilderUtils.newContainerId(attemptId1, 1), + attemptState.getMasterContainer().getId()); // Nodes on which the AM's run MockNM am1Node = nm1; @@ -463,8 +464,8 @@ public void testRMRestart(SchedulerType type) throws Exception { appState = rmAppState.get(loadedApp2.getApplicationId()); attemptState = appState.getAttempt(attemptId2); assertNotNull(attemptState); - assertEquals(BuilderUtils.newContainerId(attemptId2, 1), - attemptState.getMasterContainer().getId()); + assertEquals(BuilderUtils.newContainerId(attemptId2, 1), + attemptState.getMasterContainer().getId()); MockNM am2Node = nm1; if (attemptState.getMasterContainer().getNodeId().toString() @@ -648,6 +649,7 @@ public void testRMRestartAppRunningAMFailed(SchedulerType type) throws Exception @Timeout(60) @ParameterizedTest(name = "{0}") @MethodSource("getParameters") + @SuppressWarnings("checkstyle:MethodLength") public void testRMRestartWaitForPreviousAMToFinish(SchedulerType type) throws Exception { initTestRMRestart(type); // testing 3 cases @@ -761,7 +763,7 @@ public void testRMRestartWaitForPreviousAMToFinish(SchedulerType type) throws Ex assertEquals(RMAppAttemptState.FAILED, rmApp.getAppAttempts().get(am2.getApplicationAttemptId()) .getAppAttemptState()); - assertEquals(RMAppAttemptState.LAUNCHED,rmApp.getAppAttempts() + assertEquals(RMAppAttemptState.LAUNCHED, rmApp.getAppAttempts() .get(latestAppAttemptId).getAppAttemptState()); rm3.waitForState(latestAppAttemptId, RMAppAttemptState.FAILED); @@ -1158,7 +1160,7 @@ protected SystemMetricsPublisher createSystemMetricsPublisher() { rm1.waitForState(app2.getApplicationId(), RMAppState.KILLED); rm1.waitForState(am2.getApplicationAttemptId(), RMAppAttemptState.KILLED); - verify(rm1.getRMContext().getSystemMetricsPublisher(),times(3)) + verify(rm1.getRMContext().getSystemMetricsPublisher(), times(3)) .appCreated(any(RMApp.class), anyLong()); // restart rm @@ -1336,8 +1338,8 @@ public void testRMRestartOnMaxAppAttempts(SchedulerType type) throws Exception { ApplicationAttemptStateData attemptState = appState.getAttempt(attemptId1); assertNotNull(attemptState); - assertEquals(BuilderUtils.newContainerId(attemptId1, 1), - attemptState.getMasterContainer().getId()); + assertEquals(BuilderUtils.newContainerId(attemptId1, 1), + attemptState.getMasterContainer().getId()); // Setting AMLivelinessMonitor interval to be 3 Secs. conf.setInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 3000); @@ -1346,8 +1348,7 @@ public void testRMRestartOnMaxAppAttempts(SchedulerType type) throws Exception { rm2.start(); // verify that maxAppAttempts is set to global value - assertEquals(2, - rm2.getRMContext().getRMApps().get(app2.getApplicationId()) + assertEquals(2, rm2.getRMContext().getRMApps().get(app2.getApplicationId()) .getMaxAppAttempts()); // app1 and app2 are loaded back, but app1 failed because it's @@ -1632,6 +1633,7 @@ public void testAppAttemptTokensRestoredOnRMRestart(SchedulerType type) throws E @Timeout(60) @ParameterizedTest(name = "{0}") @MethodSource("getParameters") + @SuppressWarnings("checkstyle:MethodLength") public void testRMDelegationTokenRestoredOnRMRestart(SchedulerType type) throws Exception { initTestRMRestart(type); conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java index 864e50c4e3182..e60ed7e25d1ad 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java @@ -95,6 +95,7 @@ public void initTestAMRestart(SchedulerType type) throws IOException { @ParameterizedTest(name = "{0}") @MethodSource("getParameters") @Timeout(value = 30) + @SuppressWarnings("checkstyle:methodlength") public void testAMRestartWithExistingContainers(SchedulerType type) throws Exception { initTestAMRestart(type); getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java index fca8211fda36e..52e6415cf7aa0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java @@ -205,7 +205,7 @@ public void testLogAggregationStatus() throws Exception { .entrySet()) { if (report.getKey().equals(node1.getNodeID())) { assertEquals(LogAggregationStatus.RUNNING, report.getValue() - .getLogAggregationStatus()); + .getLogAggregationStatus()); assertEquals(messageForNode1_1, report.getValue() .getDiagnosticMessage()); } else if (report.getKey().equals(node2.getNodeID())) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestCombinedSystemMetricsPublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestCombinedSystemMetricsPublisher.java index 092310270a9b3..49baa0cb3616d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestCombinedSystemMetricsPublisher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestCombinedSystemMetricsPublisher.java @@ -421,10 +421,10 @@ private void verifyEntity(File entityFile, long expectedEvents, } finally { reader.close(); } - assertEquals( - expectedEvents, count, "Expected " + expectedEvents + " events to be published"); - assertEquals( - expectedMetrics, metricsCount, "Expected " + expectedMetrics + " metrics is incorrect"); + assertEquals(expectedEvents, count, + "Expected " + expectedEvents + " events to be published"); + assertEquals(expectedMetrics, metricsCount, + "Expected " + expectedMetrics + " metrics is incorrect"); } private String getTimelineEntityDir() { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java index 211b7d3edfa95..223400ffeac25 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java @@ -135,6 +135,7 @@ public void tearDown() throws Exception { @ParameterizedTest @MethodSource("data") @Timeout(value = 10) + @SuppressWarnings("checkstyle:methodlength") public void testPublishApplicationMetrics(boolean pRmTimelineServerV1PublisherBatchEnabled, int pRmTimelineServerV1PublisherInterval) throws Exception { initTestSystemMetricsPublisher(pRmTimelineServerV1PublisherBatchEnabled, @@ -336,7 +337,8 @@ public void testPublishApplicationMetrics(boolean pRmTimelineServerV1PublisherBa @ParameterizedTest @MethodSource("data") @Timeout(value = 10) - public void testPublishAppAttemptMetricsForUnmanagedAM(boolean pRmTimelineServerV1PublisherBatchEnabled, + public void testPublishAppAttemptMetricsForUnmanagedAM(boolean + pRmTimelineServerV1PublisherBatchEnabled, int pRmTimelineServerV1PublisherInterval) throws Exception { initTestSystemMetricsPublisher(pRmTimelineServerV1PublisherBatchEnabled, pRmTimelineServerV1PublisherInterval); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java index 75b5b909ae5cd..514d3f4ff7723 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java @@ -151,7 +151,7 @@ public int getValue() { }; @RegisterExtension - public TestName name = new TestName(); + private TestName name = new TestName(); @BeforeEach @SuppressWarnings("unchecked") diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java index 57171a7762673..a7845cea0898c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java @@ -236,6 +236,7 @@ public void testGetLabelResource() throws Exception { @Test @Timeout(value = 5) + @SuppressWarnings("checkstyle:MethodLength") public void testGetQueueResource() throws Exception { Resource clusterResource = Resource.newInstance(9999, 1); @@ -533,9 +534,9 @@ public void testReplaceLabelsFromNode() throws Exception { schedEventsHandler.receivedEvent = false; mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1"), toSet("p2"))); - assertTrue( - - schedEventsHandler.receivedEvent, "Event should be sent when labels are modified at host though labels were set @ NM level"); + assertTrue(schedEventsHandler.receivedEvent, + "Event should be sent when labels are " + + "modified at host though labels were set @ NM level"); assertEquals(1, schedEventsHandler.updatedNodeToLabels.size(), "Single node label mapping modified"); assertCollectionEquals(toSet("p2"), diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientToAMTokens.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientToAMTokens.java index 1f9b5bb03da8b..d5e2dd74d47e1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientToAMTokens.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientToAMTokens.java @@ -95,7 +95,7 @@ public void initTestClientToAMTokens(SchedulerType type) throws IOException { initParameterizedSchedulerTestBase(type); setup(); } - + public void setup() { conf = getConf(); } From 439acf85a4637db24911be390bb154240256526e Mon Sep 17 00:00:00 2001 From: slfan1989 Date: Sun, 9 Mar 2025 15:10:47 +0800 Subject: [PATCH 6/6] YARN-11262. Fix CheckStyle. --- .../hadoop/yarn/server/resourcemanager/TestRMRestart.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java index 3ddaf3d7afdd6..c5bbf2e0a5265 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java @@ -1178,7 +1178,7 @@ protected SystemMetricsPublisher createSystemMetricsPublisher() { rms.add(rm2); rm2.start(); - verify(rm2.getRMContext().getSystemMetricsPublisher(),times(3)) + verify(rm2.getRMContext().getSystemMetricsPublisher(), times(3)) .appCreated(any(RMApp.class), anyLong()); GetApplicationsRequest request1 =