From 9173aad89c8a904c586fa5863b67dff43118ce83 Mon Sep 17 00:00:00 2001 From: slfan1989 Date: Sun, 9 Mar 2025 11:32:52 +0800 Subject: [PATCH] YARN-11262. Upgrade JUnit from 4 to 5 in hadoop-yarn-server-resourcemanager Part4. --- .../scheduler/TestAbstractYarnScheduler.java | 167 +-- .../scheduler/TestAppSchedulingInfo.java | 66 +- .../scheduler/TestClusterNodeTracker.java | 94 +- .../TestConfigurationMutationACLPolicies.java | 10 +- .../scheduler/TestPartitionQueueMetrics.java | 98 +- .../scheduler/TestQueueMetrics.java | 20 +- .../TestQueueMetricsForCustomResources.java | 20 +- .../scheduler/TestResourceUsage.java | 8 +- .../TestSchedulerApplicationAttempt.java | 78 +- .../scheduler/TestSchedulerHealth.java | 113 +- .../scheduler/TestSchedulerOvercommit.java | 52 +- .../scheduler/TestSchedulerUtils.java | 100 +- ...TestSchedulingWithAllocationRequestId.java | 36 +- .../constraint/TestAllocationTagsManager.java | 148 +- .../TestAllocationTagsNamespace.java | 70 +- .../TestBatchedRequestsIterators.java | 10 +- ...TestPlacementConstraintManagerService.java | 110 +- .../TestPlacementConstraintsUtil.java | 184 +-- .../constraint/TestPlacementProcessor.java | 115 +- .../algorithm/TestCircularIterator.java | 14 +- .../TestLocalAllocationTagsManager.java | 28 +- ...alizedOpportunisticContainerAllocator.java | 12 +- .../distributed/TestNodeQueueLoadMonitor.java | 224 +-- .../scheduler/fair/FairSchedulerTestBase.java | 6 +- .../fair/TestAllocationFileLoaderService.java | 224 +-- .../scheduler/fair/TestAppRunnability.java | 18 +- .../TestApplicationMasterServiceWithFS.java | 34 +- .../scheduler/fair/TestComputeFairShares.java | 24 +- .../fair/TestConfigurableResource.java | 16 +- .../fair/TestContinuousScheduling.java | 53 +- .../scheduler/fair/TestFSAppAttempt.java | 18 +- .../scheduler/fair/TestFSAppStarvation.java | 56 +- .../scheduler/fair/TestFSLeafQueue.java | 50 +- .../scheduler/fair/TestFSParentQueue.java | 8 +- .../scheduler/fair/TestFSQueueMetrics.java | 262 ++-- .../scheduler/fair/TestFSSchedulerNode.java | 148 +- .../scheduler/fair/TestFairScheduler.java | 1212 +++++++++-------- .../fair/TestFairSchedulerConfiguration.java | 41 +- .../fair/TestFairSchedulerFairShare.java | 12 +- .../fair/TestFairSchedulerPreemption.java | 42 +- .../fair/TestFairSchedulerUtilities.java | 8 +- ...stFairSchedulerWithMultiResourceTypes.java | 12 +- .../fair/TestMaxRunningAppsEnforcer.java | 8 +- .../scheduler/fair/TestQueueManager.java | 232 ++-- .../fair/TestQueueManagerRealScheduler.java | 12 +- .../fair/TestQueuePlacementPolicy.java | 28 +- .../scheduler/fair/TestSchedulingPolicy.java | 104 +- .../scheduler/fair/TestSchedulingUpdate.java | 54 +- .../TestVisitedResourceRequestTracker.java | 32 +- .../FSConfigConverterTestCommons.java | 4 +- .../TestConvertedConfigValidator.java | 18 +- ...TestFSConfigToCSConfigArgumentHandler.java | 140 +- .../TestFSConfigToCSConfigConverter.java | 210 +-- .../TestFSConfigToCSConfigConverterMain.java | 60 +- .../TestFSConfigToCSConfigRuleHandler.java | 43 +- .../fair/converter/TestFSQueueConverter.java | 232 ++-- .../converter/TestFSYarnSiteConverter.java | 157 ++- .../TestQueuePlacementConverter.java | 70 +- .../TestWeightToPercentageConverter.java | 110 +- .../TestWeightToWeightConverter.java | 62 +- .../TestDominantResourceFairnessPolicy.java | 185 +-- .../fair/policies/TestEmptyQueues.java | 16 +- .../scheduler/fifo/TestFifoScheduler.java | 191 +-- ...SingleConstraintAppPlacementAllocator.java | 58 +- .../policy/TestFairOrderingPolicy.java | 64 +- .../policy/TestFifoOrderingPolicy.java | 58 +- .../TestFifoOrderingPolicyForPendingApps.java | 6 +- ...OrderingPolicyWithExclusivePartitions.java | 44 +- 68 files changed, 3167 insertions(+), 3052 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/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..96d169d5ce1d5 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,7 @@ 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.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -94,8 +94,9 @@ 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.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.mockito.Mockito; @SuppressWarnings("unchecked") @@ -150,45 +151,45 @@ private void testMaximumAllocationMemoryHelper( final int node1MaxMemory, final int node2MaxMemory, final int node3MaxMemory, final int... expectedMaxMemory) throws Exception { - Assert.assertEquals(6, expectedMaxMemory.length); + Assertions.assertEquals(6, expectedMaxMemory.length); - Assert.assertEquals(0, scheduler.getNumClusterNodes()); + Assertions.assertEquals(0, scheduler.getNumClusterNodes()); long maxMemory = scheduler.getMaximumResourceCapability().getMemorySize(); - Assert.assertEquals(expectedMaxMemory[0], maxMemory); + Assertions.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()); + Assertions.assertEquals(1, scheduler.getNumClusterNodes()); maxMemory = scheduler.getMaximumResourceCapability().getMemorySize(); - Assert.assertEquals(expectedMaxMemory[1], maxMemory); + Assertions.assertEquals(expectedMaxMemory[1], maxMemory); scheduler.handle(new NodeRemovedSchedulerEvent(node1)); - Assert.assertEquals(0, scheduler.getNumClusterNodes()); + Assertions.assertEquals(0, scheduler.getNumClusterNodes()); maxMemory = scheduler.getMaximumResourceCapability().getMemorySize(); - Assert.assertEquals(expectedMaxMemory[2], maxMemory); + Assertions.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()); + Assertions.assertEquals(1, scheduler.getNumClusterNodes()); maxMemory = scheduler.getMaximumResourceCapability().getMemorySize(); - Assert.assertEquals(expectedMaxMemory[3], maxMemory); + Assertions.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()); + Assertions.assertEquals(2, scheduler.getNumClusterNodes()); maxMemory = scheduler.getMaximumResourceCapability().getMemorySize(); - Assert.assertEquals(expectedMaxMemory[4], maxMemory); + Assertions.assertEquals(expectedMaxMemory[4], maxMemory); scheduler.handle(new NodeRemovedSchedulerEvent(node3)); - Assert.assertEquals(1, scheduler.getNumClusterNodes()); + Assertions.assertEquals(1, scheduler.getNumClusterNodes()); maxMemory = scheduler.getMaximumResourceCapability().getMemorySize(); - Assert.assertEquals(expectedMaxMemory[5], maxMemory); + Assertions.assertEquals(expectedMaxMemory[5], maxMemory); scheduler.handle(new NodeRemovedSchedulerEvent(node2)); - Assert.assertEquals(0, scheduler.getNumClusterNodes()); + Assertions.assertEquals(0, scheduler.getNumClusterNodes()); } @Test @@ -236,45 +237,45 @@ private void testMaximumAllocationVCoresHelper( final int node1MaxVCores, final int node2MaxVCores, final int node3MaxVCores, final int... expectedMaxVCores) throws Exception { - Assert.assertEquals(6, expectedMaxVCores.length); + Assertions.assertEquals(6, expectedMaxVCores.length); - Assert.assertEquals(0, scheduler.getNumClusterNodes()); + Assertions.assertEquals(0, scheduler.getNumClusterNodes()); int maxVCores = scheduler.getMaximumResourceCapability().getVirtualCores(); - Assert.assertEquals(expectedMaxVCores[0], maxVCores); + Assertions.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()); + Assertions.assertEquals(1, scheduler.getNumClusterNodes()); maxVCores = scheduler.getMaximumResourceCapability().getVirtualCores(); - Assert.assertEquals(expectedMaxVCores[1], maxVCores); + Assertions.assertEquals(expectedMaxVCores[1], maxVCores); scheduler.handle(new NodeRemovedSchedulerEvent(node1)); - Assert.assertEquals(0, scheduler.getNumClusterNodes()); + Assertions.assertEquals(0, scheduler.getNumClusterNodes()); maxVCores = scheduler.getMaximumResourceCapability().getVirtualCores(); - Assert.assertEquals(expectedMaxVCores[2], maxVCores); + Assertions.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()); + Assertions.assertEquals(1, scheduler.getNumClusterNodes()); maxVCores = scheduler.getMaximumResourceCapability().getVirtualCores(); - Assert.assertEquals(expectedMaxVCores[3], maxVCores); + Assertions.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()); + Assertions.assertEquals(2, scheduler.getNumClusterNodes()); maxVCores = scheduler.getMaximumResourceCapability().getVirtualCores(); - Assert.assertEquals(expectedMaxVCores[4], maxVCores); + Assertions.assertEquals(expectedMaxVCores[4], maxVCores); scheduler.handle(new NodeRemovedSchedulerEvent(node3)); - Assert.assertEquals(1, scheduler.getNumClusterNodes()); + Assertions.assertEquals(1, scheduler.getNumClusterNodes()); maxVCores = scheduler.getMaximumResourceCapability().getVirtualCores(); - Assert.assertEquals(expectedMaxVCores[5], maxVCores); + Assertions.assertEquals(expectedMaxVCores[5], maxVCores); scheduler.handle(new NodeRemovedSchedulerEvent(node2)); - Assert.assertEquals(0, scheduler.getNumClusterNodes()); + Assertions.assertEquals(0, scheduler.getNumClusterNodes()); } /** @@ -743,7 +744,8 @@ public void testMaxAllocationAfterUpdateNodeResource() throws IOException { * null (no attempt). */ @SuppressWarnings({ "rawtypes" }) - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testReleasedContainerIfAppAttemptisNull() throws Exception { YarnConfiguration conf=getConf(); MockRM rm1 = new MockRM(conf); @@ -777,9 +779,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); + Assertions.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,7 +789,8 @@ public void testReleasedContainerIfAppAttemptisNull() throws Exception { } } - @Test(timeout = 30000l) + @Test + @Timeout(30) public void testContainerReleaseWithAllocationTags() throws Exception { // Currently only can be tested against capacity scheduler. if (getSchedulerType().equals(SchedulerType.CAPACITY)) { @@ -847,13 +850,13 @@ public void testContainerReleaseWithAllocationTags() throws Exception { Thread.sleep(1000); } - Assert.assertEquals(4, allocated.size()); + Assertions.assertEquals(4, allocated.size()); Set containers = allocated.stream() .filter(container -> container.getAllocationRequestId() == 1l) .collect(Collectors.toSet()); - Assert.assertNotNull(containers); - Assert.assertEquals(1, containers.size()); + Assertions.assertNotNull(containers); + Assertions.assertEquals(1, containers.size()); ContainerId cid = containers.iterator().next().getId(); // mock container start @@ -864,8 +867,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()); + Assertions.assertNotNull(nodeTags.get(testTag1)); + Assertions.assertEquals(1, nodeTags.get(testTag1).intValue()); // release a container am1.allocate(new ArrayList<>(), Lists.newArrayList(cid)); @@ -873,9 +876,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()); + Assertions.assertNotNull(nodeTags); + Assertions.assertNotNull(nodeTags.get(testTag1)); + Assertions.assertEquals(1, nodeTags.get(testTag1).intValue()); // NM reports back that container is released // RM should cleanup the tag @@ -897,7 +900,8 @@ public void testContainerReleaseWithAllocationTags() throws Exception { } - @Test(timeout = 30000L) + @Test + @Timeout(30) public void testNodeRemovedWithAllocationTags() throws Exception { // Currently only can be tested against capacity scheduler. if (getSchedulerType().equals(SchedulerType.CAPACITY)) { @@ -948,13 +952,13 @@ public void testNodeRemovedWithAllocationTags() throws Exception { Thread.sleep(1000); } - Assert.assertEquals(1, allocated.size()); + Assertions.assertEquals(1, allocated.size()); Set containers = allocated.stream() .filter(container -> container.getAllocationRequestId() == 1L) .collect(Collectors.toSet()); - Assert.assertNotNull(containers); - Assert.assertEquals(1, containers.size()); + Assertions.assertNotNull(containers); + Assertions.assertEquals(1, containers.size()); ContainerId cid = containers.iterator().next().getId(); // mock container start @@ -965,8 +969,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()); + Assertions.assertNotNull(nodeTags.get(testTag1)); + Assertions.assertEquals(1, nodeTags.get(testTag1).intValue()); // remove the node RMNode node1 = MockNodes.newNodeInfo( @@ -977,12 +981,13 @@ 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); + Assertions.assertNull(nodeTags); } } - @Test(timeout=60000) + @Test + @Timeout(value = 60) public void testContainerReleasedByNode() throws Exception { System.out.println("Starting testContainerReleasedByNode"); YarnConfiguration conf = getConf(); @@ -1075,15 +1080,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()); + Assertions.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()); + Assertions.assertTrue( + containers.isEmpty(), "new container allocated before node freed old"); } // free the old container from the node @@ -1105,7 +1110,8 @@ public void testContainerReleasedByNode() throws Exception { } } - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testResourceRequestRestoreWhenRMContainerIsAtAllocated() throws Exception { YarnConfiguration conf = getConf(); @@ -1280,11 +1286,11 @@ public void testResourceRequestRecoveryToTheRightAppAttempt() RMAppAttempt rmAppAttempt2 = MockRM.waitForAttemptScheduled(rmApp, rm); ApplicationAttemptId applicationAttemptTwoID = rmAppAttempt2.getAppAttemptId(); - Assert.assertEquals(2, applicationAttemptTwoID.getAttemptId()); + Assertions.assertEquals(2, applicationAttemptTwoID.getAttemptId()); // All outstanding allocated containers will be killed (irrespective of // keep-alive of container across app-attempts) - Assert.assertEquals(RMContainerState.KILLED, + Assertions.assertEquals(RMContainerState.KILLED, allocatedContainer.getState()); // The core part of this test @@ -1292,11 +1298,11 @@ public void testResourceRequestRecoveryToTheRightAppAttempt() // original app-attempt, not the new one for (SchedulerRequestKey key : firstSchedulerAppAttempt.getSchedulerKeys()) { if (key.getPriority().getPriority() == 0) { - Assert.assertEquals(0, + Assertions.assertEquals(0, firstSchedulerAppAttempt.getOutstandingAsksCount(key)); } else if (key.getPriority().getPriority() == ALLOCATED_CONTAINER_PRIORITY) { - Assert.assertEquals(1, + Assertions.assertEquals(1, firstSchedulerAppAttempt.getOutstandingAsksCount(key)); } } @@ -1307,8 +1313,8 @@ public void testResourceRequestRecoveryToTheRightAppAttempt() List transferredContainers = rm.getResourceScheduler().getTransferredContainers( applicationAttemptTwoID); - Assert.assertEquals(1, transferredContainers.size()); - Assert.assertEquals(runningContainerID, transferredContainers.get(0) + Assertions.assertEquals(1, transferredContainers.size()); + Assertions.assertEquals(runningContainerID, transferredContainers.get(0) .getId()); } finally { @@ -1321,9 +1327,9 @@ private void verifyMaximumResourceCapability( final Resource schedulerMaximumResourceCapability = scheduler .getMaximumResourceCapability(); - Assert.assertEquals(expectedMaximumResource.getMemorySize(), + Assertions.assertEquals(expectedMaximumResource.getMemorySize(), schedulerMaximumResourceCapability.getMemorySize()); - Assert.assertEquals(expectedMaximumResource.getVirtualCores(), + Assertions.assertEquals(expectedMaximumResource.getVirtualCores(), schedulerMaximumResourceCapability.getVirtualCores()); } @@ -1386,7 +1392,8 @@ private ResourceTrackerService getPrivateResourceTrackerService( * that might occur due to the use of the RMNode object. * @throws Exception */ - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testNodemanagerReconnect() throws Exception { Configuration conf = getConf(); MockRM rm = new MockRM(conf); @@ -1418,8 +1425,8 @@ public void testNodemanagerReconnect() throws Exception { privateDispatcher.await(); Resource clusterResource = rm.getResourceScheduler().getClusterResource(); - Assert.assertEquals("Initial cluster resources don't match", capability, - clusterResource); + Assertions.assertEquals(capability +, clusterResource, "Initial cluster resources don't match"); Resource newCapability = Resources.createResource(1024); RegisterNodeManagerRequest request2 = @@ -1431,15 +1438,16 @@ public void testNodemanagerReconnect() throws Exception { sleepHandler.sleepFlag = true; privateResourceTrackerService.registerNodeManager(request2); privateDispatcher.await(); - Assert.assertEquals("Cluster resources don't match", newCapability, - rm.getResourceScheduler().getClusterResource()); + Assertions.assertEquals(newCapability +, rm.getResourceScheduler().getClusterResource(), "Cluster resources don't match"); privateResourceTrackerService.stop(); } finally { rm.stop(); } } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testUpdateThreadLifeCycle() throws Exception { MockRM rm = new MockRM(getConf()); try { @@ -1449,7 +1457,7 @@ public void testUpdateThreadLifeCycle() throws Exception { if (getSchedulerType().equals(SchedulerType.FAIR)) { Thread updateThread = scheduler.updateThread; - Assert.assertTrue(updateThread.isAlive()); + Assertions.assertTrue(updateThread.isAlive()); scheduler.stop(); int numRetries = 100; @@ -1457,12 +1465,12 @@ public void testUpdateThreadLifeCycle() throws Exception { Thread.sleep(50); } - Assert.assertNotEquals("The Update thread is still alive", 0, numRetries); + Assertions.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); + Assertions.assertNull( + scheduler.updateThread, "updateThread shouldn't have been created"); } else { - Assert.fail("Unhandled SchedulerType, " + getSchedulerType() + + Assertions.fail("Unhandled SchedulerType, " + getSchedulerType() + ", please update this unit test."); } } finally { @@ -1470,7 +1478,8 @@ public void testUpdateThreadLifeCycle() throws Exception { } } - @Test(timeout=60000) + @Test + @Timeout(value = 60) public void testContainerRecoveredByNode() throws Exception { System.out.println("Starting testContainerRecoveredByNode"); final int maxMemory = 10 * 1024; @@ -1521,11 +1530,11 @@ public void testContainerRecoveredByNode() throws Exception { //verify queue name when rmContainer is recovered if (scheduler instanceof CapacityScheduler) { - Assert.assertEquals( + Assertions.assertEquals( app1.getQueue(), rmContainer.getQueueName()); } else { - Assert.assertEquals(app1.getQueue(), rmContainer.getQueueName()); + Assertions.assertEquals(app1.getQueue(), rmContainer.getQueueName()); } } finally { 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/TestAppSchedulingInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.java index a7c7ce85903c7..8f28e72a1f399 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.java @@ -36,8 +36,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSLeafQueue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.LocalityAppPlacementAllocator; import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; public class TestAppSchedulingInfo { @@ -57,20 +57,20 @@ public void testBacklistChanged() { appSchedulingInfo.updatePlacesBlacklistedByApp(new ArrayList(), new ArrayList()); - Assert.assertFalse(appSchedulingInfo.getAndResetBlacklistChanged()); + Assertions.assertFalse(appSchedulingInfo.getAndResetBlacklistChanged()); ArrayList blacklistAdditions = new ArrayList(); blacklistAdditions.add("node1"); blacklistAdditions.add("node2"); appSchedulingInfo.updatePlacesBlacklistedByApp(blacklistAdditions, new ArrayList()); - Assert.assertTrue(appSchedulingInfo.getAndResetBlacklistChanged()); + Assertions.assertTrue(appSchedulingInfo.getAndResetBlacklistChanged()); blacklistAdditions.clear(); blacklistAdditions.add("node1"); appSchedulingInfo.updatePlacesBlacklistedByApp(blacklistAdditions, new ArrayList()); - Assert.assertFalse(appSchedulingInfo.getAndResetBlacklistChanged()); + Assertions.assertFalse(appSchedulingInfo.getAndResetBlacklistChanged()); ArrayList blacklistRemovals = new ArrayList(); blacklistRemovals.add("node1"); @@ -78,11 +78,11 @@ public void testBacklistChanged() { blacklistRemovals); appSchedulingInfo.updatePlacesBlacklistedByApp(new ArrayList(), blacklistRemovals); - Assert.assertTrue(appSchedulingInfo.getAndResetBlacklistChanged()); + Assertions.assertTrue(appSchedulingInfo.getAndResetBlacklistChanged()); appSchedulingInfo.updatePlacesBlacklistedByApp(new ArrayList(), blacklistRemovals); - Assert.assertFalse(appSchedulingInfo.getAndResetBlacklistChanged()); + Assertions.assertFalse(appSchedulingInfo.getAndResetBlacklistChanged()); } @Test @@ -96,23 +96,23 @@ public void testSchedulerRequestKeyOrdering() { ts.add(TestUtils.toSchedulerKey(Priority.newInstance(2), 6)); Iterator iter = ts.iterator(); SchedulerRequestKey sk = iter.next(); - Assert.assertEquals(0, sk.getPriority().getPriority()); - Assert.assertEquals(3, sk.getAllocationRequestId()); + Assertions.assertEquals(0, sk.getPriority().getPriority()); + Assertions.assertEquals(3, sk.getAllocationRequestId()); sk = iter.next(); - Assert.assertEquals(0, sk.getPriority().getPriority()); - Assert.assertEquals(4, sk.getAllocationRequestId()); + Assertions.assertEquals(0, sk.getPriority().getPriority()); + Assertions.assertEquals(4, sk.getAllocationRequestId()); sk = iter.next(); - Assert.assertEquals(1, sk.getPriority().getPriority()); - Assert.assertEquals(1, sk.getAllocationRequestId()); + Assertions.assertEquals(1, sk.getPriority().getPriority()); + Assertions.assertEquals(1, sk.getAllocationRequestId()); sk = iter.next(); - Assert.assertEquals(1, sk.getPriority().getPriority()); - Assert.assertEquals(2, sk.getAllocationRequestId()); + Assertions.assertEquals(1, sk.getPriority().getPriority()); + Assertions.assertEquals(2, sk.getAllocationRequestId()); sk = iter.next(); - Assert.assertEquals(2, sk.getPriority().getPriority()); - Assert.assertEquals(5, sk.getAllocationRequestId()); + Assertions.assertEquals(2, sk.getPriority().getPriority()); + Assertions.assertEquals(5, sk.getAllocationRequestId()); sk = iter.next(); - Assert.assertEquals(2, sk.getPriority().getPriority()); - Assert.assertEquals(6, sk.getAllocationRequestId()); + Assertions.assertEquals(2, sk.getPriority().getPriority()); + Assertions.assertEquals(6, sk.getAllocationRequestId()); } @Test @@ -128,7 +128,7 @@ public void testSchedulerKeyAccounting() { AppSchedulingInfo info = new AppSchedulingInfo( appAttemptId, "test", queue, mock(ActiveUsersManager.class), 0, new ResourceUsage(), new HashMap<>(), rmContext, false); - Assert.assertEquals(0, info.getSchedulerKeys().size()); + Assertions.assertEquals(0, info.getSchedulerKeys().size()); Priority pri1 = Priority.newInstance(1); ResourceRequest req1 = ResourceRequest.newInstance(pri1, @@ -142,16 +142,16 @@ public void testSchedulerKeyAccounting() { info.updateResourceRequests(reqs, false); ArrayList keys = new ArrayList<>(info.getSchedulerKeys()); - Assert.assertEquals(2, keys.size()); - Assert.assertEquals(SchedulerRequestKey.create(req1), keys.get(0)); - Assert.assertEquals(SchedulerRequestKey.create(req2), keys.get(1)); + Assertions.assertEquals(2, keys.size()); + Assertions.assertEquals(SchedulerRequestKey.create(req1), keys.get(0)); + Assertions.assertEquals(SchedulerRequestKey.create(req2), keys.get(1)); // iterate to verify no ConcurrentModificationException for (SchedulerRequestKey schedulerKey : info.getSchedulerKeys()) { info.allocate(NodeType.OFF_SWITCH, null, schedulerKey, null); } - Assert.assertEquals(1, info.getSchedulerKeys().size()); - Assert.assertEquals(SchedulerRequestKey.create(req2), + Assertions.assertEquals(1, info.getSchedulerKeys().size()); + Assertions.assertEquals(SchedulerRequestKey.create(req2), info.getSchedulerKeys().iterator().next()); req2 = ResourceRequest.newInstance(pri2, @@ -161,22 +161,22 @@ public void testSchedulerKeyAccounting() { info.updateResourceRequests(reqs, false); info.allocate(NodeType.OFF_SWITCH, null, SchedulerRequestKey.create(req2), null); - Assert.assertEquals(0, info.getSchedulerKeys().size()); + Assertions.assertEquals(0, info.getSchedulerKeys().size()); req1 = ResourceRequest.newInstance(pri1, ResourceRequest.ANY, Resource.newInstance(1024, 1), 5); reqs.clear(); reqs.add(req1); info.updateResourceRequests(reqs, false); - Assert.assertEquals(1, info.getSchedulerKeys().size()); - Assert.assertEquals(SchedulerRequestKey.create(req1), + Assertions.assertEquals(1, info.getSchedulerKeys().size()); + Assertions.assertEquals(SchedulerRequestKey.create(req1), info.getSchedulerKeys().iterator().next()); req1 = ResourceRequest.newInstance(pri1, ResourceRequest.ANY, Resource.newInstance(1024, 1), 0); reqs.clear(); reqs.add(req1); info.updateResourceRequests(reqs, false); - Assert.assertEquals(0, info.getSchedulerKeys().size()); + Assertions.assertEquals(0, info.getSchedulerKeys().size()); } @Test @@ -193,9 +193,9 @@ public void testApplicationPlacementType() { AppSchedulingInfo info = new AppSchedulingInfo(appAttemptId, "test", queue, mock(ActiveUsersManager.class), 0, new ResourceUsage(), new HashMap<>(), rmContext, false); - Assert.assertEquals(info.getApplicationSchedulingEnvs(), new HashMap<>()); + Assertions.assertEquals(info.getApplicationSchedulingEnvs(), new HashMap<>()); // This should return null as nothing is set in the conf. - Assert.assertNull(info.getDefaultResourceRequestAppPlacementType()); + Assertions.assertNull(info.getDefaultResourceRequestAppPlacementType()); conf = new Configuration(); conf.set(YarnConfiguration.APPLICATION_PLACEMENT_TYPE_CLASS, DEFAULT_APPLICATION_PLACEMENT_TYPE_CLASS); @@ -203,7 +203,7 @@ public void testApplicationPlacementType() { info = new AppSchedulingInfo(appAttemptId, "test", queue, mock(ActiveUsersManager.class), 0, new ResourceUsage(), new HashMap<>(), rmContext, false); - Assert.assertEquals(info.getDefaultResourceRequestAppPlacementType(), + Assertions.assertEquals(info.getDefaultResourceRequestAppPlacementType(), DEFAULT_APPLICATION_PLACEMENT_TYPE_CLASS); } @@ -223,7 +223,7 @@ public void testApplicationPlacementTypeNotConfigured() { mock(ActiveUsersManager.class), 0, new ResourceUsage(), applicationSchedulingEnvs, rmContext, false); // This should be set from applicationSchedulingEnvs - Assert.assertEquals(info.getDefaultResourceRequestAppPlacementType(), + Assertions.assertEquals(info.getDefaultResourceRequestAppPlacementType(), LocalityAppPlacementAllocator.class.getName()); } } 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/TestClusterNodeTracker.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestClusterNodeTracker.java index 14eca5ae5e3b1..f2c93ef4abfff 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestClusterNodeTracker.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestClusterNodeTracker.java @@ -30,11 +30,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSSchedulerNode; import org.apache.hadoop.yarn.util.resource.ResourceUtils; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; /** * Test class to verify ClusterNodeTracker. Using FSSchedulerNode without @@ -44,13 +44,13 @@ public class TestClusterNodeTracker { private ClusterNodeTracker nodeTracker; private ClusterMetrics metrics; - @Before + @BeforeEach public void setup() { metrics = ClusterMetrics.getMetrics(); nodeTracker = new ClusterNodeTracker<>(); } - @After + @AfterEach public void teardown() { ClusterMetrics.destroy(); } @@ -67,33 +67,33 @@ private void addEight4x4Nodes() { @Test public void testGetNodeCount() { addEight4x4Nodes(); - assertEquals("Incorrect number of nodes in the cluster", - 8, nodeTracker.nodeCount()); + assertEquals( + 8, nodeTracker.nodeCount(), "Incorrect number of nodes in the cluster"); - assertEquals("Incorrect number of nodes in each rack", - 4, nodeTracker.nodeCount("rack0")); + assertEquals( + 4, nodeTracker.nodeCount("rack0"), "Incorrect number of nodes in each rack"); } @Test public void testIncrCapability() { addEight4x4Nodes(); - assertEquals("Cluster Capability Memory incorrect", - metrics.getCapabilityMB(), (4096 * 8)); - assertEquals("Cluster Capability Vcores incorrect", - metrics.getCapabilityVirtualCores(), 4 * 8); + assertEquals( + metrics.getCapabilityMB(), (4096 * 8), "Cluster Capability Memory incorrect"); + assertEquals( + metrics.getCapabilityVirtualCores(), 4 * 8, "Cluster Capability Vcores incorrect"); } @Test public void testGetNodesForResourceName() throws Exception { addEight4x4Nodes(); - assertEquals("Incorrect number of nodes matching ANY", - 8, nodeTracker.getNodesByResourceName(ResourceRequest.ANY).size()); + assertEquals( + 8, nodeTracker.getNodesByResourceName(ResourceRequest.ANY).size(), "Incorrect number of nodes matching ANY"); - assertEquals("Incorrect number of nodes matching rack", - 4, nodeTracker.getNodesByResourceName("rack0").size()); + assertEquals( + 4, nodeTracker.getNodesByResourceName("rack0").size(), "Incorrect number of nodes matching rack"); - assertEquals("Incorrect number of nodes matching node", - 1, nodeTracker.getNodesByResourceName("host0").size()); + assertEquals( + 1, nodeTracker.getNodesByResourceName("host0").size(), "Incorrect number of nodes matching node"); } @Test @@ -113,8 +113,8 @@ public void testMaxAllowedAllocation() { Resource result = nodeTracker.getMaxAllowedAllocation(); - assertEquals("With no nodes added, the ClusterNodeTracker did not return " - + "the configured max allocation", maximum, result); + assertEquals(maximum, result, "With no nodes added, the ClusterNodeTracker did not return " + + "the configured max allocation"); List smallNodes = MockNodes.newNodes(1, 1, Resource.newInstance(1024, 2, @@ -133,52 +133,52 @@ public void testMaxAllowedAllocation() { result = nodeTracker.getMaxAllowedAllocation(); - assertEquals("With a single node added, the ClusterNodeTracker did not " - + "return that node's resources as the maximum allocation", - mediumNodes.get(0).getTotalCapability(), result); + assertEquals( + mediumNodes.get(0).getTotalCapability(), result, "With a single node added, the ClusterNodeTracker did not " + + "return that node's resources as the maximum allocation"); nodeTracker.addNode(smallNode); result = nodeTracker.getMaxAllowedAllocation(); - assertEquals("With two nodes added, the ClusterNodeTracker did not " + assertEquals( + Resource.newInstance(4096, 2, Collections.singletonMap("test1", 4L)) +, result, "With two nodes added, the ClusterNodeTracker did not " + "return a the maximum allocation that was the max of their aggregate " - + "resources", - Resource.newInstance(4096, 2, Collections.singletonMap("test1", 4L)), - result); + + "resources"); nodeTracker.removeNode(smallNode.getNodeID()); result = nodeTracker.getMaxAllowedAllocation(); - assertEquals("After removing a node, the ClusterNodeTracker did not " - + "recalculate the adjusted maximum allocation correctly", - mediumNodes.get(0).getTotalCapability(), result); + assertEquals( + mediumNodes.get(0).getTotalCapability(), result, "After removing a node, the ClusterNodeTracker did not " + + "recalculate the adjusted maximum allocation correctly"); nodeTracker.addNode(largeNode); result = nodeTracker.getMaxAllowedAllocation(); - assertEquals("With two nodes added, the ClusterNodeTracker did not " + assertEquals( + Resource.newInstance(10240, 4, Collections.singletonMap("test1", 2L)) +, result, "With two nodes added, the ClusterNodeTracker did not " + "return a the maximum allocation that was the max of their aggregate " - + "resources", - Resource.newInstance(10240, 4, Collections.singletonMap("test1", 2L)), - result); + + "resources"); nodeTracker.removeNode(largeNode.getNodeID()); result = nodeTracker.getMaxAllowedAllocation(); - assertEquals("After removing a node, the ClusterNodeTracker did not " - + "recalculate the adjusted maximum allocation correctly", - mediumNodes.get(0).getTotalCapability(), result); + assertEquals( + mediumNodes.get(0).getTotalCapability(), result, "After removing a node, the ClusterNodeTracker did not " + + "recalculate the adjusted maximum allocation correctly"); nodeTracker.removeNode(mediumNode.getNodeID()); result = nodeTracker.getMaxAllowedAllocation(); - assertEquals("After removing all nodes, the ClusterNodeTracker did not " - + "return the configured maximum allocation", maximum, result); + assertEquals(maximum, result, "After removing all nodes, the ClusterNodeTracker did not " + + "return the configured maximum allocation"); nodeTracker.addNode(smallNode); nodeTracker.addNode(mediumNode); @@ -186,11 +186,11 @@ public void testMaxAllowedAllocation() { result = nodeTracker.getMaxAllowedAllocation(); - assertEquals("With three nodes added, the ClusterNodeTracker did not " + assertEquals( + Resource.newInstance(10240, 4, Collections.singletonMap("test1", 4L)) +, result, "With three nodes added, the ClusterNodeTracker did not " + "return a the maximum allocation that was the max of their aggregate " - + "resources", - Resource.newInstance(10240, 4, Collections.singletonMap("test1", 4L)), - result); + + "resources"); nodeTracker.removeNode(smallNode.getNodeID()); nodeTracker.removeNode(mediumNode.getNodeID()); @@ -198,7 +198,7 @@ public void testMaxAllowedAllocation() { result = nodeTracker.getMaxAllowedAllocation(); - assertEquals("After removing all nodes, the ClusterNodeTracker did not " - + "return the configured maximum allocation", maximum, result); + assertEquals(maximum, result, "After removing all nodes, the ClusterNodeTracker did not " + + "return the configured maximum allocation"); } } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestConfigurationMutationACLPolicies.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestConfigurationMutationACLPolicies.java index f59c43761628d..9395092551f3b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestConfigurationMutationACLPolicies.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestConfigurationMutationACLPolicies.java @@ -27,15 +27,15 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf.QueueAdminConfigurationMutationACLPolicy; import org.apache.hadoop.yarn.webapp.dao.QueueConfigInfo; import org.apache.hadoop.yarn.webapp.dao.SchedConfUpdateInfo; -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.Collections; import java.util.Map; -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.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; @@ -54,7 +54,7 @@ public class TestConfigurationMutationACLPolicies { private static final Map EMPTY_MAP = Collections.emptyMap(); - @Before + @BeforeEach public void setUp() throws IOException { rmContext = mock(RMContext.class); scheduler = mock(MutableConfScheduler.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/scheduler/TestPartitionQueueMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestPartitionQueueMetrics.java index 8424f195e29eb..b844fb212414b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestPartitionQueueMetrics.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestPartitionQueueMetrics.java @@ -21,6 +21,7 @@ import static org.apache.hadoop.test.MetricsAsserts.assertCounter; import static org.apache.hadoop.test.MetricsAsserts.assertGauge; import static org.apache.hadoop.test.MetricsAsserts.getMetrics; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -36,10 +37,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueueMetrics; import org.apache.hadoop.yarn.server.utils.BuilderUtils; 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; public class TestPartitionQueueMetrics { @@ -48,14 +49,14 @@ public class TestPartitionQueueMetrics { private MetricsSystem ms; - @Before + @BeforeEach public void setUp() { ms = new MetricsSystemImpl(); QueueMetrics.clearQueueMetrics(); PartitionQueueMetrics.clearQueueMetrics(); } - @After + @AfterEach public void tearDown() { ms.shutdown(); } @@ -114,10 +115,10 @@ public void testSinglePartitionWithSingleLevelQueueMetrics() PartitionQueueMetrics pq1 = new PartitionQueueMetrics(ms, "root.q1", parentQueue, true, CONF, "x"); - Assert.assertTrue("Name of registry should be \"" + Assertions.assertTrue(pq1.registry.info().name() + .compareTo(PartitionQueueMetrics.P_RECORD_INFO.name()) == 0, "Name of registry should be \"" + PartitionQueueMetrics.P_RECORD_INFO.name() + "\", but was \"" - + pq1.registry.info().name() + "\".", pq1.registry.info().name() - .compareTo(PartitionQueueMetrics.P_RECORD_INFO.name()) == 0); + + pq1.registry.info().name() + "\"."); } /** @@ -640,57 +641,60 @@ public void testThreeLevelWithUserMetrics() { * * @throws Exception */ - @Test(expected = NullPointerException.class) + @Test public void testSinglePartitionWithSingleLevelQueueMetricsWithoutUserMetrics() throws Exception { - String parentQueueName = "root"; - Queue parentQueue = mock(Queue.class); - String user = "alice"; + assertThrows(NullPointerException.class, ()->{ + String parentQueueName = "root"; + Queue parentQueue = mock(Queue.class); + String user = "alice"; - QueueMetrics root = QueueMetrics.forQueue("root", null, false, CONF); - when(parentQueue.getMetrics()).thenReturn(root); - when(parentQueue.getQueueName()).thenReturn(parentQueueName); - CSQueueMetrics q1 = - CSQueueMetrics.forQueue("root.q1", parentQueue, false, CONF); - CSQueueMetrics q2 = - CSQueueMetrics.forQueue("root.q2", parentQueue, false, CONF); + QueueMetrics root = QueueMetrics.forQueue("root", null, false, CONF); + when(parentQueue.getMetrics()).thenReturn(root); + when(parentQueue.getQueueName()).thenReturn(parentQueueName); + CSQueueMetrics q1 = + CSQueueMetrics.forQueue("root.q1", parentQueue, false, CONF); + CSQueueMetrics q2 = + CSQueueMetrics.forQueue("root.q2", parentQueue, false, CONF); - AppSchedulingInfo app = mockApp(user); + AppSchedulingInfo app = mockApp(user); - q1.submitApp(user, false); - q1.submitAppAttempt(user, false); + q1.submitApp(user, false); + q1.submitAppAttempt(user, false); - root.setAvailableResourcesToQueue("x", - Resources.createResource(200 * GB, 200)); + root.setAvailableResourcesToQueue("x", + Resources.createResource(200 * GB, 200)); - q1.incrPendingResources("x", user, 2, Resource.newInstance(1024, 1)); + q1.incrPendingResources("x", user, 2, Resource.newInstance(1024, 1)); - MetricsSource partitionSource = partitionSource(q1.getMetricsSystem(), "x"); - MetricsSource rootQueueSource = - queueSource(q1.getMetricsSystem(), "x", parentQueueName); - MetricsSource q1Source = queueSource(q1.getMetricsSystem(), "x", "root.q1"); - MetricsSource q1UserSource = - userSource(q1.getMetricsSystem(), "x", user, "root.q1"); + MetricsSource partitionSource = partitionSource(q1.getMetricsSystem(), "x"); + MetricsSource rootQueueSource = + queueSource(q1.getMetricsSystem(), "x", parentQueueName); + MetricsSource q1Source = queueSource(q1.getMetricsSystem(), "x", "root.q1"); + MetricsSource q1UserSource = + userSource(q1.getMetricsSystem(), "x", user, "root.q1"); - checkResources(partitionSource, 0, 0, 0, 200 * GB, 200, 2 * GB, 2, 2); - checkResources(rootQueueSource, 0, 0, 0, 200 * GB, 200, 2 * GB, 2, 2); - checkResources(q1Source, 0, 0, 0, 0, 0, 2 * GB, 2, 2); - checkResources(q1UserSource, 0, 0, 0, 0, 0, 2 * GB, 2, 2); + checkResources(partitionSource, 0, 0, 0, 200 * GB, 200, 2 * GB, 2, 2); + checkResources(rootQueueSource, 0, 0, 0, 200 * GB, 200, 2 * GB, 2, 2); + checkResources(q1Source, 0, 0, 0, 0, 0, 2 * GB, 2, 2); + checkResources(q1UserSource, 0, 0, 0, 0, 0, 2 * GB, 2, 2); - q2.incrPendingResources("x", user, 3, Resource.newInstance(1024, 1)); - MetricsSource q2Source = queueSource(q2.getMetricsSystem(), "x", "root.q2"); - MetricsSource q2UserSource = - userSource(q1.getMetricsSystem(), "x", user, "root.q2"); + q2.incrPendingResources("x", user, 3, Resource.newInstance(1024, 1)); + MetricsSource q2Source = queueSource(q2.getMetricsSystem(), "x", "root.q2"); + MetricsSource q2UserSource = + userSource(q1.getMetricsSystem(), "x", user, "root.q2"); - checkResources(partitionSource, 0, 0, 0, 0, 0, 5 * GB, 5, 5); - checkResources(rootQueueSource, 0, 0, 0, 0, 0, 5 * GB, 5, 5); - checkResources(q2Source, 0, 0, 0, 0, 0, 3 * GB, 3, 3); - checkResources(q2UserSource, 0, 0, 0, 0, 0, 3 * GB, 3, 3); + checkResources(partitionSource, 0, 0, 0, 0, 0, 5 * GB, 5, 5); + checkResources(rootQueueSource, 0, 0, 0, 0, 0, 5 * GB, 5, 5); + checkResources(q2Source, 0, 0, 0, 0, 0, 3 * GB, 3, 3); + checkResources(q2UserSource, 0, 0, 0, 0, 0, 3 * GB, 3, 3); + + q1.finishAppAttempt(app.getApplicationId(), app.isPending(), app.getUser(), + false); + q1.finishApp(user, RMAppState.FINISHED, false); + }); - q1.finishAppAttempt(app.getApplicationId(), app.isPending(), app.getUser(), - false); - q1.finishApp(user, RMAppState.FINISHED, false); } public static MetricsSource partitionSource(MetricsSystem ms, 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/TestQueueMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestQueueMetrics.java index 68a3b0f39daef..3fb212679cade 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestQueueMetrics.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestQueueMetrics.java @@ -33,9 +33,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler; import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; @@ -65,7 +65,7 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceMetricsChecker.ResourceMetricsKey.RESERVED_MB; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceMetricsChecker.ResourceMetricsKey.RESERVED_V_CORES; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; +import static org.junit.jupiter.api.Assertions.assertNull; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -82,7 +82,7 @@ private static Queue createMockQueue(QueueMetrics metrics) { private static final Configuration conf = new Configuration(); private MetricsSystem ms; - @Before + @BeforeEach public void setUp() { ms = new MetricsSystemImpl(); QueueMetrics.clearQueueMetrics(); @@ -747,15 +747,15 @@ public void testMetricsCache() { QueueMetrics metrics = QueueMetrics.forQueue(ms, leafQueueName, parentQueue1, true, conf); - Assert.assertNotNull("QueueMetrics for A shoudn't be null", metrics); + Assertions.assertNotNull(metrics, "QueueMetrics for A shoudn't be null"); // Re-register to check for cache hit, shouldn't blow up metrics-system... // also, verify parent-metrics QueueMetrics alterMetrics = QueueMetrics.forQueue(ms, leafQueueName, parentQueue1, true, conf); - Assert.assertNotNull("QueueMetrics for alterMetrics shoudn't be null", - alterMetrics); + Assertions.assertNotNull( + alterMetrics, "QueueMetrics for alterMetrics shoudn't be null"); } finally { ms.shutdown(); } @@ -788,9 +788,9 @@ public void testCollectAllMetrics() { // do not collect all metrics AppMetricsChecker.create() .checkAgainst(queueSource, false); - Assert.fail(); + Assertions.fail(); } catch (AssertionError e) { - Assert.assertTrue( + Assertions.assertTrue( e.getMessage().contains("Expected exactly one metric for name ")); } // collect all metrics 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/TestQueueMetricsForCustomResources.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestQueueMetricsForCustomResources.java index 7acfe5da0ec68..f62432fb1b307 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestQueueMetricsForCustomResources.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestQueueMetricsForCustomResources.java @@ -32,8 +32,8 @@ 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.util.Collections; import java.util.HashMap; @@ -84,8 +84,8 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceMetricsChecker.ResourceMetricsKey.AGGREGATE_PREEMPTED_SECONDS_CUSTOM_RES1; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceMetricsChecker.ResourceMetricsKey.AGGREGATE_PREEMPTED_SECONDS_CUSTOM_RES2; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestQueueMetrics.queueSource; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; public class TestQueueMetricsForCustomResources { public enum MetricsForCustomResource { @@ -100,7 +100,7 @@ public enum MetricsForCustomResource { private Resource defaultResource; private MetricsSystem ms; - @Before + @BeforeEach public void setUp() { ms = new MetricsSystemImpl(); QueueMetrics.clearQueueMetrics(); @@ -156,13 +156,13 @@ private static void assertCustomResourceValueInternal( MetricsForCustomResource metricsType, String resourceName, long expectedValue, Long value) { assertNotNull( - "QueueMetrics should have custom resource metrics value " + - "for resource: " + resourceName, value); - assertEquals(String.format( + value, "QueueMetrics should have custom resource metrics value " + + "for resource: " + resourceName); + assertEquals(expectedValue +, (long) value, String.format( "QueueMetrics should have custom resource metrics value %d " + "for resource: %s for metrics type %s", - expectedValue, resourceName, metricsType), expectedValue, - (long) value); + expectedValue, resourceName, metricsType)); } private static Map getCustomResourcesWithValue(long value) { 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/TestResourceUsage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestResourceUsage.java index a2bd9e520ee95..aaf1f79e4b226 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestResourceUsage.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestResourceUsage.java @@ -25,8 +25,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.yarn.api.records.Resource; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -136,8 +136,8 @@ private void internalTestModifyAndRead(String label) throws Exception { } void check(int mem, int cpu, Resource res) { - Assert.assertEquals(mem, res.getMemorySize()); - Assert.assertEquals(cpu, res.getVirtualCores()); + Assertions.assertEquals(mem, res.getMemorySize()); + Assertions.assertEquals(cpu, res.getVirtualCores()); } @Test 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/TestSchedulerApplicationAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerApplicationAttempt.java index eae0bb3342b94..d45066988ff05 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerApplicationAttempt.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerApplicationAttempt.java @@ -19,7 +19,7 @@ import java.util.ArrayList; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils.toSchedulerKey; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -48,10 +48,10 @@ import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey; import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.After; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import org.junit.jupiter.api.Test; public class TestSchedulerApplicationAttempt { @@ -59,7 +59,7 @@ public class TestSchedulerApplicationAttempt { private Configuration conf = new Configuration(); - @After + @AfterEach public void tearDown() { QueueMetrics.clearQueueMetrics(); DefaultMetricsSystem.shutdown(); @@ -349,15 +349,15 @@ public void testAllResourceUsage() throws Exception { app.attemptResourceUsage.incReserved("X", r2); app.attemptResourceUsage.incReserved("Y", r2); - assertTrue("getUsedResources expected " + Resource.newInstance(3072, 4) - + " but was " + app.getResourceUsageReport().getUsedResources(), - Resources.equals(Resource.newInstance(3072, 4), - app.getResourceUsageReport().getUsedResources())); - assertTrue("getReservedResources expected " + Resource.newInstance(2048, 2) + assertTrue( + Resources.equals(Resource.newInstance(3072, 4), + app.getResourceUsageReport().getUsedResources()), "getUsedResources expected " + Resource.newInstance(3072, 4) + + " but was " + app.getResourceUsageReport().getUsedResources()); + assertTrue( + Resources.equals(Resource.newInstance(2048, 2), + app.getResourceUsageReport().getReservedResources()), "getReservedResources expected " + Resource.newInstance(2048, 2) + " but was " - + app.getResourceUsageReport().getReservedResources(), - Resources.equals(Resource.newInstance(2048, 2), - app.getResourceUsageReport().getReservedResources())); + + app.getResourceUsageReport().getReservedResources()); } @Test @@ -413,30 +413,30 @@ public void testHasPendingResourceRequest() throws Exception { requests.get(1).setAllocationRequestId(1L); app.updateResourceRequests(requests); - assertTrue("Reported no pending resource requests for no label when " - + "resource requests for no label are pending (exclusive partitions)", - app.hasPendingResourceRequest("", - SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY)); - assertTrue("Reported no pending resource requests for label with pending " - + "resource requests (exclusive partitions)", - app.hasPendingResourceRequest("label1", - SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY)); - assertFalse("Reported pending resource requests for label with no pending " - + "resource requests (exclusive partitions)", - app.hasPendingResourceRequest("label2", - SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY)); - - assertTrue("Reported no pending resource requests for no label when " - + "resource requests for no label are pending (relaxed partitions)", - app.hasPendingResourceRequest("", - SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY)); - assertTrue("Reported no pending resource requests for label with pending " - + "resource requests (relaxed partitions)", - app.hasPendingResourceRequest("label1", - SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY)); - assertTrue("Reported no pending resource requests for label with no " - + "pending resource requests (relaxed partitions)", - app.hasPendingResourceRequest("label2", - SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY)); + assertTrue( + app.hasPendingResourceRequest("", + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), "Reported no pending resource requests for no label when " + + "resource requests for no label are pending (exclusive partitions)"); + assertTrue( + app.hasPendingResourceRequest("label1", + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), "Reported no pending resource requests for label with pending " + + "resource requests (exclusive partitions)"); + assertFalse( + app.hasPendingResourceRequest("label2", + SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), "Reported pending resource requests for label with no pending " + + "resource requests (exclusive partitions)"); + + assertTrue( + app.hasPendingResourceRequest("", + SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY), "Reported no pending resource requests for no label when " + + "resource requests for no label are pending (relaxed partitions)"); + assertTrue( + app.hasPendingResourceRequest("label1", + SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY), "Reported no pending resource requests for label with pending " + + "resource requests (relaxed partitions)"); + assertTrue( + app.hasPendingResourceRequest("label2", + SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY), "Reported no pending resource requests for label with no " + + "pending resource requests (relaxed partitions)"); } } 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/TestSchedulerHealth.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerHealth.java index 933eaf91fe4f2..b0c239d1630de 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerHealth.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerHealth.java @@ -40,8 +40,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.io.IOException; @@ -84,18 +84,18 @@ public void testCounts() { sh.updateSchedulerReservationCounts(value); sh.updateSchedulerReleaseCounts(value); - Assert.assertEquals(value, sh.getAllocationCount().longValue()); - Assert.assertEquals(value, sh.getReleaseCount().longValue()); - Assert.assertEquals(value, sh.getReservationCount().longValue()); - Assert.assertEquals(value, sh.getPreemptionCount().longValue()); + Assertions.assertEquals(value, sh.getAllocationCount().longValue()); + Assertions.assertEquals(value, sh.getReleaseCount().longValue()); + Assertions.assertEquals(value, sh.getReservationCount().longValue()); + Assertions.assertEquals(value, sh.getPreemptionCount().longValue()); - Assert.assertEquals(value * (i + 1), sh.getAggregateAllocationCount() + Assertions.assertEquals(value * (i + 1), sh.getAggregateAllocationCount() .longValue()); - Assert.assertEquals(value * (i + 1), sh.getAggregateReleaseCount() + Assertions.assertEquals(value * (i + 1), sh.getAggregateReleaseCount() .longValue()); - Assert.assertEquals(value * (i + 1), sh.getAggregateReservationCount() + Assertions.assertEquals(value * (i + 1), sh.getAggregateReservationCount() .longValue()); - Assert.assertEquals(value * (i + 1), sh.getAggregatePreemptionCount() + Assertions.assertEquals(value * (i + 1), sh.getAggregatePreemptionCount() .longValue()); } @@ -109,50 +109,49 @@ public void testOperationDetails() { sh.updateRelease(now, NodeId.newInstance("testhost", 1234), ContainerId.fromString("container_1427562107907_0002_01_000001"), "testqueue"); - Assert.assertEquals("container_1427562107907_0002_01_000001", sh + Assertions.assertEquals("container_1427562107907_0002_01_000001", sh .getLastReleaseDetails().getContainerId().toString()); - Assert.assertEquals("testhost:1234", sh.getLastReleaseDetails().getNodeId() + Assertions.assertEquals("testhost:1234", sh.getLastReleaseDetails().getNodeId() .toString()); - Assert.assertEquals("testqueue", sh.getLastReleaseDetails().getQueue()); - Assert.assertEquals(now, sh.getLastReleaseDetails().getTimestamp()); - Assert.assertEquals(0, sh.getLastSchedulerRunTime()); + Assertions.assertEquals("testqueue", sh.getLastReleaseDetails().getQueue()); + Assertions.assertEquals(now, sh.getLastReleaseDetails().getTimestamp()); + Assertions.assertEquals(0, sh.getLastSchedulerRunTime()); now = Time.now(); sh.updateReservation(now, NodeId.newInstance("testhost1", 1234), ContainerId.fromString("container_1427562107907_0003_01_000001"), "testqueue1"); - Assert.assertEquals("container_1427562107907_0003_01_000001", sh + Assertions.assertEquals("container_1427562107907_0003_01_000001", sh .getLastReservationDetails().getContainerId().toString()); - Assert.assertEquals("testhost1:1234", sh.getLastReservationDetails() + Assertions.assertEquals("testhost1:1234", sh.getLastReservationDetails() .getNodeId().toString()); - Assert - .assertEquals("testqueue1", sh.getLastReservationDetails().getQueue()); - Assert.assertEquals(now, sh.getLastReservationDetails().getTimestamp()); - Assert.assertEquals(0, sh.getLastSchedulerRunTime()); + Assertions.assertEquals("testqueue1", sh.getLastReservationDetails().getQueue()); + Assertions.assertEquals(now, sh.getLastReservationDetails().getTimestamp()); + Assertions.assertEquals(0, sh.getLastSchedulerRunTime()); now = Time.now(); sh.updateAllocation(now, NodeId.newInstance("testhost2", 1234), ContainerId.fromString("container_1427562107907_0004_01_000001"), "testqueue2"); - Assert.assertEquals("container_1427562107907_0004_01_000001", sh + Assertions.assertEquals("container_1427562107907_0004_01_000001", sh .getLastAllocationDetails().getContainerId().toString()); - Assert.assertEquals("testhost2:1234", sh.getLastAllocationDetails() + Assertions.assertEquals("testhost2:1234", sh.getLastAllocationDetails() .getNodeId().toString()); - Assert.assertEquals("testqueue2", sh.getLastAllocationDetails().getQueue()); - Assert.assertEquals(now, sh.getLastAllocationDetails().getTimestamp()); - Assert.assertEquals(0, sh.getLastSchedulerRunTime()); + Assertions.assertEquals("testqueue2", sh.getLastAllocationDetails().getQueue()); + Assertions.assertEquals(now, sh.getLastAllocationDetails().getTimestamp()); + Assertions.assertEquals(0, sh.getLastSchedulerRunTime()); now = Time.now(); sh.updatePreemption(now, NodeId.newInstance("testhost3", 1234), ContainerId.fromString("container_1427562107907_0005_01_000001"), "testqueue3"); - Assert.assertEquals("container_1427562107907_0005_01_000001", sh + Assertions.assertEquals("container_1427562107907_0005_01_000001", sh .getLastPreemptionDetails().getContainerId().toString()); - Assert.assertEquals("testhost3:1234", sh.getLastPreemptionDetails() + Assertions.assertEquals("testhost3:1234", sh.getLastPreemptionDetails() .getNodeId().toString()); - Assert.assertEquals("testqueue3", sh.getLastPreemptionDetails().getQueue()); - Assert.assertEquals(now, sh.getLastPreemptionDetails().getTimestamp()); - Assert.assertEquals(0, sh.getLastSchedulerRunTime()); + Assertions.assertEquals("testqueue3", sh.getLastPreemptionDetails().getQueue()); + Assertions.assertEquals(now, sh.getLastPreemptionDetails().getTimestamp()); + Assertions.assertEquals(0, sh.getLastSchedulerRunTime()); } @Test @@ -161,15 +160,15 @@ public void testResourceUpdate() { long now = Time.now(); sh.updateSchedulerRunDetails(now, Resource.newInstance(1024, 1), Resource.newInstance(2048, 1)); - Assert.assertEquals(now, sh.getLastSchedulerRunTime()); - Assert.assertEquals(Resource.newInstance(1024, 1), + Assertions.assertEquals(now, sh.getLastSchedulerRunTime()); + Assertions.assertEquals(Resource.newInstance(1024, 1), sh.getResourcesAllocated()); - Assert.assertEquals(Resource.newInstance(2048, 1), + Assertions.assertEquals(Resource.newInstance(2048, 1), sh.getResourcesReserved()); now = Time.now(); sh.updateSchedulerReleaseDetails(now, Resource.newInstance(3072, 1)); - Assert.assertEquals(now, sh.getLastSchedulerRunTime()); - Assert.assertEquals(Resource.newInstance(3072, 1), + Assertions.assertEquals(now, sh.getLastSchedulerRunTime()); + Assertions.assertEquals(Resource.newInstance(3072, 1), sh.getResourcesReleased()); } @@ -246,13 +245,13 @@ public void testCapacitySchedulerAllocation() throws Exception { .getSchedulerHealth(); // Now SchedulerHealth records last container allocated, aggregated // allocation account will not be changed - Assert.assertEquals(1, sh.getAllocationCount().longValue()); - Assert.assertEquals(Resource.newInstance(1 * 1024, 1), + Assertions.assertEquals(1, sh.getAllocationCount().longValue()); + Assertions.assertEquals(Resource.newInstance(1 * 1024, 1), sh.getResourcesAllocated()); - Assert.assertEquals(2, sh.getAggregateAllocationCount().longValue()); - Assert.assertEquals("host_0:1234", sh.getLastAllocationDetails() + Assertions.assertEquals(2, sh.getAggregateAllocationCount().longValue()); + Assertions.assertEquals("host_0:1234", sh.getLastAllocationDetails() .getNodeId().toString()); - Assert.assertEquals("root.default", sh.getLastAllocationDetails() + Assertions.assertEquals("root.default", sh.getLastAllocationDetails() .getQueue()); Task task_0_2 = @@ -261,13 +260,13 @@ public void testCapacitySchedulerAllocation() throws Exception { application_0.schedule(); nodeUpdate(nm_0); - Assert.assertEquals(1, sh.getAllocationCount().longValue()); - Assert.assertEquals(Resource.newInstance(2 * 1024, 1), + Assertions.assertEquals(1, sh.getAllocationCount().longValue()); + Assertions.assertEquals(Resource.newInstance(2 * 1024, 1), sh.getResourcesAllocated()); - Assert.assertEquals(3, sh.getAggregateAllocationCount().longValue()); - Assert.assertEquals("host_0:1234", sh.getLastAllocationDetails() + Assertions.assertEquals(3, sh.getAggregateAllocationCount().longValue()); + Assertions.assertEquals("host_0:1234", sh.getLastAllocationDetails() .getNodeId().toString()); - Assert.assertEquals("root.default", sh.getLastAllocationDetails() + Assertions.assertEquals("root.default", sh.getLastAllocationDetails() .getQueue()); } @@ -325,13 +324,13 @@ public void testCapacitySchedulerReservation() throws Exception { SchedulerHealth sh = ((CapacityScheduler) resourceManager.getResourceScheduler()) .getSchedulerHealth(); - Assert.assertEquals(1, sh.getAllocationCount().longValue()); - Assert.assertEquals(Resource.newInstance(1024, 1), + Assertions.assertEquals(1, sh.getAllocationCount().longValue()); + Assertions.assertEquals(Resource.newInstance(1024, 1), sh.getResourcesAllocated()); - Assert.assertEquals(1, sh.getAggregateAllocationCount().longValue()); - Assert.assertEquals("host_0:1234", sh.getLastAllocationDetails() + Assertions.assertEquals(1, sh.getAggregateAllocationCount().longValue()); + Assertions.assertEquals("host_0:1234", sh.getLastAllocationDetails() .getNodeId().toString()); - Assert.assertEquals("root.default", sh.getLastAllocationDetails() + Assertions.assertEquals("root.default", sh.getLastAllocationDetails() .getQueue()); Task task_0_1 = @@ -340,14 +339,14 @@ public void testCapacitySchedulerReservation() throws Exception { application_0.schedule(); nodeUpdate(nm_0); - Assert.assertEquals(0, sh.getAllocationCount().longValue()); - Assert.assertEquals(1, sh.getReservationCount().longValue()); - Assert.assertEquals(Resource.newInstance(2 * 1024, 1), + Assertions.assertEquals(0, sh.getAllocationCount().longValue()); + Assertions.assertEquals(1, sh.getReservationCount().longValue()); + Assertions.assertEquals(Resource.newInstance(2 * 1024, 1), sh.getResourcesReserved()); - Assert.assertEquals(1, sh.getAggregateAllocationCount().longValue()); - Assert.assertEquals("host_0:1234", sh.getLastAllocationDetails() + Assertions.assertEquals(1, sh.getAggregateAllocationCount().longValue()); + Assertions.assertEquals("host_0:1234", sh.getLastAllocationDetails() .getNodeId().toString()); - Assert.assertEquals("root.default", sh.getLastAllocationDetails() + Assertions.assertEquals("root.default", sh.getLastAllocationDetails() .getQueue()); } } 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/TestSchedulerOvercommit.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerOvercommit.java index 119bc6f657e64..d9bcaa5c05e1f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerOvercommit.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerOvercommit.java @@ -21,11 +21,11 @@ import static java.util.Collections.emptyList; import static java.util.Collections.singletonList; import static java.util.Collections.singletonMap; -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 java.util.Collections; import java.util.HashSet; @@ -63,9 +63,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; import org.apache.hadoop.yarn.server.utils.BuilderUtils; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -114,7 +114,7 @@ public abstract class TestSchedulerOvercommit { * Setup the cluster with: an RM, a NM and an application for test. * @throws Exception If it cannot set up the cluster. */ - @Before + @BeforeEach public void setup() throws Exception { LOG.info("Setting up the test cluster..."); @@ -162,7 +162,7 @@ protected Configuration getConfiguration() { * Stops the default application and the RM (with the scheduler). * @throws Exception If it cannot stop the cluster. */ - @After + @AfterEach public void cleanup() throws Exception { LOG.info("Cleaning up the test cluster..."); @@ -215,14 +215,14 @@ public void testReduceNoTimeout() throws Exception { // Verify no NPE is trigger in schedule after resource is updated am.addRequests(new String[] {"127.0.0.1", "127.0.0.2"}, 3 * GB, 1, 1); AllocateResponse allocResponse2 = am.schedule(); - assertTrue("Shouldn't have enough resource to allocate containers", - allocResponse2.getAllocatedContainers().isEmpty()); + assertTrue( + allocResponse2.getAllocatedContainers().isEmpty(), "Shouldn't have enough resource to allocate containers"); // Try 10 times as scheduling is an async process for (int i = 0; i < 10; i++) { Thread.sleep(INTERVAL); allocResponse2 = am.schedule(); - assertTrue("Shouldn't have enough resource to allocate containers", - allocResponse2.getAllocatedContainers().isEmpty()); + assertTrue( + allocResponse2.getAllocatedContainers().isEmpty(), "Shouldn't have enough resource to allocate containers"); } } @@ -461,14 +461,14 @@ public void testEndToEnd() throws Exception { // verify no NPE is trigger in schedule after resource is updated am.addRequests(new String[] {"127.0.0.1", "127.0.0.2"}, 3 * GB, 1, 1); AllocateResponse allocResponse2 = am.schedule(); - assertTrue("Shouldn't have enough resource to allocate containers", - allocResponse2.getAllocatedContainers().isEmpty()); + assertTrue( + allocResponse2.getAllocatedContainers().isEmpty(), "Shouldn't have enough resource to allocate containers"); // try 10 times as scheduling is an async process for (int i = 0; i < 10; i++) { Thread.sleep(100); allocResponse2 = am.schedule(); - assertTrue("Shouldn't have enough resource to allocate containers", - allocResponse2.getAllocatedContainers().isEmpty()); + assertTrue( + allocResponse2.getAllocatedContainers().isEmpty(), "Shouldn't have enough resource to allocate containers"); } // increase the resources again to 5 GB to schedule the 3GB container @@ -560,7 +560,7 @@ protected Container createContainer( allocated = response.getAllocatedContainers(); nm.nodeHeartbeat(true); } - assertFalse("Cannot create the container", allocated.isEmpty()); + assertFalse(allocated.isEmpty(), "Cannot create the container"); assertEquals(1, allocated.size()); final Container c = allocated.get(0); @@ -608,9 +608,9 @@ public static void assertContainerKilled( * @param time Actual time to check. */ public static void assertTime(final long expectedTime, final long time) { - assertTrue("Too short: " + time + "ms", time > expectedTime); - assertTrue("Too long: " + time + "ms", - time < (expectedTime + 2 * INTERVAL)); + assertTrue(time > expectedTime, "Too short: " + time + "ms"); + assertTrue( + time < (expectedTime + 2 * INTERVAL), "Too long: " + time + "ms"); } /** @@ -632,7 +632,7 @@ public static void assertNoPreemption(final PreemptionMessage msg) { */ public static void assertPreemption( final ContainerId containerId, final PreemptionMessage msg) { - assertNotNull("Expected a preemption message", msg); + assertNotNull(msg, "Expected a preemption message"); Set preemptContainers = new HashSet<>(); if (msg.getContract() != null) { for (PreemptionContainer c : msg.getContract().getContainers()) { @@ -659,10 +659,10 @@ public static void assertMemory(ResourceScheduler scheduler, NodeId nmId, SchedulerNodeReport nmReport = scheduler.getNodeReport(nmId); assertNotNull(nmReport); Resource used = nmReport.getUsedResource(); - assertEquals("Used memory", expectedUsed, used.getMemorySize()); + assertEquals(expectedUsed, used.getMemorySize(), "Used memory"); Resource available = nmReport.getAvailableResource(); - assertEquals("Available memory", - expectedAvailable, available.getMemorySize()); + assertEquals( + expectedAvailable, available.getMemorySize(), "Available memory"); } /** 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/TestSchedulerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java index bc41c359c2e1a..c32a3fe7456a1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java @@ -24,9 +24,9 @@ .GREATER_THEN_MAX_ALLOCATION; import static org.apache.hadoop.yarn.exceptions .InvalidResourceRequestException.InvalidResourceType.LESS_THAN_ZERO; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.Mockito.mock; @@ -98,10 +98,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.Assert; -import org.junit.Before; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; import org.junit.Rule; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; import org.junit.rules.ExpectedException; @@ -127,7 +128,7 @@ private void initResourceTypes() { CustomResourceTypesConfigurationProvider.initResourceTypes(3, "G"); } - @Before + @BeforeEach public void setUp() { initResourceTypes(); //this needs to be initialized after initResourceTypes is called @@ -140,7 +141,8 @@ public void setUp() { .build()); } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testNormalizeRequest() { ResourceCalculator resourceCalculator = new DefaultResourceCalculator(); @@ -205,7 +207,8 @@ public void testNormalizeRequest() { ask.getCapability().getMemorySize()); } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testNormalizeRequestWithDominantResourceCalculator() { ResourceCalculator resourceCalculator = new DominantResourceCalculator(); @@ -238,7 +241,8 @@ public void testNormalizeRequestWithDominantResourceCalculator() { assertEquals(2048, ask.getCapability().getMemorySize()); } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testValidateResourceRequestWithErrorLabelsPermission() throws IOException { // mock queue and scheduler @@ -411,8 +415,8 @@ public void testValidateResourceRequestWithErrorLabelsPermission() rmContext.getNodeLabelManager().removeFromClusterNodeLabels( Arrays.asList("x")); } - Assert.assertTrue("InvalidLabelResourceRequestException expected", - invalidlabelexception); + Assertions.assertTrue( + invalidlabelexception, "InvalidLabelResourceRequestException expected"); // queue is "*", always succeeded try { // set queue accessible node labels to empty @@ -535,7 +539,7 @@ public void testValidateResourceRequestWithErrorLabelsPermission() .newResourceRequest(mock(Priority.class), "*", resource, 1, "x"); normalizeAndvalidateRequest(resReq1, "queue", scheduler, rmContext, maxResource); - Assert.assertEquals(RMNodeLabelsManager.NO_LABEL, + Assertions.assertEquals(RMNodeLabelsManager.NO_LABEL, resReq1.getNodeLabelExpression()); } catch (InvalidResourceRequestException e) { assertEquals("Invalid resource request, node label not enabled but " @@ -543,7 +547,8 @@ public void testValidateResourceRequestWithErrorLabelsPermission() } } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testValidateResourceRequest() throws IOException { ResourceScheduler mockScheduler = mock(ResourceScheduler.class); @@ -747,8 +752,8 @@ public ApplicationMasterProtocol run() { rm.stop(); - Assert.assertTrue( - "Didn't not catch InvalidResourceBlacklistRequestException", error); + Assertions.assertTrue( + error, "Didn't not catch InvalidResourceBlacklistRequestException"); } private void waitForLaunchedState(RMAppAttempt attempt) @@ -760,7 +765,7 @@ private void waitForLaunchedState(RMAppAttempt attempt) + "Current state is " + attempt.getAppAttemptState()); Thread.sleep(1000); } - Assert.assertEquals(attempt.getAppAttemptState(), + Assertions.assertEquals(attempt.getAppAttemptState(), RMAppAttemptState.LAUNCHED); } @@ -776,7 +781,7 @@ public void testCreateAbnormalContainerStatus() { ContainerStatus cd = SchedulerUtils.createAbnormalContainerStatus( ContainerId.newContainerId(ApplicationAttemptId.newInstance( ApplicationId.newInstance(System.currentTimeMillis(), 1), 1), 1), "x"); - Assert.assertEquals(ContainerExitStatus.ABORTED, cd.getExitStatus()); + Assertions.assertEquals(ContainerExitStatus.ABORTED, cd.getExitStatus()); } @Test @@ -784,10 +789,11 @@ public void testCreatePreemptedContainerStatus() { ContainerStatus cd = SchedulerUtils.createPreemptedContainerStatus( ContainerId.newContainerId(ApplicationAttemptId.newInstance( ApplicationId.newInstance(System.currentTimeMillis(), 1), 1), 1), "x"); - Assert.assertEquals(ContainerExitStatus.PREEMPTED, cd.getExitStatus()); + Assertions.assertEquals(ContainerExitStatus.PREEMPTED, cd.getExitStatus()); } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testNormalizeNodeLabelExpression() throws IOException { // mock queue and scheduler @@ -821,12 +827,12 @@ public void testNormalizeNodeLabelExpression() mock(Priority.class), ResourceRequest.ANY, resource, 1); normalizeAndvalidateRequest(resReq, "queue", scheduler, rmContext, maxResource); - Assert.assertEquals("x", resReq.getNodeLabelExpression()); + Assertions.assertEquals("x", resReq.getNodeLabelExpression()); resReq.setNodeLabelExpression(" y "); normalizeAndvalidateRequest(resReq, "queue", scheduler, rmContext, maxResource); - Assert.assertEquals("y", resReq.getNodeLabelExpression()); + Assertions.assertEquals("y", resReq.getNodeLabelExpression()); } catch (InvalidResourceRequestException e) { e.printStackTrace(); fail("Should be valid when request labels is a subset of queue labels"); @@ -1000,64 +1006,64 @@ public void testEnforcePartitionExclusivity() { // RR label unset and app label does not match. Nothing should happen. SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet, null); - Assert.assertNull(rr.getNodeLabelExpression()); + Assertions.assertNull(rr.getNodeLabelExpression()); SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet, appLabel); - Assert.assertNull(rr.getNodeLabelExpression()); + Assertions.assertNull(rr.getNodeLabelExpression()); // RR label and app label do not match. Nothing should happen. rr.setNodeLabelExpression(dummyLabel); SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet, null); - Assert.assertEquals(dummyLabel, rr.getNodeLabelExpression()); + Assertions.assertEquals(dummyLabel, rr.getNodeLabelExpression()); SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet, appLabel); - Assert.assertEquals(dummyLabel, rr.getNodeLabelExpression()); + Assertions.assertEquals(dummyLabel, rr.getNodeLabelExpression()); // RR label matches but app label does not. RR label should be set // to app label rr.setNodeLabelExpression(enforcedExclusiveLabel); SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet, null); - Assert.assertNull(rr.getNodeLabelExpression()); + Assertions.assertNull(rr.getNodeLabelExpression()); rr.setNodeLabelExpression(enforcedExclusiveLabel); SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet, appLabel); - Assert.assertEquals(appLabel, rr.getNodeLabelExpression()); + Assertions.assertEquals(appLabel, rr.getNodeLabelExpression()); // RR label unset and app label matches. RR label should be set // to app label rr.setNodeLabelExpression(null); SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet, enforcedExclusiveLabel); - Assert.assertEquals(enforcedExclusiveLabel, rr.getNodeLabelExpression()); + Assertions.assertEquals(enforcedExclusiveLabel, rr.getNodeLabelExpression()); // RR label does not match and app label matches. RR label should be set // to app label rr.setNodeLabelExpression(dummyLabel); SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet, enforcedExclusiveLabel); - Assert.assertEquals(enforcedExclusiveLabel, rr.getNodeLabelExpression()); + Assertions.assertEquals(enforcedExclusiveLabel, rr.getNodeLabelExpression()); // RR label and app label matches. Nothing should happen. rr.setNodeLabelExpression(enforcedExclusiveLabel); SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet, enforcedExclusiveLabel); - Assert.assertEquals(enforcedExclusiveLabel, rr.getNodeLabelExpression()); + Assertions.assertEquals(enforcedExclusiveLabel, rr.getNodeLabelExpression()); // Unconfigured label: nothing should happen. rr.setNodeLabelExpression(null); SchedulerUtils.enforcePartitionExclusivity(rr, null, appLabel); - Assert.assertNull(rr.getNodeLabelExpression()); + Assertions.assertNull(rr.getNodeLabelExpression()); rr.setNodeLabelExpression(dummyLabel); SchedulerUtils.enforcePartitionExclusivity(rr, null, appLabel); - Assert.assertEquals(dummyLabel, rr.getNodeLabelExpression()); + Assertions.assertEquals(dummyLabel, rr.getNodeLabelExpression()); rr.setNodeLabelExpression(enforcedExclusiveLabel); SchedulerUtils.enforcePartitionExclusivity(rr, null, appLabel); - Assert.assertEquals(enforcedExclusiveLabel, rr.getNodeLabelExpression()); + Assertions.assertEquals(enforcedExclusiveLabel, rr.getNodeLabelExpression()); } @Test @@ -1075,57 +1081,57 @@ public void testEnforcePartitionExclusivityMultipleLabels() { // RR label unset and app label does not match. Nothing should happen. SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet, null); - Assert.assertNull(rr.getNodeLabelExpression()); + Assertions.assertNull(rr.getNodeLabelExpression()); SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet, appLabel); - Assert.assertNull(rr.getNodeLabelExpression()); + Assertions.assertNull(rr.getNodeLabelExpression()); // RR label and app label do not match. Nothing should happen. rr.setNodeLabelExpression(dummyLabel); SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet, null); - Assert.assertEquals(dummyLabel, rr.getNodeLabelExpression()); + Assertions.assertEquals(dummyLabel, rr.getNodeLabelExpression()); SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet, appLabel); - Assert.assertEquals(dummyLabel, rr.getNodeLabelExpression()); + Assertions.assertEquals(dummyLabel, rr.getNodeLabelExpression()); // RR label matches but app label does not. RR label should be set // to app label rr.setNodeLabelExpression(enforcedLabel1); SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet, null); - Assert.assertNull(rr.getNodeLabelExpression()); + Assertions.assertNull(rr.getNodeLabelExpression()); rr.setNodeLabelExpression(enforcedLabel2); SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet, appLabel); - Assert.assertEquals(appLabel, rr.getNodeLabelExpression()); + Assertions.assertEquals(appLabel, rr.getNodeLabelExpression()); // RR label unset and app label matches. RR label should be set // to app label rr.setNodeLabelExpression(null); SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet, enforcedLabel1); - Assert.assertEquals(enforcedLabel1, rr.getNodeLabelExpression()); + Assertions.assertEquals(enforcedLabel1, rr.getNodeLabelExpression()); // RR label does not match and app label matches. RR label should be set // to app label rr.setNodeLabelExpression(dummyLabel); SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet, enforcedLabel2); - Assert.assertEquals(enforcedLabel2, rr.getNodeLabelExpression()); + Assertions.assertEquals(enforcedLabel2, rr.getNodeLabelExpression()); // RR label and app label matches. Nothing should happen. rr.setNodeLabelExpression(enforcedLabel1); SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet, enforcedLabel1); - Assert.assertEquals(enforcedLabel1, rr.getNodeLabelExpression()); + Assertions.assertEquals(enforcedLabel1, rr.getNodeLabelExpression()); // RR label and app label don't match, but they're both enforced labels. // RR label should be set to app label. rr.setNodeLabelExpression(enforcedLabel2); SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet, enforcedLabel1); - Assert.assertEquals(enforcedLabel1, rr.getNodeLabelExpression()); + Assertions.assertEquals(enforcedLabel1, rr.getNodeLabelExpression()); } public static SchedulerApplication @@ -1144,13 +1150,13 @@ public void testEnforcePartitionExclusivityMultipleLabels() { SchedulerApplication app = applications.get(appId); // verify application is added. - Assert.assertNotNull(app); - Assert.assertEquals("user", app.getUser()); + Assertions.assertNotNull(app); + Assertions.assertEquals("user", app.getUser()); AppRemovedSchedulerEvent appRemoveEvent = new AppRemovedSchedulerEvent(appId, RMAppState.FINISHED); handler.handle(appRemoveEvent); - Assert.assertNull(applications.get(appId)); + Assertions.assertNull(applications.get(appId)); return 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/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..bb07e6ce27933 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 @@ -34,8 +34,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.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -65,7 +66,8 @@ public YarnConfiguration getConf() { return conf; } - @Test (timeout = 10000) + @Test + @Timeout(value = 10) public void testMultipleAllocationRequestIds() throws Exception { YarnConfiguration conf = getConf(); MockRM rm = new MockRM(conf); @@ -90,13 +92,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()); + Assertions.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()); + Assertions.assertEquals(2, allocated.size()); for (Container container : allocated) { checkAllocatedContainer(container, 2 * GB, nm2.getNodeId(), 20); } @@ -107,7 +109,8 @@ public void testMultipleAllocationRequestIds() throws Exception { } } - @Test (timeout = 10000) + @Test + @Timeout(value = 10) public void testMultipleAllocationRequestDiffPriority() throws Exception { YarnConfiguration conf = getConf(); MockRM rm = new MockRM(conf); @@ -132,7 +135,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()); + Assertions.assertEquals(2, allocated.size()); for (Container container : allocated) { checkAllocatedContainer(container, 2 * GB, nm2.getNodeId(), 20); } @@ -140,7 +143,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()); + Assertions.assertEquals(1, allocated.size()); checkAllocatedContainer(allocated.get(0), 2 * GB, nm1.getNodeId(), 10); } finally { if (rm != null) { @@ -151,13 +154,14 @@ 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, + Assertions.assertEquals(memory, allocated.getResource().getMemorySize()); + Assertions.assertEquals(nodeId, allocated.getNodeId()); + Assertions.assertEquals(allocationRequestId, allocated.getAllocationRequestId()); } - @Test (timeout = 10000) + @Test + @Timeout(value = 10) public void testMultipleAppsWithAllocationReqId() throws Exception { YarnConfiguration conf = getConf(); MockRM rm = new MockRM(conf); @@ -193,7 +197,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()); + Assertions.assertEquals(1, allocated.size()); checkAllocatedContainer(allocated.get(0), 1 * GB, nm1.getNodeId(), 5L); // Submit another application @@ -212,7 +216,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()); + Assertions.assertEquals(1, allocated.size()); checkAllocatedContainer(allocated.get(0), 2 * GB, nm2.getNodeId(), 5L); // Now submit app2 RR with allocationReqId = 10 @@ -222,7 +226,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()); + Assertions.assertEquals(1, allocated.size()); checkAllocatedContainer(allocated.get(0), 3 * GB, nm1.getNodeId(), 10L); // Now submit app1 RR with allocationReqId = 10 @@ -232,7 +236,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()); + Assertions.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/scheduler/constraint/TestAllocationTagsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestAllocationTagsManager.java index 09e2086db91f6..ccb8566c6aa2f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestAllocationTagsManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestAllocationTagsManager.java @@ -33,9 +33,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import java.util.List; @@ -49,7 +49,7 @@ public class TestAllocationTagsManager { private RMContext rmContext; - @Before + @BeforeEach public void setup() { MockRM rm = new MockRM(); rm.start(); @@ -79,22 +79,22 @@ public void testMultipleAddRemoveContainer() { atm.addContainer(nodeId, cid1, tags1); atm.addContainer(nodeId, cid2, tags2); atm.addContainer(nodeId, cid3, tags3); - Assert.assertEquals(2L, + Assertions.assertEquals(2L, (long) atm.getAllocationTagsWithCount(nodeId).get("mapper")); - Assert.assertEquals(1L, + Assertions.assertEquals(1L, (long) atm.getAllocationTagsWithCount(nodeId).get("reducer")); // remove container1 atm.removeContainer(nodeId, cid1, tags1); - Assert.assertEquals(1L, + Assertions.assertEquals(1L, (long) atm.getAllocationTagsWithCount(nodeId).get("mapper")); - Assert.assertNull(atm.getAllocationTagsWithCount(nodeId).get("reducer")); + Assertions.assertNull(atm.getAllocationTagsWithCount(nodeId).get("reducer")); // remove the same container again, the reducer no longer exists, // make sure there is no NPE here atm.removeContainer(nodeId, cid1, tags1); - Assert.assertNull(atm.getAllocationTagsWithCount(nodeId).get("mapper")); - Assert.assertNull(atm.getAllocationTagsWithCount(nodeId).get("reducer")); + Assertions.assertNull(atm.getAllocationTagsWithCount(nodeId).get("mapper")); + Assertions.assertNull(atm.getAllocationTagsWithCount(nodeId).get("reducer")); } @Test @@ -138,7 +138,7 @@ public void testAllocationTagsManagerSimpleCases() ImmutableSet.of("service")); // Get Node Cardinality of app1 on node1, with tag "mapper" - Assert.assertEquals(1, + Assertions.assertEquals(1, atm.getNodeCardinalityByOp(NodeId.fromString("host1:123"), AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(1), @@ -146,11 +146,11 @@ public void testAllocationTagsManagerSimpleCases() Long::max)); // Get Rack Cardinality of app1 on rack0, with tag "mapper" - Assert.assertEquals(2, atm.getRackCardinality("rack0", + Assertions.assertEquals(2, atm.getRackCardinality("rack0", TestUtils.getMockApplicationId(1), "mapper")); // Get Node Cardinality of app1 on node2, with tag "mapper/reducer", op=min - Assert.assertEquals(1, + Assertions.assertEquals(1, atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"), AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(1), @@ -158,7 +158,7 @@ public void testAllocationTagsManagerSimpleCases() Long::min)); // Get Node Cardinality of app1 on node2, with tag "mapper/reducer", op=max - Assert.assertEquals(2, + Assertions.assertEquals(2, atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"), AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(1), @@ -166,7 +166,7 @@ public void testAllocationTagsManagerSimpleCases() Long::max)); // Get Node Cardinality of app1 on node2, with tag "mapper/reducer", op=sum - Assert.assertEquals(3, + Assertions.assertEquals(3, atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"), AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(1), @@ -174,17 +174,17 @@ public void testAllocationTagsManagerSimpleCases() Long::sum)); // Get Node Cardinality by passing single tag. - Assert.assertEquals(1, + Assertions.assertEquals(1, atm.getNodeCardinality(NodeId.fromString("host2:123"), TestUtils.getMockApplicationId(1), "mapper")); - Assert.assertEquals(2, + Assertions.assertEquals(2, atm.getNodeCardinality(NodeId.fromString("host2:123"), TestUtils.getMockApplicationId(1), "reducer")); // Get Node Cardinality of app1 on node2, with tag "no_existed/reducer", // op=min - Assert.assertEquals(0, + Assertions.assertEquals(0, atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"), AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(1), @@ -193,41 +193,41 @@ public void testAllocationTagsManagerSimpleCases() // Get Node Cardinality of app1 on node2, with tag "", op=max // (Expect this returns #containers from app1 on node2) - Assert.assertEquals(2, + Assertions.assertEquals(2, atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"), AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(1), null), Long::max)); // Get Node Cardinality of app1 on node2, with empty tag set, op=max - Assert.assertEquals(2, + Assertions.assertEquals(2, atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"), AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(1), null), Long::max)); // Get Cardinality of app1 on node2, with empty tag set, op=max - Assert.assertEquals(2, + Assertions.assertEquals(2, atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"), AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(1), ImmutableSet.of()), Long::max)); // Get Node Cardinality of all apps on node2, with empty tag set, op=sum - Assert.assertEquals(4, atm.getNodeCardinalityByOp( + Assertions.assertEquals(4, atm.getNodeCardinalityByOp( NodeId.fromString("host2:123"), AllocationTags.createGlobalAllocationTags(ImmutableSet.of()), Long::sum)); // Get Node Cardinality of app_1 on node2, with empty tag set, op=sum - Assert.assertEquals(3, + Assertions.assertEquals(3, atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"), AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(1), ImmutableSet.of()), Long::sum)); // Get Node Cardinality of app_1 on node2, with empty tag set, op=sum - Assert.assertEquals(1, + Assertions.assertEquals(1, atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"), AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(2), ImmutableSet.of()), @@ -253,7 +253,7 @@ public void testAllocationTagsManagerSimpleCases() // Expect all cardinality to be 0 // Get Cardinality of app1 on node1, with tag "mapper" - Assert.assertEquals(0, + Assertions.assertEquals(0, atm.getNodeCardinalityByOp(NodeId.fromString("host1:123"), AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(1), @@ -261,7 +261,7 @@ public void testAllocationTagsManagerSimpleCases() Long::max)); // Get Node Cardinality of app1 on node2, with tag "mapper/reducer", op=min - Assert.assertEquals(0, + Assertions.assertEquals(0, atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"), AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(1), @@ -269,7 +269,7 @@ public void testAllocationTagsManagerSimpleCases() Long::min)); // Get Node Cardinality of app1 on node2, with tag "mapper/reducer", op=max - Assert.assertEquals(0, + Assertions.assertEquals(0, atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"), AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(1), @@ -277,7 +277,7 @@ public void testAllocationTagsManagerSimpleCases() Long::max)); // Get Node Cardinality of app1 on node2, with tag "mapper/reducer", op=sum - Assert.assertEquals(0, + Assertions.assertEquals(0, atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"), AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(1), @@ -286,20 +286,20 @@ public void testAllocationTagsManagerSimpleCases() // Get Node Cardinality of app1 on node2, with tag "", op=max // (Expect this returns #containers from app1 on node2) - Assert.assertEquals(0, + Assertions.assertEquals(0, atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"), AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(1), ImmutableSet.of(TestUtils.getMockApplicationId(1).toString())), Long::max)); - Assert.assertEquals(0, + Assertions.assertEquals(0, atm.getNodeCardinality(NodeId.fromString("host2:123"), TestUtils.getMockApplicationId(1), TestUtils.getMockApplicationId(1).toString())); // Get Node Cardinality of app1 on node2, with empty tag set, op=max - Assert.assertEquals(0, + Assertions.assertEquals(0, atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"), AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(1), @@ -307,13 +307,13 @@ public void testAllocationTagsManagerSimpleCases() Long::max)); // Get Node Cardinality of all apps on node2, with empty tag set, op=sum - Assert.assertEquals(0, atm.getNodeCardinalityByOp( + Assertions.assertEquals(0, atm.getNodeCardinalityByOp( NodeId.fromString("host2:123"), AllocationTags.createGlobalAllocationTags(ImmutableSet.of()), Long::sum)); // Get Node Cardinality of app_1 on node2, with empty tag set, op=sum - Assert.assertEquals(0, + Assertions.assertEquals(0, atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"), AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(1), @@ -321,7 +321,7 @@ public void testAllocationTagsManagerSimpleCases() Long::sum)); // Get Node Cardinality of app_2 on node2, with empty tag set, op=sum - Assert.assertEquals(0, + Assertions.assertEquals(0, atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"), AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(1), @@ -370,32 +370,32 @@ public void testAllocationTagsManagerRackMapping() TestUtils.getMockContainerId(2, 3), ImmutableSet.of("service")); // Get Rack Cardinality of app1 on rack0, with tag "mapper" - Assert.assertEquals(1, atm.getRackCardinality("rack0", + Assertions.assertEquals(1, atm.getRackCardinality("rack0", TestUtils.getMockApplicationId(1), "mapper")); // Get Rack Cardinality of app2 on rack0, with tag "reducer" - Assert.assertEquals(2, atm.getRackCardinality("rack0", + Assertions.assertEquals(2, atm.getRackCardinality("rack0", TestUtils.getMockApplicationId(2), "reducer")); // Get Rack Cardinality of all apps on rack0, with tag "reducer" - Assert.assertEquals(3, atm.getRackCardinality("rack0", null, "reducer")); + Assertions.assertEquals(3, atm.getRackCardinality("rack0", null, "reducer")); // Get Rack Cardinality of app_1 on rack0, with empty tag set, op=max - Assert.assertEquals(1, atm.getRackCardinalityByOp("rack0", + Assertions.assertEquals(1, atm.getRackCardinalityByOp("rack0", AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(1), ImmutableSet.of()), Long::max)); // Get Rack Cardinality of app_1 on rack0, with empty tag set, op=min - Assert.assertEquals(1, atm.getRackCardinalityByOp("rack0", + Assertions.assertEquals(1, atm.getRackCardinalityByOp("rack0", AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(1), ImmutableSet.of()), Long::min)); // Get Rack Cardinality of all apps on rack0, with empty tag set, op=min - Assert.assertEquals(3, atm.getRackCardinalityByOp("rack0", + Assertions.assertEquals(3, atm.getRackCardinalityByOp("rack0", AllocationTags.createGlobalAllocationTags(ImmutableSet.of()), Long::max)); } @@ -445,12 +445,12 @@ public void testAllocationTagsManagerMemoryAfterCleanup() { TestUtils.getMockContainerId(2, 3), ImmutableSet.of("service")); // Check internal data structure - Assert.assertEquals(0, + Assertions.assertEquals(0, atm.getGlobalNodeMapping().getTypeToTagsWithCount().size()); - Assert.assertEquals(0, atm.getPerAppNodeMappings().size()); - Assert.assertEquals(0, + Assertions.assertEquals(0, atm.getPerAppNodeMappings().size()); + Assertions.assertEquals(0, atm.getGlobalRackMapping().getTypeToTagsWithCount().size()); - Assert.assertEquals(0, atm.getPerAppRackMappings().size()); + Assertions.assertEquals(0, atm.getPerAppRackMappings().size()); } @Test @@ -490,8 +490,8 @@ public void testQueryCardinalityWithIllegalParameters() { } catch (InvalidAllocationTagsQueryException e1) { caughtException = true; } - Assert.assertTrue("should fail because of nodeId specified", - caughtException); + Assertions.assertTrue( + caughtException, "should fail because of nodeId specified"); // No op caughtException = false; @@ -504,8 +504,8 @@ public void testQueryCardinalityWithIllegalParameters() { } catch (InvalidAllocationTagsQueryException e1) { caughtException = true; } - Assert.assertTrue("should fail because of nodeId specified", - caughtException); + Assertions.assertTrue( + caughtException, "should fail because of nodeId specified"); } @Test @@ -588,12 +588,12 @@ public void testNodeAllocationTagsAggregation() //******************************** AllocationTags tags = AllocationTags .createSingleAppAllocationTags(app1, ImmutableSet.of("A", "C")); - Assert.assertEquals(2, atm.getNodeCardinalityByOp(host1, tags, Long::max)); - Assert.assertEquals(0, atm.getNodeCardinalityByOp(host1, tags, Long::min)); - Assert.assertEquals(1, atm.getNodeCardinalityByOp(host2, tags, Long::max)); - Assert.assertEquals(0, atm.getNodeCardinalityByOp(host2, tags, Long::min)); - Assert.assertEquals(0, atm.getNodeCardinalityByOp(host3, tags, Long::max)); - Assert.assertEquals(0, atm.getNodeCardinalityByOp(host3, tags, Long::min)); + Assertions.assertEquals(2, atm.getNodeCardinalityByOp(host1, tags, Long::max)); + Assertions.assertEquals(0, atm.getNodeCardinalityByOp(host1, tags, Long::min)); + Assertions.assertEquals(1, atm.getNodeCardinalityByOp(host2, tags, Long::max)); + Assertions.assertEquals(0, atm.getNodeCardinalityByOp(host2, tags, Long::min)); + Assertions.assertEquals(0, atm.getNodeCardinalityByOp(host3, tags, Long::max)); + Assertions.assertEquals(0, atm.getNodeCardinalityByOp(host3, tags, Long::min)); //******************************** // 2) not-self (app2, app3) @@ -618,20 +618,20 @@ public void testNodeAllocationTagsAggregation() tags = AllocationTags.createOtherAppAllocationTags(app1, ImmutableSet.of("A", "B")); - Assert.assertEquals(4, atm.getNodeCardinalityByOp(host1, tags, Long::max)); - Assert.assertEquals(0, atm.getNodeCardinalityByOp(host1, tags, Long::min)); - Assert.assertEquals(4, atm.getNodeCardinalityByOp(host1, tags, Long::sum)); + Assertions.assertEquals(4, atm.getNodeCardinalityByOp(host1, tags, Long::max)); + Assertions.assertEquals(0, atm.getNodeCardinalityByOp(host1, tags, Long::min)); + Assertions.assertEquals(4, atm.getNodeCardinalityByOp(host1, tags, Long::sum)); //******************************** // 3) app-id/app2 (app2) //******************************** tags = AllocationTags .createSingleAppAllocationTags(app2, ImmutableSet.of("A", "B")); - Assert.assertEquals(3, atm.getNodeCardinalityByOp(host1, tags, Long::max)); - Assert.assertEquals(0, atm.getNodeCardinalityByOp(host1, tags, Long::min)); - Assert.assertEquals(2, atm.getNodeCardinalityByOp(host2, tags, Long::max)); - Assert.assertEquals(1, atm.getNodeCardinalityByOp(host2, tags, Long::min)); - Assert.assertEquals(3, atm.getNodeCardinalityByOp(host2, tags, Long::sum)); + Assertions.assertEquals(3, atm.getNodeCardinalityByOp(host1, tags, Long::max)); + Assertions.assertEquals(0, atm.getNodeCardinalityByOp(host1, tags, Long::min)); + Assertions.assertEquals(2, atm.getNodeCardinalityByOp(host2, tags, Long::max)); + Assertions.assertEquals(1, atm.getNodeCardinalityByOp(host2, tags, Long::min)); + Assertions.assertEquals(3, atm.getNodeCardinalityByOp(host2, tags, Long::sum)); //******************************** @@ -639,20 +639,20 @@ public void testNodeAllocationTagsAggregation() //******************************** tags = AllocationTags .createGlobalAllocationTags(ImmutableSet.of("A")); - Assert.assertEquals(6, atm.getNodeCardinalityByOp(host1, tags, Long::sum)); - Assert.assertEquals(1, atm.getNodeCardinalityByOp(host2, tags, Long::sum)); - Assert.assertEquals(0, atm.getNodeCardinalityByOp(host3, tags, Long::sum)); + Assertions.assertEquals(6, atm.getNodeCardinalityByOp(host1, tags, Long::sum)); + Assertions.assertEquals(1, atm.getNodeCardinalityByOp(host2, tags, Long::sum)); + Assertions.assertEquals(0, atm.getNodeCardinalityByOp(host3, tags, Long::sum)); tags = AllocationTags .createGlobalAllocationTags(ImmutableSet.of("A", "B")); - Assert.assertEquals(7, atm.getNodeCardinalityByOp(host1, tags, Long::sum)); - Assert.assertEquals(4, atm.getNodeCardinalityByOp(host2, tags, Long::sum)); - Assert.assertEquals(0, atm.getNodeCardinalityByOp(host3, tags, Long::sum)); - Assert.assertEquals(6, atm.getNodeCardinalityByOp(host1, tags, Long::max)); - Assert.assertEquals(3, atm.getNodeCardinalityByOp(host2, tags, Long::max)); - Assert.assertEquals(0, atm.getNodeCardinalityByOp(host3, tags, Long::max)); - Assert.assertEquals(1, atm.getNodeCardinalityByOp(host1, tags, Long::min)); - Assert.assertEquals(1, atm.getNodeCardinalityByOp(host2, tags, Long::min)); - Assert.assertEquals(0, atm.getNodeCardinalityByOp(host3, tags, Long::min)); + Assertions.assertEquals(7, atm.getNodeCardinalityByOp(host1, tags, Long::sum)); + Assertions.assertEquals(4, atm.getNodeCardinalityByOp(host2, tags, Long::sum)); + Assertions.assertEquals(0, atm.getNodeCardinalityByOp(host3, tags, Long::sum)); + Assertions.assertEquals(6, atm.getNodeCardinalityByOp(host1, tags, Long::max)); + Assertions.assertEquals(3, atm.getNodeCardinalityByOp(host2, tags, Long::max)); + Assertions.assertEquals(0, atm.getNodeCardinalityByOp(host3, tags, Long::max)); + Assertions.assertEquals(1, atm.getNodeCardinalityByOp(host1, tags, Long::min)); + Assertions.assertEquals(1, atm.getNodeCardinalityByOp(host2, tags, Long::min)); + Assertions.assertEquals(0, atm.getNodeCardinalityByOp(host3, tags, Long::min)); } } 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/constraint/TestAllocationTagsNamespace.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestAllocationTagsNamespace.java index b54631c4c7b46..cc8422512ec36 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestAllocationTagsNamespace.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestAllocationTagsNamespace.java @@ -19,8 +19,8 @@ import org.apache.hadoop.util.Sets; import org.apache.hadoop.yarn.api.records.AllocationTagNamespaceType; import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.util.HashMap; import java.util.Map; @@ -37,50 +37,50 @@ public void testNamespaceParse() throws InvalidAllocationTagsQueryException { String namespaceStr = "self"; namespace = TargetApplicationsNamespace.parse(namespaceStr); - Assert.assertEquals(AllocationTagNamespaceType.SELF, + Assertions.assertEquals(AllocationTagNamespaceType.SELF, namespace.getNamespaceType()); namespaceStr = "not-self"; namespace = TargetApplicationsNamespace.parse(namespaceStr); - Assert.assertEquals(AllocationTagNamespaceType.NOT_SELF, + Assertions.assertEquals(AllocationTagNamespaceType.NOT_SELF, namespace.getNamespaceType()); namespaceStr = "all"; namespace = TargetApplicationsNamespace.parse(namespaceStr); - Assert.assertEquals(AllocationTagNamespaceType.ALL, + Assertions.assertEquals(AllocationTagNamespaceType.ALL, namespace.getNamespaceType()); namespaceStr = "app-tag/spark-jobs"; namespace = TargetApplicationsNamespace.parse(namespaceStr); - Assert.assertEquals(AllocationTagNamespaceType.APP_TAG, + Assertions.assertEquals(AllocationTagNamespaceType.APP_TAG, namespace.getNamespaceType()); // Invalid app-tag namespace syntax try { namespaceStr = "app-tag/tag123/tag234"; TargetApplicationsNamespace.parse(namespaceStr); - Assert.fail("Parsing should fail as the given namespace is invalid"); + Assertions.fail("Parsing should fail as the given namespace is invalid"); } catch (Exception e) { e.printStackTrace(); - Assert.assertTrue(e instanceof InvalidAllocationTagsQueryException); - Assert.assertTrue(e.getMessage().startsWith( + Assertions.assertTrue(e instanceof InvalidAllocationTagsQueryException); + Assertions.assertTrue(e.getMessage().startsWith( "Invalid namespace string")); } ApplicationId applicationId = ApplicationId.newInstance(12345, 1); namespaceStr = "app-id/" + applicationId.toString(); namespace = TargetApplicationsNamespace.parse(namespaceStr); - Assert.assertEquals(AllocationTagNamespaceType.APP_ID, + Assertions.assertEquals(AllocationTagNamespaceType.APP_ID, namespace.getNamespaceType()); // Invalid app-id namespace syntax, invalid app ID. try { namespaceStr = "app-id/apppppp_12345_99999"; TargetApplicationsNamespace.parse(namespaceStr); - Assert.fail("Parsing should fail as the given app ID is invalid"); + Assertions.fail("Parsing should fail as the given app ID is invalid"); } catch (Exception e) { - Assert.assertTrue(e instanceof InvalidAllocationTagsQueryException); - Assert.assertTrue(e.getMessage().startsWith( + Assertions.assertTrue(e instanceof InvalidAllocationTagsQueryException); + Assertions.assertTrue(e.getMessage().startsWith( "Invalid application ID for app-id")); } @@ -88,11 +88,11 @@ public void testNamespaceParse() throws InvalidAllocationTagsQueryException { try { namespaceStr = "app-id"; TargetApplicationsNamespace.parse(namespaceStr); - Assert.fail("Parsing should fail as the given namespace" + Assertions.fail("Parsing should fail as the given namespace" + " is missing application ID"); } catch (Exception e) { - Assert.assertTrue(e instanceof InvalidAllocationTagsQueryException); - Assert.assertTrue(e.getMessage().startsWith( + Assertions.assertTrue(e instanceof InvalidAllocationTagsQueryException); + Assertions.assertTrue(e.getMessage().startsWith( "Missing the application ID in the namespace string")); } @@ -100,10 +100,10 @@ public void testNamespaceParse() throws InvalidAllocationTagsQueryException { try { namespaceStr = "non_exist_ns"; TargetApplicationsNamespace.parse(namespaceStr); - Assert.fail("Parsing should fail as the giving type is not supported."); + Assertions.fail("Parsing should fail as the giving type is not supported."); } catch (Exception e) { - Assert.assertTrue(e instanceof InvalidAllocationTagsQueryException); - Assert.assertTrue(e.getMessage().startsWith( + Assertions.assertTrue(e instanceof InvalidAllocationTagsQueryException); + Assertions.assertTrue(e.getMessage().startsWith( "Invalid namespace prefix")); } } @@ -124,10 +124,10 @@ public void testNamespaceEvaluation() throws namespace = TargetApplicationsNamespace.parse(namespaceStr); try { namespace.getNamespaceScope(); - Assert.fail("Call getNamespaceScope before evaluate is not allowed."); + Assertions.fail("Call getNamespaceScope before evaluate is not allowed."); } catch (Exception e) { - Assert.assertTrue(e instanceof IllegalStateException); - Assert.assertTrue(e.getMessage().contains( + Assertions.assertTrue(e instanceof IllegalStateException); + Assertions.assertTrue(e.getMessage().contains( "Evaluate must be called before a namespace can be consumed.")); } @@ -135,24 +135,24 @@ public void testNamespaceEvaluation() throws namespace = TargetApplicationsNamespace.parse(namespaceStr); targetApplications = new TargetApplications(app1, ImmutableSet.of(app1)); namespace.evaluate(targetApplications); - Assert.assertEquals(1, namespace.getNamespaceScope().size()); - Assert.assertEquals(app1, namespace.getNamespaceScope().iterator().next()); + Assertions.assertEquals(1, namespace.getNamespaceScope().size()); + Assertions.assertEquals(app1, namespace.getNamespaceScope().iterator().next()); namespaceStr = "not-self"; namespace = TargetApplicationsNamespace.parse(namespaceStr); targetApplications = new TargetApplications(app1, ImmutableSet.of(app1)); namespace.evaluate(targetApplications); - Assert.assertEquals(0, namespace.getNamespaceScope().size()); + Assertions.assertEquals(0, namespace.getNamespaceScope().size()); targetApplications = new TargetApplications(app1, ImmutableSet.of(app1, app2, app3)); namespace.evaluate(targetApplications); - Assert.assertEquals(2, namespace.getNamespaceScope().size()); - Assert.assertFalse(namespace.getNamespaceScope().contains(app1)); + Assertions.assertEquals(2, namespace.getNamespaceScope().size()); + Assertions.assertFalse(namespace.getNamespaceScope().contains(app1)); namespaceStr = "all"; namespace = TargetApplicationsNamespace.parse(namespaceStr); - Assert.assertEquals(AllocationTagNamespaceType.ALL, + Assertions.assertEquals(AllocationTagNamespaceType.ALL, namespace.getNamespaceType()); namespaceStr = "app-id/" + app2.toString(); @@ -160,8 +160,8 @@ public void testNamespaceEvaluation() throws targetApplications = new TargetApplications(app1, ImmutableSet.of(app1, app2, app3, app4, app5)); namespace.evaluate(targetApplications); - Assert.assertEquals(1, namespace.getNamespaceScope().size()); - Assert.assertEquals(app2, namespace.getNamespaceScope().iterator().next()); + Assertions.assertEquals(1, namespace.getNamespaceScope().size()); + Assertions.assertEquals(app2, namespace.getNamespaceScope().iterator().next()); /** * App to Application Tags @@ -182,21 +182,21 @@ public void testNamespaceEvaluation() throws namespace = TargetApplicationsNamespace.parse(namespaceStr); targetApplications = new TargetApplications(app1, appsWithTags); namespace.evaluate(targetApplications); - Assert.assertEquals(3, namespace.getNamespaceScope().size()); - Assert.assertTrue(Sets.difference(namespace.getNamespaceScope(), + Assertions.assertEquals(3, namespace.getNamespaceScope().size()); + Assertions.assertTrue(Sets.difference(namespace.getNamespaceScope(), ImmutableSet.of(app1, app2, app5)).isEmpty()); namespaceStr = "app-tag/B"; namespace = TargetApplicationsNamespace.parse(namespaceStr); namespace.evaluate(targetApplications); - Assert.assertEquals(2, namespace.getNamespaceScope().size()); - Assert.assertTrue(Sets.difference(namespace.getNamespaceScope(), + Assertions.assertEquals(2, namespace.getNamespaceScope().size()); + Assertions.assertTrue(Sets.difference(namespace.getNamespaceScope(), ImmutableSet.of(app1, app5)).isEmpty()); // Not exist namespaceStr = "app-tag/xyz"; namespace = TargetApplicationsNamespace.parse(namespaceStr); namespace.evaluate(targetApplications); - Assert.assertEquals(0, namespace.getNamespaceScope().size()); + Assertions.assertEquals(0, namespace.getNamespaceScope().size()); } } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestBatchedRequestsIterators.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestBatchedRequestsIterators.java index 0e7b7157c20bb..3ccf393af3371 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestBatchedRequestsIterators.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestBatchedRequestsIterators.java @@ -25,8 +25,8 @@ import org.apache.hadoop.yarn.api.records.SchedulingRequest; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.processor.BatchedRequests; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; /** * Test Request Iterator. @@ -48,7 +48,7 @@ public void testSerialIterator() throws Exception { long prevAllocId = 0; while (requestIterator.hasNext()) { SchedulingRequest request = requestIterator.next(); - Assert.assertTrue(request.getAllocationRequestId() > prevAllocId); + Assertions.assertTrue(request.getAllocationRequestId() > prevAllocId); prevAllocId = request.getAllocationRequestId(); } } @@ -71,9 +71,9 @@ public void testPopularTagsIterator() throws Exception { while (requestIterator.hasNext()) { SchedulingRequest request = requestIterator.next(); if (recCcount < 3) { - Assert.assertTrue(request.getAllocationTags().contains("pri")); + Assertions.assertTrue(request.getAllocationTags().contains("pri")); } else { - Assert.assertTrue(request.getAllocationTags().contains("bar") + Assertions.assertTrue(request.getAllocationTags().contains("bar") || request.getAllocationTags().contains("test")); } recCcount++; 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/constraint/TestPlacementConstraintManagerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestPlacementConstraintManagerService.java index d9376037de67d..78f753ec15ec3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestPlacementConstraintManagerService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestPlacementConstraintManagerService.java @@ -41,9 +41,9 @@ import org.apache.hadoop.yarn.api.resource.PlacementConstraint.And; import org.apache.hadoop.yarn.api.resource.PlacementConstraints; import org.apache.hadoop.yarn.server.utils.BuilderUtils; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** * Unit tests for {@link PlacementConstraintManagerService}. @@ -61,7 +61,7 @@ protected PlacementConstraintManagerService createPCM() { private Set sourceTag1, sourceTag2, sourceTag3, sourceTag4; private Map, PlacementConstraint> constraintMap1, constraintMap2; - @Before + @BeforeEach public void before() { this.pcm = createPCM(); @@ -93,94 +93,94 @@ public void before() { @Test public void testRegisterUnregisterApps() { - Assert.assertEquals(0, pcm.getNumRegisteredApplications()); + Assertions.assertEquals(0, pcm.getNumRegisteredApplications()); // Register two applications. pcm.registerApplication(appId1, constraintMap1); - Assert.assertEquals(1, pcm.getNumRegisteredApplications()); + Assertions.assertEquals(1, pcm.getNumRegisteredApplications()); Map, PlacementConstraint> constrMap = pcm.getConstraints(appId1); - Assert.assertNotNull(constrMap); - Assert.assertEquals(2, constrMap.size()); - Assert.assertNotNull(constrMap.get(sourceTag1)); - Assert.assertNotNull(constrMap.get(sourceTag2)); + Assertions.assertNotNull(constrMap); + Assertions.assertEquals(2, constrMap.size()); + Assertions.assertNotNull(constrMap.get(sourceTag1)); + Assertions.assertNotNull(constrMap.get(sourceTag2)); pcm.registerApplication(appId2, constraintMap2); - Assert.assertEquals(2, pcm.getNumRegisteredApplications()); + Assertions.assertEquals(2, pcm.getNumRegisteredApplications()); constrMap = pcm.getConstraints(appId2); - Assert.assertNotNull(constrMap); - Assert.assertEquals(1, constrMap.size()); - Assert.assertNotNull(constrMap.get(sourceTag3)); - Assert.assertNull(constrMap.get(sourceTag2)); + Assertions.assertNotNull(constrMap); + Assertions.assertEquals(1, constrMap.size()); + Assertions.assertNotNull(constrMap.get(sourceTag3)); + Assertions.assertNull(constrMap.get(sourceTag2)); // Try to register the same app again. pcm.registerApplication(appId2, constraintMap1); - Assert.assertEquals(2, pcm.getNumRegisteredApplications()); + Assertions.assertEquals(2, pcm.getNumRegisteredApplications()); // Unregister appId1. pcm.unregisterApplication(appId1); - Assert.assertEquals(1, pcm.getNumRegisteredApplications()); - Assert.assertNull(pcm.getConstraints(appId1)); - Assert.assertNotNull(pcm.getConstraints(appId2)); + Assertions.assertEquals(1, pcm.getNumRegisteredApplications()); + Assertions.assertNull(pcm.getConstraints(appId1)); + Assertions.assertNotNull(pcm.getConstraints(appId2)); } @Test public void testAddConstraint() { // Cannot add constraint to unregistered app. - Assert.assertEquals(0, pcm.getNumRegisteredApplications()); + Assertions.assertEquals(0, pcm.getNumRegisteredApplications()); pcm.addConstraint(appId1, sourceTag1, c1, false); - Assert.assertEquals(0, pcm.getNumRegisteredApplications()); + Assertions.assertEquals(0, pcm.getNumRegisteredApplications()); // Register application. pcm.registerApplication(appId1, new HashMap<>()); - Assert.assertEquals(1, pcm.getNumRegisteredApplications()); - Assert.assertEquals(0, pcm.getConstraints(appId1).size()); + Assertions.assertEquals(1, pcm.getNumRegisteredApplications()); + Assertions.assertEquals(0, pcm.getConstraints(appId1).size()); // Add two constraints. pcm.addConstraint(appId1, sourceTag1, c1, false); pcm.addConstraint(appId1, sourceTag2, c3, false); - Assert.assertEquals(2, pcm.getConstraints(appId1).size()); + Assertions.assertEquals(2, pcm.getConstraints(appId1).size()); // Constraint for sourceTag1 should not be replaced. pcm.addConstraint(appId1, sourceTag1, c2, false); - Assert.assertEquals(2, pcm.getConstraints(appId1).size()); - Assert.assertEquals(c1, pcm.getConstraint(appId1, sourceTag1)); - Assert.assertNotEquals(c2, pcm.getConstraint(appId1, sourceTag1)); + Assertions.assertEquals(2, pcm.getConstraints(appId1).size()); + Assertions.assertEquals(c1, pcm.getConstraint(appId1, sourceTag1)); + Assertions.assertNotEquals(c2, pcm.getConstraint(appId1, sourceTag1)); // Now c2 should replace c1 for sourceTag1. pcm.addConstraint(appId1, sourceTag1, c2, true); - Assert.assertEquals(2, pcm.getConstraints(appId1).size()); - Assert.assertEquals(c2, pcm.getConstraint(appId1, sourceTag1)); + Assertions.assertEquals(2, pcm.getConstraints(appId1).size()); + Assertions.assertEquals(c2, pcm.getConstraint(appId1, sourceTag1)); } @Test public void testGlobalConstraints() { - Assert.assertEquals(0, pcm.getNumGlobalConstraints()); + Assertions.assertEquals(0, pcm.getNumGlobalConstraints()); pcm.addGlobalConstraint(sourceTag1, c1, false); - Assert.assertEquals(1, pcm.getNumGlobalConstraints()); - Assert.assertNotNull(pcm.getGlobalConstraint(sourceTag1)); + Assertions.assertEquals(1, pcm.getNumGlobalConstraints()); + Assertions.assertNotNull(pcm.getGlobalConstraint(sourceTag1)); // Constraint for sourceTag1 should not be replaced. pcm.addGlobalConstraint(sourceTag1, c2, false); - Assert.assertEquals(1, pcm.getNumGlobalConstraints()); - Assert.assertEquals(c1, pcm.getGlobalConstraint(sourceTag1)); - Assert.assertNotEquals(c2, pcm.getGlobalConstraint(sourceTag1)); + Assertions.assertEquals(1, pcm.getNumGlobalConstraints()); + Assertions.assertEquals(c1, pcm.getGlobalConstraint(sourceTag1)); + Assertions.assertNotEquals(c2, pcm.getGlobalConstraint(sourceTag1)); // Now c2 should replace c1 for sourceTag1. pcm.addGlobalConstraint(sourceTag1, c2, true); - Assert.assertEquals(1, pcm.getNumGlobalConstraints()); - Assert.assertEquals(c2, pcm.getGlobalConstraint(sourceTag1)); + Assertions.assertEquals(1, pcm.getNumGlobalConstraints()); + Assertions.assertEquals(c2, pcm.getGlobalConstraint(sourceTag1)); pcm.removeGlobalConstraint(sourceTag1); - Assert.assertEquals(0, pcm.getNumGlobalConstraints()); + Assertions.assertEquals(0, pcm.getNumGlobalConstraints()); } @Test public void testValidateConstraint() { // At the moment we only disallow multiple source tags to be associated with // a constraint. TODO: More tests to be added for YARN-6621. - Assert.assertTrue(pcm.validateConstraint(sourceTag1, c1)); - Assert.assertFalse(pcm.validateConstraint(sourceTag4, c1)); + Assertions.assertTrue(pcm.validateConstraint(sourceTag1, c1)); + Assertions.assertFalse(pcm.validateConstraint(sourceTag4, c1)); } @Test @@ -194,10 +194,10 @@ public void testGetRequestConstraint() { // AC = null // GC = null constraint = pcm.getMultilevelConstraint(appId1, null, c1); - Assert.assertTrue(constraint.getConstraintExpr() instanceof And); + Assertions.assertTrue(constraint.getConstraintExpr() instanceof And); mergedConstraint = (And) constraint.getConstraintExpr(); - Assert.assertEquals(1, mergedConstraint.getChildren().size()); - Assert.assertEquals(c1, mergedConstraint.getChildren().get(0).build()); + Assertions.assertEquals(1, mergedConstraint.getChildren().size()); + Assertions.assertEquals(c1, mergedConstraint.getChildren().get(0).build()); // RC = null // AC = tag1->c1, tag2->c2 @@ -207,17 +207,17 @@ public void testGetRequestConstraint() { // registered constraint, we should get an empty AND constraint. constraint = pcm.getMultilevelConstraint(appId1, Sets.newHashSet("not_exist_tag"), null); - Assert.assertTrue(constraint.getConstraintExpr() instanceof And); + Assertions.assertTrue(constraint.getConstraintExpr() instanceof And); mergedConstraint = (And) constraint.getConstraintExpr(); // AND() - Assert.assertEquals(0, mergedConstraint.getChildren().size()); + Assertions.assertEquals(0, mergedConstraint.getChildren().size()); // if a mapping is found for a given source tag constraint = pcm.getMultilevelConstraint(appId1, sourceTag1, null); - Assert.assertTrue(constraint.getConstraintExpr() instanceof And); + Assertions.assertTrue(constraint.getConstraintExpr() instanceof And); mergedConstraint = (And) constraint.getConstraintExpr(); // AND(c1) - Assert.assertEquals(1, mergedConstraint.getChildren().size()); - Assert.assertEquals(c1, mergedConstraint.getChildren().get(0).build()); + Assertions.assertEquals(1, mergedConstraint.getChildren().size()); + Assertions.assertEquals(c1, mergedConstraint.getChildren().get(0).build()); pcm.unregisterApplication(appId1); // RC = null @@ -226,11 +226,11 @@ public void testGetRequestConstraint() { pcm.addGlobalConstraint(sourceTag1, c1, true); constraint = pcm.getMultilevelConstraint(appId1, Sets.newHashSet(sourceTag1), null); - Assert.assertTrue(constraint.getConstraintExpr() instanceof And); + Assertions.assertTrue(constraint.getConstraintExpr() instanceof And); mergedConstraint = (And) constraint.getConstraintExpr(); // AND(c1) - Assert.assertEquals(1, mergedConstraint.getChildren().size()); - Assert.assertEquals(c1, mergedConstraint.getChildren().get(0).build()); + Assertions.assertEquals(1, mergedConstraint.getChildren().size()); + Assertions.assertEquals(c1, mergedConstraint.getChildren().get(0).build()); pcm.removeGlobalConstraint(sourceTag1); // RC = c2 @@ -240,10 +240,10 @@ public void testGetRequestConstraint() { pcm.registerApplication(appId1, constraintMap1); // both RC, AC and GC should be respected constraint = pcm.getMultilevelConstraint(appId1, sourceTag1, c2); - Assert.assertTrue(constraint.getConstraintExpr() instanceof And); + Assertions.assertTrue(constraint.getConstraintExpr() instanceof And); mergedConstraint = (And) constraint.getConstraintExpr(); // AND(c1, c2, c3) - Assert.assertEquals(3, mergedConstraint.getChildren().size()); + Assertions.assertEquals(3, mergedConstraint.getChildren().size()); pcm.removeGlobalConstraint(sourceTag1); pcm.unregisterApplication(appId1); @@ -254,10 +254,10 @@ public void testGetRequestConstraint() { pcm.registerApplication(appId1, constraintMap1); constraint = pcm.getMultilevelConstraint(appId1, Sets.newHashSet(sourceTag1), c1); - Assert.assertTrue(constraint.getConstraintExpr() instanceof And); + Assertions.assertTrue(constraint.getConstraintExpr() instanceof And); mergedConstraint = (And) constraint.getConstraintExpr(); // AND(c1, c2) - Assert.assertEquals(2, mergedConstraint.getChildren().size()); + Assertions.assertEquals(2, mergedConstraint.getChildren().size()); pcm.removeGlobalConstraint(sourceTag1); pcm.unregisterApplication(appId1); } 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/constraint/TestPlacementConstraintsUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestPlacementConstraintsUtil.java index 7821bc5567104..3d528bb434c02 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestPlacementConstraintsUtil.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestPlacementConstraintsUtil.java @@ -67,9 +67,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.DiagnosticsCollector; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.GenericDiagnosticsCollector; import org.apache.hadoop.yarn.server.utils.BuilderUtils; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.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.mockito.Mockito; @@ -89,7 +89,7 @@ public class TestPlacementConstraintsUtil { constraintMap2, constraintMap3, constraintMap4; private AtomicLong requestID = new AtomicLong(0); - @Before + @BeforeEach public void setup() { MockRM rm = new MockRM(); rm.start(); @@ -196,9 +196,9 @@ public void testNodeAffinityAssignment() SchedulerNode schedulerNode =newSchedulerNode(currentNode.getHostName(), currentNode.getRackName(), currentNode.getNodeID()); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag1), schedulerNode, pcm, tm)); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag2), schedulerNode, pcm, tm)); } /** @@ -225,24 +225,24 @@ public void testNodeAffinityAssignment() tm.addContainer(n0_r1.getNodeID(), hbase_m, ImmutableSet.of("hbase-m")); // 'spark' placement on Node0 should now SUCCEED - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag1), schedulerNode0, pcm, tm)); // FAIL on the rest of the nodes - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag1), schedulerNode1, pcm, tm)); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag1), schedulerNode2, pcm, tm)); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag1), schedulerNode3, pcm, tm)); // Test diagnostics collector DiagnosticsCollector collector = new GenericDiagnosticsCollector(); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag1), schedulerNode1, pcm, tm, Optional.of(collector))); - Assert.assertNotNull(collector.getDiagnostics()); - Assert.assertTrue(collector.getDiagnostics().contains("ALLOCATION_TAG")); + Assertions.assertNotNull(collector.getDiagnostics()); + Assertions.assertTrue(collector.getDiagnostics().contains("ALLOCATION_TAG")); } @Test @@ -296,13 +296,13 @@ public void testMultiTagsPlacementConstraints() // n0 and n1 has A/B so they cannot satisfy the PC // n2 and n3 doesn't have A or B, so they can satisfy the PC - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(st1), schedulerNode0, pcm, tm)); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(st1), schedulerNode1, pcm, tm)); - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(st1), schedulerNode2, pcm, tm)); - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(st1), schedulerNode3, pcm, tm)); /** @@ -318,13 +318,13 @@ public void testMultiTagsPlacementConstraints() tm.addContainer(n2_r2.getNodeID(), cb1, ImmutableSet.of("B")); // Only n2 has both A and B so only it can satisfy the PC - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(st2), schedulerNode0, pcm, tm)); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(st2), schedulerNode1, pcm, tm)); - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(st2), schedulerNode2, pcm, tm)); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(st2), schedulerNode3, pcm, tm)); } @@ -360,15 +360,15 @@ public void testRackAffinityAssignment() n3_r2.getRackName(), n3_r2.getNodeID()); // 'zk' placement on Rack1 should now SUCCEED - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag2), schedulerNode0, pcm, tm)); - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag2), schedulerNode1, pcm, tm)); // FAIL on the rest of the RACKs - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag2), schedulerNode2, pcm, tm)); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag2), schedulerNode3, pcm, tm)); } @@ -405,14 +405,14 @@ public void testNodeAntiAffinityAssignment() tm.addContainer(n0_r1.getNodeID(), hbase_m, ImmutableSet.of("hbase-m")); // 'spark' placement on Node0 should now FAIL - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag1), schedulerNode0, pcm, tm)); // SUCCEED on the rest of the nodes - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag1), schedulerNode1, pcm, tm)); - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag1), schedulerNode2, pcm, tm)); - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag1), schedulerNode3, pcm, tm)); } @@ -448,15 +448,15 @@ public void testRackAntiAffinityAssignment() n3_r2.getRackName(), n3_r2.getNodeID()); // 'zk' placement on Rack1 should FAIL - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag2), schedulerNode0, pcm, tm)); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag2), schedulerNode1, pcm, tm)); // SUCCEED on the rest of the RACKs - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag2), schedulerNode2, pcm, tm)); - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag2), schedulerNode3, pcm, tm)); } @@ -484,9 +484,9 @@ public void testORConstraintAssignment() newContainerId(appId1, 1), ImmutableSet.of("hbase-m")); tm.addContainer(n2r2.getNodeID(), newContainerId(appId1, 2), ImmutableSet.of("hbase-rs")); - Assert.assertEquals(1L, tm.getAllocationTagsWithCount(n0r1.getNodeID()) + Assertions.assertEquals(1L, tm.getAllocationTagsWithCount(n0r1.getNodeID()) .get("hbase-m").longValue()); - Assert.assertEquals(1L, tm.getAllocationTagsWithCount(n2r2.getNodeID()) + Assertions.assertEquals(1L, tm.getAllocationTagsWithCount(n2r2.getNodeID()) .get("hbase-rs").longValue()); SchedulerNode schedulerNode0 =newSchedulerNode(n0r1.getHostName(), @@ -500,13 +500,13 @@ public void testORConstraintAssignment() // n0 and n2 should be qualified for allocation as // they either have hbase-m or hbase-rs tag - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag1), schedulerNode0, pcm, tm)); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag1), schedulerNode1, pcm, tm)); - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag1), schedulerNode2, pcm, tm)); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag1), schedulerNode3, pcm, tm)); /** @@ -519,7 +519,7 @@ public void testORConstraintAssignment() tm.addContainer(n3r2.getNodeID(), newContainerId(appId1, 2), ImmutableSet.of("hbase-rs")); // n3 is qualified now because it is allocated with hbase-rs tag - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag1), schedulerNode3, pcm, tm)); /** @@ -534,13 +534,13 @@ public void testORConstraintAssignment() newContainerId(appId1, 3), ImmutableSet.of("spark")); // According to constraint, "zk" is allowed to be placed on a node // has "hbase-m" tag OR a node has both "hbase-rs" and "spark" tags. - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag2), schedulerNode0, pcm, tm)); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag2), schedulerNode1, pcm, tm)); - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag2), schedulerNode2, pcm, tm)); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag2), schedulerNode3, pcm, tm)); } @@ -568,9 +568,9 @@ public void testANDConstraintAssignment() newContainerId(appId1, 0), ImmutableSet.of("hbase-m")); tm.addContainer(n2r2.getNodeID(), newContainerId(appId1, 1), ImmutableSet.of("hbase-m")); - Assert.assertEquals(1L, tm.getAllocationTagsWithCount(n0r1.getNodeID()) + Assertions.assertEquals(1L, tm.getAllocationTagsWithCount(n0r1.getNodeID()) .get("hbase-m").longValue()); - Assert.assertEquals(1L, tm.getAllocationTagsWithCount(n2r2.getNodeID()) + Assertions.assertEquals(1L, tm.getAllocationTagsWithCount(n2r2.getNodeID()) .get("hbase-m").longValue()); SchedulerNode schedulerNode0 =newSchedulerNode(n0r1.getHostName(), @@ -584,13 +584,13 @@ public void testANDConstraintAssignment() // Anti-affinity with hbase-m so it should not be able to be placed // onto n0 and n2 as they already have hbase-m allocated. - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag1), schedulerNode0, pcm, tm)); - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag1), schedulerNode1, pcm, tm)); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag1), schedulerNode2, pcm, tm)); - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag1), schedulerNode3, pcm, tm)); /** @@ -604,17 +604,17 @@ public void testANDConstraintAssignment() tm.addContainer(n1r1.getNodeID(), newContainerId(appId1, i+2), ImmutableSet.of("spark")); } - Assert.assertEquals(4L, tm.getAllocationTagsWithCount(n1r1.getNodeID()) + Assertions.assertEquals(4L, tm.getAllocationTagsWithCount(n1r1.getNodeID()) .get("spark").longValue()); // Violate cardinality constraint - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag1), schedulerNode0, pcm, tm)); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag1), schedulerNode1, pcm, tm)); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag1), schedulerNode2, pcm, tm)); - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1, createSchedulingRequest(sourceTag1), schedulerNode3, pcm, tm)); } @@ -684,16 +684,16 @@ NODE, allocationTagWithNamespace(namespaceAll.toString(), "A")) constraintMap.put(srcTags1, constraint1); pcm.registerApplication(application1, constraintMap); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( application1, createSchedulingRequest(srcTags1), schedulerNode0, pcm, tm)); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( application1, createSchedulingRequest(srcTags1), schedulerNode1, pcm, tm)); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( application1, createSchedulingRequest(srcTags1), schedulerNode2, pcm, tm)); - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints( application1, createSchedulingRequest(srcTags1), schedulerNode3, pcm, tm)); @@ -710,16 +710,16 @@ application1, createSchedulingRequest(srcTags1), constraintMap.put(srcTags2, constraint2); pcm.registerApplication(application2, constraintMap); - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints( application2, createSchedulingRequest(srcTags2), schedulerNode0, pcm, tm)); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( application2, createSchedulingRequest(srcTags2), schedulerNode1, pcm, tm)); - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints( application2, createSchedulingRequest(srcTags2), schedulerNode2, pcm, tm)); - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints( application2, createSchedulingRequest(srcTags2), schedulerNode3, pcm, tm)); @@ -736,16 +736,16 @@ application2, createSchedulingRequest(srcTags2), constraintMap.put(srcTags3, constraint3); pcm.registerApplication(application3, constraintMap); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( application3, createSchedulingRequest(srcTags3), schedulerNode0, pcm, tm)); - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints( application3, createSchedulingRequest(srcTags3), schedulerNode1, pcm, tm)); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( application3, createSchedulingRequest(srcTags3), schedulerNode2, pcm, tm)); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( application3, createSchedulingRequest(srcTags3), schedulerNode3, pcm, tm)); @@ -831,16 +831,16 @@ NODE, allocationTagWithNamespace(notSelf.toString(), "A")) constraintMap.put(srcTags1, constraint1); pcm.registerApplication(application1, constraintMap); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( application1, createSchedulingRequest(srcTags1), schedulerNode0, pcm, tm)); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( application1, createSchedulingRequest(srcTags1), schedulerNode1, pcm, tm)); - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints( application1, createSchedulingRequest(srcTags1), schedulerNode2, pcm, tm)); - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints( application1, createSchedulingRequest(srcTags1), schedulerNode3, pcm, tm)); @@ -859,16 +859,16 @@ NODE, allocationTagWithNamespace(notSelf.toString(), "A")) cm2.put(srcTags2, constraint2); pcm.registerApplication(application1, cm2); - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints( application1, createSchedulingRequest(srcTags2), schedulerNode0, pcm, tm)); - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints( application1, createSchedulingRequest(srcTags2), schedulerNode1, pcm, tm)); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( application1, createSchedulingRequest(srcTags2), schedulerNode2, pcm, tm)); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( application1, createSchedulingRequest(srcTags2), schedulerNode3, pcm, tm)); @@ -906,9 +906,9 @@ public void testInterAppConstraintsByAppID() newContainerId(application1, 0), ImmutableSet.of("hbase-m")); tm.addContainer(n2r2.getNodeID(), newContainerId(application1, 1), ImmutableSet.of("hbase-m")); - Assert.assertEquals(1L, tm.getAllocationTagsWithCount(n0r1.getNodeID()) + Assertions.assertEquals(1L, tm.getAllocationTagsWithCount(n0r1.getNodeID()) .get("hbase-m").longValue()); - Assert.assertEquals(1L, tm.getAllocationTagsWithCount(n2r2.getNodeID()) + Assertions.assertEquals(1L, tm.getAllocationTagsWithCount(n2r2.getNodeID()) .get("hbase-m").longValue()); SchedulerNode schedulerNode0 =newSchedulerNode(n0r1.getHostName(), @@ -935,16 +935,16 @@ public void testInterAppConstraintsByAppID() // Anti-affinity with app1/hbase-m so it should not be able to be placed // onto n0 and n2 as they already have hbase-m allocated. - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( application2, createSchedulingRequest(srcTags2), schedulerNode0, pcm, tm)); - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints( application2, createSchedulingRequest(srcTags2), schedulerNode1, pcm, tm)); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( application2, createSchedulingRequest(srcTags2), schedulerNode2, pcm, tm)); - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints( application2, createSchedulingRequest(srcTags2), schedulerNode3, pcm, tm)); @@ -971,16 +971,16 @@ application2, createSchedulingRequest(srcTags2), newContainerId(application3, 0), ImmutableSet.of("hbase-m")); // Anti-affinity to self/hbase-m - Assert.assertFalse(PlacementConstraintsUtil + Assertions.assertFalse(PlacementConstraintsUtil .canSatisfyConstraints(application3, createSchedulingRequest(srcTags3), schedulerNode0, pcm, tm)); - Assert.assertTrue(PlacementConstraintsUtil + Assertions.assertTrue(PlacementConstraintsUtil .canSatisfyConstraints(application3, createSchedulingRequest(srcTags3), schedulerNode1, pcm, tm)); - Assert.assertTrue(PlacementConstraintsUtil + Assertions.assertTrue(PlacementConstraintsUtil .canSatisfyConstraints(application3, createSchedulingRequest(srcTags3), schedulerNode2, pcm, tm)); - Assert.assertTrue(PlacementConstraintsUtil + Assertions.assertTrue(PlacementConstraintsUtil .canSatisfyConstraints(application3, createSchedulingRequest(srcTags3), schedulerNode3, pcm, tm)); @@ -1052,16 +1052,16 @@ public void testInterAppConstriantsByAppTag() // Anti-affinity with app-tag/test-tag/hbase-m, // app1 has tag "test-tag" so the constraint is equally to work on app1 // onto n1 and n3 as they don't have "hbase-m" from app1. - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( application2, createSchedulingRequest(srcTags2), schedulerNode0, pcm, tm)); - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints( application2, createSchedulingRequest(srcTags2), schedulerNode1, pcm, tm)); - Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints( application2, createSchedulingRequest(srcTags2), schedulerNode2, pcm, tm)); - Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints( + Assertions.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints( application2, createSchedulingRequest(srcTags2), schedulerNode3, pcm, tm)); @@ -1094,10 +1094,10 @@ public void testInvalidAllocationTagNamespace() { PlacementConstraintsUtil.canSatisfyConstraints(application1, createSchedulingRequest(srcTags1, constraint1), schedulerNode0, pcm, tm); - Assert.fail("This should fail because we gave an invalid namespace"); + Assertions.fail("This should fail because we gave an invalid namespace"); } catch (Exception e) { - Assert.assertTrue(e instanceof InvalidAllocationTagsQueryException); - Assert.assertTrue(e.getMessage() + Assertions.assertTrue(e instanceof InvalidAllocationTagsQueryException); + Assertions.assertTrue(e.getMessage() .contains("Invalid namespace prefix: unknown_namespace")); } } 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/constraint/TestPlacementProcessor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestPlacementProcessor.java index e7c1d0ca6d7e9..9effebdcc3c16 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestPlacementProcessor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TestPlacementProcessor.java @@ -46,10 +46,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -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.util.ArrayList; import java.util.Arrays; @@ -82,7 +83,7 @@ public class TestPlacementProcessor { private MockRM rm; private DrainDispatcher dispatcher; - @Before + @BeforeEach public void createAndStartRM() { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); @@ -107,14 +108,15 @@ protected Dispatcher createDispatcher() { rm.start(); } - @After + @AfterEach public void stopRM() { if (rm != null) { rm.stop(); } } - @Test(timeout = 300000) + @Test + @Timeout(value = 300) public void testAntiAffinityPlacement() throws Exception { HashMap nodes = new HashMap<>(); MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService()); @@ -157,18 +159,19 @@ public void testAntiAffinityPlacement() throws Exception { waitForContainerAllocation(nodes.values(), am1, allocatedContainers, new ArrayList<>(), 4); - Assert.assertEquals(4, allocatedContainers.size()); + Assertions.assertEquals(4, allocatedContainers.size()); Set nodeIds = allocatedContainers.stream().map(x -> x.getNodeId()) .collect(Collectors.toSet()); // Ensure unique nodes (antiaffinity) - Assert.assertEquals(4, nodeIds.size()); + Assertions.assertEquals(4, nodeIds.size()); QueueMetrics metrics = rm.getResourceScheduler().getRootQueueMetrics(); // Verify Metrics verifyMetrics(metrics, 11264, 11, 5120, 5, 5); } - @Test(timeout = 300000) + @Test + @Timeout(value = 300) public void testMutualAntiAffinityPlacement() throws Exception { HashMap nodes = new HashMap<>(); MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService()); @@ -219,18 +222,19 @@ public void testMutualAntiAffinityPlacement() throws Exception { waitForContainerAllocation(nodes.values(), am1, allocatedContainers, new ArrayList<>(), 5); - Assert.assertEquals(5, allocatedContainers.size()); + Assertions.assertEquals(5, allocatedContainers.size()); Set nodeIds = allocatedContainers.stream().map(x -> x.getNodeId()) .collect(Collectors.toSet()); // Ensure unique nodes (antiaffinity) - Assert.assertEquals(5, nodeIds.size()); + Assertions.assertEquals(5, nodeIds.size()); QueueMetrics metrics = rm.getResourceScheduler().getRootQueueMetrics(); // Verify Metrics verifyMetrics(metrics, 14336, 14, 6144, 6, 6); } - @Test(timeout = 300000) + @Test + @Timeout(value = 300) public void testCardinalityPlacement() throws Exception { HashMap nodes = new HashMap<>(); MockNM nm1 = new MockNM("h1:1234", 8192, rm.getResourceTrackerService()); @@ -277,13 +281,13 @@ public void testCardinalityPlacement() throws Exception { waitForContainerAllocation(nodes.values(), am1, allocatedContainers, new ArrayList<>(), 8); - Assert.assertEquals(8, allocatedContainers.size()); + Assertions.assertEquals(8, allocatedContainers.size()); Map nodeIdContainerIdMap = allocatedContainers.stream().collect( Collectors.groupingBy(c -> c.getNodeId(), Collectors.counting())); // Ensure no more than 4 containers per node for (NodeId n : nodeIdContainerIdMap.keySet()) { - Assert.assertTrue(nodeIdContainerIdMap.get(n) < 5); + Assertions.assertTrue(nodeIdContainerIdMap.get(n) < 5); } QueueMetrics metrics = rm.getResourceScheduler().getRootQueueMetrics(); @@ -291,7 +295,8 @@ public void testCardinalityPlacement() throws Exception { verifyMetrics(metrics, 23552, 23, 9216, 9, 9); } - @Test(timeout = 300000) + @Test + @Timeout(value = 300) public void testAffinityPlacement() throws Exception { HashMap nodes = new HashMap<>(); MockNM nm1 = new MockNM("h1:1234", 8192, rm.getResourceTrackerService()); @@ -336,18 +341,19 @@ public void testAffinityPlacement() throws Exception { waitForContainerAllocation(nodes.values(), am1, allocatedContainers, new ArrayList<>(), 5); - Assert.assertEquals(5, allocatedContainers.size()); + Assertions.assertEquals(5, allocatedContainers.size()); Set nodeIds = allocatedContainers.stream().map(x -> x.getNodeId()) .collect(Collectors.toSet()); // Ensure all containers end up on the same node (affinity) - Assert.assertEquals(1, nodeIds.size()); + Assertions.assertEquals(1, nodeIds.size()); QueueMetrics metrics = rm.getResourceScheduler().getRootQueueMetrics(); // Verify Metrics verifyMetrics(metrics, 26624, 26, 6144, 6, 6); } - @Test(timeout = 300000) + @Test + @Timeout(value = 300) public void testComplexPlacement() throws Exception { HashMap nodes = new HashMap<>(); MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService()); @@ -398,13 +404,13 @@ public void testComplexPlacement() throws Exception { waitForContainerAllocation(nodes.values(), am1, allocatedContainers, new ArrayList<>(), 6); - Assert.assertEquals(6, allocatedContainers.size()); + Assertions.assertEquals(6, allocatedContainers.size()); Map nodeIdContainerIdMap = allocatedContainers.stream().collect( Collectors.groupingBy(c -> c.getNodeId(), Collectors.counting())); // Ensure no more than 3 containers per node (1 'bar', 2 'foo') for (NodeId n : nodeIdContainerIdMap.keySet()) { - Assert.assertTrue(nodeIdContainerIdMap.get(n) < 4); + Assertions.assertTrue(nodeIdContainerIdMap.get(n) < 4); } QueueMetrics metrics = rm.getResourceScheduler().getRootQueueMetrics(); @@ -412,7 +418,8 @@ public void testComplexPlacement() throws Exception { verifyMetrics(metrics, 9216, 9, 7168, 7, 7); } - @Test(timeout = 300000) + @Test + @Timeout(value = 300) public void testSchedulerRejection() throws Exception { stopRM(); CapacitySchedulerConfiguration csConf = @@ -494,14 +501,14 @@ public void testSchedulerRejection() throws Exception { } } - Assert.assertEquals(3, allocatedContainers.size()); + Assertions.assertEquals(3, allocatedContainers.size()); Set nodeIds = allocatedContainers.stream() .map(x -> x.getNodeId()).collect(Collectors.toSet()); // Ensure unique nodes - Assert.assertEquals(3, nodeIds.size()); + Assertions.assertEquals(3, nodeIds.size()); RejectedSchedulingRequest rej = rejectedReqs.get(0); - Assert.assertEquals(4, rej.getRequest().getAllocationRequestId()); - Assert.assertEquals(RejectionReason.COULD_NOT_SCHEDULE_ON_NODE, + Assertions.assertEquals(4, rej.getRequest().getAllocationRequestId()); + Assertions.assertEquals(RejectionReason.COULD_NOT_SCHEDULE_ON_NODE, rej.getReason()); QueueMetrics metrics = rm.getResourceScheduler().getRootQueueMetrics(); @@ -509,7 +516,8 @@ public void testSchedulerRejection() throws Exception { verifyMetrics(metrics, 12288, 12, 4096, 4, 4); } - @Test(timeout = 300000) + @Test + @Timeout(value = 300) public void testNodeCapacityRejection() throws Exception { HashMap nodes = new HashMap<>(); MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService()); @@ -573,14 +581,14 @@ public void testNodeCapacityRejection() throws Exception { } } - Assert.assertEquals(3, allocatedContainers.size()); + Assertions.assertEquals(3, allocatedContainers.size()); Set nodeIds = allocatedContainers.stream() .map(x -> x.getNodeId()).collect(Collectors.toSet()); // Ensure unique nodes - Assert.assertEquals(3, nodeIds.size()); + Assertions.assertEquals(3, nodeIds.size()); RejectedSchedulingRequest rej = rejectedReqs.get(0); - Assert.assertEquals(4, rej.getRequest().getAllocationRequestId()); - Assert.assertEquals(RejectionReason.COULD_NOT_PLACE_ON_NODE, + Assertions.assertEquals(4, rej.getRequest().getAllocationRequestId()); + Assertions.assertEquals(RejectionReason.COULD_NOT_PLACE_ON_NODE, rej.getReason()); QueueMetrics metrics = rm.getResourceScheduler().getRootQueueMetrics(); @@ -588,7 +596,8 @@ public void testNodeCapacityRejection() throws Exception { verifyMetrics(metrics, 12288, 12, 4096, 4, 4); } - @Test(timeout = 300000) + @Test + @Timeout(value = 300) public void testRePlacementAfterSchedulerRejection() throws Exception { stopRM(); CapacitySchedulerConfiguration csConf = @@ -672,18 +681,19 @@ public void testRePlacementAfterSchedulerRejection() throws Exception { } } - Assert.assertEquals(4, allocatedContainers.size()); + Assertions.assertEquals(4, allocatedContainers.size()); Set nodeIds = allocatedContainers.stream() .map(x -> x.getNodeId()).collect(Collectors.toSet()); // Ensure unique nodes - Assert.assertEquals(4, nodeIds.size()); + Assertions.assertEquals(4, nodeIds.size()); QueueMetrics metrics = rm.getResourceScheduler().getRootQueueMetrics(); // Verify Metrics verifyMetrics(metrics, 15360, 19, 9216, 5, 5); } - @Test(timeout = 300000) + @Test + @Timeout(value = 300) public void testPlacementRejection() throws Exception { HashMap nodes = new HashMap<>(); MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService()); @@ -748,13 +758,13 @@ public void testPlacementRejection() throws Exception { } } - Assert.assertEquals(4, allocatedContainers.size()); + Assertions.assertEquals(4, allocatedContainers.size()); Set nodeIds = allocatedContainers.stream() .map(x -> x.getNodeId()).collect(Collectors.toSet()); // Ensure unique nodes - Assert.assertEquals(4, nodeIds.size()); + Assertions.assertEquals(4, nodeIds.size()); RejectedSchedulingRequest rej = rejectedReqs.get(0); - Assert.assertEquals(COULD_NOT_PLACE_ON_NODE, + Assertions.assertEquals(COULD_NOT_PLACE_ON_NODE, rej.getReason()); QueueMetrics metrics = rm.getResourceScheduler().getRootQueueMetrics(); @@ -762,7 +772,8 @@ public void testPlacementRejection() throws Exception { verifyMetrics(metrics, 11264, 11, 5120, 5, 5); } - @Test(timeout = 300000) + @Test + @Timeout(value = 300) public void testAndOrPlacement() throws Exception { HashMap nodes = new HashMap<>(); MockNM nm1 = new MockNM("h1:1234", 40960, 100, @@ -824,7 +835,7 @@ public void testAndOrPlacement() throws Exception { waitForContainerAllocation(nodes.values(), am1, allocatedContainers, new ArrayList<>(), 3); printTags(nodes.values(), rm.getRMContext().getAllocationTagsManager()); - Assert.assertEquals(3, allocatedContainers.size()); + Assertions.assertEquals(3, allocatedContainers.size()); /** Testing AND placement constraint**/ // Now allocates a bar container, as restricted by the AND constraint, @@ -837,7 +848,7 @@ public void testAndOrPlacement() throws Exception { waitForContainerAllocation(nodes.values(), am1, allocatedContainers, new ArrayList<>(), 1); printTags(nodes.values(), rm.getRMContext().getAllocationTagsManager()); - Assert.assertEquals(1, allocatedContainers.size()); + Assertions.assertEquals(1, allocatedContainers.size()); NodeId barNode = allocatedContainers.get(0).getNodeId(); // Sends another 3 bar request, 2 of them can be allocated @@ -854,15 +865,15 @@ public void testAndOrPlacement() throws Exception { waitForContainerAllocation(nodes.values(), am1, allocatedContainers, rejectedContainers, 2); printTags(nodes.values(), rm.getRMContext().getAllocationTagsManager()); - Assert.assertEquals(2, allocatedContainers.size()); - Assert.assertTrue(allocatedContainers.stream().allMatch( + Assertions.assertEquals(2, allocatedContainers.size()); + Assertions.assertTrue(allocatedContainers.stream().allMatch( container -> container.getNodeId().equals(barNode))); // The third request could not be satisfied because it violates // the cardinality constraint. Validate rejected request correctly // capture this. - Assert.assertEquals(1, rejectedContainers.size()); - Assert.assertEquals(COULD_NOT_PLACE_ON_NODE, + Assertions.assertEquals(1, rejectedContainers.size()); + Assertions.assertEquals(COULD_NOT_PLACE_ON_NODE, rejectedContainers.get(0).getReason()); /** Testing OR placement constraint**/ @@ -884,10 +895,10 @@ public void testAndOrPlacement() throws Exception { // All 20 containers should be allocated onto nodes besides nm5, // because moo affinity to foo or bar which only exists on rest of nodes. - Assert.assertEquals(20, allocatedContainers.size()); + Assertions.assertEquals(20, allocatedContainers.size()); for (Container mooContainer : allocatedContainers) { // nm5 has no moo allocated containers. - Assert.assertFalse(mooContainer.getNodeId().equals(nm5.getNodeId())); + Assertions.assertFalse(mooContainer.getNodeId().equals(nm5.getNodeId())); } } @@ -946,12 +957,12 @@ protected static SchedulingRequest schedulingRequest( private static void verifyMetrics(QueueMetrics metrics, long availableMB, int availableVirtualCores, long allocatedMB, int allocatedVirtualCores, int allocatedContainers) { - Assert.assertEquals(availableMB, metrics.getAvailableMB()); - Assert.assertEquals(availableVirtualCores, + Assertions.assertEquals(availableMB, metrics.getAvailableMB()); + Assertions.assertEquals(availableVirtualCores, metrics.getAvailableVirtualCores()); - Assert.assertEquals(allocatedMB, metrics.getAllocatedMB()); - Assert.assertEquals(allocatedVirtualCores, + Assertions.assertEquals(allocatedMB, metrics.getAllocatedMB()); + Assertions.assertEquals(allocatedVirtualCores, metrics.getAllocatedVirtualCores()); - Assert.assertEquals(allocatedContainers, metrics.getAllocatedContainers()); + Assertions.assertEquals(allocatedContainers, metrics.getAllocatedContainers()); } } 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/constraint/algorithm/TestCircularIterator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/algorithm/TestCircularIterator.java index bb005a04ee826..3cb3773dd9374 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/algorithm/TestCircularIterator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/algorithm/TestCircularIterator.java @@ -17,8 +17,8 @@ */ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.algorithm; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.Arrays; @@ -39,7 +39,7 @@ public void testIteration() throws Exception { while (ci.hasNext()) { sb.append(ci.next()); } - Assert.assertEquals("abcd", sb.toString()); + Assertions.assertEquals("abcd", sb.toString()); Iterator lIter = list.iterator(); lIter.next(); @@ -49,7 +49,7 @@ public void testIteration() throws Exception { while (ci.hasNext()) { sb.append(ci.next()); } - Assert.assertEquals("cdab", sb.toString()); + Assertions.assertEquals("cdab", sb.toString()); lIter = list.iterator(); lIter.next(); @@ -60,7 +60,7 @@ public void testIteration() throws Exception { while (ci.hasNext()) { sb.append(ci.next()); } - Assert.assertEquals("xdabc", sb.toString()); + Assertions.assertEquals("xdabc", sb.toString()); list = Arrays.asList("a"); lIter = list.iterator(); @@ -70,13 +70,13 @@ public void testIteration() throws Exception { while (ci.hasNext()) { sb.append(ci.next()); } - Assert.assertEquals("ya", sb.toString()); + Assertions.assertEquals("ya", sb.toString()); try { list = new ArrayList<>(); lIter = list.iterator(); new CircularIterator<>("y", lIter, list); - Assert.fail("Should fail.."); + Assertions.fail("Should fail.."); } catch (Exception e) { // foo bar } 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/constraint/algorithm/TestLocalAllocationTagsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/algorithm/TestLocalAllocationTagsManager.java index 5ce3130c5e08e..fbf37c2eae366 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/algorithm/TestLocalAllocationTagsManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/algorithm/TestLocalAllocationTagsManager.java @@ -28,9 +28,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.AllocationTags; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.AllocationTagsManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.InvalidAllocationTagsQueryException; -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 java.util.List; @@ -41,7 +41,7 @@ public class TestLocalAllocationTagsManager { private RMContext rmContext; - @Before + @BeforeEach public void setup() { MockRM rm = new MockRM(); rm.start(); @@ -84,21 +84,21 @@ public void testTempContainerAllocations() TestUtils.getMockApplicationId(2), ImmutableSet.of("service")); // Expect tag mappings to be present including temp Tags - Assert.assertEquals(1, + Assertions.assertEquals(1, atm.getNodeCardinalityByOp(NodeId.fromString("host1:123"), AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(1), ImmutableSet.of("mapper")), Long::sum)); - Assert.assertEquals(1, + Assertions.assertEquals(1, atm.getNodeCardinalityByOp(NodeId.fromString("host1:123"), AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(1), ImmutableSet.of("service")), Long::sum)); - Assert.assertEquals(1, + Assertions.assertEquals(1, atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"), AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(2), @@ -107,14 +107,14 @@ public void testTempContainerAllocations() // Do a temp Tag cleanup on app2 ephAtm.cleanTempContainers(TestUtils.getMockApplicationId(2)); - Assert.assertEquals(0, + Assertions.assertEquals(0, atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"), AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(2), ImmutableSet.of("service")), Long::sum)); // Expect app1 to be unaffected - Assert.assertEquals(1, + Assertions.assertEquals(1, atm.getNodeCardinalityByOp(NodeId.fromString("host1:123"), AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(1), @@ -122,7 +122,7 @@ public void testTempContainerAllocations() Long::sum)); // Do a cleanup on app1 as well ephAtm.cleanTempContainers(TestUtils.getMockApplicationId(1)); - Assert.assertEquals(0, + Assertions.assertEquals(0, atm.getNodeCardinalityByOp(NodeId.fromString("host1:123"), AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(1), @@ -130,14 +130,14 @@ public void testTempContainerAllocations() Long::sum)); // Non temp-tags should be unaffected - Assert.assertEquals(1, + Assertions.assertEquals(1, atm.getNodeCardinalityByOp(NodeId.fromString("host1:123"), AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(1), ImmutableSet.of("service")), Long::sum)); - Assert.assertEquals(0, + Assertions.assertEquals(0, atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"), AllocationTags.createSingleAppAllocationTags( TestUtils.getMockApplicationId(2), @@ -145,11 +145,11 @@ public void testTempContainerAllocations() Long::sum)); // Expect app2 with no containers, and app1 with 2 containers across 2 nodes - Assert.assertEquals(2, + Assertions.assertEquals(2, atm.getPerAppNodeMappings().get(TestUtils.getMockApplicationId(1)) .getTypeToTagsWithCount().size()); - Assert.assertNull( + Assertions.assertNull( atm.getPerAppNodeMappings().get(TestUtils.getMockApplicationId(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/scheduler/distributed/TestCentralizedOpportunisticContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/TestCentralizedOpportunisticContainerAllocator.java index 8301c13dfd375..38d47d69e3ccc 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/TestCentralizedOpportunisticContainerAllocator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/TestCentralizedOpportunisticContainerAllocator.java @@ -36,8 +36,8 @@ import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerContext; import org.apache.hadoop.yarn.server.security.BaseContainerTokenSecretManager; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,9 +52,9 @@ import java.util.Map; import java.util.Set; -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.ArgumentMatchers.anyLong; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; @@ -79,7 +79,7 @@ public class TestCentralizedOpportunisticContainerAllocator { ResourceBlacklistRequest.newInstance( new ArrayList<>(), new ArrayList<>()); - @Before + @BeforeEach public void setup() { // creating a dummy master key to be used for creation of container. final MasterKey mKey = new MasterKey() { 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/distributed/TestNodeQueueLoadMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/TestNodeQueueLoadMonitor.java index c8ebe6f8f2067..8757ee82b2502 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/TestNodeQueueLoadMonitor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/TestNodeQueueLoadMonitor.java @@ -31,9 +31,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.util.resource.ResourceUtils; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import java.util.Collections; @@ -123,7 +123,7 @@ public static void addNewTypesToResources(String... resourceTypes) { ResourceUtils.initializeResourcesFromResourceInformationMap(riMap); } - @BeforeClass + @BeforeAll public static void classSetUp() { addNewTypesToResources(NETWORK_RESOURCE); defaultResourceRequested = newResourceInstance(128, 1, 1); @@ -139,43 +139,43 @@ public void testWaitTimeSort() { selector.updateNode(createRMNode("h3", 3, 10, 10)); selector.computeTask.run(); List nodeIds = selector.selectNodes(); - Assert.assertEquals("h2:2", nodeIds.get(0).toString()); - Assert.assertEquals("h3:3", nodeIds.get(1).toString()); - Assert.assertEquals("h1:1", nodeIds.get(2).toString()); + Assertions.assertEquals("h2:2", nodeIds.get(0).toString()); + Assertions.assertEquals("h3:3", nodeIds.get(1).toString()); + Assertions.assertEquals("h1:1", nodeIds.get(2).toString()); // Now update node3 selector.updateNode(createRMNode("h3", 3, 2, 10)); selector.computeTask.run(); nodeIds = selector.selectNodes(); - Assert.assertEquals("h3:3", nodeIds.get(0).toString()); - Assert.assertEquals("h2:2", nodeIds.get(1).toString()); - Assert.assertEquals("h1:1", nodeIds.get(2).toString()); + Assertions.assertEquals("h3:3", nodeIds.get(0).toString()); + Assertions.assertEquals("h2:2", nodeIds.get(1).toString()); + Assertions.assertEquals("h1:1", nodeIds.get(2).toString()); // Now send update with -1 wait time selector.updateNode(createRMNode("h4", 4, -1, 10)); selector.computeTask.run(); nodeIds = selector.selectNodes(); // No change - Assert.assertEquals("h3:3", nodeIds.get(0).toString()); - Assert.assertEquals("h2:2", nodeIds.get(1).toString()); - Assert.assertEquals("h1:1", nodeIds.get(2).toString()); + Assertions.assertEquals("h3:3", nodeIds.get(0).toString()); + Assertions.assertEquals("h2:2", nodeIds.get(1).toString()); + Assertions.assertEquals("h1:1", nodeIds.get(2).toString()); // Now update node 2 to DECOMMISSIONING state selector .updateNode(createRMNode("h2", 2, 1, 10, NodeState.DECOMMISSIONING)); selector.computeTask.run(); nodeIds = selector.selectNodes(); - Assert.assertEquals(2, nodeIds.size()); - Assert.assertEquals("h3:3", nodeIds.get(0).toString()); - Assert.assertEquals("h1:1", nodeIds.get(1).toString()); + Assertions.assertEquals(2, nodeIds.size()); + Assertions.assertEquals("h3:3", nodeIds.get(0).toString()); + Assertions.assertEquals("h1:1", nodeIds.get(1).toString()); // Now update node 2 back to RUNNING state selector.updateNode(createRMNode("h2", 2, 1, 10, NodeState.RUNNING)); selector.computeTask.run(); nodeIds = selector.selectNodes(); - Assert.assertEquals("h2:2", nodeIds.get(0).toString()); - Assert.assertEquals("h3:3", nodeIds.get(1).toString()); - Assert.assertEquals("h1:1", nodeIds.get(2).toString()); + Assertions.assertEquals("h2:2", nodeIds.get(0).toString()); + Assertions.assertEquals("h3:3", nodeIds.get(1).toString()); + Assertions.assertEquals("h1:1", nodeIds.get(2).toString()); } @Test @@ -188,18 +188,18 @@ public void testQueueLengthSort() { selector.computeTask.run(); List nodeIds = selector.selectNodes(); System.out.println("1-> " + nodeIds); - Assert.assertEquals("h2:2", nodeIds.get(0).toString()); - Assert.assertEquals("h3:3", nodeIds.get(1).toString()); - Assert.assertEquals("h1:1", nodeIds.get(2).toString()); + Assertions.assertEquals("h2:2", nodeIds.get(0).toString()); + Assertions.assertEquals("h3:3", nodeIds.get(1).toString()); + Assertions.assertEquals("h1:1", nodeIds.get(2).toString()); // Now update node3 selector.updateNode(createRMNode("h3", 3, -1, 2)); selector.computeTask.run(); nodeIds = selector.selectNodes(); System.out.println("2-> "+ nodeIds); - Assert.assertEquals("h3:3", nodeIds.get(0).toString()); - Assert.assertEquals("h2:2", nodeIds.get(1).toString()); - Assert.assertEquals("h1:1", nodeIds.get(2).toString()); + Assertions.assertEquals("h3:3", nodeIds.get(0).toString()); + Assertions.assertEquals("h2:2", nodeIds.get(1).toString()); + Assertions.assertEquals("h1:1", nodeIds.get(2).toString()); // Now send update with -1 wait time but valid length selector.updateNode(createRMNode("h4", 4, -1, 20)); @@ -207,10 +207,10 @@ public void testQueueLengthSort() { nodeIds = selector.selectNodes(); System.out.println("3-> "+ nodeIds); // No change - Assert.assertEquals("h3:3", nodeIds.get(0).toString()); - Assert.assertEquals("h2:2", nodeIds.get(1).toString()); - Assert.assertEquals("h1:1", nodeIds.get(2).toString()); - Assert.assertEquals("h4:4", nodeIds.get(3).toString()); + Assertions.assertEquals("h3:3", nodeIds.get(0).toString()); + Assertions.assertEquals("h2:2", nodeIds.get(1).toString()); + Assertions.assertEquals("h1:1", nodeIds.get(2).toString()); + Assertions.assertEquals("h4:4", nodeIds.get(3).toString()); // Now update h3 and fill its queue. selector.updateNode(createRMNode("h3", 3, -1, @@ -218,29 +218,29 @@ public void testQueueLengthSort() { selector.computeTask.run(); nodeIds = selector.selectNodes(); System.out.println("4-> "+ nodeIds); - Assert.assertEquals(3, nodeIds.size()); - Assert.assertEquals("h2:2", nodeIds.get(0).toString()); - Assert.assertEquals("h1:1", nodeIds.get(1).toString()); - Assert.assertEquals("h4:4", nodeIds.get(2).toString()); + Assertions.assertEquals(3, nodeIds.size()); + Assertions.assertEquals("h2:2", nodeIds.get(0).toString()); + Assertions.assertEquals("h1:1", nodeIds.get(1).toString()); + Assertions.assertEquals("h4:4", nodeIds.get(2).toString()); // Now update h2 to Decommissioning state selector.updateNode(createRMNode("h2", 2, -1, 5, NodeState.DECOMMISSIONING)); selector.computeTask.run(); nodeIds = selector.selectNodes(); - Assert.assertEquals(2, nodeIds.size()); - Assert.assertEquals("h1:1", nodeIds.get(0).toString()); - Assert.assertEquals("h4:4", nodeIds.get(1).toString()); + Assertions.assertEquals(2, nodeIds.size()); + Assertions.assertEquals("h1:1", nodeIds.get(0).toString()); + Assertions.assertEquals("h4:4", nodeIds.get(1).toString()); // Now update h2 back to Running state selector.updateNode(createRMNode("h2", 2, -1, 5, NodeState.RUNNING)); selector.computeTask.run(); nodeIds = selector.selectNodes(); - Assert.assertEquals(3, nodeIds.size()); - Assert.assertEquals("h2:2", nodeIds.get(0).toString()); - Assert.assertEquals("h1:1", nodeIds.get(1).toString()); - Assert.assertEquals("h4:4", nodeIds.get(2).toString()); + Assertions.assertEquals(3, nodeIds.size()); + Assertions.assertEquals("h2:2", nodeIds.get(0).toString()); + Assertions.assertEquals("h1:1", nodeIds.get(1).toString()); + Assertions.assertEquals("h4:4", nodeIds.get(2).toString()); } @Test @@ -265,10 +265,10 @@ public void testQueueLengthThenResourcesSort() { Resources.multiply(defaultResourceRequested, 2), defaultCapacity)); selector.computeTask.run(); List nodeIds = selector.selectNodes(); - Assert.assertEquals("h2:2", nodeIds.get(0).toString()); - Assert.assertEquals("h1:1", nodeIds.get(1).toString()); - Assert.assertEquals("h3:3", nodeIds.get(2).toString()); - Assert.assertEquals("h4:4", nodeIds.get(3).toString()); + Assertions.assertEquals("h2:2", nodeIds.get(0).toString()); + Assertions.assertEquals("h1:1", nodeIds.get(1).toString()); + Assertions.assertEquals("h3:3", nodeIds.get(2).toString()); + Assertions.assertEquals("h4:4", nodeIds.get(3).toString()); // Now update node3 // node3 should now rank after node4 since it has the same queue length @@ -278,10 +278,10 @@ public void testQueueLengthThenResourcesSort() { Resources.multiply(defaultResourceRequested, 3), defaultCapacity)); selector.computeTask.run(); nodeIds = selector.selectNodes(); - Assert.assertEquals("h2:2", nodeIds.get(0).toString()); - Assert.assertEquals("h1:1", nodeIds.get(1).toString()); - Assert.assertEquals("h4:4", nodeIds.get(2).toString()); - Assert.assertEquals("h3:3", nodeIds.get(3).toString()); + Assertions.assertEquals("h2:2", nodeIds.get(0).toString()); + Assertions.assertEquals("h1:1", nodeIds.get(1).toString()); + Assertions.assertEquals("h4:4", nodeIds.get(2).toString()); + Assertions.assertEquals("h3:3", nodeIds.get(3).toString()); // Now update h3 and fill its queue -- it should no longer be available selector.updateNode(createRMNode("h3", 3, -1, @@ -289,10 +289,10 @@ public void testQueueLengthThenResourcesSort() { selector.computeTask.run(); nodeIds = selector.selectNodes(); // h3 is queued up, so we should only have 3 nodes left - Assert.assertEquals(3, nodeIds.size()); - Assert.assertEquals("h2:2", nodeIds.get(0).toString()); - Assert.assertEquals("h1:1", nodeIds.get(1).toString()); - Assert.assertEquals("h4:4", nodeIds.get(2).toString()); + Assertions.assertEquals(3, nodeIds.size()); + Assertions.assertEquals("h2:2", nodeIds.get(0).toString()); + Assertions.assertEquals("h1:1", nodeIds.get(1).toString()); + Assertions.assertEquals("h4:4", nodeIds.get(2).toString()); // Now update h2 to Decommissioning state selector.updateNode(createRMNode("h2", 2, -1, @@ -300,9 +300,9 @@ public void testQueueLengthThenResourcesSort() { selector.computeTask.run(); nodeIds = selector.selectNodes(); // h2 is decommissioned, and h3 is full, so we should only have 2 nodes - Assert.assertEquals(2, nodeIds.size()); - Assert.assertEquals("h1:1", nodeIds.get(0).toString()); - Assert.assertEquals("h4:4", nodeIds.get(1).toString()); + Assertions.assertEquals(2, nodeIds.size()); + Assertions.assertEquals("h1:1", nodeIds.get(0).toString()); + Assertions.assertEquals("h4:4", nodeIds.get(1).toString()); // Now update h2 back to Running state selector.updateNode(createRMNode( @@ -310,10 +310,10 @@ public void testQueueLengthThenResourcesSort() { Resources.multiply(defaultResourceRequested, 2), defaultCapacity)); selector.computeTask.run(); nodeIds = selector.selectNodes(); - Assert.assertEquals(3, nodeIds.size()); - Assert.assertEquals("h2:2", nodeIds.get(0).toString()); - Assert.assertEquals("h1:1", nodeIds.get(1).toString()); - Assert.assertEquals("h4:4", nodeIds.get(2).toString()); + Assertions.assertEquals(3, nodeIds.size()); + Assertions.assertEquals("h2:2", nodeIds.get(0).toString()); + Assertions.assertEquals("h1:1", nodeIds.get(1).toString()); + Assertions.assertEquals("h4:4", nodeIds.get(2).toString()); // Now update h2 to have a zero queue capacity. // Make sure that here it is still in the pool. @@ -323,10 +323,10 @@ public void testQueueLengthThenResourcesSort() { defaultCapacity)); selector.computeTask.run(); nodeIds = selector.selectNodes(); - Assert.assertEquals(3, nodeIds.size()); - Assert.assertEquals("h2:2", nodeIds.get(0).toString()); - Assert.assertEquals("h1:1", nodeIds.get(1).toString()); - Assert.assertEquals("h4:4", nodeIds.get(2).toString()); + Assertions.assertEquals(3, nodeIds.size()); + Assertions.assertEquals("h2:2", nodeIds.get(0).toString()); + Assertions.assertEquals("h1:1", nodeIds.get(1).toString()); + Assertions.assertEquals("h4:4", nodeIds.get(2).toString()); // Now update h2 to have a positive queue length but a zero queue capacity. // Make sure that here it is no longer in the pool. @@ -341,9 +341,9 @@ public void testQueueLengthThenResourcesSort() { defaultCapacity)); selector.computeTask.run(); nodeIds = selector.selectNodes(); - Assert.assertEquals(2, nodeIds.size()); - Assert.assertEquals("h1:1", nodeIds.get(0).toString()); - Assert.assertEquals("h4:4", nodeIds.get(1).toString()); + Assertions.assertEquals(2, nodeIds.size()); + Assertions.assertEquals("h1:1", nodeIds.get(0).toString()); + Assertions.assertEquals("h4:4", nodeIds.get(1).toString()); } /** @@ -360,33 +360,33 @@ public void testQueueLengthThenResourcesDecrementsAvailable() { selector.updateSortedNodes(); ClusterNode clusterNode = selector.getClusterNodes().get(node.getNodeID()); - Assert.assertEquals(Resources.none(), + Assertions.assertEquals(Resources.none(), clusterNode.getAllocatedResource()); // Has enough resources RMNode selectedNode = selector.selectAnyNode( Collections.emptySet(), defaultResourceRequested); - Assert.assertNotNull(selectedNode); - Assert.assertEquals(node.getNodeID(), selectedNode.getNodeID()); + Assertions.assertNotNull(selectedNode); + Assertions.assertEquals(node.getNodeID(), selectedNode.getNodeID()); clusterNode = selector.getClusterNodes().get(node.getNodeID()); - Assert.assertEquals(defaultResourceRequested, + Assertions.assertEquals(defaultResourceRequested, clusterNode.getAllocatedResource()); // Does not have enough resources, but can queue selectedNode = selector.selectAnyNode( Collections.emptySet(), defaultCapacity); - Assert.assertNotNull(selectedNode); - Assert.assertEquals(node.getNodeID(), selectedNode.getNodeID()); + Assertions.assertNotNull(selectedNode); + Assertions.assertEquals(node.getNodeID(), selectedNode.getNodeID()); clusterNode = selector.getClusterNodes().get(node.getNodeID()); - Assert.assertEquals(1, clusterNode.getQueueLength()); + Assertions.assertEquals(1, clusterNode.getQueueLength()); // Does not have enough resources and cannot queue selectedNode = selector.selectAnyNode( Collections.emptySet(), Resources.add(defaultResourceRequested, defaultCapacity)); - Assert.assertNull(selectedNode); + Assertions.assertNull(selectedNode); } @Test @@ -411,10 +411,10 @@ public void testQueueLengthThenResourcesCapabilityChange() { Resources.multiply(defaultResourceRequested, 4), defaultCapacity)); selector.computeTask.run(); List nodeIds = selector.selectNodes(); - Assert.assertEquals("h1:1", nodeIds.get(0).toString()); - Assert.assertEquals("h2:2", nodeIds.get(1).toString()); - Assert.assertEquals("h3:3", nodeIds.get(2).toString()); - Assert.assertEquals("h4:4", nodeIds.get(3).toString()); + Assertions.assertEquals("h1:1", nodeIds.get(0).toString()); + Assertions.assertEquals("h2:2", nodeIds.get(1).toString()); + Assertions.assertEquals("h3:3", nodeIds.get(2).toString()); + Assertions.assertEquals("h4:4", nodeIds.get(3).toString()); // Now update node1 to have only defaultResourceRequested available // by changing its capability to 2x defaultResourceReqeusted @@ -425,10 +425,10 @@ public void testQueueLengthThenResourcesCapabilityChange() { Resources.multiply(defaultResourceRequested, 2))); selector.computeTask.run(); nodeIds = selector.selectNodes(); - Assert.assertEquals("h2:2", nodeIds.get(0).toString()); - Assert.assertEquals("h3:3", nodeIds.get(1).toString()); - Assert.assertEquals("h4:4", nodeIds.get(2).toString()); - Assert.assertEquals("h1:1", nodeIds.get(3).toString()); + Assertions.assertEquals("h2:2", nodeIds.get(0).toString()); + Assertions.assertEquals("h3:3", nodeIds.get(1).toString()); + Assertions.assertEquals("h4:4", nodeIds.get(2).toString()); + Assertions.assertEquals("h1:1", nodeIds.get(3).toString()); // Now update node2 to have no resources available // by changing its capability to 1x defaultResourceReqeusted @@ -439,10 +439,10 @@ public void testQueueLengthThenResourcesCapabilityChange() { Resources.multiply(defaultResourceRequested, 1))); selector.computeTask.run(); nodeIds = selector.selectNodes(); - Assert.assertEquals("h3:3", nodeIds.get(0).toString()); - Assert.assertEquals("h4:4", nodeIds.get(1).toString()); - Assert.assertEquals("h1:1", nodeIds.get(2).toString()); - Assert.assertEquals("h2:2", nodeIds.get(3).toString()); + Assertions.assertEquals("h3:3", nodeIds.get(0).toString()); + Assertions.assertEquals("h4:4", nodeIds.get(1).toString()); + Assertions.assertEquals("h1:1", nodeIds.get(2).toString()); + Assertions.assertEquals("h2:2", nodeIds.get(3).toString()); } @Test @@ -458,12 +458,12 @@ public void testContainerQueuingLimit() { QueueLimitCalculator calculator = selector.getThresholdCalculator(); ContainerQueuingLimit containerQueuingLimit = calculator .createContainerQueuingLimit(); - Assert.assertEquals(6, containerQueuingLimit.getMaxQueueLength()); - Assert.assertEquals(-1, containerQueuingLimit.getMaxQueueWaitTimeInMs()); + Assertions.assertEquals(6, containerQueuingLimit.getMaxQueueLength()); + Assertions.assertEquals(-1, containerQueuingLimit.getMaxQueueWaitTimeInMs()); selector.computeTask.run(); containerQueuingLimit = calculator.createContainerQueuingLimit(); - Assert.assertEquals(10, containerQueuingLimit.getMaxQueueLength()); - Assert.assertEquals(-1, containerQueuingLimit.getMaxQueueWaitTimeInMs()); + Assertions.assertEquals(10, containerQueuingLimit.getMaxQueueLength()); + Assertions.assertEquals(-1, containerQueuingLimit.getMaxQueueWaitTimeInMs()); // Test Limits do not exceed specified max selector.updateNode(createRMNode("h1", 1, -1, 110)); @@ -474,7 +474,7 @@ public void testContainerQueuingLimit() { selector.updateNode(createRMNode("h6", 6, -1, 160)); selector.computeTask.run(); containerQueuingLimit = calculator.createContainerQueuingLimit(); - Assert.assertEquals(100, containerQueuingLimit.getMaxQueueLength()); + Assertions.assertEquals(100, containerQueuingLimit.getMaxQueueLength()); // Test Limits do not go below specified min selector.updateNode(createRMNode("h1", 1, -1, 1)); @@ -485,7 +485,7 @@ public void testContainerQueuingLimit() { selector.updateNode(createRMNode("h6", 6, -1, 6)); selector.computeTask.run(); containerQueuingLimit = calculator.createContainerQueuingLimit(); - Assert.assertEquals(6, containerQueuingLimit.getMaxQueueLength()); + Assertions.assertEquals(6, containerQueuingLimit.getMaxQueueLength()); } @@ -515,21 +515,21 @@ public void testSelectLocalNode() { Set blacklist = new HashSet<>(); RMNode node = selector.selectLocalNode( "h1", blacklist, defaultResourceRequested); - Assert.assertEquals("h1", node.getHostName()); + Assertions.assertEquals("h1", node.getHostName()); // if node has been added to blacklist blacklist.add("h1"); node = selector.selectLocalNode( "h1", blacklist, defaultResourceRequested); - Assert.assertNull(node); + Assertions.assertNull(node); node = selector.selectLocalNode( "h2", blacklist, defaultResourceRequested); - Assert.assertNull(node); + Assertions.assertNull(node); node = selector.selectLocalNode( "h3", blacklist, defaultResourceRequested); - Assert.assertEquals("h3", node.getHostName()); + Assertions.assertEquals("h3", node.getHostName()); } /** @@ -558,22 +558,22 @@ public void testSelectRackLocalNode() { Set blacklist = new HashSet<>(); RMNode node = selector.selectRackLocalNode( "rack1", blacklist, defaultResourceRequested); - Assert.assertEquals("h1", node.getHostName()); + Assertions.assertEquals("h1", node.getHostName()); // if node has been added to blacklist blacklist.add("h1"); node = selector.selectRackLocalNode( "rack1", blacklist, defaultResourceRequested); - Assert.assertNull(node); + Assertions.assertNull(node); node = selector.selectRackLocalNode( "rack2", blacklist, defaultResourceRequested); - Assert.assertEquals("h3", node.getHostName()); + Assertions.assertEquals("h3", node.getHostName()); blacklist.add("h3"); node = selector.selectRackLocalNode( "rack2", blacklist, defaultResourceRequested); - Assert.assertNull(node); + Assertions.assertNull(node); } /** @@ -599,23 +599,23 @@ public void testSelectAnyNode() { selector.computeTask.run(); - Assert.assertEquals(2, selector.getSortedNodes().size()); + Assertions.assertEquals(2, selector.getSortedNodes().size()); // basic test for selecting node which has queue length // less than queue capacity. Set blacklist = new HashSet<>(); RMNode node = selector.selectAnyNode(blacklist, defaultResourceRequested); - Assert.assertTrue(node.getHostName().equals("h1") || + Assertions.assertTrue(node.getHostName().equals("h1") || node.getHostName().equals("h3")); // if node has been added to blacklist blacklist.add("h1"); node = selector.selectAnyNode(blacklist, defaultResourceRequested); - Assert.assertEquals("h3", node.getHostName()); + Assertions.assertEquals("h3", node.getHostName()); blacklist.add("h3"); node = selector.selectAnyNode(blacklist, defaultResourceRequested); - Assert.assertNull(node); + Assertions.assertNull(node); } @Test @@ -646,7 +646,7 @@ public void testQueueLengthThenResourcesComparator() { comparator.setClusterResource( Resources.add(cn1.getCapability(), cn2.getCapability())); - Assert.assertTrue(comparator.compare(cn1, cn2) < 0); + Assertions.assertTrue(comparator.compare(cn1, cn2) < 0); } // Case 2: Shorter queue should be ranked first before comparing resources @@ -669,7 +669,7 @@ public void testQueueLengthThenResourcesComparator() { comparator.setClusterResource( Resources.add(cn1.getCapability(), cn2.getCapability())); - Assert.assertTrue(comparator.compare(cn1, cn2) < 0); + Assertions.assertTrue(comparator.compare(cn1, cn2) < 0); } // Case 3: No capability vs with capability, @@ -693,7 +693,7 @@ public void testQueueLengthThenResourcesComparator() { comparator.setClusterResource( Resources.add(cn1.getCapability(), cn2.getCapability())); - Assert.assertTrue(comparator.compare(cn1, cn2) < 0); + Assertions.assertTrue(comparator.compare(cn1, cn2) < 0); } // Case 4: Compare same values @@ -716,7 +716,7 @@ public void testQueueLengthThenResourcesComparator() { comparator.setClusterResource( Resources.add(cn1.getCapability(), cn2.getCapability())); - Assert.assertEquals(0, comparator.compare(cn1, cn2)); + Assertions.assertEquals(0, comparator.compare(cn1, cn2)); } // Case 5: If ratio is the same, compare raw values @@ -741,7 +741,7 @@ public void testQueueLengthThenResourcesComparator() { comparator.setClusterResource( Resources.add(cn1.getCapability(), cn2.getCapability())); // Both are 60% allocated, but CN1 has 5 avail VCores, CN2 only has 4 - Assert.assertTrue(comparator.compare(cn1, cn2) < 0); + Assertions.assertTrue(comparator.compare(cn1, cn2) < 0); } // Case 6: by VCores absolute value @@ -764,7 +764,7 @@ public void testQueueLengthThenResourcesComparator() { comparator.setClusterResource( Resources.add(cn1.getCapability(), cn2.getCapability())); - Assert.assertTrue(comparator.compare(cn2, cn1) < 0); + Assertions.assertTrue(comparator.compare(cn2, cn1) < 0); } // Case 7: by memory absolute value @@ -787,7 +787,7 @@ public void testQueueLengthThenResourcesComparator() { comparator.setClusterResource( Resources.add(cn1.getCapability(), cn2.getCapability())); - Assert.assertTrue(comparator.compare(cn2, cn1) < 0); + Assertions.assertTrue(comparator.compare(cn2, cn1) < 0); } // Case 8: Memory should be more constraining in the overall cluster, @@ -811,7 +811,7 @@ public void testQueueLengthThenResourcesComparator() { comparator.setClusterResource( Resources.add(cn1.getCapability(), cn2.getCapability())); - Assert.assertTrue(comparator.compare(cn1, cn2) < 0); + Assertions.assertTrue(comparator.compare(cn1, cn2) < 0); } } 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/fair/FairSchedulerTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerTestBase.java index a3d8f10ab2f9b..a55e3ac7be473 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerTestBase.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerTestBase.java @@ -53,7 +53,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import java.io.File; @@ -379,9 +379,9 @@ protected void checkAppConsumption(FSAppAttempt app, Resource resource) } // available resource - Assert.assertEquals(resource.getMemorySize(), + Assertions.assertEquals(resource.getMemorySize(), app.getCurrentConsumption().getMemorySize()); - Assert.assertEquals(resource.getVirtualCores(), + Assertions.assertEquals(resource.getVirtualCores(), app.getCurrentConsumption().getVirtualCores()); } 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/fair/TestAllocationFileLoaderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestAllocationFileLoaderService.java index 26ddb3aabe195..a6e5048d52043 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestAllocationFileLoaderService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestAllocationFileLoaderService.java @@ -47,9 +47,10 @@ import org.apache.hadoop.yarn.util.SystemClock; import org.apache.hadoop.yarn.util.resource.CustomResourceTypesConfigurationProvider; 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; +import org.junit.jupiter.api.Timeout; import java.io.File; import java.io.IOException; @@ -59,11 +60,7 @@ import java.util.Map; import java.util.Set; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.*; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -84,7 +81,7 @@ public class TestAllocationFileLoaderService { private FairScheduler scheduler; private Configuration conf; - @Before + @BeforeEach public void setup() { SystemClock clock = SystemClock.getInstance(); PlacementManager placementManager = new PlacementManager(); @@ -100,7 +97,7 @@ public void setup() { when(scheduler.getRMContext()).thenReturn(rmContext); } - @After + @AfterEach public void teardown() { new File(ALLOC_FILE).delete(); } @@ -132,14 +129,16 @@ public void testGetAllocationFileFromFileSystem() hdfsCluster.shutdown(true); } - @Test (expected = UnsupportedFileSystemException.class) + @Test public void testDenyGetAllocationFileFromUnsupportedFileSystem() throws UnsupportedFileSystemException { - conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, "badfs:///badfile"); - AllocationFileLoaderService allocLoader = - new AllocationFileLoaderService(scheduler); + assertThrows(UnsupportedFileSystemException.class, ()->{ + conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, "badfs:///badfile"); + AllocationFileLoaderService allocLoader = + new AllocationFileLoaderService(scheduler); - allocLoader.getAllocationFile(conf); + allocLoader.getAllocationFile(conf); + }); } @Test @@ -158,7 +157,8 @@ public void testGetAllocationFileFromClasspath() { } } - @Test (timeout = 10000) + @Test + @Timeout(value = 10) public void testReload() throws Exception { AllocationFileWriter.create() .addQueue(new AllocationFileQueue.Builder("queueA") @@ -359,23 +359,23 @@ public void testAllocationFileParsing() throws Exception { assertEquals(Resources.createResource(0), queueConf.getMinResources("root.queueG.queueH")); - assertNull("Max child resources unexpectedly set for queue root.queueA", - queueConf.getMaxChildResources("root.queueA")); - assertNull("Max child resources unexpectedly set for queue root.queueB", - queueConf.getMaxChildResources("root.queueB")); - assertNull("Max child resources unexpectedly set for queue root.queueC", - queueConf.getMaxChildResources("root.queueC")); - assertNull("Max child resources unexpectedly set for queue root.queueD", - queueConf.getMaxChildResources("root.queueD")); - assertNull("Max child resources unexpectedly set for queue root.queueE", - queueConf.getMaxChildResources("root.queueE")); + assertNull( + queueConf.getMaxChildResources("root.queueA"), "Max child resources unexpectedly set for queue root.queueA"); + assertNull( + queueConf.getMaxChildResources("root.queueB"), "Max child resources unexpectedly set for queue root.queueB"); + assertNull( + queueConf.getMaxChildResources("root.queueC"), "Max child resources unexpectedly set for queue root.queueC"); + assertNull( + queueConf.getMaxChildResources("root.queueD"), "Max child resources unexpectedly set for queue root.queueD"); + assertNull( + queueConf.getMaxChildResources("root.queueE"), "Max child resources unexpectedly set for queue root.queueE"); assertEquals(Resources.createResource(2048, 64), queueConf.getMaxChildResources("root.queueF").getResource()); assertEquals(Resources.createResource(2048, 64), queueConf.getMaxChildResources("root.queueG").getResource()); - assertNull("Max child resources unexpectedly set for " - + "queue root.queueG.queueH", - queueConf.getMaxChildResources("root.queueG.queueH")); + assertNull( + queueConf.getMaxChildResources("root.queueG.queueH"), "Max child resources unexpectedly set for " + + "queue root.queueG.queueH"); assertEquals(15, queueConf.getQueueMaxApps("root." + YarnConfiguration.DEFAULT_QUEUE_NAME)); @@ -618,8 +618,8 @@ public void testSimplePlacementPolicyFromConf() throws Exception { .getQueuePlacementManager().getPlacementRules(); assertEquals(2, rules.size()); assertEquals(SpecifiedPlacementRule.class, rules.get(0).getClass()); - assertFalse("Create flag was not set to false", - ((FSPlacementRule)rules.get(0)).getCreateFlag()); + assertFalse( + ((FSPlacementRule)rules.get(0)).getCreateFlag(), "Create flag was not set to false"); assertEquals(DefaultPlacementRule.class, rules.get(1).getClass()); } @@ -627,61 +627,67 @@ public void testSimplePlacementPolicyFromConf() throws Exception { * Verify that you can't place queues at the same level as the root queue in * the allocations file. */ - @Test (expected = AllocationConfigurationException.class) + @Test public void testQueueAlongsideRoot() throws Exception { - conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); + assertThrows(AllocationConfigurationException.class, ()->{ + conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); - AllocationFileWriter.create() - .addQueue(new AllocationFileQueue.Builder("root").build()) - .addQueue(new AllocationFileQueue.Builder("other").build()) - .writeToFile(ALLOC_FILE); + AllocationFileWriter.create() + .addQueue(new AllocationFileQueue.Builder("root").build()) + .addQueue(new AllocationFileQueue.Builder("other").build()) + .writeToFile(ALLOC_FILE); - AllocationFileLoaderService allocLoader = - new AllocationFileLoaderService(scheduler); - allocLoader.init(conf); - ReloadListener confHolder = new ReloadListener(); - allocLoader.setReloadListener(confHolder); - allocLoader.reloadAllocations(); + AllocationFileLoaderService allocLoader = + new AllocationFileLoaderService(scheduler); + allocLoader.init(conf); + ReloadListener confHolder = new ReloadListener(); + allocLoader.setReloadListener(confHolder); + allocLoader.reloadAllocations(); + }); } /** * Verify that you can't include periods as the queue name in the allocations * file. */ - @Test (expected = AllocationConfigurationException.class) + @Test public void testQueueNameContainingPeriods() throws Exception { - conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); + assertThrows(AllocationConfigurationException.class, ()->{ + conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); - AllocationFileWriter.create() - .addQueue(new AllocationFileQueue.Builder("parent1.child").build()) - .writeToFile(ALLOC_FILE); + AllocationFileWriter.create() + .addQueue(new AllocationFileQueue.Builder("parent1.child").build()) + .writeToFile(ALLOC_FILE); - AllocationFileLoaderService allocLoader = - new AllocationFileLoaderService(scheduler); - allocLoader.init(conf); - ReloadListener confHolder = new ReloadListener(); - allocLoader.setReloadListener(confHolder); - allocLoader.reloadAllocations(); + AllocationFileLoaderService allocLoader = + new AllocationFileLoaderService(scheduler); + allocLoader.init(conf); + ReloadListener confHolder = new ReloadListener(); + allocLoader.setReloadListener(confHolder); + allocLoader.reloadAllocations(); + }); } /** * Verify that you can't have the queue name with whitespace only in the * allocations file. */ - @Test (expected = AllocationConfigurationException.class) + @Test public void testQueueNameContainingOnlyWhitespace() throws Exception { - conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); + assertThrows(AllocationConfigurationException.class, ()->{ + conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); - AllocationFileWriter.create() - .addQueue(new AllocationFileQueue.Builder(" ").build()) - .writeToFile(ALLOC_FILE); + AllocationFileWriter.create() + .addQueue(new AllocationFileQueue.Builder(" ").build()) + .writeToFile(ALLOC_FILE); - AllocationFileLoaderService allocLoader = - new AllocationFileLoaderService(scheduler); - allocLoader.init(conf); - ReloadListener confHolder = new ReloadListener(); - allocLoader.setReloadListener(confHolder); - allocLoader.reloadAllocations(); + AllocationFileLoaderService allocLoader = + new AllocationFileLoaderService(scheduler); + allocLoader.init(conf); + ReloadListener confHolder = new ReloadListener(); + allocLoader.setReloadListener(confHolder); + allocLoader.reloadAllocations(); + }); } @Test @@ -832,39 +838,43 @@ public void testParentTagWithChild() throws Exception { * Verify that you can't have the queue name with just a non breaking * whitespace in the allocations file. */ - @Test (expected = AllocationConfigurationException.class) + @Test public void testQueueNameContainingNBWhitespace() throws Exception { - conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); + assertThrows(AllocationConfigurationException.class, ()->{ + conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); - AllocationFileWriter.create() - .addQueue(new AllocationFileQueue.Builder("\u00a0").build()) - .writeToFile(ALLOC_FILE); + AllocationFileWriter.create() + .addQueue(new AllocationFileQueue.Builder("\u00a0").build()) + .writeToFile(ALLOC_FILE); - AllocationFileLoaderService allocLoader = - new AllocationFileLoaderService(scheduler); - allocLoader.init(conf); - ReloadListener confHolder = new ReloadListener(); - allocLoader.setReloadListener(confHolder); - allocLoader.reloadAllocations(); + AllocationFileLoaderService allocLoader = + new AllocationFileLoaderService(scheduler); + allocLoader.init(conf); + ReloadListener confHolder = new ReloadListener(); + allocLoader.setReloadListener(confHolder); + allocLoader.reloadAllocations(); + }); } /** * Verify that defaultQueueSchedulingMode can't accept FIFO as a value. */ - @Test (expected = AllocationConfigurationException.class) + @Test public void testDefaultQueueSchedulingModeIsFIFO() throws Exception { - conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); + assertThrows(AllocationConfigurationException.class, ()->{ + conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); - AllocationFileWriter.create() - .fifoDefaultQueueSchedulingPolicy() - .writeToFile(ALLOC_FILE); + AllocationFileWriter.create() + .fifoDefaultQueueSchedulingPolicy() + .writeToFile(ALLOC_FILE); - AllocationFileLoaderService allocLoader = - new AllocationFileLoaderService(scheduler); - allocLoader.init(conf); - ReloadListener confHolder = new ReloadListener(); - allocLoader.setReloadListener(confHolder); - allocLoader.reloadAllocations(); + AllocationFileLoaderService allocLoader = + new AllocationFileLoaderService(scheduler); + allocLoader.init(conf); + ReloadListener confHolder = new ReloadListener(); + allocLoader.setReloadListener(confHolder); + allocLoader.reloadAllocations(); + }); } @Test @@ -896,11 +906,11 @@ public void testReservableQueue() throws Exception { assertTrue(allocConf.isReservable(reservableQueuePath)); Map> configuredQueues = allocConf.getConfiguredQueues(); - assertTrue("reservable queue is expected be to a parent queue", - configuredQueues.get(FSQueueType.PARENT).contains(reservableQueueName)); - assertFalse("reservable queue should not be a leaf queue", - configuredQueues.get(FSQueueType.LEAF) - .contains(reservableQueueName)); + assertTrue( + configuredQueues.get(FSQueueType.PARENT).contains(reservableQueueName), "reservable queue is expected be to a parent queue"); + assertFalse( + configuredQueues.get(FSQueueType.LEAF) + .contains(reservableQueueName), "reservable queue should not be a leaf queue"); assertTrue(allocConf.getMoveOnExpiry(reservableQueuePath)); assertEquals(ReservationSchedulerConfiguration.DEFAULT_RESERVATION_WINDOW, @@ -925,24 +935,26 @@ public void testReservableQueue() throws Exception { * Verify that you can't have dynamic user queue and reservable queue on * the same queue. */ - @Test (expected = AllocationConfigurationException.class) + @Test public void testReservableCannotBeCombinedWithDynamicUserQueue() throws Exception { - conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); + assertThrows(AllocationConfigurationException.class, ()->{ + conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); - AllocationFileWriter.create() - .addQueue(new AllocationFileQueue.Builder("notboth") - .parent(true) - .reservation() - .build()) - .writeToFile(ALLOC_FILE); + AllocationFileWriter.create() + .addQueue(new AllocationFileQueue.Builder("notboth") + .parent(true) + .reservation() + .build()) + .writeToFile(ALLOC_FILE); - AllocationFileLoaderService allocLoader = - new AllocationFileLoaderService(scheduler); - allocLoader.init(conf); - ReloadListener confHolder = new ReloadListener(); - allocLoader.setReloadListener(confHolder); - allocLoader.reloadAllocations(); + AllocationFileLoaderService allocLoader = + new AllocationFileLoaderService(scheduler); + allocLoader.init(conf); + ReloadListener confHolder = new ReloadListener(); + allocLoader.setReloadListener(confHolder); + allocLoader.reloadAllocations(); + }); } private class ReloadListener implements AllocationFileLoaderService.Listener { 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/fair/TestAppRunnability.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestAppRunnability.java index be685eebc8d5d..deb22ca632995 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestAppRunnability.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestAppRunnability.java @@ -18,10 +18,10 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertSame; -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.assertSame; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.File; import java.io.IOException; @@ -48,9 +48,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair .allocationfile.AllocationFileWriter; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** * This class is to test the fair scheduler functionality of @@ -60,7 +60,7 @@ public class TestAppRunnability extends FairSchedulerTestBase { private final static String ALLOC_FILE = new File(TEST_DIR, "test-queues").getAbsolutePath(); - @Before + @BeforeEach public void setUp() throws IOException { conf = createConfiguration(); resourceManager = new MockRM(conf); @@ -68,7 +68,7 @@ public void setUp() throws IOException { scheduler = (FairScheduler) resourceManager.getResourceScheduler(); } - @After + @AfterEach public void tearDown() { if (resourceManager != null) { resourceManager.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/fair/TestApplicationMasterServiceWithFS.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestApplicationMasterServiceWithFS.java index def37493b1f5c..4e16d5b43af60 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestApplicationMasterServiceWithFS.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestApplicationMasterServiceWithFS.java @@ -41,10 +41,11 @@ .allocationfile.AllocationFileQueue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair .allocationfile.AllocationFileWriter; -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; /** * Test Application master service using Fair scheduler. @@ -60,7 +61,7 @@ public class TestApplicationMasterServiceWithFS { private AllocateResponse allocateResponse; private static YarnConfiguration configuration; - @BeforeClass + @BeforeAll public static void setup() { String allocFile = GenericTestUtils.getTestDir(TEST_FOLDER).getAbsolutePath(); @@ -81,13 +82,14 @@ public static void setup() { .writeToFile(allocFile); } - @AfterClass + @AfterAll public static void teardown(){ File allocFile = GenericTestUtils.getTestDir(TEST_FOLDER); allocFile.delete(); } - @Test(timeout = 3000000) + @Test + @Timeout(value = 3000) public void testQueueLevelContainerAllocationFail() throws Exception { MockRM rm = new MockRM(configuration); rm.start(); @@ -111,10 +113,10 @@ public void testQueueLevelContainerAllocationFail() throws Exception { am1.addRequests(new String[] { "127.0.0.1" }, MEMORY_ALLOCATION, 1, 1); try { allocateResponse = am1.schedule(); // send the request - Assert.fail(); + Assertions.fail(); } catch (Exception e) { - Assert.assertTrue(e instanceof InvalidResourceRequestException); - Assert.assertEquals( + Assertions.assertTrue(e instanceof InvalidResourceRequestException); + Assertions.assertEquals( InvalidResourceRequestException.InvalidResourceType.GREATER_THEN_MAX_ALLOCATION, ((InvalidResourceRequestException) e).getInvalidResourceType()); @@ -123,12 +125,14 @@ public void testQueueLevelContainerAllocationFail() throws Exception { } } - @Test(timeout = 3000000) + @Test + @Timeout(value = 3000) public void testQueueLevelContainerAllocationSuccess() throws Exception { testFairSchedulerContainerAllocationSuccess("queueB"); } - @Test(timeout = 3000000) + @Test + @Timeout(value = 3000) public void testSchedulerLevelContainerAllocationSuccess() throws Exception { testFairSchedulerContainerAllocationSuccess("queueC"); } @@ -164,16 +168,16 @@ private void testFairSchedulerContainerAllocationSuccess(String queueName) try { allocateResponse = am1.schedule(); } catch (Exception e) { - Assert.fail("Allocation should be successful"); + Assertions.fail("Allocation should be successful"); } return allocateResponse.getAllocatedContainers().size() > 0; }, 1000, 10000); Container allocatedContainer = allocateResponse.getAllocatedContainers().get(0); - Assert.assertEquals(MEMORY_ALLOCATION, + Assertions.assertEquals(MEMORY_ALLOCATION, allocatedContainer.getResource().getMemorySize()); - Assert.assertEquals(1, allocatedContainer.getResource().getVirtualCores()); + Assertions.assertEquals(1, allocatedContainer.getResource().getVirtualCores()); 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/fair/TestComputeFairShares.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestComputeFairShares.java index 5d3d49ab82217..09e4e2bbe65fa 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestComputeFairShares.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestComputeFairShares.java @@ -23,12 +23,12 @@ import org.apache.hadoop.yarn.api.records.ResourceInformation; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.ComputeFairShares; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** * Exercise the computeFairShares method in SchedulingAlgorithms. @@ -36,7 +36,7 @@ public class TestComputeFairShares { private List scheds; - @Before + @BeforeEach public void setUp() throws Exception { scheds = new ArrayList<>(); } @@ -241,11 +241,11 @@ public void testCPU() { * Check that a given list of shares have been assigned to this.scheds. */ private void verifyMemoryShares(long... shares) { - Assert.assertEquals("Number of shares and schedulables are not consistent", - scheds.size(), shares.length); + Assertions.assertEquals( + scheds.size(), shares.length, "Number of shares and schedulables are not consistent"); for (int i = 0; i < shares.length; i++) { - Assert.assertEquals("Expected share number " + i + " in list wrong", - shares[i], scheds.get(i).getFairShare().getMemorySize()); + Assertions.assertEquals( + shares[i], scheds.get(i).getFairShare().getMemorySize(), "Expected share number " + i + " in list wrong"); } } @@ -253,11 +253,11 @@ private void verifyMemoryShares(long... shares) { * Check that a given list of shares have been assigned to this.scheds. */ private void verifyCPUShares(int... shares) { - Assert.assertEquals("Number of shares and schedulables are not consistent", - scheds.size(), shares.length); + Assertions.assertEquals( + scheds.size(), shares.length, "Number of shares and schedulables are not consistent"); for (int i = 0; i < shares.length; i++) { - Assert.assertEquals("Expected share number " + i + " in list wrong", - shares[i], scheds.get(i).getFairShare().getVirtualCores()); + Assertions.assertEquals( + shares[i], scheds.get(i).getFairShare().getVirtualCores(), "Expected share number " + i + " in list wrong"); } } } 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/fair/TestConfigurableResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestConfigurableResource.java index debd8ae7ebe53..951938edf7c7f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestConfigurableResource.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestConfigurableResource.java @@ -20,11 +20,11 @@ import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.Test; +import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; /** * To test class {@link ConfigurableResource}. @@ -42,11 +42,11 @@ public void testGetResourceWithPercentage() { assertEquals( configurableResource.getResource(clusterResource).getVirtualCores(), 1); - assertNull("The absolute resource should be null since object" - + " configurableResource is initialized with percentages", - configurableResource.getResource()); - assertNull("The absolute resource should be null since cluster resource" - + " is null", configurableResource.getResource(null)); + assertNull( + configurableResource.getResource(), "The absolute resource should be null since object" + + " configurableResource is initialized with percentages"); + assertNull(configurableResource.getResource(null), "The absolute resource should be null since cluster resource" + + " is null"); } @Test 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/fair/TestContinuousScheduling.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestContinuousScheduling.java index 0d1f6294d8701..c2df8521ecaa6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestContinuousScheduling.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestContinuousScheduling.java @@ -45,19 +45,20 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent; import org.apache.hadoop.yarn.util.ControlledClock; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.After; -import org.junit.Assert; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; -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.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.isA; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.spy; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.ArrayList; import java.util.HashSet; @@ -86,7 +87,7 @@ public Configuration createConfiguration() { } @SuppressWarnings("deprecation") - @Before + @BeforeEach public void setup() { QueueMetrics.clearQueueMetrics(); DefaultMetricsSystem.setMiniClusterMode(true); @@ -106,7 +107,7 @@ public void setup() { assertEquals(mockClock, scheduler.getClock()); } - @After + @AfterEach public void teardown() { if (resourceManager != null) { resourceManager.stop(); @@ -114,7 +115,8 @@ public void teardown() { } } - @Test (timeout = 60000) + @Test + @Timeout(value = 60) public void testBasic() throws InterruptedException { // Add one node String host = "127.0.0.1"; @@ -146,7 +148,8 @@ public void testBasic() throws InterruptedException { checkAppConsumption(app, Resources.createResource(1024, 1)); } - @Test (timeout = 10000) + @Test + @Timeout(value = 10) public void testSortedNodes() throws Exception { // Add two nodes RMNode node1 = @@ -204,7 +207,7 @@ public void testSortedNodes() throws Exception { while (it.hasNext()) { nodes.add(it.next().getContainer().getNodeId()); } - Assert.assertEquals(2, nodes.size()); + Assertions.assertEquals(2, nodes.size()); } @SuppressWarnings("deprecation") @@ -226,8 +229,8 @@ public void testWithNodeRemoved() throws Exception { .rollMasterKey(); scheduler.setRMContext(resourceManager.getRMContext()); - Assert.assertTrue("Continuous scheduling should be disabled.", - !scheduler.isContinuousSchedulingEnabled()); + Assertions.assertTrue( + !scheduler.isContinuousSchedulingEnabled(), "Continuous scheduling should be disabled."); scheduler.init(conf); scheduler.start(); @@ -242,15 +245,15 @@ public void testWithNodeRemoved() throws Exception { "127.0.0.2"); NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2); scheduler.handle(nodeEvent2); - Assert.assertEquals("We should have two alive nodes.", - 2, scheduler.getNumClusterNodes()); + Assertions.assertEquals( + 2, scheduler.getNumClusterNodes(), "We should have two alive nodes."); // Remove one node NodeRemovedSchedulerEvent removeNode1 = new NodeRemovedSchedulerEvent(node1); scheduler.handle(removeNode1); - Assert.assertEquals("We should only have one alive node.", - 1, scheduler.getNumClusterNodes()); + Assertions.assertEquals( + 1, scheduler.getNumClusterNodes(), "We should only have one alive node."); // Invoke the continuous scheduling once try { @@ -284,16 +287,16 @@ public void testInterruptedException() scheduler.init(conf); scheduler.start(); FairScheduler spyScheduler = spy(scheduler); - Assert.assertTrue("Continuous scheduling should be disabled.", - !spyScheduler.isContinuousSchedulingEnabled()); + Assertions.assertTrue( + !spyScheduler.isContinuousSchedulingEnabled(), "Continuous scheduling should be disabled."); // Add one node RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(8 * 1024, 8), 1, "127.0.0.1"); NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); spyScheduler.handle(nodeEvent1); - Assert.assertEquals("We should have one alive node.", - 1, spyScheduler.getNumClusterNodes()); + Assertions.assertEquals( + 1, spyScheduler.getNumClusterNodes(), "We should have one alive node."); InterruptedException ie = new InterruptedException(); doThrow(new YarnRuntimeException(ie)).when(spyScheduler). attemptScheduling(isA(FSSchedulerNode.class)); @@ -302,7 +305,7 @@ public void testInterruptedException() spyScheduler.continuousSchedulingAttempt(); fail("Expected InterruptedException to stop schedulingThread"); } catch (InterruptedException e) { - Assert.assertEquals(ie, e); + Assertions.assertEquals(ie, e); } } @@ -320,7 +323,7 @@ public void testSchedulerThreadLifeCycle() throws InterruptedException { Thread.sleep(50); } - assertNotEquals("The Scheduling thread is still alive", 0, numRetries); + assertNotEquals(0, numRetries, "The Scheduling thread is still alive"); } @SuppressWarnings("deprecation") 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/fair/TestFSAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.java index bda04187ef1a1..880fa9057c040 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.java @@ -28,10 +28,10 @@ import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.server.resourcemanager.MockNodes; import org.apache.hadoop.yarn.server.resourcemanager.MockRM; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.spy; @@ -57,13 +57,13 @@ import org.apache.hadoop.yarn.util.ControlledClock; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; public class TestFSAppAttempt extends FairSchedulerTestBase { - @Before + @BeforeEach public void setup() { Configuration conf = createConfiguration(); resourceManager = new MockRM(conf); @@ -297,8 +297,8 @@ public void testHeadroomWithBlackListedNodes() { "127.0.0.2"); NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2); scheduler.handle(nodeEvent2); - assertEquals("We should have two alive nodes.", - 2, scheduler.getNumClusterNodes()); + assertEquals( + 2, scheduler.getNumClusterNodes(), "We should have two alive nodes."); Resource clusterResource = scheduler.getClusterResource(); Resource clusterUsage = scheduler.getRootQueueMetrics() .getAllocatedResources(); 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/fair/TestFSAppStarvation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppStarvation.java index aeefbcd8202fc..a80ab83292691 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppStarvation.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppStarvation.java @@ -29,13 +29,13 @@ .allocationfile.AllocationFileWriter; import org.apache.hadoop.yarn.util.ControlledClock; -import org.junit.After; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +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 org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.File; @@ -55,7 +55,7 @@ public class TestFSAppStarvation extends FairSchedulerTestBase { private FairSchedulerWithMockPreemption.MockPreemptionThread preemptionThread; - @Before + @BeforeEach public void setup() { createConfiguration(); conf.set(YarnConfiguration.RM_SCHEDULER, @@ -69,7 +69,7 @@ public void setup() { conf.setLong(FairSchedulerConfiguration.UPDATE_INTERVAL_MS, Long.MAX_VALUE); } - @After + @AfterEach public void teardown() { ALLOC_FILE.delete(); conf = null; @@ -89,8 +89,8 @@ public void testPreemptionDisabled() throws Exception { setupClusterAndSubmitJobs(); - assertNull("Found starved apps even when preemption is turned off", - scheduler.getContext().getStarvedApps()); + assertNull( + scheduler.getContext().getStarvedApps(), "Found starved apps even when preemption is turned off"); } /* @@ -109,18 +109,18 @@ public void testPreemptionEnabled() throws Exception { Thread.sleep(10); } - assertNotNull("FSContext does not have an FSStarvedApps instance", - scheduler.getContext().getStarvedApps()); - assertEquals("Expecting 3 starved applications, one each for the " - + "minshare and fairshare queues", - 3, preemptionThread.uniqueAppsAdded()); + assertNotNull( + scheduler.getContext().getStarvedApps(), "FSContext does not have an FSStarvedApps instance"); + assertEquals( + 3, preemptionThread.uniqueAppsAdded(), "Expecting 3 starved applications, one each for the " + + "minshare and fairshare queues"); // Verify apps are added again only after the set delay for starvation has // passed. clock.tickSec(1); scheduler.update(); - assertEquals("Apps re-added even before starvation delay passed", - preemptionThread.totalAppsAdded(), preemptionThread.uniqueAppsAdded()); + assertEquals( + preemptionThread.totalAppsAdded(), preemptionThread.uniqueAppsAdded(), "Apps re-added even before starvation delay passed"); verifyLeafQueueStarvation(); clock.tickMsec( @@ -136,10 +136,10 @@ public void testPreemptionEnabled() throws Exception { Thread.sleep(10); } - assertEquals("Each app should be marked as starved once" + - " at each scheduler update above", - preemptionThread.totalAppsAdded(), - preemptionThread.uniqueAppsAdded() * 2); + assertEquals( + preemptionThread.totalAppsAdded() +, preemptionThread.uniqueAppsAdded() * 2, "Each app should be marked as starved once" + + " at each scheduler update above"); } /* @@ -153,10 +153,10 @@ public void testClusterUtilizationThreshold() throws Exception { setupClusterAndSubmitJobs(); - assertNotNull("FSContext does not have an FSStarvedApps instance", - scheduler.getContext().getStarvedApps()); - assertEquals("Found starved apps when preemption threshold is over 100%", 0, - preemptionThread.totalAppsAdded()); + assertNotNull( + scheduler.getContext().getStarvedApps(), "FSContext does not have an FSStarvedApps instance"); + assertEquals(0 +, preemptionThread.totalAppsAdded(), "Found starved apps when preemption threshold is over 100%"); } private void verifyLeafQueueStarvation() { @@ -224,8 +224,8 @@ private void setupStarvedCluster() { .build()) .writeToFile(ALLOC_FILE.getAbsolutePath()); - assertTrue("Allocation file does not exist, not running the test", - ALLOC_FILE.exists()); + assertTrue( + ALLOC_FILE.exists(), "Allocation file does not exist, not running the test"); resourceManager = new MockRM(conf); scheduler = (FairScheduler) resourceManager.getResourceScheduler(); 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/fair/TestFSLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSLeafQueue.java index 61122833f12de..45d02eb8fa612 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSLeafQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSLeafQueue.java @@ -19,8 +19,8 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; import java.io.File; @@ -50,13 +50,14 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.allocationfile.AllocationFileQueue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.allocationfile.AllocationFileWriter; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.mockito.Mockito; import java.util.Map; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; public class TestFSLeafQueue extends FairSchedulerTestBase { private final static String ALLOC_FILE = new File(TEST_DIR, @@ -65,14 +66,14 @@ public class TestFSLeafQueue extends FairSchedulerTestBase { private static final float MAX_AM_SHARE = 0.5f; private static final String CUSTOM_RESOURCE = "test1"; - @Before + @BeforeEach public void setup() throws IOException { conf = createConfiguration(); conf.setClass(YarnConfiguration.RM_SCHEDULER, FairScheduler.class, ResourceScheduler.class); } - @After + @AfterEach public void teardown() { if (resourceManager != null) { resourceManager.stop(); @@ -105,11 +106,12 @@ public void testUpdateDemand() { schedulable.updateDemand(); - assertTrue("Demand is greater than max allowed ", - Resources.equals(schedulable.getDemand(), maxResource)); + assertTrue( + Resources.equals(schedulable.getDemand(), maxResource), "Demand is greater than max allowed "); } - @Test (timeout = 5000) + @Test + @Timeout(value = 5) public void test() { conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); @@ -220,15 +222,15 @@ public void run() { // start all test runners startBlocker.countDown(); int testTimeout = 2; - assertTrue("Timeout waiting for more than " + testTimeout + " seconds", - allDone.await(testTimeout, TimeUnit.SECONDS)); + assertTrue( + allDone.await(testTimeout, TimeUnit.SECONDS), "Timeout waiting for more than " + testTimeout + " seconds"); } catch (InterruptedException ie) { exceptions.add(ie); } finally { threadPool.shutdownNow(); } - assertTrue("Test failed with exception(s)" + exceptions, - exceptions.isEmpty()); + assertTrue( + exceptions.isEmpty(), "Test failed with exception(s)" + exceptions); } @Test @@ -268,7 +270,7 @@ public void testCanRunAppAMReturnsTrue() { verifyQueueMetricsForCustomResources(queue); boolean result = queue.canRunAppAM(appAMResource); - assertTrue("AM should have been allocated!", result); + assertTrue(result, "AM should have been allocated!"); verifyAMShare(queue, expectedAMShare, customResourceValues); } @@ -318,7 +320,7 @@ public void testCanRunAppAMReturnsFalse() { verifyQueueMetricsForCustomResources(queue); boolean result = queue.canRunAppAM(appAMResource); - assertFalse("AM should not have been allocated!", result); + assertFalse(result, "AM should not have been allocated!"); verifyAMShare(queue, expectedAMShare, customResourceValues); } @@ -338,8 +340,8 @@ private void verifyAMShare(FSLeafQueue schedulable, //make sure to verify custom resource value explicitly! assertEquals(5L, customResourceValue); - assertEquals("AM share is not the expected!", expectedAMShare, - actualAMShare); + assertEquals(expectedAMShare +, actualAMShare, "AM share is not the expected!"); } private Map verifyQueueMetricsForCustomResources( @@ -348,10 +350,10 @@ private Map verifyQueueMetricsForCustomResources( schedulable.getMetrics().getCustomResources().getMaxAMShare(); Map customResourceValues = maxAMShareCustomResources .getValues(); - assertNotNull("Queue metrics for custom resources should not be null!", - maxAMShareCustomResources); - assertNotNull("Queue metrics for custom resources resource values " + - "should not be null!", customResourceValues); + assertNotNull( + maxAMShareCustomResources, "Queue metrics for custom resources should not be null!"); + assertNotNull(customResourceValues, "Queue metrics for custom resources resource values " + + "should not be null!"); return customResourceValues; } } 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/fair/TestFSParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSParentQueue.java index 32da3a7c5fa9c..fbda1f959aa3b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSParentQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSParentQueue.java @@ -22,10 +22,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager; import org.apache.hadoop.yarn.util.SystemClock; import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -33,7 +33,7 @@ public class TestFSParentQueue { private QueueManager queueManager; - @Before + @BeforeEach public void setUp() { FairSchedulerConfiguration conf = new FairSchedulerConfiguration(); RMContext rmContext = mock(RMContext.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/scheduler/fair/TestFSQueueMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSQueueMetrics.java index 15e2d02421725..355a2eb6af3fd 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSQueueMetrics.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSQueueMetrics.java @@ -30,10 +30,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestQueueMetrics; 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 static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; /** * The test class for {@link FSQueueMetrics}. @@ -45,7 +45,7 @@ public class TestFSQueueMetrics { private static final String RESOURCE_NAME = "test1"; private static final String QUEUE_NAME = "single"; - @Before + @BeforeEach public void setUp() { ms = new MetricsSystemImpl(); QueueMetrics.clearQueueMetrics(); @@ -95,30 +95,30 @@ public void testSetFairShare() { 20L)); metrics.setFairShare(res); - assertEquals(getErrorMessage("fairShareMB"), - 2048L, metrics.getFairShareMB()); - assertEquals(getErrorMessage("fairShareVcores"), - 4L, metrics.getFairShareVirtualCores()); - assertEquals(getErrorMessage("fairShareMB"), - 2048L, metrics.getFairShare().getMemorySize()); - assertEquals(getErrorMessage("fairShareVcores"), - 4L, metrics.getFairShare().getVirtualCores()); - assertEquals(getErrorMessage("fairShare for resource: " + RESOURCE_NAME), - 20L, metrics.getFairShare().getResourceValue(RESOURCE_NAME)); + assertEquals( + 2048L, metrics.getFairShareMB(), getErrorMessage("fairShareMB")); + assertEquals( + 4L, metrics.getFairShareVirtualCores(), getErrorMessage("fairShareVcores")); + assertEquals( + 2048L, metrics.getFairShare().getMemorySize(), getErrorMessage("fairShareMB")); + assertEquals( + 4L, metrics.getFairShare().getVirtualCores(), getErrorMessage("fairShareVcores")); + assertEquals( + 20L, metrics.getFairShare().getResourceValue(RESOURCE_NAME), getErrorMessage("fairShare for resource: " + RESOURCE_NAME)); res = Resource.newInstance(2049L, 5); metrics.setFairShare(res); - assertEquals(getErrorMessage("fairShareMB"), - 2049L, metrics.getFairShareMB()); - assertEquals(getErrorMessage("fairShareVcores"), - 5L, metrics.getFairShareVirtualCores()); - assertEquals(getErrorMessage("fairShareMB"), - 2049L, metrics.getFairShare().getMemorySize()); - assertEquals(getErrorMessage("fairShareVcores"), - 5L, metrics.getFairShare().getVirtualCores()); - assertEquals(getErrorMessage("fairShare for resource: " + RESOURCE_NAME), - 0, metrics.getFairShare().getResourceValue(RESOURCE_NAME)); + assertEquals( + 2049L, metrics.getFairShareMB(), getErrorMessage("fairShareMB")); + assertEquals( + 5L, metrics.getFairShareVirtualCores(), getErrorMessage("fairShareVcores")); + assertEquals( + 2049L, metrics.getFairShare().getMemorySize(), getErrorMessage("fairShareMB")); + assertEquals( + 5L, metrics.getFairShare().getVirtualCores(), getErrorMessage("fairShareVcores")); + assertEquals( + 0, metrics.getFairShare().getResourceValue(RESOURCE_NAME), getErrorMessage("fairShare for resource: " + RESOURCE_NAME)); } @Test @@ -129,36 +129,36 @@ public void testSetSteadyFairShare() { 20L)); metrics.setSteadyFairShare(res); - assertEquals(getErrorMessage("steadyFairShareMB"), - 2048L, metrics.getSteadyFairShareMB()); - assertEquals(getErrorMessage("steadyFairShareVcores"), - 4L, metrics.getSteadyFairShareVCores()); + assertEquals( + 2048L, metrics.getSteadyFairShareMB(), getErrorMessage("steadyFairShareMB")); + assertEquals( + 4L, metrics.getSteadyFairShareVCores(), getErrorMessage("steadyFairShareVcores")); Resource steadyFairShare = metrics.getSteadyFairShare(); - assertEquals(getErrorMessage("steadyFairShareMB"), - 2048L, steadyFairShare.getMemorySize()); - assertEquals(getErrorMessage("steadyFairShareVcores"), - 4L, steadyFairShare.getVirtualCores()); - assertEquals(getErrorMessage("steadyFairShare for resource: " + - RESOURCE_NAME), - 20L, steadyFairShare.getResourceValue(RESOURCE_NAME)); + assertEquals( + 2048L, steadyFairShare.getMemorySize(), getErrorMessage("steadyFairShareMB")); + assertEquals( + 4L, steadyFairShare.getVirtualCores(), getErrorMessage("steadyFairShareVcores")); + assertEquals( + 20L, steadyFairShare.getResourceValue(RESOURCE_NAME), getErrorMessage("steadyFairShare for resource: " + + RESOURCE_NAME)); res = Resource.newInstance(2049L, 5); metrics.setSteadyFairShare(res); - assertEquals(getErrorMessage("steadyFairShareMB"), - 2049L, metrics.getSteadyFairShareMB()); - assertEquals(getErrorMessage("steadyFairShareVcores"), - 5L, metrics.getSteadyFairShareVCores()); + assertEquals( + 2049L, metrics.getSteadyFairShareMB(), getErrorMessage("steadyFairShareMB")); + assertEquals( + 5L, metrics.getSteadyFairShareVCores(), getErrorMessage("steadyFairShareVcores")); steadyFairShare = metrics.getSteadyFairShare(); - assertEquals(getErrorMessage("steadyFairShareMB"), - 2049L, steadyFairShare.getMemorySize()); - assertEquals(getErrorMessage("steadyFairShareVcores"), - 5L, steadyFairShare.getVirtualCores()); - assertEquals(getErrorMessage("steadyFairShare for resource: " + - RESOURCE_NAME), - 0, steadyFairShare.getResourceValue(RESOURCE_NAME)); + assertEquals( + 2049L, steadyFairShare.getMemorySize(), getErrorMessage("steadyFairShareMB")); + assertEquals( + 5L, steadyFairShare.getVirtualCores(), getErrorMessage("steadyFairShareVcores")); + assertEquals( + 0, steadyFairShare.getResourceValue(RESOURCE_NAME), getErrorMessage("steadyFairShare for resource: " + + RESOURCE_NAME)); } @Test @@ -169,30 +169,30 @@ public void testSetMinShare() { 20L)); metrics.setMinShare(res); - assertEquals(getErrorMessage("minShareMB"), - 2048L, metrics.getMinShareMB()); - assertEquals(getErrorMessage("minShareVcores"), - 4L, metrics.getMinShareVirtualCores()); - assertEquals(getErrorMessage("minShareMB"), - 2048L, metrics.getMinShare().getMemorySize()); - assertEquals(getErrorMessage("minShareVcores"), - 4L, metrics.getMinShare().getVirtualCores()); - assertEquals(getErrorMessage("minShare for resource: " + RESOURCE_NAME), - 20L, metrics.getMinShare().getResourceValue(RESOURCE_NAME)); + assertEquals( + 2048L, metrics.getMinShareMB(), getErrorMessage("minShareMB")); + assertEquals( + 4L, metrics.getMinShareVirtualCores(), getErrorMessage("minShareVcores")); + assertEquals( + 2048L, metrics.getMinShare().getMemorySize(), getErrorMessage("minShareMB")); + assertEquals( + 4L, metrics.getMinShare().getVirtualCores(), getErrorMessage("minShareVcores")); + assertEquals( + 20L, metrics.getMinShare().getResourceValue(RESOURCE_NAME), getErrorMessage("minShare for resource: " + RESOURCE_NAME)); res = Resource.newInstance(2049L, 5); metrics.setMinShare(res); - assertEquals(getErrorMessage("minShareMB"), - 2049L, metrics.getMinShareMB()); - assertEquals(getErrorMessage("minShareVcores"), - 5L, metrics.getMinShareVirtualCores()); - assertEquals(getErrorMessage("minShareMB"), - 2049L, metrics.getMinShare().getMemorySize()); - assertEquals(getErrorMessage("minShareVcores"), - 5L, metrics.getMinShare().getVirtualCores()); - assertEquals(getErrorMessage("minShare for resource: " + RESOURCE_NAME), - 0, metrics.getMinShare().getResourceValue(RESOURCE_NAME)); + assertEquals( + 2049L, metrics.getMinShareMB(), getErrorMessage("minShareMB")); + assertEquals( + 5L, metrics.getMinShareVirtualCores(), getErrorMessage("minShareVcores")); + assertEquals( + 2049L, metrics.getMinShare().getMemorySize(), getErrorMessage("minShareMB")); + assertEquals( + 5L, metrics.getMinShare().getVirtualCores(), getErrorMessage("minShareVcores")); + assertEquals( + 0, metrics.getMinShare().getResourceValue(RESOURCE_NAME), getErrorMessage("minShare for resource: " + RESOURCE_NAME)); } @Test @@ -203,30 +203,30 @@ public void testSetMaxShare() { 20L)); metrics.setMaxShare(res); - assertEquals(getErrorMessage("maxShareMB"), - 2048L, metrics.getMaxShareMB()); - assertEquals(getErrorMessage("maxShareVcores"), - 4L, metrics.getMaxShareVirtualCores()); - assertEquals(getErrorMessage("maxShareMB"), - 2048L, metrics.getMaxShare().getMemorySize()); - assertEquals(getErrorMessage("maxShareVcores"), - 4L, metrics.getMaxShare().getVirtualCores()); - assertEquals(getErrorMessage("maxShare for resource: " + RESOURCE_NAME), - 20L, metrics.getMaxShare().getResourceValue(RESOURCE_NAME)); + assertEquals( + 2048L, metrics.getMaxShareMB(), getErrorMessage("maxShareMB")); + assertEquals( + 4L, metrics.getMaxShareVirtualCores(), getErrorMessage("maxShareVcores")); + assertEquals( + 2048L, metrics.getMaxShare().getMemorySize(), getErrorMessage("maxShareMB")); + assertEquals( + 4L, metrics.getMaxShare().getVirtualCores(), getErrorMessage("maxShareVcores")); + assertEquals( + 20L, metrics.getMaxShare().getResourceValue(RESOURCE_NAME), getErrorMessage("maxShare for resource: " + RESOURCE_NAME)); res = Resource.newInstance(2049L, 5); metrics.setMaxShare(res); - assertEquals(getErrorMessage("maxShareMB"), - 2049L, metrics.getMaxShareMB()); - assertEquals(getErrorMessage("maxShareVcores"), - 5L, metrics.getMaxShareVirtualCores()); - assertEquals(getErrorMessage("maxShareMB"), - 2049L, metrics.getMaxShare().getMemorySize()); - assertEquals(getErrorMessage("maxShareVcores"), - 5L, metrics.getMaxShare().getVirtualCores()); - assertEquals(getErrorMessage("maxShare for resource: " + RESOURCE_NAME), - 0, metrics.getMaxShare().getResourceValue(RESOURCE_NAME)); + assertEquals( + 2049L, metrics.getMaxShareMB(), getErrorMessage("maxShareMB")); + assertEquals( + 5L, metrics.getMaxShareVirtualCores(), getErrorMessage("maxShareVcores")); + assertEquals( + 2049L, metrics.getMaxShare().getMemorySize(), getErrorMessage("maxShareMB")); + assertEquals( + 5L, metrics.getMaxShare().getVirtualCores(), getErrorMessage("maxShareVcores")); + assertEquals( + 0, metrics.getMaxShare().getResourceValue(RESOURCE_NAME), getErrorMessage("maxShare for resource: " + RESOURCE_NAME)); } @Test @@ -237,32 +237,32 @@ public void testSetMaxAMShare() { 20L)); metrics.setMaxAMShare(res); - assertEquals(getErrorMessage("maxAMShareMB"), - 2048L, metrics.getMaxAMShareMB()); - assertEquals(getErrorMessage("maxAMShareVcores"), - 4L, metrics.getMaxAMShareVCores()); - assertEquals(getErrorMessage("maxAMShareMB"), - 2048L, metrics.getMaxAMShare().getMemorySize()); - assertEquals(getErrorMessage("maxAMShareVcores"), - 4L, metrics.getMaxAMShare().getVirtualCores()); - assertEquals(getErrorMessage( - "maxAMShare for resource: " + RESOURCE_NAME), - 20L, metrics.getMaxAMShare().getResourceValue(RESOURCE_NAME)); + assertEquals( + 2048L, metrics.getMaxAMShareMB(), getErrorMessage("maxAMShareMB")); + assertEquals( + 4L, metrics.getMaxAMShareVCores(), getErrorMessage("maxAMShareVcores")); + assertEquals(2048L, metrics.getMaxAMShare().getMemorySize(), + getErrorMessage("maxAMShareMB")); + assertEquals( + 4L, metrics.getMaxAMShare().getVirtualCores(), getErrorMessage("maxAMShareVcores")); + assertEquals( + 20L, metrics.getMaxAMShare().getResourceValue(RESOURCE_NAME),getErrorMessage( + "maxAMShare for resource: " + RESOURCE_NAME)); res = Resource.newInstance(2049L, 5); metrics.setMaxAMShare(res); - assertEquals(getErrorMessage("maxAMShareMB"), - 2049L, metrics.getMaxAMShareMB()); - assertEquals(getErrorMessage("maxAMShareVcores"), - 5L, metrics.getMaxAMShareVCores()); - assertEquals(getErrorMessage("maxAMShareMB"), - 2049L, metrics.getMaxAMShare().getMemorySize()); - assertEquals(getErrorMessage("maxAMShareVcores"), - 5L, metrics.getMaxAMShare().getVirtualCores()); - assertEquals(getErrorMessage( - "maxAMShare for resource: " + RESOURCE_NAME), - 0, metrics.getMaxAMShare().getResourceValue(RESOURCE_NAME)); + assertEquals( + 2049L, metrics.getMaxAMShareMB(), getErrorMessage("maxAMShareMB")); + assertEquals( + 5L, metrics.getMaxAMShareVCores(), getErrorMessage("maxAMShareVcores")); + assertEquals( + 2049L, metrics.getMaxAMShare().getMemorySize(), getErrorMessage("maxAMShareMB")); + assertEquals( + 5L, metrics.getMaxAMShare().getVirtualCores(), getErrorMessage("maxAMShareVcores")); + assertEquals( + 0, metrics.getMaxAMShare().getResourceValue(RESOURCE_NAME), getErrorMessage( + "maxAMShare for resource: " + RESOURCE_NAME)); } @Test @@ -273,42 +273,42 @@ public void testSetAMResourceUsage() { 20L)); metrics.setAMResourceUsage(res); - assertEquals(getErrorMessage("AMResourceUsageMB"), - 2048L, metrics.getAMResourceUsageMB()); - assertEquals(getErrorMessage("AMResourceUsageVcores"), - 4L, metrics.getAMResourceUsageVCores()); + assertEquals( + 2048L, metrics.getAMResourceUsageMB(), getErrorMessage("AMResourceUsageMB")); + assertEquals( + 4L, metrics.getAMResourceUsageVCores(), getErrorMessage("AMResourceUsageVcores")); Resource amResourceUsage = metrics.getAMResourceUsage(); - assertEquals(getErrorMessage("AMResourceUsageMB"), - 2048L, amResourceUsage.getMemorySize()); - assertEquals(getErrorMessage("AMResourceUsageVcores"), - 4L, amResourceUsage.getVirtualCores()); - assertEquals(getErrorMessage("AMResourceUsage for resource: " + - RESOURCE_NAME), - 20L, amResourceUsage.getResourceValue(RESOURCE_NAME)); + assertEquals( + 2048L, amResourceUsage.getMemorySize(), getErrorMessage("AMResourceUsageMB")); + assertEquals( + 4L, amResourceUsage.getVirtualCores(), getErrorMessage("AMResourceUsageVcores")); + assertEquals( + 20L, amResourceUsage.getResourceValue(RESOURCE_NAME), getErrorMessage("AMResourceUsage for resource: " + + RESOURCE_NAME)); res = Resource.newInstance(2049L, 5); metrics.setAMResourceUsage(res); - assertEquals(getErrorMessage("AMResourceUsageMB"), - 2049L, metrics.getAMResourceUsageMB()); - assertEquals(getErrorMessage("AMResourceUsageVcores"), - 5L, metrics.getAMResourceUsageVCores()); + assertEquals( + 2049L, metrics.getAMResourceUsageMB(), getErrorMessage("AMResourceUsageMB")); + assertEquals( + 5L, metrics.getAMResourceUsageVCores(), getErrorMessage("AMResourceUsageVcores")); amResourceUsage = metrics.getAMResourceUsage(); - assertEquals(getErrorMessage("AMResourceUsageMB"), - 2049L, amResourceUsage.getMemorySize()); - assertEquals(getErrorMessage("AMResourceUsageVcores"), - 5L, amResourceUsage.getVirtualCores()); - assertEquals(getErrorMessage("AMResourceUsage for resource: " + - RESOURCE_NAME), - 0, amResourceUsage.getResourceValue(RESOURCE_NAME)); + assertEquals( + 2049L, amResourceUsage.getMemorySize(), getErrorMessage("AMResourceUsageMB")); + assertEquals( + 5L, amResourceUsage.getVirtualCores(), getErrorMessage("AMResourceUsageVcores")); + assertEquals( + 0, amResourceUsage.getResourceValue(RESOURCE_NAME), getErrorMessage("AMResourceUsage for resource: " + + RESOURCE_NAME)); } @Test public void testSetMaxApps() { FSQueueMetrics metrics = setupMetrics(RESOURCE_NAME); metrics.setMaxApps(25); - assertEquals(getErrorMessage("maxApps"), 25L, metrics.getMaxApps()); + assertEquals(25L, metrics.getMaxApps(), getErrorMessage("maxApps")); } } 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/fair/TestFSSchedulerNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSSchedulerNode.java index 913ad94fd8237..f08e2cef870d6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSSchedulerNode.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSSchedulerNode.java @@ -21,16 +21,16 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import java.util.ArrayList; import java.util.Collections; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -129,16 +129,16 @@ public Boolean answer(InvocationOnMock invocationOnMock) } private void finalValidation(FSSchedulerNode schedulerNode) { - assertEquals("Everything should have been released", - Resources.none(), schedulerNode.getAllocatedResource()); - assertTrue("No containers should be reserved for preemption", - schedulerNode.containersForPreemption.isEmpty()); - assertTrue("No resources should be reserved for preemptors", - schedulerNode.resourcesPreemptedForApp.isEmpty()); assertEquals( - "No amount of resource should be reserved for preemptees", - Resources.none(), - schedulerNode.getTotalReserved()); + Resources.none(), schedulerNode.getAllocatedResource(), "Everything should have been released"); + assertTrue( + schedulerNode.containersForPreemption.isEmpty(), "No containers should be reserved for preemption"); + assertTrue( + schedulerNode.resourcesPreemptedForApp.isEmpty(), "No resources should be reserved for preemptors"); + assertEquals( + + Resources.none() +, schedulerNode.getTotalReserved(), "No amount of resource should be reserved for preemptees"); } private void allocateContainers(FSSchedulerNode schedulerNode) { @@ -154,15 +154,15 @@ public void testSimpleAllocation() { FSSchedulerNode schedulerNode = new FSSchedulerNode(node, false); createDefaultContainer(); - assertEquals("Nothing should have been allocated, yet", - Resources.none(), schedulerNode.getAllocatedResource()); + assertEquals( + Resources.none(), schedulerNode.getAllocatedResource(), "Nothing should have been allocated, yet"); schedulerNode.allocateContainer(containers.get(0)); - assertEquals("Container should be allocated", - containers.get(0).getContainer().getResource(), - schedulerNode.getAllocatedResource()); + assertEquals( + containers.get(0).getContainer().getResource() +, schedulerNode.getAllocatedResource(), "Container should be allocated"); schedulerNode.releaseContainer(containers.get(0).getContainerId(), true); - assertEquals("Everything should have been released", - Resources.none(), schedulerNode.getAllocatedResource()); + assertEquals( + Resources.none(), schedulerNode.getAllocatedResource(), "Everything should have been released"); // Check that we are error prone schedulerNode.releaseContainer(containers.get(0).getContainerId(), true); @@ -180,16 +180,16 @@ public void testMultipleAllocations() { createDefaultContainer(); createDefaultContainer(); createDefaultContainer(); - assertEquals("Nothing should have been allocated, yet", - Resources.none(), schedulerNode.getAllocatedResource()); + assertEquals( + Resources.none(), schedulerNode.getAllocatedResource(), "Nothing should have been allocated, yet"); schedulerNode.allocateContainer(containers.get(0)); schedulerNode.containerStarted(containers.get(0).getContainerId()); schedulerNode.allocateContainer(containers.get(1)); schedulerNode.containerStarted(containers.get(1).getContainerId()); schedulerNode.allocateContainer(containers.get(2)); - assertEquals("Container should be allocated", - Resources.multiply(containers.get(0).getContainer().getResource(), 3.0), - schedulerNode.getAllocatedResource()); + assertEquals( + Resources.multiply(containers.get(0).getContainer().getResource(), 3.0) +, schedulerNode.getAllocatedResource(), "Container should be allocated"); schedulerNode.releaseContainer(containers.get(1).getContainerId(), true); schedulerNode.releaseContainer(containers.get(2).getContainerId(), true); schedulerNode.releaseContainer(containers.get(0).getContainerId(), true); @@ -206,10 +206,10 @@ public void testSimplePreemption() { // Launch containers and saturate the cluster saturateCluster(schedulerNode); - assertEquals("Container should be allocated", - Resources.multiply(containers.get(0).getContainer().getResource(), - containers.size()), - schedulerNode.getAllocatedResource()); + assertEquals( + Resources.multiply(containers.get(0).getContainer().getResource(), + containers.size()) +, schedulerNode.getAllocatedResource(), "Container should be allocated"); // Request preemption FSAppAttempt starvingApp = createStarvingApp(schedulerNode, @@ -217,16 +217,16 @@ public void testSimplePreemption() { schedulerNode.addContainersForPreemption( Collections.singletonList(containers.get(0)), starvingApp); assertEquals( - "No resource amount should be reserved for preemptees", - containers.get(0).getAllocatedResource(), - schedulerNode.getTotalReserved()); + + containers.get(0).getAllocatedResource() +, schedulerNode.getTotalReserved(), "No resource amount should be reserved for preemptees"); // Preemption occurs release one container schedulerNode.releaseContainer(containers.get(0).getContainerId(), true); allocateContainers(schedulerNode); - assertEquals("Container should be allocated", - schedulerNode.getTotalResource(), - schedulerNode.getAllocatedResource()); + assertEquals( + schedulerNode.getTotalResource() +, schedulerNode.getAllocatedResource(), "Container should be allocated"); // Release all remaining containers for (int i = 1; i < containers.size(); ++i) { @@ -245,10 +245,10 @@ public void testDuplicatePreemption() { // Launch containers and saturate the cluster saturateCluster(schedulerNode); - assertEquals("Container should be allocated", - Resources.multiply(containers.get(0).getContainer().getResource(), - containers.size()), - schedulerNode.getAllocatedResource()); + assertEquals( + Resources.multiply(containers.get(0).getContainer().getResource(), + containers.size()) +, schedulerNode.getAllocatedResource(), "Container should be allocated"); // Request preemption twice FSAppAttempt starvingApp = createStarvingApp(schedulerNode, @@ -258,16 +258,16 @@ public void testDuplicatePreemption() { schedulerNode.addContainersForPreemption( Collections.singletonList(containers.get(0)), starvingApp); assertEquals( - "No resource amount should be reserved for preemptees", - containers.get(0).getAllocatedResource(), - schedulerNode.getTotalReserved()); + + containers.get(0).getAllocatedResource() +, schedulerNode.getTotalReserved(), "No resource amount should be reserved for preemptees"); // Preemption occurs release one container schedulerNode.releaseContainer(containers.get(0).getContainerId(), true); allocateContainers(schedulerNode); - assertEquals("Container should be allocated", - schedulerNode.getTotalResource(), - schedulerNode.getAllocatedResource()); + assertEquals( + schedulerNode.getTotalResource() +, schedulerNode.getAllocatedResource(), "Container should be allocated"); // Release all remaining containers for (int i = 1; i < containers.size(); ++i) { @@ -286,10 +286,10 @@ public void testComplexPreemption() { // Launch containers and saturate the cluster saturateCluster(schedulerNode); - assertEquals("Container should be allocated", - Resources.multiply(containers.get(0).getContainer().getResource(), - containers.size()), - schedulerNode.getAllocatedResource()); + assertEquals( + Resources.multiply(containers.get(0).getContainer().getResource(), + containers.size()) +, schedulerNode.getAllocatedResource(), "Container should be allocated"); // Preempt a container FSAppAttempt starvingApp1 = createStarvingApp(schedulerNode, @@ -311,9 +311,9 @@ public void testComplexPreemption() { schedulerNode.releaseContainer(containers.get(1).getContainerId(), true); allocateContainers(schedulerNode); - assertEquals("Container should be allocated", - schedulerNode.getTotalResource(), - schedulerNode.getAllocatedResource()); + assertEquals( + schedulerNode.getTotalResource() +, schedulerNode.getAllocatedResource(), "Container should be allocated"); // Release all containers for (int i = 3; i < containers.size(); ++i) { @@ -332,10 +332,10 @@ public void testMultiplePreemptionEvents() { // Launch containers and saturate the cluster saturateCluster(schedulerNode); - assertEquals("Container should be allocated", - Resources.multiply(containers.get(0).getContainer().getResource(), - containers.size()), - schedulerNode.getAllocatedResource()); + assertEquals( + Resources.multiply(containers.get(0).getContainer().getResource(), + containers.size()) +, schedulerNode.getAllocatedResource(), "Container should be allocated"); // Preempt a container FSAppAttempt starvingApp1 = createStarvingApp(schedulerNode, @@ -359,9 +359,9 @@ public void testMultiplePreemptionEvents() { schedulerNode.releaseContainer(containers.get(0).getContainerId(), true); allocateContainers(schedulerNode); - assertEquals("Container should be allocated", - schedulerNode.getTotalResource(), - schedulerNode.getAllocatedResource()); + assertEquals( + schedulerNode.getTotalResource() +, schedulerNode.getAllocatedResource(), "Container should be allocated"); // Release all containers for (int i = 3; i < containers.size(); ++i) { @@ -380,10 +380,10 @@ public void testPreemptionToCompletedApp() { // Launch containers and saturate the cluster saturateCluster(schedulerNode); - assertEquals("Container should be allocated", - Resources.multiply(containers.get(0).getContainer().getResource(), - containers.size()), - schedulerNode.getAllocatedResource()); + assertEquals( + Resources.multiply(containers.get(0).getContainer().getResource(), + containers.size()) +, schedulerNode.getAllocatedResource(), "Container should be allocated"); // Preempt a container FSAppAttempt starvingApp = createStarvingApp(schedulerNode, @@ -398,9 +398,9 @@ public void testPreemptionToCompletedApp() { // the deleted app when(starvingApp.isStopped()).thenReturn(true); allocateContainers(schedulerNode); - assertNotEquals("Container should be allocated", - schedulerNode.getTotalResource(), - schedulerNode.getAllocatedResource()); + assertNotEquals( + schedulerNode.getTotalResource() +, schedulerNode.getAllocatedResource(), "Container should be allocated"); // Release all containers for (int i = 1; i < containers.size(); ++i) { @@ -419,10 +419,10 @@ public void testPartialReservedPreemption() { // Launch containers and saturate the cluster saturateCluster(schedulerNode); - assertEquals("Container should be allocated", - Resources.multiply(containers.get(0).getContainer().getResource(), - containers.size()), - schedulerNode.getAllocatedResource()); + assertEquals( + Resources.multiply(containers.get(0).getContainer().getResource(), + containers.size()) +, schedulerNode.getAllocatedResource(), "Container should be allocated"); // Preempt a container Resource originalStarvingAppDemand = Resource.newInstance(512, 1); @@ -436,10 +436,10 @@ public void testPartialReservedPreemption() { // Container partially reassigned allocateContainers(schedulerNode); - assertEquals("Container should be allocated", - Resources.subtract(schedulerNode.getTotalResource(), - Resource.newInstance(512, 0)), - schedulerNode.getAllocatedResource()); + assertEquals( + Resources.subtract(schedulerNode.getTotalResource(), + Resource.newInstance(512, 0)) +, schedulerNode.getAllocatedResource(), "Container should be allocated"); // Cleanup simulating node update schedulerNode.getPreemptionList(); 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/fair/TestFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java index 30fb894be98f0..d43720197ca9b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java @@ -107,10 +107,11 @@ import org.apache.hadoop.yarn.util.ControlledClock; 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.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.mockito.Mockito; import java.io.File; @@ -128,13 +129,7 @@ import static org.apache.hadoop.yarn.server.resourcemanager.MockNM.createMockNodeStatus; import static org.assertj.core.api.Assertions.assertThat; import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.*; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -146,7 +141,7 @@ public class TestFairScheduler extends FairSchedulerTestBase { private final static String ALLOC_FILE = new File(TEST_DIR, "test-queues").getAbsolutePath(); - @Before + @BeforeEach public void setUp() throws IOException { DefaultMetricsSystem.setMiniClusterMode(true); scheduler = new FairScheduler(); @@ -162,7 +157,7 @@ public void setUp() throws IOException { scheduler.setRMContext(resourceManager.getRMContext()); } - @After + @AfterEach public void tearDown() { if (scheduler != null) { scheduler.stop(); @@ -178,7 +173,8 @@ public void tearDown() { } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testConfValidation() throws Exception { Configuration conf = new YarnConfiguration(); conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 2048); @@ -189,9 +185,9 @@ public void testConfValidation() throws Exception { " larger than the max memory allocation."); } catch (YarnRuntimeException e) { // Exception is expected. - assertTrue("The thrown exception is not the expected one.", - e.getMessage().startsWith( - "Invalid resource scheduler memory")); + assertTrue( + e.getMessage().startsWith( + "Invalid resource scheduler memory"), "The thrown exception is not the expected one."); } conf = new YarnConfiguration(); @@ -203,14 +199,15 @@ public void testConfValidation() throws Exception { " larger than the max vcores allocation."); } catch (YarnRuntimeException e) { // Exception is expected. - assertTrue("The thrown exception is not the expected one.", - e.getMessage().startsWith( - "Invalid resource scheduler vcores")); + assertTrue( + e.getMessage().startsWith( + "Invalid resource scheduler vcores"), "The thrown exception is not the expected one."); } } @SuppressWarnings("deprecation") - @Test(timeout=2000) + @Test + @Timeout(value = 2) public void testLoadConfigurationOnInitialize() throws IOException { conf.setBoolean(FairSchedulerConfiguration.ASSIGN_MULTIPLE, true); conf.setInt(FairSchedulerConfiguration.MAX_ASSIGN, 3); @@ -233,19 +230,19 @@ public void testLoadConfigurationOnInitialize() throws IOException { scheduler.init(conf); scheduler.start(); scheduler.reinitialize(conf, resourceManager.getRMContext()); - Assert.assertEquals(true, scheduler.assignMultiple); - Assert.assertEquals(3, scheduler.maxAssign); - Assert.assertEquals(true, scheduler.sizeBasedWeight); - Assert.assertEquals(.5, scheduler.nodeLocalityThreshold, .01); - Assert.assertEquals(.7, scheduler.rackLocalityThreshold, .01); - Assert.assertTrue("The continuous scheduling should be enabled", - scheduler.continuousSchedulingEnabled); - Assert.assertEquals(10, scheduler.continuousSchedulingSleepMs); - Assert.assertEquals(5000, scheduler.nodeLocalityDelayMs); - Assert.assertEquals(5000, scheduler.rackLocalityDelayMs); - Assert.assertEquals(1024, scheduler.getMaximumResourceCapability().getMemorySize()); - Assert.assertEquals(512, scheduler.getMinimumResourceCapability().getMemorySize()); - Assert.assertEquals(128, + Assertions.assertEquals(true, scheduler.assignMultiple); + Assertions.assertEquals(3, scheduler.maxAssign); + Assertions.assertEquals(true, scheduler.sizeBasedWeight); + Assertions.assertEquals(.5, scheduler.nodeLocalityThreshold, .01); + Assertions.assertEquals(.7, scheduler.rackLocalityThreshold, .01); + Assertions.assertTrue( + scheduler.continuousSchedulingEnabled, "The continuous scheduling should be enabled"); + Assertions.assertEquals(10, scheduler.continuousSchedulingSleepMs); + Assertions.assertEquals(5000, scheduler.nodeLocalityDelayMs); + Assertions.assertEquals(5000, scheduler.rackLocalityDelayMs); + Assertions.assertEquals(1024, scheduler.getMaximumResourceCapability().getMemorySize()); + Assertions.assertEquals(512, scheduler.getMinimumResourceCapability().getMemorySize()); + Assertions.assertEquals(128, scheduler.getIncrementResourceCapability().getMemorySize()); } @@ -261,10 +258,10 @@ public void testNonMinZeroResourcesSettings() throws IOException { ResourceUtils.resetResourceTypes(conf); scheduler.init(conf); scheduler.reinitialize(conf, null); - Assert.assertEquals(256, scheduler.getMinimumResourceCapability().getMemorySize()); - Assert.assertEquals(1, scheduler.getMinimumResourceCapability().getVirtualCores()); - Assert.assertEquals(512, scheduler.getIncrementResourceCapability().getMemorySize()); - Assert.assertEquals(2, scheduler.getIncrementResourceCapability().getVirtualCores()); + Assertions.assertEquals(256, scheduler.getMinimumResourceCapability().getMemorySize()); + Assertions.assertEquals(1, scheduler.getMinimumResourceCapability().getVirtualCores()); + Assertions.assertEquals(512, scheduler.getIncrementResourceCapability().getMemorySize()); + Assertions.assertEquals(2, scheduler.getIncrementResourceCapability().getVirtualCores()); } @Test @@ -279,10 +276,10 @@ public void testMinZeroResourcesSettings() throws IOException { ResourceUtils.resetResourceTypes(conf); scheduler.init(conf); scheduler.reinitialize(conf, null); - Assert.assertEquals(0, scheduler.getMinimumResourceCapability().getMemorySize()); - Assert.assertEquals(0, scheduler.getMinimumResourceCapability().getVirtualCores()); - Assert.assertEquals(512, scheduler.getIncrementResourceCapability().getMemorySize()); - Assert.assertEquals(2, scheduler.getIncrementResourceCapability().getVirtualCores()); + Assertions.assertEquals(0, scheduler.getMinimumResourceCapability().getMemorySize()); + Assertions.assertEquals(0, scheduler.getMinimumResourceCapability().getVirtualCores()); + Assertions.assertEquals(512, scheduler.getIncrementResourceCapability().getMemorySize()); + Assertions.assertEquals(2, scheduler.getIncrementResourceCapability().getVirtualCores()); } @Test @@ -368,28 +365,28 @@ public void testQueueMaximumCapacityAllocations() throws IOException { scheduler.init(conf); - Assert.assertEquals(1, scheduler.getMaximumResourceCapability("root.queueA") + Assertions.assertEquals(1, scheduler.getMaximumResourceCapability("root.queueA") .getVirtualCores()); - Assert.assertEquals(512, + Assertions.assertEquals(512, scheduler.getMaximumResourceCapability("root.queueA").getMemorySize()); - Assert.assertEquals(DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES, + Assertions.assertEquals(DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES, scheduler.getMaximumResourceCapability("root.queueB") .getVirtualCores()); - Assert.assertEquals(RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE, + Assertions.assertEquals(RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE, scheduler.getMaximumResourceCapability("root.queueB").getMemorySize()); - Assert.assertEquals(3, scheduler.getMaximumResourceCapability("root.queueC") + Assertions.assertEquals(3, scheduler.getMaximumResourceCapability("root.queueC") .getVirtualCores()); - Assert.assertEquals(2048, + Assertions.assertEquals(2048, scheduler.getMaximumResourceCapability("root.queueC").getMemorySize()); - Assert.assertEquals(3, scheduler + Assertions.assertEquals(3, scheduler .getMaximumResourceCapability("root.queueC.queueD").getVirtualCores()); - Assert.assertEquals(2048, scheduler + Assertions.assertEquals(2048, scheduler .getMaximumResourceCapability("root.queueC.queueD").getMemorySize()); - Assert.assertEquals(RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE, scheduler + Assertions.assertEquals(RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE, scheduler .getMaximumResourceCapability("root.queueE").getMemorySize()); } @@ -486,13 +483,13 @@ public void testFairShareWithHighMaxResources() throws IOException { FSLeafQueue queue = scheduler.getQueueManager().getLeafQueue( "queueA", false); // queueA's weight is 0.25, so its fair share should be 2 * 1024. - assertEquals("Queue A did not get its expected fair share", - 2 * 1024, queue.getFairShare().getMemorySize()); + assertEquals( + 2 * 1024, queue.getFairShare().getMemorySize(), "Queue A did not get its expected fair share"); // queueB's weight is 0.75, so its fair share should be 6 * 1024. queue = scheduler.getQueueManager().getLeafQueue( "queueB", false); - assertEquals("Queue B did not get its expected fair share", - 6 * 1024, queue.getFairShare().getMemorySize()); + assertEquals( + 6 * 1024, queue.getFairShare().getMemorySize(), "Queue B did not get its expected fair share"); } /** @@ -540,15 +537,15 @@ public void testFairShareWithLowMaxResources() throws IOException { scheduler.getQueueManager().getLeafQueue("queueA", false); // queueA's weight is 0.5, so its fair share should be 6GB, but it's // capped at 1GB. - assertEquals("Queue A did not get its expected fair share", - 1 * 1024, queue.getFairShare().getMemorySize()); + assertEquals( + 1 * 1024, queue.getFairShare().getMemorySize(), "Queue A did not get its expected fair share"); // queueB's weight is 0.5, so its fair share should be 2GB, but the // other queue is capped at 1GB, so queueB's share is 7GB, // capped at 3GB. queue = scheduler.getQueueManager().getLeafQueue( "queueB", false); - assertEquals("Queue B did not get its expected fair share", - 3 * 1024, queue.getFairShare().getMemorySize()); + assertEquals( + 3 * 1024, queue.getFairShare().getMemorySize(), "Queue B did not get its expected fair share"); NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1); scheduler.handle(updateEvent); @@ -559,11 +556,11 @@ public void testFairShareWithLowMaxResources() throws IOException { scheduler.handle(updateEvent); // App 1 should be running with 1 container - assertEquals("App 1 is not running with the correct number of containers", - 1, scheduler.getSchedulerApp(attId1).getLiveContainers().size()); + assertEquals( + 1, scheduler.getSchedulerApp(attId1).getLiveContainers().size(), "App 1 is not running with the correct number of containers"); // App 2 should be running with 3 containers - assertEquals("App 2 is not running with the correct number of containers", - 3, scheduler.getSchedulerApp(attId2).getLiveContainers().size()); + assertEquals( + 3, scheduler.getSchedulerApp(attId2).getLiveContainers().size(), "App 2 is not running with the correct number of containers"); } /** @@ -612,20 +609,20 @@ public void testChildMaxResources() throws IOException { drainEventsOnRM(); // Apps should be running with 2 containers - assertEquals("App 1 is not running with the correct number of containers", - 2, scheduler.getSchedulerApp(attId1).getLiveContainers().size()); - assertEquals("App 2 is not running with the correct number of containers", - 2, scheduler.getSchedulerApp(attId2).getLiveContainers().size()); + assertEquals( + 2, scheduler.getSchedulerApp(attId1).getLiveContainers().size(), "App 1 is not running with the correct number of containers"); + assertEquals( + 2, scheduler.getSchedulerApp(attId2).getLiveContainers().size(), "App 2 is not running with the correct number of containers"); //ensure that a 5th node heartbeat does not allocate more containers scheduler.handle(nodeEvent); drainEventsOnRM(); // Apps should be running with 2 containers - assertEquals("App 1 is not running with the correct number of containers", - 2, scheduler.getSchedulerApp(attId1).getLiveContainers().size()); - assertEquals("App 2 is not running with the correct number of containers", - 2, scheduler.getSchedulerApp(attId2).getLiveContainers().size()); + assertEquals( + 2, scheduler.getSchedulerApp(attId1).getLiveContainers().size(), "App 1 is not running with the correct number of containers"); + assertEquals( + 2, scheduler.getSchedulerApp(attId2).getLiveContainers().size(), "App 2 is not running with the correct number of containers"); AllocationFileWriter.create() .addQueue(new AllocationFileQueue.Builder("queueA") @@ -646,10 +643,10 @@ public void testChildMaxResources() throws IOException { drainEventsOnRM(); // Apps should be running with 3 containers now - assertEquals("App 1 is not running with the correct number of containers", - 3, scheduler.getSchedulerApp(attId1).getLiveContainers().size()); - assertEquals("App 2 is not running with the correct number of containers", - 3, scheduler.getSchedulerApp(attId2).getLiveContainers().size()); + assertEquals( + 3, scheduler.getSchedulerApp(attId1).getLiveContainers().size(), "App 1 is not running with the correct number of containers"); + assertEquals( + 3, scheduler.getSchedulerApp(attId2).getLiveContainers().size(), "App 2 is not running with the correct number of containers"); AllocationFileWriter.create() .addQueue(new AllocationFileQueue.Builder("queueA") @@ -670,10 +667,10 @@ public void testChildMaxResources() throws IOException { drainEventsOnRM(); // Apps still should be running with 3 containers because we don't preempt - assertEquals("App 1 is not running with the correct number of containers", - 3, scheduler.getSchedulerApp(attId1).getLiveContainers().size()); - assertEquals("App 2 is not running with the correct number of containers", - 3, scheduler.getSchedulerApp(attId2).getLiveContainers().size()); + assertEquals( + 3, scheduler.getSchedulerApp(attId1).getLiveContainers().size(), "App 1 is not running with the correct number of containers"); + assertEquals( + 3, scheduler.getSchedulerApp(attId2).getLiveContainers().size(), "App 2 is not running with the correct number of containers"); } private void drainEventsOnRM() { @@ -778,20 +775,20 @@ public void testComputeMaxAMResource() throws IOException { // queueFSZeroWithMax's weight is 0.0, so its fair share should be 0, we use // the min(maxShare, available resource) to compute maxAMShare, in this // case, we use maxShare, since it is smaller than available resource. - assertEquals("QueueFSZeroWithMax's fair share should be zero", - 0, queueFSZeroWithMax.getFairShare().getMemorySize()); + assertEquals( + 0, queueFSZeroWithMax.getFairShare().getMemorySize(), "QueueFSZeroWithMax's fair share should be zero"); Resource expectedAMResource = Resources.multiplyAndRoundUp( queueFSZeroWithMax.getMaxShare(), queueFSZeroWithMax.getMaxAMShare()); - assertEquals("QueueFSZeroWithMax's maximum AM resource should be " - + "maxShare * maxAMShare", expectedAMResource.getMemorySize(), - queueFSZeroWithMax.getMetrics().getMaxAMShareMB()); - assertEquals("QueueFSZeroWithMax's maximum AM resource should be " - + "maxShare * maxAMShare", expectedAMResource.getVirtualCores(), - queueFSZeroWithMax.getMetrics().getMaxAMShareVCores()); - assertEquals("QueueFSZeroWithMax's AM resource usage should be the same to " - + "AM resource request", - amResource.getMemorySize(), - queueFSZeroWithMax.getMetrics().getAMResourceUsageMB()); + assertEquals(expectedAMResource.getMemorySize() +, queueFSZeroWithMax.getMetrics().getMaxAMShareMB(), "QueueFSZeroWithMax's maximum AM resource should be " + + "maxShare * maxAMShare"); + assertEquals(expectedAMResource.getVirtualCores() +, queueFSZeroWithMax.getMetrics().getMaxAMShareVCores(), "QueueFSZeroWithMax's maximum AM resource should be " + + "maxShare * maxAMShare"); + assertEquals( + amResource.getMemorySize() +, queueFSZeroWithMax.getMetrics().getAMResourceUsageMB(), "QueueFSZeroWithMax's AM resource usage should be the same to " + + "AM resource request"); // queueFSZeroWithAVL amResource = Resources.createResource(1 * GB, 1); @@ -812,20 +809,20 @@ public void testComputeMaxAMResource() throws IOException { Resources.createResource(memCapacity - amResource.getMemorySize(), cpuCapacity - amResource.getVirtualCores()), queueFSZeroWithAVL.getMaxAMShare()); - assertEquals("QueueFSZeroWithAVL's fair share should be zero", - 0, queueFSZeroWithAVL.getFairShare().getMemorySize()); - assertEquals("QueueFSZeroWithAVL's maximum AM resource should be " - + " available resource * maxAMShare", - expectedAMResource.getMemorySize(), - queueFSZeroWithAVL.getMetrics().getMaxAMShareMB()); - assertEquals("QueueFSZeroWithAVL's maximum AM resource should be " - + " available resource * maxAMShare", - expectedAMResource.getVirtualCores(), - queueFSZeroWithAVL.getMetrics().getMaxAMShareVCores()); - assertEquals("QueueFSZeroWithMax's AM resource usage should be the same to " - + "AM resource request", - amResource.getMemorySize(), - queueFSZeroWithAVL.getMetrics().getAMResourceUsageMB()); + assertEquals( + 0, queueFSZeroWithAVL.getFairShare().getMemorySize(), "QueueFSZeroWithAVL's fair share should be zero"); + assertEquals( + expectedAMResource.getMemorySize() +, queueFSZeroWithAVL.getMetrics().getMaxAMShareMB(), "QueueFSZeroWithAVL's maximum AM resource should be " + + " available resource * maxAMShare"); + assertEquals( + expectedAMResource.getVirtualCores() +, queueFSZeroWithAVL.getMetrics().getMaxAMShareVCores(), "QueueFSZeroWithAVL's maximum AM resource should be " + + " available resource * maxAMShare"); + assertEquals( + amResource.getMemorySize() +, queueFSZeroWithAVL.getMetrics().getAMResourceUsageMB(), "QueueFSZeroWithMax's AM resource usage should be the same to " + + "AM resource request"); // queueFSNonZero amResource = Resources.createResource(1 * GB, 1); @@ -840,20 +837,20 @@ public void testComputeMaxAMResource() throws IOException { // queueFSNonZero's weight is 1, so its fair share is not 0, and we use the // fair share to compute maxAMShare - assertNotEquals("QueueFSNonZero's fair share shouldn't be zero", - 0, queueFSNonZero.getFairShare().getMemorySize()); + assertNotEquals( + 0, queueFSNonZero.getFairShare().getMemorySize(), "QueueFSNonZero's fair share shouldn't be zero"); expectedAMResource = Resources.multiplyAndRoundUp( queueFSNonZero.getFairShare(), queueFSNonZero.getMaxAMShare()); - assertEquals("QueueFSNonZero's maximum AM resource should be " - + " fair share * maxAMShare", expectedAMResource.getMemorySize(), - queueFSNonZero.getMetrics().getMaxAMShareMB()); - assertEquals("QueueFSNonZero's maximum AM resource should be " - + " fair share * maxAMShare", expectedAMResource.getVirtualCores(), - queueFSNonZero.getMetrics().getMaxAMShareVCores()); - assertEquals("QueueFSNonZero's AM resource usage should be the same to " - + "AM resource request", - amResource.getMemorySize(), - queueFSNonZero.getMetrics().getAMResourceUsageMB()); + assertEquals(expectedAMResource.getMemorySize() +, queueFSNonZero.getMetrics().getMaxAMShareMB(), "QueueFSNonZero's maximum AM resource should be " + + " fair share * maxAMShare"); + assertEquals(expectedAMResource.getVirtualCores() +, queueFSNonZero.getMetrics().getMaxAMShareVCores(), "QueueFSNonZero's maximum AM resource should be " + + " fair share * maxAMShare"); + assertEquals( + amResource.getMemorySize() +, queueFSNonZero.getMetrics().getAMResourceUsageMB(), "QueueFSNonZero's AM resource usage should be the same to " + + "AM resource request"); } @Test @@ -984,16 +981,16 @@ public void testQueueInfo() throws IOException { // because the sum of all active Queues' weight are 1. // Before NodeUpdate Event, CurrentCapacity should be 0 QueueInfo queueInfo = scheduler.getQueueInfo("queueA", false, false); - Assert.assertEquals(0.25f, queueInfo.getCapacity(), 0.0f); - Assert.assertEquals(0.0f, queueInfo.getCurrentCapacity(), 0.0f); + Assertions.assertEquals(0.25f, queueInfo.getCapacity(), 0.0f); + Assertions.assertEquals(0.0f, queueInfo.getCurrentCapacity(), 0.0f); // test queueMetrics - Assert.assertEquals(0, queueInfo.getQueueStatistics() + Assertions.assertEquals(0, queueInfo.getQueueStatistics() .getAllocatedContainers()); - Assert.assertEquals(0, queueInfo.getQueueStatistics() + Assertions.assertEquals(0, queueInfo.getQueueStatistics() .getAllocatedMemoryMB()); queueInfo = scheduler.getQueueInfo("queueB", false, false); - Assert.assertEquals(0.75f, queueInfo.getCapacity(), 0.0f); - Assert.assertEquals(0.0f, queueInfo.getCurrentCapacity(), 0.0f); + Assertions.assertEquals(0.75f, queueInfo.getCapacity(), 0.0f); + Assertions.assertEquals(0.0f, queueInfo.getCurrentCapacity(), 0.0f); // Each NodeUpdate Event will only assign one container. // To assign two containers, call handle NodeUpdate Event twice. @@ -1004,20 +1001,20 @@ public void testQueueInfo() throws IOException { // After NodeUpdate Event, CurrentCapacity for queueA should be 1/2=0.5 // and CurrentCapacity for queueB should be 6/6=1. queueInfo = scheduler.getQueueInfo("queueA", false, false); - Assert.assertEquals(0.25f, queueInfo.getCapacity(), 0.0f); - Assert.assertEquals(0.5f, queueInfo.getCurrentCapacity(), 0.0f); + Assertions.assertEquals(0.25f, queueInfo.getCapacity(), 0.0f); + Assertions.assertEquals(0.5f, queueInfo.getCurrentCapacity(), 0.0f); // test queueMetrics - Assert.assertEquals(1, queueInfo.getQueueStatistics() + Assertions.assertEquals(1, queueInfo.getQueueStatistics() .getAllocatedContainers()); - Assert.assertEquals(1024, queueInfo.getQueueStatistics() + Assertions.assertEquals(1024, queueInfo.getQueueStatistics() .getAllocatedMemoryMB()); queueInfo = scheduler.getQueueInfo("queueB", false, false); - Assert.assertEquals(0.75f, queueInfo.getCapacity(), 0.0f); - Assert.assertEquals(1.0f, queueInfo.getCurrentCapacity(), 0.0f); + Assertions.assertEquals(0.75f, queueInfo.getCapacity(), 0.0f); + Assertions.assertEquals(1.0f, queueInfo.getCurrentCapacity(), 0.0f); // test queueMetrics - Assert.assertEquals(1, queueInfo.getQueueStatistics() + Assertions.assertEquals(1, queueInfo.getQueueStatistics() .getAllocatedContainers()); - Assert.assertEquals(6144, queueInfo.getQueueStatistics() + Assertions.assertEquals(6144, queueInfo.getQueueStatistics() .getAllocatedMemoryMB()); } @@ -1074,22 +1071,22 @@ public void testHierarchicalQueuesSimilarParents() throws IOException { QueueManager queueManager = scheduler.getQueueManager(); FSLeafQueue leafQueue = queueManager.getLeafQueue("parent.child", true); - Assert.assertEquals(1, queueManager.getLeafQueues().size()); - Assert.assertNotNull(leafQueue); - Assert.assertEquals("root.parent.child", leafQueue.getName()); + Assertions.assertEquals(1, queueManager.getLeafQueues().size()); + Assertions.assertNotNull(leafQueue); + Assertions.assertEquals("root.parent.child", leafQueue.getName()); FSLeafQueue leafQueue2 = queueManager.getLeafQueue("parent", true); - Assert.assertNull(leafQueue2); - Assert.assertEquals(1, queueManager.getLeafQueues().size()); + Assertions.assertNull(leafQueue2); + Assertions.assertEquals(1, queueManager.getLeafQueues().size()); FSLeafQueue leafQueue3 = queueManager.getLeafQueue("parent.child.grandchild", true); - Assert.assertNull(leafQueue3); - Assert.assertEquals(1, queueManager.getLeafQueues().size()); + Assertions.assertNull(leafQueue3); + Assertions.assertEquals(1, queueManager.getLeafQueues().size()); FSLeafQueue leafQueue4 = queueManager.getLeafQueue("parent.sister", true); - Assert.assertNotNull(leafQueue4); - Assert.assertEquals("root.parent.sister", leafQueue4.getName()); - Assert.assertEquals(2, queueManager.getLeafQueues().size()); + Assertions.assertNotNull(leafQueue4); + Assertions.assertEquals("root.parent.sister", leafQueue4.getName()); + Assertions.assertEquals(2, queueManager.getLeafQueues().size()); } @Test @@ -1133,7 +1130,8 @@ public void testSchedulerRootQueueMetrics() throws Exception { assertEquals(0, scheduler.rootMetrics.getReservedMB()); } - @Test (timeout = 5000) + @Test + @Timeout(value = 5) public void testSimpleContainerAllocation() throws IOException { scheduler.init(conf); scheduler.start(); @@ -1184,7 +1182,8 @@ public void testSimpleContainerAllocation() throws IOException { assertEquals(4, scheduler.getRootQueueMetrics().getAvailableVirtualCores()); } - @Test (timeout = 5000) + @Test + @Timeout(value = 5) public void testSimpleContainerReservation() throws Exception { scheduler.init(conf); scheduler.start(); @@ -1240,7 +1239,8 @@ public void testSimpleContainerReservation() throws Exception { } - @Test (timeout = 5000) + @Test + @Timeout(value = 5) public void testOffSwitchAppReservationThreshold() throws Exception { conf.setFloat(FairSchedulerConfiguration.RESERVABLE_NODES, 0.50f); scheduler.init(conf); @@ -1324,7 +1324,8 @@ public void testOffSwitchAppReservationThreshold() throws Exception { scheduler.getSchedulerApp(attId).getNumReservations(null, true)); } - @Test (timeout = 5000) + @Test + @Timeout(value = 5) public void testRackLocalAppReservationThreshold() throws Exception { conf.setFloat(FairSchedulerConfiguration.RESERVABLE_NODES, 0.50f); scheduler.init(conf); @@ -1434,7 +1435,8 @@ public void testRackLocalAppReservationThreshold() throws Exception { scheduler.getSchedulerApp(attId).getNumReservations(null, true)); } - @Test (timeout = 5000) + @Test + @Timeout(value = 5) public void testReservationThresholdWithAssignMultiple() throws Exception { // set reservable-nodes to 0 which make reservation exceed conf.setFloat(FairSchedulerConfiguration.RESERVABLE_NODES, 0f); @@ -1472,7 +1474,8 @@ public void testReservationThresholdWithAssignMultiple() throws Exception { scheduler.getSchedulerApp(attId).getNumReservations(null, true)); } - @Test (timeout = 500000) + @Test + @Timeout(value = 500) public void testContainerReservationAttemptExceedingQueueMax() throws Exception { conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); @@ -1576,8 +1579,8 @@ public void testRequestAMResourceInZeroFairShareQueue() throws Exception { FSAppAttempt app1 = scheduler.getSchedulerApp(attId1); scheduler.update(); scheduler.handle(updateEvent); - assertEquals("Application 1 should not be running", - 0, app1.getLiveContainers().size()); + assertEquals( + 0, app1.getLiveContainers().size(), "Application 1 should not be running"); // A managed AM which need 2G memory will get resource, // since it request no more than the maxAMShare (4G * 0.5 = 2G). @@ -1587,8 +1590,8 @@ public void testRequestAMResourceInZeroFairShareQueue() throws Exception { FSAppAttempt app2 = scheduler.getSchedulerApp(attId2); scheduler.update(); scheduler.handle(updateEvent); - assertEquals("Application 2 should be running", - 1, app2.getLiveContainers().size()); + assertEquals( + 1, app2.getLiveContainers().size(), "Application 2 should be running"); // A managed AM which need 1G memory will get resource, even thought its // fair share is 0 because its weight is tiny(0.000001). @@ -1598,12 +1601,13 @@ public void testRequestAMResourceInZeroFairShareQueue() throws Exception { FSAppAttempt app3 = scheduler.getSchedulerApp(attId3); scheduler.update(); scheduler.handle(updateEvent); - assertEquals("Application 3 should be running", - 1, app3.getLiveContainers().size()); + assertEquals( + 1, app3.getLiveContainers().size(), "Application 3 should be running"); } - @Test (timeout = 500000) - public void testContainerReservationNotExceedingQueueMax() throws Exception { + @Test + @Timeout(value = 500) + public void testContainerReservationNotExceedingQueueMax() throws Exception { conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); AllocationFileWriter.create() @@ -2144,12 +2148,12 @@ public void testHierarchicalQueueAllocationFileParsing() throws IOException { QueueManager queueManager = scheduler.getQueueManager(); Collection leafQueues = queueManager.getLeafQueues(); - Assert.assertEquals(3, leafQueues.size()); - Assert.assertNotNull(queueManager.getLeafQueue("queueA", false)); - Assert.assertNotNull(queueManager.getLeafQueue("queueB.queueC", false)); - Assert.assertNotNull(queueManager.getLeafQueue("queueB.queueD", false)); + Assertions.assertEquals(3, leafQueues.size()); + Assertions.assertNotNull(queueManager.getLeafQueue("queueA", false)); + Assertions.assertNotNull(queueManager.getLeafQueue("queueB.queueC", false)); + Assertions.assertNotNull(queueManager.getLeafQueue("queueB.queueD", false)); // Make sure querying for queues didn't create any new ones: - Assert.assertEquals(3, leafQueues.size()); + Assertions.assertEquals(3, leafQueues.size()); } @Test @@ -2189,7 +2193,8 @@ public void testConfigureRootQueue() throws Exception { assertEquals(0.5f, root.getFairSharePreemptionThreshold(), 0.01); } - @Test(timeout = 5000) + @Test + @Timeout(value = 5) public void testMultipleContainersWaitingForReservation() throws IOException { scheduler.init(conf); scheduler.start(); @@ -2221,7 +2226,8 @@ public void testMultipleContainersWaitingForReservation() throws IOException { scheduler.getSchedulerApp(attId2).getCurrentReservation().getMemorySize()); } - @Test (timeout = 5000) + @Test + @Timeout(value = 5) public void testUserMaxRunningApps() throws Exception { // Set max running apps conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); @@ -2272,7 +2278,8 @@ public void testUserMaxRunningApps() throws Exception { assertEquals(2, scheduler.getSchedulerApp(attId1).getLiveContainers().size()); } - @Test (timeout = 5000) + @Test + @Timeout(value = 5) public void testIncreaseQueueMaxRunningAppsOnTheFly() throws Exception { AllocationFileWriter allocBefore = AllocationFileWriter.create() .addQueue(new AllocationFileQueue.Builder("root") @@ -2293,7 +2300,8 @@ public void testIncreaseQueueMaxRunningAppsOnTheFly() throws Exception { testIncreaseQueueSettingOnTheFlyInternal(allocBefore, allocAfter); } - @Test (timeout = 5000) + @Test + @Timeout(value = 5) public void testIncreaseUserMaxRunningAppsOnTheFly() throws Exception { AllocationFileWriter allocBefore = AllocationFileWriter.create() .addQueue(new AllocationFileQueue.Builder("root") @@ -2405,7 +2413,8 @@ private void testIncreaseQueueSettingOnTheFlyInternal( assertEquals(1, scheduler.getSchedulerApp(attId4).getLiveContainers().size()); } - @Test (timeout = 5000) + @Test + @Timeout(value = 5) public void testDecreaseQueueMaxRunningAppsOnTheFly() throws Exception { AllocationFileWriter allocBefore = AllocationFileWriter.create() .addQueue(new AllocationFileQueue.Builder("root") @@ -2426,7 +2435,8 @@ public void testDecreaseQueueMaxRunningAppsOnTheFly() throws Exception { testDecreaseQueueSettingOnTheFlyInternal(allocBefore, allocAfter); } - @Test (timeout = 5000) + @Test + @Timeout(value = 5) public void testDecreaseUserMaxRunningAppsOnTheFly() throws Exception { AllocationFileWriter allocBefore = AllocationFileWriter.create() .addQueue(new AllocationFileQueue.Builder("root") @@ -2564,7 +2574,8 @@ private void testDecreaseQueueSettingOnTheFlyInternal( * Reserve at a lower priority and verify the lower priority request gets * allocated */ - @Test (timeout = 5000) + @Test + @Timeout(value = 5) public void testReservationWithMultiplePriorities() throws IOException { scheduler.init(conf); scheduler.start(); @@ -2582,7 +2593,7 @@ public void testReservationWithMultiplePriorities() throws IOException { createSchedulingRequest(1024, 1, "queue", "user", 1)); scheduler.update(); scheduler.handle(updateEvent); - assertEquals("Basic allocation failed", 1, app1.getLiveContainers().size()); + assertEquals(1, app1.getLiveContainers().size(), "Basic allocation failed"); // Create another app and reserve at a lower priority first ApplicationAttemptId attId = @@ -2590,8 +2601,8 @@ public void testReservationWithMultiplePriorities() throws IOException { FSAppAttempt app2 = scheduler.getSchedulerApp(attId); scheduler.update(); scheduler.handle(updateEvent); - assertEquals("Reservation at lower priority failed", - 1, app2.getReservedContainers().size()); + assertEquals( + 1, app2.getReservedContainers().size(), "Reservation at lower priority failed"); // Request container on the second app at a higher priority createSchedulingRequestExistingApplication(2048, 2, 1, attId); @@ -2607,10 +2618,10 @@ public void testReservationWithMultiplePriorities() throws IOException { // Reserved container (at lower priority) should be run Collection liveContainers = app2.getLiveContainers(); - assertEquals("Allocation post completion failed", 1, liveContainers.size()); - assertEquals("High prio container allocated against low prio reservation", - 2, liveContainers.iterator().next().getContainer(). - getPriority().getPriority()); + assertEquals(1, liveContainers.size(), "Allocation post completion failed"); + assertEquals( + 2, liveContainers.iterator().next().getContainer(). + getPriority().getPriority(), "High prio container allocated against low prio reservation"); } @Test @@ -2639,12 +2650,13 @@ public void testAclSubmitApplication() throws Exception { "norealuserhasthisname2", 1); FSAppAttempt app1 = scheduler.getSchedulerApp(attId1); - assertNotNull("The application was not allowed", app1); + assertNotNull(app1, "The application was not allowed"); FSAppAttempt app2 = scheduler.getSchedulerApp(attId2); - assertNull("The application was allowed", app2); + assertNull(app2, "The application was allowed"); } - @Test (timeout = 5000) + @Test + @Timeout(value = 5) public void testMultipleNodesSingleRackRequest() throws Exception { scheduler.init(conf); scheduler.start(); @@ -2704,7 +2716,8 @@ public void testMultipleNodesSingleRackRequest() throws Exception { .size()); } - @Test (timeout = 5000) + @Test + @Timeout(value = 5) public void testFifoWithinQueue() throws Exception { scheduler.init(conf); scheduler.start(); @@ -2749,7 +2762,8 @@ public void testFifoWithinQueue() throws Exception { assertEquals(1, app2.getLiveContainers().size()); } - @Test(timeout = 3000) + @Test + @Timeout(value = 3) public void testFixedMaxAssign() throws Exception { conf.setBoolean(FairSchedulerConfiguration.ASSIGN_MULTIPLE, true); conf.setBoolean(FairSchedulerConfiguration.DYNAMIC_MAX_ASSIGN, false); @@ -2772,15 +2786,15 @@ public void testFixedMaxAssign() throws Exception { scheduler.maxAssign = 2; scheduler.update(); scheduler.handle(updateEvent); - assertEquals("Incorrect number of containers allocated", 2, app - .getLiveContainers().size()); + assertEquals(2, app + .getLiveContainers().size(), "Incorrect number of containers allocated"); // set maxAssign to -1: all remaining containers should be allocated scheduler.maxAssign = -1; scheduler.update(); scheduler.handle(updateEvent); - assertEquals("Incorrect number of containers allocated", 8, app - .getLiveContainers().size()); + assertEquals(8, app + .getLiveContainers().size(), "Incorrect number of containers allocated"); } @@ -2790,7 +2804,8 @@ public void testFixedMaxAssign() throws Exception { * affected. * 2. Verify the node is fully allocated. */ - @Test(timeout = 3000) + @Test + @Timeout(value = 3) public void testDynamicMaxAssign() throws Exception { conf.setBoolean(FairSchedulerConfiguration.ASSIGN_MULTIPLE, true); scheduler.init(conf); @@ -2813,25 +2828,26 @@ public void testDynamicMaxAssign() throws Exception { scheduler.update(); scheduler.handle(updateEvent); // New container allocations should be floor(8/2) + 1 = 5 - assertEquals("Incorrect number of containers allocated", 5, - app.getLiveContainers().size()); + assertEquals(5 +, app.getLiveContainers().size(), "Incorrect number of containers allocated"); // Set maxassign to a value larger than half the remaining resources scheduler.maxAssign = 4; scheduler.update(); scheduler.handle(updateEvent); // New container allocations should be floor(3/2) + 1 = 2 - assertEquals("Incorrect number of containers allocated", 7, - app.getLiveContainers().size()); + assertEquals(7 +, app.getLiveContainers().size(), "Incorrect number of containers allocated"); scheduler.update(); scheduler.handle(updateEvent); // New container allocations should be 1 - assertEquals("Incorrect number of containers allocated", 8, - app.getLiveContainers().size()); + assertEquals(8 +, app.getLiveContainers().size(), "Incorrect number of containers allocated"); } - @Test(timeout = 3000) + @Test + @Timeout(value = 3) public void testMaxAssignWithZeroMemoryContainers() throws Exception { conf.setBoolean(FairSchedulerConfiguration.ASSIGN_MULTIPLE, true); conf.setBoolean(FairSchedulerConfiguration.DYNAMIC_MAX_ASSIGN, false); @@ -2856,15 +2872,15 @@ public void testMaxAssignWithZeroMemoryContainers() throws Exception { scheduler.maxAssign = 2; scheduler.update(); scheduler.handle(updateEvent); - assertEquals("Incorrect number of containers allocated", 2, app - .getLiveContainers().size()); + assertEquals(2, app + .getLiveContainers().size(), "Incorrect number of containers allocated"); // set maxAssign to -1: all remaining containers should be allocated scheduler.maxAssign = -1; scheduler.update(); scheduler.handle(updateEvent); - assertEquals("Incorrect number of containers allocated", 8, app - .getLiveContainers().size()); + assertEquals(8, app + .getLiveContainers().size(), "Incorrect number of containers allocated"); } /** @@ -2880,7 +2896,8 @@ public void testMaxAssignWithZeroMemoryContainers() throws Exception { * * @throws Exception */ - @Test(timeout = 5000) + @Test + @Timeout(value = 5) public void testAssignContainer() throws Exception { scheduler.init(conf); scheduler.start(); @@ -2936,15 +2953,15 @@ public void testAssignContainer() throws Exception { "Wrong number of assigned containers after " + (i + 1) + " updates"; if (i < 4) { // app1 req still not met - assertEquals(ERR, (i + 1), app1.getLiveContainers().size()); - assertEquals(ERR, 0, app4.getLiveContainers().size()); + assertEquals((i + 1), app1.getLiveContainers().size(), ERR); + assertEquals(0, app4.getLiveContainers().size(), ERR); } else { // app1 req has been met, app4 should be served now - assertEquals(ERR, 4, app1.getLiveContainers().size()); - assertEquals(ERR, (i - 3), app4.getLiveContainers().size()); + assertEquals(4, app1.getLiveContainers().size(), ERR); + assertEquals((i - 3), app4.getLiveContainers().size(), ERR); } - assertEquals(ERR, (i + 1) / 2, app2.getLiveContainers().size()); - assertEquals(ERR, (i + 1) / 2, app3.getLiveContainers().size()); + assertEquals((i + 1) / 2, app2.getLiveContainers().size(), ERR); + assertEquals((i + 1) / 2, app3.getLiveContainers().size(), ERR); } } } @@ -3000,8 +3017,8 @@ public void testNotAllowSubmitApplication() throws Exception { } catch (InterruptedException ex) {ex.printStackTrace();} numTries++; } - assertEquals("The application doesn't reach SUBMITTED.", - RMAppState.SUBMITTED, application.getState()); + assertEquals( + RMAppState.SUBMITTED, application.getState(), "The application doesn't reach SUBMITTED."); ApplicationAttemptId attId = ApplicationAttemptId.newInstance(applicationId, this.ATTEMPT_ID++); @@ -3259,11 +3276,11 @@ public void testReservationsStrictLocality() throws IOException { // Heartbeat from node1. App shouldn't get an allocation or reservation NodeUpdateSchedulerEvent nodeUpdateEvent = new NodeUpdateSchedulerEvent(node1); scheduler.handle(nodeUpdateEvent); - assertEquals("App assigned a container on the wrong node", - 0, app.getLiveContainers().size()); + assertEquals( + 0, app.getLiveContainers().size(), "App assigned a container on the wrong node"); scheduler.handle(nodeUpdateEvent); - assertEquals("App reserved a container on the wrong node", - 0, app.getReservedContainers().size()); + assertEquals( + 0, app.getReservedContainers().size(), "App reserved a container on the wrong node"); } @Test @@ -3316,16 +3333,16 @@ public void testBasicDRFAssignment() throws Exception { // 2048/8192 is less than the other's of 2/5 NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node); scheduler.handle(updateEvent); - Assert.assertEquals(1, app1.getLiveContainers().size()); - Assert.assertEquals(0, app2.getLiveContainers().size()); + Assertions.assertEquals(1, app1.getLiveContainers().size()); + Assertions.assertEquals(0, app2.getLiveContainers().size()); scheduler.handle(updateEvent); - Assert.assertEquals(1, app1.getLiveContainers().size()); - Assert.assertEquals(1, app2.getLiveContainers().size()); + Assertions.assertEquals(1, app1.getLiveContainers().size()); + Assertions.assertEquals(1, app2.getLiveContainers().size()); scheduler.handle(updateEvent); - Assert.assertEquals(2, app1.getLiveContainers().size()); - Assert.assertEquals(1, app2.getLiveContainers().size()); + Assertions.assertEquals(2, app1.getLiveContainers().size()); + Assertions.assertEquals(1, app2.getLiveContainers().size()); } /** @@ -3360,13 +3377,13 @@ public void testBasicDRFWithQueues() throws Exception { NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node); scheduler.handle(updateEvent); - Assert.assertEquals(1, app1.getLiveContainers().size()); + Assertions.assertEquals(1, app1.getLiveContainers().size()); scheduler.handle(updateEvent); - Assert.assertEquals(1, app3.getLiveContainers().size()); + Assertions.assertEquals(1, app3.getLiveContainers().size()); scheduler.handle(updateEvent); - Assert.assertEquals(2, app3.getLiveContainers().size()); + Assertions.assertEquals(2, app3.getLiveContainers().size()); scheduler.handle(updateEvent); - Assert.assertEquals(1, app2.getLiveContainers().size()); + Assertions.assertEquals(1, app2.getLiveContainers().size()); } @Test @@ -3407,37 +3424,38 @@ public void testDRFHierarchicalQueues() throws Exception { NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node); scheduler.handle(updateEvent); // app1 gets first container because it asked first - Assert.assertEquals(1, app1.getLiveContainers().size()); + Assertions.assertEquals(1, app1.getLiveContainers().size()); scheduler.handle(updateEvent); // app4 gets second container because it's on queue2 - Assert.assertEquals(1, app4.getLiveContainers().size()); + Assertions.assertEquals(1, app4.getLiveContainers().size()); scheduler.handle(updateEvent); // app4 gets another container because queue2's dominant share of memory // is still less than queue1's of cpu - Assert.assertEquals(2, app4.getLiveContainers().size()); + Assertions.assertEquals(2, app4.getLiveContainers().size()); scheduler.handle(updateEvent); // app3 gets one because queue1 gets one and queue1.subqueue2 is behind // queue1.subqueue1 - Assert.assertEquals(1, app3.getLiveContainers().size()); + Assertions.assertEquals(1, app3.getLiveContainers().size()); scheduler.handle(updateEvent); // app4 would get another one, but it doesn't have any requests // queue1.subqueue2 is still using less than queue1.subqueue1, so it // gets another - Assert.assertEquals(2, app3.getLiveContainers().size()); + Assertions.assertEquals(2, app3.getLiveContainers().size()); // queue1.subqueue1 is behind again, so it gets one, which it gives to app2 scheduler.handle(updateEvent); - Assert.assertEquals(1, app2.getLiveContainers().size()); + Assertions.assertEquals(1, app2.getLiveContainers().size()); // at this point, we've used all our CPU up, so nobody else should get a container scheduler.handle(updateEvent); - Assert.assertEquals(1, app1.getLiveContainers().size()); - Assert.assertEquals(1, app2.getLiveContainers().size()); - Assert.assertEquals(2, app3.getLiveContainers().size()); - Assert.assertEquals(2, app4.getLiveContainers().size()); + Assertions.assertEquals(1, app1.getLiveContainers().size()); + Assertions.assertEquals(1, app2.getLiveContainers().size()); + Assertions.assertEquals(2, app3.getLiveContainers().size()); + Assertions.assertEquals(2, app4.getLiveContainers().size()); } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testHostPortNodeName() throws Exception { conf.setBoolean(YarnConfiguration .RM_SCHEDULER_INCLUDE_PORT_IN_NODE_NAME, true); @@ -3618,8 +3636,8 @@ public void testQueueMaxAMShare() throws Exception { scheduler.update(); FSLeafQueue queue1 = scheduler.getQueueManager().getLeafQueue("queue1", true); - assertEquals("Queue queue1's fair share should be 0", 0, queue1 - .getFairShare().getMemorySize()); + assertEquals(0, queue1 + .getFairShare().getMemorySize(), "Queue queue1's fair share should be 0"); createSchedulingRequest(1 * 1024, "default", "user1"); scheduler.update(); @@ -3636,12 +3654,12 @@ public void testQueueMaxAMShare() throws Exception { FSAppAttempt app1 = scheduler.getSchedulerApp(attId1); scheduler.update(); scheduler.handle(updateEvent); - assertEquals("Application1's AM requests 1024 MB memory", - 1024, app1.getAMResource().getMemorySize()); - assertEquals("Application1's AM should be running", - 1, app1.getLiveContainers().size()); - assertEquals("Queue1's AM resource usage should be 1024 MB memory", - 1024, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 1024, app1.getAMResource().getMemorySize(), "Application1's AM requests 1024 MB memory"); + assertEquals( + 1, app1.getLiveContainers().size(), "Application1's AM should be running"); + assertEquals( + 1024, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 1024 MB memory"); // Exceeds no limits ApplicationAttemptId attId2 = createAppAttemptId(2, 1); @@ -3650,12 +3668,12 @@ public void testQueueMaxAMShare() throws Exception { FSAppAttempt app2 = scheduler.getSchedulerApp(attId2); scheduler.update(); scheduler.handle(updateEvent); - assertEquals("Application2's AM requests 1024 MB memory", - 1024, app2.getAMResource().getMemorySize()); - assertEquals("Application2's AM should be running", - 1, app2.getLiveContainers().size()); - assertEquals("Queue1's AM resource usage should be 2048 MB memory", - 2048, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 1024, app2.getAMResource().getMemorySize(), "Application2's AM requests 1024 MB memory"); + assertEquals( + 1, app2.getLiveContainers().size(), "Application2's AM should be running"); + assertEquals( + 2048, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 2048 MB memory"); // Exceeds queue limit ApplicationAttemptId attId3 = createAppAttemptId(3, 1); @@ -3664,21 +3682,21 @@ public void testQueueMaxAMShare() throws Exception { FSAppAttempt app3 = scheduler.getSchedulerApp(attId3); scheduler.update(); scheduler.handle(updateEvent); - assertEquals("Application3's AM resource shouldn't be updated", - 0, app3.getAMResource().getMemorySize()); - assertEquals("Application3's AM should not be running", - 0, app3.getLiveContainers().size()); - assertEquals("Queue1's AM resource usage should be 2048 MB memory", - 2048, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 0, app3.getAMResource().getMemorySize(), "Application3's AM resource shouldn't be updated"); + assertEquals( + 0, app3.getLiveContainers().size(), "Application3's AM should not be running"); + assertEquals( + 2048, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 2048 MB memory"); // Still can run non-AM container createSchedulingRequestExistingApplication(1024, 1, attId1); scheduler.update(); scheduler.handle(updateEvent); - assertEquals("Application1 should have two running containers", - 2, app1.getLiveContainers().size()); - assertEquals("Queue1's AM resource usage should be 2048 MB memory", - 2048, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 2, app1.getLiveContainers().size(), "Application1 should have two running containers"); + assertEquals( + 2048, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 2048 MB memory"); // Remove app1, app3's AM should become running AppAttemptRemovedSchedulerEvent appRemovedEvent1 = @@ -3686,16 +3704,16 @@ public void testQueueMaxAMShare() throws Exception { scheduler.update(); scheduler.handle(appRemovedEvent1); scheduler.handle(updateEvent); - assertEquals("Application1's AM should be finished", - 0, app1.getLiveContainers().size()); - assertEquals("Finished application usage should be none", - Resources.none(), app1.getResourceUsage()); - assertEquals("Application3's AM should be running", - 1, app3.getLiveContainers().size()); - assertEquals("Application3's AM requests 1024 MB memory", - 1024, app3.getAMResource().getMemorySize()); - assertEquals("Queue1's AM resource usage should be 2048 MB memory", - 2048, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 0, app1.getLiveContainers().size(), "Application1's AM should be finished"); + assertEquals( + Resources.none(), app1.getResourceUsage(), "Finished application usage should be none"); + assertEquals( + 1, app3.getLiveContainers().size(), "Application3's AM should be running"); + assertEquals( + 1024, app3.getAMResource().getMemorySize(), "Application3's AM requests 1024 MB memory"); + assertEquals( + 2048, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 2048 MB memory"); // Exceeds queue limit ApplicationAttemptId attId4 = createAppAttemptId(4, 1); @@ -3704,14 +3722,14 @@ public void testQueueMaxAMShare() throws Exception { FSAppAttempt app4 = scheduler.getSchedulerApp(attId4); scheduler.update(); scheduler.handle(updateEvent); - assertEquals("Application4's AM resource shouldn't be updated", - 0, app4.getAMResource().getMemorySize()); - assertEquals("Application4's AM should not be running", - 0, app4.getLiveContainers().size()); - assertEquals("Finished application usage should be none", - Resources.none(), app4.getResourceUsage()); - assertEquals("Queue1's AM resource usage should be 2048 MB memory", - 2048, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 0, app4.getAMResource().getMemorySize(), "Application4's AM resource shouldn't be updated"); + assertEquals( + 0, app4.getLiveContainers().size(), "Application4's AM should not be running"); + assertEquals( + Resources.none(), app4.getResourceUsage(), "Finished application usage should be none"); + assertEquals( + 2048, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 2048 MB memory"); // Exceeds queue limit ApplicationAttemptId attId5 = createAppAttemptId(5, 1); @@ -3720,14 +3738,14 @@ public void testQueueMaxAMShare() throws Exception { FSAppAttempt app5 = scheduler.getSchedulerApp(attId5); scheduler.update(); scheduler.handle(updateEvent); - assertEquals("Application5's AM resource shouldn't be updated", - 0, app5.getAMResource().getMemorySize()); - assertEquals("Application5's AM should not be running", - 0, app5.getLiveContainers().size()); - assertEquals("Finished application usage should be none", - Resources.none(), app5.getResourceUsage()); - assertEquals("Queue1's AM resource usage should be 2048 MB memory", - 2048, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 0, app5.getAMResource().getMemorySize(), "Application5's AM resource shouldn't be updated"); + assertEquals( + 0, app5.getLiveContainers().size(), "Application5's AM should not be running"); + assertEquals( + Resources.none(), app5.getResourceUsage(), "Finished application usage should be none"); + assertEquals( + 2048, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 2048 MB memory"); // Remove un-running app doesn't affect others AppAttemptRemovedSchedulerEvent appRemovedEvent4 = @@ -3735,12 +3753,12 @@ public void testQueueMaxAMShare() throws Exception { scheduler.handle(appRemovedEvent4); scheduler.update(); scheduler.handle(updateEvent); - assertEquals("Application5's AM should not be running", - 0, app5.getLiveContainers().size()); - assertEquals("Finished application usage should be none", - Resources.none(), app5.getResourceUsage()); - assertEquals("Queue1's AM resource usage should be 2048 MB memory", - 2048, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 0, app5.getLiveContainers().size(), "Application5's AM should not be running"); + assertEquals( + Resources.none(), app5.getResourceUsage(), "Finished application usage should be none"); + assertEquals( + 2048, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 2048 MB memory"); // Remove app2 and app3, app5's AM should become running AppAttemptRemovedSchedulerEvent appRemovedEvent2 = @@ -3751,47 +3769,47 @@ public void testQueueMaxAMShare() throws Exception { scheduler.handle(appRemovedEvent3); scheduler.update(); scheduler.handle(updateEvent); - assertEquals("Application2's AM should be finished", - 0, app2.getLiveContainers().size()); - assertEquals("Finished application usage should be none", - Resources.none(), app2.getResourceUsage()); - assertEquals("Application3's AM should be finished", - 0, app3.getLiveContainers().size()); - assertEquals("Finished application usage should be none", - Resources.none(), app3.getResourceUsage()); - assertEquals("Application5's AM should be running", - 1, app5.getLiveContainers().size()); - assertEquals("Application5's AM requests 2048 MB memory", - 2048, app5.getAMResource().getMemorySize()); - assertEquals("Queue1's AM resource usage should be 2048 MB memory", - 2048, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 0, app2.getLiveContainers().size(), "Application2's AM should be finished"); + assertEquals( + Resources.none(), app2.getResourceUsage(), "Finished application usage should be none"); + assertEquals( + 0, app3.getLiveContainers().size(), "Application3's AM should be finished"); + assertEquals( + Resources.none(), app3.getResourceUsage(), "Finished application usage should be none"); + assertEquals( + 1, app5.getLiveContainers().size(), "Application5's AM should be running"); + assertEquals( + 2048, app5.getAMResource().getMemorySize(), "Application5's AM requests 2048 MB memory"); + assertEquals( + 2048, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 2048 MB memory"); // request non-AM container for app5 createSchedulingRequestExistingApplication(1024, 1, attId5); - assertEquals("Application5's AM should have 1 container", - 1, app5.getLiveContainers().size()); + assertEquals( + 1, app5.getLiveContainers().size(), "Application5's AM should have 1 container"); // complete AM container before non-AM container is allocated. // spark application hit this situation. RMContainer amContainer5 = (RMContainer)app5.getLiveContainers().toArray()[0]; ContainerExpiredSchedulerEvent containerExpired = new ContainerExpiredSchedulerEvent(amContainer5.getContainerId()); scheduler.handle(containerExpired); - assertEquals("Application5's AM should have 0 container", - 0, app5.getLiveContainers().size()); - assertEquals("Finished application usage should be none", - Resources.none(), app5.getResourceUsage()); - assertEquals("Queue1's AM resource usage should be 2048 MB memory", - 2048, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 0, app5.getLiveContainers().size(), "Application5's AM should have 0 container"); + assertEquals( + Resources.none(), app5.getResourceUsage(), "Finished application usage should be none"); + assertEquals( + 2048, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 2048 MB memory"); scheduler.update(); scheduler.handle(updateEvent); // non-AM container should be allocated // check non-AM container allocation is not rejected // due to queue MaxAMShare limitation. - assertEquals("Application5 should have 1 container", - 1, app5.getLiveContainers().size()); + assertEquals( + 1, app5.getLiveContainers().size(), "Application5 should have 1 container"); // check non-AM container allocation won't affect queue AmResourceUsage - assertEquals("Queue1's AM resource usage should be 2048 MB memory", - 2048, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 2048, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 2048 MB memory"); // Check amResource normalization ApplicationAttemptId attId6 = createAppAttemptId(6, 1); @@ -3800,14 +3818,14 @@ public void testQueueMaxAMShare() throws Exception { FSAppAttempt app6 = scheduler.getSchedulerApp(attId6); scheduler.update(); scheduler.handle(updateEvent); - assertEquals("Application6's AM should not be running", - 0, app6.getLiveContainers().size()); - assertEquals("Finished application usage should be none", - Resources.none(), app6.getResourceUsage()); - assertEquals("Application6's AM resource shouldn't be updated", - 0, app6.getAMResource().getMemorySize()); - assertEquals("Queue1's AM resource usage should be 2048 MB memory", - 2048, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 0, app6.getLiveContainers().size(), "Application6's AM should not be running"); + assertEquals( + Resources.none(), app6.getResourceUsage(), "Finished application usage should be none"); + assertEquals( + 0, app6.getAMResource().getMemorySize(), "Application6's AM resource shouldn't be updated"); + assertEquals( + 2048, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 2048 MB memory"); // Remove all apps AppAttemptRemovedSchedulerEvent appRemovedEvent5 = @@ -3817,8 +3835,8 @@ public void testQueueMaxAMShare() throws Exception { scheduler.handle(appRemovedEvent5); scheduler.handle(appRemovedEvent6); scheduler.update(); - assertEquals("Queue1's AM resource usage should be 0", - 0, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 0, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 0"); } @Test @@ -3853,24 +3871,24 @@ public void testQueueMaxAMShareDefault() throws Exception { FSLeafQueue queue1 = scheduler.getQueueManager().getLeafQueue("queue1", true); - assertEquals("Queue queue1's fair share should be 0", 0, queue1 - .getFairShare().getMemorySize()); + assertEquals(0, queue1 + .getFairShare().getMemorySize(), "Queue queue1's fair share should be 0"); FSLeafQueue queue2 = scheduler.getQueueManager().getLeafQueue("queue2", true); - assertEquals("Queue queue2's fair share should be 0", 0, queue2 - .getFairShare().getMemorySize()); + assertEquals(0, queue2 + .getFairShare().getMemorySize(), "Queue queue2's fair share should be 0"); FSLeafQueue queue3 = scheduler.getQueueManager().getLeafQueue("queue3", true); - assertEquals("Queue queue3's fair share should be 0", 0, queue3 - .getFairShare().getMemorySize()); + assertEquals(0, queue3 + .getFairShare().getMemorySize(), "Queue queue3's fair share should be 0"); FSLeafQueue queue4 = scheduler.getQueueManager().getLeafQueue("queue4", true); - assertEquals("Queue queue4's fair share should be 0", 0, queue4 - .getFairShare().getMemorySize()); + assertEquals(0, queue4 + .getFairShare().getMemorySize(), "Queue queue4's fair share should be 0"); FSLeafQueue queue5 = scheduler.getQueueManager().getLeafQueue("queue5", true); - assertEquals("Queue queue5's fair share should be 0", 0, queue5 - .getFairShare().getMemorySize()); + assertEquals(0, queue5 + .getFairShare().getMemorySize(), "Queue queue5's fair share should be 0"); List queues = Arrays.asList("root.queue3", "root.queue4", "root.queue5"); @@ -3891,12 +3909,12 @@ public void testQueueMaxAMShareDefault() throws Exception { FSAppAttempt app1 = scheduler.getSchedulerApp(attId1); scheduler.update(); scheduler.handle(updateEvent); - assertEquals("Application1's AM requests 1024 MB memory", - 1024, app1.getAMResource().getMemorySize()); - assertEquals("Application1's AM should be running", - 1, app1.getLiveContainers().size()); - assertEquals("Queue1's AM resource usage should be 1024 MB memory", - 1024, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 1024, app1.getAMResource().getMemorySize(), "Application1's AM requests 1024 MB memory"); + assertEquals( + 1, app1.getLiveContainers().size(), "Application1's AM should be running"); + assertEquals( + 1024, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 1024 MB memory"); // Now the fair share is 1639 MB, and the maxAMShare is 0.4f, // so the AM is not accepted. @@ -3906,12 +3924,12 @@ public void testQueueMaxAMShareDefault() throws Exception { FSAppAttempt app2 = scheduler.getSchedulerApp(attId2); scheduler.update(); scheduler.handle(updateEvent); - assertEquals("Application2's AM resource shouldn't be updated", - 0, app2.getAMResource().getMemorySize()); - assertEquals("Application2's AM should not be running", - 0, app2.getLiveContainers().size()); - assertEquals("Queue2's AM resource usage should be 0 MB memory", - 0, queue2.getAmResourceUsage().getMemorySize()); + assertEquals( + 0, app2.getAMResource().getMemorySize(), "Application2's AM resource shouldn't be updated"); + assertEquals( + 0, app2.getLiveContainers().size(), "Application2's AM should not be running"); + assertEquals( + 0, queue2.getAmResourceUsage().getMemorySize(), "Queue2's AM resource usage should be 0 MB memory"); // Remove the app2 AppAttemptRemovedSchedulerEvent appRemovedEvent2 = @@ -3928,12 +3946,12 @@ public void testQueueMaxAMShareDefault() throws Exception { FSAppAttempt app3 = scheduler.getSchedulerApp(attId3); scheduler.update(); scheduler.handle(updateEvent); - assertEquals("Application3's AM resource shouldn't be updated", - 0, app3.getAMResource().getMemorySize()); - assertEquals("Application3's AM should not be running", - 0, app3.getLiveContainers().size()); - assertEquals("Queue3's AM resource usage should be 0 MB memory", - 0, queue3.getAmResourceUsage().getMemorySize()); + assertEquals( + 0, app3.getAMResource().getMemorySize(), "Application3's AM resource shouldn't be updated"); + assertEquals( + 0, app3.getLiveContainers().size(), "Application3's AM should not be running"); + assertEquals( + 0, queue3.getAmResourceUsage().getMemorySize(), "Queue3's AM resource usage should be 0 MB memory"); // AM4 can pass the fair share checking and it doesn't takes all // available VCore, but it need 5 VCores which are more than @@ -3944,12 +3962,12 @@ public void testQueueMaxAMShareDefault() throws Exception { FSAppAttempt app4 = scheduler.getSchedulerApp(attId4); scheduler.update(); scheduler.handle(updateEvent); - assertEquals("Application4's AM resource shouldn't be updated", - 0, app4.getAMResource().getMemorySize()); - assertEquals("Application4's AM should not be running", - 0, app4.getLiveContainers().size()); - assertEquals("Queue3's AM resource usage should be 0 MB memory", - 0, queue3.getAmResourceUsage().getMemorySize()); + assertEquals( + 0, app4.getAMResource().getMemorySize(), "Application4's AM resource shouldn't be updated"); + assertEquals( + 0, app4.getLiveContainers().size(), "Application4's AM should not be running"); + assertEquals( + 0, queue3.getAmResourceUsage().getMemorySize(), "Queue3's AM resource usage should be 0 MB memory"); } /** @@ -4028,12 +4046,12 @@ public void testQueueMaxAMShareWithContainerReservation() throws Exception { scheduler.update(); // Allocate app1's AM container on node1. scheduler.handle(updateE1); - assertEquals("Application1's AM requests 1024 MB memory", - 1024, app1.getAMResource().getMemorySize()); - assertEquals("Application1's AM should be running", - 1, app1.getLiveContainers().size()); - assertEquals("Queue1's AM resource usage should be 1024 MB memory", - 1024, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 1024, app1.getAMResource().getMemorySize(), "Application1's AM requests 1024 MB memory"); + assertEquals( + 1, app1.getLiveContainers().size(), "Application1's AM should be running"); + assertEquals( + 1024, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 1024 MB memory"); ApplicationAttemptId attId2 = createAppAttemptId(2, 1); createApplicationWithAMResource(attId2, "queue1", "user1", amResource2); @@ -4042,12 +4060,12 @@ public void testQueueMaxAMShareWithContainerReservation() throws Exception { scheduler.update(); // Allocate app2's AM container on node2. scheduler.handle(updateE2); - assertEquals("Application2's AM requests 1024 MB memory", - 1024, app2.getAMResource().getMemorySize()); - assertEquals("Application2's AM should be running", - 1, app2.getLiveContainers().size()); - assertEquals("Queue1's AM resource usage should be 2048 MB memory", - 2048, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 1024, app2.getAMResource().getMemorySize(), "Application2's AM requests 1024 MB memory"); + assertEquals( + 1, app2.getLiveContainers().size(), "Application2's AM should be running"); + assertEquals( + 2048, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 2048 MB memory"); ApplicationAttemptId attId3 = createAppAttemptId(3, 1); createApplicationWithAMResource(attId3, "queue1", "user1", amResource3); @@ -4060,12 +4078,12 @@ public void testQueueMaxAMShareWithContainerReservation() throws Exception { scheduler.handle(updateE1); // Similarly app3 reserves a container on node2. scheduler.handle(updateE2); - assertEquals("Application3's AM resource shouldn't be updated", - 0, app3.getAMResource().getMemorySize()); - assertEquals("Application3's AM should not be running", - 0, app3.getLiveContainers().size()); - assertEquals("Queue1's AM resource usage should be 2048 MB memory", - 2048, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 0, app3.getAMResource().getMemorySize(), "Application3's AM resource shouldn't be updated"); + assertEquals( + 0, app3.getLiveContainers().size(), "Application3's AM should not be running"); + assertEquals( + 2048, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 2048 MB memory"); ApplicationAttemptId attId4 = createAppAttemptId(4, 1); createApplicationWithAMResource(attId4, "queue1", "user1", amResource4); @@ -4075,30 +4093,30 @@ public void testQueueMaxAMShareWithContainerReservation() throws Exception { // app4 can't allocate its AM container on node1 because // app3 already reserved its container on node1. scheduler.handle(updateE1); - assertEquals("Application4's AM resource shouldn't be updated", - 0, app4.getAMResource().getMemorySize()); - assertEquals("Application4's AM should not be running", - 0, app4.getLiveContainers().size()); - assertEquals("Queue1's AM resource usage should be 2048 MB memory", - 2048, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 0, app4.getAMResource().getMemorySize(), "Application4's AM resource shouldn't be updated"); + assertEquals( + 0, app4.getLiveContainers().size(), "Application4's AM should not be running"); + assertEquals( + 2048, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 2048 MB memory"); scheduler.update(); // Allocate app4's AM container on node3. scheduler.handle(updateE3); - assertEquals("Application4's AM requests 5120 MB memory", - 5120, app4.getAMResource().getMemorySize()); - assertEquals("Application4's AM should be running", - 1, app4.getLiveContainers().size()); - assertEquals("Queue1's AM resource usage should be 7168 MB memory", - 7168, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 5120, app4.getAMResource().getMemorySize(), "Application4's AM requests 5120 MB memory"); + assertEquals( + 1, app4.getLiveContainers().size(), "Application4's AM should be running"); + assertEquals( + 7168, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 7168 MB memory"); AppAttemptRemovedSchedulerEvent appRemovedEvent1 = new AppAttemptRemovedSchedulerEvent(attId1, RMAppAttemptState.FINISHED, false); // Release app1's AM container on node1. scheduler.handle(appRemovedEvent1); - assertEquals("Queue1's AM resource usage should be 6144 MB memory", - 6144, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 6144, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 6144 MB memory"); ApplicationAttemptId attId5 = createAppAttemptId(5, 1); createApplicationWithAMResource(attId5, "queue1", "user1", amResource5); @@ -4109,20 +4127,20 @@ public void testQueueMaxAMShareWithContainerReservation() throws Exception { // app3 unreserve its container on node1 due to // exceeding queue MaxAMShare limit. scheduler.handle(updateE1); - assertEquals("Application5's AM requests 1024 MB memory", - 1024, app5.getAMResource().getMemorySize()); - assertEquals("Application5's AM should be running", - 1, app5.getLiveContainers().size()); - assertEquals("Queue1's AM resource usage should be 7168 MB memory", - 7168, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 1024, app5.getAMResource().getMemorySize(), "Application5's AM requests 1024 MB memory"); + assertEquals( + 1, app5.getLiveContainers().size(), "Application5's AM should be running"); + assertEquals( + 7168, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 7168 MB memory"); AppAttemptRemovedSchedulerEvent appRemovedEvent3 = new AppAttemptRemovedSchedulerEvent(attId3, RMAppAttemptState.FINISHED, false); // Remove app3. scheduler.handle(appRemovedEvent3); - assertEquals("Queue1's AM resource usage should be 7168 MB memory", - 7168, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 7168, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 7168 MB memory"); ApplicationAttemptId attId6 = createAppAttemptId(6, 1); createApplicationWithAMResource(attId6, "queue1", "user1", amResource6); @@ -4133,12 +4151,12 @@ public void testQueueMaxAMShareWithContainerReservation() throws Exception { // app6 can't reserve a container on node1 because // it exceeds queue MaxAMShare limit. scheduler.handle(updateE1); - assertEquals("Application6's AM resource shouldn't be updated", - 0, app6.getAMResource().getMemorySize()); - assertEquals("Application6's AM should not be running", - 0, app6.getLiveContainers().size()); - assertEquals("Queue1's AM resource usage should be 7168 MB memory", - 7168, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 0, app6.getAMResource().getMemorySize(), "Application6's AM resource shouldn't be updated"); + assertEquals( + 0, app6.getLiveContainers().size(), "Application6's AM should not be running"); + assertEquals( + 7168, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 7168 MB memory"); ApplicationAttemptId attId7 = createAppAttemptId(7, 1); createApplicationWithAMResource(attId7, "queue1", "user1", amResource7); @@ -4148,28 +4166,28 @@ public void testQueueMaxAMShareWithContainerReservation() throws Exception { // Allocate app7's AM container on node1 to prove // app6 didn't reserve a container on node1. scheduler.handle(updateE1); - assertEquals("Application7's AM requests 1024 MB memory", - 1024, app7.getAMResource().getMemorySize()); - assertEquals("Application7's AM should be running", - 1, app7.getLiveContainers().size()); - assertEquals("Queue1's AM resource usage should be 8192 MB memory", - 8192, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 1024, app7.getAMResource().getMemorySize(), "Application7's AM requests 1024 MB memory"); + assertEquals( + 1, app7.getLiveContainers().size(), "Application7's AM should be running"); + assertEquals( + 8192, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 8192 MB memory"); AppAttemptRemovedSchedulerEvent appRemovedEvent4 = new AppAttemptRemovedSchedulerEvent(attId4, RMAppAttemptState.FINISHED, false); // Release app4's AM container on node3. scheduler.handle(appRemovedEvent4); - assertEquals("Queue1's AM resource usage should be 3072 MB memory", - 3072, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 3072, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 3072 MB memory"); AppAttemptRemovedSchedulerEvent appRemovedEvent5 = new AppAttemptRemovedSchedulerEvent(attId5, RMAppAttemptState.FINISHED, false); // Release app5's AM container on node1. scheduler.handle(appRemovedEvent5); - assertEquals("Queue1's AM resource usage should be 2048 MB memory", - 2048, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 2048, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 2048 MB memory"); scheduler.update(); // app6 reserves a container on node1 because node1's available resource @@ -4187,53 +4205,53 @@ public void testQueueMaxAMShareWithContainerReservation() throws Exception { // app8 can't allocate a container on node1 because // app6 already reserved a container on node1. scheduler.handle(updateE1); - assertEquals("Application8's AM resource shouldn't be updated", - 0, app8.getAMResource().getMemorySize()); - assertEquals("Application8's AM should not be running", - 0, app8.getLiveContainers().size()); - assertEquals("Queue1's AM resource usage should be 2048 MB memory", - 2048, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 0, app8.getAMResource().getMemorySize(), "Application8's AM resource shouldn't be updated"); + assertEquals( + 0, app8.getLiveContainers().size(), "Application8's AM should not be running"); + assertEquals( + 2048, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 2048 MB memory"); scheduler.update(); // app8 can't allocate a container on node2 because // app6 already reserved a container on node2. scheduler.handle(updateE2); - assertEquals("Application8's AM resource shouldn't be updated", - 0, app8.getAMResource().getMemorySize()); - assertEquals("Application8's AM should not be running", - 0, app8.getLiveContainers().size()); - assertEquals("Queue1's AM resource usage should be 2048 MB memory", - 2048, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 0, app8.getAMResource().getMemorySize(), "Application8's AM resource shouldn't be updated"); + assertEquals( + 0, app8.getLiveContainers().size(), "Application8's AM should not be running"); + assertEquals( + 2048, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 2048 MB memory"); AppAttemptRemovedSchedulerEvent appRemovedEvent2 = new AppAttemptRemovedSchedulerEvent(attId2, RMAppAttemptState.FINISHED, false); // Release app2's AM container on node2. scheduler.handle(appRemovedEvent2); - assertEquals("Queue1's AM resource usage should be 1024 MB memory", - 1024, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 1024, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 1024 MB memory"); scheduler.update(); // app6 turns the reservation into an allocation on node2. scheduler.handle(updateE2); - assertEquals("Application6's AM requests 10240 MB memory", - RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE, - app6.getAMResource().getMemorySize()); - assertEquals("Application6's AM should be running", - 1, app6.getLiveContainers().size()); - assertEquals("Queue1's AM resource usage should be 11264 MB memory", - 11264, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE +, app6.getAMResource().getMemorySize(), "Application6's AM requests 10240 MB memory"); + assertEquals( + 1, app6.getLiveContainers().size(), "Application6's AM should be running"); + assertEquals( + 11264, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 11264 MB memory"); scheduler.update(); // app6 unreserve its container on node1 because // it already got a container on node2. // Now app8 can allocate its AM container on node1. scheduler.handle(updateE1); - assertEquals("Application8's AM requests 1024 MB memory", - 1024, app8.getAMResource().getMemorySize()); - assertEquals("Application8's AM should be running", - 1, app8.getLiveContainers().size()); - assertEquals("Queue1's AM resource usage should be 12288 MB memory", - 12288, queue1.getAmResourceUsage().getMemorySize()); + assertEquals( + 1024, app8.getAMResource().getMemorySize(), "Application8's AM requests 1024 MB memory"); + assertEquals( + 1, app8.getLiveContainers().size(), "Application8's AM should be running"); + assertEquals( + 12288, queue1.getAmResourceUsage().getMemorySize(), "Queue1's AM resource usage should be 12288 MB memory"); } @Test @@ -4315,8 +4333,8 @@ public void testSchedulingOnRemovedNode() throws Exception { // Disable continuous scheduling, will invoke continuous scheduling manually scheduler.init(conf); scheduler.start(); - Assert.assertTrue("Continuous scheduling should be disabled.", - !scheduler.isContinuousSchedulingEnabled()); + Assertions.assertTrue( + !scheduler.isContinuousSchedulingEnabled(), "Continuous scheduling should be disabled."); ApplicationAttemptId id11 = createAppAttemptId(1, 1); createMockRMApp(id11); @@ -4422,8 +4440,8 @@ public void testBlacklistNodes() throws Exception { assertTrue(app.isPlaceBlacklisted(host)); scheduler.update(); scheduler.handle(updateEvent); - assertEquals("Incorrect number of containers allocated", 0, app - .getLiveContainers().size()); + assertEquals(0, app + .getLiveContainers().size(), "Incorrect number of containers allocated"); // Verify a container gets placed on the empty blacklist scheduler.allocate(appAttemptId, update, null, Collections.emptyList(), null, @@ -4432,8 +4450,8 @@ public void testBlacklistNodes() throws Exception { createSchedulingRequest(GB, "root.default", "user", 1); scheduler.update(); scheduler.handle(updateEvent); - assertEquals("Incorrect number of containers allocated", 1, app - .getLiveContainers().size()); + assertEquals(1, app + .getLiveContainers().size(), "Incorrect number of containers allocated"); } @Test @@ -4468,7 +4486,7 @@ public void testGetAppsInQueue() throws Exception { // apps in subqueues should be included apps = scheduler.getAppsInQueue("queue1"); - Assert.assertEquals(2, apps.size()); + Assertions.assertEquals(2, apps.size()); Set appAttIds = Sets.newHashSet(apps.get(0), apps.get(1)); assertTrue(appAttIds.contains(appAttId1)); assertTrue(appAttIds.contains(appAttId2)); @@ -4520,58 +4538,64 @@ public void testResourceUsageByMoveApp() throws Exception { assertThat(queue1.getResourceUsage().getMemorySize()).isEqualTo(0); } - @Test (expected = YarnException.class) + @Test public void testMoveWouldViolateMaxAppsConstraints() throws Exception { - scheduler.init(conf); - scheduler.start(); - scheduler.reinitialize(conf, resourceManager.getRMContext()); + assertThrows(YarnException.class, ()->{ + scheduler.init(conf); + scheduler.start(); + scheduler.reinitialize(conf, resourceManager.getRMContext()); - QueueManager queueMgr = scheduler.getQueueManager(); - FSQueue queue2 = queueMgr.getLeafQueue("queue2", true); - queue2.setMaxRunningApps(0); + QueueManager queueMgr = scheduler.getQueueManager(); + FSQueue queue2 = queueMgr.getLeafQueue("queue2", true); + queue2.setMaxRunningApps(0); - ApplicationAttemptId appAttId = - createSchedulingRequest(1024, 1, "queue1", "user1", 3); + ApplicationAttemptId appAttId = + createSchedulingRequest(1024, 1, "queue1", "user1", 3); - scheduler.moveApplication(appAttId.getApplicationId(), "queue2"); + scheduler.moveApplication(appAttId.getApplicationId(), "queue2"); + }); } - @Test (expected = YarnException.class) + @Test public void testMoveWouldViolateMaxResourcesConstraints() throws Exception { - scheduler.init(conf); - scheduler.start(); - scheduler.reinitialize(conf, resourceManager.getRMContext()); - - QueueManager queueMgr = scheduler.getQueueManager(); - FSLeafQueue oldQueue = queueMgr.getLeafQueue("queue1", true); - FSQueue queue2 = queueMgr.getLeafQueue("queue2", true); - queue2.setMaxShare( - new ConfigurableResource(Resource.newInstance(1024, 1))); + assertThrows(YarnException.class, ()->{ + scheduler.init(conf); + scheduler.start(); + scheduler.reinitialize(conf, resourceManager.getRMContext()); + + QueueManager queueMgr = scheduler.getQueueManager(); + FSLeafQueue oldQueue = queueMgr.getLeafQueue("queue1", true); + FSQueue queue2 = queueMgr.getLeafQueue("queue2", true); + queue2.setMaxShare( + new ConfigurableResource(Resource.newInstance(1024, 1))); + + ApplicationAttemptId appAttId = + createSchedulingRequest(1024, 1, "queue1", "user1", 3); + RMNode node = MockNodes.newNodeInfo(1, Resources.createResource(2048, 2)); + NodeAddedSchedulerEvent nodeEvent = new NodeAddedSchedulerEvent(node); + NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node); + scheduler.handle(nodeEvent); + scheduler.handle(updateEvent); + scheduler.handle(updateEvent); - ApplicationAttemptId appAttId = - createSchedulingRequest(1024, 1, "queue1", "user1", 3); - RMNode node = MockNodes.newNodeInfo(1, Resources.createResource(2048, 2)); - NodeAddedSchedulerEvent nodeEvent = new NodeAddedSchedulerEvent(node); - NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node); - scheduler.handle(nodeEvent); - scheduler.handle(updateEvent); - scheduler.handle(updateEvent); - - assertEquals(Resource.newInstance(2048, 2), oldQueue.getResourceUsage()); - scheduler.moveApplication(appAttId.getApplicationId(), "queue2"); + assertEquals(Resource.newInstance(2048, 2), oldQueue.getResourceUsage()); + scheduler.moveApplication(appAttId.getApplicationId(), "queue2"); + }); } - @Test (expected = YarnException.class) + @Test public void testMoveToNonexistentQueue() throws Exception { - scheduler.init(conf); - scheduler.start(); - scheduler.reinitialize(conf, resourceManager.getRMContext()); + assertThrows(YarnException.class, ()->{ + scheduler.init(conf); + scheduler.start(); + scheduler.reinitialize(conf, resourceManager.getRMContext()); - scheduler.getQueueManager().getLeafQueue("queue1", true); - - ApplicationAttemptId appAttId = - createSchedulingRequest(1024, 1, "queue1", "user1", 3); - scheduler.moveApplication(appAttId.getApplicationId(), "queue2"); + scheduler.getQueueManager().getLeafQueue("queue1", true); + + ApplicationAttemptId appAttId = + createSchedulingRequest(1024, 1, "queue1", "user1", 3); + scheduler.moveApplication(appAttId.getApplicationId(), "queue2"); + }); } @Test @@ -4671,67 +4695,69 @@ public void testDoubleRemoval() throws Exception { // Make sure the app attempt is in the queue. List attemptList = scheduler.getAppsInQueue(testUser); - assertNotNull("Queue missing", attemptList); - assertTrue("Attempt should be in the queue", - attemptList.contains(attemptId)); - assertFalse("Attempt is stopped", attempt.isStopped()); + assertNotNull(attemptList, "Queue missing"); + assertTrue( + attemptList.contains(attemptId), "Attempt should be in the queue"); + assertFalse(attempt.isStopped(), "Attempt is stopped"); // Now remove the app attempt scheduler.handle(attemptRemovedEvent); // The attempt is not in the queue, and stopped attemptList = scheduler.getAppsInQueue(testUser); - assertFalse("Attempt should not be in the queue", - attemptList.contains(attemptId)); - assertTrue("Attempt should have been stopped", attempt.isStopped()); + assertFalse( + attemptList.contains(attemptId), "Attempt should not be in the queue"); + assertTrue(attempt.isStopped(), "Attempt should have been stopped"); // Now remove the app attempt again, since it is stopped nothing happens. scheduler.handle(attemptRemovedEvent); // The attempt should still show the original queue info. - assertTrue("Attempt queue has changed", - attempt.getQueue().getName().endsWith(testUser)); + assertTrue( + attempt.getQueue().getName().endsWith(testUser), "Attempt queue has changed"); } - @Test (expected = YarnException.class) + @Test public void testMoveAfterRemoval() throws Exception { - String testUser = "user1"; // convenience var - scheduler.init(conf); - scheduler.start(); - scheduler.reinitialize(conf, resourceManager.getRMContext()); - - ApplicationAttemptId attemptId = createAppAttemptId(1, 1); - ApplicationPlacementContext apc = - new ApplicationPlacementContext(testUser); - AppAddedSchedulerEvent appAddedEvent = - new AppAddedSchedulerEvent(attemptId.getApplicationId(), testUser, - testUser, apc); - scheduler.handle(appAddedEvent); - AppAttemptAddedSchedulerEvent attemptAddedEvent = - new AppAttemptAddedSchedulerEvent(createAppAttemptId(1, 1), false); - scheduler.handle(attemptAddedEvent); - - // Get a handle on the attempt. - FSAppAttempt attempt = scheduler.getSchedulerApp(attemptId); - - AppAttemptRemovedSchedulerEvent attemptRemovedEvent = - new AppAttemptRemovedSchedulerEvent(createAppAttemptId(1, 1), - RMAppAttemptState.FINISHED, false); - - // Remove the app attempt - scheduler.handle(attemptRemovedEvent); - // Make sure the app attempt is not in the queue and stopped. - List attemptList = - scheduler.getAppsInQueue(testUser); - assertNotNull("Queue missing", attemptList); - assertFalse("Attempt should not be in the queue", - attemptList.contains(attemptId)); - assertTrue("Attempt should have been stopped", attempt.isStopped()); - // The attempt should still show the original queue info. - assertTrue("Attempt queue has changed", - attempt.getQueue().getName().endsWith(testUser)); + assertThrows(YarnException.class, ()->{ + String testUser = "user1"; // convenience var + scheduler.init(conf); + scheduler.start(); + scheduler.reinitialize(conf, resourceManager.getRMContext()); + + ApplicationAttemptId attemptId = createAppAttemptId(1, 1); + ApplicationPlacementContext apc = + new ApplicationPlacementContext(testUser); + AppAddedSchedulerEvent appAddedEvent = + new AppAddedSchedulerEvent(attemptId.getApplicationId(), testUser, + testUser, apc); + scheduler.handle(appAddedEvent); + AppAttemptAddedSchedulerEvent attemptAddedEvent = + new AppAttemptAddedSchedulerEvent(createAppAttemptId(1, 1), false); + scheduler.handle(attemptAddedEvent); + + // Get a handle on the attempt. + FSAppAttempt attempt = scheduler.getSchedulerApp(attemptId); + + AppAttemptRemovedSchedulerEvent attemptRemovedEvent = + new AppAttemptRemovedSchedulerEvent(createAppAttemptId(1, 1), + RMAppAttemptState.FINISHED, false); + + // Remove the app attempt + scheduler.handle(attemptRemovedEvent); + // Make sure the app attempt is not in the queue and stopped. + List attemptList = + scheduler.getAppsInQueue(testUser); + assertNotNull(attemptList, "Queue missing"); + assertFalse( + attemptList.contains(attemptId), "Attempt should not be in the queue"); + assertTrue(attempt.isStopped(), "Attempt should have been stopped"); + // The attempt should still show the original queue info. + assertTrue( + attempt.getQueue().getName().endsWith(testUser), "Attempt queue has changed"); - // Now move the app: not using an event since there is none - // in the scheduler. This should throw. - scheduler.moveApplication(attemptId.getApplicationId(), "default"); + // Now move the app: not using an event since there is none + // in the scheduler. This should throw. + scheduler.moveApplication(attemptId.getApplicationId(), "default"); + }); } @Test @@ -4740,8 +4766,8 @@ public void testPerfMetricsInited() { scheduler.start(); MetricsCollectorImpl collector = new MetricsCollectorImpl(); scheduler.fsOpDurations.getMetrics(collector, true); - assertEquals("Incorrect number of perf metrics", 1, - collector.getRecords().size()); + assertEquals(1 +, collector.getRecords().size(), "Incorrect number of perf metrics"); } @Test @@ -4780,7 +4806,7 @@ public void testQueueNameWithTrailingSpace() throws Exception { appAttemptId2.getApplicationId(), "A ", "user1", apc); try { scheduler.handle(appAddedEvent2); - Assert.fail("Submit should have failed with InvalidQueueNameException"); + Assertions.fail("Submit should have failed with InvalidQueueNameException"); } catch (InvalidQueueNameException iqne) { // expected ignore: rules should have filtered this out } @@ -4836,10 +4862,10 @@ public void testEmptyQueueNameInConfigFile() { try { scheduler.init(conf); - Assert.fail("scheduler init should fail because" + + Assertions.fail("scheduler init should fail because" + " empty queue name."); } catch (Exception e) { - Assert.assertTrue(e.getMessage().contains( + Assertions.assertTrue(e.getMessage().contains( "Failed to initialize FairScheduler")); } } @@ -4954,7 +4980,7 @@ public void handle(Event event) { new NodeUpdateSchedulerEvent(spyNode)); // Check the used resource is 0 GB 0 core - // Assert.assertEquals(1 * GB, nm_0.getUsed().getMemory()); + // Assertions.assertEquals(1 * GB, nm_0.getUsed().getMemory()); Resource usedResource = resourceManager.getResourceScheduler() .getSchedulerNode(nm_0.getNodeId()).getAllocatedResource(); @@ -5001,7 +5027,8 @@ private NodeManager registerNode(String hostName, int containerManagerPort, return nm; } - @Test(timeout = 120000) + @Test + @Timeout(value = 120) public void testContainerAllocationWithContainerIdLeap() throws Exception { conf.setFloat(FairSchedulerConfiguration.RESERVABLE_NODES, 0.50f); scheduler.init(conf); @@ -5072,7 +5099,8 @@ public void testContainerAllocationWithContainerIdLeap() throws Exception { assertEquals(reservedId + 1, maxId); } - @Test(timeout = 120000) + @Test + @Timeout(value = 120) public void testRefreshQueuesWhenRMHA() throws Exception { conf.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false); conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true); @@ -5229,11 +5257,11 @@ public void testUpdateDemand() throws IOException { queue1.updateDemand(); - assertTrue("Demand is greater than max allowed ", - Resources.equals(queue1.getDemand(), maxResource)); - assertTrue("Demand of child queue not updated ", - Resources.equals(aQueue.getDemand(), maxResource) && - Resources.equals(bQueue.getDemand(), maxResource)); + assertTrue( + Resources.equals(queue1.getDemand(), maxResource), "Demand is greater than max allowed "); + assertTrue( + Resources.equals(aQueue.getDemand(), maxResource) && + Resources.equals(bQueue.getDemand(), maxResource), "Demand of child queue not updated "); } @Test @@ -5282,8 +5310,8 @@ public void testDumpState() throws IOException { + " LastTimeAtMinShare: " + clock.getTime() + "}"; - assertEquals("Unexpected state dump string", - childQueueString, child1.dumpState()); + assertEquals( + childQueueString, child1.dumpState(), "Unexpected state dump string"); FSParentQueue parent = scheduler.getQueueManager().getParentQueue("parent", false); parent.setMaxShare(new ConfigurableResource(resource)); @@ -5347,7 +5375,7 @@ public void testCompletedContainerOnRemovedNode() throws IOException { // Get the allocated containers for the application (list can not be null) Collection clist = scheduler.getSchedulerApp(appAttemptId) .getLiveContainers(); - Assert.assertEquals(1, clist.size()); + Assertions.assertEquals(1, clist.size()); // Make sure that we remove the correct node (should never fail) RMContainer rmc = clist.iterator().next(); @@ -5411,19 +5439,19 @@ private void testAppRejectedToQueueWithZeroCapacityOfResource(String resource) Resource.newInstance(GB, 1), Lists.newArrayList(amReqs)); scheduler.update(); - assertEquals("Exactly one APP_REJECTED event is expected", 1, - recordedEvents.size()); + assertEquals(1 +, recordedEvents.size(), "Exactly one APP_REJECTED event is expected"); Event event = recordedEvents.get(0); RMAppEvent rmAppEvent = (RMAppEvent) event; assertEquals(RMAppEventType.APP_REJECTED, rmAppEvent.getType()); - assertTrue("Diagnostic message does not match: " + - rmAppEvent.getDiagnosticMsg(), - rmAppEvent.getDiagnosticMsg() + assertTrue( + rmAppEvent.getDiagnosticMsg() .matches("Cannot submit application application[\\d_]+ to queue " + "root.queueA because it has zero amount of resource " + "for a requested resource! " + "Invalid requested AM resources: .+, " - + "maximum queue resources: .+")); + + "maximum queue resources: .+"), "Diagnostic message does not match: " + + rmAppEvent.getDiagnosticMsg()); } private void generateAllocationFileWithZeroResource(String resource) { @@ -5515,32 +5543,32 @@ private void testSchedulingRejectedToQueueZeroCapacityOfResource( .collect(Collectors.toList())); } catch (SchedulerInvalidResourceRequestException e) { assertTrue( - "The thrown exception is not the expected one. Exception message: " - + e.getMessage(), - e.getMessage() + + e.getMessage() .matches("Resource request is invalid for application " + "application[\\d_]+ because queue root\\.queueA has 0 " + "amount of resource for a resource type! " - + "Validation result:.*")); + + "Validation result:.*"), "The thrown exception is not the expected one. Exception message: " + + e.getMessage()); List appsInQueue = scheduler.getAppsInQueue("queueA"); - assertEquals("Number of apps in queue 'queueA' should be one!", 1, - appsInQueue.size()); + assertEquals(1 +, appsInQueue.size(), "Number of apps in queue 'queueA' should be one!"); ApplicationAttemptId appAttemptId = scheduler.getAppsInQueue("queueA").get(0); assertNotNull( - "Scheduler app for appAttemptId " + appAttemptId - + " should not be null!", - scheduler.getSchedulerApp(appAttemptId)); + + scheduler.getSchedulerApp(appAttemptId), "Scheduler app for appAttemptId " + appAttemptId + + " should not be null!"); FSAppAttempt schedulerApp = scheduler.getSchedulerApp(appAttemptId); - assertNotNull("Scheduler app queueInfo for appAttemptId " + appAttemptId - + " should not be null!", schedulerApp.getAppSchedulingInfo()); + assertNotNull(schedulerApp.getAppSchedulingInfo(), "Scheduler app queueInfo for appAttemptId " + appAttemptId + + " should not be null!"); - assertTrue("There should be no requests accepted", schedulerApp - .getAppSchedulingInfo().getAllResourceRequests().isEmpty()); + assertTrue(schedulerApp + .getAppSchedulingInfo().getAllResourceRequests().isEmpty(), "There should be no requests accepted"); } } @@ -5586,19 +5614,19 @@ public void testRestoreToExistingQueue() throws IOException { List appsInQueue = scheduler.getAppsInQueue(queueId); - assertEquals("Number of apps in queue 'root.parent.queueA' should be one!", - 1, appsInQueue.size()); + assertEquals( + 1, appsInQueue.size(), "Number of apps in queue 'root.parent.queueA' should be one!"); appAttemptId = scheduler.getAppsInQueue(queueId).get(0); - assertNotNull("Scheduler app for appAttemptId " + appAttemptId - + " should not be null!", scheduler.getSchedulerApp(appAttemptId)); + assertNotNull(scheduler.getSchedulerApp(appAttemptId), "Scheduler app for appAttemptId " + appAttemptId + + " should not be null!"); FSAppAttempt schedulerApp = scheduler.getSchedulerApp(appAttemptId); - assertNotNull("Scheduler app queueInfo for appAttemptId " + appAttemptId - + " should not be null!", schedulerApp.getAppSchedulingInfo()); + assertNotNull(schedulerApp.getAppSchedulingInfo(), "Scheduler app queueInfo for appAttemptId " + appAttemptId + + " should not be null!"); - assertTrue("There should be no requests accepted", schedulerApp - .getAppSchedulingInfo().getAllResourceRequests().isEmpty()); + assertTrue(schedulerApp + .getAppSchedulingInfo().getAllResourceRequests().isEmpty(), "There should be no requests accepted"); // Restore an applications with a user that has no access to the queue try { @@ -5611,19 +5639,19 @@ public void testRestoreToExistingQueue() throws IOException { scheduler.addApplicationAttempt(appAttemptId, false, true); appsInQueue = scheduler.getAppsInQueue(queueId); - assertEquals("Number of apps in queue 'root.parent.queueA' should be two!", - 2, appsInQueue.size()); + assertEquals( + 2, appsInQueue.size(), "Number of apps in queue 'root.parent.queueA' should be two!"); appAttemptId = scheduler.getAppsInQueue(queueId).get(1); - assertNotNull("Scheduler app for appAttemptId " + appAttemptId - + " should not be null!", scheduler.getSchedulerApp(appAttemptId)); + assertNotNull(scheduler.getSchedulerApp(appAttemptId), "Scheduler app for appAttemptId " + appAttemptId + + " should not be null!"); schedulerApp = scheduler.getSchedulerApp(appAttemptId); - assertNotNull("Scheduler app queueInfo for appAttemptId " + appAttemptId - + " should not be null!", schedulerApp.getAppSchedulingInfo()); + assertNotNull(schedulerApp.getAppSchedulingInfo(), "Scheduler app queueInfo for appAttemptId " + appAttemptId + + " should not be null!"); - assertTrue("There should be no requests accepted", schedulerApp - .getAppSchedulingInfo().getAllResourceRequests().isEmpty()); + assertTrue(schedulerApp + .getAppSchedulingInfo().getAllResourceRequests().isEmpty(), "There should be no requests accepted"); } @Test @@ -5652,20 +5680,20 @@ public void testRestoreToParentQueue() throws IOException { String recoveredQueue = "root.recovery"; List appsInQueue = scheduler.getAppsInQueue(recoveredQueue); - assertEquals("Number of apps in queue 'root.recovery' should be one!", - 1, appsInQueue.size()); + assertEquals( + 1, appsInQueue.size(), "Number of apps in queue 'root.recovery' should be one!"); appAttemptId = scheduler.getAppsInQueue(recoveredQueue).get(0); - assertNotNull("Scheduler app for appAttemptId " + appAttemptId - + " should not be null!", scheduler.getSchedulerApp(appAttemptId)); + assertNotNull(scheduler.getSchedulerApp(appAttemptId), "Scheduler app for appAttemptId " + appAttemptId + + " should not be null!"); FSAppAttempt schedulerApp = scheduler.getSchedulerApp(appAttemptId); - assertNotNull("Scheduler app queueInfo for appAttemptId " + appAttemptId - + " should not be null!", schedulerApp.getAppSchedulingInfo()); + assertNotNull(schedulerApp.getAppSchedulingInfo(), "Scheduler app queueInfo for appAttemptId " + appAttemptId + + " should not be null!"); - assertTrue("There should be no requests accepted", schedulerApp - .getAppSchedulingInfo().getAllResourceRequests().isEmpty()); + assertTrue(schedulerApp + .getAppSchedulingInfo().getAllResourceRequests().isEmpty(), "There should be no requests accepted"); } private void generateAllocationFilePercentageResource() { 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/fair/TestFairSchedulerConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerConfiguration.java index 38fbcd8415300..943beba9aa351 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerConfiguration.java @@ -34,9 +34,9 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.log4j.spi.LoggingEvent; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import org.junit.Rule; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.junit.rules.ExpectedException; import java.util.Collections; @@ -44,8 +44,7 @@ import java.util.concurrent.CopyOnWriteArrayList; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedulerConfiguration.parseResourceConfigValue; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.*; /** * Tests fair scheduler configuration. @@ -638,13 +637,15 @@ public void testAllocationIncrementMemoryNonDefaultUnit() throws Exception { .getMemorySize()); } - @Test(expected=IllegalArgumentException.class) + @Test public void testAllocationIncrementInvalidUnit() throws Exception { - Configuration conf = new Configuration(); - conf.set(YarnConfiguration.RESOURCE_TYPES + "." + - ResourceInformation.MEMORY_MB.getName() + - FairSchedulerConfiguration.INCREMENT_ALLOCATION, "1 Xi"); - new FairSchedulerConfiguration(conf).getIncrementAllocation(); + assertThrows(IllegalArgumentException.class, ()->{ + Configuration conf = new Configuration(); + conf.set(YarnConfiguration.RESOURCE_TYPES + "." + + ResourceInformation.MEMORY_MB.getName() + + FairSchedulerConfiguration.INCREMENT_ALLOCATION, "1 Xi"); + new FairSchedulerConfiguration(conf).getIncrementAllocation(); + }); } @Test @@ -762,14 +763,14 @@ public void testMemoryIncrementConfiguredViaMultipleProperties() { FairSchedulerConfiguration.INCREMENT_ALLOCATION, "13"); FairSchedulerConfiguration fsc = new FairSchedulerConfiguration(conf); Resource increment = fsc.getIncrementAllocation(); - Assert.assertEquals(13L, increment.getMemorySize()); - assertTrue("Warning message is not logged when specifying memory " + - "increment via multiple properties", - testAppender.getLogEvents().stream().anyMatch( + Assertions.assertEquals(13L, increment.getMemorySize()); + assertTrue( + testAppender.getLogEvents().stream().anyMatch( e -> e.getLevel() == Level.WARN && ("Configuration " + "yarn.resource-types.memory-mb.increment-allocation=13 is " + "overriding the yarn.scheduler.increment-allocation-mb=7 " + - "property").equals(e.getMessage()))); + "property").equals(e.getMessage())), "Warning message is not logged when specifying memory " + + "increment via multiple properties"); } finally { logger.removeAppender(testAppender); } @@ -788,14 +789,14 @@ public void testCpuIncrementConfiguredViaMultipleProperties() { FairSchedulerConfiguration.INCREMENT_ALLOCATION, "13"); FairSchedulerConfiguration fsc = new FairSchedulerConfiguration(conf); Resource increment = fsc.getIncrementAllocation(); - Assert.assertEquals(13, increment.getVirtualCores()); - assertTrue("Warning message is not logged when specifying CPU vCores " + - "increment via multiple properties", - testAppender.getLogEvents().stream().anyMatch( + Assertions.assertEquals(13, increment.getVirtualCores()); + assertTrue( + testAppender.getLogEvents().stream().anyMatch( e -> e.getLevel() == Level.WARN && ("Configuration " + "yarn.resource-types.vcores.increment-allocation=13 is " + "overriding the yarn.scheduler.increment-allocation-vcores=7 " + - "property").equals(e.getMessage()))); + "property").equals(e.getMessage())), "Warning message is not logged when specifying CPU vCores " + + "increment via multiple properties"); } finally { logger.removeAppender(testAppender); } 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/fair/TestFairSchedulerFairShare.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerFairShare.java index db612f23aad5c..ad178249faa55 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerFairShare.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerFairShare.java @@ -18,7 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import java.io.File; import java.io.IOException; @@ -37,21 +37,21 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair .allocationfile.AllocationFileWriter; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.After; -import org.junit.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 TestFairSchedulerFairShare extends FairSchedulerTestBase { private final static String ALLOC_FILE = new File(TEST_DIR, TestFairSchedulerFairShare.class.getName() + ".xml").getAbsolutePath(); - @Before + @BeforeEach public void setup() throws IOException { conf = createConfiguration(); conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); } - @After + @AfterEach public void teardown() { if (resourceManager != null) { resourceManager.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/fair/TestFairSchedulerPreemption.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java index 8d7665a7f5058..db8d73dc53188 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java @@ -30,13 +30,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.allocationfile.AllocationFileWriter; import org.apache.hadoop.yarn.util.ControlledClock; import org.apache.hadoop.yarn.util.SystemClock; -import org.junit.After; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import org.junit.Before; +import org.junit.jupiter.api.AfterEach; +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 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.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -93,7 +93,7 @@ public TestFairSchedulerPreemption(String name, int mode) writeAllocFile(); } - @Before + @BeforeEach public void setup() throws IOException { createConfiguration(); conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, @@ -109,7 +109,7 @@ public void setup() throws IOException { setupCluster(); } - @After + @AfterEach public void teardown() { ALLOC_FILE.delete(); conf = null; @@ -190,8 +190,8 @@ private void writeAllocFile() { } allocationFileWriter.writeToFile(ALLOC_FILE.getAbsolutePath()); - assertTrue("Allocation file does not exist, not running the test", - ALLOC_FILE.exists()); + assertTrue( + ALLOC_FILE.exists(), "Allocation file does not exist, not running the test"); } private void setupCluster() throws IOException { @@ -298,30 +298,30 @@ private void verifyPreemption(int numStarvedAppContainers, } // Post preemption, verify the greedyApp has the correct # of containers. - assertEquals("Incorrect # of containers on the greedy app", - numGreedyAppContainers, greedyApp.getLiveContainers().size()); + assertEquals( + numGreedyAppContainers, greedyApp.getLiveContainers().size(), "Incorrect # of containers on the greedy app"); // Verify the queue metrics are set appropriately. The greedyApp started // with 8 1GB, 1vcore containers. - assertEquals("Incorrect # of preempted containers in QueueMetrics", - 8 - numGreedyAppContainers, - greedyApp.getQueue().getMetrics().getAggregatePreemptedContainers()); + assertEquals( + 8 - numGreedyAppContainers +, greedyApp.getQueue().getMetrics().getAggregatePreemptedContainers(), "Incorrect # of preempted containers in QueueMetrics"); // Verify the node is reserved for the starvingApp for (RMNode rmNode : rmNodes) { FSSchedulerNode node = (FSSchedulerNode) scheduler.getNodeTracker().getNode(rmNode.getNodeID()); if (node.getContainersForPreemption().size() > 0) { - assertTrue("node should be reserved for the starvingApp", - node.getPreemptionList().keySet().contains(starvingApp)); + assertTrue( + node.getPreemptionList().keySet().contains(starvingApp), "node should be reserved for the starvingApp"); } } sendEnoughNodeUpdatesToAssignFully(); // Verify the preempted containers are assigned to starvingApp - assertEquals("Starved app is not assigned the right # of containers", - numStarvedAppContainers, starvingApp.getLiveContainers().size()); + assertEquals( + numStarvedAppContainers, starvingApp.getLiveContainers().size(), "Starved app is not assigned the right # of containers"); // Verify the node is not reserved for the starvingApp anymore for (RMNode rmNode : rmNodes) { @@ -406,8 +406,8 @@ public void testPreemptionSelectNonAMContainer() throws Exception { String host1 = containers.get(1).getNodeId().getHost(); // Each node provides two and only two non-AM containers to be preempted, so // the preemption happens on both nodes. - assertTrue("Preempted containers should come from two different " - + "nodes.", !host0.equals(host1)); + assertTrue(!host0.equals(host1), "Preempted containers should come from two different " + + "nodes."); } @Test 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/fair/TestFairSchedulerUtilities.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerUtilities.java index 37f686e79e3cb..ecf2805e51360 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerUtilities.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerUtilities.java @@ -17,12 +17,12 @@ */ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; -import org.junit.Test; +import org.junit.jupiter.api.Test; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedulerUtilities.trimQueueName; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Tests for {@link FairSchedulerUtilities}. 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/fair/TestFairSchedulerWithMultiResourceTypes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerWithMultiResourceTypes.java index 2785baa87301d..fef70d45629d6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerWithMultiResourceTypes.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerWithMultiResourceTypes.java @@ -24,9 +24,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager; import org.apache.hadoop.yarn.util.resource.ResourceUtils; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.HashMap; @@ -34,7 +34,7 @@ import static org.apache.hadoop.yarn.util.resource.ResourceUtils.MAXIMUM_ALLOCATION; import static org.apache.hadoop.yarn.util.resource.ResourceUtils.UNITS; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -43,7 +43,7 @@ public class TestFairSchedulerWithMultiResourceTypes private static final String CUSTOM_RESOURCE = "custom-resource"; - @Before + @BeforeEach public void setUp() throws IOException { scheduler = new FairScheduler(); conf = createConfiguration(); @@ -55,7 +55,7 @@ public void setUp() throws IOException { scheduler.setRMContext(rmContext); } - @After + @AfterEach public void tearDown() { if (scheduler != null) { scheduler.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/fair/TestMaxRunningAppsEnforcer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestMaxRunningAppsEnforcer.java index 52f929e0fb4a2..20d0440eee87c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestMaxRunningAppsEnforcer.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestMaxRunningAppsEnforcer.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -33,8 +33,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager; import org.apache.hadoop.yarn.util.ControlledClock; import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestMaxRunningAppsEnforcer { private QueueManager queueManager; @@ -45,7 +45,7 @@ public class TestMaxRunningAppsEnforcer { private RMContext rmContext; private FairScheduler scheduler; - @Before + @BeforeEach public void setup() { FairSchedulerConfiguration conf = new FairSchedulerConfiguration(); PlacementManager placementManager = new PlacementManager(); 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/fair/TestQueueManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestQueueManager.java index 3400822074e05..5141e148f1ed5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestQueueManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestQueueManager.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; -import static org.junit.Assert.*; +import static org.junit.jupiter.api.Assertions.*; import static org.mockito.Mockito.*; import java.util.Collections; @@ -32,8 +32,8 @@ import org.apache.hadoop.yarn.util.SystemClock; import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; @@ -46,7 +46,7 @@ public class TestQueueManager { private QueueManager queueManager; private FairScheduler scheduler; - @Before + @BeforeEach public void setUp() { PlacementManager placementManager = new PlacementManager(); FairSchedulerConfiguration conf = new FairSchedulerConfiguration(); @@ -194,8 +194,8 @@ private void updateConfiguredLeafQueues(QueueManager queueMgr, public void testCreateLeafQueue() { FSQueue q1 = queueManager.createQueue("root.queue1", FSQueueType.LEAF); - assertNotNull("Leaf queue root.queue1 was not created", - queueManager.getLeafQueue("root.queue1", false)); + assertNotNull( + queueManager.getLeafQueue("root.queue1", false), "Leaf queue root.queue1 was not created"); assertEquals("createQueue() returned wrong queue", "root.queue1", q1.getName()); } @@ -208,10 +208,10 @@ public void testCreateLeafQueueAndParent() { FSQueue q2 = queueManager.createQueue("root.queue1.queue2", FSQueueType.LEAF); - assertNotNull("Parent queue root.queue1 was not created", - queueManager.getParentQueue("root.queue1", false)); - assertNotNull("Leaf queue root.queue1.queue2 was not created", - queueManager.getLeafQueue("root.queue1.queue2", false)); + assertNotNull( + queueManager.getParentQueue("root.queue1", false), "Parent queue root.queue1 was not created"); + assertNotNull( + queueManager.getLeafQueue("root.queue1.queue2", false), "Leaf queue root.queue1.queue2 was not created"); assertEquals("createQueue() returned wrong queue", "root.queue1.queue2", q2.getName()); } @@ -228,38 +228,38 @@ public void testCreateQueueWithChildDefaults() { new ConfigurableResource(Resources.createResource(8192, 256))); FSQueue q1 = queueManager.createQueue("root.test.childC", FSQueueType.LEAF); - assertNotNull("Leaf queue root.test.childC was not created", - queueManager.getLeafQueue("root.test.childC", false)); + assertNotNull( + queueManager.getLeafQueue("root.test.childC", false), "Leaf queue root.test.childC was not created"); assertEquals("createQueue() returned wrong queue", "root.test.childC", q1.getName()); - assertEquals("Max resources for root.queue1 were not inherited from " - + "parent's max child resources", Resources.createResource(8192, 256), - q1.getMaxShare()); + assertEquals(Resources.createResource(8192, 256) +, q1.getMaxShare(), "Max resources for root.queue1 were not inherited from " + + "parent's max child resources"); FSQueue q2 = queueManager.createQueue("root.test.childD", FSQueueType.PARENT); - assertNotNull("Leaf queue root.test.childD was not created", - queueManager.getParentQueue("root.test.childD", false)); + assertNotNull( + queueManager.getParentQueue("root.test.childD", false), "Leaf queue root.test.childD was not created"); assertEquals("createQueue() returned wrong queue", "root.test.childD", q2.getName()); - assertEquals("Max resources for root.test.childD were not inherited " - + "from parent's max child resources", - Resources.createResource(8192, 256), - q2.getMaxShare()); + assertEquals( + Resources.createResource(8192, 256) +, q2.getMaxShare(), "Max resources for root.test.childD were not inherited " + + "from parent's max child resources"); // Check that the childA and childB queues weren't impacted // by the child defaults - assertNotNull("Leaf queue root.test.childA was not created during setup", - queueManager.getLeafQueue("root.test.childA", false)); - assertEquals("Max resources for root.test.childA were inherited from " - + "parent's max child resources", Resources.unbounded(), - queueManager.getLeafQueue("root.test.childA", false).getMaxShare()); - assertNotNull("Leaf queue root.test.childB was not created during setup", - queueManager.getParentQueue("root.test.childB", false)); - assertEquals("Max resources for root.test.childB were inherited from " - + "parent's max child resources", Resources.unbounded(), - queueManager.getParentQueue("root.test.childB", false).getMaxShare()); + assertNotNull( + queueManager.getLeafQueue("root.test.childA", false), "Leaf queue root.test.childA was not created during setup"); + assertEquals(Resources.unbounded() +, queueManager.getLeafQueue("root.test.childA", false).getMaxShare(), "Max resources for root.test.childA were inherited from " + + "parent's max child resources"); + assertNotNull( + queueManager.getParentQueue("root.test.childB", false), "Leaf queue root.test.childB was not created during setup"); + assertEquals(Resources.unbounded() +, queueManager.getParentQueue("root.test.childB", false).getMaxShare(), "Max resources for root.test.childB were inherited from " + + "parent's max child resources"); } /** @@ -269,19 +269,19 @@ public void testCreateQueueWithChildDefaults() { public void testCreateLeafQueueWithDefaults() { FSQueue q1 = queueManager.createQueue("root.queue1", FSQueueType.LEAF); - assertNotNull("Leaf queue root.queue1 was not created", - queueManager.getLeafQueue("root.queue1", false)); + assertNotNull( + queueManager.getLeafQueue("root.queue1", false), "Leaf queue root.queue1 was not created"); assertEquals("createQueue() returned wrong queue", "root.queue1", q1.getName()); // Min default is 0,0 - assertEquals("Min resources were not set to default", - Resources.createResource(0, 0), - q1.getMinShare()); + assertEquals( + Resources.createResource(0, 0) +, q1.getMinShare(), "Min resources were not set to default"); // Max default is unbounded - assertEquals("Max resources were not set to default", Resources.unbounded(), - q1.getMaxShare()); + assertEquals(Resources.unbounded() +, q1.getMaxShare(), "Max resources were not set to default"); } /** @@ -291,8 +291,8 @@ public void testCreateLeafQueueWithDefaults() { public void testCreateParentQueue() { FSQueue q1 = queueManager.createQueue("root.queue1", FSQueueType.PARENT); - assertNotNull("Parent queue root.queue1 was not created", - queueManager.getParentQueue("root.queue1", false)); + assertNotNull( + queueManager.getParentQueue("root.queue1", false), "Parent queue root.queue1 was not created"); assertEquals("createQueue() returned wrong queue", "root.queue1", q1.getName()); } @@ -305,10 +305,10 @@ public void testCreateParentQueueAndParent() { FSQueue q2 = queueManager.createQueue("root.queue1.queue2", FSQueueType.PARENT); - assertNotNull("Parent queue root.queue1 was not created", - queueManager.getParentQueue("root.queue1", false)); - assertNotNull("Leaf queue root.queue1.queue2 was not created", - queueManager.getParentQueue("root.queue1.queue2", false)); + assertNotNull( + queueManager.getParentQueue("root.queue1", false), "Parent queue root.queue1 was not created"); + assertNotNull( + queueManager.getParentQueue("root.queue1.queue2", false), "Leaf queue root.queue1.queue2 was not created"); assertEquals("createQueue() returned wrong queue", "root.queue1.queue2", q2.getName()); } @@ -321,11 +321,11 @@ public void testRemovalOfDynamicLeafQueue() { FSLeafQueue q1 = queueManager.getLeafQueue("root.test.childB.dynamic1", true); - assertNotNull("Queue root.test.childB.dynamic1 was not created", q1); + assertNotNull(q1, "Queue root.test.childB.dynamic1 was not created"); assertEquals("createQueue() returned wrong queue", "root.test.childB.dynamic1", q1.getName()); - assertTrue("root.test.childB.dynamic1 is not a dynamic queue", - q1.isDynamic()); + assertTrue( + q1.isDynamic(), "root.test.childB.dynamic1 is not a dynamic queue"); // an application is submitted to root.test.childB.dynamic1 ApplicationId appId = ApplicationId.newInstance(0, 0); @@ -335,7 +335,7 @@ public void testRemovalOfDynamicLeafQueue() { queueManager.removePendingIncompatibleQueues(); queueManager.removeEmptyDynamicQueues(); q1 = queueManager.getLeafQueue("root.test.childB.dynamic1", false); - assertNotNull("Queue root.test.childB.dynamic1 was deleted", q1); + assertNotNull(q1, "Queue root.test.childB.dynamic1 was deleted"); // the application finishes, the next removeEmptyDynamicQueues() should // clean root.test.childB.dynamic1 up, but keep its static parent @@ -344,9 +344,9 @@ public void testRemovalOfDynamicLeafQueue() { queueManager.removePendingIncompatibleQueues(); queueManager.removeEmptyDynamicQueues(); q1 = queueManager.getLeafQueue("root.test.childB.dynamic1", false); - assertNull("Queue root.test.childB.dynamic1 was not deleted", q1); - assertNotNull("The static parent of root.test.childB.dynamic1 was deleted", - queueManager.getParentQueue("root.test.childB", false)); + assertNull(q1, "Queue root.test.childB.dynamic1 was not deleted"); + assertNotNull( + queueManager.getParentQueue("root.test.childB", false), "The static parent of root.test.childB.dynamic1 was deleted"); } /** @@ -356,22 +356,22 @@ public void testRemovalOfDynamicLeafQueue() { public void testRemovalOfDynamicParentQueue() { FSQueue q1 = queueManager.getLeafQueue("root.parent1.dynamic1", true); - assertNotNull("Queue root.parent1.dynamic1 was not created", q1); + assertNotNull(q1, "Queue root.parent1.dynamic1 was not created"); assertEquals("createQueue() returned wrong queue", "root.parent1.dynamic1", q1.getName()); - assertTrue("root.parent1.dynamic1 is not a dynamic queue", q1.isDynamic()); + assertTrue(q1.isDynamic(), "root.parent1.dynamic1 is not a dynamic queue"); FSQueue p1 = queueManager.getParentQueue("root.parent1", false); - assertNotNull("Queue root.parent1 was not created", p1); - assertTrue("root.parent1 is not a dynamic queue", p1.isDynamic()); + assertNotNull(p1, "Queue root.parent1 was not created"); + assertTrue(p1.isDynamic(), "root.parent1 is not a dynamic queue"); queueManager.removePendingIncompatibleQueues(); queueManager.removeEmptyDynamicQueues(); q1 = queueManager.getLeafQueue("root.parent1.dynamic1", false); p1 = queueManager.getParentQueue("root.parent1", false); - assertNull("Queue root.parent1.dynamic1 was not deleted", q1); - assertNull("Queue root.parent1 was not deleted", p1); + assertNull(q1, "Queue root.parent1.dynamic1 was not deleted"); + assertNull(p1, "Queue root.parent1 was not deleted"); } /** @@ -381,10 +381,10 @@ public void testRemovalOfDynamicParentQueue() { public void testNonEmptyDynamicQueueBecomingStaticQueue() { FSLeafQueue q1 = queueManager.getLeafQueue("root.leaf1", true); - assertNotNull("Queue root.leaf1 was not created", q1); + assertNotNull(q1, "Queue root.leaf1 was not created"); assertEquals("createQueue() returned wrong queue", "root.leaf1", q1.getName()); - assertTrue("root.leaf1 is not a dynamic queue", q1.isDynamic()); + assertTrue(q1.isDynamic(), "root.leaf1 is not a dynamic queue"); // pretend that we submitted an app to the queue ApplicationId appId = ApplicationId.newInstance(0, 0); @@ -394,7 +394,7 @@ public void testNonEmptyDynamicQueueBecomingStaticQueue() { queueManager.removePendingIncompatibleQueues(); queueManager.removeEmptyDynamicQueues(); q1 = queueManager.getLeafQueue("root.leaf1", false); - assertNotNull("Queue root.leaf1 was deleted", q1); + assertNotNull(q1, "Queue root.leaf1 was deleted"); // next we add leaf1 under root in the allocation config AllocationConfiguration allocConf = scheduler.getAllocationConfiguration(); @@ -402,7 +402,7 @@ public void testNonEmptyDynamicQueueBecomingStaticQueue() { queueManager.updateAllocationConfiguration(allocConf); // updateAllocationConfiguration() should make root.leaf1 a dynamic queue - assertFalse("root.leaf1 is not a static queue", q1.isDynamic()); + assertFalse(q1.isDynamic(), "root.leaf1 is not a static queue"); // application finished now and the queue is empty, but since leaf1 is a // static queue at this point, hence not affected by @@ -411,8 +411,8 @@ public void testNonEmptyDynamicQueueBecomingStaticQueue() { queueManager.removePendingIncompatibleQueues(); queueManager.removeEmptyDynamicQueues(); q1 = queueManager.getLeafQueue("root.leaf1", false); - assertNotNull("Queue root.leaf1 was deleted", q1); - assertFalse("root.leaf1 is not a static queue", q1.isDynamic()); + assertNotNull(q1, "Queue root.leaf1 was deleted"); + assertFalse(q1.isDynamic(), "root.leaf1 is not a static queue"); } /** @@ -422,10 +422,10 @@ public void testNonEmptyDynamicQueueBecomingStaticQueue() { public void testNonEmptyStaticQueueBecomingDynamicQueue() { FSLeafQueue q1 = queueManager.getLeafQueue("root.test.childA", false); - assertNotNull("Queue root.test.childA does not exist", q1); + assertNotNull(q1, "Queue root.test.childA does not exist"); assertEquals("createQueue() returned wrong queue", "root.test.childA", q1.getName()); - assertFalse("root.test.childA is not a static queue", q1.isDynamic()); + assertFalse(q1.isDynamic(), "root.test.childA is not a static queue"); // we submitted an app to the queue ApplicationId appId = ApplicationId.newInstance(0, 0); @@ -436,8 +436,8 @@ public void testNonEmptyStaticQueueBecomingDynamicQueue() { queueManager.removePendingIncompatibleQueues(); queueManager.removeEmptyDynamicQueues(); q1 = queueManager.getLeafQueue("root.test.childA", false); - assertNotNull("Queue root.test.childA was deleted", q1); - assertFalse("root.test.childA is not a dynamic queue", q1.isDynamic()); + assertNotNull(q1, "Queue root.test.childA was deleted"); + assertFalse(q1.isDynamic(), "root.test.childA is not a dynamic queue"); // next we remove all queues from the allocation config, // this causes all queues to change to dynamic @@ -449,8 +449,8 @@ public void testNonEmptyStaticQueueBecomingDynamicQueue() { queueManager.updateAllocationConfiguration(allocConf); q1 = queueManager.getLeafQueue("root.test.childA", false); - assertNotNull("Queue root.test.childA was deleted", q1); - assertTrue("root.test.childA is not a dynamic queue", q1.isDynamic()); + assertNotNull(q1, "Queue root.test.childA was deleted"); + assertTrue(q1.isDynamic(), "root.test.childA is not a dynamic queue"); // application finished - the queue does not have runnable app // the next removeEmptyDynamicQueues() call should remove the queues @@ -460,10 +460,10 @@ public void testNonEmptyStaticQueueBecomingDynamicQueue() { queueManager.removeEmptyDynamicQueues(); q1 = queueManager.getLeafQueue("root.test.childA", false); - assertNull("Queue root.test.childA was not deleted", q1); + assertNull(q1, "Queue root.test.childA was not deleted"); FSParentQueue p1 = queueManager.getParentQueue("root.test", false); - assertNull("Queue root.test was not deleted", p1); + assertNull(p1, "Queue root.test was not deleted"); } /** @@ -473,16 +473,16 @@ public void testNonEmptyStaticQueueBecomingDynamicQueue() { public void testRemovalOfChildlessParentQueue() { FSParentQueue q1 = queueManager.getParentQueue("root.test.childB", false); - assertNotNull("Queue root.test.childB was not created", q1); + assertNotNull(q1, "Queue root.test.childB was not created"); assertEquals("createQueue() returned wrong queue", "root.test.childB", q1.getName()); - assertFalse("root.test.childB is a dynamic queue", q1.isDynamic()); + assertFalse(q1.isDynamic(), "root.test.childB is a dynamic queue"); // static queues should not be deleted queueManager.removePendingIncompatibleQueues(); queueManager.removeEmptyDynamicQueues(); q1 = queueManager.getParentQueue("root.test.childB", false); - assertNotNull("Queue root.test.childB was deleted", q1); + assertNotNull(q1, "Queue root.test.childB was deleted"); // next we remove root.test.childB from the allocation config AllocationConfiguration allocConf = scheduler.getAllocationConfiguration(); @@ -496,7 +496,7 @@ public void testRemovalOfChildlessParentQueue() { queueManager.removePendingIncompatibleQueues(); queueManager.removeEmptyDynamicQueues(); q1 = queueManager.getParentQueue("root.leaf1", false); - assertNull("Queue root.leaf1 was not deleted", q1); + assertNull(q1, "Queue root.leaf1 was not deleted"); } /** @@ -506,14 +506,14 @@ public void testRemovalOfChildlessParentQueue() { @Test public void testQueueTypeChange() { FSQueue q1 = queueManager.getLeafQueue("root.parent1.leaf1", true); - assertNotNull("Queue root.parent1.leaf1 was not created", q1); + assertNotNull(q1, "Queue root.parent1.leaf1 was not created"); assertEquals("createQueue() returned wrong queue", "root.parent1.leaf1", q1.getName()); - assertTrue("root.parent1.leaf1 is not a dynamic queue", q1.isDynamic()); + assertTrue(q1.isDynamic(), "root.parent1.leaf1 is not a dynamic queue"); FSQueue p1 = queueManager.getParentQueue("root.parent1", false); - assertNotNull("Queue root.parent1 was not created", p1); - assertTrue("root.parent1 is not a dynamic queue", p1.isDynamic()); + assertNotNull(p1, "Queue root.parent1 was not created"); + assertTrue(p1.isDynamic(), "root.parent1 is not a dynamic queue"); // adding root.parent1.leaf1 and root.parent1 to the allocation config AllocationConfiguration allocConf = scheduler.getAllocationConfiguration(); @@ -524,9 +524,9 @@ public void testQueueTypeChange() { // updateAllocationConfiguration() should change both queues over to static queueManager.updateAllocationConfiguration(allocConf); q1 = queueManager.getLeafQueue("root.parent1.leaf1", false); - assertFalse("root.parent1.leaf1 is not a static queue", q1.isDynamic()); + assertFalse(q1.isDynamic(), "root.parent1.leaf1 is not a static queue"); p1 = queueManager.getParentQueue("root.parent1", false); - assertFalse("root.parent1 is not a static queue", p1.isDynamic()); + assertFalse(p1.isDynamic(), "root.parent1 is not a static queue"); // removing root.parent1.leaf1 and root.parent1 from the allocation // config @@ -540,9 +540,9 @@ public void testQueueTypeChange() { queueManager.setQueuesToDynamic( ImmutableSet.of("root.parent1", "root.parent1.leaf1")); q1 = queueManager.getLeafQueue("root.parent1.leaf1", false); - assertTrue("root.parent1.leaf1 is not a dynamic queue", q1.isDynamic()); + assertTrue(q1.isDynamic(), "root.parent1.leaf1 is not a dynamic queue"); p1 = queueManager.getParentQueue("root.parent1", false); - assertTrue("root.parent1 is not a dynamic queue", p1.isDynamic()); + assertTrue(p1.isDynamic(), "root.parent1 is not a dynamic queue"); } /** @@ -551,21 +551,21 @@ public void testQueueTypeChange() { @Test public void testApplicationAssignmentPreventsRemovalOfDynamicQueue() { FSLeafQueue q = queueManager.getLeafQueue("root.leaf1", true); - assertNotNull("root.leaf1 does not exist", q); - assertTrue("root.leaf1 is not empty", q.isEmpty()); + assertNotNull(q, "root.leaf1 does not exist"); + assertTrue(q.isEmpty(), "root.leaf1 is not empty"); // assigning an application (without an appAttempt so far) to the queue // removeEmptyDynamicQueues() should not remove the queue ApplicationId applicationId = ApplicationId.newInstance(1L, 0); q.addAssignedApp(applicationId); q = queueManager.getLeafQueue("root.leaf1", false); - assertFalse("root.leaf1 is empty", q.isEmpty()); + assertFalse(q.isEmpty(), "root.leaf1 is empty"); queueManager.removePendingIncompatibleQueues(); queueManager.removeEmptyDynamicQueues(); q = queueManager.getLeafQueue("root.leaf1", false); - assertNotNull("root.leaf1 has been removed", q); - assertFalse("root.leaf1 is empty", q.isEmpty()); + assertNotNull(q, "root.leaf1 has been removed"); + assertFalse(q.isEmpty(), "root.leaf1 is empty"); ApplicationAttemptId applicationAttemptId = ApplicationAttemptId.newInstance(applicationId, 0); @@ -581,19 +581,19 @@ public void testApplicationAssignmentPreventsRemovalOfDynamicQueue() { q.addApp(appAttempt, true); queueManager.removeEmptyDynamicQueues(); q = queueManager.getLeafQueue("root.leaf1", false); - assertNotNull("root.leaf1 has been removed", q); - assertFalse("root.leaf1 is empty", q.isEmpty()); + assertNotNull(q, "root.leaf1 has been removed"); + assertFalse(q.isEmpty(), "root.leaf1 is empty"); // the appAttempt finished, the queue should be empty q.removeApp(appAttempt); q = queueManager.getLeafQueue("root.leaf1", false); - assertTrue("root.leaf1 is not empty", q.isEmpty()); + assertTrue(q.isEmpty(), "root.leaf1 is not empty"); // removeEmptyDynamicQueues() should remove the queue queueManager.removePendingIncompatibleQueues(); queueManager.removeEmptyDynamicQueues(); q = queueManager.getLeafQueue("root.leaf1", false); - assertNull("root.leaf1 has not been removed", q); + assertNull(q, "root.leaf1 has not been removed"); } /** @@ -608,20 +608,20 @@ public void testRemovalOfIncompatibleNonEmptyQueue() { queueManager.updateAllocationConfiguration(allocConf); FSLeafQueue q = queueManager.getLeafQueue("root.a", true); - assertNotNull("root.a does not exist", q); - assertTrue("root.a is not empty", q.isEmpty()); + assertNotNull(q, "root.a does not exist"); + assertTrue(q.isEmpty(), "root.a is not empty"); // we start to run an application on root.a ApplicationId appId = ApplicationId.newInstance(0, 0); q.addAssignedApp(appId); - assertFalse("root.a is empty", q.isEmpty()); + assertFalse(q.isEmpty(), "root.a is empty"); // root.a should not be removed by removeEmptyDynamicQueues or by // removePendingIncompatibleQueues queueManager.removePendingIncompatibleQueues(); queueManager.removeEmptyDynamicQueues(); q = queueManager.getLeafQueue("root.a", false); - assertNotNull("root.a does not exist", q); + assertNotNull(q, "root.a does not exist"); // let's introduce queue incompatibility allocConf.configuredQueues.get(FSQueueType.LEAF).remove("root.a"); @@ -631,14 +631,14 @@ public void testRemovalOfIncompatibleNonEmptyQueue() { // since root.a has running applications, it should be still a leaf queue q = queueManager.getLeafQueue("root.a", false); - assertNotNull("root.a has been removed", q); - assertFalse("root.a is empty", q.isEmpty()); + assertNotNull(q, "root.a has been removed"); + assertFalse(q.isEmpty(), "root.a is empty"); // removePendingIncompatibleQueues should still keep root.a as a leaf queue queueManager.removePendingIncompatibleQueues(); q = queueManager.getLeafQueue("root.a", false); - assertNotNull("root.a has been removed", q); - assertFalse("root.a is empty", q.isEmpty()); + assertNotNull(q, "root.a has been removed"); + assertFalse(q.isEmpty(), "root.a is empty"); // when the application finishes, root.a will become a parent queue on next // config cleanup. The leaf queue will be created below it on reload of the @@ -647,10 +647,10 @@ public void testRemovalOfIncompatibleNonEmptyQueue() { queueManager.removePendingIncompatibleQueues(); queueManager.removeEmptyDynamicQueues(); FSParentQueue p = queueManager.getParentQueue("root.a", false); - assertNotNull("root.a does not exist", p); + assertNotNull(p, "root.a does not exist"); queueManager.updateAllocationConfiguration(allocConf); q = queueManager.getLeafQueue("root.a.b", false); - assertNotNull("root.a.b was not created", q); + assertNotNull(q, "root.a.b was not created"); } /** @@ -660,8 +660,8 @@ public void testRemovalOfIncompatibleNonEmptyQueue() { public void testRemoveDeepHierarchy() { // create a deeper queue hierarchy FSLeafQueue q = queueManager.getLeafQueue("root.p1.p2.p3.leaf", true); - assertNotNull("root.p1.p2.p3.leaf does not exist", q); - assertTrue("root.p1.p2.p3.leaf is not empty", q.isEmpty()); + assertNotNull(q, "root.p1.p2.p3.leaf does not exist"); + assertTrue(q.isEmpty(), "root.p1.p2.p3.leaf is not empty"); // Add an application to make the queue not empty ApplicationId appId = ApplicationId.newInstance(0, 0); @@ -671,16 +671,16 @@ public void testRemoveDeepHierarchy() { queueManager.removePendingIncompatibleQueues(); queueManager.removeEmptyDynamicQueues(); q = queueManager.getLeafQueue("root.p1.p2.p3.leaf", false); - assertNotNull("root.p1.p2.p3.leaf does not exist", q); + assertNotNull(q, "root.p1.p2.p3.leaf does not exist"); // Remove the application q.removeAssignedApp(appId); // Cleanup should remove the whole tree queueManager.removeEmptyDynamicQueues(); q = queueManager.getLeafQueue("root.p1.p2.p3.leaf", false); - assertNull("root.p1.p2.p3.leaf does exist", q); + assertNull(q, "root.p1.p2.p3.leaf does exist"); FSParentQueue p = queueManager.getParentQueue("root.p1", false); - assertNull("root.p1 does exist", p); + assertNull(p, "root.p1 does exist"); } /** @@ -691,12 +691,12 @@ public void testRemoveDeepHierarchy() { public void testRemoveSplitHierarchy() { // create a deeper queue hierarchy FSLeafQueue leaf1 = queueManager.getLeafQueue("root.p1.p2-1.leaf-1", true); - assertNotNull("root.p1.p2-1.leaf-1 does not exist", leaf1); - assertTrue("root.p1.p2-1.leaf1 is not empty", leaf1.isEmpty()); + assertNotNull(leaf1, "root.p1.p2-1.leaf-1 does not exist"); + assertTrue(leaf1.isEmpty(), "root.p1.p2-1.leaf1 is not empty"); // Create a split below the first level FSLeafQueue leaf2 = queueManager.getLeafQueue("root.p1.p2-2.leaf-2", true); - assertNotNull("root.p1.p2-2.leaf2 does not exist", leaf2); - assertTrue("root.p1.p2-2.leaf2 is not empty", leaf2.isEmpty()); + assertNotNull(leaf2, "root.p1.p2-2.leaf2 does not exist"); + assertTrue(leaf2.isEmpty(), "root.p1.p2-2.leaf2 is not empty"); // Add an application to make one of the queues not empty ApplicationId appId = ApplicationId.newInstance(0, 0); @@ -706,19 +706,19 @@ public void testRemoveSplitHierarchy() { queueManager.removePendingIncompatibleQueues(); queueManager.removeEmptyDynamicQueues(); leaf1 = queueManager.getLeafQueue("root.p1.p2-1.leaf-1", false); - assertNotNull("root.p1.p2-1.leaf-1 does not exist", leaf1); + assertNotNull(leaf1, "root.p1.p2-1.leaf-1 does not exist"); leaf2 = queueManager.getLeafQueue("root.p1.p2-2.leaf-2", false); - assertNull("root.p1.p2-2.leaf2 does exist", leaf2); + assertNull(leaf2, "root.p1.p2-2.leaf2 does exist"); FSParentQueue p = queueManager.getParentQueue("root.p1.p2-2", false); - assertNull("root.p1.p2-2 does exist", p); + assertNull(p, "root.p1.p2-2 does exist"); // Remove the application leaf1.removeAssignedApp(appId); // Cleanup should remove the whole tree queueManager.removeEmptyDynamicQueues(); leaf1 = queueManager.getLeafQueue("root.p1.p2-1.leaf-1", false); - assertNull("root.p1.p2-1.leaf-1 does exist", leaf1); + assertNull(leaf1, "root.p1.p2-1.leaf-1 does exist"); p = queueManager.getParentQueue("root.p1", false); - assertNull("root.p1 does exist", p); + assertNull(p, "root.p1 does exist"); } } 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/fair/TestQueueManagerRealScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestQueueManagerRealScheduler.java index e9a0b39dd82fb..0bb40922dbd8d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestQueueManagerRealScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestQueueManagerRealScheduler.java @@ -22,14 +22,14 @@ .allocationfile.AllocationFileQueue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair .allocationfile.AllocationFileWriter; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.File; import java.io.IOException; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; /** * QueueManager tests that require a real scheduler @@ -37,7 +37,7 @@ public class TestQueueManagerRealScheduler extends FairSchedulerTestBase { private final static File ALLOC_FILE = new File(TEST_DIR, "test-queue-mgr"); - @Before + @BeforeEach public void setup() throws IOException { createConfiguration(); writeAllocFile(30); @@ -49,7 +49,7 @@ public void setup() throws IOException { scheduler = (FairScheduler) resourceManager.getResourceScheduler(); } - @After + @AfterEach public void teardown() { ALLOC_FILE.deleteOnExit(); if (resourceManager != 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/scheduler/fair/TestQueuePlacementPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestQueuePlacementPolicy.java index 5e8f60253cba5..1509ea381261a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestQueuePlacementPolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestQueuePlacementPolicy.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; -import static org.junit.Assert.*; +import static org.junit.jupiter.api.Assertions.*; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -44,10 +44,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementRule; import org.apache.hadoop.yarn.server.resourcemanager.placement.UserPlacementRule; import org.apache.hadoop.yarn.util.SystemClock; -import org.junit.After; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import org.w3c.dom.Document; import org.w3c.dom.Element; @@ -66,13 +66,13 @@ public class TestQueuePlacementPolicy { private ApplicationSubmissionContext asc; private ApplicationPlacementContext context; - @BeforeClass + @BeforeAll public static void setup() { CONF.setClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING, SimpleGroupsMapping.class, GroupMappingServiceProvider.class); } - @Before + @BeforeEach public void initTest() { SystemClock clock = SystemClock.getInstance(); RMContext rmContext = mock(RMContext.class); @@ -90,7 +90,7 @@ public void initTest() { when(scheduler.getQueueManager()).thenReturn(queueManager); } - @After + @AfterEach public void cleanTest() { placementManager = null; queueManager = null; @@ -155,7 +155,7 @@ public void testSpecifiedThenReject() throws Exception { assertEquals("root.specifiedq", context.getQueue()); asc = newAppSubmissionContext("default"); context = placementManager.placeApplication(asc, "someuser"); - assertNull("Assignment should have been rejected and was not", context); + assertNull(context, "Assignment should have been rejected and was not"); } @Test @@ -265,8 +265,8 @@ public void testMultipleParentRules() throws Exception { PlacementRule nested = placementManager.getPlacementRules().get(0); if (nested instanceof UserPlacementRule) { PlacementRule parent = ((FSPlacementRule)nested).getParentRule(); - assertTrue("Nested rule should have been Default rule", - parent instanceof DefaultPlacementRule); + assertTrue( + parent instanceof DefaultPlacementRule, "Nested rule should have been Default rule"); } else { fail("Policy parsing failed: rule with multiple parents not set"); } @@ -374,7 +374,7 @@ public void testNestedUserQueuePrimaryGroup() throws Exception { createQueue(FSQueueType.LEAF, "root.user3group"); asc = newAppSubmissionContext("default"); context = placementManager.placeApplication(asc, "user3"); - assertNull("Submission should have failed and did not", context); + assertNull(context, "Submission should have failed and did not"); } @Test @@ -520,7 +520,7 @@ public void testNestedUserQueueDefaultRule() throws Exception { createPolicy(sb.toString()); asc = newAppSubmissionContext("default"); context = placementManager.placeApplication(asc, "user1"); - assertNull("Submission should have failed and did not", context); + assertNull(context, "Submission should have failed and did not"); } @Test @@ -642,7 +642,7 @@ private ApplicationSubmissionContext newAppSubmissionContext(String queue) { private void createQueue(FSQueueType type, String name) { // Create a queue as if it is in the config. FSQueue queue = queueManager.createQueue(name, type); - assertNotNull("Queue not created", queue); + assertNotNull(queue, "Queue not created"); // walk up the list till we have a non dynamic queue // root is always non dynamic do { 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/fair/TestSchedulingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestSchedulingPolicy.java index edb43dbf1f1d8..2b090f8cca023 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestSchedulingPolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestSchedulingPolicy.java @@ -37,13 +37,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FairSharePolicy; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy; -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 static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -55,7 +55,7 @@ public class TestSchedulingPolicy { private FairSchedulerConfiguration conf; private FairScheduler scheduler; - @Before + @BeforeEach public void setUp() throws Exception { scheduler = new FairScheduler(); conf = new FairSchedulerConfiguration(); @@ -72,34 +72,34 @@ public void testParseSchedulingPolicy() // Class name SchedulingPolicy sm = SchedulingPolicy .parse(FairSharePolicy.class.getName()); - assertTrue("Invalid scheduler name", - sm.getName().equals(FairSharePolicy.NAME)); + assertTrue( + sm.getName().equals(FairSharePolicy.NAME), "Invalid scheduler name"); // Canonical name sm = SchedulingPolicy.parse(FairSharePolicy.class .getCanonicalName()); - assertTrue("Invalid scheduler name", - sm.getName().equals(FairSharePolicy.NAME)); + assertTrue( + sm.getName().equals(FairSharePolicy.NAME), "Invalid scheduler name"); // Class sm = SchedulingPolicy.getInstance(FairSharePolicy.class); - assertTrue("Invalid scheduler name", - sm.getName().equals(FairSharePolicy.NAME)); + assertTrue( + sm.getName().equals(FairSharePolicy.NAME), "Invalid scheduler name"); // Shortname - drf sm = SchedulingPolicy.parse("drf"); - assertTrue("Invalid scheduler name", - sm.getName().equals(DominantResourceFairnessPolicy.NAME)); + assertTrue( + sm.getName().equals(DominantResourceFairnessPolicy.NAME), "Invalid scheduler name"); // Shortname - fair sm = SchedulingPolicy.parse("fair"); - assertTrue("Invalid scheduler name", - sm.getName().equals(FairSharePolicy.NAME)); + assertTrue( + sm.getName().equals(FairSharePolicy.NAME), "Invalid scheduler name"); // Shortname - fifo sm = SchedulingPolicy.parse("fifo"); - assertTrue("Invalid scheduler name", - sm.getName().equals(FifoPolicy.NAME)); + assertTrue( + sm.getName().equals(FifoPolicy.NAME), "Invalid scheduler name"); } /** @@ -160,8 +160,8 @@ private void generateAndTest(Stack genSchedulable) { if (genSchedulable.size() == 3) { // We get three Schedulable objects, let's use them to check the // comparator. - Assert.assertTrue("The comparator must ensure transitivity", - checkTransitivity(genSchedulable)); + Assertions.assertTrue( + checkTransitivity(genSchedulable), "The comparator must ensure transitivity"); return; } @@ -192,7 +192,7 @@ private Schedulable createSchedulable( private boolean checkTransitivity( Collection schedulableObjs) { - Assert.assertEquals(3, schedulableObjs.size()); + Assertions.assertEquals(3, schedulableObjs.size()); Schedulable[] copy = schedulableObjs.toArray(new Schedulable[3]); if (fairShareComparator.compare(copy[0], copy[1]) > 0) { @@ -335,14 +335,14 @@ public void testSchedulingPolicyViolation() throws IOException { scheduler.init(conf); FSQueue child1 = scheduler.getQueueManager().getQueue("child1"); - assertNull("Queue 'child1' should be null since its policy isn't allowed to" - + " be 'drf' if its parent policy is 'fair'.", child1); + assertNull(child1, "Queue 'child1' should be null since its policy isn't allowed to" + + " be 'drf' if its parent policy is 'fair'."); // dynamic queue FSQueue dynamicQueue = scheduler.getQueueManager(). getLeafQueue("dynamicQueue", true); - assertNull("Dynamic queue should be null since it isn't allowed to be 'drf'" - + " policy if its parent policy is 'fair'.", dynamicQueue); + assertNull(dynamicQueue, "Dynamic queue should be null since it isn't allowed to be 'drf'" + + " policy if its parent policy is 'fair'."); // Set child1 to 'fair' and child2 to 'drf', the reload the allocation file. AllocationFileWriter.create() @@ -358,14 +358,14 @@ public void testSchedulingPolicyViolation() throws IOException { scheduler.reinitialize(conf, null); child1 = scheduler.getQueueManager().getQueue("child1"); - assertNotNull("Queue 'child1' should be not null since its policy is " - + "allowed to be 'fair' if its parent policy is 'fair'.", child1); + assertNotNull(child1, "Queue 'child1' should be not null since its policy is " + + "allowed to be 'fair' if its parent policy is 'fair'."); // Detect the policy violation of Child2, keep the original policy instead // of setting the new policy. FSQueue child2 = scheduler.getQueueManager().getQueue("child2"); - assertTrue("Queue 'child2' should be 'fair' since its new policy 'drf' " - + "is not allowed.", child2.getPolicy() instanceof FairSharePolicy); + assertTrue(child2.getPolicy() instanceof FairSharePolicy, "Queue 'child2' should be 'fair' since its new policy 'drf' " + + "is not allowed."); } @Test @@ -388,14 +388,14 @@ public void testSchedulingPolicyViolationInTheMiddleLevel() { scheduler.init(conf); FSQueue level2 = scheduler.getQueueManager().getQueue("level2"); - assertNotNull("Queue 'level2' shouldn't be null since its policy is allowed" - + " to be 'fair' if its parent policy is 'fair'.", level2); + assertNotNull(level2, "Queue 'level2' shouldn't be null since its policy is allowed" + + " to be 'fair' if its parent policy is 'fair'."); FSQueue level3 = scheduler.getQueueManager().getQueue("level2.level3"); - assertNull("Queue 'level3' should be null since its policy isn't allowed" - + " to be 'drf' if its parent policy is 'fair'.", level3); + assertNull(level3, "Queue 'level3' should be null since its policy isn't allowed" + + " to be 'drf' if its parent policy is 'fair'."); FSQueue leaf = scheduler.getQueueManager().getQueue("level2.level3.leaf"); - assertNull("Queue 'leaf' should be null since its parent failed to create.", - leaf); + assertNull( + leaf, "Queue 'leaf' should be null since its parent failed to create."); } @Test @@ -416,8 +416,8 @@ public void testFIFOPolicyOnlyForLeafQueues() scheduler.init(conf); FSQueue intermediate = scheduler.getQueueManager().getQueue("intermediate"); - assertNull("Queue 'intermediate' should be null since 'fifo' is only for " - + "leaf queue.", intermediate); + assertNull(intermediate, "Queue 'intermediate' should be null since 'fifo' is only for " + + "leaf queue."); AllocationFileWriter.create() .addQueue(new AllocationFileQueue.Builder("root") @@ -434,8 +434,8 @@ public void testFIFOPolicyOnlyForLeafQueues() assertNotNull(scheduler.getQueueManager().getQueue("intermediate")); FSQueue leaf = scheduler.getQueueManager().getQueue("intermediate.leaf"); - assertNotNull("Queue 'leaf' should be null since 'fifo' is only for " - + "leaf queue.", leaf); + assertNotNull(leaf, "Queue 'leaf' should be null since 'fifo' is only for " + + "leaf queue."); } @Test @@ -466,13 +466,13 @@ public void testPolicyReinitialization() throws IOException { scheduler.reinitialize(conf, null); FSQueue child1 = scheduler.getQueueManager().getQueue("child1"); - assertTrue("Queue 'child1' should still be 'fair' since 'drf' isn't allowed" - + " if its parent policy is 'fair'.", - child1.getPolicy() instanceof FairSharePolicy); + assertTrue( + child1.getPolicy() instanceof FairSharePolicy, "Queue 'child1' should still be 'fair' since 'drf' isn't allowed" + + " if its parent policy is 'fair'."); FSQueue child2 = scheduler.getQueueManager().getQueue("child2"); - assertTrue("Queue 'child2' should still be 'fair' there is a policy" - + " violation while reinitialization.", - child2.getPolicy() instanceof FairSharePolicy); + assertTrue( + child2.getPolicy() instanceof FairSharePolicy, "Queue 'child2' should still be 'fair' there is a policy" + + " violation while reinitialization."); // Set both child1 and root to 'drf', then reload the allocation file AllocationFileWriter.create() @@ -488,12 +488,12 @@ public void testPolicyReinitialization() throws IOException { scheduler.reinitialize(conf, null); child1 = scheduler.getQueueManager().getQueue("child1"); - assertTrue("Queue 'child1' should be 'drf' since both 'root' and 'child1'" - + " are 'drf'.", - child1.getPolicy() instanceof DominantResourceFairnessPolicy); + assertTrue( + child1.getPolicy() instanceof DominantResourceFairnessPolicy, "Queue 'child1' should be 'drf' since both 'root' and 'child1'" + + " are 'drf'."); child2 = scheduler.getQueueManager().getQueue("child2"); - assertTrue("Queue 'child2' should still be 'fifo' there is no policy" - + " violation while reinitialization.", - child2.getPolicy() instanceof FifoPolicy); + assertTrue( + child2.getPolicy() instanceof FifoPolicy, "Queue 'child2' should still be 'fifo' there is no policy" + + " violation while reinitialization."); } } 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/fair/TestSchedulingUpdate.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestSchedulingUpdate.java index 94298f42ea1ba..28b2e38ebe7c0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestSchedulingUpdate.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestSchedulingUpdate.java @@ -28,13 +28,14 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent; 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; +import org.junit.jupiter.api.Timeout; -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; public class TestSchedulingUpdate extends FairSchedulerTestBase { @@ -49,7 +50,7 @@ public Configuration createConfiguration() { return conf; } - @Before + @BeforeEach public void setup() { conf = createConfiguration(); resourceManager = new MockRM(conf); @@ -58,7 +59,7 @@ public void setup() { scheduler = (FairScheduler) resourceManager.getResourceScheduler(); } - @After + @AfterEach public void teardown() { if (resourceManager != null) { resourceManager.stop(); @@ -66,7 +67,8 @@ public void teardown() { } } - @Test (timeout = 3000) + @Test + @Timeout(value = 3) public void testSchedulingUpdateOnNodeJoinLeave() throws InterruptedException { verifyNoCalls(); @@ -102,34 +104,34 @@ private void verifyExpectedCalls(long expectedCalls, int memory, int vcores) count++; Thread.sleep(10); } - assertTrue("Update Thread has not run based on its metrics", - scheduler.fsOpDurations.hasUpdateThreadRunChanged()); - assertEquals("Root queue metrics memory does not have expected value", - memory, scheduler.getRootQueueMetrics().getAvailableMB()); - assertEquals("Root queue metrics cpu does not have expected value", - vcores, scheduler.getRootQueueMetrics().getAvailableVirtualCores()); + assertTrue( + scheduler.fsOpDurations.hasUpdateThreadRunChanged(), "Update Thread has not run based on its metrics"); + assertEquals( + memory, scheduler.getRootQueueMetrics().getAvailableMB(), "Root queue metrics memory does not have expected value"); + assertEquals( + vcores, scheduler.getRootQueueMetrics().getAvailableVirtualCores(), "Root queue metrics cpu does not have expected value"); MetricsCollectorImpl collector = new MetricsCollectorImpl(); scheduler.fsOpDurations.getMetrics(collector, true); MetricsRecord record = collector.getRecords().get(0); for (AbstractMetric abstractMetric : record.metrics()) { if (abstractMetric.name().contains("UpdateThreadRunNumOps")) { - assertEquals("Update Thread did not run expected number of times " + - "based on metric record count", - expectedCalls, - abstractMetric.value()); + assertEquals( + expectedCalls +, abstractMetric.value(), "Update Thread did not run expected number of times " + + "based on metric record count"); verified = true; } } - assertTrue("Did not find metric for UpdateThreadRunNumOps", verified); + assertTrue(verified, "Did not find metric for UpdateThreadRunNumOps"); } private void verifyNoCalls() { - assertFalse("Update thread should not have executed", - scheduler.fsOpDurations.hasUpdateThreadRunChanged()); - assertEquals("Scheduler queue memory should not have been updated", - 0, scheduler.getRootQueueMetrics().getAvailableMB()); - assertEquals("Scheduler queue cpu should not have been updated", - 0,scheduler.getRootQueueMetrics().getAvailableVirtualCores()); + assertFalse( + scheduler.fsOpDurations.hasUpdateThreadRunChanged(), "Update thread should not have executed"); + assertEquals( + 0, scheduler.getRootQueueMetrics().getAvailableMB(), "Scheduler queue memory should not have been updated"); + assertEquals( + 0, scheduler.getRootQueueMetrics().getAvailableVirtualCores(), "Scheduler queue cpu should not have been updated"); } } 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/fair/TestVisitedResourceRequestTracker.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestVisitedResourceRequestTracker.java index b36fc45c63246..e8bf32745f059 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestVisitedResourceRequestTracker.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestVisitedResourceRequestTracker.java @@ -24,9 +24,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ClusterNodeTracker; import org.apache.hadoop.yarn.util.resource.Resources; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import org.junit.Test; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import org.junit.jupiter.api.Test; import java.util.List; @@ -71,12 +71,12 @@ public void testVisitAnyRequestFirst() { new VisitedResourceRequestTracker(nodeTracker); // Visit ANY request first - assertTrue(FIRST_CALL_FAILURE, tracker.visit(anyRequest)); + assertTrue(tracker.visit(anyRequest), FIRST_CALL_FAILURE); // All other requests should return false - assertFalse(ANY_VISITED + RACK_FAILURE, tracker.visit(rackRequest)); - assertFalse(ANY_VISITED + NODE_FAILURE, tracker.visit(node1Request)); - assertFalse(ANY_VISITED + NODE_FAILURE, tracker.visit(node2Request)); + assertFalse(tracker.visit(rackRequest), ANY_VISITED + RACK_FAILURE); + assertFalse(tracker.visit(node1Request), ANY_VISITED + NODE_FAILURE); + assertFalse(tracker.visit(node2Request), ANY_VISITED + NODE_FAILURE); } @Test @@ -85,12 +85,12 @@ public void testVisitRackRequestFirst() { new VisitedResourceRequestTracker(nodeTracker); // Visit rack request first - assertTrue(FIRST_CALL_FAILURE, tracker.visit(rackRequest)); + assertTrue(tracker.visit(rackRequest), FIRST_CALL_FAILURE); // All other requests should return false - assertFalse(RACK_VISITED + ANY_FAILURE, tracker.visit(anyRequest)); - assertFalse(RACK_VISITED + NODE_FAILURE, tracker.visit(node1Request)); - assertFalse(RACK_VISITED + NODE_FAILURE, tracker.visit(node2Request)); + assertFalse(tracker.visit(anyRequest), RACK_VISITED + ANY_FAILURE); + assertFalse(tracker.visit(node1Request), RACK_VISITED + NODE_FAILURE); + assertFalse(tracker.visit(node2Request), RACK_VISITED + NODE_FAILURE); } @Test @@ -99,14 +99,14 @@ public void testVisitNodeRequestFirst() { new VisitedResourceRequestTracker(nodeTracker); // Visit node1 first - assertTrue(FIRST_CALL_FAILURE, tracker.visit(node1Request)); + assertTrue(tracker.visit(node1Request), FIRST_CALL_FAILURE); // Rack and ANY should return false - assertFalse(NODE_VISITED + ANY_FAILURE, tracker.visit(anyRequest)); - assertFalse(NODE_VISITED + RACK_FAILURE, tracker.visit(rackRequest)); + assertFalse(tracker.visit(anyRequest), NODE_VISITED + ANY_FAILURE); + assertFalse(tracker.visit(rackRequest), NODE_VISITED + RACK_FAILURE); // The other node should return true - assertTrue(NODE_VISITED + "Different node visit failed", - tracker.visit(node2Request)); + assertTrue( + tracker.visit(node2Request), NODE_VISITED + "Different node visit failed"); } } 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/fair/converter/FSConfigConverterTestCommons.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/FSConfigConverterTestCommons.java index af915d459f7e8..0e24a10ef26ee 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/FSConfigConverterTestCommons.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/FSConfigConverterTestCommons.java @@ -28,7 +28,7 @@ import java.io.PrintStream; import java.io.PrintWriter; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Helper methods for FS->CS converter testing. @@ -62,7 +62,7 @@ public void setUp() throws IOException { FileUtils.deleteDirectory(d); } boolean success = d.mkdirs(); - assertTrue("Can't create directory: " + d.getAbsolutePath(), success); + assertTrue(success, "Can't create directory: " + d.getAbsolutePath()); } public void tearDown() { 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/fair/converter/TestConvertedConfigValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestConvertedConfigValidator.java index 4feb4e93b999e..19394a16ff4f9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestConvertedConfigValidator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestConvertedConfigValidator.java @@ -21,12 +21,14 @@ import java.io.File; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.junit.runner.RunWith; import org.mockito.junit.MockitoJUnitRunner; +import static org.junit.jupiter.api.Assertions.assertThrows; + @RunWith(MockitoJUnitRunner.class) public class TestConvertedConfigValidator { private static final String CONFIG_DIR_PASSES = @@ -36,13 +38,13 @@ public class TestConvertedConfigValidator { private ConvertedConfigValidator validator; - @Before + @BeforeEach public void setup() { QueueMetrics.clearQueueMetrics(); validator = new ConvertedConfigValidator(); } - @After + @AfterEach public void after() { QueueMetrics.clearQueueMetrics(); } @@ -54,8 +56,10 @@ public void testValidationPassed() throws Exception { // expected: no exception } - @Test(expected = VerificationException.class) + @Test public void testValidationFails() throws Exception { - validator.validateConvertedConfig(CONFIG_DIR_FAIL); + assertThrows(VerificationException.class, ()->{ + validator.validateConvertedConfig(CONFIG_DIR_FAIL); + }); } } 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/fair/converter/TestFSConfigToCSConfigArgumentHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSConfigToCSConfigArgumentHandler.java index 26416350857d7..3ee3aa58c88f0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSConfigToCSConfigArgumentHandler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSConfigToCSConfigArgumentHandler.java @@ -17,9 +17,9 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.converter; import static org.apache.hadoop.test.MockitoUtil.verifyZeroInteractions; -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.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.doThrow; @@ -34,10 +34,10 @@ import org.apache.hadoop.util.Lists; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; -import org.junit.After; -import org.junit.Before; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; import org.junit.Rule; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.mockito.ArgumentCaptor; @@ -71,7 +71,7 @@ public class TestFSConfigToCSConfigArgumentHandler { private FSConfigConverterTestCommons fsTestCommons; - @Before + @BeforeEach public void setUp() throws IOException { fsTestCommons = new FSConfigConverterTestCommons(); fsTestCommons.setUp(); @@ -79,7 +79,7 @@ public void setUp() throws IOException { conversionOptions = new ConversionOptions(dryRunResultHolder, false); } - @After + @AfterEach public void tearDown() { QueueMetrics.clearQueueMetrics(); fsTestCommons.tearDown(); @@ -154,10 +154,10 @@ public void testMissingYarnSiteXmlArgument() throws Exception { FSConfigConverterTestCommons.OUTPUT_DIR}; int retVal = argumentHandler.parseAndConvert(args); - assertEquals("Return value", -1, retVal); + assertEquals(-1, retVal, "Return value"); - assertTrue("Error content missing", fsTestCommons.getErrContent() - .toString().contains("Missing yarn-site.xml parameter")); + assertTrue(fsTestCommons.getErrContent() + .toString().contains("Missing yarn-site.xml parameter"), "Error content missing"); } @Test @@ -180,11 +180,11 @@ public void testMissingOutputDirArgument() throws Exception { FSConfigConverterTestCommons.YARN_SITE_XML}; int retVal = argumentHandler.parseAndConvert(args); - assertEquals("Return value", -1, retVal); + assertEquals(-1, retVal, "Return value"); - assertTrue("Error content missing", fsTestCommons.getErrContent() + assertTrue(fsTestCommons.getErrContent() .toString() - .contains("Output directory or console mode was not defined")); + .contains("Output directory or console mode was not defined"), "Error content missing"); } @Test @@ -223,11 +223,11 @@ public void testInvalidOutputDir() throws Exception { FSConfigConverterTestCommons.YARN_SITE_XML); int retVal = argumentHandler.parseAndConvert(args); - assertEquals("Return value", -1, retVal); - assertTrue("Error content missing", fsTestCommons.getErrContent() + assertEquals(-1, retVal, "Return value"); + assertTrue(fsTestCommons.getErrContent() .toString() .contains("Cannot start FS config conversion due to the following " + - "precondition error")); + "precondition error"), "Error content missing"); } @Test @@ -324,15 +324,15 @@ public void testConvertFSConfigurationDefaults() throws Exception { FSConfigToCSConfigConverterParams params = conversionParams.getValue(); LOG.info("FS config converter parameters: " + params); - assertEquals("Yarn site config", - FSConfigConverterTestCommons.YARN_SITE_XML, - params.getYarnSiteXmlConfig()); - assertEquals("FS xml", FSConfigConverterTestCommons.FS_ALLOC_FILE, - params.getFairSchedulerXmlConfig()); - assertEquals("Conversion rules config", - FSConfigConverterTestCommons.CONVERSION_RULES_FILE, - params.getConversionRulesConfig()); - assertFalse("Console mode", params.isConsole()); + assertEquals( + FSConfigConverterTestCommons.YARN_SITE_XML +, params.getYarnSiteXmlConfig(), "Yarn site config"); + assertEquals(FSConfigConverterTestCommons.FS_ALLOC_FILE +, params.getFairSchedulerXmlConfig(), "FS xml"); + assertEquals( + FSConfigConverterTestCommons.CONVERSION_RULES_FILE +, params.getConversionRulesConfig(), "Conversion rules config"); + assertFalse(params.isConsole(), "Console mode"); } @Test @@ -356,15 +356,15 @@ public void testConvertFSConfigurationWithConsoleParam() FSConfigToCSConfigConverterParams params = conversionParams.getValue(); LOG.info("FS config converter parameters: " + params); - assertEquals("Yarn site config", - FSConfigConverterTestCommons.YARN_SITE_XML, - params.getYarnSiteXmlConfig()); - assertEquals("FS xml", FSConfigConverterTestCommons.FS_ALLOC_FILE, - params.getFairSchedulerXmlConfig()); - assertEquals("Conversion rules config", - FSConfigConverterTestCommons.CONVERSION_RULES_FILE, - params.getConversionRulesConfig()); - assertTrue("Console mode", params.isConsole()); + assertEquals( + FSConfigConverterTestCommons.YARN_SITE_XML +, params.getYarnSiteXmlConfig(), "Yarn site config"); + assertEquals(FSConfigConverterTestCommons.FS_ALLOC_FILE +, params.getFairSchedulerXmlConfig(), "FS xml"); + assertEquals( + FSConfigConverterTestCommons.CONVERSION_RULES_FILE +, params.getConversionRulesConfig(), "Conversion rules config"); + assertTrue(params.isConsole(), "Console mode"); } @Test @@ -389,18 +389,18 @@ public void testConvertFSConfigurationClusterResource() FSConfigToCSConfigConverterParams params = conversionParams.getValue(); LOG.info("FS config converter parameters: " + params); - assertEquals("Yarn site config", - FSConfigConverterTestCommons.YARN_SITE_XML, - params.getYarnSiteXmlConfig()); - assertEquals("FS xml", - FSConfigConverterTestCommons.FS_ALLOC_FILE, - params.getFairSchedulerXmlConfig()); - assertEquals("Conversion rules config", - FSConfigConverterTestCommons.CONVERSION_RULES_FILE, - params.getConversionRulesConfig()); + assertEquals( + FSConfigConverterTestCommons.YARN_SITE_XML +, params.getYarnSiteXmlConfig(), "Yarn site config"); + assertEquals( + FSConfigConverterTestCommons.FS_ALLOC_FILE +, params.getFairSchedulerXmlConfig(), "FS xml"); + assertEquals( + FSConfigConverterTestCommons.CONVERSION_RULES_FILE +, params.getConversionRulesConfig(), "Conversion rules config"); assertEquals("Cluster resource", "vcores=20, memory-mb=240", params.getClusterResource()); - assertTrue("Console mode", params.isConsole()); + assertTrue(params.isConsole(), "Console mode"); } @Test @@ -417,9 +417,9 @@ public void testConvertFSConfigurationErrorHandling() throws Exception { .when(mockConverter) .convert(ArgumentMatchers.any(FSConfigToCSConfigConverterParams.class)); int retVal = argumentHandler.parseAndConvert(args); - assertEquals("Return value", -1, retVal); - assertTrue("Error content missing", fsTestCommons.getErrContent() - .toString().contains("Unsupported property/setting encountered")); + assertEquals(-1, retVal, "Return value"); + assertTrue(fsTestCommons.getErrContent() + .toString().contains("Unsupported property/setting encountered"), "Error content missing"); } @Test @@ -435,9 +435,9 @@ public void testConvertFSConfigurationErrorHandling2() throws Exception { Mockito.doThrow(ConversionException.class).when(mockConverter) .convert(ArgumentMatchers.any(FSConfigToCSConfigConverterParams.class)); int retVal = argumentHandler.parseAndConvert(args); - assertEquals("Return value", -1, retVal); - assertTrue("Error content missing", fsTestCommons.getErrContent() - .toString().contains("Fatal error during FS config conversion")); + assertEquals(-1, retVal, "Return value"); + assertTrue(fsTestCommons.getErrContent() + .toString().contains("Fatal error during FS config conversion"), "Error content missing"); } @Test @@ -483,11 +483,11 @@ private void testDryRunWithException(Exception exception, .convert(ArgumentMatchers.any(FSConfigToCSConfigConverterParams.class)); int retVal = argumentHandler.parseAndConvert(args); - assertEquals("Return value", -1, retVal); - assertEquals("Number of errors", 1, dryRunResultHolder.getErrors().size()); + assertEquals(-1, retVal, "Return value"); + assertEquals(1, dryRunResultHolder.getErrors().size(), "Number of errors"); String error = dryRunResultHolder.getErrors().iterator().next(); - assertTrue("Unexpected error message", - error.contains(expectedErrorMessage)); + assertTrue( + error.contains(expectedErrorMessage), "Unexpected error message"); } @Test @@ -505,8 +505,8 @@ public void testDisabledTerminalRuleCheck() throws Exception { argumentHandler.parseAndConvert(args); - assertTrue("-t switch had no effect", - conversionOptions.isNoRuleTerminalCheck()); + assertTrue( + conversionOptions.isNoRuleTerminalCheck(), "-t switch had no effect"); } @Test @@ -523,8 +523,8 @@ public void testEnabledTerminalRuleCheck() throws Exception { argumentHandler.parseAndConvert(args); - assertFalse("No terminal rule check was enabled", - conversionOptions.isNoRuleTerminalCheck()); + assertFalse( + conversionOptions.isNoRuleTerminalCheck(), "No terminal rule check was enabled"); } @Test @@ -539,7 +539,7 @@ public void testYarnSiteOptionInOutputFolder() throws Exception { "-o", FSConfigConverterTestCommons.TEST_DIR}; int retVal = argumentHandler.parseAndConvert(args); - assertEquals("Return value", -1, retVal); + assertEquals(-1, retVal, "Return value"); assertTrue(fsTestCommons.getErrContent() .toString().contains("contains the yarn-site.xml")); @@ -561,7 +561,7 @@ private void testFileExistsInOutputFolder(String file) throws Exception { "-e"}; int retVal = argumentHandler.parseAndConvert(args); - assertEquals("Return value", -1, retVal); + assertEquals(-1, retVal, "Return value"); String expectedMessage = String.format( "already contains a file or directory named %s", file); @@ -631,11 +631,11 @@ private void testPlacementRuleConversion(boolean enabled) throws Exception { FSConfigToCSConfigConverterParams params = captor.getValue(); if (enabled) { - assertTrue("Conversion should be enabled by default", - params.isConvertPlacementRules()); + assertTrue( + params.isConvertPlacementRules(), "Conversion should be enabled by default"); } else { - assertFalse("-sp switch had no effect", - params.isConvertPlacementRules()); + assertFalse( + params.isConvertPlacementRules(), "-sp switch had no effect"); } } @@ -696,8 +696,8 @@ public void testEnabledAsyncScheduling() throws Exception { "-a"); argumentHandler.parseAndConvert(args); - assertTrue("-a switch had no effect", - conversionOptions.isEnableAsyncScheduler()); + assertTrue( + conversionOptions.isEnableAsyncScheduler(), "-a switch had no effect"); } @Test @@ -711,8 +711,8 @@ public void testDisabledAsyncScheduling() throws Exception { FSConfigConverterTestCommons.FS_ALLOC_FILE, "-p"); argumentHandler.parseAndConvert(args); - assertFalse("-a switch wasn't provided but async scheduling option is true", - conversionOptions.isEnableAsyncScheduler()); + assertFalse( + conversionOptions.isEnableAsyncScheduler(), "-a switch wasn't provided but async scheduling option is true"); } @Test @@ -749,6 +749,6 @@ private void testUsePercentages(boolean enabled) throws Exception { verify(mockConverter).convert(captor.capture()); FSConfigToCSConfigConverterParams params = captor.getValue(); - assertEquals("Use percentages", enabled, params.isUsePercentages()); + assertEquals(enabled, params.isUsePercentages(), "Use percentages"); } } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSConfigToCSConfigConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSConfigToCSConfigConverter.java index 99b1f6b4ef0f8..b62dffaf2d370 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSConfigToCSConfigConverter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSConfigToCSConfigConverter.java @@ -31,9 +31,9 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.converter.FSConfigToCSConfigRuleHandler.RuleAction.ABORT; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.converter.FSConfigToCSConfigRuleHandler.RuleAction.WARNING; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.Mockito.verify; @@ -55,10 +55,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.placement.schema.MappingRulesDescription; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedulerConfiguration; import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator; -import org.junit.After; -import org.junit.Before; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; import org.junit.Rule; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.mockito.Mock; @@ -136,7 +136,7 @@ private ConversionOptions createDefaultConversionOptions() { return new ConversionOptions(new DryRunResultHolder(), false); } - @Before + @BeforeEach public void setup() throws IOException { config = new Configuration(false); config.set(FairSchedulerConfiguration.ALLOCATION_FILE, FAIR_SCHEDULER_XML); @@ -147,7 +147,7 @@ public void setup() throws IOException { converterTestCommons.setUp(); } - @After + @AfterEach public void tearDown() { converterTestCommons.tearDown(); } @@ -183,17 +183,17 @@ public void testDefaultMaxAMShare() throws Exception { Float maxAmShare = conf.getMaximumApplicationMasterResourcePercent(); - assertEquals("Default max AM share", 0.16f, maxAmShare, 0.0f); + assertEquals( 0.16f, maxAmShare, 0.0f, "Default max AM share"); - assertEquals("root.admins.alice max-am-resource-percent", 0.15f, + assertEquals(0.15f, conf.getMaximumApplicationMasterResourcePerQueuePercent(ADMINS_ALICE), - 0.0f); + 0.0f, "root.admins.alice max-am-resource-percent"); //root.users.joe don’t have maximum-am-resource-percent set // so falling back to the global value - assertEquals("root.users.joe maximum-am-resource-percent", 0.16f, + assertEquals(0.16f, conf.getMaximumApplicationMasterResourcePerQueuePercent(USERS_JOE), - 0.0f); + 0.0f, "root.users.joe maximum-am-resource-percent"); } @Test @@ -202,21 +202,21 @@ public void testDefaultUserLimitFactor() throws Exception { CapacitySchedulerConfiguration conf = converter.getCapacitySchedulerConfig(); - assertEquals("root.users user-limit-factor", 1.0f, - conf.getUserLimitFactor(USERS), 0.0f); - assertEquals("root.users auto-queue-creation-v2.enabled", true, - conf.isAutoQueueCreationV2Enabled(USERS)); + assertEquals(1.0f, + conf.getUserLimitFactor(USERS), 0.0f, "root.users user-limit-factor"); + assertEquals(true +, conf.isAutoQueueCreationV2Enabled(USERS), "root.users auto-queue-creation-v2.enabled"); - assertEquals("root.default user-limit-factor", -1.0f, - conf.getUserLimitFactor(DEFAULT), 0.0f); + assertEquals(-1.0f, + conf.getUserLimitFactor(DEFAULT), 0.0f, "root.default user-limit-factor"); - assertEquals("root.users.joe user-limit-factor", -1.0f, - conf.getUserLimitFactor(USERS_JOE), 0.0f); + assertEquals(-1.0f, + conf.getUserLimitFactor(USERS_JOE), 0.0f, "root.users.joe user-limit-factor"); - assertEquals("root.admins.bob user-limit-factor", -1.0f, - conf.getUserLimitFactor(ADMINS_BOB), 0.0f); - assertEquals("root.admin.bob auto-queue-creation-v2.enabled", false, - conf.isAutoQueueCreationV2Enabled(ADMINS_BOB)); + assertEquals(-1.0f, + conf.getUserLimitFactor(ADMINS_BOB), 0.0f, "root.admins.bob user-limit-factor"); + assertEquals(false +, conf.isAutoQueueCreationV2Enabled(ADMINS_BOB), "root.admin.bob auto-queue-creation-v2.enabled"); } @Test @@ -231,20 +231,20 @@ public void testDefaultMaxAMShareDisabled() throws Exception { CapacitySchedulerConfiguration conf = converter.getCapacitySchedulerConfig(); // -1.0 means disabled ==> 1.0 in CS - assertEquals("Default max-am-resource-percent", 1.0f, - conf.getMaximumApplicationMasterResourcePercent(), 0.0f); + assertEquals(1.0f, + conf.getMaximumApplicationMasterResourcePercent(), 0.0f, "Default max-am-resource-percent"); // root.admins.bob is unset,so falling back to the global value - assertEquals("root.admins.bob maximum-am-resource-percent", 1.0f, - conf.getMaximumApplicationMasterResourcePerQueuePercent(ADMINS_BOB), 0.0f); + assertEquals(1.0f, + conf.getMaximumApplicationMasterResourcePerQueuePercent(ADMINS_BOB), 0.0f, "root.admins.bob maximum-am-resource-percent"); // root.admins.alice 0.15 != -1.0 - assertEquals("root.admins.alice max-am-resource-percent", 0.15f, - conf.getMaximumApplicationMasterResourcePerQueuePercent(ADMINS_ALICE), 0.0f); + assertEquals(0.15f, + conf.getMaximumApplicationMasterResourcePerQueuePercent(ADMINS_ALICE), 0.0f, "root.admins.alice max-am-resource-percent"); // root.users.joe is unset,so falling back to the global value - assertEquals("root.users.joe maximum-am-resource-percent", 1.0f, - conf.getMaximumApplicationMasterResourcePerQueuePercent(USERS_JOE), 0.0f); + assertEquals(1.0f, + conf.getMaximumApplicationMasterResourcePerQueuePercent(USERS_JOE), 0.0f, "root.users.joe maximum-am-resource-percent"); } @Test @@ -290,8 +290,8 @@ public void testDefaultQueueMaxParallelApps() throws Exception { CapacitySchedulerConfiguration conf = converter.getCapacitySchedulerConfig(); - assertEquals("Default max parallel apps", 15, - conf.getDefaultMaxParallelApps(), 0); + assertEquals(15, + conf.getDefaultMaxParallelApps(), 0, "Default max parallel apps"); } @Test @@ -300,8 +300,8 @@ public void testSpecificQueueMaxParallelApps() throws Exception { CapacitySchedulerConfiguration conf = converter.getCapacitySchedulerConfig(); - assertEquals("root.admins.alice max parallel apps", 2, - conf.getMaxParallelAppsForQueue(ADMINS_ALICE), 0); + assertEquals(2, + conf.getMaxParallelAppsForQueue(ADMINS_ALICE), 0, "root.admins.alice max parallel apps"); } @Test @@ -310,8 +310,8 @@ public void testDefaultUserMaxParallelApps() throws Exception { CapacitySchedulerConfiguration conf = converter.getCapacitySchedulerConfig(); - assertEquals("Default user max parallel apps", 10, - conf.getDefaultMaxParallelAppsPerUser(), 0); + assertEquals(10, + conf.getDefaultMaxParallelAppsPerUser(), 0, "Default user max parallel apps"); } @Test @@ -320,17 +320,17 @@ public void testSpecificUserMaxParallelApps() throws Exception { CapacitySchedulerConfiguration conf = converter.getCapacitySchedulerConfig(); - assertEquals("Max parallel apps for alice", 30, - conf.getMaxParallelAppsForUser("alice"), 0); + assertEquals(30, + conf.getMaxParallelAppsForUser("alice"), 0, "Max parallel apps for alice"); //users.bob, user.joe, user.john don’t have max-parallel-app set // so falling back to the global value for .user to 10 - assertEquals("Max parallel apps for user bob", 10, - conf.getMaxParallelAppsForUser("bob"), 0); - assertEquals("Max parallel apps for user joe", 10, - conf.getMaxParallelAppsForUser("joe"), 0); - assertEquals("Max parallel apps for user john", 10, - conf.getMaxParallelAppsForUser("john"), 0); + assertEquals(10, + conf.getMaxParallelAppsForUser("bob"), 0, "Max parallel apps for user bob"); + assertEquals(10, + conf.getMaxParallelAppsForUser("joe"), 0, "Max parallel apps for user joe"); + assertEquals(10, + conf.getMaxParallelAppsForUser("john"), 0, "Max parallel apps for user john"); } @Test @@ -362,8 +362,8 @@ public void testConvertFSConfigurationClusterResource() throws Exception { .withClusterResource(CLUSTER_RESOURCE_STRING) .build(); converter.convert(params); - assertEquals("Resource", Resource.newInstance(240, 20), - converter.getClusterResource()); + assertEquals(Resource.newInstance(240, 20) +, converter.getClusterResource(), "Resource"); } @Test @@ -373,8 +373,8 @@ public void testConvertFSConfigPctModeUsedAndClusterResourceDefined() .withClusterResource(CLUSTER_RESOURCE_STRING) .build(); converter.convert(params); - assertEquals("Resource", Resource.newInstance(240, 20), - converter.getClusterResource()); + assertEquals(Resource.newInstance(240, 20) +, converter.getClusterResource(), "Resource"); } @Test @@ -426,16 +426,16 @@ public void testConvertFSConfigurationRulesFile() throws Exception { Map actions = ruleHandler.getActions(); - assertEquals("maxCapacityPercentage", - ABORT, actions.get(MAX_CAPACITY_PERCENTAGE)); - assertEquals("maxChildCapacity", - ABORT, actions.get(MAX_CHILD_CAPACITY)); - assertEquals("dynamicMaxAssign", - ABORT, actions.get(DYNAMIC_MAX_ASSIGN)); - assertEquals("reservationSystem", - ABORT, actions.get(RESERVATION_SYSTEM)); - assertEquals("queueAutoCreate", - ABORT, actions.get(QUEUE_AUTO_CREATE)); + assertEquals( + ABORT, actions.get(MAX_CAPACITY_PERCENTAGE), "maxCapacityPercentage"); + assertEquals( + ABORT, actions.get(MAX_CHILD_CAPACITY), "maxChildCapacity"); + assertEquals( + ABORT, actions.get(DYNAMIC_MAX_ASSIGN), "dynamicMaxAssign"); + assertEquals( + ABORT, actions.get(RESERVATION_SYSTEM), "reservationSystem"); + assertEquals( + ABORT, actions.get(QUEUE_AUTO_CREATE), "queueAutoCreate"); } @Test @@ -455,28 +455,28 @@ public void testConvertFSConfigurationWithoutRulesFile() throws Exception { Map actions = ruleHandler.getActions(); - assertEquals("maxCapacityPercentage", - WARNING, actions.get(MAX_CAPACITY_PERCENTAGE)); - assertEquals("maxChildCapacity", - WARNING, actions.get(MAX_CHILD_CAPACITY)); - assertEquals("dynamicMaxAssign", - WARNING, actions.get(DYNAMIC_MAX_ASSIGN)); - assertEquals("reservationSystem", - WARNING, actions.get(RESERVATION_SYSTEM)); - assertEquals("queueAutoCreate", - WARNING, actions.get(QUEUE_AUTO_CREATE)); - assertEquals("childStaticDynamicConflict", - WARNING, actions.get(CHILD_STATIC_DYNAMIC_CONFLICT)); - assertEquals("parentChildCreateDiffers", - WARNING, actions.get(PARENT_CHILD_CREATE_DIFFERS)); - assertEquals("fairAsDrf", - WARNING, actions.get(FAIR_AS_DRF)); - assertEquals("maxResources", - WARNING, actions.get(MAX_RESOURCES)); - assertEquals("minResources", - WARNING, actions.get(MIN_RESOURCES)); - assertEquals("parentDynamicCreate", - WARNING, actions.get(PARENT_DYNAMIC_CREATE)); + assertEquals( + WARNING, actions.get(MAX_CAPACITY_PERCENTAGE), "maxCapacityPercentage"); + assertEquals( + WARNING, actions.get(MAX_CHILD_CAPACITY), "maxChildCapacity"); + assertEquals( + WARNING, actions.get(DYNAMIC_MAX_ASSIGN), "dynamicMaxAssign"); + assertEquals( + WARNING, actions.get(RESERVATION_SYSTEM), "reservationSystem"); + assertEquals( + WARNING, actions.get(QUEUE_AUTO_CREATE), "queueAutoCreate"); + assertEquals( + WARNING, actions.get(CHILD_STATIC_DYNAMIC_CONFLICT), "childStaticDynamicConflict"); + assertEquals( + WARNING, actions.get(PARENT_CHILD_CREATE_DIFFERS), "parentChildCreateDiffers"); + assertEquals( + WARNING, actions.get(FAIR_AS_DRF), "fairAsDrf"); + assertEquals( + WARNING, actions.get(MAX_RESOURCES), "maxResources"); + assertEquals( + WARNING, actions.get(MIN_RESOURCES), "minResources"); + assertEquals( + WARNING, actions.get(PARENT_DYNAMIC_CREATE), "parentDynamicCreate"); } @Test @@ -510,19 +510,19 @@ public void testConvertCheckOutputDir() throws Exception { File capacityFile = new File(FSConfigConverterTestCommons.OUTPUT_DIR, "capacity-scheduler.xml"); - assertTrue("Capacity file exists", capacityFile.exists()); - assertTrue("Capacity file length > 0", capacityFile.length() > 0); - assertTrue("No. of configuration elements > 0", conf.size() > 0); + assertTrue(capacityFile.exists(), "Capacity file exists"); + assertTrue(capacityFile.length() > 0, "Capacity file length > 0"); + assertTrue(conf.size() > 0, "No. of configuration elements > 0"); File yarnSiteFile = new File(FSConfigConverterTestCommons.OUTPUT_DIR, "yarn-site.xml"); - assertTrue("Yarn site exists", yarnSiteFile.exists()); - assertTrue("Yarn site length > 0", yarnSiteFile.length() > 0); + assertTrue(yarnSiteFile.exists(), "Yarn site exists"); + assertTrue(yarnSiteFile.length() > 0, "Yarn site length > 0"); File mappingRulesFile = new File(FSConfigConverterTestCommons.OUTPUT_DIR, "mapping-rules.json"); - assertTrue("Mapping rules file exists", mappingRulesFile.exists()); - assertTrue("Mapping rules file length > 0", mappingRulesFile.length() > 0); + assertTrue(mappingRulesFile.exists(), "Mapping rules file exists"); + assertTrue(mappingRulesFile.length() > 0, "Mapping rules file length > 0"); } @Test @@ -601,9 +601,9 @@ public void testConversionWhenOnlyFairPolicyIsUsed() throws Exception { Configuration convertedConfig = converter.getYarnSiteConfig(); - assertEquals("Resource calculator class shouldn't be set", null, - convertedConfig.getClass( - CapacitySchedulerConfiguration.RESOURCE_CALCULATOR_CLASS, null)); + assertEquals(null +, convertedConfig.getClass( + CapacitySchedulerConfiguration.RESOURCE_CALCULATOR_CLASS, null), "Resource calculator class shouldn't be set"); } @Test @@ -617,9 +617,9 @@ public void testConversionWhenMixedPolicyIsUsed() throws Exception { Configuration convertedConfig = converter.getYarnSiteConfig(); - assertEquals("Resource calculator type", DominantResourceCalculator.class, - convertedConfig.getClass( - CapacitySchedulerConfiguration.RESOURCE_CALCULATOR_CLASS, null)); + assertEquals(DominantResourceCalculator.class +, convertedConfig.getClass( + CapacitySchedulerConfiguration.RESOURCE_CALCULATOR_CLASS, null), "Resource calculator type"); } @Test @@ -664,10 +664,10 @@ private void testUserAsDefaultQueueAndPlacementRules( if (hasPlacementRules) { // fs.xml defines 5 rules - assertEquals("Number of rules", 5, description.getRules().size()); + assertEquals(5, description.getRules().size(), "Number of rules"); } else { // by default, FS internally creates 2 rules - assertEquals("Number of rules", 2, description.getRules().size()); + assertEquals(2, description.getRules().size(), "Number of rules"); } } @@ -709,15 +709,15 @@ public void testPlacementRulesConversionEnabled() throws Exception { public void testConversionWhenAsyncSchedulingIsEnabled() throws Exception { boolean schedulingEnabledValue = testConversionWithAsyncSchedulingOption(true); - assertTrue("Asynchronous scheduling should be true", schedulingEnabledValue); + assertTrue(schedulingEnabledValue, "Asynchronous scheduling should be true"); } @Test public void testConversionWhenAsyncSchedulingIsDisabled() throws Exception { boolean schedulingEnabledValue = testConversionWithAsyncSchedulingOption(false); - assertEquals("Asynchronous scheduling should be the default value", - CapacitySchedulerConfiguration.DEFAULT_SCHEDULE_ASYNCHRONOUSLY_ENABLE, - schedulingEnabledValue); + assertEquals( + CapacitySchedulerConfiguration.DEFAULT_SCHEDULE_ASYNCHRONOUSLY_ENABLE +, schedulingEnabledValue, "Asynchronous scheduling should be the default value"); } @Test @@ -729,9 +729,9 @@ public void testSiteDisabledPreemptionWithObserveOnlyConversion() .build(); converter.convert(params); - assertTrue("The observe only should be true", - converter.getCapacitySchedulerConfig(). - getPreemptionObserveOnly()); + assertTrue( + converter.getCapacitySchedulerConfig(). + getPreemptionObserveOnly(), "The observe only should be true"); } private boolean testConversionWithAsyncSchedulingOption(boolean enabled) throws Exception { 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/fair/converter/TestFSConfigToCSConfigConverterMain.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSConfigToCSConfigConverterMain.java index 87259cbe7920c..aed3416ccd2c0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSConfigToCSConfigConverterMain.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSConfigToCSConfigConverterMain.java @@ -21,8 +21,8 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.converter.FSConfigConverterTestCommons.OUTPUT_DIR; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.converter.FSConfigConverterTestCommons.YARN_SITE_XML; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.converter.FSConfigConverterTestCommons.setupFSConfigConversionFiles; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.File; import java.io.IOException; @@ -30,9 +30,9 @@ import java.util.function.Consumer; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; -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; /** @@ -43,7 +43,7 @@ public class TestFSConfigToCSConfigConverterMain { private FSConfigConverterTestCommons converterTestCommons; private ExitFunc exitFunc; - @Before + @BeforeEach public void setUp() throws Exception { exitFunc = new ExitFunc(); converterTestCommons = new FSConfigConverterTestCommons(); @@ -51,7 +51,7 @@ public void setUp() throws Exception { FSConfigToCSConfigConverterMain.setExit(exitFunc); } - @After + @AfterEach public void tearDown() throws Exception { QueueMetrics.clearQueueMetrics(); FSConfigToCSConfigConverterMain.setExit(System::exit); @@ -108,9 +108,9 @@ private void testConvertFSConfigurationDefaults(boolean percentage) boolean yarnSiteConfigExists = new File(OUTPUT_DIR, "yarn-site.xml").exists(); - assertTrue("capacity-scheduler.xml was not generated", csConfigExists); - assertTrue("yarn-site.xml was not generated", yarnSiteConfigExists); - assertEquals("Exit code", 0, exitFunc.exitCode); + assertTrue(csConfigExists, "capacity-scheduler.xml was not generated"); + assertTrue(yarnSiteConfigExists, "yarn-site.xml was not generated"); + assertEquals(0, exitFunc.exitCode, "Exit code"); } @Test @@ -126,13 +126,13 @@ public void testConvertFSConfigurationWithConsoleParam() "-r", CONVERSION_RULES_FILE}); String stdout = converterTestCommons.getStdOutContent().toString(); - assertTrue("Stdout doesn't contain yarn-site.xml", - stdout.contains("======= yarn-site.xml =======")); - assertTrue("Stdout doesn't contain capacity-scheduler.xml", - stdout.contains("======= capacity-scheduler.xml =======")); - assertTrue("Stdout doesn't contain mapping-rules.json", - stdout.contains("======= mapping-rules.json =======")); - assertEquals("Exit code", 0, exitFunc.exitCode); + assertTrue( + stdout.contains("======= yarn-site.xml ======="), "Stdout doesn't contain yarn-site.xml"); + assertTrue( + stdout.contains("======= capacity-scheduler.xml ======="), "Stdout doesn't contain capacity-scheduler.xml"); + assertTrue( + stdout.contains("======= mapping-rules.json ======="), "Stdout doesn't contain mapping-rules.json"); + assertEquals(0, exitFunc.exitCode, "Exit code"); } @Test @@ -140,7 +140,7 @@ public void testShortHelpSwitch() { FSConfigToCSConfigConverterMain.main(new String[] {"-h"}); verifyHelpText(); - assertEquals("Exit code", 0, exitFunc.exitCode); + assertEquals(0, exitFunc.exitCode, "Exit code"); } @Test @@ -148,7 +148,7 @@ public void testLongHelpSwitch() { FSConfigToCSConfigConverterMain.main(new String[] {"--help"}); verifyHelpText(); - assertEquals("Exit code", 0, exitFunc.exitCode); + assertEquals(0, exitFunc.exitCode, "Exit code"); } @Test @@ -156,7 +156,7 @@ public void testHelpDisplayedWithoutArgs() { FSConfigToCSConfigConverterMain.main(new String[] {}); verifyHelpText(); - assertEquals("Exit code", 0, exitFunc.exitCode); + assertEquals(0, exitFunc.exitCode, "Exit code"); } @Test @@ -173,13 +173,13 @@ public void testConvertFSConfigurationWithLongSwitches() "--rulesconfig", CONVERSION_RULES_FILE}); String stdout = converterTestCommons.getStdOutContent().toString(); - assertTrue("Stdout doesn't contain yarn-site.xml", - stdout.contains("======= yarn-site.xml =======")); - assertTrue("Stdout doesn't contain capacity-scheduler.xml", - stdout.contains("======= capacity-scheduler.xml =======")); - assertTrue("Stdout doesn't contain mapping-rules.json", - stdout.contains("======= mapping-rules.json =======")); - assertEquals("Exit code", 0, exitFunc.exitCode); + assertTrue( + stdout.contains("======= yarn-site.xml ======="), "Stdout doesn't contain yarn-site.xml"); + assertTrue( + stdout.contains("======= capacity-scheduler.xml ======="), "Stdout doesn't contain capacity-scheduler.xml"); + assertTrue( + stdout.contains("======= mapping-rules.json ======="), "Stdout doesn't contain mapping-rules.json"); + assertEquals(0, exitFunc.exitCode, "Exit code"); } @Test @@ -188,13 +188,13 @@ public void testNegativeReturnValueOnError() { "--print", "--yarnsiteconfig"}); - assertEquals("Exit code", -1, exitFunc.exitCode); + assertEquals(-1, exitFunc.exitCode, "Exit code"); } private void verifyHelpText() { String stdout = converterTestCommons.getStdOutContent().toString(); - assertTrue("Help was not displayed", - stdout.contains("General options are:")); + assertTrue( + stdout.contains("General options are:"), "Help was not displayed"); } @SuppressWarnings("checkstyle:visibilitymodifier") 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/fair/converter/TestFSConfigToCSConfigRuleHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSConfigToCSConfigRuleHandler.java index 11ecca88f77ed..383b4cb125f48 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSConfigToCSConfigRuleHandler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSConfigToCSConfigRuleHandler.java @@ -25,14 +25,13 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.converter.FSConfigToCSConfigRuleHandler.QUEUE_AUTO_CREATE; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.converter.FSConfigToCSConfigRuleHandler.RESERVATION_SYSTEM; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.converter.FSConfigToCSConfigRuleHandler.FAIR_AS_DRF; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.*; import java.io.IOException; import java.util.Properties; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; /** * Unit tests for FSConfigToCSConfigRuleHandler. @@ -45,7 +44,7 @@ public class TestFSConfigToCSConfigRuleHandler { private FSConfigToCSConfigRuleHandler ruleHandler; private DryRunResultHolder dryRunResultHolder; - @Before + @BeforeEach public void setup() { dryRunResultHolder = new DryRunResultHolder(); } @@ -123,15 +122,17 @@ public void testAllRulesAbort() throws IOException { expectAbort(() -> ruleHandler.handleFairAsDrf("test")); } - @Test(expected = ConversionException.class) + @Test public void testMaxChildQueueCountNotInteger() throws IOException { - Properties rules = new Properties(); - rules.put(MAX_CHILD_QUEUE_LIMIT, "abc"); + assertThrows(ConversionException.class, ()->{ + Properties rules = new Properties(); + rules.put(MAX_CHILD_QUEUE_LIMIT, "abc"); - ruleHandler = new FSConfigToCSConfigRuleHandler(rules, - createDefaultConversionOptions()); + ruleHandler = new FSConfigToCSConfigRuleHandler(rules, + createDefaultConversionOptions()); - ruleHandler.handleChildQueueCount("test", 1); + ruleHandler.handleChildQueueCount("test", 1); + }); } @Test @@ -144,10 +145,10 @@ public void testDryRunWarning() { ruleHandler.handleDynamicMaxAssign(); ruleHandler.handleMaxChildCapacity(); - assertEquals("Number of warnings", 2, - dryRunResultHolder.getWarnings().size()); - assertEquals("Number of errors", 0, - dryRunResultHolder.getErrors().size()); + assertEquals(2 +, dryRunResultHolder.getWarnings().size(), "Number of warnings"); + assertEquals(0 +, dryRunResultHolder.getErrors().size(), "Number of errors"); } @Test @@ -162,10 +163,10 @@ public void testDryRunError() { ruleHandler.handleDynamicMaxAssign(); ruleHandler.handleMaxChildCapacity(); - assertEquals("Number of warnings", 0, - dryRunResultHolder.getWarnings().size()); - assertEquals("Number of errors", 2, - dryRunResultHolder.getErrors().size()); + assertEquals(0 +, dryRunResultHolder.getWarnings().size(), "Number of warnings"); + assertEquals(2 +, dryRunResultHolder.getErrors().size(), "Number of errors"); } private void expectAbort(VoidCall call) { @@ -183,8 +184,8 @@ private void expectAbort(VoidCall call, Class exceptionClass) { exceptionThrown = true; } - assertTrue("Exception was not thrown", exceptionThrown); - assertEquals("Unexpected exception", exceptionClass, thrown.getClass()); + assertTrue(exceptionThrown, "Exception was not thrown"); + assertEquals(exceptionClass, thrown.getClass(), "Unexpected exception"); } @FunctionalInterface 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/fair/converter/TestFSQueueConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSQueueConverter.java index 47f8e1edfec3f..68a7655e177e1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSQueueConverter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSQueueConverter.java @@ -18,9 +18,9 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.DEFAULT_MAX_PARALLEL_APPLICATIONS; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.PREFIX; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -42,10 +42,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueue; 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.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; import org.junit.Rule; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.mockito.Mock; @@ -111,7 +111,7 @@ private static String prepareFileName(String f) { @Rule public ExpectedException expectedException = ExpectedException.none(); - @Before + @BeforeEach public void setup() { yarnConfig = new Configuration(false); yarnConfig.set(FairSchedulerConfiguration.ALLOCATION_FILE, @@ -129,7 +129,7 @@ public void setup() { rootQueue = fs.getQueueManager().getRootQueue(); } - @After + @AfterEach public void tearDown() throws IOException { if (fs != null) { fs.close(); @@ -167,20 +167,20 @@ public void testConvertQueueHierarchy() { converter.convertQueueHierarchy(rootQueue); // root children - assertEquals("root children", Arrays.asList("admins", "users", "misc", "default"), - csConfig.getQueues(ROOT)); + assertEquals(Arrays.asList("admins", "users", "misc", "default") +, csConfig.getQueues(ROOT), "root children"); // root.admins children - assertEquals("root.admins children", Arrays.asList("bob", "alice"), - csConfig.getQueues(ADMINS)); + assertEquals(Arrays.asList("bob", "alice") +, csConfig.getQueues(ADMINS), "root.admins children"); // root.default children - none - assertTrue("root.default children", - csConfig.getQueues(DEFAULT).isEmpty()); + assertTrue( + csConfig.getQueues(DEFAULT).isEmpty(), "root.default children"); // root.users children - assertEquals("root.users children", Arrays.asList("john", "joe"), - csConfig.getQueues(USERS)); + assertEquals(Arrays.asList("john", "joe") +, csConfig.getQueues(USERS), "root.users children"); Set leafs = Sets.difference(ALL_QUEUES, Sets.newHashSet("root", @@ -191,8 +191,8 @@ public void testConvertQueueHierarchy() { for (String queue : leafs) { key = PREFIX + queue + ".queues"; - assertTrue("Key " + key + " has value, but it should be empty", - csConfig.getQueues(new QueuePath(queue)).isEmpty()); + assertTrue( + csConfig.getQueues(new QueuePath(queue)).isEmpty(), "Key " + key + " has value, but it should be empty"); } } @@ -204,23 +204,23 @@ public void testQueueMaxAMShare() { converter.convertQueueHierarchy(rootQueue); // root.admins.bob - assertEquals("root.admins.bob AM share", 1.0f, + assertEquals(1.0f, csConfig.getMaximumApplicationMasterResourcePerQueuePercent( - ADMINS_BOB), 0.0f); + ADMINS_BOB), 0.0f, "root.admins.bob AM share"); // root.admins.alice - assertEquals("root.admins.alice AM share", 0.15f, + assertEquals(0.15f, csConfig.getMaximumApplicationMasterResourcePerQueuePercent( - ADMINS_ALICE), 0.0f); + ADMINS_ALICE), 0.0f, "root.admins.alice AM share"); Set remaining = Sets.difference(ALL_QUEUES, Sets.newHashSet("root.admins.bob", "root.admins.alice")); for (String queue : remaining) { key = PREFIX + queue + ".maximum-am-resource-percent"; - assertEquals("Key " + key + " has different value", + assertEquals( 0.1f, csConfig - .getMaximumApplicationMasterResourcePerQueuePercent(new QueuePath(queue)), 0.0f); + .getMaximumApplicationMasterResourcePerQueuePercent(new QueuePath(queue)), 0.0f, "Key " + key + " has different value"); } } @@ -230,17 +230,17 @@ public void testQueueMaxParallelApps() { converter.convertQueueHierarchy(rootQueue); - assertEquals("root.admins.alice max apps", 2, - csConfig.getMaxParallelAppsForQueue(ADMINS_ALICE), 0); + assertEquals(2, + csConfig.getMaxParallelAppsForQueue(ADMINS_ALICE), 0, "root.admins.alice max apps"); Set remaining = Sets.difference(ALL_QUEUES, Sets.newHashSet("root.admins.alice")); for (String queue : remaining) { key = PREFIX + queue + ".max-parallel-apps"; - assertEquals("Key " + key + " has different value", + assertEquals( DEFAULT_MAX_PARALLEL_APPLICATIONS, csConfig - .getMaxParallelAppsForQueue(new QueuePath(queue)), 0); + .getMaxParallelAppsForQueue(new QueuePath(queue)), 0, "Key " + key + " has different value"); } } @@ -251,30 +251,30 @@ public void testQueueMaxAllocations() { converter.convertQueueHierarchy(rootQueue); // root.admins vcores + mb - assertEquals("root.admins max vcores", 3, - csConfig.getQueueMaximumAllocationVcores(ADMINS)); - assertEquals("root.admins max memory", 4096, - csConfig.getQueueMaximumAllocationMb(ADMINS)); + assertEquals(3 +, csConfig.getQueueMaximumAllocationVcores(ADMINS), "root.admins max vcores"); + assertEquals(4096 +, csConfig.getQueueMaximumAllocationMb(ADMINS), "root.admins max memory"); // root.users.john max vcores + mb - assertEquals("root.users.john max vcores", 2, - csConfig.getQueueMaximumAllocationVcores(USERS_JOHN)); - assertEquals("root.users.john max memory", 8192, - csConfig.getQueueMaximumAllocationMb(USERS_JOHN)); + assertEquals(2 +, csConfig.getQueueMaximumAllocationVcores(USERS_JOHN), "root.users.john max vcores"); + assertEquals(8192 +, csConfig.getQueueMaximumAllocationMb(USERS_JOHN), "root.users.john max memory"); Set remaining = Sets.difference(ALL_QUEUES, Sets.newHashSet("root.admins", "root.users.john")); for (String queue : remaining) { key = PREFIX + queue + ".maximum-allocation-vcores"; - assertEquals("Key " + key + " has different value", + assertEquals( -1.0, csConfig - .getQueueMaximumAllocationVcores(new QueuePath(queue)), 0.0f); + .getQueueMaximumAllocationVcores(new QueuePath(queue)), 0.0f, "Key " + key + " has different value"); key = PREFIX + queue + ".maximum-allocation-mb"; - assertEquals("Key " + key + " has different value", + assertEquals( -1.0, csConfig - .getQueueMaximumAllocationMb(new QueuePath(queue)), 0.0f); + .getQueueMaximumAllocationMb(new QueuePath(queue)), 0.0f, "Key " + key + " has different value"); } } @@ -284,20 +284,20 @@ public void testQueuePreemptionDisabled() { converter.convertQueueHierarchy(rootQueue); - assertTrue("root.admins.alice preemption setting", - csConfig.getPreemptionDisabled( - ADMINS_ALICE, false)); - assertTrue("root.users.joe preemption setting", - csConfig.getPreemptionDisabled( - USERS_JOE, false)); + assertTrue( + csConfig.getPreemptionDisabled( + ADMINS_ALICE, false), "root.admins.alice preemption setting"); + assertTrue( + csConfig.getPreemptionDisabled( + USERS_JOE, false), "root.users.joe preemption setting"); Set remaining = Sets.difference(ALL_QUEUES, Sets.newHashSet("root.admins.alice", "root.users.joe")); for (String queue : remaining) { key = PREFIX + queue + ".disable_preemption"; - assertEquals("Key " + key + " has different value", - false, csConfig.getPreemptionDisabled(new QueuePath(queue), false)); + assertEquals( + false, csConfig.getPreemptionDisabled(new QueuePath(queue), false), "Key " + key + " has different value"); } } @@ -309,8 +309,8 @@ public void testQueuePreemptionDisabledWhenGlobalPreemptionDisabled() { for (String queue : ALL_QUEUES) { key = PREFIX + queue + ".disable_preemption"; - assertEquals("Key " + key + " has different value", - false, csConfig.getPreemptionDisabled(new QueuePath(queue), false)); + assertEquals( + false, csConfig.getPreemptionDisabled(new QueuePath(queue), false), "Key " + key + " has different value"); } } @@ -321,32 +321,32 @@ public void testChildCapacityInCapacityMode() { converter.convertQueueHierarchy(rootQueue); // root - assertEquals("root.default capacity", 33.333f, - csConfig.getNonLabeledQueueCapacity(DEFAULT), 0.0f); - assertEquals("root.admins capacity", 33.333f, - csConfig.getNonLabeledQueueCapacity(ADMINS), 0.0f); - assertEquals("root.users capacity", 33.334f, - csConfig.getNonLabeledQueueCapacity(USERS), 0.0f); + assertEquals(33.333f, + csConfig.getNonLabeledQueueCapacity(DEFAULT), 0.0f, "root.default capacity"); + assertEquals(33.333f, + csConfig.getNonLabeledQueueCapacity(ADMINS), 0.0f, "root.admins capacity"); + assertEquals(33.334f, + csConfig.getNonLabeledQueueCapacity(USERS), 0.0f, "root.users capacity"); // root.users - assertEquals("root.users.john capacity", 25.000f, - csConfig.getNonLabeledQueueCapacity(USERS_JOHN), 0.0f); - assertEquals("root.users.joe capacity", 75.000f, - csConfig.getNonLabeledQueueCapacity(USERS_JOE), 0.0f); + assertEquals(25.000f, + csConfig.getNonLabeledQueueCapacity(USERS_JOHN), 0.0f, "root.users.john capacity"); + assertEquals(75.000f, + csConfig.getNonLabeledQueueCapacity(USERS_JOE), 0.0f, "root.users.joe capacity"); // root.admins - assertEquals("root.admins.alice capacity", 75.000f, - csConfig.getNonLabeledQueueCapacity(ADMINS_ALICE), 0.0f); - assertEquals("root.admins.bob capacity", 25.000f, - csConfig.getNonLabeledQueueCapacity(ADMINS_BOB), 0.0f); + assertEquals(5.000f, + csConfig.getNonLabeledQueueCapacity(ADMINS_ALICE), 0.0f, "root.admins.alice capacity"); + assertEquals(25.000f, + csConfig.getNonLabeledQueueCapacity(ADMINS_BOB), 0.0f, "root.admins.bob capacity"); // root.misc - assertEquals("root.misc capacity", 0.000f, - csConfig.getNonLabeledQueueCapacity(MISC), 0.000f); - assertEquals("root.misc.a capacity", 0.000f, - csConfig.getNonLabeledQueueCapacity(MISC_A), 0.000f); - assertEquals("root.misc.b capacity", 0.000f, - csConfig.getNonLabeledQueueCapacity(MISC_B), 0.000f); + assertEquals(0.000f, + csConfig.getNonLabeledQueueCapacity(MISC), 0.000f, "root.misc capacity"); + assertEquals(0.000f, + csConfig.getNonLabeledQueueCapacity(MISC_A), 0.000f, "root.misc.a capacity"); + assertEquals(0.000f, + csConfig.getNonLabeledQueueCapacity(MISC_B), 0.000f, "root.misc.b capacity"); } @Test @@ -356,32 +356,32 @@ public void testChildCapacityInWeightMode() { converter.convertQueueHierarchy(rootQueue); // root - assertEquals("root.default weight", 1.0f, - csConfig.getNonLabeledQueueWeight(DEFAULT), 0.01f); - assertEquals("root.admins weight", 1.0f, - csConfig.getNonLabeledQueueWeight(ADMINS), 0.01f); - assertEquals("root.users weight", 1.0f, - csConfig.getNonLabeledQueueWeight(USERS), 0.01f); + assertEquals(1.0f, + csConfig.getNonLabeledQueueWeight(DEFAULT), 0.01f, "root.default weight"); + assertEquals(1.0f, + csConfig.getNonLabeledQueueWeight(ADMINS), 0.01f, "root.admins weight"); + assertEquals(1.0f, + csConfig.getNonLabeledQueueWeight(USERS), 0.01f, "root.users weight"); // root.users - assertEquals("root.users.john weight", 1.0f, - csConfig.getNonLabeledQueueWeight(USERS_JOHN), 0.01f); - assertEquals("root.users.joe weight", 3.0f, - csConfig.getNonLabeledQueueWeight(USERS_JOE), 0.01f); + assertEquals(1.0f, + csConfig.getNonLabeledQueueWeight(USERS_JOHN), 0.01f, "root.users.john weight"); + assertEquals(3.0f, + csConfig.getNonLabeledQueueWeight(USERS_JOE), 0.01f, "root.users.joe weight"); // root.admins - assertEquals("root.admins.alice weight", 3.0f, - csConfig.getNonLabeledQueueWeight(ADMINS_ALICE), 0.01f); - assertEquals("root.admins.bob weight", 1.0f, - csConfig.getNonLabeledQueueWeight(ADMINS_BOB), 0.01f); + assertEquals( 3.0f, + csConfig.getNonLabeledQueueWeight(ADMINS_ALICE), 0.01f, "root.admins.alice weight"); + assertEquals(1.0f, + csConfig.getNonLabeledQueueWeight(ADMINS_BOB), 0.01f, "root.admins.bob weight"); // root.misc - assertEquals("root.misc weight", 0.0f, - csConfig.getNonLabeledQueueWeight(MISC), 0.00f); - assertEquals("root.misc.a weight", 0.0f, - csConfig.getNonLabeledQueueWeight(MISC_A), 0.00f); - assertEquals("root.misc.b weight", 0.0f, - csConfig.getNonLabeledQueueWeight(MISC_B), 0.00f); + assertEquals(0.0f, + csConfig.getNonLabeledQueueWeight(MISC), 0.00f, "root.misc weight"); + assertEquals(0.0f, + csConfig.getNonLabeledQueueWeight(MISC_A), 0.00f, "root.misc.a weight"); + assertEquals(0.0f, + csConfig.getNonLabeledQueueWeight(MISC_B), 0.00f, "root.misc.b weight"); } @Test @@ -390,16 +390,16 @@ public void testAutoCreateV2FlagsInWeightMode() { converter.convertQueueHierarchy(rootQueue); - assertTrue("root autocreate v2 flag", - csConfig.isAutoQueueCreationV2Enabled(ROOT)); - assertTrue("root.admins autocreate v2 flag", - csConfig.isAutoQueueCreationV2Enabled(ADMINS)); - assertTrue("root.admins.alice autocreate v2 flag", - csConfig.isAutoQueueCreationV2Enabled(ADMINS_ALICE)); - assertTrue("root.users autocreate v2 flag", - csConfig.isAutoQueueCreationV2Enabled(USERS)); - assertTrue("root.misc autocreate v2 flag", - csConfig.isAutoQueueCreationV2Enabled(MISC)); + assertTrue( + csConfig.isAutoQueueCreationV2Enabled(ROOT), "root autocreate v2 flag"); + assertTrue( + csConfig.isAutoQueueCreationV2Enabled(ADMINS), "root.admins autocreate v2 flag"); + assertTrue( + csConfig.isAutoQueueCreationV2Enabled(ADMINS_ALICE), "root.admins.alice autocreate v2 flag"); + assertTrue( + csConfig.isAutoQueueCreationV2Enabled(USERS), "root.users autocreate v2 flag"); + assertTrue( + csConfig.isAutoQueueCreationV2Enabled(MISC), "root.misc autocreate v2 flag"); //leaf queue root.admins.alice is removed from the below list //adding reservation to a leaf, it's queueType changes to FSParentQueue @@ -412,9 +412,9 @@ public void testAutoCreateV2FlagsInWeightMode() { for (String queue : leafs) { key = PREFIX + queue + ".auto-queue-creation-v2.enabled"; - assertEquals("Key " + key + " has different value", - false, csConfig - .isAutoQueueCreationV2Enabled(new QueuePath(queue))); + assertEquals( + false, csConfig + .isAutoQueueCreationV2Enabled(new QueuePath(queue)), "Key " + key + " has different value"); } } @@ -430,13 +430,13 @@ public void testZeroSumCapacityValidation() { for (String queue : noZeroSumAllowedQueues) { key = PREFIX + queue + ".allow-zero-capacity-sum"; - assertEquals("Key " + key + " has different value", - false, csConfig - .getAllowZeroCapacitySum(new QueuePath(queue))); + assertEquals( + false, csConfig + .getAllowZeroCapacitySum(new QueuePath(queue)), "Key " + key + " has different value"); } - assertTrue("root.misc allow zero capacities", - csConfig.getAllowZeroCapacitySum(MISC)); + assertTrue( + csConfig.getAllowZeroCapacitySum(MISC), "root.misc allow zero capacities"); } @Test @@ -447,9 +447,9 @@ public void testQueueMaximumCapacity() { for (String queue : ALL_QUEUES) { key = PREFIX + queue + ".maximum-capacity"; - assertEquals("Key " + key + " has different value", + assertEquals( 100.0, csConfig - .getNonLabeledQueueMaximumCapacity(new QueuePath(queue)), 0.0f); + .getNonLabeledQueueMaximumCapacity(new QueuePath(queue)), 0.0f, "Key " + key + " has different value"); } verify(ruleHandler, times(3)).handleMaxResources(); } @@ -473,8 +473,8 @@ public void testQueueWithNoAutoCreateChildQueue() { for (String queue : ALL_QUEUES) { key = PREFIX + queue + ".auto-create-child-queue.enabled"; - assertEquals("Key " + key + " has different value", - false, csConfig.isAutoCreateChildQueueEnabled(new QueuePath(queue))); + assertEquals( + false, csConfig.isAutoCreateChildQueueEnabled(new QueuePath(queue)), "Key " + key + " has different value"); } } @@ -486,8 +486,8 @@ public void testQueueSizeBasedWeightEnabled() { for (String queue : ALL_QUEUES) { key = PREFIX + queue + ".ordering-policy.fair.enable-size-based-weight"; - assertTrue("Key " + key + " has different value", - csConfig.getBoolean(key, false)); + assertTrue( + csConfig.getBoolean(key, false), "Key " + key + " has different value"); } } @@ -499,8 +499,8 @@ public void testQueueSizeBasedWeightDisabled() { for (String queue : ALL_QUEUES) { key = PREFIX + queue + ".ordering-policy.fair.enable-size-based-weight"; - assertNull("Key " + key + " has different value", - csConfig.get(key)); + assertNull( + csConfig.get(key), "Key " + key + " has different value"); } } 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/fair/converter/TestFSYarnSiteConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSYarnSiteConverter.java index 3fcf213d61168..5ae4f27170349 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSYarnSiteConverter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSYarnSiteConverter.java @@ -25,13 +25,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedulerConfiguration; import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertNotEquals; +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.assertNotEquals; /** * Unit tests for FSYarnSiteConverter. @@ -44,7 +44,7 @@ public class TestFSYarnSiteConverter { private static final String DELETION_POLICY_CLASS = AutoCreatedQueueDeletionPolicy.class.getCanonicalName(); - @Before + @BeforeEach public void setup() { yarnConfig = new Configuration(false); yarnConvertedConfig = new Configuration(false); @@ -62,25 +62,25 @@ public void testSiteContinuousSchedulingConversion() { converter.convertSiteProperties(yarnConfig, yarnConvertedConfig, false, true, false, null); - assertTrue("Cont. scheduling", yarnConvertedConfig.getBoolean( - CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_ENABLE, false)); - assertEquals("Scheduling interval", 666, - yarnConvertedConfig.getInt( + assertTrue(yarnConvertedConfig.getBoolean( + CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_ENABLE, false), "Cont. scheduling"); + assertEquals(666 +, yarnConvertedConfig.getInt( "yarn.scheduler.capacity.schedule-asynchronously" + - ".scheduling-interval-ms", -1)); + ".scheduling-interval-ms", -1), "Scheduling interval"); } @Test public void testSiteQueueConfAutoRefreshConversion() { converter.convertSiteProperties(yarnConfig, yarnConvertedConfig, false, false, false, null); - assertTrue(yarnConvertedConfig.get(YarnConfiguration. - RM_SCHEDULER_ENABLE_MONITORS), true); - assertTrue("Scheduling Policies contains queue conf auto refresh", - yarnConvertedConfig. + assertTrue(true, yarnConvertedConfig.get(YarnConfiguration. + RM_SCHEDULER_ENABLE_MONITORS)); + assertTrue( + yarnConvertedConfig. get(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES) .contains(QueueConfigurationAutoRefreshPolicy. - class.getCanonicalName())); + class.getCanonicalName()), "Scheduling Policies contains queue conf auto refresh"); } @Test @@ -94,28 +94,28 @@ public void testSitePreemptionConversion() { converter.convertSiteProperties(yarnConfig, yarnConvertedConfig, false, false, false, null); - assertTrue("Preemption enabled", - yarnConvertedConfig.getBoolean( + assertTrue( + yarnConvertedConfig.getBoolean( YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS, - false)); - assertEquals("Wait time before kill", 123, - yarnConvertedConfig.getInt( + false), "Preemption enabled"); + assertEquals(123 +, yarnConvertedConfig.getInt( CapacitySchedulerConfiguration.PREEMPTION_WAIT_TIME_BEFORE_KILL, - -1)); - assertEquals("Starvation check wait time", 321, - yarnConvertedConfig.getInt( + -1), "Wait time before kill"); + assertEquals(321 +, yarnConvertedConfig.getInt( CapacitySchedulerConfiguration.PREEMPTION_MONITORING_INTERVAL, - -1)); + -1), "Starvation check wait time"); - assertFalse("Observe_only should be false", - yarnConvertedConfig.getBoolean(CapacitySchedulerConfiguration. - PREEMPTION_OBSERVE_ONLY, false)); + assertFalse( + yarnConvertedConfig.getBoolean(CapacitySchedulerConfiguration. + PREEMPTION_OBSERVE_ONLY, false), "Observe_only should be false"); - assertTrue("Should contain ProportionalCapacityPreemptionPolicy.", - yarnConvertedConfig. + assertTrue( + yarnConvertedConfig. get(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES). contains(ProportionalCapacityPreemptionPolicy. - class.getCanonicalName())); + class.getCanonicalName()), "Should contain ProportionalCapacityPreemptionPolicy."); } @Test @@ -125,22 +125,22 @@ public void testSiteDisabledPreemptionWithNoPolicyConversion() { converter.convertSiteProperties(yarnConfig, yarnConvertedConfig, false, false, false, null); - assertFalse("Should not contain ProportionalCapacityPreemptionPolicy.", - yarnConvertedConfig. + assertFalse( + yarnConvertedConfig. get(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES). contains(ProportionalCapacityPreemptionPolicy. - class.getCanonicalName())); + class.getCanonicalName()), "Should not contain ProportionalCapacityPreemptionPolicy."); yarnConfig.setBoolean(FairSchedulerConfiguration.PREEMPTION, false); converter.convertSiteProperties(yarnConfig, yarnConvertedConfig, false, false, false, FSConfigToCSConfigConverterParams.PreemptionMode.NO_POLICY); - assertFalse("Should not contain ProportionalCapacityPreemptionPolicy.", - yarnConvertedConfig. + assertFalse( + yarnConvertedConfig. get(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES). contains(ProportionalCapacityPreemptionPolicy. - class.getCanonicalName())); + class.getCanonicalName()), "Should not contain ProportionalCapacityPreemptionPolicy."); } @Test @@ -150,10 +150,10 @@ public void testSiteAssignMultipleConversion() { converter.convertSiteProperties(yarnConfig, yarnConvertedConfig, false, false, false, null); - assertTrue("Assign multiple", - yarnConvertedConfig.getBoolean( + assertTrue( + yarnConvertedConfig.getBoolean( CapacitySchedulerConfiguration.ASSIGN_MULTIPLE_ENABLED, - false)); + false), "Assign multiple"); } @Test @@ -163,9 +163,9 @@ public void testSiteMaxAssignConversion() { converter.convertSiteProperties(yarnConfig, yarnConvertedConfig, false, false, false, null); - assertEquals("Max assign", 111, - yarnConvertedConfig.getInt( - CapacitySchedulerConfiguration.MAX_ASSIGN_PER_HEARTBEAT, -1)); + assertEquals(111 +, yarnConvertedConfig.getInt( + CapacitySchedulerConfiguration.MAX_ASSIGN_PER_HEARTBEAT, -1), "Max assign"); } @Test @@ -191,9 +191,9 @@ public void testSiteDrfEnabledConversion() { converter.convertSiteProperties(yarnConfig, yarnConvertedConfig, true, false, false, null); - assertEquals("Resource calculator type", DominantResourceCalculator.class, - yarnConvertedConfig.getClass( - CapacitySchedulerConfiguration.RESOURCE_CALCULATOR_CLASS, null)); + assertEquals(DominantResourceCalculator.class +, yarnConvertedConfig.getClass( + CapacitySchedulerConfiguration.RESOURCE_CALCULATOR_CLASS, null), "Resource calculator type"); } @Test @@ -201,10 +201,10 @@ public void testSiteDrfDisabledConversion() { converter.convertSiteProperties(yarnConfig, yarnConvertedConfig, false, false, false, null); - assertEquals("Resource calculator type", DefaultResourceCalculator.class, - yarnConvertedConfig.getClass( + assertEquals(DefaultResourceCalculator.class +, yarnConvertedConfig.getClass( CapacitySchedulerConfiguration.RESOURCE_CALCULATOR_CLASS, - CapacitySchedulerConfiguration.DEFAULT_RESOURCE_CALCULATOR_CLASS)); + CapacitySchedulerConfiguration.DEFAULT_RESOURCE_CALCULATOR_CLASS), "Resource calculator type"); } @Test @@ -212,9 +212,9 @@ public void testAsyncSchedulingEnabledConversion() { converter.convertSiteProperties(yarnConfig, yarnConvertedConfig, true, true, false, null); - assertTrue("Asynchronous scheduling", yarnConvertedConfig.getBoolean( + assertTrue(yarnConvertedConfig.getBoolean( CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_ENABLE, - CapacitySchedulerConfiguration.DEFAULT_SCHEDULE_ASYNCHRONOUSLY_ENABLE)); + CapacitySchedulerConfiguration.DEFAULT_SCHEDULE_ASYNCHRONOUSLY_ENABLE), "Asynchronous scheduling"); } @Test @@ -222,21 +222,21 @@ public void testAsyncSchedulingDisabledConversion() { converter.convertSiteProperties(yarnConfig, yarnConvertedConfig, false, false, false, null); - assertFalse("Asynchronous scheduling", yarnConvertedConfig.getBoolean( + assertFalse(yarnConvertedConfig.getBoolean( CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_ENABLE, - false)); + false), "Asynchronous scheduling"); } @Test public void testSiteQueueAutoDeletionConversionWithWeightMode() { converter.convertSiteProperties(yarnConfig, yarnConvertedConfig, false, false, false, null); - assertTrue(yarnConvertedConfig.get(YarnConfiguration. - RM_SCHEDULER_ENABLE_MONITORS), true); - assertTrue("Scheduling Policies contain auto deletion policy", - yarnConvertedConfig. + assertTrue(true, yarnConvertedConfig.get(YarnConfiguration. + RM_SCHEDULER_ENABLE_MONITORS)); + assertTrue( + yarnConvertedConfig. get(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES) - .contains(DELETION_POLICY_CLASS)); + .contains(DELETION_POLICY_CLASS), "Scheduling Policies contain auto deletion policy"); // Test when policy has existed. yarnConvertedConfig. @@ -244,17 +244,17 @@ public void testSiteQueueAutoDeletionConversionWithWeightMode() { "testPolicy"); converter.convertSiteProperties(yarnConfig, yarnConvertedConfig, false, false, false, null); - assertTrue("Scheduling Policies contain auto deletion policy", - yarnConvertedConfig. + assertTrue( + yarnConvertedConfig. get(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES) - .contains(DELETION_POLICY_CLASS)); + .contains(DELETION_POLICY_CLASS), "Scheduling Policies contain auto deletion policy"); - assertEquals("Auto deletion policy expired time should be 10s", - 10, yarnConvertedConfig. + assertEquals( + 10, yarnConvertedConfig. getLong(CapacitySchedulerConfiguration. AUTO_CREATE_CHILD_QUEUE_EXPIRED_TIME, CapacitySchedulerConfiguration. - DEFAULT_AUTO_CREATE_CHILD_QUEUE_EXPIRED_TIME)); + DEFAULT_AUTO_CREATE_CHILD_QUEUE_EXPIRED_TIME), "Auto deletion policy expired time should be 10s"); } @Test @@ -264,35 +264,34 @@ public void testSiteQueueAutoDeletionConversionWithWeightMode() { // test percentage mode converter.convertSiteProperties(yarnConfig, yarnConvertedConfig, false, false, true, null); - assertTrue(yarnConvertedConfig.get(YarnConfiguration. - RM_SCHEDULER_ENABLE_MONITORS), true); + assertTrue(true, yarnConvertedConfig.get(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS)); - assertTrue("Scheduling Policies should not" + - "contain auto deletion policy in percentage mode", - yarnConvertedConfig. + assertTrue( + yarnConvertedConfig. get(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES) == null || !yarnConvertedConfig. get(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES). - contains(DELETION_POLICY_CLASS)); + contains(DELETION_POLICY_CLASS), "Scheduling Policies should not" + + "contain auto deletion policy in percentage mode"); yarnConvertedConfig. set(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES, "testPolicy"); converter.convertSiteProperties(yarnConfig, yarnConvertedConfig, false, false, true, null); - assertFalse("Scheduling Policies should not " + - "contain auto deletion policy in percentage mode", - yarnConvertedConfig. + assertFalse( + yarnConvertedConfig. get(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES) - .contains(DELETION_POLICY_CLASS)); + .contains(DELETION_POLICY_CLASS), "Scheduling Policies should not " + + "contain auto deletion policy in percentage mode"); - assertNotEquals("Auto deletion policy expired time should not " + - "be set in percentage mode", - 10, yarnConvertedConfig. + assertNotEquals( + 10, yarnConvertedConfig. getLong(CapacitySchedulerConfiguration. AUTO_CREATE_CHILD_QUEUE_EXPIRED_TIME, CapacitySchedulerConfiguration. - DEFAULT_AUTO_CREATE_CHILD_QUEUE_EXPIRED_TIME)); + DEFAULT_AUTO_CREATE_CHILD_QUEUE_EXPIRED_TIME), "Auto deletion policy expired time should not " + + "be set in percentage mode"); } } 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/fair/converter/TestQueuePlacementConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestQueuePlacementConverter.java index 109f0e6270f5e..2c2df54c9dcbe 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestQueuePlacementConverter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestQueuePlacementConverter.java @@ -16,7 +16,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.converter; import static org.apache.hadoop.test.MockitoUtil.verifyZeroInteractions; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.*; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; @@ -47,8 +47,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.placement.schema.Rule.FallbackResult; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.placement.schema.Rule.Policy; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.placement.schema.Rule.Type; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.junit.MockitoJUnitRunner; @@ -71,7 +71,7 @@ public class TestQueuePlacementConverter { private CapacitySchedulerConfiguration csConf; - @Before + @BeforeEach public void setup() { this.converter = new QueuePlacementConverter(); this.csConf = new CapacitySchedulerConfiguration( @@ -84,7 +84,7 @@ public void testConvertUserRule() { initPlacementManagerMock(fsRule); MappingRulesDescription description = convert(); - assertEquals("Number of rules", 1, description.getRules().size()); + assertEquals(1, description.getRules().size(), "Number of rules"); verifyRule(description.getRules().get(0), Policy.USER); verifyZeroInteractions(ruleHandler); } @@ -95,7 +95,7 @@ public void testConvertSpecifiedRule() { initPlacementManagerMock(fsRule); MappingRulesDescription description = convert(); - assertEquals("Number of rules", 1, description.getRules().size()); + assertEquals(1, description.getRules().size(), "Number of rules"); verifyRule(description.getRules().get(0), Policy.SPECIFIED); verifyZeroInteractions(ruleHandler); } @@ -107,7 +107,7 @@ public void testConvertPrimaryGroupRule() { MappingRulesDescription description = convert(); - assertEquals("Number of rules", 1, description.getRules().size()); + assertEquals(1, description.getRules().size(), "Number of rules"); verifyRule(description.getRules().get(0), Policy.PRIMARY_GROUP); verifyZeroInteractions(ruleHandler); } @@ -119,7 +119,7 @@ public void testConvertSecondaryGroupRule() { MappingRulesDescription description = convert(); - assertEquals("Number of rules", 1, description.getRules().size()); + assertEquals(1, description.getRules().size(), "Number of rules"); verifyRule(description.getRules().get(0), Policy.SECONDARY_GROUP); verifyZeroInteractions(ruleHandler); } @@ -132,7 +132,7 @@ public void testConvertDefaultRuleWithQueueName() { MappingRulesDescription description = convert(); - assertEquals("Number of rules", 1, description.getRules().size()); + assertEquals(1, description.getRules().size(), "Number of rules"); verifyRule(description.getRules().get(0), Policy.CUSTOM); verifyZeroInteractions(ruleHandler); @@ -146,18 +146,20 @@ public void testConvertDefaultRule() { MappingRulesDescription description = convert(); - assertEquals("Number of rules", 1, description.getRules().size()); + assertEquals(1, description.getRules().size(), "Number of rules"); verifyRule(description.getRules().get(0), Policy.DEFAULT_QUEUE); verifyZeroInteractions(ruleHandler); } - @Test(expected = IllegalArgumentException.class) + @Test public void testConvertUnsupportedRule() { - PlacementRule rule = mock(TestPlacementRule.class); - initPlacementManagerMock(rule); + assertThrows(IllegalArgumentException.class, ()->{ + PlacementRule rule = mock(TestPlacementRule.class); + initPlacementManagerMock(rule); - // throws exception - convert(); + // throws exception + convert(); + }); } @Test @@ -167,7 +169,7 @@ public void testConvertRejectRule() { MappingRulesDescription description = convert(); - assertEquals("Number of rules", 1, description.getRules().size()); + assertEquals(1, description.getRules().size(), "Number of rules"); verifyRule(description.getRules().get(0), Policy.REJECT); verifyZeroInteractions(ruleHandler); } @@ -181,7 +183,7 @@ public void testConvertNestedPrimaryGroupRule() { MappingRulesDescription description = convert(); - assertEquals("Number of rules", 1, description.getRules().size()); + assertEquals(1, description.getRules().size(), "Number of rules"); verifyRule(description.getRules().get(0), Policy.PRIMARY_GROUP_USER); verifyZeroInteractions(ruleHandler); } @@ -196,7 +198,7 @@ public void testConvertNestedSecondaryGroupRule() { MappingRulesDescription description = convert(); - assertEquals("Number of rules", 1, description.getRules().size()); + assertEquals(1, description.getRules().size(), "Number of rules"); verifyRule(description.getRules().get(0), Policy.SECONDARY_GROUP_USER); verifyZeroInteractions(ruleHandler); } @@ -212,23 +214,25 @@ public void testConvertNestedDefaultRule() { MappingRulesDescription description = convert(); - assertEquals("Number of rules", 1, description.getRules().size()); + assertEquals(1, description.getRules().size(), "Number of rules"); Rule rule = description.getRules().get(0); verifyRule(description.getRules().get(0), Policy.USER); assertEquals("Parent path", "root.abc", rule.getParentQueue()); verifyZeroInteractions(ruleHandler); } - @Test(expected = IllegalArgumentException.class) + @Test public void testUnsupportedNestedParentRule() { - UserPlacementRule fsRule = mock(UserPlacementRule.class); - TestPlacementRule parent = - mock(TestPlacementRule.class); - when(fsRule.getParentRule()).thenReturn(parent); - initPlacementManagerMock(fsRule); + assertThrows(IllegalArgumentException.class, ()->{ + UserPlacementRule fsRule = mock(UserPlacementRule.class); + TestPlacementRule parent = + mock(TestPlacementRule.class); + when(fsRule.getParentRule()).thenReturn(parent); + initPlacementManagerMock(fsRule); - // throws exception - convert(); + // throws exception + convert(); + }); } @Test @@ -242,7 +246,7 @@ public void testConvertMultiplePlacementRules() { MappingRulesDescription description = convert(); - assertEquals("Number of rules", 3, description.getRules().size()); + assertEquals(3, description.getRules().size(), "Number of rules"); verifyRule(description.getRules().get(0), Policy.USER); verifyRule(description.getRules().get(1), Policy.PRIMARY_GROUP); verifyRule(description.getRules().get(2), Policy.SECONDARY_GROUP); @@ -357,7 +361,7 @@ private void testConvertNestedRuleCreateFlagInWeightMode( MappingRulesDescription desc = convertInWeightMode(); Rule rule = desc.getRules().get(0); - assertEquals("Expected create flag", expectedFlagOnRule, rule.getCreate()); + assertEquals(expectedFlagOnRule, rule.getCreate(), "Expected create flag"); if (ruleHandlerShouldBeInvoked) { verify(ruleHandler).handleFSParentAndChildCreateFlagDiff( @@ -493,11 +497,11 @@ private MappingRulesDescription convertInWeightMode() { } private void verifyRule(Rule rule, Policy expectedPolicy) { - assertEquals("Policy type", expectedPolicy, rule.getPolicy()); + assertEquals(expectedPolicy, rule.getPolicy(), "Policy type"); assertEquals("Match string", "*", rule.getMatches()); - assertEquals("Fallback result", - FallbackResult.SKIP, rule.getFallbackResult()); - assertEquals("Type", Type.USER, rule.getType()); + assertEquals( + FallbackResult.SKIP, rule.getFallbackResult(), "Fallback result"); + assertEquals(Type.USER, rule.getType(), "Type"); } private class TestPlacementRule extends FSPlacementRule { 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/fair/converter/weightconversion/TestWeightToPercentageConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/weightconversion/TestWeightToPercentageConverter.java index f1f205dcab03f..a2c90aef9e632 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/weightconversion/TestWeightToPercentageConverter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/weightconversion/TestWeightToPercentageConverter.java @@ -19,9 +19,9 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.converter.weightconversion; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.PREFIX; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.math.BigDecimal; import java.util.HashMap; @@ -31,8 +31,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueue; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestWeightToPercentageConverter extends WeightConverterTestBase { @@ -43,7 +43,7 @@ public class TestWeightToPercentageConverter public static final QueuePath ROOT_B = new QueuePath("root", "b"); public static final QueuePath ROOT_C = new QueuePath("root", "c"); - @Before + @BeforeEach public void setup() { converter = new WeightToPercentConverter(); csConfig = new CapacitySchedulerConfiguration( @@ -55,10 +55,10 @@ public void testSingleWeightConversion() { FSQueue root = createFSQueues(1); converter.convertWeightsForChildQueues(root, csConfig); - assertFalse("Capacity zerosum allowed", - csConfig.getAllowZeroCapacitySum(ROOT)); - assertEquals("root.a capacity", 100.000f, - csConfig.getNonLabeledQueueCapacity(new QueuePath("root.a")), 0.0f); + assertFalse( + csConfig.getAllowZeroCapacitySum(ROOT), "Capacity zerosum allowed"); + assertEquals(100.000f, + csConfig.getNonLabeledQueueCapacity(new QueuePath("root.a")), 0.0f, "root.a capacity"); } @Test @@ -66,8 +66,8 @@ public void testNoChildQueueConversion() { FSQueue root = createFSQueues(); converter.convertWeightsForChildQueues(root, csConfig); - assertEquals("Converted items", 20, - csConfig.getPropsWithPrefix(PREFIX).size()); + assertEquals(20 +, csConfig.getPropsWithPrefix(PREFIX).size(), "Converted items"); } @Test @@ -76,15 +76,15 @@ public void testMultiWeightConversion() { converter.convertWeightsForChildQueues(root, csConfig); - assertEquals("Number of properties", 23, - csConfig.getPropsWithPrefix(PREFIX).size()); + assertEquals(23 +, csConfig.getPropsWithPrefix(PREFIX).size(), "Number of properties"); // this is no fixing - it's the result of BigDecimal rounding - assertEquals("root.a capacity", 16.667f, - csConfig.getNonLabeledQueueCapacity(ROOT_A), 0.0f); - assertEquals("root.b capacity", 33.333f, - csConfig.getNonLabeledQueueCapacity(ROOT_B), 0.0f); - assertEquals("root.c capacity", 50.000f, - csConfig.getNonLabeledQueueCapacity(ROOT_C), 0.0f); + assertEquals(16.667f, + csConfig.getNonLabeledQueueCapacity(ROOT_A), 0.0f, "root.a capacity"); + assertEquals(33.333f, + csConfig.getNonLabeledQueueCapacity(ROOT_B), 0.0f, "root.b capacity"); + assertEquals(50.000f, + csConfig.getNonLabeledQueueCapacity(ROOT_C), 0.0f, "root.c capacity"); } @Test @@ -93,16 +93,16 @@ public void testMultiWeightConversionWhenOfThemIsZero() { converter.convertWeightsForChildQueues(root, csConfig); - assertFalse("Capacity zerosum allowed", - csConfig.getAllowZeroCapacitySum(ROOT)); - assertEquals("Number of properties", 23, - csConfig.getPropsWithPrefix(PREFIX).size()); - assertEquals("root.a capacity", 0.000f, - csConfig.getNonLabeledQueueCapacity(ROOT_A), 0.0f); - assertEquals("root.b capacity", 50.000f, - csConfig.getNonLabeledQueueCapacity(ROOT_B), 0.0f); - assertEquals("root.c capacity", 50.000f, - csConfig.getNonLabeledQueueCapacity(ROOT_C), 0.0f); + assertFalse( + csConfig.getAllowZeroCapacitySum(ROOT), "Capacity zerosum allowed"); + assertEquals(23 +, csConfig.getPropsWithPrefix(PREFIX).size(), "Number of properties"); + assertEquals(0.000f, + csConfig.getNonLabeledQueueCapacity(ROOT_A), 0.0f,"root.a capacity"); + assertEquals(50.000f, + csConfig.getNonLabeledQueueCapacity(ROOT_B), 0.0f, "root.b capacity"); + assertEquals(50.000f, + csConfig.getNonLabeledQueueCapacity(ROOT_C), 0.0f, "root.c capacity"); } @Test @@ -111,16 +111,16 @@ public void testMultiWeightConversionWhenAllOfThemAreZero() { converter.convertWeightsForChildQueues(root, csConfig); - assertEquals("Number of properties", 24, - csConfig.getPropsWithPrefix(PREFIX).size()); - assertTrue("Capacity zerosum allowed", - csConfig.getAllowZeroCapacitySum(ROOT)); - assertEquals("root.a capacity", 0.000f, - csConfig.getNonLabeledQueueCapacity(ROOT_A), 0.0f); - assertEquals("root.b capacity", 0.000f, - csConfig.getNonLabeledQueueCapacity(ROOT_B), 0.0f); - assertEquals("root.c capacity", 0.000f, - csConfig.getNonLabeledQueueCapacity(ROOT_C), 0.0f); + assertEquals(24 +, csConfig.getPropsWithPrefix(PREFIX).size(), "Number of properties"); + assertTrue( + csConfig.getAllowZeroCapacitySum(ROOT), "Capacity zerosum allowed"); + assertEquals(0.000f, + csConfig.getNonLabeledQueueCapacity(ROOT_A), 0.0f, "root.a capacity"); + assertEquals(0.000f, + csConfig.getNonLabeledQueueCapacity(ROOT_B), 0.0f, "root.b capacity"); + assertEquals(0.000f, + csConfig.getNonLabeledQueueCapacity(ROOT_C), 0.0f, "root.c capacity"); } @Test @@ -129,14 +129,14 @@ public void testCapacityFixingWithThreeQueues() { converter.convertWeightsForChildQueues(root, csConfig); - assertEquals("Number of properties", 23, - csConfig.getPropsWithPrefix(PREFIX).size()); - assertEquals("root.a capacity", 33.334f, - csConfig.getNonLabeledQueueCapacity(ROOT_A), 0.0f); - assertEquals("root.b capacity", 33.333f, - csConfig.getNonLabeledQueueCapacity(ROOT_B), 0.0f); - assertEquals("root.c capacity", 33.333f, - csConfig.getNonLabeledQueueCapacity(ROOT_C), 0.0f); + assertEquals(23 +, csConfig.getPropsWithPrefix(PREFIX).size(), "Number of properties"); + assertEquals(33.334f, + csConfig.getNonLabeledQueueCapacity(ROOT_A), 0.0f, "root.a capacity"); + assertEquals(33.333f, + csConfig.getNonLabeledQueueCapacity(ROOT_B), 0.0f, "root.b capacity"); + assertEquals(33.333f, + csConfig.getNonLabeledQueueCapacity(ROOT_C), 0.0f, "root.c capacity"); } @Test @@ -167,12 +167,12 @@ private void testCapacityFixing(Map capacities, converter.fixCapacities(capacities, total); - assertFalse("Capacity zerosum allowed", needCapacityValidationRelax); - assertEquals("root.a capacity", new BigDecimal("50.000"), - capacities.get("root.a")); - assertEquals("root.b capacity", new BigDecimal("25.500"), - capacities.get("root.b")); - assertEquals("root.c capacity", new BigDecimal("25.500"), - capacities.get("root.c")); + assertFalse(needCapacityValidationRelax, "Capacity zerosum allowed"); + assertEquals(new BigDecimal("50.000") +, capacities.get("root.a"), "root.a capacity"); + assertEquals(new BigDecimal("25.500") +, capacities.get("root.b"), "root.b capacity"); + assertEquals(new BigDecimal("25.500") +, capacities.get("root.c"), "root.c capacity"); } } 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/fair/converter/weightconversion/TestWeightToWeightConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/weightconversion/TestWeightToWeightConverter.java index 8992553a86ad3..9942af2622f92 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/weightconversion/TestWeightToWeightConverter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/weightconversion/TestWeightToWeightConverter.java @@ -19,8 +19,8 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.converter.weightconversion; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.PREFIX; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.ArrayList; @@ -28,8 +28,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueue; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestWeightToWeightConverter extends WeightConverterTestBase { private WeightToWeightConverter converter; @@ -40,7 +40,7 @@ public class TestWeightToWeightConverter extends WeightConverterTestBase { public static final QueuePath ROOT_B = new QueuePath("root", "b"); public static final QueuePath ROOT_C = new QueuePath("root", "c"); - @Before + @BeforeEach public void setup() { converter = new WeightToWeightConverter(); csConfig = new CapacitySchedulerConfiguration( @@ -52,10 +52,10 @@ public void testNoChildQueueConversion() { FSQueue root = createFSQueues(); converter.convertWeightsForChildQueues(root, csConfig); - assertEquals("root weight", 1.0f, - csConfig.getNonLabeledQueueWeight(ROOT), 0.0f); - assertEquals("Converted items", 22, - csConfig.getPropsWithPrefix(PREFIX).size()); + assertEquals(1.0f, + csConfig.getNonLabeledQueueWeight(ROOT), 0.0f, "root weight"); + assertEquals(22 +, csConfig.getPropsWithPrefix(PREFIX).size(), "Converted items"); } @Test @@ -63,12 +63,12 @@ public void testSingleWeightConversion() { FSQueue root = createFSQueues(1); converter.convertWeightsForChildQueues(root, csConfig); - assertEquals("root weight", 1.0f, - csConfig.getNonLabeledQueueWeight(ROOT), 0.0f); - assertEquals("root.a weight", 1.0f, - csConfig.getNonLabeledQueueWeight(ROOT_A), 0.0f); - assertEquals("Number of properties", 23, - csConfig.getPropsWithPrefix(PREFIX).size()); + assertEquals(1.0f, + csConfig.getNonLabeledQueueWeight(ROOT), 0.0f, "root weight"); + assertEquals(1.0f, + csConfig.getNonLabeledQueueWeight(ROOT_A), 0.0f, "root.a weight"); + assertEquals(23 +, csConfig.getPropsWithPrefix(PREFIX).size(), "Number of properties"); } @Test @@ -77,16 +77,16 @@ public void testMultiWeightConversion() { converter.convertWeightsForChildQueues(root, csConfig); - assertEquals("Number of properties", 25, - csConfig.getPropsWithPrefix(PREFIX).size()); - assertEquals("root weight", 1.0f, - csConfig.getNonLabeledQueueWeight(ROOT), 0.0f); - assertEquals("root.a weight", 1.0f, - csConfig.getNonLabeledQueueWeight(ROOT_A), 0.0f); - assertEquals("root.b weight", 2.0f, - csConfig.getNonLabeledQueueWeight(ROOT_B), 0.0f); - assertEquals("root.c weight", 3.0f, - csConfig.getNonLabeledQueueWeight(ROOT_C), 0.0f); + assertEquals(25 +, csConfig.getPropsWithPrefix(PREFIX).size(), "Number of properties"); + assertEquals(1.0f, + csConfig.getNonLabeledQueueWeight(ROOT), 0.0f, "root weight"); + assertEquals(1.0f, + csConfig.getNonLabeledQueueWeight(ROOT_A), 0.0f, "root.a weight"); + assertEquals(2.0f, + csConfig.getNonLabeledQueueWeight(ROOT_B), 0.0f, "root.b weight"); + assertEquals(3.0f, + csConfig.getNonLabeledQueueWeight(ROOT_C), 0.0f, "root.c weight"); } @Test @@ -94,8 +94,8 @@ public void testAutoCreateV2FlagOnParent() { FSQueue root = createFSQueues(1); converter.convertWeightsForChildQueues(root, csConfig); - assertTrue("root autocreate v2 enabled", - csConfig.isAutoQueueCreationV2Enabled(ROOT)); + assertTrue( + csConfig.isAutoQueueCreationV2Enabled(ROOT), "root autocreate v2 enabled"); } @Test @@ -103,9 +103,9 @@ public void testAutoCreateV2FlagOnParentWithoutChildren() { FSQueue root = createParent(new ArrayList<>()); converter.convertWeightsForChildQueues(root, csConfig); - assertEquals("Number of properties", 22, - csConfig.getPropsWithPrefix(PREFIX).size()); - assertTrue("root autocreate v2 enabled", - csConfig.isAutoQueueCreationV2Enabled(ROOT)); + assertEquals(22 +, csConfig.getPropsWithPrefix(PREFIX).size(), "Number of properties"); + assertTrue( + csConfig.isAutoQueueCreationV2Enabled(ROOT), "root autocreate v2 enabled"); } } 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/fair/policies/TestDominantResourceFairnessPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/TestDominantResourceFairnessPolicy.java index 1cb25242b15bc..0af915c8d7154 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/TestDominantResourceFairnessPolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/TestDominantResourceFairnessPolicy.java @@ -17,9 +17,9 @@ */ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -41,16 +41,16 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy.DominantResourceFairnessComparator2; import org.apache.hadoop.yarn.util.resource.ResourceUtils; 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; /** * comparator.compare(sched1, sched2) < 0 means that sched1 should get a * container before sched2 */ public class TestDominantResourceFairnessPolicy { - @Before + @BeforeEach public void setup() { addResources("test"); } @@ -96,8 +96,8 @@ public void testSameDominantResource() { Schedulable s1 = createSchedulable(1000, 1); Schedulable s2 = createSchedulable(2000, 1); - assertTrue("Comparison didn't return a value less than 0", - c.compare(s1, s2) < 0); + assertTrue( + c.compare(s1, s2) < 0, "Comparison didn't return a value less than 0"); } @Test @@ -112,8 +112,8 @@ public void testDifferentDominantResource() { Schedulable s1 = createSchedulable(4000, 3); Schedulable s2 = createSchedulable(2000, 5); - assertTrue("Comparison didn't return a value less than 0", - c.compare(s1, s2) < 0); + assertTrue( + c.compare(s1, s2) < 0, "Comparison didn't return a value less than 0"); } @Test @@ -128,8 +128,8 @@ public void testOneIsNeedy() { Schedulable s1 = createSchedulable(2000, 5, 0, 6); Schedulable s2 = createSchedulable(4000, 3, 0, 0); - assertTrue("Comparison didn't return a value less than 0", - c.compare(s1, s2) < 0); + assertTrue( + c.compare(s1, s2) < 0, "Comparison didn't return a value less than 0"); } @Test @@ -146,16 +146,16 @@ public void testBothAreNeedy() { // dominant share is 4000/8000 Schedulable s2 = createSchedulable(4000, 3); - assertTrue("Comparison didn't return a value less than 0", - c.compare(s1, s2) < 0); + assertTrue( + c.compare(s1, s2) < 0, "Comparison didn't return a value less than 0"); // dominant min share is 2/3 s1 = createSchedulable(2000, 5, 3000, 6); // dominant min share is 4/5 s2 = createSchedulable(4000, 3, 5000, 4); - assertTrue("Comparison didn't return a value less than 0", - c.compare(s1, s2) < 0); + assertTrue( + c.compare(s1, s2) < 0, "Comparison didn't return a value less than 0"); } @Test @@ -208,12 +208,12 @@ public void testSortShares() { comparator.sortRatios(ratios1, ratios2); for (int i = 0; i < ratios1.length; i++) { - Assert.assertArrayEquals("The shares array was not sorted into the " - + "expected order: incorrect inner array encountered", - expected1[i], ratios1[i], 0.00001f); - Assert.assertArrayEquals("The shares array was not sorted into the " - + "expected order: incorrect inner array encountered", - expected2[i], ratios2[i], 0.00001f); + Assertions.assertArrayEquals( + expected1[i], ratios1[i], 0.00001f, "The shares array was not sorted into the " + + "expected order: incorrect inner array encountered"); + Assertions.assertArrayEquals( + expected2[i], ratios2[i], 0.00001f, "The shares array was not sorted into the " + + "expected order: incorrect inner array encountered"); } } @@ -232,17 +232,18 @@ public void testCalculateClusterAndFairRatios() { int dominant = comparator.calculateClusterAndFairRatios(used, capacity, shares, 1.0f); - assertEquals("Calculated usage ratio for memory (10MB out of 100MB) is " - + "incorrect", 0.1, - shares[index.get(ResourceInformation.MEMORY_MB.getName())][0], .00001); - assertEquals("Calculated usage ratio for vcores (5 out of 10) is " - + "incorrect", 0.5, - shares[index.get(ResourceInformation.VCORES.getName())][0], .00001); - assertEquals("Calculated usage ratio for test resource (2 out of 5) is " - + "incorrect", 0.4, shares[index.get("test")][0], .00001); - assertEquals("The wrong dominant resource index was returned", - index.get(ResourceInformation.VCORES.getName()).intValue(), - dominant); + assertEquals(0.1, + shares[index.get(ResourceInformation.MEMORY_MB.getName())][0], .00001, + "Calculated usage ratio for memory (10MB out of 100MB) is " + + "incorrect"); + assertEquals(0.5, + shares[index.get(ResourceInformation.VCORES.getName())][0], .00001, "Calculated usage ratio for vcores (5 out of 10) is " + + "incorrect"); + assertEquals( 0.4, shares[index.get("test")][0], .00001, "Calculated usage ratio for test resource (2 out of 5) is " + + "incorrect"); + assertEquals( + index.get(ResourceInformation.VCORES.getName()).intValue() +, dominant, "The wrong dominant resource index was returned"); } @Test @@ -257,12 +258,12 @@ public void testCalculateClusterAndFairRatios2() { comparator.calculateClusterAndFairRatios(used.getResources(), 1.0f, capacity.getResources(), shares); - assertEquals("Calculated usage ratio for memory (10MB out of 100MB) is " - + "incorrect", 0.1, shares[Resource.MEMORY_INDEX], .00001); - assertEquals("Calculated usage ratio for vcores (5 out of 10) is " - + "incorrect", 0.5, shares[Resource.VCORES_INDEX], .00001); - assertEquals("The wrong dominant resource index was returned", - Resource.VCORES_INDEX, dominant); + assertEquals( 0.1, shares[Resource.MEMORY_INDEX], .00001, "Calculated usage ratio for memory (10MB out of 100MB) is " + + "incorrect"); + assertEquals(0.5, shares[Resource.VCORES_INDEX], .00001, "Calculated usage ratio for vcores (5 out of 10) is " + + "incorrect"); + assertEquals( + Resource.VCORES_INDEX, dominant, "The wrong dominant resource index was returned"); } @Test @@ -279,15 +280,15 @@ public void testCalculateMinShareRatios() { comparator.calculateMinShareRatios(used, minShares, ratios); - assertEquals("Calculated min share ratio for memory (10MB out of 5MB) is " - + "incorrect", 2.0, - ratios[index.get(ResourceInformation.MEMORY_MB.getName())][2], .00001f); - assertEquals("Calculated min share ratio for vcores (5 out of 10) is " - + "incorrect", 0.5, - ratios[index.get(ResourceInformation.VCORES.getName())][2], .00001f); - assertEquals("Calculated min share ratio for test resource (0 out of 5) is " - + "incorrect", Float.POSITIVE_INFINITY, ratios[index.get("test")][2], - 0.00001f); + assertEquals(2.0, + ratios[index.get(ResourceInformation.MEMORY_MB.getName())][2], .00001f, "Calculated min share ratio for memory (10MB out of 5MB) is " + + "incorrect"); + assertEquals(0.5, + ratios[index.get(ResourceInformation.VCORES.getName())][2], .00001f, "Calculated min share ratio for vcores (5 out of 10) is " + + "incorrect"); + assertEquals(Float.POSITIVE_INFINITY, ratios[index.get("test")][2], + 0.00001f, "Calculated min share ratio for test resource (0 out of 5) is " + + "incorrect"); } @Test @@ -302,10 +303,12 @@ public void testCalculateMinShareRatios2() { comparator.calculateMinShareRatios(used.getResources(), minShares.getResources()); - assertEquals("Calculated min share ratio for memory (10MB out of 5MB) is " - + "incorrect", 2.0, ratios[Resource.MEMORY_INDEX], .00001f); - assertEquals("Calculated min share ratio for vcores (5 out of 10) is " - + "incorrect", 0.5, ratios[Resource.VCORES_INDEX], .00001f); + assertEquals(2.0, ratios[Resource.MEMORY_INDEX], .00001f, + "Calculated min share ratio for memory (10MB out of 5MB) is " + + "incorrect"); + assertEquals( 0.5, ratios[Resource.VCORES_INDEX], .00001f, + "Calculated min share ratio for vcores (5 out of 10) is " + + "incorrect"); } @Test @@ -330,96 +333,96 @@ public void testCompareShares() { int ret = comparator.compareRatios(ratios1, ratios2, 0); - assertEquals("Expected the first array to be larger because the first " - + "usage ratio element is larger", 1, ret); + assertEquals(1, ret, "Expected the first array to be larger because the first " + + "usage ratio element is larger"); ret = comparator.compareRatios(ratios2, ratios1, 0); - assertEquals("Expected the first array to be smaller because the first " - + "usage ratio element is smaller", -1, ret); + assertEquals(-1, ret, "Expected the first array to be smaller because the first " + + "usage ratio element is smaller"); ret = comparator.compareRatios(ratios1, ratios1, 0); - assertEquals("Expected the arrays to be equal, since they're the same " - + "array", 0, ret); + assertEquals(0, ret, "Expected the arrays to be equal, since they're the same " + + "array"); ret = comparator.compareRatios(ratios2, ratios2, 0); - assertEquals("Expected the arrays to be equal, since they're the same " - + "array", 0, ret); + assertEquals(0, ret, "Expected the arrays to be equal, since they're the same " + + "array"); ret = comparator.compareRatios(ratios3, ratios3, 0); - assertEquals("Expected the arrays to be equal, since they're the same " - + "array", 0, ret); + assertEquals(0, ret, "Expected the arrays to be equal, since they're the same " + + "array"); ret = comparator.compareRatios(ratios2, ratios3, 0); - assertEquals("Expected the first array to be larger because the last " - + "usage ratio element is larger, and all other elements are equal", - 1, ret); + assertEquals( + 1, ret, "Expected the first array to be larger because the last " + + "usage ratio element is larger, and all other elements are equal"); ret = comparator.compareRatios(ratios1, ratios2, 1); - assertEquals("Expected the first array to be smaller because the first " - + "fair share ratio element is smaller", -1, ret); + assertEquals(-1, ret, "Expected the first array to be smaller because the first " + + "fair share ratio element is smaller"); ret = comparator.compareRatios(ratios2, ratios1, 1); - assertEquals("Expected the first array to be larger because the first " - + "fair share ratio element is larger", 1, ret); + assertEquals(1, ret, "Expected the first array to be larger because the first " + + "fair share ratio element is larger"); ret = comparator.compareRatios(ratios1, ratios1, 1); - assertEquals("Expected the arrays to be equal, since they're the same " - + "array", 0, ret); + assertEquals(0, ret, "Expected the arrays to be equal, since they're the same " + + "array"); ret = comparator.compareRatios(ratios2, ratios2, 1); - assertEquals("Expected the arrays to be equal, since they're the same " - + "array", 0, ret); + assertEquals(0, ret, "Expected the arrays to be equal, since they're the same " + + "array"); ret = comparator.compareRatios(ratios3, ratios3, 1); - assertEquals("Expected the arrays to be equal, since they're the same " - + "array", 0, ret); + assertEquals(0, ret, "Expected the arrays to be equal, since they're the same " + + "array"); ret = comparator.compareRatios(ratios2, ratios3, 1); - assertEquals("Expected the first array to be smaller because the last " - + "usage ratio element is smaller, and all other elements are equal", - -1, ret); + assertEquals( + -1, ret, "Expected the first array to be smaller because the last " + + "usage ratio element is smaller, and all other elements are equal"); ret = comparator.compareRatios(ratios1, ratios2, 2); - assertEquals("Expected the first array to be larger because the first " - + "min share ratio element is larger", 1, ret); + assertEquals(1, ret, "Expected the first array to be larger because the first " + + "min share ratio element is larger"); ret = comparator.compareRatios(ratios2, ratios1, 2); - assertEquals("Expected the first array to be smaller because the first " - + "min share ratio element is smaller", -1, ret); + assertEquals(-1, ret, "Expected the first array to be smaller because the first " + + "min share ratio element is smaller"); ret = comparator.compareRatios(ratios1, ratios1, 2); - assertEquals("Expected the arrays to be equal, since they're the same " - + "array", 0, ret); + assertEquals(0, ret, "Expected the arrays to be equal, since they're the same " + + "array"); ret = comparator.compareRatios(ratios2, ratios2, 2); - assertEquals("Expected the arrays to be equal, since they're the same " - + "array", 0, ret); + assertEquals(0, ret, "Expected the arrays to be equal, since they're the same " + + "array"); ret = comparator.compareRatios(ratios3, ratios3, 2); - assertEquals("Expected the arrays to be equal, since they're the same " - + "array", 0, ret); + assertEquals(0, ret, "Expected the arrays to be equal, since they're the same " + + "array"); ret = comparator.compareRatios(ratios2, ratios3, 2); - assertEquals("Expected the first array to be smaller because the second " + assertEquals(-1, ret, "Expected the first array to be smaller because the second " + "min share ratio element is smaller, and all the first elements are " - + "equal", -1, ret); + + "equal"); } @Test 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/fair/policies/TestEmptyQueues.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/TestEmptyQueues.java index 7511cecc68a19..12f2015076f9d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/TestEmptyQueues.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/TestEmptyQueues.java @@ -21,8 +21,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.Schedulable; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.SchedulingPolicy; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.ArrayList; import java.util.Collection; @@ -30,7 +31,7 @@ public class TestEmptyQueues { private Collection schedulables; - @Before + @BeforeEach public void setup() { schedulables = new ArrayList(); } @@ -39,17 +40,20 @@ private void testComputeShares(SchedulingPolicy policy) { policy.computeShares(schedulables, Resources.none()); } - @Test (timeout = 10000) + @Test + @Timeout(value = 10) public void testFifoPolicy() { testComputeShares(SchedulingPolicy.getInstance(FifoPolicy.class)); } - @Test (timeout = 10000) + @Test + @Timeout(value = 10) public void testFairSharePolicy() { testComputeShares(SchedulingPolicy.getInstance(FairSharePolicy.class)); } - @Test (timeout = 10000) + @Test + @Timeout(value = 10) public void testDRFPolicy() { testComputeShares( SchedulingPolicy.getInstance(DominantResourceFairnessPolicy.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/scheduler/fifo/TestFifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java index 682a01d985fa4..8499c35e96b2f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java @@ -20,9 +20,9 @@ import static org.apache.hadoop.yarn.server.resourcemanager.MockNM.createMockNodeStatus; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -114,10 +114,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager; import org.apache.hadoop.yarn.server.utils.BuilderUtils; 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.mockito.Mockito; public class TestFifoScheduler { @@ -133,7 +134,7 @@ public class TestFifoScheduler { private final static ContainerUpdates NULL_UPDATE_REQUESTS = new ContainerUpdates(); - @Before + @BeforeEach public void setUp() throws Exception { conf = new Configuration(); conf.setClass(YarnConfiguration.RM_SCHEDULER, @@ -141,7 +142,7 @@ public void setUp() throws Exception { resourceManager = new MockRM(conf); } - @After + @AfterEach public void tearDown() throws Exception { resourceManager.stop(); } @@ -179,14 +180,16 @@ private ResourceRequest createResourceRequest(int memory, String host, return request; } - @Test(timeout=5000) + @Test + @Timeout(value = 5) public void testFifoSchedulerCapacityWhenNoNMs() { FifoScheduler scheduler = new FifoScheduler(); QueueInfo queueInfo = scheduler.getQueueInfo(null, false, false); - Assert.assertEquals(0.0f, queueInfo.getCurrentCapacity(), 0.0f); + Assertions.assertEquals(0.0f, queueInfo.getCurrentCapacity(), 0.0f); } - @Test(timeout=5000) + @Test + @Timeout(value = 5) public void testAppAttemptMetrics() throws Exception { AsyncDispatcher dispatcher = new InlineDispatcher(); @@ -223,11 +226,12 @@ public void testAppAttemptMetrics() throws Exception { scheduler.handle(attemptEvent2); int afterAppsSubmitted = metrics.getAppsSubmitted(); - Assert.assertEquals(1, afterAppsSubmitted - beforeAppsSubmitted); + Assertions.assertEquals(1, afterAppsSubmitted - beforeAppsSubmitted); scheduler.stop(); } - @Test(timeout=2000) + @Test + @Timeout(value = 2) public void testNodeLocalAssignment() throws Exception { AsyncDispatcher dispatcher = new InlineDispatcher(); Configuration conf = new Configuration(); @@ -294,20 +298,21 @@ public void testNodeLocalAssignment() throws Exception { NodeUpdateSchedulerEvent node0Update = new NodeUpdateSchedulerEvent(node0); // Before the node update event, there are 3 local requests outstanding - Assert.assertEquals(3, nodeLocal.getNumContainers()); + Assertions.assertEquals(3, nodeLocal.getNumContainers()); scheduler.handle(node0Update); // After the node update event, check that there are no more local requests // outstanding - Assert.assertEquals(0, nodeLocal.getNumContainers()); + Assertions.assertEquals(0, nodeLocal.getNumContainers()); //Also check that the containers were scheduled SchedulerAppReport info = scheduler.getSchedulerAppInfo(appAttemptId); - Assert.assertEquals(3, info.getLiveContainers().size()); + Assertions.assertEquals(3, info.getLiveContainers().size()); scheduler.stop(); } - @Test(timeout=2000) + @Test + @Timeout(value = 2) public void testUpdateResourceOnNode() throws Exception { AsyncDispatcher dispatcher = new InlineDispatcher(); Configuration conf = new Configuration(); @@ -356,7 +361,7 @@ public void testUpdateResourceOnNode() throws Exception { assertEquals(1024, scheduler.getNodeTracker().getNode(node0.getNodeID()). getUnallocatedResource().getMemorySize(), 1024); QueueInfo queueInfo = scheduler.getQueueInfo(null, false, false); - Assert.assertEquals(0.0f, queueInfo.getCurrentCapacity(), 0.0f); + Assertions.assertEquals(0.0f, queueInfo.getCurrentCapacity(), 0.0f); int _appId = 1; int _appAttemptId = 1; @@ -389,20 +394,20 @@ public void testUpdateResourceOnNode() throws Exception { null, null, NULL_UPDATE_REQUESTS); // Before the node update event, there are one local request - Assert.assertEquals(1, nodeLocal.getNumContainers()); + Assertions.assertEquals(1, nodeLocal.getNumContainers()); NodeUpdateSchedulerEvent node0Update = new NodeUpdateSchedulerEvent(node0); // Now schedule. scheduler.handle(node0Update); // After the node update event, check no local request - Assert.assertEquals(0, nodeLocal.getNumContainers()); + Assertions.assertEquals(0, nodeLocal.getNumContainers()); // Also check that one container was scheduled SchedulerAppReport info = scheduler.getSchedulerAppInfo(appAttemptId); - Assert.assertEquals(1, info.getLiveContainers().size()); + Assertions.assertEquals(1, info.getLiveContainers().size()); // And check the default Queue now is full. queueInfo = scheduler.getQueueInfo(null, false, false); - Assert.assertEquals(1.0f, queueInfo.getCurrentCapacity(), 0.0f); + Assertions.assertEquals(1.0f, queueInfo.getCurrentCapacity(), 0.0f); } // @Test @@ -627,7 +632,7 @@ public void testGetAppsInQueue() throws Exception { assertTrue(appsInDefault.contains(application_1.getApplicationAttemptId())); assertEquals(2, appsInDefault.size()); - Assert.assertNull(scheduler.getAppsInQueue("someotherqueue")); + Assertions.assertNull(scheduler.getAppsInQueue("someotherqueue")); } @Test @@ -644,7 +649,8 @@ public void testAddAndRemoveAppFromFiFoScheduler() throws Exception { fs.getSchedulerApplications(), fs, "queue"); } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testConfValidation() throws Exception { FifoScheduler scheduler = new FifoScheduler(); Configuration conf = new YarnConfiguration(); @@ -656,12 +662,13 @@ public void testConfValidation() throws Exception { + " larger than the max memory allocation."); } catch (YarnRuntimeException e) { // Exception is expected. - assertTrue("The thrown exception is not the expected one.", e - .getMessage().startsWith("Invalid resource scheduler memory")); + assertTrue(e + .getMessage().startsWith("Invalid resource scheduler memory"), "The thrown exception is not the expected one."); } } - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testAllocateContainerOnNodeWithoutOffSwitchSpecified() throws Exception { GenericTestUtils.setRootLogLevel(Level.DEBUG); @@ -687,13 +694,14 @@ public void testAllocateContainerOnNodeWithoutOffSwitchSpecified() // kick the schedule nm1.nodeHeartbeat(true); } catch (NullPointerException e) { - Assert.fail("NPE when allocating container on node but " + Assertions.fail("NPE when allocating container on node but " + "forget to set off-switch request should be handled"); } rm.stop(); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testFifoScheduling() throws Exception { GenericTestUtils.setRootLogLevel(Level.DEBUG); MockRM rm = new MockRM(conf); @@ -709,7 +717,7 @@ public void testFifoScheduling() throws Exception { am1.registerAppAttempt(); SchedulerNodeReport report_nm1 = rm.getResourceScheduler().getNodeReport(nm1.getNodeId()); - Assert.assertEquals(2 * GB, report_nm1.getUsedResource().getMemorySize()); + Assertions.assertEquals(2 * GB, report_nm1.getUsedResource().getMemorySize()); RMApp app2 = MockRMAppSubmitter.submitWithMemory(2048, rm); // kick the scheduling, 2GB given to AM, remaining 2 GB on nm2 @@ -719,7 +727,7 @@ public void testFifoScheduling() throws Exception { am2.registerAppAttempt(); SchedulerNodeReport report_nm2 = rm.getResourceScheduler().getNodeReport(nm2.getNodeId()); - Assert.assertEquals(2 * GB, report_nm2.getUsedResource().getMemorySize()); + Assertions.assertEquals(2 * GB, report_nm2.getUsedResource().getMemorySize()); // add request for containers am1.addRequests(new String[] { "127.0.0.1", "127.0.0.2" }, GB, 1, 1); @@ -744,25 +752,25 @@ public void testFifoScheduling() throws Exception { nm2.nodeHeartbeat(true); List allocated1 = alloc1Response.getAllocatedContainers(); - Assert.assertEquals(1, allocated1.size()); - Assert.assertEquals(1 * GB, allocated1.get(0).getResource().getMemorySize()); - Assert.assertEquals(nm1.getNodeId(), allocated1.get(0).getNodeId()); + Assertions.assertEquals(1, allocated1.size()); + Assertions.assertEquals(1 * GB, allocated1.get(0).getResource().getMemorySize()); + Assertions.assertEquals(nm1.getNodeId(), allocated1.get(0).getNodeId()); List allocated2 = alloc2Response.getAllocatedContainers(); - Assert.assertEquals(1, allocated2.size()); - Assert.assertEquals(3 * GB, allocated2.get(0).getResource().getMemorySize()); - Assert.assertEquals(nm1.getNodeId(), allocated2.get(0).getNodeId()); + Assertions.assertEquals(1, allocated2.size()); + Assertions.assertEquals(3 * GB, allocated2.get(0).getResource().getMemorySize()); + Assertions.assertEquals(nm1.getNodeId(), allocated2.get(0).getNodeId()); report_nm1 = rm.getResourceScheduler().getNodeReport(nm1.getNodeId()); report_nm2 = rm.getResourceScheduler().getNodeReport(nm2.getNodeId()); - Assert.assertEquals(0, report_nm1.getAvailableResource().getMemorySize()); - Assert.assertEquals(2 * GB, report_nm2.getAvailableResource().getMemorySize()); + Assertions.assertEquals(0, report_nm1.getAvailableResource().getMemorySize()); + Assertions.assertEquals(2 * GB, report_nm2.getAvailableResource().getMemorySize()); - Assert.assertEquals(6 * GB, report_nm1.getUsedResource().getMemorySize()); - Assert.assertEquals(2 * GB, report_nm2.getUsedResource().getMemorySize()); + Assertions.assertEquals(6 * GB, report_nm1.getUsedResource().getMemorySize()); + Assertions.assertEquals(2 * GB, report_nm2.getUsedResource().getMemorySize()); Container c1 = allocated1.get(0); - Assert.assertEquals(GB, c1.getResource().getMemorySize()); + Assertions.assertEquals(GB, c1.getResource().getMemorySize()); ContainerStatus containerStatus = BuilderUtils.newContainerStatus(c1.getId(), ContainerState.COMPLETE, "", 0, c1.getResource()); @@ -773,16 +781,17 @@ public void testFifoScheduling() throws Exception { + waitCount + " times already.."); Thread.sleep(1000); } - Assert.assertEquals(1, attempt1.getJustFinishedContainers().size()); - Assert.assertEquals(1, am1.schedule().getCompletedContainersStatuses() + Assertions.assertEquals(1, attempt1.getJustFinishedContainers().size()); + Assertions.assertEquals(1, am1.schedule().getCompletedContainersStatuses() .size()); report_nm1 = rm.getResourceScheduler().getNodeReport(nm1.getNodeId()); - Assert.assertEquals(5 * GB, report_nm1.getUsedResource().getMemorySize()); + Assertions.assertEquals(5 * GB, report_nm1.getUsedResource().getMemorySize()); rm.stop(); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testNodeUpdateBeforeAppAttemptInit() throws Exception { FifoScheduler scheduler = new FifoScheduler(); MockRM rm = new MockRM(conf); @@ -803,7 +812,7 @@ public void testNodeUpdateBeforeAppAttemptInit() throws Exception { try { scheduler.handle(updateEvent); } catch (NullPointerException e) { - Assert.fail(); + Assertions.fail(); } ApplicationAttemptId attId = ApplicationAttemptId.newInstance(appId, 1); @@ -834,19 +843,21 @@ private void testMinimumAllocation(YarnConfiguration conf, int testAlloc) int checkAlloc = conf.getInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB); - Assert.assertEquals(checkAlloc, report_nm1.getUsedResource().getMemorySize()); + Assertions.assertEquals(checkAlloc, report_nm1.getUsedResource().getMemorySize()); rm.stop(); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testDefaultMinimumAllocation() throws Exception { // Test with something lesser than default testMinimumAllocation(new YarnConfiguration(TestFifoScheduler.conf), YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB / 2); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testNonDefaultMinimumAllocation() throws Exception { // Set custom min-alloc to test tweaking it int allocMB = 1536; @@ -858,7 +869,8 @@ public void testNonDefaultMinimumAllocation() throws Exception { testMinimumAllocation(conf, allocMB / 2); } - @Test(timeout = 50000) + @Test + @Timeout(value = 50) public void testReconnectedNode() throws Exception { CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); QueuePath defaultQueuePath = new QueuePath("default"); @@ -880,7 +892,7 @@ public void testReconnectedNode() throws Exception { fs.handle(new NodeAddedSchedulerEvent(n1)); fs.handle(new NodeAddedSchedulerEvent(n2)); fs.handle(new NodeUpdateSchedulerEvent(n1)); - Assert.assertEquals(6 * GB, fs.getRootQueueMetrics().getAvailableMB()); + Assertions.assertEquals(6 * GB, fs.getRootQueueMetrics().getAvailableMB()); // reconnect n1 with downgraded memory n1 = @@ -889,11 +901,12 @@ public void testReconnectedNode() throws Exception { fs.handle(new NodeAddedSchedulerEvent(n1)); fs.handle(new NodeUpdateSchedulerEvent(n1)); - Assert.assertEquals(4 * GB, fs.getRootQueueMetrics().getAvailableMB()); + Assertions.assertEquals(4 * GB, fs.getRootQueueMetrics().getAvailableMB()); fs.stop(); } - @Test(timeout = 50000) + @Test + @Timeout(value = 50) public void testBlackListNodes() throws Exception { Configuration conf = new Configuration(); @@ -974,18 +987,18 @@ public void testBlackListNodes() throws Exception { fs.allocate(appAttemptId1, emptyAsk, null, emptyId, null, null, NULL_UPDATE_REQUESTS); - Assert.assertEquals("allocation1", 0, allocation1.getContainers().size()); + Assertions.assertEquals(0, allocation1.getContainers().size(), "allocation1"); // verify host_1_1 can get allocated as not in blacklist fs.handle(new NodeUpdateSchedulerEvent(n4)); Allocation allocation2 = fs.allocate(appAttemptId1, emptyAsk, null, emptyId, null, null, NULL_UPDATE_REQUESTS); - Assert.assertEquals("allocation2", 1, allocation2.getContainers().size()); + Assertions.assertEquals(1, allocation2.getContainers().size(), "allocation2"); List containerList = allocation2.getContainers(); for (Container container : containerList) { - Assert.assertEquals("Container is allocated on n4", - container.getNodeId(), n4.getNodeID()); + Assertions.assertEquals( + container.getNodeId(), n4.getNodeID(), "Container is allocated on n4"); } // Ask for a 1 GB container again for app 1 @@ -1002,38 +1015,39 @@ public void testBlackListNodes() throws Exception { Allocation allocation3 = fs.allocate(appAttemptId1, emptyAsk, null, emptyId, null, null, NULL_UPDATE_REQUESTS); - Assert.assertEquals("allocation3", 0, allocation3.getContainers().size()); + Assertions.assertEquals(0, allocation3.getContainers().size(), "allocation3"); // verify n2 is not qualified to be allocated fs.handle(new NodeUpdateSchedulerEvent(n2)); Allocation allocation4 = fs.allocate(appAttemptId1, emptyAsk, null, emptyId, null, null, NULL_UPDATE_REQUESTS); - Assert.assertEquals("allocation4", 0, allocation4.getContainers().size()); + Assertions.assertEquals(0, allocation4.getContainers().size(), "allocation4"); // verify n3 is not qualified to be allocated fs.handle(new NodeUpdateSchedulerEvent(n3)); Allocation allocation5 = fs.allocate(appAttemptId1, emptyAsk, null, emptyId, null, null, NULL_UPDATE_REQUESTS); - Assert.assertEquals("allocation5", 0, allocation5.getContainers().size()); + Assertions.assertEquals(0, allocation5.getContainers().size(), "allocation5"); fs.handle(new NodeUpdateSchedulerEvent(n4)); Allocation allocation6 = fs.allocate(appAttemptId1, emptyAsk, null, emptyId, null, null, NULL_UPDATE_REQUESTS); - Assert.assertEquals("allocation6", 1, allocation6.getContainers().size()); + Assertions.assertEquals(1, allocation6.getContainers().size(), "allocation6"); containerList = allocation6.getContainers(); for (Container container : containerList) { - Assert.assertEquals("Container is allocated on n4", - container.getNodeId(), n4.getNodeID()); + Assertions.assertEquals( + container.getNodeId(), n4.getNodeID(), "Container is allocated on n4"); } rm.stop(); } - @Test(timeout = 50000) + @Test + @Timeout(value = 50) public void testHeadroom() throws Exception { Configuration conf = new Configuration(); @@ -1097,19 +1111,20 @@ public void testHeadroom() throws Exception { Allocation allocation1 = fs.allocate(appAttemptId1, emptyAsk, null, emptyId, null, null, NULL_UPDATE_REQUESTS); - Assert.assertEquals("Allocation headroom", 1 * GB, allocation1 - .getResourceLimit().getMemorySize()); + Assertions.assertEquals(1 * GB, allocation1 + .getResourceLimit().getMemorySize(), "Allocation headroom"); Allocation allocation2 = fs.allocate(appAttemptId2, emptyAsk, null, emptyId, null, null, NULL_UPDATE_REQUESTS); - Assert.assertEquals("Allocation headroom", 1 * GB, allocation2 - .getResourceLimit().getMemorySize()); + Assertions.assertEquals(1 * GB, allocation2 + .getResourceLimit().getMemorySize(), "Allocation headroom"); rm.stop(); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void testResourceOverCommit() throws Exception { int waitCount; MockRM rm = new MockRM(conf); @@ -1126,8 +1141,8 @@ public void testResourceOverCommit() throws Exception { SchedulerNodeReport report_nm1 = rm.getResourceScheduler().getNodeReport(nm1.getNodeId()); // check node report, 2 GB used and 2 GB available - Assert.assertEquals(2 * GB, report_nm1.getUsedResource().getMemorySize()); - Assert.assertEquals(2 * GB, report_nm1.getAvailableResource().getMemorySize()); + Assertions.assertEquals(2 * GB, report_nm1.getUsedResource().getMemorySize()); + Assertions.assertEquals(2 * GB, report_nm1.getAvailableResource().getMemorySize()); // add request for containers am1.addRequests(new String[] { "127.0.0.1", "127.0.0.2" }, 2 * GB, 1, 1); @@ -1142,18 +1157,18 @@ public void testResourceOverCommit() throws Exception { } List allocated1 = alloc1Response.getAllocatedContainers(); - Assert.assertEquals(1, allocated1.size()); - Assert.assertEquals(2 * GB, allocated1.get(0).getResource().getMemorySize()); - Assert.assertEquals(nm1.getNodeId(), allocated1.get(0).getNodeId()); + Assertions.assertEquals(1, allocated1.size()); + Assertions.assertEquals(2 * GB, allocated1.get(0).getResource().getMemorySize()); + Assertions.assertEquals(nm1.getNodeId(), allocated1.get(0).getNodeId()); report_nm1 = rm.getResourceScheduler().getNodeReport(nm1.getNodeId()); // check node report, 4 GB used and 0 GB available - Assert.assertEquals(0, report_nm1.getAvailableResource().getMemorySize()); - Assert.assertEquals(4 * GB, report_nm1.getUsedResource().getMemorySize()); + Assertions.assertEquals(0, report_nm1.getAvailableResource().getMemorySize()); + Assertions.assertEquals(4 * GB, report_nm1.getUsedResource().getMemorySize()); // check container is assigned with 2 GB. Container c1 = allocated1.get(0); - Assert.assertEquals(2 * GB, c1.getResource().getMemorySize()); + Assertions.assertEquals(2 * GB, c1.getResource().getMemorySize()); // update node resource to 2 GB, so resource is over-consumed. Map nodeResourceMap = @@ -1178,8 +1193,8 @@ public void testResourceOverCommit() throws Exception { // Now, the used resource is still 4 GB, and available resource is minus // value. report_nm1 = rm.getResourceScheduler().getNodeReport(nm1.getNodeId()); - Assert.assertEquals(4 * GB, report_nm1.getUsedResource().getMemorySize()); - Assert.assertEquals(-2 * GB, report_nm1.getAvailableResource().getMemorySize()); + Assertions.assertEquals(4 * GB, report_nm1.getUsedResource().getMemorySize()); + Assertions.assertEquals(-2 * GB, report_nm1.getAvailableResource().getMemorySize()); // Check container can complete successfully in case of resource // over-commitment. @@ -1193,13 +1208,13 @@ public void testResourceOverCommit() throws Exception { + waitCount + " times already.."); Thread.sleep(100); } - Assert.assertEquals(1, attempt1.getJustFinishedContainers().size()); - Assert.assertEquals(1, am1.schedule().getCompletedContainersStatuses() + Assertions.assertEquals(1, attempt1.getJustFinishedContainers().size()); + Assertions.assertEquals(1, am1.schedule().getCompletedContainersStatuses() .size()); report_nm1 = rm.getResourceScheduler().getNodeReport(nm1.getNodeId()); - Assert.assertEquals(2 * GB, report_nm1.getUsedResource().getMemorySize()); + Assertions.assertEquals(2 * GB, report_nm1.getUsedResource().getMemorySize()); // As container return 2 GB back, the available resource becomes 0 again. - Assert.assertEquals(0 * GB, report_nm1.getAvailableResource().getMemorySize()); + Assertions.assertEquals(0 * GB, report_nm1.getAvailableResource().getMemorySize()); rm.stop(); } @@ -1300,7 +1315,7 @@ public void handle(Event event) { application_0.schedule(); // Check the used resource is 1 GB 1 core - // Assert.assertEquals(1 * GB, nm_0.getUsed().getMemory()); + // Assertions.assertEquals(1 * GB, nm_0.getUsed().getMemory()); Resource usedResource = resourceManager.getResourceScheduler() .getSchedulerNode(nm_0.getNodeId()).getAllocatedResource(); @@ -1329,12 +1344,12 @@ public void handle(Event event) { private void checkApplicationResourceUsage(int expected, Application application) { - Assert.assertEquals(expected, application.getUsedResources().getMemorySize()); + Assertions.assertEquals(expected, application.getUsedResources().getMemorySize()); } private void checkNodeResourceUsage(int expected, org.apache.hadoop.yarn.server.resourcemanager.NodeManager node) { - Assert.assertEquals(expected, node.getUsed().getMemorySize()); + Assertions.assertEquals(expected, node.getUsed().getMemorySize()); node.checkResourceUsage(); } 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/placement/TestSingleConstraintAppPlacementAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/TestSingleConstraintAppPlacementAllocator.java index 14b9e4b0a6d75..2d77b4a7aafbe 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/TestSingleConstraintAppPlacementAllocator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/TestSingleConstraintAppPlacementAllocator.java @@ -33,9 +33,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.MemoryPlacementConstraintManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.PlacementConstraintManager; import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import java.util.HashSet; @@ -59,7 +59,7 @@ public class TestSingleConstraintAppPlacementAllocator { private SchedulerRequestKey schedulerRequestKey; private SingleConstraintAppPlacementAllocator allocator; - @Before + @BeforeEach public void setup() throws Exception { // stub app scheduling info. appSchedulingInfo = mock(AppSchedulingInfo.class); @@ -108,7 +108,7 @@ private void assertInvalidSchedulingRequest( // Expected return; } - Assert.fail( + Assertions.fail( "Expect failure for schedulingRequest=" + schedulingRequest.toString()); } @@ -126,7 +126,7 @@ public void testSchedulingRequestValidation() { .build()).resourceSizing( ResourceSizing.newInstance(1, Resource.newInstance(1024, 1))) .build()); - Assert.assertEquals("", allocator.getTargetNodePartition()); + Assertions.assertEquals("", allocator.getTargetNodePartition()); // Valid (with partition) assertValidSchedulingRequest(SchedulingRequest.newBuilder().executionType( @@ -140,7 +140,7 @@ public void testSchedulingRequestValidation() { .build()).resourceSizing( ResourceSizing.newInstance(1, Resource.newInstance(1024, 1))) .build()); - Assert.assertEquals("x", allocator.getTargetNodePartition()); + Assertions.assertEquals("x", allocator.getTargetNodePartition()); // Valid (without specifying node partition) assertValidSchedulingRequest(SchedulingRequest.newBuilder().executionType( @@ -154,7 +154,7 @@ public void testSchedulingRequestValidation() { ResourceSizing.newInstance(1, Resource.newInstance(1024, 1))) .build()); // Node partition is unspecified, use the default node label expression y - Assert.assertEquals("y", allocator.getTargetNodePartition()); + Assertions.assertEquals("y", allocator.getTargetNodePartition()); // Valid (with application Id target) assertValidSchedulingRequest(SchedulingRequest.newBuilder().executionType( @@ -168,7 +168,7 @@ public void testSchedulingRequestValidation() { ResourceSizing.newInstance(1, Resource.newInstance(1024, 1))) .build()); // Allocation tags should not include application Id - Assert.assertEquals("y", allocator.getTargetNodePartition()); + Assertions.assertEquals("y", allocator.getTargetNodePartition()); // Invalid (without sizing) assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType( @@ -279,7 +279,7 @@ public void testSchedulingRequestUpdate() { ResourceSizing.newInstance(1, Resource.newInstance(1024, 1))) .build(); allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest, true); - Assert.assertEquals(existingNumAllocations + 1, + Assertions.assertEquals(existingNumAllocations + 1, allocator.getSchedulingRequest().getResourceSizing() .getNumAllocations()); } @@ -330,13 +330,13 @@ public void testFunctionality() throws InvalidAllocationTagsQueryException { when(node1.getPartition()).thenReturn("x"); when(node1.getNodeID()).thenReturn(NodeId.fromString("host1:123")); - Assert.assertTrue(allocator + Assertions.assertTrue(allocator .precheckNode(node1, SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY)); SchedulerNode node2 = mock(SchedulerNode.class); when(node1.getPartition()).thenReturn(""); when(node1.getNodeID()).thenReturn(NodeId.fromString("host2:123")); - Assert.assertFalse(allocator + Assertions.assertFalse(allocator .precheckNode(node2, SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY)); } @@ -363,7 +363,7 @@ public void testNodeAttributesFunctionality() { boolean result = allocator.canAllocate(NodeType.NODE_LOCAL, TestUtils.getMockNodeWithAttributes("host1", "/rack1", 123, 1024, attributes)); - Assert.assertTrue("Allocation should be success for java=1.8", result); + Assertions.assertTrue(result, "Allocation should be success for java=1.8"); // 2. verify python!=3 validation SchedulingRequest schedulingRequest2 = @@ -387,8 +387,8 @@ public void testNodeAttributesFunctionality() { result = allocator.canAllocate(NodeType.NODE_LOCAL, TestUtils.getMockNodeWithAttributes("host1", "/rack1", 123, 1024, attributes)); - Assert.assertTrue("Allocation should be success as python doesn't exist", - result); + Assertions.assertTrue( + result, "Allocation should be success as python doesn't exist"); // 3. verify python!=3 validation when node has python=2 allocator = new SingleConstraintAppPlacementAllocator(); @@ -400,9 +400,9 @@ public void testNodeAttributesFunctionality() { result = allocator.canAllocate(NodeType.NODE_LOCAL, TestUtils.getMockNodeWithAttributes("host1", "/rack1", 123, 1024, attributes)); - Assert.assertTrue( - "Allocation should be success as python=3 doesn't exist in node", - result); + Assertions.assertTrue( + + result, "Allocation should be success as python=3 doesn't exist in node"); // 4. verify python!=3 validation when node has python=3 allocator = new SingleConstraintAppPlacementAllocator(); @@ -414,8 +414,8 @@ public void testNodeAttributesFunctionality() { result = allocator.canAllocate(NodeType.NODE_LOCAL, TestUtils.getMockNodeWithAttributes("host1", "/rack1", 123, 1024, attributes)); - Assert.assertFalse("Allocation should fail as python=3 exist in node", - result); + Assertions.assertFalse( + result, "Allocation should fail as python=3 exist in node"); } @Test @@ -451,8 +451,8 @@ public void testConjunctionNodeAttributesFunctionality() { boolean result = allocator.canAllocate(NodeType.NODE_LOCAL, TestUtils.getMockNodeWithAttributes("host1", "/rack1", 123, 1024, attributes)); - Assert.assertFalse("Allocation should fail as python=3 exists in node", - result); + Assertions.assertFalse( + result, "Allocation should fail as python=3 exists in node"); // 2. verify and(python!=3:java=1.8) validation when node has python=2 // and java=1.8 @@ -467,8 +467,8 @@ public void testConjunctionNodeAttributesFunctionality() { result = allocator.canAllocate(NodeType.NODE_LOCAL, TestUtils.getMockNodeWithAttributes("host1", "/rack1", 123, 1024, attributes)); - Assert.assertTrue("Allocation should be success as python=2 exists in node", - result); + Assertions.assertTrue( + result, "Allocation should be success as python=2 exists in node"); // 3. verify or(python!=3:java=1.8) validation when node has python=3 SchedulingRequest schedulingRequest2 = @@ -501,8 +501,8 @@ public void testConjunctionNodeAttributesFunctionality() { result = allocator.canAllocate(NodeType.NODE_LOCAL, TestUtils.getMockNodeWithAttributes("host1", "/rack1", 123, 1024, attributes)); - Assert.assertTrue("Allocation should be success as java=1.8 exists in node", - result); + Assertions.assertTrue( + result, "Allocation should be success as java=1.8 exists in node"); // 4. verify or(python!=3:java=1.8) validation when node has python=3 // and java=1.7. @@ -517,8 +517,8 @@ public void testConjunctionNodeAttributesFunctionality() { result = allocator.canAllocate(NodeType.NODE_LOCAL, TestUtils.getMockNodeWithAttributes("host1", "/rack1", 123, 1024, attributes)); - Assert - .assertFalse("Allocation should fail as java=1.8 doesnt exist in node", - result); + Assertions + .assertFalse( + result, "Allocation should fail as java=1.8 doesnt exist in 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/scheduler/policy/TestFairOrderingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFairOrderingPolicy.java index 9cfbb7da197d2..a759fe05bd352 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFairOrderingPolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFairOrderingPolicy.java @@ -18,7 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import java.util.*; @@ -33,8 +33,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; @@ -52,14 +52,14 @@ public void testSimpleComparison() { MockSchedulableEntity r1 = new MockSchedulableEntity(); MockSchedulableEntity r2 = new MockSchedulableEntity(); - assertEquals("Comparator Output", 0, - policy.getComparator().compare(r1, r2)); + assertEquals(0 +, policy.getComparator().compare(r1, r2), "Comparator Output"); //consumption r1.setUsed(Resources.createResource(1, 0)); AbstractComparatorOrderingPolicy.updateSchedulingResourceUsage( r1.getSchedulingResourceUsage()); - Assert.assertTrue(policy.getComparator().compare(r1, r2) > 0); + Assertions.assertTrue(policy.getComparator().compare(r1, r2) > 0); } @Test @@ -71,8 +71,8 @@ public void testSizeBasedWeight() { MockSchedulableEntity r2 = new MockSchedulableEntity(); //No changes, equal - assertEquals("Comparator Output", 0, - policy.getComparator().compare(r1, r2)); + assertEquals(0 +, policy.getComparator().compare(r1, r2), "Comparator Output"); r1.setUsed(Resources.createResource(4 * GB)); r2.setUsed(Resources.createResource(4 * GB)); @@ -86,8 +86,8 @@ public void testSizeBasedWeight() { r2.getSchedulingResourceUsage()); //Same, equal - assertEquals("Comparator Output", 0, - policy.getComparator().compare(r1, r2)); + assertEquals(0 +, policy.getComparator().compare(r1, r2), "Comparator Output"); r2.setUsed(Resources.createResource(5 * GB)); r2.setPending(Resources.createResource(5 * GB)); @@ -97,13 +97,13 @@ public void testSizeBasedWeight() { //More demand and consumption, but not enough more demand to overcome //additional consumption - Assert.assertTrue(policy.getComparator().compare(r1, r2) < 0); + Assertions.assertTrue(policy.getComparator().compare(r1, r2) < 0); //High demand, enough to reverse sbw r2.setPending(Resources.createResource(100 * GB)); AbstractComparatorOrderingPolicy.updateSchedulingResourceUsage( r2.getSchedulingResourceUsage()); - Assert.assertTrue(policy.getComparator().compare(r1, r2) > 0); + Assertions.assertTrue(policy.getComparator().compare(r1, r2) > 0); } @Test @@ -183,8 +183,8 @@ public void testSizeBasedWeightNotAffectAppActivation() throws Exception { // Get LeafQueue LeafQueue lq = (LeafQueue) cs.getQueue("default"); OrderingPolicy policy = lq.getOrderingPolicy(); - Assert.assertTrue(policy instanceof FairOrderingPolicy); - Assert.assertTrue(((FairOrderingPolicy)policy).getSizeBasedWeight()); + Assertions.assertTrue(policy instanceof FairOrderingPolicy); + Assertions.assertTrue(((FairOrderingPolicy)policy).getSizeBasedWeight()); rm.registerNode("h1:1234", 10 * GB); @@ -226,20 +226,20 @@ public void testSizeBasedWeightNotAffectAppActivation() throws Exception { .build(); MockRMAppSubmitter.submit(rm, data); - Assert.assertEquals(1, lq.getNumActiveApplications()); - Assert.assertEquals(3, lq.getNumPendingApplications()); + Assertions.assertEquals(1, lq.getNumActiveApplications()); + Assertions.assertEquals(3, lq.getNumPendingApplications()); // Try allocate once, #active-apps and #pending-apps should be still correct cs.handle(new NodeUpdateSchedulerEvent( rm.getRMContext().getRMNodes().get(NodeId.newInstance("h1", 1234)))); - Assert.assertEquals(1, lq.getNumActiveApplications()); - Assert.assertEquals(3, lq.getNumPendingApplications()); + Assertions.assertEquals(1, lq.getNumActiveApplications()); + Assertions.assertEquals(3, lq.getNumPendingApplications()); } public void checkIds(Iterator si, String[] ids) { for (int i = 0;i < ids.length;i++) { - Assert.assertEquals(si.next().getId(), + Assertions.assertEquals(si.next().getId(), ids[i]); } } @@ -264,8 +264,8 @@ public void testOrderingUsingUsedAndPendingResources() { .updateSchedulingResourceUsage(r2.getSchedulingResourceUsage()); // Same, equal - assertEquals("Comparator Output", 0, - policy.getComparator().compare(r1, r2)); + assertEquals(0 +, policy.getComparator().compare(r1, r2), "Comparator Output"); r1.setUsed(Resources.createResource(4 * GB)); r2.setUsed(Resources.createResource(8 * GB)); @@ -278,7 +278,7 @@ public void testOrderingUsingUsedAndPendingResources() { AbstractComparatorOrderingPolicy .updateSchedulingResourceUsage(r2.getSchedulingResourceUsage()); - Assert.assertTrue(policy.getComparator().compare(r1, r2) < 0); + Assertions.assertTrue(policy.getComparator().compare(r1, r2) < 0); } @Test @@ -293,20 +293,20 @@ public void testOrderingUsingAppSubmitTime() { assertEquals(r1.getStartTime(), r2.getStartTime()); // No changes, equal - assertEquals("Comparator Output", 0, - policy.getComparator().compare(r1, r2)); + assertEquals(0 +, policy.getComparator().compare(r1, r2), "Comparator Output"); // R2 has been started after R1 r1.setStartTime(5); r2.setStartTime(10); - Assert.assertTrue(policy.getComparator().compare(r1, r2) < 0); + Assertions.assertTrue(policy.getComparator().compare(r1, r2) < 0); // R1 has been started after R2 r1.setStartTime(10); r2.setStartTime(5); - Assert.assertTrue(policy.getComparator().compare(r1, r2) > 0); + Assertions.assertTrue(policy.getComparator().compare(r1, r2) > 0); } @Test @@ -325,8 +325,8 @@ public void testOrderingUsingAppDemand() { .updateSchedulingResourceUsage(r2.getSchedulingResourceUsage()); // Same, equal - assertEquals("Comparator Output", 0, - policy.getComparator().compare(r1, r2)); + assertEquals(0 +, policy.getComparator().compare(r1, r2), "Comparator Output"); // Compare demands ensures entity without resource demands gets lower // priority @@ -337,7 +337,7 @@ public void testOrderingUsingAppDemand() { AbstractComparatorOrderingPolicy .updateSchedulingResourceUsage(r2.getSchedulingResourceUsage()); - Assert.assertTrue(policy.getComparator().compare(r1, r2) > 0); + Assertions.assertTrue(policy.getComparator().compare(r1, r2) > 0); // When both entity has certain demands, then there is no actual comparison r1.setPending(Resources.createResource(4 * GB)); @@ -347,8 +347,8 @@ public void testOrderingUsingAppDemand() { AbstractComparatorOrderingPolicy .updateSchedulingResourceUsage(r2.getSchedulingResourceUsage()); - assertEquals("Comparator Output", 0, - policy.getComparator().compare(r1, r2)); + assertEquals(0 +, policy.getComparator().compare(r1, r2), "Comparator Output"); } @Test @@ -388,6 +388,6 @@ public void testRemoveEntitiesWithSizeBasedWeightAsCompletedJobs() { policy.removeSchedulableEntity(entities.get(i)); } - Assert.assertEquals(5, policy.getNumSchedulableEntities()); + Assertions.assertEquals(5, policy.getNumSchedulableEntities()); } } 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/policy/TestFifoOrderingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFifoOrderingPolicy.java index 62bc7124c4b9d..5745a1f191f4d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFifoOrderingPolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFifoOrderingPolicy.java @@ -18,16 +18,16 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.Arrays; import java.util.Iterator; import java.util.List; import org.apache.hadoop.yarn.api.records.Priority; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; public class TestFifoOrderingPolicy { @@ -39,17 +39,17 @@ public void testFifoOrderingPolicy() { MockSchedulableEntity r1 = new MockSchedulableEntity(); MockSchedulableEntity r2 = new MockSchedulableEntity(); - assertEquals("The comparator should return 0 because the entities are created with " + - "the same values.", 0, - policy.getComparator().compare(r1, r2)); + assertEquals(0 +, policy.getComparator().compare(r1, r2), "The comparator should return 0 because the entities are created with " + + "the same values."); r1.setSerial(1); - assertEquals("The lhs entity has a larger serial, the comparator return " + - "value should be 1.", 1, policy.getComparator().compare(r1, r2)); + assertEquals(1, policy.getComparator().compare(r1, r2), "The lhs entity has a larger serial, the comparator return " + + "value should be 1."); r2.setSerial(2); - Assert.assertEquals("The rhs entity has a larger serial, the comparator return " + - "value should be -1.", -1, policy.getComparator().compare(r1, r2)); + Assertions.assertEquals(-1, policy.getComparator().compare(r1, r2), "The rhs entity has a larger serial, the comparator return " + + "value should be -1."); } @Test @@ -91,32 +91,32 @@ public void testFifoOrderingPolicyAlongWithPriority() { MockSchedulableEntity r1 = new MockSchedulableEntity(); MockSchedulableEntity r2 = new MockSchedulableEntity(); - assertEquals("Both r1 and r2 priority is null, the comparator should return 0.", 0, - policy.getComparator().compare(r1, r2)); + assertEquals(0 +, policy.getComparator().compare(r1, r2), "Both r1 and r2 priority is null, the comparator should return 0."); Priority p2 = Priority.newInstance(0); // r1 is null and r2 is not null r2.setApplicationPriority(p2); - Assert.assertTrue("The priority of r1 is null, the priority of r2 is not null, " + - "the comparator should return a negative value.", - policy.getComparator().compare(r1, r2) < 0); + Assertions.assertTrue( + policy.getComparator().compare(r1, r2) < 0, "The priority of r1 is null, the priority of r2 is not null, " + + "the comparator should return a negative value."); Priority p1 = Priority.newInstance(1); // r1 is not null and r2 is null r1.setApplicationPriority(p1); r2.setApplicationPriority(null); - assertTrue("The priority of r1 is not null, the priority of r2 is null," + - "the comparator should return a positive value.", - policy.getComparator().compare(r1, r2) > 0); + assertTrue( + policy.getComparator().compare(r1, r2) > 0, "The priority of r1 is not null, the priority of r2 is null," + + "the comparator should return a positive value."); // r1 is not null and r2 is not null r1.setApplicationPriority(p1); r2.setApplicationPriority(p2); - Assert.assertTrue("Both priorities are not null, the r1 has higher priority, " + - "the result should be a negative value.", - policy.getComparator().compare(r1, r2) < 0); + Assertions.assertTrue( + policy.getComparator().compare(r1, r2) < 0, "Both priorities are not null, the r1 has higher priority, " + + "the result should be a negative value."); } @Test @@ -130,20 +130,20 @@ public void testOrderingUsingAppSubmitTime() { assertEquals(r1.getStartTime(), r2.getStartTime()); // No changes, equal - assertEquals("The submit times are the same, the comparator should return 0.", 0, - policy.getComparator().compare(r1, r2)); + assertEquals(0 +, policy.getComparator().compare(r1, r2), "The submit times are the same, the comparator should return 0."); // R2 has been started after R1 r1.setStartTime(5); r2.setStartTime(10); - Assert.assertTrue("r2 was started after r1, " + - "the comparator should return a negative value.", - policy.getComparator().compare(r1, r2) < 0); + Assertions.assertTrue( + policy.getComparator().compare(r1, r2) < 0, "r2 was started after r1, " + + "the comparator should return a negative value."); // R1 has been started after R2 r1.setStartTime(10); r2.setStartTime(5); - Assert.assertTrue("r2 was started before r1, the comparator should return a positive value.", - policy.getComparator().compare(r1, r2) > 0); + Assertions.assertTrue( + policy.getComparator().compare(r1, r2) > 0, "r2 was started before r1, the comparator should return a positive value."); } } 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/policy/TestFifoOrderingPolicyForPendingApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFifoOrderingPolicyForPendingApps.java index 56fccc25b0185..76dfe2fea63c3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFifoOrderingPolicyForPendingApps.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFifoOrderingPolicyForPendingApps.java @@ -20,8 +20,8 @@ import java.util.*; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThat; @@ -86,7 +86,7 @@ public void testIterators() { public void checkSerials(Iterator si, long[] serials) { for (int i = 0; i < serials.length; i++) { - Assert.assertEquals(si.next().getSerial(), serials[i]); + Assertions.assertEquals(si.next().getSerial(), serials[i]); } } } 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/policy/TestFifoOrderingPolicyWithExclusivePartitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFifoOrderingPolicyWithExclusivePartitions.java index 499a70a053c00..7fc6b11954f85 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFifoOrderingPolicyWithExclusivePartitions.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFifoOrderingPolicyWithExclusivePartitions.java @@ -23,8 +23,8 @@ import java.util.Iterator; import java.util.List; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; /** @@ -55,8 +55,8 @@ public void testNoConfiguredExclusiveEnforcedPartitions() { policy.addSchedulableEntity(p1); policy.addAllSchedulableEntities(Arrays.asList(p2, r1, r2)); - Assert.assertEquals(4, policy.getNumSchedulableEntities()); - Assert.assertEquals(4, policy.getSchedulableEntities().size()); + Assertions.assertEquals(4, policy.getNumSchedulableEntities()); + Assertions.assertEquals(4, policy.getSchedulableEntities().size()); IteratorSelector sel = new IteratorSelector(); // Should behave like FifoOrderingPolicy, regardless of partition verifyAssignmentIteratorOrder(policy, @@ -68,8 +68,8 @@ public void testNoConfiguredExclusiveEnforcedPartitions() { policy.removeSchedulableEntity(p2); policy.removeSchedulableEntity(r2); - Assert.assertEquals(2, policy.getNumSchedulableEntities()); - Assert.assertEquals(2, policy.getSchedulableEntities().size()); + Assertions.assertEquals(2, policy.getNumSchedulableEntities()); + Assertions.assertEquals(2, policy.getSchedulableEntities().size()); verifyAssignmentIteratorOrder(policy, IteratorSelector.EMPTY_ITERATOR_SELECTOR, "r1", "p1"); verifyPreemptionIteratorOrder(policy, "p1", "r1"); @@ -105,8 +105,8 @@ public void testSingleExclusiveEnforcedPartition() { r3.setId("r3"); policy.addSchedulableEntity(r1); - Assert.assertEquals(1, policy.getNumSchedulableEntities()); - Assert.assertEquals("r1", policy.getSchedulableEntities() + Assertions.assertEquals(1, policy.getNumSchedulableEntities()); + Assertions.assertEquals("r1", policy.getSchedulableEntities() .iterator().next().getId()); verifyAssignmentIteratorOrder(policy, IteratorSelector.EMPTY_ITERATOR_SELECTOR, "r1"); @@ -115,8 +115,8 @@ public void testSingleExclusiveEnforcedPartition() { List entities = Arrays.asList(r2, r3, p1, p2); policy.addAllSchedulableEntities(entities); policy.addSchedulableEntity(p3); - Assert.assertEquals(6, policy.getNumSchedulableEntities()); - Assert.assertEquals(6, policy.getSchedulableEntities().size()); + Assertions.assertEquals(6, policy.getNumSchedulableEntities()); + Assertions.assertEquals(6, policy.getSchedulableEntities().size()); // Assignment iterator should return non-PARTITION entities, // in order based on FifoOrderingPolicy verifyAssignmentIteratorOrder(policy, @@ -135,8 +135,8 @@ public void testSingleExclusiveEnforcedPartition() { policy.removeSchedulableEntity(p2); policy.removeSchedulableEntity(r2); - Assert.assertEquals(4, policy.getNumSchedulableEntities()); - Assert.assertEquals(4, policy.getSchedulableEntities().size()); + Assertions.assertEquals(4, policy.getNumSchedulableEntities()); + Assertions.assertEquals(4, policy.getSchedulableEntities().size()); verifyAssignmentIteratorOrder(policy, IteratorSelector.EMPTY_ITERATOR_SELECTOR, "r3", "r1"); verifyPreemptionIteratorOrder(policy, "r1", "p3", "p1", "r3"); @@ -146,8 +146,8 @@ public void testSingleExclusiveEnforcedPartition() { policy.removeSchedulableEntity(p1); policy.removeSchedulableEntity(p3); - Assert.assertEquals(2, policy.getNumSchedulableEntities()); - Assert.assertEquals(2, policy.getSchedulableEntities().size()); + Assertions.assertEquals(2, policy.getNumSchedulableEntities()); + Assertions.assertEquals(2, policy.getSchedulableEntities().size()); verifyAssignmentIteratorOrder(policy, IteratorSelector.EMPTY_ITERATOR_SELECTOR, "r3", "r1"); verifyPreemptionIteratorOrder(policy, "r1", "r3"); @@ -187,8 +187,8 @@ public void testMultipleExclusiveEnforcedPartitions() { s2.setId("s2"); policy.addAllSchedulableEntities(Arrays.asList(s1, s2, r1)); - Assert.assertEquals(3, policy.getNumSchedulableEntities()); - Assert.assertEquals(3, policy.getSchedulableEntities().size()); + Assertions.assertEquals(3, policy.getNumSchedulableEntities()); + Assertions.assertEquals(3, policy.getSchedulableEntities().size()); IteratorSelector sel = new IteratorSelector(); // assignment iterator returns only default (non-partitioned) entities verifyAssignmentIteratorOrder(policy, @@ -198,8 +198,8 @@ public void testMultipleExclusiveEnforcedPartitions() { verifyAssignmentIteratorOrder(policy, sel, "r1"); policy.addAllSchedulableEntities(Arrays.asList(r2, p1, p2)); - Assert.assertEquals(6, policy.getNumSchedulableEntities()); - Assert.assertEquals(6, policy.getSchedulableEntities().size()); + Assertions.assertEquals(6, policy.getNumSchedulableEntities()); + Assertions.assertEquals(6, policy.getSchedulableEntities().size()); verifyAssignmentIteratorOrder(policy, IteratorSelector.EMPTY_ITERATOR_SELECTOR, "s2", "s1"); sel.setPartition(PARTITION); @@ -211,8 +211,8 @@ public void testMultipleExclusiveEnforcedPartitions() { policy.removeSchedulableEntity(p2); policy.removeSchedulableEntity(r1); policy.removeSchedulableEntity(r2); - Assert.assertEquals(3, policy.getNumSchedulableEntities()); - Assert.assertEquals(3, policy.getSchedulableEntities().size()); + Assertions.assertEquals(3, policy.getNumSchedulableEntities()); + Assertions.assertEquals(3, policy.getSchedulableEntities().size()); verifyAssignmentIteratorOrder(policy, IteratorSelector.EMPTY_ITERATOR_SELECTOR, "s2", "s1"); sel.setPartition(PARTITION); @@ -237,8 +237,8 @@ private void verifyPreemptionIteratorOrder( private void verifyIteratorOrder(Iterator itr, String... ids) { for (String id : ids) { - Assert.assertEquals(id, itr.next().getId()); + Assertions.assertEquals(id, itr.next().getId()); } - Assert.assertFalse(itr.hasNext()); + Assertions.assertFalse(itr.hasNext()); } }