Переглянути джерело

YARN-11262. Upgrade JUnit from 4 to 5 in hadoop-yarn-server-resourcemanager Part4. (#7488)

Co-authored-by: Chris Nauroth <cnauroth@apache.org>
Co-authored-by: Hualong Zhang <hualong.z@hotmail.com>
Reviewed-by: Chris Nauroth <cnauroth@apache.org>
Reviewed-by: Hualong Zhang <hualong.z@hotmail.com>
Signed-off-by: Shilun Fan <slfan1989@apache.org>
slfan1989 1 місяць тому
батько
коміт
68fcde01fb
68 змінених файлів з 3223 додано та 3154 видалено
  1. 1 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
  2. 36 33
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.java
  3. 46 47
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestClusterNodeTracker.java
  4. 5 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestConfigurationMutationACLPolicies.java
  5. 51 47
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestPartitionQueueMetrics.java
  6. 13 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestQueueMetrics.java
  7. 11 13
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestQueueMetricsForCustomResources.java
  8. 12 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestResourceUsage.java
  9. 40 40
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerApplicationAttempt.java
  10. 62 62
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerHealth.java
  11. 26 26
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerOvercommit.java
  12. 109 97
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java
  13. 8 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueCapacities.java
  14. 81 77
      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
  15. 40 35
      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
  16. 6 5
      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
  17. 61 55
      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
  18. 99 95
      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
  19. 66 52
      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
  20. 9 7
      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
  21. 16 14
      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
  22. 17 18
      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
  23. 116 112
      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
  24. 4 3
      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
  25. 122 107
      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
  26. 9 9
      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
  27. 22 15
      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
  28. 13 13
      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
  29. 9 8
      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
  30. 27 25
      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
  31. 46 45
      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
  32. 28 28
      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
  33. 28 27
      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
  34. 4 4
      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
  35. 83 131
      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
  36. 60 77
      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
  37. 259 232
      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
  38. 211 152
      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
  39. 6 6
      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
  40. 76 57
      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
  41. 4 4
      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
  42. 6 6
      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
  43. 4 4
      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
  44. 147 146
      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
  45. 6 6
      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
  46. 18 14
      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
  47. 52 52
      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
  48. 27 26
      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
  49. 15 16
      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
  50. 2 2
      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
  51. 14 10
      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
  52. 73 85
      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
  53. 185 204
      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
  54. 30 30
      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
  55. 24 21
      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
  56. 150 164
      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
  57. 96 116
      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
  58. 43 39
      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
  59. 46 55
      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
  60. 18 31
      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
  61. 92 91
      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
  62. 10 6
      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
  63. 109 93
      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
  64. 29 34
      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
  65. 28 33
      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
  66. 30 29
      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
  67. 3 3
      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
  68. 24 22
      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

+ 1 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java

@@ -69,7 +69,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSec
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
-import org.junit.jupiter.api.Assertions;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
@@ -121,8 +120,7 @@ public class ReservationSystemTestUtil {
     assertEquals(8192, plan.getTotalCapacity().getMemorySize());
     assertTrue(
         plan.getReservationAgent() instanceof AlignedPlannerWithGreedy);
-    Assertions
-        .assertTrue(plan.getSharingPolicy() instanceof CapacityOverTimePolicy);
+    assertTrue(plan.getSharingPolicy() instanceof CapacityOverTimePolicy);
   }
 
   public static void setupFSAllocationFile(String allocationFile) {

+ 36 - 33
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.java

@@ -18,6 +18,10 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -36,8 +40,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtil
 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.Test;
 
 public class TestAppSchedulingInfo {
 
@@ -57,20 +60,20 @@ public class TestAppSchedulingInfo {
 
     appSchedulingInfo.updatePlacesBlacklistedByApp(new ArrayList<String>(),
         new ArrayList<String>());
-    Assert.assertFalse(appSchedulingInfo.getAndResetBlacklistChanged());
+    assertFalse(appSchedulingInfo.getAndResetBlacklistChanged());
 
     ArrayList<String> blacklistAdditions = new ArrayList<String>();
     blacklistAdditions.add("node1");
     blacklistAdditions.add("node2");
     appSchedulingInfo.updatePlacesBlacklistedByApp(blacklistAdditions,
         new ArrayList<String>());
-    Assert.assertTrue(appSchedulingInfo.getAndResetBlacklistChanged());
+    assertTrue(appSchedulingInfo.getAndResetBlacklistChanged());
 
     blacklistAdditions.clear();
     blacklistAdditions.add("node1");
     appSchedulingInfo.updatePlacesBlacklistedByApp(blacklistAdditions,
         new ArrayList<String>());
-    Assert.assertFalse(appSchedulingInfo.getAndResetBlacklistChanged());
+    assertFalse(appSchedulingInfo.getAndResetBlacklistChanged());
 
     ArrayList<String> blacklistRemovals = new ArrayList<String>();
     blacklistRemovals.add("node1");
@@ -78,11 +81,11 @@ public class TestAppSchedulingInfo {
         blacklistRemovals);
     appSchedulingInfo.updatePlacesBlacklistedByApp(new ArrayList<String>(),
         blacklistRemovals);
-    Assert.assertTrue(appSchedulingInfo.getAndResetBlacklistChanged());
+    assertTrue(appSchedulingInfo.getAndResetBlacklistChanged());
 
     appSchedulingInfo.updatePlacesBlacklistedByApp(new ArrayList<String>(),
         blacklistRemovals);
-    Assert.assertFalse(appSchedulingInfo.getAndResetBlacklistChanged());
+    assertFalse(appSchedulingInfo.getAndResetBlacklistChanged());
   }
 
   @Test
@@ -96,23 +99,23 @@ public class TestAppSchedulingInfo {
     ts.add(TestUtils.toSchedulerKey(Priority.newInstance(2), 6));
     Iterator<SchedulerRequestKey> iter = ts.iterator();
     SchedulerRequestKey sk = iter.next();
-    Assert.assertEquals(0, sk.getPriority().getPriority());
-    Assert.assertEquals(3, sk.getAllocationRequestId());
+    assertEquals(0, sk.getPriority().getPriority());
+    assertEquals(3, sk.getAllocationRequestId());
     sk = iter.next();
-    Assert.assertEquals(0, sk.getPriority().getPriority());
-    Assert.assertEquals(4, sk.getAllocationRequestId());
+    assertEquals(0, sk.getPriority().getPriority());
+    assertEquals(4, sk.getAllocationRequestId());
     sk = iter.next();
-    Assert.assertEquals(1, sk.getPriority().getPriority());
-    Assert.assertEquals(1, sk.getAllocationRequestId());
+    assertEquals(1, sk.getPriority().getPriority());
+    assertEquals(1, sk.getAllocationRequestId());
     sk = iter.next();
-    Assert.assertEquals(1, sk.getPriority().getPriority());
-    Assert.assertEquals(2, sk.getAllocationRequestId());
+    assertEquals(1, sk.getPriority().getPriority());
+    assertEquals(2, sk.getAllocationRequestId());
     sk = iter.next();
-    Assert.assertEquals(2, sk.getPriority().getPriority());
-    Assert.assertEquals(5, sk.getAllocationRequestId());
+    assertEquals(2, sk.getPriority().getPriority());
+    assertEquals(5, sk.getAllocationRequestId());
     sk = iter.next();
-    Assert.assertEquals(2, sk.getPriority().getPriority());
-    Assert.assertEquals(6, sk.getAllocationRequestId());
+    assertEquals(2, sk.getPriority().getPriority());
+    assertEquals(6, sk.getAllocationRequestId());
   }
 
   @Test
@@ -128,7 +131,7 @@ public class TestAppSchedulingInfo {
     AppSchedulingInfo  info = new AppSchedulingInfo(
         appAttemptId, "test", queue, mock(ActiveUsersManager.class), 0,
         new ResourceUsage(), new HashMap<>(), rmContext, false);
-    Assert.assertEquals(0, info.getSchedulerKeys().size());
+    assertEquals(0, info.getSchedulerKeys().size());
 
     Priority pri1 = Priority.newInstance(1);
     ResourceRequest req1 = ResourceRequest.newInstance(pri1,
@@ -142,16 +145,16 @@ public class TestAppSchedulingInfo {
     info.updateResourceRequests(reqs, false);
     ArrayList<SchedulerRequestKey> 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));
+    assertEquals(2, keys.size());
+    assertEquals(SchedulerRequestKey.create(req1), keys.get(0));
+    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),
+    assertEquals(1, info.getSchedulerKeys().size());
+    assertEquals(SchedulerRequestKey.create(req2),
         info.getSchedulerKeys().iterator().next());
 
     req2 = ResourceRequest.newInstance(pri2,
@@ -161,22 +164,22 @@ public class TestAppSchedulingInfo {
     info.updateResourceRequests(reqs, false);
     info.allocate(NodeType.OFF_SWITCH, null, SchedulerRequestKey.create(req2),
         null);
-    Assert.assertEquals(0, info.getSchedulerKeys().size());
+    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),
+    assertEquals(1, info.getSchedulerKeys().size());
+    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());
+    assertEquals(0, info.getSchedulerKeys().size());
   }
 
   @Test
@@ -193,9 +196,9 @@ public class TestAppSchedulingInfo {
     AppSchedulingInfo info = new AppSchedulingInfo(appAttemptId, "test", queue,
         mock(ActiveUsersManager.class), 0, new ResourceUsage(), new HashMap<>(),
         rmContext, false);
-    Assert.assertEquals(info.getApplicationSchedulingEnvs(), new HashMap<>());
+    assertEquals(info.getApplicationSchedulingEnvs(), new HashMap<>());
     // This should return null as nothing is set in the conf.
-    Assert.assertNull(info.getDefaultResourceRequestAppPlacementType());
+    assertNull(info.getDefaultResourceRequestAppPlacementType());
     conf = new Configuration();
     conf.set(YarnConfiguration.APPLICATION_PLACEMENT_TYPE_CLASS,
         DEFAULT_APPLICATION_PLACEMENT_TYPE_CLASS);
@@ -203,7 +206,7 @@ public class TestAppSchedulingInfo {
     info = new AppSchedulingInfo(appAttemptId, "test", queue,
         mock(ActiveUsersManager.class), 0, new ResourceUsage(), new HashMap<>(),
         rmContext, false);
-    Assert.assertEquals(info.getDefaultResourceRequestAppPlacementType(),
+    assertEquals(info.getDefaultResourceRequestAppPlacementType(),
         DEFAULT_APPLICATION_PLACEMENT_TYPE_CLASS);
   }
 
@@ -223,7 +226,7 @@ public class TestAppSchedulingInfo {
         mock(ActiveUsersManager.class), 0, new ResourceUsage(),
         applicationSchedulingEnvs, rmContext, false);
     // This should be set from applicationSchedulingEnvs
-    Assert.assertEquals(info.getDefaultResourceRequestAppPlacementType(),
+    assertEquals(info.getDefaultResourceRequestAppPlacementType(),
         LocalityAppPlacementAllocator.class.getName());
   }
 }

+ 46 - 47
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.rmnode.RMNode;
 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<FSSchedulerNode> 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 @@ public class TestClusterNodeTracker {
   @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,9 @@ public class TestClusterNodeTracker {
 
     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<RMNode> smallNodes =
         MockNodes.newNodes(1, 1, Resource.newInstance(1024, 2,
@@ -133,52 +134,51 @@ public class TestClusterNodeTracker {
 
     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,10 @@ public class TestClusterNodeTracker {
 
     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 +197,7 @@ public class TestClusterNodeTracker {
 
     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");
   }
 }

+ 5 - 5
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.RMContext;
 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<String, String> EMPTY_MAP =
       Collections.<String, String>emptyMap();
 
-  @Before
+  @BeforeEach
   public void setUp() throws IOException {
     rmContext = mock(RMContext.class);
     scheduler = mock(MutableConfScheduler.class);

+ 51 - 47
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,8 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 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.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -36,10 +38,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 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.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 class TestPartitionQueueMetrics {
 
     PartitionQueueMetrics pq1 =
         new PartitionQueueMetrics(ms, "root.q1", parentQueue, true, CONF, "x");
-    Assert.assertTrue("Name of registry should be \""
+    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 class TestPartitionQueueMetrics {
    *
    * @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,

+ 13 - 11
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,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 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.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -64,8 +63,11 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceMe
 import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceMetricsChecker.ResourceMetricsKey.RESERVED_CONTAINERS;
 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.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -82,7 +84,7 @@ public class TestQueueMetrics {
   private static final Configuration conf = new Configuration();
   private MetricsSystem ms;
 
-  @Before
+  @BeforeEach
   public void setUp() {
     ms = new MetricsSystemImpl();
     QueueMetrics.clearQueueMetrics();
@@ -747,15 +749,15 @@ public class TestQueueMetrics {
       QueueMetrics metrics =
           QueueMetrics.forQueue(ms, leafQueueName, parentQueue1, true, conf);
 
-      Assert.assertNotNull("QueueMetrics for A shoudn't be null", metrics);
+      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);
+      assertNotNull(alterMetrics,
+          "QueueMetrics for alterMetrics shoudn't be null");
     } finally {
       ms.shutdown();
     }
@@ -788,9 +790,9 @@ public class TestQueueMetrics {
       // do not collect all metrics
       AppMetricsChecker.create()
           .checkAgainst(queueSource, false);
-      Assert.fail();
+      fail();
     } catch (AssertionError e) {
-      Assert.assertTrue(
+      assertTrue(
               e.getMessage().contains("Expected exactly one metric for name "));
     }
     // collect all metrics

+ 11 - 13
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.server.resourcemanager.nodelabels.RMNodeLabelsMana
 
 
 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.ResourceMe
 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 class TestQueueMetricsForCustomResources {
   private Resource defaultResource;
   private MetricsSystem ms;
 
-  @Before
+  @BeforeEach
   public void setUp() {
     ms = new MetricsSystemImpl();
     QueueMetrics.clearQueueMetrics();
@@ -155,14 +155,12 @@ public class TestQueueMetricsForCustomResources {
   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(
-            "QueueMetrics should have custom resource metrics value %d " +
-                "for resource: %s for metrics type %s",
-            expectedValue, resourceName, metricsType), expectedValue,
-            (long) value);
+    assertNotNull(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));
   }
 
   private static Map<String, String> getCustomResourcesWithValue(long value) {

+ 12 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestResourceUsage.java

@@ -18,32 +18,30 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
 import java.lang.reflect.Method;
 import java.util.Arrays;
 import java.util.Collection;
 
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
 
-@RunWith(Parameterized.class)
 public class TestResourceUsage {
   private static final Logger LOG =
       LoggerFactory.getLogger(TestResourceUsage.class);
   private String suffix;
 
-  @Parameterized.Parameters
   public static Collection<String[]> getParameters() {
     return Arrays.asList(new String[][]{{"Pending"}, {"Used"}, {"Reserved"},
         {"AMUsed"}, {"AMLimit"}, {"CachedUsed"}, {"CachedPending"}});
   }
 
-  public TestResourceUsage(String suffix) {
-    this.suffix = suffix;
+  public void initTestResourceUsage(String pSuffix) {
+    this.suffix = pSuffix;
   }
 
   private static void dec(ResourceUsage obj, String suffix, Resource res,
@@ -136,12 +134,14 @@ public class TestResourceUsage {
   }
 
   void check(int mem, int cpu, Resource res) {
-    Assert.assertEquals(mem, res.getMemorySize());
-    Assert.assertEquals(cpu, res.getVirtualCores());
+    assertEquals(mem, res.getMemorySize());
+    assertEquals(cpu, res.getVirtualCores());
   }
 
-  @Test
-  public void testModifyAndRead() throws Exception {
+  @ParameterizedTest
+  @MethodSource("getParameters")
+  public void testModifyAndRead(String pSuffix) throws Exception {
+    initTestResourceUsage(pSuffix);
     LOG.info("Test - " + suffix);
     internalTestModifyAndRead(null);
     internalTestModifyAndRead("label");

+ 40 - 40
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 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
 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.resourcemanager.scheduler.fifo.FifoSchedule
 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 class TestSchedulerApplicationAttempt {
     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)
-               + " but was "
-               + app.getResourceUsageReport().getReservedResources(),
-        Resources.equals(Resource.newInstance(2048, 2),
-        app.getResourceUsageReport().getReservedResources()));
+    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());
   }
 
   @Test
@@ -413,30 +413,30 @@ public class TestSchedulerApplicationAttempt {
     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)");
   }
 }

+ 62 - 62
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerHealth.java

@@ -18,6 +18,9 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assumptions.assumeTrue;
+
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.util.Time;
@@ -40,13 +43,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 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.Test;
 
 import java.io.IOException;
 
 import static org.apache.hadoop.yarn.server.resourcemanager.MockNM.createMockNodeStatus;
-import static org.junit.Assume.assumeTrue;
 
 public class TestSchedulerHealth {
 
@@ -84,18 +85,18 @@ public class TestSchedulerHealth {
       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());
+      assertEquals(value, sh.getAllocationCount().longValue());
+      assertEquals(value, sh.getReleaseCount().longValue());
+      assertEquals(value, sh.getReservationCount().longValue());
+      assertEquals(value, sh.getPreemptionCount().longValue());
 
-      Assert.assertEquals(value * (i + 1), sh.getAggregateAllocationCount()
+      assertEquals(value * (i + 1), sh.getAggregateAllocationCount()
         .longValue());
-      Assert.assertEquals(value * (i + 1), sh.getAggregateReleaseCount()
+      assertEquals(value * (i + 1), sh.getAggregateReleaseCount()
         .longValue());
-      Assert.assertEquals(value * (i + 1), sh.getAggregateReservationCount()
+      assertEquals(value * (i + 1), sh.getAggregateReservationCount()
         .longValue());
-      Assert.assertEquals(value * (i + 1), sh.getAggregatePreemptionCount()
+      assertEquals(value * (i + 1), sh.getAggregatePreemptionCount()
         .longValue());
 
     }
@@ -109,50 +110,49 @@ public class TestSchedulerHealth {
     sh.updateRelease(now, NodeId.newInstance("testhost", 1234),
       ContainerId.fromString("container_1427562107907_0002_01_000001"),
       "testqueue");
-    Assert.assertEquals("container_1427562107907_0002_01_000001", sh
+    assertEquals("container_1427562107907_0002_01_000001", sh
       .getLastReleaseDetails().getContainerId().toString());
-    Assert.assertEquals("testhost:1234", sh.getLastReleaseDetails().getNodeId()
+    assertEquals("testhost:1234", sh.getLastReleaseDetails().getNodeId()
       .toString());
-    Assert.assertEquals("testqueue", sh.getLastReleaseDetails().getQueue());
-    Assert.assertEquals(now, sh.getLastReleaseDetails().getTimestamp());
-    Assert.assertEquals(0, sh.getLastSchedulerRunTime());
+    assertEquals("testqueue", sh.getLastReleaseDetails().getQueue());
+    assertEquals(now, sh.getLastReleaseDetails().getTimestamp());
+    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
+    assertEquals("container_1427562107907_0003_01_000001", sh
       .getLastReservationDetails().getContainerId().toString());
-    Assert.assertEquals("testhost1:1234", sh.getLastReservationDetails()
+    assertEquals("testhost1:1234", sh.getLastReservationDetails()
       .getNodeId().toString());
-    Assert
-      .assertEquals("testqueue1", sh.getLastReservationDetails().getQueue());
-    Assert.assertEquals(now, sh.getLastReservationDetails().getTimestamp());
-    Assert.assertEquals(0, sh.getLastSchedulerRunTime());
+    assertEquals("testqueue1", sh.getLastReservationDetails().getQueue());
+    assertEquals(now, sh.getLastReservationDetails().getTimestamp());
+    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
+    assertEquals("container_1427562107907_0004_01_000001", sh
       .getLastAllocationDetails().getContainerId().toString());
-    Assert.assertEquals("testhost2:1234", sh.getLastAllocationDetails()
+    assertEquals("testhost2:1234", sh.getLastAllocationDetails()
       .getNodeId().toString());
-    Assert.assertEquals("testqueue2", sh.getLastAllocationDetails().getQueue());
-    Assert.assertEquals(now, sh.getLastAllocationDetails().getTimestamp());
-    Assert.assertEquals(0, sh.getLastSchedulerRunTime());
+    assertEquals("testqueue2", sh.getLastAllocationDetails().getQueue());
+    assertEquals(now, sh.getLastAllocationDetails().getTimestamp());
+    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
+    assertEquals("container_1427562107907_0005_01_000001", sh
       .getLastPreemptionDetails().getContainerId().toString());
-    Assert.assertEquals("testhost3:1234", sh.getLastPreemptionDetails()
+    assertEquals("testhost3:1234", sh.getLastPreemptionDetails()
       .getNodeId().toString());
-    Assert.assertEquals("testqueue3", sh.getLastPreemptionDetails().getQueue());
-    Assert.assertEquals(now, sh.getLastPreemptionDetails().getTimestamp());
-    Assert.assertEquals(0, sh.getLastSchedulerRunTime());
+    assertEquals("testqueue3", sh.getLastPreemptionDetails().getQueue());
+    assertEquals(now, sh.getLastPreemptionDetails().getTimestamp());
+    assertEquals(0, sh.getLastSchedulerRunTime());
   }
 
   @Test
@@ -161,15 +161,15 @@ public class TestSchedulerHealth {
     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),
+    assertEquals(now, sh.getLastSchedulerRunTime());
+    assertEquals(Resource.newInstance(1024, 1),
       sh.getResourcesAllocated());
-    Assert.assertEquals(Resource.newInstance(2048, 1),
+    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),
+    assertEquals(now, sh.getLastSchedulerRunTime());
+    assertEquals(Resource.newInstance(3072, 1),
       sh.getResourcesReleased());
   }
 
@@ -201,8 +201,8 @@ public class TestSchedulerHealth {
 
     boolean isCapacityScheduler =
         resourceManager.getResourceScheduler() instanceof CapacityScheduler;
-    assumeTrue("This test is only supported on Capacity Scheduler",
-      isCapacityScheduler);
+    assumeTrue(isCapacityScheduler,
+        "This test is only supported on Capacity Scheduler");
 
     NodeStatus mockNodeStatus = createMockNodeStatus();
 
@@ -246,13 +246,13 @@ public class TestSchedulerHealth {
           .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),
+    assertEquals(1, sh.getAllocationCount().longValue());
+    assertEquals(Resource.newInstance(1 * 1024, 1),
       sh.getResourcesAllocated());
-    Assert.assertEquals(2, sh.getAggregateAllocationCount().longValue());
-    Assert.assertEquals("host_0:1234", sh.getLastAllocationDetails()
+    assertEquals(2, sh.getAggregateAllocationCount().longValue());
+    assertEquals("host_0:1234", sh.getLastAllocationDetails()
       .getNodeId().toString());
-    Assert.assertEquals("root.default", sh.getLastAllocationDetails()
+    assertEquals("root.default", sh.getLastAllocationDetails()
       .getQueue());
 
     Task task_0_2 =
@@ -261,13 +261,13 @@ public class TestSchedulerHealth {
     application_0.schedule();
 
     nodeUpdate(nm_0);
-    Assert.assertEquals(1, sh.getAllocationCount().longValue());
-    Assert.assertEquals(Resource.newInstance(2 * 1024, 1),
+    assertEquals(1, sh.getAllocationCount().longValue());
+    assertEquals(Resource.newInstance(2 * 1024, 1),
       sh.getResourcesAllocated());
-    Assert.assertEquals(3, sh.getAggregateAllocationCount().longValue());
-    Assert.assertEquals("host_0:1234", sh.getLastAllocationDetails()
+    assertEquals(3, sh.getAggregateAllocationCount().longValue());
+    assertEquals("host_0:1234", sh.getLastAllocationDetails()
       .getNodeId().toString());
-    Assert.assertEquals("root.default", sh.getLastAllocationDetails()
+    assertEquals("root.default", sh.getLastAllocationDetails()
       .getQueue());
   }
 
@@ -278,8 +278,8 @@ public class TestSchedulerHealth {
 
     boolean isCapacityScheduler =
         resourceManager.getResourceScheduler() instanceof CapacityScheduler;
-    assumeTrue("This test is only supported on Capacity Scheduler",
-      isCapacityScheduler);
+    assumeTrue(isCapacityScheduler,
+        "This test is only supported on Capacity Scheduler");
 
     NodeStatus mockNodeStatus = createMockNodeStatus();
 
@@ -325,13 +325,13 @@ public class TestSchedulerHealth {
     SchedulerHealth sh =
         ((CapacityScheduler) resourceManager.getResourceScheduler())
           .getSchedulerHealth();
-    Assert.assertEquals(1, sh.getAllocationCount().longValue());
-    Assert.assertEquals(Resource.newInstance(1024, 1),
+    assertEquals(1, sh.getAllocationCount().longValue());
+    assertEquals(Resource.newInstance(1024, 1),
       sh.getResourcesAllocated());
-    Assert.assertEquals(1, sh.getAggregateAllocationCount().longValue());
-    Assert.assertEquals("host_0:1234", sh.getLastAllocationDetails()
+    assertEquals(1, sh.getAggregateAllocationCount().longValue());
+    assertEquals("host_0:1234", sh.getLastAllocationDetails()
       .getNodeId().toString());
-    Assert.assertEquals("root.default", sh.getLastAllocationDetails()
+    assertEquals("root.default", sh.getLastAllocationDetails()
       .getQueue());
 
     Task task_0_1 =
@@ -340,14 +340,14 @@ public class TestSchedulerHealth {
     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),
+    assertEquals(0, sh.getAllocationCount().longValue());
+    assertEquals(1, sh.getReservationCount().longValue());
+    assertEquals(Resource.newInstance(2 * 1024, 1),
       sh.getResourcesReserved());
-    Assert.assertEquals(1, sh.getAggregateAllocationCount().longValue());
-    Assert.assertEquals("host_0:1234", sh.getLastAllocationDetails()
+    assertEquals(1, sh.getAggregateAllocationCount().longValue());
+    assertEquals("host_0:1234", sh.getLastAllocationDetails()
       .getNodeId().toString());
-    Assert.assertEquals("root.default", sh.getLastAllocationDetails()
+    assertEquals("root.default", sh.getLastAllocationDetails()
       .getQueue());
   }
 }

+ 26 - 26
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 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 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.RMApp;
 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 @@ public abstract class TestSchedulerOvercommit {
    * 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 abstract class TestSchedulerOvercommit {
     // 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 abstract class TestSchedulerOvercommit {
     // 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 @@ public abstract class TestSchedulerOvercommit {
       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 abstract class TestSchedulerOvercommit {
    * @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 abstract class TestSchedulerOvercommit {
    */
   public static void assertPreemption(
       final ContainerId containerId, final PreemptionMessage msg) {
-    assertNotNull("Expected a preemption message", msg);
+    assertNotNull(msg, "Expected a preemption message");
     Set<ContainerId> preemptContainers = new HashSet<>();
     if (msg.getContract() != null) {
       for (PreemptionContainer c : msg.getContract().getContainers()) {
@@ -659,10 +659,10 @@ public abstract class TestSchedulerOvercommit {
     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");
   }
 
   /**

+ 109 - 97
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,12 @@ import static org.apache.hadoop.yarn.exceptions
     .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.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyBoolean;
 import static org.mockito.Mockito.mock;
@@ -98,13 +101,11 @@ import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 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.Rule;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet;
-import org.junit.rules.ExpectedException;
 import org.mockito.Mockito;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -120,14 +121,11 @@ public class TestSchedulerUtils {
 
   private static YarnConfiguration conf = new YarnConfiguration();
 
-  @Rule
-  public ExpectedException exception = ExpectedException.none();
-
   private void initResourceTypes() {
     CustomResourceTypesConfigurationProvider.initResourceTypes(3, "G");
   }
 
-  @Before
+  @BeforeEach
   public void setUp() {
     initResourceTypes();
     //this needs to be initialized after initResourceTypes is called
@@ -140,7 +138,8 @@ public class TestSchedulerUtils {
                     .build());
   }
 
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testNormalizeRequest() {
     ResourceCalculator resourceCalculator = new DefaultResourceCalculator();
 
@@ -205,7 +204,8 @@ public class TestSchedulerUtils {
             ask.getCapability().getMemorySize());
   }
 
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testNormalizeRequestWithDominantResourceCalculator() {
     ResourceCalculator resourceCalculator = new DominantResourceCalculator();
 
@@ -238,7 +238,9 @@ public class TestSchedulerUtils {
     assertEquals(2048, ask.getCapability().getMemorySize());
   }
 
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
+  @SuppressWarnings("checkstyle:methodlength")
   public void testValidateResourceRequestWithErrorLabelsPermission()
           throws IOException {
     // mock queue and scheduler
@@ -411,8 +413,7 @@ public class TestSchedulerUtils {
       rmContext.getNodeLabelManager().removeFromClusterNodeLabels(
               Arrays.asList("x"));
     }
-    Assert.assertTrue("InvalidLabelResourceRequestException expected",
-            invalidlabelexception);
+    assertTrue(invalidlabelexception, "InvalidLabelResourceRequestException expected");
     // queue is "*", always succeeded
     try {
       // set queue accessible node labels to empty
@@ -535,7 +536,7 @@ public class TestSchedulerUtils {
               .newResourceRequest(mock(Priority.class), "*", resource, 1, "x");
       normalizeAndvalidateRequest(resReq1, "queue",
               scheduler, rmContext, maxResource);
-      Assert.assertEquals(RMNodeLabelsManager.NO_LABEL,
+      assertEquals(RMNodeLabelsManager.NO_LABEL,
               resReq1.getNodeLabelExpression());
     } catch (InvalidResourceRequestException e) {
       assertEquals("Invalid resource request, node label not enabled but "
@@ -543,7 +544,8 @@ public class TestSchedulerUtils {
     }
   }
 
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testValidateResourceRequest() throws IOException {
     ResourceScheduler mockScheduler = mock(ResourceScheduler.class);
 
@@ -747,8 +749,7 @@ public class TestSchedulerUtils {
 
     rm.stop();
 
-    Assert.assertTrue(
-            "Didn't not catch InvalidResourceBlacklistRequestException", error);
+    assertTrue(error, "Didn't not catch InvalidResourceBlacklistRequestException");
   }
 
   private void waitForLaunchedState(RMAppAttempt attempt)
@@ -760,8 +761,7 @@ public class TestSchedulerUtils {
               + "Current state is " + attempt.getAppAttemptState());
       Thread.sleep(1000);
     }
-    Assert.assertEquals(attempt.getAppAttemptState(),
-            RMAppAttemptState.LAUNCHED);
+    assertEquals(attempt.getAppAttemptState(), RMAppAttemptState.LAUNCHED);
   }
 
   @Test
@@ -776,7 +776,7 @@ public class TestSchedulerUtils {
     ContainerStatus cd = SchedulerUtils.createAbnormalContainerStatus(
             ContainerId.newContainerId(ApplicationAttemptId.newInstance(
                     ApplicationId.newInstance(System.currentTimeMillis(), 1), 1), 1), "x");
-    Assert.assertEquals(ContainerExitStatus.ABORTED, cd.getExitStatus());
+    assertEquals(ContainerExitStatus.ABORTED, cd.getExitStatus());
   }
 
   @Test
@@ -784,10 +784,11 @@ public class TestSchedulerUtils {
     ContainerStatus cd = SchedulerUtils.createPreemptedContainerStatus(
             ContainerId.newContainerId(ApplicationAttemptId.newInstance(
                     ApplicationId.newInstance(System.currentTimeMillis(), 1), 1), 1), "x");
-    Assert.assertEquals(ContainerExitStatus.PREEMPTED, cd.getExitStatus());
+    assertEquals(ContainerExitStatus.PREEMPTED, cd.getExitStatus());
   }
 
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testNormalizeNodeLabelExpression()
           throws IOException {
     // mock queue and scheduler
@@ -821,12 +822,12 @@ public class TestSchedulerUtils {
               mock(Priority.class), ResourceRequest.ANY, resource, 1);
       normalizeAndvalidateRequest(resReq, "queue",
               scheduler, rmContext, maxResource);
-      Assert.assertEquals("x", resReq.getNodeLabelExpression());
+      assertEquals("x", resReq.getNodeLabelExpression());
 
       resReq.setNodeLabelExpression(" y ");
       normalizeAndvalidateRequest(resReq, "queue",
               scheduler, rmContext, maxResource);
-      Assert.assertEquals("y", resReq.getNodeLabelExpression());
+      assertEquals("y", resReq.getNodeLabelExpression());
     } catch (InvalidResourceRequestException e) {
       e.printStackTrace();
       fail("Should be valid when request labels is a subset of queue labels");
@@ -840,24 +841,27 @@ public class TestSchedulerUtils {
   public void testCustomResourceRequestedUnitIsSmallerThanAvailableUnit()
           throws InvalidResourceRequestException {
     Resource requestedResource =
-            ResourceTypesTestHelper.newResource(1, 1,
-                    ImmutableMap.of("custom-resource-1", "11"));
+        ResourceTypesTestHelper.newResource(1, 1,
+        ImmutableMap.of("custom-resource-1", "11"));
 
     Resource availableResource =
-            ResourceTypesTestHelper.newResource(1, 1,
-                    ImmutableMap.of("custom-resource-1", "0G"));
-
-    exception.expect(InvalidResourceRequestException.class);
-    exception.expectMessage(InvalidResourceRequestExceptionMessageGenerator
-            .create().withRequestedResourceType("custom-resource-1")
-            .withRequestedResource(requestedResource)
-            .withAvailableAllocation(availableResource)
-            .withMaxAllocation(configuredMaxAllocation)
-            .withInvalidResourceType(GREATER_THEN_MAX_ALLOCATION)
-            .build());
-
-    SchedulerUtils.checkResourceRequestAgainstAvailableResource(
-            requestedResource, availableResource);
+        ResourceTypesTestHelper.newResource(1, 1,
+        ImmutableMap.of("custom-resource-1", "0G"));
+
+    String errorMessage = InvalidResourceRequestExceptionMessageGenerator
+        .create().withRequestedResourceType("custom-resource-1")
+        .withRequestedResource(requestedResource)
+        .withAvailableAllocation(availableResource)
+        .withMaxAllocation(configuredMaxAllocation)
+        .withInvalidResourceType(GREATER_THEN_MAX_ALLOCATION)
+        .build();
+
+    InvalidResourceRequestException invalidResourceRequestException =
+        assertThrows(InvalidResourceRequestException.class, () -> {
+          SchedulerUtils.checkResourceRequestAgainstAvailableResource(
+              requestedResource, availableResource);
+        });
+    assertTrue(invalidResourceRequestException.getMessage().contains(errorMessage));
   }
 
   @Test
@@ -882,26 +886,30 @@ public class TestSchedulerUtils {
 
   @Test
   public void testCustomResourceRequestedUnitIsGreaterThanAvailableUnit()
-          throws InvalidResourceRequestException {
+      throws InvalidResourceRequestException {
     Resource requestedResource =
-            ResourceTypesTestHelper.newResource(1, 1,
-                    ImmutableMap.of("custom-resource-1", "1M"));
+        ResourceTypesTestHelper.newResource(1, 1,
+        ImmutableMap.of("custom-resource-1", "1M"));
 
     Resource availableResource = ResourceTypesTestHelper.newResource(1, 1,
-            ImmutableMap.<String, String>builder().put("custom-resource-1",
-                    "120k")
-                    .build());
-
-    exception.expect(InvalidResourceRequestException.class);
-    exception.expectMessage(InvalidResourceRequestExceptionMessageGenerator
-            .create().withRequestedResourceType("custom-resource-1")
-            .withRequestedResource(requestedResource)
-            .withAvailableAllocation(availableResource)
-            .withMaxAllocation(configuredMaxAllocation)
-            .withInvalidResourceType(GREATER_THEN_MAX_ALLOCATION)
-            .build());
-    SchedulerUtils.checkResourceRequestAgainstAvailableResource(
-            requestedResource, availableResource);
+        ImmutableMap.<String, String>builder().put("custom-resource-1",
+        "120k").build());
+
+    String errorMessage = InvalidResourceRequestExceptionMessageGenerator
+        .create().withRequestedResourceType("custom-resource-1")
+        .withRequestedResource(requestedResource)
+        .withAvailableAllocation(availableResource)
+        .withMaxAllocation(configuredMaxAllocation)
+        .withInvalidResourceType(GREATER_THEN_MAX_ALLOCATION)
+        .build();
+
+    InvalidResourceRequestException invalidResourceRequestException =
+        assertThrows(InvalidResourceRequestException.class, () -> {
+          SchedulerUtils.checkResourceRequestAgainstAvailableResource(
+               requestedResource, availableResource);
+        });
+
+    assertTrue(invalidResourceRequestException.getMessage().contains(errorMessage));
   }
 
   @Test
@@ -951,17 +959,21 @@ public class TestSchedulerUtils {
     Resource availableResource = ResourceTypesTestHelper.newResource(1, 1,
             ImmutableMap.of("custom-resource-1", "100M"));
 
-    exception.expect(InvalidResourceRequestException.class);
-    exception.expectMessage(InvalidResourceRequestExceptionMessageGenerator
-            .create().withRequestedResourceType("custom-resource-1")
-            .withRequestedResource(requestedResource)
-            .withAvailableAllocation(availableResource)
-            .withInvalidResourceType(GREATER_THEN_MAX_ALLOCATION)
-            .withMaxAllocation(configuredMaxAllocation)
-            .build());
-
-    SchedulerUtils.checkResourceRequestAgainstAvailableResource(
-            requestedResource, availableResource);
+    String errorMessage = InvalidResourceRequestExceptionMessageGenerator
+        .create().withRequestedResourceType("custom-resource-1")
+        .withRequestedResource(requestedResource)
+        .withAvailableAllocation(availableResource)
+        .withInvalidResourceType(GREATER_THEN_MAX_ALLOCATION)
+        .withMaxAllocation(configuredMaxAllocation)
+        .build();
+
+    InvalidResourceRequestException invalidResourceRequestException =
+        assertThrows(InvalidResourceRequestException.class, () -> {
+          SchedulerUtils.checkResourceRequestAgainstAvailableResource(
+              requestedResource, availableResource);
+        });
+
+    assertTrue(invalidResourceRequestException.getMessage().contains(errorMessage));
   }
 
   public static void waitSchedulerApplicationAttemptStopped(
@@ -1000,64 +1012,64 @@ public class TestSchedulerUtils {
     // RR label unset and app label does not match. Nothing should happen.
     SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet,
         null);
-    Assert.assertNull(rr.getNodeLabelExpression());
+    assertNull(rr.getNodeLabelExpression());
     SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet,
         appLabel);
-    Assert.assertNull(rr.getNodeLabelExpression());
+    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());
+    assertEquals(dummyLabel, rr.getNodeLabelExpression());
     SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet,
         appLabel);
-    Assert.assertEquals(dummyLabel, rr.getNodeLabelExpression());
+    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());
+    assertNull(rr.getNodeLabelExpression());
     rr.setNodeLabelExpression(enforcedExclusiveLabel);
     SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet,
         appLabel);
-    Assert.assertEquals(appLabel, rr.getNodeLabelExpression());
+    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());
+    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());
+    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());
+    assertEquals(enforcedExclusiveLabel, rr.getNodeLabelExpression());
 
     // Unconfigured label: nothing should happen.
     rr.setNodeLabelExpression(null);
     SchedulerUtils.enforcePartitionExclusivity(rr, null,
         appLabel);
-    Assert.assertNull(rr.getNodeLabelExpression());
+    assertNull(rr.getNodeLabelExpression());
     rr.setNodeLabelExpression(dummyLabel);
     SchedulerUtils.enforcePartitionExclusivity(rr, null,
         appLabel);
-    Assert.assertEquals(dummyLabel, rr.getNodeLabelExpression());
+    assertEquals(dummyLabel, rr.getNodeLabelExpression());
     rr.setNodeLabelExpression(enforcedExclusiveLabel);
     SchedulerUtils.enforcePartitionExclusivity(rr, null,
         appLabel);
-    Assert.assertEquals(enforcedExclusiveLabel, rr.getNodeLabelExpression());
+    assertEquals(enforcedExclusiveLabel, rr.getNodeLabelExpression());
   }
 
   @Test
@@ -1075,57 +1087,57 @@ public class TestSchedulerUtils {
     // RR label unset and app label does not match. Nothing should happen.
     SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet,
         null);
-    Assert.assertNull(rr.getNodeLabelExpression());
+    assertNull(rr.getNodeLabelExpression());
     SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet,
         appLabel);
-    Assert.assertNull(rr.getNodeLabelExpression());
+    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());
+    assertEquals(dummyLabel, rr.getNodeLabelExpression());
     SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet,
         appLabel);
-    Assert.assertEquals(dummyLabel, rr.getNodeLabelExpression());
+    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());
+    assertNull(rr.getNodeLabelExpression());
     rr.setNodeLabelExpression(enforcedLabel2);
     SchedulerUtils.enforcePartitionExclusivity(rr, enforcedExclusiveLabelSet,
         appLabel);
-    Assert.assertEquals(appLabel, rr.getNodeLabelExpression());
+    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());
+    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());
+    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());
+    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());
+    assertEquals(enforcedLabel1, rr.getNodeLabelExpression());
   }
 
   public static SchedulerApplication<SchedulerApplicationAttempt>
@@ -1144,13 +1156,13 @@ public class TestSchedulerUtils {
     SchedulerApplication<SchedulerApplicationAttempt> app =
             applications.get(appId);
     // verify application is added.
-    Assert.assertNotNull(app);
-    Assert.assertEquals("user", app.getUser());
+    assertNotNull(app);
+    assertEquals("user", app.getUser());
 
     AppRemovedSchedulerEvent appRemoveEvent =
             new AppRemovedSchedulerEvent(appId, RMAppState.FINISHED);
     handler.handle(appRemoveEvent);
-    Assert.assertNull(applications.get(appId));
+    assertNull(applications.get(appId));
     return app;
   }
 

+ 8 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueCapacities.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
 import java.lang.reflect.Method;
 import java.util.Arrays;
 import java.util.Collection;
@@ -26,7 +28,6 @@ import org.junit.jupiter.params.ParameterizedTest;
 import org.junit.jupiter.params.provider.MethodSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.junit.jupiter.api.Assertions;
 
 public class TestQueueCapacities {
   private static final Logger LOG =
@@ -105,11 +106,11 @@ public class TestQueueCapacities {
 
     // Set to 1, and check
     set(qc, suffix, label, 1f);
-    Assertions.assertEquals(1f, get(qc, suffix, label), 1e-8);
+    assertEquals(1f, get(qc, suffix, label), 1e-8);
 
     // Set to 2, and check
     set(qc, suffix, label, 2f);
-    Assertions.assertEquals(2f, get(qc, suffix, label), 1e-8);
+    assertEquals(2f, get(qc, suffix, label), 1e-8);
   }
 
   @MethodSource("getParameters")
@@ -126,9 +127,9 @@ public class TestQueueCapacities {
   public void testDefaultValues(String pSuffix) {
     initTestQueueCapacities(pSuffix);
     QueueCapacities qc = new QueueCapacities(false);
-    Assertions.assertEquals(-1, qc.getWeight(""), 1e-6);
-    Assertions.assertEquals(-1, qc.getWeight("x"), 1e-6);
-    Assertions.assertEquals(0, qc.getCapacity(""), 1e-6);
-    Assertions.assertEquals(0, qc.getCapacity("x"), 1e-6);
+    assertEquals(-1, qc.getWeight(""), 1e-6);
+    assertEquals(-1, qc.getWeight("x"), 1e-6);
+    assertEquals(0, qc.getCapacity(""), 1e-6);
+    assertEquals(0, qc.getCapacity("x"), 1e-6);
   }
 }

+ 81 - 77
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

@@ -20,6 +20,12 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint;
 
+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.Mockito.spy;
+import static org.mockito.Mockito.when;
+
 import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -33,10 +39,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.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.mockito.Mockito;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 import java.util.List;
 import java.util.Set;
@@ -49,7 +53,7 @@ import java.util.concurrent.ConcurrentMap;
 public class TestAllocationTagsManager {
   private RMContext rmContext;
 
-  @Before
+  @BeforeEach
   public void setup() {
     MockRM rm = new MockRM();
     rm.start();
@@ -79,22 +83,22 @@ public class TestAllocationTagsManager {
     atm.addContainer(nodeId, cid1, tags1);
     atm.addContainer(nodeId, cid2, tags2);
     atm.addContainer(nodeId, cid3, tags3);
-    Assert.assertEquals(2L,
+    assertEquals(2L,
         (long) atm.getAllocationTagsWithCount(nodeId).get("mapper"));
-    Assert.assertEquals(1L,
+    assertEquals(1L,
         (long) atm.getAllocationTagsWithCount(nodeId).get("reducer"));
 
     // remove container1
     atm.removeContainer(nodeId, cid1, tags1);
-    Assert.assertEquals(1L,
+    assertEquals(1L,
         (long) atm.getAllocationTagsWithCount(nodeId).get("mapper"));
-    Assert.assertNull(atm.getAllocationTagsWithCount(nodeId).get("reducer"));
+    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"));
+    assertNull(atm.getAllocationTagsWithCount(nodeId).get("mapper"));
+    assertNull(atm.getAllocationTagsWithCount(nodeId).get("reducer"));
   }
 
   @Test
@@ -138,7 +142,7 @@ public class TestAllocationTagsManager {
         ImmutableSet.of("service"));
 
     // Get Node Cardinality of app1 on node1, with tag "mapper"
-    Assert.assertEquals(1,
+    assertEquals(1,
         atm.getNodeCardinalityByOp(NodeId.fromString("host1:123"),
             AllocationTags.createSingleAppAllocationTags(
                 TestUtils.getMockApplicationId(1),
@@ -146,11 +150,11 @@ public class TestAllocationTagsManager {
             Long::max));
 
     // Get Rack Cardinality of app1 on rack0, with tag "mapper"
-    Assert.assertEquals(2, atm.getRackCardinality("rack0",
+    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,
+    assertEquals(1,
         atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"),
             AllocationTags.createSingleAppAllocationTags(
                 TestUtils.getMockApplicationId(1),
@@ -158,7 +162,7 @@ public class TestAllocationTagsManager {
             Long::min));
 
     // Get Node Cardinality of app1 on node2, with tag "mapper/reducer", op=max
-    Assert.assertEquals(2,
+    assertEquals(2,
         atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"),
             AllocationTags.createSingleAppAllocationTags(
                 TestUtils.getMockApplicationId(1),
@@ -166,7 +170,7 @@ public class TestAllocationTagsManager {
             Long::max));
 
     // Get Node Cardinality of app1 on node2, with tag "mapper/reducer", op=sum
-    Assert.assertEquals(3,
+    assertEquals(3,
         atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"),
             AllocationTags.createSingleAppAllocationTags(
                 TestUtils.getMockApplicationId(1),
@@ -174,17 +178,17 @@ public class TestAllocationTagsManager {
             Long::sum));
 
     // Get Node Cardinality by passing single tag.
-    Assert.assertEquals(1,
+    assertEquals(1,
         atm.getNodeCardinality(NodeId.fromString("host2:123"),
             TestUtils.getMockApplicationId(1), "mapper"));
 
-    Assert.assertEquals(2,
+    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,
+    assertEquals(0,
         atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"),
             AllocationTags.createSingleAppAllocationTags(
                 TestUtils.getMockApplicationId(1),
@@ -193,41 +197,41 @@ public class TestAllocationTagsManager {
 
     // Get Node Cardinality of app1 on node2, with tag "<applicationId>", op=max
     // (Expect this returns #containers from app1 on node2)
-    Assert.assertEquals(2,
+    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,
+    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,
+    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(
+    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,
+    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,
+    assertEquals(1,
         atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"),
             AllocationTags.createSingleAppAllocationTags(
                 TestUtils.getMockApplicationId(2), ImmutableSet.of()),
@@ -253,7 +257,7 @@ public class TestAllocationTagsManager {
 
     // Expect all cardinality to be 0
     // Get Cardinality of app1 on node1, with tag "mapper"
-    Assert.assertEquals(0,
+    assertEquals(0,
         atm.getNodeCardinalityByOp(NodeId.fromString("host1:123"),
             AllocationTags.createSingleAppAllocationTags(
                 TestUtils.getMockApplicationId(1),
@@ -261,7 +265,7 @@ public class TestAllocationTagsManager {
             Long::max));
 
     // Get Node Cardinality of app1 on node2, with tag "mapper/reducer", op=min
-    Assert.assertEquals(0,
+    assertEquals(0,
         atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"),
             AllocationTags.createSingleAppAllocationTags(
                 TestUtils.getMockApplicationId(1),
@@ -269,7 +273,7 @@ public class TestAllocationTagsManager {
             Long::min));
 
     // Get Node Cardinality of app1 on node2, with tag "mapper/reducer", op=max
-    Assert.assertEquals(0,
+    assertEquals(0,
         atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"),
             AllocationTags.createSingleAppAllocationTags(
                 TestUtils.getMockApplicationId(1),
@@ -277,7 +281,7 @@ public class TestAllocationTagsManager {
             Long::max));
 
     // Get Node Cardinality of app1 on node2, with tag "mapper/reducer", op=sum
-    Assert.assertEquals(0,
+    assertEquals(0,
         atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"),
             AllocationTags.createSingleAppAllocationTags(
                 TestUtils.getMockApplicationId(1),
@@ -286,20 +290,20 @@ public class TestAllocationTagsManager {
 
     // Get Node Cardinality of app1 on node2, with tag "<applicationId>", op=max
     // (Expect this returns #containers from app1 on node2)
-    Assert.assertEquals(0,
+    assertEquals(0,
         atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"),
             AllocationTags.createSingleAppAllocationTags(
                 TestUtils.getMockApplicationId(1),
                 ImmutableSet.of(TestUtils.getMockApplicationId(1).toString())),
             Long::max));
 
-    Assert.assertEquals(0,
+    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,
+    assertEquals(0,
         atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"),
             AllocationTags.createSingleAppAllocationTags(
                 TestUtils.getMockApplicationId(1),
@@ -307,13 +311,13 @@ public class TestAllocationTagsManager {
             Long::max));
 
     // Get Node Cardinality of all apps on node2, with empty tag set, op=sum
-    Assert.assertEquals(0, atm.getNodeCardinalityByOp(
+    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,
+    assertEquals(0,
         atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"),
             AllocationTags.createSingleAppAllocationTags(
                 TestUtils.getMockApplicationId(1),
@@ -321,7 +325,7 @@ public class TestAllocationTagsManager {
             Long::sum));
 
     // Get Node Cardinality of app_2 on node2, with empty tag set, op=sum
-    Assert.assertEquals(0,
+    assertEquals(0,
         atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"),
             AllocationTags.createSingleAppAllocationTags(
                 TestUtils.getMockApplicationId(1),
@@ -370,32 +374,32 @@ public class TestAllocationTagsManager {
         TestUtils.getMockContainerId(2, 3), ImmutableSet.of("service"));
 
     // Get Rack Cardinality of app1 on rack0, with tag "mapper"
-    Assert.assertEquals(1, atm.getRackCardinality("rack0",
+    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",
+    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"));
+    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",
+    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",
+    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",
+    assertEquals(3, atm.getRackCardinalityByOp("rack0",
         AllocationTags.createGlobalAllocationTags(ImmutableSet.of()),
         Long::max));
   }
@@ -445,12 +449,12 @@ public class TestAllocationTagsManager {
         TestUtils.getMockContainerId(2, 3), ImmutableSet.of("service"));
 
     // Check internal data structure
-    Assert.assertEquals(0,
+    assertEquals(0,
         atm.getGlobalNodeMapping().getTypeToTagsWithCount().size());
-    Assert.assertEquals(0, atm.getPerAppNodeMappings().size());
-    Assert.assertEquals(0,
+    assertEquals(0, atm.getPerAppNodeMappings().size());
+    assertEquals(0,
         atm.getGlobalRackMapping().getTypeToTagsWithCount().size());
-    Assert.assertEquals(0, atm.getPerAppRackMappings().size());
+    assertEquals(0, atm.getPerAppRackMappings().size());
   }
 
   @Test
@@ -490,8 +494,8 @@ public class TestAllocationTagsManager {
     } catch (InvalidAllocationTagsQueryException e1) {
       caughtException = true;
     }
-    Assert.assertTrue("should fail because of nodeId specified",
-        caughtException);
+    assertTrue(caughtException,
+        "should fail because of nodeId specified");
 
     // No op
     caughtException = false;
@@ -504,14 +508,14 @@ public class TestAllocationTagsManager {
     } catch (InvalidAllocationTagsQueryException e1) {
       caughtException = true;
     }
-    Assert.assertTrue("should fail because of nodeId specified",
-        caughtException);
+    assertTrue(caughtException,
+        "should fail because of nodeId specified");
   }
 
   @Test
   public void testNodeAllocationTagsAggregation()
       throws InvalidAllocationTagsQueryException {
-    RMContext mockContext = Mockito.spy(rmContext);
+    RMContext mockContext = spy(rmContext);
 
     ApplicationId app1 = TestUtils.getMockApplicationId(1);
     ApplicationId app2 = TestUtils.getMockApplicationId(2);
@@ -528,7 +532,7 @@ public class TestAllocationTagsManager {
         RMAppState.NEW, "userA", ImmutableSet.of("")));
     allApps.put(app3, new MockRMApp(125, 1002,
         RMAppState.NEW, "userA", ImmutableSet.of("")));
-    Mockito.when(mockContext.getRMApps()).thenReturn(allApps);
+    when(mockContext.getRMApps()).thenReturn(allApps);
 
     AllocationTagsManager atm = new AllocationTagsManager(mockContext);
 
@@ -588,12 +592,12 @@ public class TestAllocationTagsManager {
     //********************************
     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));
+    assertEquals(2, atm.getNodeCardinalityByOp(host1, tags, Long::max));
+    assertEquals(0, atm.getNodeCardinalityByOp(host1, tags, Long::min));
+    assertEquals(1, atm.getNodeCardinalityByOp(host2, tags, Long::max));
+    assertEquals(0, atm.getNodeCardinalityByOp(host2, tags, Long::min));
+    assertEquals(0, atm.getNodeCardinalityByOp(host3, tags, Long::max));
+    assertEquals(0, atm.getNodeCardinalityByOp(host3, tags, Long::min));
 
     //********************************
     // 2) not-self (app2, app3)
@@ -618,20 +622,20 @@ public class TestAllocationTagsManager {
     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));
+    assertEquals(4, atm.getNodeCardinalityByOp(host1, tags, Long::max));
+    assertEquals(0, atm.getNodeCardinalityByOp(host1, tags, Long::min));
+    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));
+    assertEquals(3, atm.getNodeCardinalityByOp(host1, tags, Long::max));
+    assertEquals(0, atm.getNodeCardinalityByOp(host1, tags, Long::min));
+    assertEquals(2, atm.getNodeCardinalityByOp(host2, tags, Long::max));
+    assertEquals(1, atm.getNodeCardinalityByOp(host2, tags, Long::min));
+    assertEquals(3, atm.getNodeCardinalityByOp(host2, tags, Long::sum));
 
 
     //********************************
@@ -639,20 +643,20 @@ public class TestAllocationTagsManager {
     //********************************
     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));
+    assertEquals(6, atm.getNodeCardinalityByOp(host1, tags, Long::sum));
+    assertEquals(1, atm.getNodeCardinalityByOp(host2, tags, Long::sum));
+    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));
+    assertEquals(7, atm.getNodeCardinalityByOp(host1, tags, Long::sum));
+    assertEquals(4, atm.getNodeCardinalityByOp(host2, tags, Long::sum));
+    assertEquals(0, atm.getNodeCardinalityByOp(host3, tags, Long::sum));
+    assertEquals(6, atm.getNodeCardinalityByOp(host1, tags, Long::max));
+    assertEquals(3, atm.getNodeCardinalityByOp(host2, tags, Long::max));
+    assertEquals(0, atm.getNodeCardinalityByOp(host3, tags, Long::max));
+    assertEquals(1, atm.getNodeCardinalityByOp(host1, tags, Long::min));
+    assertEquals(1, atm.getNodeCardinalityByOp(host2, tags, Long::min));
+    assertEquals(0, atm.getNodeCardinalityByOp(host3, tags, Long::min));
   }
 }

+ 40 - 35
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

@@ -15,12 +15,17 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint; /**
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+
 import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet;
 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.Test;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -37,50 +42,50 @@ public class TestAllocationTagsNamespace {
 
     String namespaceStr = "self";
     namespace = TargetApplicationsNamespace.parse(namespaceStr);
-    Assert.assertEquals(AllocationTagNamespaceType.SELF,
+    assertEquals(AllocationTagNamespaceType.SELF,
         namespace.getNamespaceType());
 
     namespaceStr = "not-self";
     namespace = TargetApplicationsNamespace.parse(namespaceStr);
-    Assert.assertEquals(AllocationTagNamespaceType.NOT_SELF,
+    assertEquals(AllocationTagNamespaceType.NOT_SELF,
         namespace.getNamespaceType());
 
     namespaceStr = "all";
     namespace = TargetApplicationsNamespace.parse(namespaceStr);
-    Assert.assertEquals(AllocationTagNamespaceType.ALL,
+    assertEquals(AllocationTagNamespaceType.ALL,
         namespace.getNamespaceType());
 
     namespaceStr = "app-tag/spark-jobs";
     namespace = TargetApplicationsNamespace.parse(namespaceStr);
-    Assert.assertEquals(AllocationTagNamespaceType.APP_TAG,
+    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");
+      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(
+      assertTrue(e instanceof InvalidAllocationTagsQueryException);
+      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,
+    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");
+      fail("Parsing should fail as the given app ID is invalid");
     } catch (Exception e) {
-      Assert.assertTrue(e instanceof InvalidAllocationTagsQueryException);
-      Assert.assertTrue(e.getMessage().startsWith(
+      assertTrue(e instanceof InvalidAllocationTagsQueryException);
+      assertTrue(e.getMessage().startsWith(
           "Invalid application ID for app-id"));
     }
 
@@ -88,11 +93,11 @@ public class TestAllocationTagsNamespace {
     try {
       namespaceStr = "app-id";
       TargetApplicationsNamespace.parse(namespaceStr);
-      Assert.fail("Parsing should fail as the given namespace"
+      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(
+      assertTrue(e instanceof InvalidAllocationTagsQueryException);
+      assertTrue(e.getMessage().startsWith(
           "Missing the application ID in the namespace string"));
     }
 
@@ -100,10 +105,10 @@ public class TestAllocationTagsNamespace {
     try {
       namespaceStr = "non_exist_ns";
       TargetApplicationsNamespace.parse(namespaceStr);
-      Assert.fail("Parsing should fail as the giving type is not supported.");
+      fail("Parsing should fail as the giving type is not supported.");
     } catch (Exception e) {
-      Assert.assertTrue(e instanceof InvalidAllocationTagsQueryException);
-      Assert.assertTrue(e.getMessage().startsWith(
+      assertTrue(e instanceof InvalidAllocationTagsQueryException);
+      assertTrue(e.getMessage().startsWith(
           "Invalid namespace prefix"));
     }
   }
@@ -124,10 +129,10 @@ public class TestAllocationTagsNamespace {
     namespace = TargetApplicationsNamespace.parse(namespaceStr);
     try {
       namespace.getNamespaceScope();
-      Assert.fail("Call getNamespaceScope before evaluate is not allowed.");
+      fail("Call getNamespaceScope before evaluate is not allowed.");
     } catch (Exception e) {
-      Assert.assertTrue(e instanceof IllegalStateException);
-      Assert.assertTrue(e.getMessage().contains(
+      assertTrue(e instanceof IllegalStateException);
+      assertTrue(e.getMessage().contains(
           "Evaluate must be called before a namespace can be consumed."));
     }
 
@@ -135,24 +140,24 @@ public class TestAllocationTagsNamespace {
     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());
+    assertEquals(1, namespace.getNamespaceScope().size());
+    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());
+    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));
+    assertEquals(2, namespace.getNamespaceScope().size());
+    assertFalse(namespace.getNamespaceScope().contains(app1));
 
     namespaceStr = "all";
     namespace = TargetApplicationsNamespace.parse(namespaceStr);
-    Assert.assertEquals(AllocationTagNamespaceType.ALL,
+    assertEquals(AllocationTagNamespaceType.ALL,
         namespace.getNamespaceType());
 
     namespaceStr = "app-id/" + app2.toString();
@@ -160,8 +165,8 @@ public class TestAllocationTagsNamespace {
     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());
+    assertEquals(1, namespace.getNamespaceScope().size());
+    assertEquals(app2, namespace.getNamespaceScope().iterator().next());
 
     /**
      * App to Application Tags
@@ -182,21 +187,21 @@ public class TestAllocationTagsNamespace {
     namespace = TargetApplicationsNamespace.parse(namespaceStr);
     targetApplications = new TargetApplications(app1, appsWithTags);
     namespace.evaluate(targetApplications);
-    Assert.assertEquals(3, namespace.getNamespaceScope().size());
-    Assert.assertTrue(Sets.difference(namespace.getNamespaceScope(),
+    assertEquals(3, namespace.getNamespaceScope().size());
+    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(),
+    assertEquals(2, namespace.getNamespaceScope().size());
+    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());
+    assertEquals(0, namespace.getNamespaceScope().size());
   }
 }

+ 6 - 5
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,9 @@ import java.util.List;
 
 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.Test;
+
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 /**
  * Test Request Iterator.
@@ -48,7 +49,7 @@ public class TestBatchedRequestsIterators {
     long prevAllocId = 0;
     while (requestIterator.hasNext()) {
       SchedulingRequest request = requestIterator.next();
-      Assert.assertTrue(request.getAllocationRequestId() > prevAllocId);
+      assertTrue(request.getAllocationRequestId() > prevAllocId);
       prevAllocId = request.getAllocationRequestId();
     }
   }
@@ -71,9 +72,9 @@ public class TestBatchedRequestsIterators {
     while (requestIterator.hasNext()) {
       SchedulingRequest request = requestIterator.next();
       if (recCcount < 3) {
-        Assert.assertTrue(request.getAllocationTags().contains("pri"));
+        assertTrue(request.getAllocationTags().contains("pri"));
       } else {
-        Assert.assertTrue(request.getAllocationTags().contains("bar")
+        assertTrue(request.getAllocationTags().contains("bar")
             || request.getAllocationTags().contains("test"));
       }
       recCcount++;

+ 61 - 55
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,15 @@ import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
 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.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 /**
  * Unit tests for {@link PlacementConstraintManagerService}.
@@ -61,7 +67,7 @@ public class TestPlacementConstraintManagerService {
   private Set<String> sourceTag1, sourceTag2, sourceTag3, sourceTag4;
   private Map<Set<String>, PlacementConstraint> constraintMap1, constraintMap2;
 
-  @Before
+  @BeforeEach
   public void before() {
     this.pcm = createPCM();
 
@@ -93,94 +99,94 @@ public class TestPlacementConstraintManagerService {
 
   @Test
   public void testRegisterUnregisterApps() {
-    Assert.assertEquals(0, pcm.getNumRegisteredApplications());
+    assertEquals(0, pcm.getNumRegisteredApplications());
 
     // Register two applications.
     pcm.registerApplication(appId1, constraintMap1);
-    Assert.assertEquals(1, pcm.getNumRegisteredApplications());
+    assertEquals(1, pcm.getNumRegisteredApplications());
     Map<Set<String>, PlacementConstraint> constrMap =
         pcm.getConstraints(appId1);
-    Assert.assertNotNull(constrMap);
-    Assert.assertEquals(2, constrMap.size());
-    Assert.assertNotNull(constrMap.get(sourceTag1));
-    Assert.assertNotNull(constrMap.get(sourceTag2));
+    assertNotNull(constrMap);
+    assertEquals(2, constrMap.size());
+    assertNotNull(constrMap.get(sourceTag1));
+    assertNotNull(constrMap.get(sourceTag2));
 
     pcm.registerApplication(appId2, constraintMap2);
-    Assert.assertEquals(2, pcm.getNumRegisteredApplications());
+    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));
+    assertNotNull(constrMap);
+    assertEquals(1, constrMap.size());
+    assertNotNull(constrMap.get(sourceTag3));
+    assertNull(constrMap.get(sourceTag2));
 
     // Try to register the same app again.
     pcm.registerApplication(appId2, constraintMap1);
-    Assert.assertEquals(2, pcm.getNumRegisteredApplications());
+    assertEquals(2, pcm.getNumRegisteredApplications());
 
     // Unregister appId1.
     pcm.unregisterApplication(appId1);
-    Assert.assertEquals(1, pcm.getNumRegisteredApplications());
-    Assert.assertNull(pcm.getConstraints(appId1));
-    Assert.assertNotNull(pcm.getConstraints(appId2));
+    assertEquals(1, pcm.getNumRegisteredApplications());
+    assertNull(pcm.getConstraints(appId1));
+    assertNotNull(pcm.getConstraints(appId2));
   }
 
   @Test
   public void testAddConstraint() {
     // Cannot add constraint to unregistered app.
-    Assert.assertEquals(0, pcm.getNumRegisteredApplications());
+    assertEquals(0, pcm.getNumRegisteredApplications());
     pcm.addConstraint(appId1, sourceTag1, c1, false);
-    Assert.assertEquals(0, pcm.getNumRegisteredApplications());
+    assertEquals(0, pcm.getNumRegisteredApplications());
 
     // Register application.
     pcm.registerApplication(appId1, new HashMap<>());
-    Assert.assertEquals(1, pcm.getNumRegisteredApplications());
-    Assert.assertEquals(0, pcm.getConstraints(appId1).size());
+    assertEquals(1, pcm.getNumRegisteredApplications());
+    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());
+    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));
+    assertEquals(2, pcm.getConstraints(appId1).size());
+    assertEquals(c1, pcm.getConstraint(appId1, sourceTag1));
+    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));
+    assertEquals(2, pcm.getConstraints(appId1).size());
+    assertEquals(c2, pcm.getConstraint(appId1, sourceTag1));
   }
 
   @Test
   public void testGlobalConstraints() {
-    Assert.assertEquals(0, pcm.getNumGlobalConstraints());
+    assertEquals(0, pcm.getNumGlobalConstraints());
     pcm.addGlobalConstraint(sourceTag1, c1, false);
-    Assert.assertEquals(1, pcm.getNumGlobalConstraints());
-    Assert.assertNotNull(pcm.getGlobalConstraint(sourceTag1));
+    assertEquals(1, pcm.getNumGlobalConstraints());
+    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));
+    assertEquals(1, pcm.getNumGlobalConstraints());
+    assertEquals(c1, pcm.getGlobalConstraint(sourceTag1));
+    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));
+    assertEquals(1, pcm.getNumGlobalConstraints());
+    assertEquals(c2, pcm.getGlobalConstraint(sourceTag1));
 
     pcm.removeGlobalConstraint(sourceTag1);
-    Assert.assertEquals(0, pcm.getNumGlobalConstraints());
+    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));
+    assertTrue(pcm.validateConstraint(sourceTag1, c1));
+    assertFalse(pcm.validateConstraint(sourceTag4, c1));
   }
 
   @Test
@@ -194,10 +200,10 @@ public class TestPlacementConstraintManagerService {
     // AC = null
     // GC = null
     constraint = pcm.getMultilevelConstraint(appId1, null, c1);
-    Assert.assertTrue(constraint.getConstraintExpr() instanceof And);
+    assertTrue(constraint.getConstraintExpr() instanceof And);
     mergedConstraint = (And) constraint.getConstraintExpr();
-    Assert.assertEquals(1, mergedConstraint.getChildren().size());
-    Assert.assertEquals(c1, mergedConstraint.getChildren().get(0).build());
+    assertEquals(1, mergedConstraint.getChildren().size());
+    assertEquals(c1, mergedConstraint.getChildren().get(0).build());
 
     // RC = null
     // AC = tag1->c1, tag2->c2
@@ -207,17 +213,17 @@ public class TestPlacementConstraintManagerService {
     // 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);
+    assertTrue(constraint.getConstraintExpr() instanceof And);
     mergedConstraint = (And) constraint.getConstraintExpr();
     // AND()
-    Assert.assertEquals(0, mergedConstraint.getChildren().size());
+    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);
+    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());
+    assertEquals(1, mergedConstraint.getChildren().size());
+    assertEquals(c1, mergedConstraint.getChildren().get(0).build());
     pcm.unregisterApplication(appId1);
 
     // RC = null
@@ -226,11 +232,11 @@ public class TestPlacementConstraintManagerService {
     pcm.addGlobalConstraint(sourceTag1, c1, true);
     constraint = pcm.getMultilevelConstraint(appId1,
         Sets.newHashSet(sourceTag1), null);
-    Assert.assertTrue(constraint.getConstraintExpr() instanceof And);
+    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());
+    assertEquals(1, mergedConstraint.getChildren().size());
+    assertEquals(c1, mergedConstraint.getChildren().get(0).build());
     pcm.removeGlobalConstraint(sourceTag1);
 
     // RC = c2
@@ -240,10 +246,10 @@ public class TestPlacementConstraintManagerService {
     pcm.registerApplication(appId1, constraintMap1);
     // both RC, AC and GC should be respected
     constraint = pcm.getMultilevelConstraint(appId1, sourceTag1, c2);
-    Assert.assertTrue(constraint.getConstraintExpr() instanceof And);
+    assertTrue(constraint.getConstraintExpr() instanceof And);
     mergedConstraint = (And) constraint.getConstraintExpr();
     // AND(c1, c2, c3)
-    Assert.assertEquals(3, mergedConstraint.getChildren().size());
+    assertEquals(3, mergedConstraint.getChildren().size());
     pcm.removeGlobalConstraint(sourceTag1);
     pcm.unregisterApplication(appId1);
 
@@ -254,10 +260,10 @@ public class TestPlacementConstraintManagerService {
     pcm.registerApplication(appId1, constraintMap1);
     constraint = pcm.getMultilevelConstraint(appId1,
         Sets.newHashSet(sourceTag1), c1);
-    Assert.assertTrue(constraint.getConstraintExpr() instanceof And);
+    assertTrue(constraint.getConstraintExpr() instanceof And);
     mergedConstraint = (And) constraint.getConstraintExpr();
     // AND(c1, c2)
-    Assert.assertEquals(2, mergedConstraint.getChildren().size());
+    assertEquals(2, mergedConstraint.getChildren().size());
     pcm.removeGlobalConstraint(sourceTag1);
     pcm.unregisterApplication(appId1);
   }

+ 99 - 95
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

@@ -26,7 +26,13 @@ import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.maxCardin
 import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.and;
 import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.or;
 import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.PlacementTargets.allocationTag;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.when;
 
 import java.util.AbstractMap;
@@ -67,12 +73,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 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.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet;
-import org.mockito.Mockito;
 
 /**
  * Test the PlacementConstraint Utility class functionality.
@@ -89,7 +93,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 +200,9 @@ public class TestPlacementConstraintsUtil {
       SchedulerNode schedulerNode =newSchedulerNode(currentNode.getHostName(),
           currentNode.getRackName(), currentNode.getNodeID());
 
-      Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+      assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
           createSchedulingRequest(sourceTag1), schedulerNode, pcm, tm));
-      Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+      assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
           createSchedulingRequest(sourceTag2), schedulerNode, pcm, tm));
     }
     /**
@@ -225,24 +229,24 @@ public class TestPlacementConstraintsUtil {
     tm.addContainer(n0_r1.getNodeID(), hbase_m, ImmutableSet.of("hbase-m"));
 
     // 'spark' placement on Node0 should now SUCCEED
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag1), schedulerNode0, pcm, tm));
     // FAIL on the rest of the nodes
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag1), schedulerNode1, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag1), schedulerNode2, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag1), schedulerNode3, pcm, tm));
 
     // Test diagnostics collector
     DiagnosticsCollector collector =
         new GenericDiagnosticsCollector();
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag1), schedulerNode1, pcm, tm,
         Optional.of(collector)));
-    Assert.assertNotNull(collector.getDiagnostics());
-    Assert.assertTrue(collector.getDiagnostics().contains("ALLOCATION_TAG"));
+    assertNotNull(collector.getDiagnostics());
+    assertTrue(collector.getDiagnostics().contains("ALLOCATION_TAG"));
   }
 
   @Test
@@ -296,13 +300,13 @@ public class TestPlacementConstraintsUtil {
 
     // 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,
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(st1), schedulerNode0, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(st1), schedulerNode1, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(st1), schedulerNode2, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(st1), schedulerNode3, pcm, tm));
 
     /**
@@ -318,13 +322,13 @@ public class TestPlacementConstraintsUtil {
     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,
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(st2), schedulerNode0, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(st2), schedulerNode1, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(st2), schedulerNode2, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(st2), schedulerNode3, pcm, tm));
   }
 
@@ -360,15 +364,15 @@ public class TestPlacementConstraintsUtil {
         n3_r2.getRackName(), n3_r2.getNodeID());
 
     // 'zk' placement on Rack1 should now SUCCEED
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag2), schedulerNode0, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag2), schedulerNode1, pcm, tm));
 
     // FAIL on the rest of the RACKs
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag2), schedulerNode2, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag2), schedulerNode3, pcm, tm));
   }
 
@@ -405,14 +409,14 @@ public class TestPlacementConstraintsUtil {
     tm.addContainer(n0_r1.getNodeID(), hbase_m, ImmutableSet.of("hbase-m"));
 
     // 'spark' placement on Node0 should now FAIL
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag1), schedulerNode0, pcm, tm));
     // SUCCEED on the rest of the nodes
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag1), schedulerNode1, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag1), schedulerNode2, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag1), schedulerNode3, pcm, tm));
   }
 
@@ -448,15 +452,15 @@ public class TestPlacementConstraintsUtil {
         n3_r2.getRackName(), n3_r2.getNodeID());
 
     // 'zk' placement on Rack1 should FAIL
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag2), schedulerNode0, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag2), schedulerNode1, pcm, tm));
 
     // SUCCEED on the rest of the RACKs
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag2), schedulerNode2, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag2), schedulerNode3, pcm, tm));
   }
 
@@ -484,9 +488,9 @@ public class TestPlacementConstraintsUtil {
         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())
+    assertEquals(1L, tm.getAllocationTagsWithCount(n0r1.getNodeID())
         .get("hbase-m").longValue());
-    Assert.assertEquals(1L, tm.getAllocationTagsWithCount(n2r2.getNodeID())
+    assertEquals(1L, tm.getAllocationTagsWithCount(n2r2.getNodeID())
         .get("hbase-rs").longValue());
 
     SchedulerNode schedulerNode0 =newSchedulerNode(n0r1.getHostName(),
@@ -500,13 +504,13 @@ public class TestPlacementConstraintsUtil {
 
     // n0 and n2 should be qualified for allocation as
     // they either have hbase-m or hbase-rs tag
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag1), schedulerNode0, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag1), schedulerNode1, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag1), schedulerNode2, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag1), schedulerNode3, pcm, tm));
 
     /**
@@ -519,7 +523,7 @@ public class TestPlacementConstraintsUtil {
     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,
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag1), schedulerNode3, pcm, tm));
 
     /**
@@ -534,13 +538,13 @@ public class TestPlacementConstraintsUtil {
         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,
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag2), schedulerNode0, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag2), schedulerNode1, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag2), schedulerNode2, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag2), schedulerNode3, pcm, tm));
   }
 
@@ -568,9 +572,9 @@ public class TestPlacementConstraintsUtil {
         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())
+    assertEquals(1L, tm.getAllocationTagsWithCount(n0r1.getNodeID())
         .get("hbase-m").longValue());
-    Assert.assertEquals(1L, tm.getAllocationTagsWithCount(n2r2.getNodeID())
+    assertEquals(1L, tm.getAllocationTagsWithCount(n2r2.getNodeID())
         .get("hbase-m").longValue());
 
     SchedulerNode schedulerNode0 =newSchedulerNode(n0r1.getHostName(),
@@ -584,13 +588,13 @@ public class TestPlacementConstraintsUtil {
 
     // 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,
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag1), schedulerNode0, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag1), schedulerNode1, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag1), schedulerNode2, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag1), schedulerNode3, pcm, tm));
 
     /**
@@ -604,17 +608,17 @@ public class TestPlacementConstraintsUtil {
       tm.addContainer(n1r1.getNodeID(),
           newContainerId(appId1, i+2), ImmutableSet.of("spark"));
     }
-    Assert.assertEquals(4L, tm.getAllocationTagsWithCount(n1r1.getNodeID())
+    assertEquals(4L, tm.getAllocationTagsWithCount(n1r1.getNodeID())
         .get("spark").longValue());
 
     // Violate cardinality constraint
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag1), schedulerNode0, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag1), schedulerNode1, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag1), schedulerNode2, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
         createSchedulingRequest(sourceTag1), schedulerNode3, pcm, tm));
   }
 
@@ -684,16 +688,16 @@ public class TestPlacementConstraintsUtil {
     constraintMap.put(srcTags1, constraint1);
     pcm.registerApplication(application1, constraintMap);
 
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
         application1, createSchedulingRequest(srcTags1),
         schedulerNode0, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
         application1, createSchedulingRequest(srcTags1),
         schedulerNode1, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
         application1, createSchedulingRequest(srcTags1),
         schedulerNode2, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
         application1, createSchedulingRequest(srcTags1),
         schedulerNode3, pcm, tm));
 
@@ -710,16 +714,16 @@ public class TestPlacementConstraintsUtil {
     constraintMap.put(srcTags2, constraint2);
     pcm.registerApplication(application2, constraintMap);
 
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
         application2, createSchedulingRequest(srcTags2),
         schedulerNode0, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
         application2, createSchedulingRequest(srcTags2),
         schedulerNode1, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
         application2, createSchedulingRequest(srcTags2),
         schedulerNode2, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
         application2, createSchedulingRequest(srcTags2),
         schedulerNode3, pcm, tm));
 
@@ -736,16 +740,16 @@ public class TestPlacementConstraintsUtil {
     constraintMap.put(srcTags3, constraint3);
     pcm.registerApplication(application3, constraintMap);
 
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
         application3, createSchedulingRequest(srcTags3),
         schedulerNode0, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
         application3, createSchedulingRequest(srcTags3),
         schedulerNode1, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
         application3, createSchedulingRequest(srcTags3),
         schedulerNode2, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
         application3, createSchedulingRequest(srcTags3),
         schedulerNode3, pcm, tm));
 
@@ -768,7 +772,7 @@ public class TestPlacementConstraintsUtil {
     allApps.put(application3, new MockRMApp(125, 1002,
         RMAppState.NEW, "userA", ImmutableSet.of("")));
 
-    RMContext mockedContext = Mockito.spy(rmContext);
+    RMContext mockedContext = spy(rmContext);
     when(mockedContext.getRMApps()).thenReturn(allApps);
 
     AllocationTagsManager tm = new AllocationTagsManager(mockedContext);
@@ -831,16 +835,16 @@ public class TestPlacementConstraintsUtil {
     constraintMap.put(srcTags1, constraint1);
     pcm.registerApplication(application1, constraintMap);
 
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
         application1, createSchedulingRequest(srcTags1),
         schedulerNode0, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
         application1, createSchedulingRequest(srcTags1),
         schedulerNode1, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
         application1, createSchedulingRequest(srcTags1),
         schedulerNode2, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
         application1, createSchedulingRequest(srcTags1),
         schedulerNode3, pcm, tm));
 
@@ -859,16 +863,16 @@ public class TestPlacementConstraintsUtil {
     cm2.put(srcTags2, constraint2);
     pcm.registerApplication(application1, cm2);
 
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
         application1, createSchedulingRequest(srcTags2),
         schedulerNode0, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
         application1, createSchedulingRequest(srcTags2),
         schedulerNode1, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
         application1, createSchedulingRequest(srcTags2),
         schedulerNode2, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
         application1, createSchedulingRequest(srcTags2),
         schedulerNode3, pcm, tm));
 
@@ -906,9 +910,9 @@ public class TestPlacementConstraintsUtil {
         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())
+    assertEquals(1L, tm.getAllocationTagsWithCount(n0r1.getNodeID())
         .get("hbase-m").longValue());
-    Assert.assertEquals(1L, tm.getAllocationTagsWithCount(n2r2.getNodeID())
+    assertEquals(1L, tm.getAllocationTagsWithCount(n2r2.getNodeID())
         .get("hbase-m").longValue());
 
     SchedulerNode schedulerNode0 =newSchedulerNode(n0r1.getHostName(),
@@ -935,16 +939,16 @@ public class TestPlacementConstraintsUtil {
 
     // 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(
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
         application2, createSchedulingRequest(srcTags2),
         schedulerNode0, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
         application2, createSchedulingRequest(srcTags2),
         schedulerNode1, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
         application2, createSchedulingRequest(srcTags2),
         schedulerNode2, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
         application2, createSchedulingRequest(srcTags2),
         schedulerNode3, pcm, tm));
 
@@ -971,16 +975,16 @@ public class TestPlacementConstraintsUtil {
         newContainerId(application3, 0), ImmutableSet.of("hbase-m"));
 
     // Anti-affinity to self/hbase-m
-    Assert.assertFalse(PlacementConstraintsUtil
+    assertFalse(PlacementConstraintsUtil
         .canSatisfyConstraints(application3, createSchedulingRequest(srcTags3),
             schedulerNode0, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil
+    assertTrue(PlacementConstraintsUtil
         .canSatisfyConstraints(application3, createSchedulingRequest(srcTags3),
             schedulerNode1, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil
+    assertTrue(PlacementConstraintsUtil
         .canSatisfyConstraints(application3, createSchedulingRequest(srcTags3),
             schedulerNode2, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil
+    assertTrue(PlacementConstraintsUtil
         .canSatisfyConstraints(application3, createSchedulingRequest(srcTags3),
             schedulerNode3, pcm, tm));
 
@@ -996,7 +1000,7 @@ public class TestPlacementConstraintsUtil {
 
     // app1: test-tag
     // app2: N/A
-    RMContext mockedContext = Mockito.spy(rmContext);
+    RMContext mockedContext = spy(rmContext);
     ConcurrentMap<ApplicationId, RMApp> allApps = new ConcurrentHashMap<>();
     allApps.put(application1, new MockRMApp(123, 1000,
         RMAppState.NEW, "userA", ImmutableSet.of("test-tag")));
@@ -1052,16 +1056,16 @@ public class TestPlacementConstraintsUtil {
     // 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(
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
         application2, createSchedulingRequest(srcTags2),
         schedulerNode0, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
         application2, createSchedulingRequest(srcTags2),
         schedulerNode1, pcm, tm));
-    Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
+    assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
         application2, createSchedulingRequest(srcTags2),
         schedulerNode2, pcm, tm));
-    Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
+    assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
         application2, createSchedulingRequest(srcTags2),
         schedulerNode3, pcm, tm));
 
@@ -1094,10 +1098,10 @@ public class TestPlacementConstraintsUtil {
       PlacementConstraintsUtil.canSatisfyConstraints(application1,
           createSchedulingRequest(srcTags1, constraint1), schedulerNode0,
           pcm, tm);
-      Assert.fail("This should fail because we gave an invalid namespace");
+      fail("This should fail because we gave an invalid namespace");
     } catch (Exception e) {
-      Assert.assertTrue(e instanceof InvalidAllocationTagsQueryException);
-      Assert.assertTrue(e.getMessage()
+      assertTrue(e instanceof InvalidAllocationTagsQueryException);
+      assertTrue(e.getMessage()
           .contains("Invalid namespace prefix: unknown_namespace"));
     }
   }

+ 66 - 52
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,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 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.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -70,6 +70,10 @@ import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetCar
 import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetIn;
 import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetNotIn;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
 /**
  * This tests end2end workflow of the constraint placement framework.
  */
@@ -82,7 +86,7 @@ public class TestPlacementProcessor {
   private MockRM rm;
   private DrainDispatcher dispatcher;
 
-  @Before
+  @BeforeEach
   public void createAndStartRM() {
     CapacitySchedulerConfiguration csConf =
         new CapacitySchedulerConfiguration();
@@ -107,14 +111,15 @@ public class TestPlacementProcessor {
     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<NodeId, MockNM> nodes = new HashMap<>();
     MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
@@ -157,18 +162,19 @@ public class TestPlacementProcessor {
     waitForContainerAllocation(nodes.values(), am1,
         allocatedContainers, new ArrayList<>(), 4);
 
-    Assert.assertEquals(4, allocatedContainers.size());
+    assertEquals(4, allocatedContainers.size());
     Set<NodeId> nodeIds = allocatedContainers.stream().map(x -> x.getNodeId())
         .collect(Collectors.toSet());
     // Ensure unique nodes (antiaffinity)
-    Assert.assertEquals(4, nodeIds.size());
+    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<NodeId, MockNM> nodes = new HashMap<>();
     MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
@@ -219,18 +225,19 @@ public class TestPlacementProcessor {
     waitForContainerAllocation(nodes.values(), am1,
         allocatedContainers, new ArrayList<>(), 5);
 
-    Assert.assertEquals(5, allocatedContainers.size());
+    assertEquals(5, allocatedContainers.size());
     Set<NodeId> nodeIds = allocatedContainers.stream().map(x -> x.getNodeId())
         .collect(Collectors.toSet());
     // Ensure unique nodes (antiaffinity)
-    Assert.assertEquals(5, nodeIds.size());
+    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<NodeId, MockNM> nodes = new HashMap<>();
     MockNM nm1 = new MockNM("h1:1234", 8192, rm.getResourceTrackerService());
@@ -277,13 +284,13 @@ public class TestPlacementProcessor {
     waitForContainerAllocation(nodes.values(), am1,
         allocatedContainers, new ArrayList<>(), 8);
 
-    Assert.assertEquals(8, allocatedContainers.size());
+    assertEquals(8, allocatedContainers.size());
     Map<NodeId, Long> 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);
+      assertTrue(nodeIdContainerIdMap.get(n) < 5);
     }
 
     QueueMetrics metrics = rm.getResourceScheduler().getRootQueueMetrics();
@@ -291,7 +298,8 @@ public class TestPlacementProcessor {
     verifyMetrics(metrics, 23552, 23, 9216, 9, 9);
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testAffinityPlacement() throws Exception {
     HashMap<NodeId, MockNM> nodes = new HashMap<>();
     MockNM nm1 = new MockNM("h1:1234", 8192, rm.getResourceTrackerService());
@@ -336,18 +344,19 @@ public class TestPlacementProcessor {
     waitForContainerAllocation(nodes.values(), am1,
         allocatedContainers, new ArrayList<>(), 5);
 
-    Assert.assertEquals(5, allocatedContainers.size());
+    assertEquals(5, allocatedContainers.size());
     Set<NodeId> 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());
+    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<NodeId, MockNM> nodes = new HashMap<>();
     MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
@@ -398,13 +407,13 @@ public class TestPlacementProcessor {
     waitForContainerAllocation(nodes.values(), am1,
         allocatedContainers, new ArrayList<>(), 6);
 
-    Assert.assertEquals(6, allocatedContainers.size());
+    assertEquals(6, allocatedContainers.size());
     Map<NodeId, Long> 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);
+      assertTrue(nodeIdContainerIdMap.get(n) < 4);
     }
 
     QueueMetrics metrics = rm.getResourceScheduler().getRootQueueMetrics();
@@ -412,7 +421,8 @@ public class TestPlacementProcessor {
     verifyMetrics(metrics, 9216, 9, 7168, 7, 7);
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testSchedulerRejection() throws Exception {
     stopRM();
     CapacitySchedulerConfiguration csConf =
@@ -494,14 +504,14 @@ public class TestPlacementProcessor {
       }
     }
 
-    Assert.assertEquals(3, allocatedContainers.size());
+    assertEquals(3, allocatedContainers.size());
     Set<NodeId> nodeIds = allocatedContainers.stream()
         .map(x -> x.getNodeId()).collect(Collectors.toSet());
     // Ensure unique nodes
-    Assert.assertEquals(3, nodeIds.size());
+    assertEquals(3, nodeIds.size());
     RejectedSchedulingRequest rej = rejectedReqs.get(0);
-    Assert.assertEquals(4, rej.getRequest().getAllocationRequestId());
-    Assert.assertEquals(RejectionReason.COULD_NOT_SCHEDULE_ON_NODE,
+    assertEquals(4, rej.getRequest().getAllocationRequestId());
+    assertEquals(RejectionReason.COULD_NOT_SCHEDULE_ON_NODE,
         rej.getReason());
 
     QueueMetrics metrics = rm.getResourceScheduler().getRootQueueMetrics();
@@ -509,7 +519,8 @@ public class TestPlacementProcessor {
     verifyMetrics(metrics, 12288, 12, 4096, 4, 4);
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testNodeCapacityRejection() throws Exception {
     HashMap<NodeId, MockNM> nodes = new HashMap<>();
     MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
@@ -573,14 +584,14 @@ public class TestPlacementProcessor {
       }
     }
 
-    Assert.assertEquals(3, allocatedContainers.size());
+    assertEquals(3, allocatedContainers.size());
     Set<NodeId> nodeIds = allocatedContainers.stream()
         .map(x -> x.getNodeId()).collect(Collectors.toSet());
     // Ensure unique nodes
-    Assert.assertEquals(3, nodeIds.size());
+    assertEquals(3, nodeIds.size());
     RejectedSchedulingRequest rej = rejectedReqs.get(0);
-    Assert.assertEquals(4, rej.getRequest().getAllocationRequestId());
-    Assert.assertEquals(RejectionReason.COULD_NOT_PLACE_ON_NODE,
+    assertEquals(4, rej.getRequest().getAllocationRequestId());
+    assertEquals(RejectionReason.COULD_NOT_PLACE_ON_NODE,
         rej.getReason());
 
     QueueMetrics metrics = rm.getResourceScheduler().getRootQueueMetrics();
@@ -588,7 +599,8 @@ public class TestPlacementProcessor {
     verifyMetrics(metrics, 12288, 12, 4096, 4, 4);
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testRePlacementAfterSchedulerRejection() throws Exception {
     stopRM();
     CapacitySchedulerConfiguration csConf =
@@ -672,18 +684,19 @@ public class TestPlacementProcessor {
       }
     }
 
-    Assert.assertEquals(4, allocatedContainers.size());
+    assertEquals(4, allocatedContainers.size());
     Set<NodeId> nodeIds = allocatedContainers.stream()
         .map(x -> x.getNodeId()).collect(Collectors.toSet());
     // Ensure unique nodes
-    Assert.assertEquals(4, nodeIds.size());
+    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<NodeId, MockNM> nodes = new HashMap<>();
     MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
@@ -748,13 +761,13 @@ public class TestPlacementProcessor {
       }
     }
 
-    Assert.assertEquals(4, allocatedContainers.size());
+    assertEquals(4, allocatedContainers.size());
     Set<NodeId> nodeIds = allocatedContainers.stream()
         .map(x -> x.getNodeId()).collect(Collectors.toSet());
     // Ensure unique nodes
-    Assert.assertEquals(4, nodeIds.size());
+    assertEquals(4, nodeIds.size());
     RejectedSchedulingRequest rej = rejectedReqs.get(0);
-    Assert.assertEquals(COULD_NOT_PLACE_ON_NODE,
+    assertEquals(COULD_NOT_PLACE_ON_NODE,
         rej.getReason());
 
     QueueMetrics metrics = rm.getResourceScheduler().getRootQueueMetrics();
@@ -762,7 +775,8 @@ public class TestPlacementProcessor {
     verifyMetrics(metrics, 11264, 11, 5120, 5, 5);
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testAndOrPlacement() throws Exception {
     HashMap<NodeId, MockNM> nodes = new HashMap<>();
     MockNM nm1 = new MockNM("h1:1234", 40960, 100,
@@ -824,7 +838,7 @@ public class TestPlacementProcessor {
     waitForContainerAllocation(nodes.values(), am1,
         allocatedContainers, new ArrayList<>(), 3);
     printTags(nodes.values(), rm.getRMContext().getAllocationTagsManager());
-    Assert.assertEquals(3, allocatedContainers.size());
+    assertEquals(3, allocatedContainers.size());
 
     /** Testing AND placement constraint**/
     // Now allocates a bar container, as restricted by the AND constraint,
@@ -837,7 +851,7 @@ public class TestPlacementProcessor {
     waitForContainerAllocation(nodes.values(), am1,
         allocatedContainers, new ArrayList<>(), 1);
     printTags(nodes.values(), rm.getRMContext().getAllocationTagsManager());
-    Assert.assertEquals(1, allocatedContainers.size());
+    assertEquals(1, allocatedContainers.size());
     NodeId barNode = allocatedContainers.get(0).getNodeId();
 
     // Sends another 3 bar request, 2 of them can be allocated
@@ -854,15 +868,15 @@ public class TestPlacementProcessor {
     waitForContainerAllocation(nodes.values(), am1,
         allocatedContainers, rejectedContainers, 2);
     printTags(nodes.values(), rm.getRMContext().getAllocationTagsManager());
-    Assert.assertEquals(2, allocatedContainers.size());
-    Assert.assertTrue(allocatedContainers.stream().allMatch(
+    assertEquals(2, allocatedContainers.size());
+    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,
+    assertEquals(1, rejectedContainers.size());
+    assertEquals(COULD_NOT_PLACE_ON_NODE,
         rejectedContainers.get(0).getReason());
 
     /** Testing OR placement constraint**/
@@ -884,10 +898,10 @@ public class TestPlacementProcessor {
 
     // 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());
+    assertEquals(20, allocatedContainers.size());
     for (Container mooContainer : allocatedContainers) {
       // nm5 has no moo allocated containers.
-      Assert.assertFalse(mooContainer.getNodeId().equals(nm5.getNodeId()));
+      assertFalse(mooContainer.getNodeId().equals(nm5.getNodeId()));
     }
   }
 
@@ -946,12 +960,12 @@ public class TestPlacementProcessor {
   private static void verifyMetrics(QueueMetrics metrics, long availableMB,
       int availableVirtualCores, long allocatedMB,
       int allocatedVirtualCores, int allocatedContainers) {
-    Assert.assertEquals(availableMB, metrics.getAvailableMB());
-    Assert.assertEquals(availableVirtualCores,
+    assertEquals(availableMB, metrics.getAvailableMB());
+    assertEquals(availableVirtualCores,
         metrics.getAvailableVirtualCores());
-    Assert.assertEquals(allocatedMB, metrics.getAllocatedMB());
-    Assert.assertEquals(allocatedVirtualCores,
+    assertEquals(allocatedMB, metrics.getAllocatedMB());
+    assertEquals(allocatedVirtualCores,
         metrics.getAllocatedVirtualCores());
-    Assert.assertEquals(allocatedContainers, metrics.getAllocatedContainers());
+    assertEquals(allocatedContainers, metrics.getAllocatedContainers());
   }
 }

+ 9 - 7
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,10 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.algorithm;
 
-import org.junit.Assert;
-import org.junit.Test;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+import org.junit.jupiter.api.Test;
 
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -39,7 +41,7 @@ public class TestCircularIterator {
     while (ci.hasNext()) {
       sb.append(ci.next());
     }
-    Assert.assertEquals("abcd", sb.toString());
+    assertEquals("abcd", sb.toString());
 
     Iterator<String> lIter = list.iterator();
     lIter.next();
@@ -49,7 +51,7 @@ public class TestCircularIterator {
     while (ci.hasNext()) {
       sb.append(ci.next());
     }
-    Assert.assertEquals("cdab", sb.toString());
+    assertEquals("cdab", sb.toString());
 
     lIter = list.iterator();
     lIter.next();
@@ -60,7 +62,7 @@ public class TestCircularIterator {
     while (ci.hasNext()) {
       sb.append(ci.next());
     }
-    Assert.assertEquals("xdabc", sb.toString());
+    assertEquals("xdabc", sb.toString());
 
     list = Arrays.asList("a");
     lIter = list.iterator();
@@ -70,13 +72,13 @@ public class TestCircularIterator {
     while (ci.hasNext()) {
       sb.append(ci.next());
     }
-    Assert.assertEquals("ya", sb.toString());
+    assertEquals("ya", sb.toString());
 
     try {
       list = new ArrayList<>();
       lIter = list.iterator();
       new CircularIterator<>("y", lIter, list);
-      Assert.fail("Should fail..");
+      fail("Should fail..");
     } catch (Exception e) {
       // foo bar
     }

+ 16 - 14
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

@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.algorithm;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+
 import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -28,9 +31,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtil
 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.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 import java.util.List;
 
@@ -41,7 +43,7 @@ public class TestLocalAllocationTagsManager {
 
   private RMContext rmContext;
 
-  @Before
+  @BeforeEach
   public void setup() {
     MockRM rm = new MockRM();
     rm.start();
@@ -84,21 +86,21 @@ public class TestLocalAllocationTagsManager {
         TestUtils.getMockApplicationId(2), ImmutableSet.of("service"));
 
     // Expect tag mappings to be present including temp Tags
-    Assert.assertEquals(1,
+    assertEquals(1,
         atm.getNodeCardinalityByOp(NodeId.fromString("host1:123"),
             AllocationTags.createSingleAppAllocationTags(
                 TestUtils.getMockApplicationId(1),
                 ImmutableSet.of("mapper")),
             Long::sum));
 
-    Assert.assertEquals(1,
+    assertEquals(1,
         atm.getNodeCardinalityByOp(NodeId.fromString("host1:123"),
             AllocationTags.createSingleAppAllocationTags(
                 TestUtils.getMockApplicationId(1),
                 ImmutableSet.of("service")),
             Long::sum));
 
-    Assert.assertEquals(1,
+    assertEquals(1,
         atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"),
             AllocationTags.createSingleAppAllocationTags(
                 TestUtils.getMockApplicationId(2),
@@ -107,14 +109,14 @@ public class TestLocalAllocationTagsManager {
 
     // Do a temp Tag cleanup on app2
     ephAtm.cleanTempContainers(TestUtils.getMockApplicationId(2));
-    Assert.assertEquals(0,
+    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,
+    assertEquals(1,
         atm.getNodeCardinalityByOp(NodeId.fromString("host1:123"),
             AllocationTags.createSingleAppAllocationTags(
                 TestUtils.getMockApplicationId(1),
@@ -122,7 +124,7 @@ public class TestLocalAllocationTagsManager {
             Long::sum));
     // Do a cleanup on app1 as well
     ephAtm.cleanTempContainers(TestUtils.getMockApplicationId(1));
-    Assert.assertEquals(0,
+    assertEquals(0,
         atm.getNodeCardinalityByOp(NodeId.fromString("host1:123"),
             AllocationTags.createSingleAppAllocationTags(
                 TestUtils.getMockApplicationId(1),
@@ -130,14 +132,14 @@ public class TestLocalAllocationTagsManager {
             Long::sum));
 
     // Non temp-tags should be unaffected
-    Assert.assertEquals(1,
+    assertEquals(1,
         atm.getNodeCardinalityByOp(NodeId.fromString("host1:123"),
             AllocationTags.createSingleAppAllocationTags(
                 TestUtils.getMockApplicationId(1),
                 ImmutableSet.of("service")),
             Long::sum));
 
-    Assert.assertEquals(0,
+    assertEquals(0,
         atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"),
             AllocationTags.createSingleAppAllocationTags(
                 TestUtils.getMockApplicationId(2),
@@ -145,11 +147,11 @@ public class TestLocalAllocationTagsManager {
             Long::sum));
 
     // Expect app2 with no containers, and app1 with 2 containers across 2 nodes
-    Assert.assertEquals(2,
+    assertEquals(2,
         atm.getPerAppNodeMappings().get(TestUtils.getMockApplicationId(1))
             .getTypeToTagsWithCount().size());
 
-    Assert.assertNull(
+    assertNull(
         atm.getPerAppNodeMappings().get(TestUtils.getMockApplicationId(2)));
   }
 

+ 17 - 18
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,9 +36,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 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.mockito.Mockito;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -52,10 +51,10 @@ import java.util.List;
 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.mockito.ArgumentMatchers.anyLong;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.anyLong;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
@@ -79,7 +78,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() {
@@ -649,21 +648,21 @@ public class TestCentralizedOpportunisticContainerAllocator {
 
   private RMNode createRMNode(String host, int port, String rack,
       int queueLength, int queueCapacity) {
-    RMNode node1 = Mockito.mock(RMNode.class);
+    RMNode node1 = mock(RMNode.class);
     NodeId nID1 = new TestNodeQueueLoadMonitor.FakeNodeId(host, port);
-    Mockito.when(node1.getHostName()).thenReturn(host);
-    Mockito.when(node1.getRackName()).thenReturn(rack);
-    Mockito.when(node1.getNodeID()).thenReturn(nID1);
-    Mockito.when(node1.getState()).thenReturn(NodeState.RUNNING);
+    when(node1.getHostName()).thenReturn(host);
+    when(node1.getRackName()).thenReturn(rack);
+    when(node1.getNodeID()).thenReturn(nID1);
+    when(node1.getState()).thenReturn(NodeState.RUNNING);
     OpportunisticContainersStatus status1 =
-        Mockito.mock(OpportunisticContainersStatus.class);
-    Mockito.when(status1.getEstimatedQueueWaitTime())
+        mock(OpportunisticContainersStatus.class);
+    when(status1.getEstimatedQueueWaitTime())
         .thenReturn(-1);
-    Mockito.when(status1.getWaitQueueLength())
+    when(status1.getWaitQueueLength())
         .thenReturn(queueLength);
-    Mockito.when(status1.getOpportQueueCapacity())
+    when(status1.getOpportQueueCapacity())
         .thenReturn(queueCapacity);
-    Mockito.when(node1.getOpportunisticContainersStatus()).thenReturn(status1);
+    when(node1.getOpportunisticContainersStatus()).thenReturn(status1);
     return node1;
   }
 }

+ 116 - 112
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

@@ -18,6 +18,11 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.distributed;
 
+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.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap;
 import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes;
@@ -31,9 +36,8 @@ import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
 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.BeforeAll;
+import org.junit.jupiter.api.Test;
 import org.mockito.Mockito;
 
 import java.util.Collections;
@@ -123,7 +127,7 @@ public class TestNodeQueueLoadMonitor {
     ResourceUtils.initializeResourcesFromResourceInformationMap(riMap);
   }
 
-  @BeforeClass
+  @BeforeAll
   public static void classSetUp() {
     addNewTypesToResources(NETWORK_RESOURCE);
     defaultResourceRequested = newResourceInstance(128, 1, 1);
@@ -139,43 +143,43 @@ public class TestNodeQueueLoadMonitor {
     selector.updateNode(createRMNode("h3", 3, 10, 10));
     selector.computeTask.run();
     List<NodeId> 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());
+    assertEquals("h2:2", nodeIds.get(0).toString());
+    assertEquals("h3:3", nodeIds.get(1).toString());
+    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());
+    assertEquals("h3:3", nodeIds.get(0).toString());
+    assertEquals("h2:2", nodeIds.get(1).toString());
+    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());
+    assertEquals("h3:3", nodeIds.get(0).toString());
+    assertEquals("h2:2", nodeIds.get(1).toString());
+    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());
+    assertEquals(2, nodeIds.size());
+    assertEquals("h3:3", nodeIds.get(0).toString());
+    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());
+    assertEquals("h2:2", nodeIds.get(0).toString());
+    assertEquals("h3:3", nodeIds.get(1).toString());
+    assertEquals("h1:1", nodeIds.get(2).toString());
   }
 
   @Test
@@ -188,18 +192,18 @@ public class TestNodeQueueLoadMonitor {
     selector.computeTask.run();
     List<NodeId> 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());
+    assertEquals("h2:2", nodeIds.get(0).toString());
+    assertEquals("h3:3", nodeIds.get(1).toString());
+    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());
+    assertEquals("h3:3", nodeIds.get(0).toString());
+    assertEquals("h2:2", nodeIds.get(1).toString());
+    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 +211,10 @@ public class TestNodeQueueLoadMonitor {
     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());
+    assertEquals("h3:3", nodeIds.get(0).toString());
+    assertEquals("h2:2", nodeIds.get(1).toString());
+    assertEquals("h1:1", nodeIds.get(2).toString());
+    assertEquals("h4:4", nodeIds.get(3).toString());
 
     // Now update h3 and fill its queue.
     selector.updateNode(createRMNode("h3", 3, -1,
@@ -218,29 +222,29 @@ public class TestNodeQueueLoadMonitor {
     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());
+    assertEquals(3, nodeIds.size());
+    assertEquals("h2:2", nodeIds.get(0).toString());
+    assertEquals("h1:1", nodeIds.get(1).toString());
+    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());
+    assertEquals(2, nodeIds.size());
+    assertEquals("h1:1", nodeIds.get(0).toString());
+    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());
+    assertEquals(3, nodeIds.size());
+    assertEquals("h2:2", nodeIds.get(0).toString());
+    assertEquals("h1:1", nodeIds.get(1).toString());
+    assertEquals("h4:4", nodeIds.get(2).toString());
   }
 
   @Test
@@ -265,10 +269,10 @@ public class TestNodeQueueLoadMonitor {
         Resources.multiply(defaultResourceRequested, 2), defaultCapacity));
     selector.computeTask.run();
     List<NodeId> 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());
+    assertEquals("h2:2", nodeIds.get(0).toString());
+    assertEquals("h1:1", nodeIds.get(1).toString());
+    assertEquals("h3:3", nodeIds.get(2).toString());
+    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 +282,10 @@ public class TestNodeQueueLoadMonitor {
         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());
+    assertEquals("h2:2", nodeIds.get(0).toString());
+    assertEquals("h1:1", nodeIds.get(1).toString());
+    assertEquals("h4:4", nodeIds.get(2).toString());
+    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 +293,10 @@ public class TestNodeQueueLoadMonitor {
     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());
+    assertEquals(3, nodeIds.size());
+    assertEquals("h2:2", nodeIds.get(0).toString());
+    assertEquals("h1:1", nodeIds.get(1).toString());
+    assertEquals("h4:4", nodeIds.get(2).toString());
 
     // Now update h2 to Decommissioning state
     selector.updateNode(createRMNode("h2", 2, -1,
@@ -300,9 +304,9 @@ public class TestNodeQueueLoadMonitor {
     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());
+    assertEquals(2, nodeIds.size());
+    assertEquals("h1:1", nodeIds.get(0).toString());
+    assertEquals("h4:4", nodeIds.get(1).toString());
 
     // Now update h2 back to Running state
     selector.updateNode(createRMNode(
@@ -310,10 +314,10 @@ public class TestNodeQueueLoadMonitor {
         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());
+    assertEquals(3, nodeIds.size());
+    assertEquals("h2:2", nodeIds.get(0).toString());
+    assertEquals("h1:1", nodeIds.get(1).toString());
+    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 +327,10 @@ public class TestNodeQueueLoadMonitor {
         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());
+    assertEquals(3, nodeIds.size());
+    assertEquals("h2:2", nodeIds.get(0).toString());
+    assertEquals("h1:1", nodeIds.get(1).toString());
+    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 +345,9 @@ public class TestNodeQueueLoadMonitor {
         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());
+    assertEquals(2, nodeIds.size());
+    assertEquals("h1:1", nodeIds.get(0).toString());
+    assertEquals("h4:4", nodeIds.get(1).toString());
   }
 
   /**
@@ -360,33 +364,33 @@ public class TestNodeQueueLoadMonitor {
     selector.updateSortedNodes();
 
     ClusterNode clusterNode = selector.getClusterNodes().get(node.getNodeID());
-    Assert.assertEquals(Resources.none(),
+    assertEquals(Resources.none(),
         clusterNode.getAllocatedResource());
 
     // Has enough resources
     RMNode selectedNode = selector.selectAnyNode(
         Collections.emptySet(), defaultResourceRequested);
-    Assert.assertNotNull(selectedNode);
-    Assert.assertEquals(node.getNodeID(), selectedNode.getNodeID());
+    assertNotNull(selectedNode);
+    assertEquals(node.getNodeID(), selectedNode.getNodeID());
 
     clusterNode = selector.getClusterNodes().get(node.getNodeID());
-    Assert.assertEquals(defaultResourceRequested,
+    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());
+    assertNotNull(selectedNode);
+    assertEquals(node.getNodeID(), selectedNode.getNodeID());
 
     clusterNode = selector.getClusterNodes().get(node.getNodeID());
-    Assert.assertEquals(1, clusterNode.getQueueLength());
+    assertEquals(1, clusterNode.getQueueLength());
 
     // Does not have enough resources and cannot queue
     selectedNode = selector.selectAnyNode(
         Collections.emptySet(),
         Resources.add(defaultResourceRequested, defaultCapacity));
-    Assert.assertNull(selectedNode);
+    assertNull(selectedNode);
   }
 
   @Test
@@ -411,10 +415,10 @@ public class TestNodeQueueLoadMonitor {
         Resources.multiply(defaultResourceRequested, 4), defaultCapacity));
     selector.computeTask.run();
     List<NodeId> 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());
+    assertEquals("h1:1", nodeIds.get(0).toString());
+    assertEquals("h2:2", nodeIds.get(1).toString());
+    assertEquals("h3:3", nodeIds.get(2).toString());
+    assertEquals("h4:4", nodeIds.get(3).toString());
 
     // Now update node1 to have only defaultResourceRequested available
     // by changing its capability to 2x defaultResourceReqeusted
@@ -425,10 +429,10 @@ public class TestNodeQueueLoadMonitor {
         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());
+    assertEquals("h2:2", nodeIds.get(0).toString());
+    assertEquals("h3:3", nodeIds.get(1).toString());
+    assertEquals("h4:4", nodeIds.get(2).toString());
+    assertEquals("h1:1", nodeIds.get(3).toString());
 
     // Now update node2 to have no resources available
     // by changing its capability to 1x defaultResourceReqeusted
@@ -439,10 +443,10 @@ public class TestNodeQueueLoadMonitor {
         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());
+    assertEquals("h3:3", nodeIds.get(0).toString());
+    assertEquals("h4:4", nodeIds.get(1).toString());
+    assertEquals("h1:1", nodeIds.get(2).toString());
+    assertEquals("h2:2", nodeIds.get(3).toString());
   }
 
   @Test
@@ -458,12 +462,12 @@ public class TestNodeQueueLoadMonitor {
     QueueLimitCalculator calculator = selector.getThresholdCalculator();
     ContainerQueuingLimit containerQueuingLimit = calculator
         .createContainerQueuingLimit();
-    Assert.assertEquals(6, containerQueuingLimit.getMaxQueueLength());
-    Assert.assertEquals(-1, containerQueuingLimit.getMaxQueueWaitTimeInMs());
+    assertEquals(6, containerQueuingLimit.getMaxQueueLength());
+    assertEquals(-1, containerQueuingLimit.getMaxQueueWaitTimeInMs());
     selector.computeTask.run();
     containerQueuingLimit = calculator.createContainerQueuingLimit();
-    Assert.assertEquals(10, containerQueuingLimit.getMaxQueueLength());
-    Assert.assertEquals(-1, containerQueuingLimit.getMaxQueueWaitTimeInMs());
+    assertEquals(10, containerQueuingLimit.getMaxQueueLength());
+    assertEquals(-1, containerQueuingLimit.getMaxQueueWaitTimeInMs());
 
     // Test Limits do not exceed specified max
     selector.updateNode(createRMNode("h1", 1, -1, 110));
@@ -474,7 +478,7 @@ public class TestNodeQueueLoadMonitor {
     selector.updateNode(createRMNode("h6", 6, -1, 160));
     selector.computeTask.run();
     containerQueuingLimit = calculator.createContainerQueuingLimit();
-    Assert.assertEquals(100, containerQueuingLimit.getMaxQueueLength());
+    assertEquals(100, containerQueuingLimit.getMaxQueueLength());
 
     // Test Limits do not go below specified min
     selector.updateNode(createRMNode("h1", 1, -1, 1));
@@ -485,7 +489,7 @@ public class TestNodeQueueLoadMonitor {
     selector.updateNode(createRMNode("h6", 6, -1, 6));
     selector.computeTask.run();
     containerQueuingLimit = calculator.createContainerQueuingLimit();
-    Assert.assertEquals(6, containerQueuingLimit.getMaxQueueLength());
+    assertEquals(6, containerQueuingLimit.getMaxQueueLength());
 
   }
 
@@ -515,21 +519,21 @@ public class TestNodeQueueLoadMonitor {
     Set<String> blacklist = new HashSet<>();
     RMNode node = selector.selectLocalNode(
         "h1", blacklist, defaultResourceRequested);
-    Assert.assertEquals("h1", node.getHostName());
+    assertEquals("h1", node.getHostName());
 
     // if node has been added to blacklist
     blacklist.add("h1");
     node = selector.selectLocalNode(
         "h1", blacklist, defaultResourceRequested);
-    Assert.assertNull(node);
+    assertNull(node);
 
     node = selector.selectLocalNode(
         "h2", blacklist, defaultResourceRequested);
-    Assert.assertNull(node);
+    assertNull(node);
 
     node = selector.selectLocalNode(
         "h3", blacklist, defaultResourceRequested);
-    Assert.assertEquals("h3", node.getHostName());
+    assertEquals("h3", node.getHostName());
   }
 
   /**
@@ -558,22 +562,22 @@ public class TestNodeQueueLoadMonitor {
     Set<String> blacklist = new HashSet<>();
     RMNode node = selector.selectRackLocalNode(
         "rack1", blacklist, defaultResourceRequested);
-    Assert.assertEquals("h1", node.getHostName());
+    assertEquals("h1", node.getHostName());
 
     // if node has been added to blacklist
     blacklist.add("h1");
     node = selector.selectRackLocalNode(
         "rack1", blacklist, defaultResourceRequested);
-    Assert.assertNull(node);
+    assertNull(node);
 
     node = selector.selectRackLocalNode(
         "rack2", blacklist, defaultResourceRequested);
-    Assert.assertEquals("h3", node.getHostName());
+    assertEquals("h3", node.getHostName());
 
     blacklist.add("h3");
     node = selector.selectRackLocalNode(
         "rack2", blacklist, defaultResourceRequested);
-    Assert.assertNull(node);
+    assertNull(node);
   }
 
   /**
@@ -599,23 +603,23 @@ public class TestNodeQueueLoadMonitor {
 
     selector.computeTask.run();
 
-    Assert.assertEquals(2, selector.getSortedNodes().size());
+    assertEquals(2, selector.getSortedNodes().size());
 
     // basic test for selecting node which has queue length
     // less than queue capacity.
     Set<String> blacklist = new HashSet<>();
     RMNode node = selector.selectAnyNode(blacklist, defaultResourceRequested);
-    Assert.assertTrue(node.getHostName().equals("h1") ||
+    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());
+    assertEquals("h3", node.getHostName());
 
     blacklist.add("h3");
     node = selector.selectAnyNode(blacklist, defaultResourceRequested);
-    Assert.assertNull(node);
+    assertNull(node);
   }
 
   @Test
@@ -646,7 +650,7 @@ public class TestNodeQueueLoadMonitor {
 
       comparator.setClusterResource(
           Resources.add(cn1.getCapability(), cn2.getCapability()));
-      Assert.assertTrue(comparator.compare(cn1, cn2) < 0);
+      assertTrue(comparator.compare(cn1, cn2) < 0);
     }
 
     // Case 2: Shorter queue should be ranked first before comparing resources
@@ -669,7 +673,7 @@ public class TestNodeQueueLoadMonitor {
 
       comparator.setClusterResource(
           Resources.add(cn1.getCapability(), cn2.getCapability()));
-      Assert.assertTrue(comparator.compare(cn1, cn2) < 0);
+      assertTrue(comparator.compare(cn1, cn2) < 0);
     }
 
     // Case 3: No capability vs with capability,
@@ -693,7 +697,7 @@ public class TestNodeQueueLoadMonitor {
 
       comparator.setClusterResource(
           Resources.add(cn1.getCapability(), cn2.getCapability()));
-      Assert.assertTrue(comparator.compare(cn1, cn2) < 0);
+      assertTrue(comparator.compare(cn1, cn2) < 0);
     }
 
     // Case 4: Compare same values
@@ -716,7 +720,7 @@ public class TestNodeQueueLoadMonitor {
 
       comparator.setClusterResource(
           Resources.add(cn1.getCapability(), cn2.getCapability()));
-      Assert.assertEquals(0, comparator.compare(cn1, cn2));
+      assertEquals(0, comparator.compare(cn1, cn2));
     }
 
     // Case 5: If ratio is the same, compare raw values
@@ -741,7 +745,7 @@ public class TestNodeQueueLoadMonitor {
       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);
+      assertTrue(comparator.compare(cn1, cn2) < 0);
     }
 
     // Case 6: by VCores absolute value
@@ -764,7 +768,7 @@ public class TestNodeQueueLoadMonitor {
 
       comparator.setClusterResource(
           Resources.add(cn1.getCapability(), cn2.getCapability()));
-      Assert.assertTrue(comparator.compare(cn2, cn1) < 0);
+      assertTrue(comparator.compare(cn2, cn1) < 0);
     }
 
     // Case 7: by memory absolute value
@@ -787,7 +791,7 @@ public class TestNodeQueueLoadMonitor {
 
       comparator.setClusterResource(
           Resources.add(cn1.getCapability(), cn2.getCapability()));
-      Assert.assertTrue(comparator.compare(cn2, cn1) < 0);
+      assertTrue(comparator.compare(cn2, cn1) < 0);
     }
 
     // Case 8: Memory should be more constraining in the overall cluster,
@@ -811,7 +815,7 @@ public class TestNodeQueueLoadMonitor {
 
       comparator.setClusterResource(
           Resources.add(cn1.getCapability(), cn2.getCapability()));
-      Assert.assertTrue(comparator.compare(cn1, cn2) < 0);
+      assertTrue(comparator.compare(cn1, cn2) < 0);
     }
   }
 

+ 4 - 3
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,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptA
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
-import org.junit.Assert;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 import java.io.File;
@@ -379,9 +380,9 @@ public class FairSchedulerTestBase {
     }
 
     // available resource
-    Assert.assertEquals(resource.getMemorySize(),
+    assertEquals(resource.getMemorySize(),
         app.getCurrentConsumption().getMemorySize());
-    Assert.assertEquals(resource.getVirtualCores(),
+    assertEquals(resource.getVirtualCores(),
         app.getCurrentConsumption().getVirtualCores());
   }
 

+ 122 - 107
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.ControlledClock;
 import org.apache.hadoop.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,12 @@ import java.util.List;
 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.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -84,7 +86,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 +102,7 @@ public class TestAllocationFileLoaderService {
     when(scheduler.getRMContext()).thenReturn(rmContext);
   }
 
-  @After
+  @AfterEach
   public void teardown() {
     new File(ALLOC_FILE).delete();
   }
@@ -132,14 +134,15 @@ public class TestAllocationFileLoaderService {
     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);
-
-    allocLoader.getAllocationFile(conf);
+    assertThrows(UnsupportedFileSystemException.class, () -> {
+      conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, "badfs:///badfile");
+      AllocationFileLoaderService allocLoader =
+          new AllocationFileLoaderService(scheduler);
+      allocLoader.getAllocationFile(conf);
+    });
   }
 
   @Test
@@ -158,7 +161,8 @@ public class TestAllocationFileLoaderService {
     }
   }
 
-  @Test (timeout = 10000)
+  @Test
+  @Timeout(value = 10)
   public void testReload() throws Exception {
     AllocationFileWriter.create()
         .addQueue(new AllocationFileQueue.Builder("queueA")
@@ -359,23 +363,23 @@ public class TestAllocationFileLoaderService {
     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 +622,8 @@ public class TestAllocationFileLoaderService {
         .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 +631,67 @@ public class TestAllocationFileLoaderService {
    * 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 +842,43 @@ public class TestAllocationFileLoaderService {
    * 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 +910,10 @@ public class TestAllocationFileLoaderService {
     assertTrue(allocConf.isReservable(reservableQueuePath));
     Map<FSQueueType, Set<String>> 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 +938,26 @@ public class TestAllocationFileLoaderService {
    * 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 {

+ 9 - 9
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
 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 class TestAppRunnability extends FairSchedulerTestBase {
     scheduler = (FairScheduler) resourceManager.getResourceScheduler();
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
     if (resourceManager != null) {
       resourceManager.stop();

+ 22 - 15
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

@@ -18,6 +18,10 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+
 import java.io.File;
 
 import org.slf4j.Logger;
@@ -41,10 +45,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair
     .allocationfile.AllocationFileQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair
     .allocationfile.AllocationFileWriter;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 /**
  * Test Application master service using Fair scheduler.
@@ -60,7 +64,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 +85,14 @@ public class TestApplicationMasterServiceWithFS {
         .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 +116,10 @@ public class TestApplicationMasterServiceWithFS {
     am1.addRequests(new String[] { "127.0.0.1" }, MEMORY_ALLOCATION, 1, 1);
     try {
       allocateResponse = am1.schedule(); // send the request
-      Assert.fail();
+      fail();
     } catch (Exception e) {
-      Assert.assertTrue(e instanceof InvalidResourceRequestException);
-      Assert.assertEquals(
+      assertTrue(e instanceof InvalidResourceRequestException);
+      assertEquals(
           InvalidResourceRequestException.InvalidResourceType.GREATER_THEN_MAX_ALLOCATION,
           ((InvalidResourceRequestException) e).getInvalidResourceType());
 
@@ -123,12 +128,14 @@ public class TestApplicationMasterServiceWithFS {
     }
   }
 
-  @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 +171,16 @@ public class TestApplicationMasterServiceWithFS {
       try {
         allocateResponse = am1.schedule();
       } catch (Exception e) {
-        Assert.fail("Allocation should be successful");
+        fail("Allocation should be successful");
       }
       return allocateResponse.getAllocatedContainers().size() > 0;
     }, 1000, 10000);
 
     Container allocatedContainer =
         allocateResponse.getAllocatedContainers().get(0);
-    Assert.assertEquals(MEMORY_ALLOCATION,
+    assertEquals(MEMORY_ALLOCATION,
         allocatedContainer.getResource().getMemorySize());
-    Assert.assertEquals(1, allocatedContainer.getResource().getVirtualCores());
+    assertEquals(1, allocatedContainer.getResource().getVirtualCores());
     rm.stop();
   }
 }

+ 13 - 13
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

@@ -18,17 +18,17 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
 import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.yarn.api.records.ResourceInformation;
 
-import org.junit.Assert;
-
 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 @@ import org.junit.Test;
 public class TestComputeFairShares {
   private List<Schedulable> scheds;
   
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     scheds = new ArrayList<>();
   }
@@ -241,11 +241,11 @@ public class TestComputeFairShares {
    * 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);
+    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());
+      assertEquals(shares[i], scheds.get(i).getFairShare().getMemorySize(),
+          "Expected share number " + i + " in list wrong");
     }
   }
   
@@ -253,11 +253,11 @@ public class TestComputeFairShares {
    * 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);
+    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());
+      assertEquals(shares[i], scheds.get(i).getFairShare().getVirtualCores(),
+          "Expected share number " + i + " in list wrong");
     }
   }
 }

+ 9 - 8
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 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
 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,12 @@ public class TestConfigurableResource {
     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

+ 27 - 25
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,19 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemoved
 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 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 +86,7 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
   }
 
   @SuppressWarnings("deprecation")
-  @Before
+  @BeforeEach
   public void setup() {
     QueueMetrics.clearQueueMetrics();
     DefaultMetricsSystem.setMiniClusterMode(true);
@@ -106,7 +106,7 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
     assertEquals(mockClock, scheduler.getClock());
   }
 
-  @After
+  @AfterEach
   public void teardown() {
     if (resourceManager != null) {
       resourceManager.stop();
@@ -114,7 +114,8 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
     }
   }
 
-  @Test (timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testBasic() throws InterruptedException {
     // Add one node
     String host = "127.0.0.1";
@@ -146,7 +147,8 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
     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 +206,7 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
     while (it.hasNext()) {
       nodes.add(it.next().getContainer().getNodeId());
     }
-    Assert.assertEquals(2, nodes.size());
+    assertEquals(2, nodes.size());
   }
 
   @SuppressWarnings("deprecation")
@@ -226,8 +228,8 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
         .rollMasterKey();
 
     scheduler.setRMContext(resourceManager.getRMContext());
-    Assert.assertTrue("Continuous scheduling should be disabled.",
-        !scheduler.isContinuousSchedulingEnabled());
+    assertTrue(!scheduler.isContinuousSchedulingEnabled(),
+        "Continuous scheduling should be disabled.");
     scheduler.init(conf);
     scheduler.start();
 
@@ -242,15 +244,15 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
             "127.0.0.2");
     NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2);
     scheduler.handle(nodeEvent2);
-    Assert.assertEquals("We should have two alive nodes.",
-        2, scheduler.getNumClusterNodes());
+    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());
+    assertEquals(1, scheduler.getNumClusterNodes(),
+        "We should only have one alive node.");
 
     // Invoke the continuous scheduling once
     try {
@@ -284,16 +286,16 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
     scheduler.init(conf);
     scheduler.start();
     FairScheduler spyScheduler = spy(scheduler);
-    Assert.assertTrue("Continuous scheduling should be disabled.",
-        !spyScheduler.isContinuousSchedulingEnabled());
+    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());
+    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 +304,7 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
       spyScheduler.continuousSchedulingAttempt();
       fail("Expected InterruptedException to stop schedulingThread");
     } catch (InterruptedException e) {
-      Assert.assertEquals(ie, e);
+      assertEquals(ie, e);
     }
   }
 
@@ -320,7 +322,7 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
       Thread.sleep(50);
     }
 
-    assertNotEquals("The Scheduling thread is still alive", 0, numRetries);
+    assertNotEquals(0, numRetries, "The Scheduling thread is still alive");
   }
 
   @SuppressWarnings("deprecation")

+ 46 - 45
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,12 +28,14 @@ import org.apache.hadoop.conf.Configuration;
 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.mock;
 import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
 
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -57,13 +59,12 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.Fif
 
 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.mockito.Mockito;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 public class TestFSAppAttempt extends FairSchedulerTestBase {
 
-  @Before
+  @BeforeEach
   public void setup() {
     Configuration conf = createConfiguration();
     resourceManager = new MockRM(conf);
@@ -73,10 +74,10 @@ public class TestFSAppAttempt extends FairSchedulerTestBase {
 
   @Test
   public void testDelayScheduling() {
-    FSLeafQueue queue = Mockito.mock(FSLeafQueue.class);
-    Priority pri = Mockito.mock(Priority.class);
+    FSLeafQueue queue = mock(FSLeafQueue.class);
+    Priority pri = mock(Priority.class);
     SchedulerRequestKey prio = TestUtils.toSchedulerKey(pri);
-    Mockito.when(pri.getPriority()).thenReturn(1);
+    when(pri.getPriority()).thenReturn(1);
     double nodeLocalityThreshold = .5;
     double rackLocalityThreshold = .6;
 
@@ -134,9 +135,9 @@ public class TestFSAppAttempt extends FairSchedulerTestBase {
   public void testDelaySchedulingForContinuousScheduling()
           throws InterruptedException {
     FSLeafQueue queue = scheduler.getQueueManager().getLeafQueue("queue", true);
-    Priority pri = Mockito.mock(Priority.class);
+    Priority pri = mock(Priority.class);
     SchedulerRequestKey prio = TestUtils.toSchedulerKey(pri);
-    Mockito.when(pri.getPriority()).thenReturn(1);
+    when(pri.getPriority()).thenReturn(1);
 
     ControlledClock clock = new ControlledClock();
     scheduler.setClock(clock);
@@ -192,10 +193,10 @@ public class TestFSAppAttempt extends FairSchedulerTestBase {
    * no tin use), the least restrictive locality level is returned.
    */
   public void testLocalityLevelWithoutDelays() {
-    FSLeafQueue queue = Mockito.mock(FSLeafQueue.class);
-    Priority pri = Mockito.mock(Priority.class);
+    FSLeafQueue queue = mock(FSLeafQueue.class);
+    Priority pri = mock(Priority.class);
     SchedulerRequestKey prio = TestUtils.toSchedulerKey(pri);
-    Mockito.when(pri.getPriority()).thenReturn(1);
+    when(pri.getPriority()).thenReturn(1);
 
     RMContext rmContext = resourceManager.getRMContext();
     ApplicationAttemptId applicationAttemptId = createAppAttemptId(1, 1);
@@ -208,10 +209,10 @@ public class TestFSAppAttempt extends FairSchedulerTestBase {
 
   @Test
   public void testHeadroom() {
-    final FairScheduler mockScheduler = Mockito.mock(FairScheduler.class);
-    Mockito.when(mockScheduler.getClock()).thenReturn(scheduler.getClock());
+    final FairScheduler mockScheduler = mock(FairScheduler.class);
+    when(mockScheduler.getClock()).thenReturn(scheduler.getClock());
 
-    final FSLeafQueue mockQueue = Mockito.mock(FSLeafQueue.class);
+    final FSLeafQueue mockQueue = mock(FSLeafQueue.class);
 
     final Resource queueMaxResources = Resource.newInstance(5 * 1024, 3);
     final Resource queueFairShare = Resources.createResource(4096, 2);
@@ -227,19 +228,19 @@ public class TestFSAppAttempt extends FairSchedulerTestBase {
     final Resource clusterAvailable =
         Resources.subtract(clusterResource, clusterUsage);
 
-    final QueueMetrics fakeRootQueueMetrics = Mockito.mock(QueueMetrics.class);
+    final QueueMetrics fakeRootQueueMetrics = mock(QueueMetrics.class);
 
-    Mockito.when(mockQueue.getMaxShare()).thenReturn(queueMaxResources);
-    Mockito.when(mockQueue.getFairShare()).thenReturn(queueFairShare);
-    Mockito.when(mockQueue.getResourceUsage()).thenReturn(queueUsage);
-    Mockito.when(mockScheduler.getClusterResource()).thenReturn
+    when(mockQueue.getMaxShare()).thenReturn(queueMaxResources);
+    when(mockQueue.getFairShare()).thenReturn(queueFairShare);
+    when(mockQueue.getResourceUsage()).thenReturn(queueUsage);
+    when(mockScheduler.getClusterResource()).thenReturn
         (clusterResource);
-    Mockito.when(fakeRootQueueMetrics.getAllocatedResources()).thenReturn
+    when(fakeRootQueueMetrics.getAllocatedResources()).thenReturn
         (clusterUsage);
-    Mockito.when(mockScheduler.getRootQueueMetrics()).thenReturn
+    when(mockScheduler.getRootQueueMetrics()).thenReturn
         (fakeRootQueueMetrics);
-    Mockito.when(mockScheduler.getConf()).thenReturn
-        (Mockito.mock(FairSchedulerConfiguration.class));
+    when(mockScheduler.getConf()).thenReturn(
+        mock(FairSchedulerConfiguration.class));
 
     ApplicationAttemptId applicationAttemptId = createAppAttemptId(1, 1);
     RMContext rmContext = resourceManager.getRMContext();
@@ -249,7 +250,7 @@ public class TestFSAppAttempt extends FairSchedulerTestBase {
 
     // Min of Memory and CPU across cluster and queue is used in
     // DominantResourceFairnessPolicy
-    Mockito.when(mockQueue.getPolicy()).thenReturn(SchedulingPolicy
+    when(mockQueue.getPolicy()).thenReturn(SchedulingPolicy
         .getInstance(DominantResourceFairnessPolicy.class));
     verifyHeadroom(schedulerApp,
         min(queueStarvation.getMemorySize(),
@@ -261,7 +262,7 @@ public class TestFSAppAttempt extends FairSchedulerTestBase {
     );
 
     // Fair and Fifo ignore CPU of queue, so use cluster available CPU
-    Mockito.when(mockQueue.getPolicy()).thenReturn(SchedulingPolicy
+    when(mockQueue.getPolicy()).thenReturn(SchedulingPolicy
         .getInstance(FairSharePolicy.class));
     verifyHeadroom(schedulerApp,
         min(queueStarvation.getMemorySize(),
@@ -272,7 +273,7 @@ public class TestFSAppAttempt extends FairSchedulerTestBase {
             queueMaxResourcesAvailable.getVirtualCores())
     );
 
-    Mockito.when(mockQueue.getPolicy()).thenReturn(SchedulingPolicy
+    when(mockQueue.getPolicy()).thenReturn(SchedulingPolicy
         .getInstance(FifoPolicy.class));
     verifyHeadroom(schedulerApp,
         min(queueStarvation.getMemorySize(),
@@ -297,8 +298,8 @@ public class TestFSAppAttempt extends FairSchedulerTestBase {
             "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();
@@ -359,28 +360,28 @@ public class TestFSAppAttempt extends FairSchedulerTestBase {
    */
   @Test
   public void testNoNextPendingAsk() {
-    FSLeafQueue queue = Mockito.mock(FSLeafQueue.class);
+    FSLeafQueue queue = mock(FSLeafQueue.class);
     ApplicationAttemptId applicationAttemptId = createAppAttemptId(1, 1);
-    RMContext rmContext = Mockito.mock(RMContext.class);
+    RMContext rmContext = mock(RMContext.class);
     ConcurrentMap<ApplicationId, RMApp> rmApps = new ConcurrentHashMap<>();
-    RMApp rmApp = Mockito.mock(RMApp.class);
+    RMApp rmApp = mock(RMApp.class);
     rmApps.put(applicationAttemptId.getApplicationId(), rmApp);
     ApplicationSubmissionContext appContext =
-        Mockito.mock(ApplicationSubmissionContext.class);
-    Mockito.when(appContext.getUnmanagedAM()).thenReturn(false);
-    Mockito.when(appContext.getLogAggregationContext())
-        .thenReturn(Mockito.mock(LogAggregationContext.class));
-    Mockito.when(rmApp.getApplicationSchedulingEnvs())
+        mock(ApplicationSubmissionContext.class);
+    when(appContext.getUnmanagedAM()).thenReturn(false);
+    when(appContext.getLogAggregationContext())
+        .thenReturn(mock(LogAggregationContext.class));
+    when(rmApp.getApplicationSchedulingEnvs())
         .thenReturn(new HashMap<>());
-    Mockito.when(rmApp.getApplicationSubmissionContext())
+    when(rmApp.getApplicationSubmissionContext())
       .thenReturn(appContext);
-    Mockito.when(rmContext.getRMApps()).thenReturn(rmApps);
-    Mockito.when(rmContext.getYarnConfiguration()).thenReturn(conf);
+    when(rmContext.getRMApps()).thenReturn(rmApps);
+    when(rmContext.getYarnConfiguration()).thenReturn(conf);
     FSAppAttempt schedulerApp =
         new FSAppAttempt(scheduler, applicationAttemptId, "user1", queue,
             null, rmContext);
     schedulerApp.setAmRunning(false);
-    FSSchedulerNode schedulerNode = Mockito.mock(FSSchedulerNode.class);
+    FSSchedulerNode schedulerNode = mock(FSSchedulerNode.class);
 
     Resource resource = schedulerApp.assignContainer(schedulerNode);
 

+ 28 - 28
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 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair
     .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 class TestFSAppStarvation extends FairSchedulerTestBase {
     conf.setLong(FairSchedulerConfiguration.UPDATE_INTERVAL_MS, Long.MAX_VALUE);
   }
 
-  @After
+  @AfterEach
   public void teardown() {
     ALLOC_FILE.delete();
     conf = null;
@@ -89,8 +89,8 @@ public class TestFSAppStarvation extends FairSchedulerTestBase {
 
     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 class TestFSAppStarvation extends FairSchedulerTestBase {
       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 class TestFSAppStarvation extends FairSchedulerTestBase {
       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 class TestFSAppStarvation extends FairSchedulerTestBase {
 
     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 @@ public class TestFSAppStarvation extends FairSchedulerTestBase {
             .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();

+ 28 - 27
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,9 +19,12 @@
 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.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 import java.io.File;
 import java.io.IOException;
@@ -50,13 +53,11 @@ import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 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.mockito.Mockito;
+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.util.Map;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.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();
@@ -97,19 +98,20 @@ public class TestFSLeafQueue extends FairSchedulerTestBase {
         SchedulingPolicy.DEFAULT_POLICY.getName());
 
     FSAppAttempt app = mock(FSAppAttempt.class);
-    Mockito.when(app.getDemand()).thenReturn(maxResource);
-    Mockito.when(app.getResourceUsage()).thenReturn(Resources.none());
+    when(app.getDemand()).thenReturn(maxResource);
+    when(app.getResourceUsage()).thenReturn(Resources.none());
 
     schedulable.addApp(app, true);
     schedulable.addApp(app, true);
 
     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,14 @@ public class TestFSLeafQueue extends FairSchedulerTestBase {
       // 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 +269,7 @@ public class TestFSLeafQueue extends FairSchedulerTestBase {
         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 +319,7 @@ public class TestFSLeafQueue extends FairSchedulerTestBase {
         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 +339,7 @@ public class TestFSLeafQueue extends FairSchedulerTestBase {
 
     //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<String, Long> verifyQueueMetricsForCustomResources(
@@ -348,10 +348,11 @@ public class TestFSLeafQueue extends FairSchedulerTestBase {
         schedulable.getMetrics().getCustomResources().getMaxAMShare();
     Map<String, Long> 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;
   }
 }

+ 4 - 4
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.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager;
 import org.apache.hadoop.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);

+ 83 - 131
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.conf.YarnConfiguration;
 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,22 @@ public class TestFSQueueMetrics {
         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 +121,29 @@ public class TestFSQueueMetrics {
         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 +154,22 @@ public class TestFSQueueMetrics {
         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 +180,22 @@ public class TestFSQueueMetrics {
         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 +206,25 @@ public class TestFSQueueMetrics {
         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 +235,32 @@ public class TestFSQueueMetrics {
         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"));
   }
 }

+ 60 - 77
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.api.records.*;
 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,14 @@ public class TestFSSchedulerNode {
   }
 
   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());
+    assertEquals(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 +152,14 @@ public class TestFSSchedulerNode {
     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 +177,15 @@ public class TestFSSchedulerNode {
     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,27 +202,25 @@ public class TestFSSchedulerNode {
 
     // 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,
         Resource.newInstance(1024, 1));
     schedulerNode.addContainersForPreemption(
         Collections.singletonList(containers.get(0)), starvingApp);
-    assertEquals(
-        "No resource amount should be reserved for preemptees",
-        containers.get(0).getAllocatedResource(),
-        schedulerNode.getTotalReserved());
+    assertEquals(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 +239,9 @@ public class TestFSSchedulerNode {
 
     // 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,
@@ -257,17 +250,15 @@ public class TestFSSchedulerNode {
         Collections.singletonList(containers.get(0)), starvingApp);
     schedulerNode.addContainersForPreemption(
         Collections.singletonList(containers.get(0)), starvingApp);
-    assertEquals(
-        "No resource amount should be reserved for preemptees",
-        containers.get(0).getAllocatedResource(),
-        schedulerNode.getTotalReserved());
+    assertEquals(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 +277,9 @@ public class TestFSSchedulerNode {
 
     // 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 +301,8 @@ public class TestFSSchedulerNode {
     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 +321,9 @@ public class TestFSSchedulerNode {
 
     // 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 +347,8 @@ public class TestFSSchedulerNode {
     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 +367,9 @@ public class TestFSSchedulerNode {
 
     // 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 +384,8 @@ public class TestFSSchedulerNode {
     // 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 +404,9 @@ public class TestFSSchedulerNode {
 
     // 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 +420,9 @@ public class TestFSSchedulerNode {
 
     // 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();

Різницю між файлами не показано, бо вона завелика
+ 259 - 232
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


+ 211 - 152
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,18 +34,17 @@ import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 import org.apache.log4j.spi.LoggingEvent;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.function.Executable;
 
 import java.util.Collections;
 import java.util.List;
 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.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 /**
  * Tests fair scheduler configuration.
@@ -77,49 +76,57 @@ public class TestFairSchedulerConfiguration {
     }
   }
 
-  @Rule
-  public ExpectedException exception = ExpectedException.none();
-
-  private void expectMissingResource(String resource) {
-    exception.expect(AllocationConfigurationException.class);
-    exception.expectMessage("Missing resource: " + resource);
+  private void expectMissingResource(String resource, Executable call) {
+    AllocationConfigurationException exception =
+        assertThrows(AllocationConfigurationException.class, call);
+    assertTrue(exception.getMessage().
+        contains("Missing resource: " + resource));
   }
 
-  private void expectUnparsableResource(String resource) {
-    exception.expect(AllocationConfigurationException.class);
-    exception.expectMessage("Cannot parse resource values from input: "
-        + resource);
+  private void expectUnparsableResource(String resource, Executable call) {
+    AllocationConfigurationException exception =
+        assertThrows(AllocationConfigurationException.class, call);
+    assertTrue(exception.getMessage().
+        contains("Cannot parse resource values from input: " + resource));
   }
 
-  private void expectInvalidResource(String resource) {
-    exception.expect(AllocationConfigurationException.class);
-    exception.expectMessage("Invalid value of " + resource + ": ");
+  private void expectInvalidResource(String resource, Executable call) {
+    AllocationConfigurationException exception =
+        assertThrows(AllocationConfigurationException.class, call);
+    assertTrue(exception.getMessage().
+        contains("Invalid value of " + resource + ": "));
   }
 
-  private void expectInvalidResourcePercentage(String resource) {
-    exception.expect(AllocationConfigurationException.class);
-    exception.expectMessage("Invalid percentage of " + resource + ": ");
+  private void expectInvalidResourcePercentage(String resource, Executable call) {
+    AllocationConfigurationException exception =
+        assertThrows(AllocationConfigurationException.class, call);
+    assertTrue(exception.getMessage().
+        contains("Invalid percentage of " + resource + ": "));
   }
 
-  private void expectInvalidResourcePercentageNewStyle(String value) {
-    exception.expect(AllocationConfigurationException.class);
-    exception.expectMessage("\"" + value + "\" is either " +
-        "not a non-negative number");
+  private void expectInvalidResourcePercentageNewStyle(String value, Executable call) {
+    AllocationConfigurationException exception =
+        assertThrows(AllocationConfigurationException.class, call);
+    assertTrue(exception.getMessage().
+        contains("\"" + value + "\" is either " + "not a non-negative number"));
   }
 
-  private void expectNegativePercentageOldStyle() {
-    exception.expect(AllocationConfigurationException.class);
-    exception.expectMessage("percentage should not be negative");
+  private void expectNegativePercentageOldStyle(Executable call) {
+    AllocationConfigurationException exception =
+        assertThrows(AllocationConfigurationException.class, call);
+    assertTrue(exception.getMessage().contains("percentage should not be negative"));
   }
 
-  private void expectNegativePercentageNewStyle() {
-    exception.expect(AllocationConfigurationException.class);
-    exception.expectMessage("is either not a non-negative number");
+  private void expectNegativePercentageNewStyle(Executable call) {
+    AllocationConfigurationException exception =
+        assertThrows(AllocationConfigurationException.class, call);
+    assertTrue(exception.getMessage().contains("is either not a non-negative number"));
   }
 
-  private void expectNegativeValueOfResource(String resource) {
-    exception.expect(AllocationConfigurationException.class);
-    exception.expectMessage("Invalid value of " + resource);
+  private void expectNegativeValueOfResource(String resource, Executable call) {
+    AllocationConfigurationException exception =
+        assertThrows(AllocationConfigurationException.class, call);
+    assertTrue(exception.getMessage().contains("Invalid value of " + resource));
   }
 
   @Test
@@ -270,181 +277,210 @@ public class TestFairSchedulerConfiguration {
   @Test
   public void testNoUnits() throws Exception {
     String value = "1024";
-    expectUnparsableResource(value);
-    parseResourceConfigValue(value);
+    expectUnparsableResource(value, () -> {
+      parseResourceConfigValue(value);
+    });
   }
 
   @Test
   public void testOnlyMemory() throws Exception {
     String value = "1024mb";
-    expectUnparsableResource(value);
-    parseResourceConfigValue(value);
+    expectUnparsableResource(value, () -> {
+      parseResourceConfigValue(value);
+    });
   }
 
   @Test
   public void testOnlyCPU() throws Exception {
     String value = "1024vcores";
-    expectUnparsableResource(value);
-    parseResourceConfigValue(value);
+    expectUnparsableResource(value, () -> {
+      parseResourceConfigValue(value);
+    });
   }
 
   @Test
   public void testGibberish() throws Exception {
     String value = "1o24vc0res";
-    expectUnparsableResource(value);
-    parseResourceConfigValue(value);
+    expectUnparsableResource(value, () -> {
+      parseResourceConfigValue(value);
+    });
   }
 
   @Test
   public void testNoUnitsPercentage() throws Exception {
-    expectMissingResource("cpu");
-    parseResourceConfigValue("95%, 50% memory");
+    expectMissingResource("cpu", () -> {
+      parseResourceConfigValue("95%, 50% memory");
+    });
   }
 
   @Test
   public void testInvalidNumPercentage() throws Exception {
-    expectInvalidResourcePercentage("cpu");
-    parseResourceConfigValue("95A% cpu, 50% memory");
+    expectInvalidResourcePercentage("cpu", () -> {
+      parseResourceConfigValue("95A% cpu, 50% memory");
+    });
   }
 
   @Test
   public void testCpuPercentageMemoryAbsolute() throws Exception {
-    expectMissingResource("memory");
-    parseResourceConfigValue("50% cpu, 1024 mb");
+    expectMissingResource("memory", () -> {
+      parseResourceConfigValue("50% cpu, 1024 mb");
+    });
   }
 
   @Test
   public void testMemoryPercentageCpuAbsolute() throws Exception {
-    expectMissingResource("cpu");
-    parseResourceConfigValue("50% memory, 2 vcores");
+    expectMissingResource("cpu", () -> {
+      parseResourceConfigValue("50% memory, 2 vcores");
+    });
   }
 
   @Test
   public void testDuplicateVcoresDefinitionAbsolute() throws Exception {
-    expectInvalidResource("vcores");
-    parseResourceConfigValue("1024 mb, 2 4 vcores");
+    expectInvalidResource("vcores", () -> {
+      parseResourceConfigValue("1024 mb, 2 4 vcores");
+    });
   }
 
   @Test
   public void testDuplicateMemoryDefinitionAbsolute() throws Exception {
-    expectInvalidResource("memory");
-    parseResourceConfigValue("2048 1024 mb, 2 vcores");
+    expectInvalidResource("memory", () -> {
+      parseResourceConfigValue("2048 1024 mb, 2 vcores");
+    });
   }
 
   @Test
   public void testDuplicateVcoresDefinitionPercentage() throws Exception {
-    expectInvalidResourcePercentage("cpu");
-    parseResourceConfigValue("50% memory, 50% 100%cpu");
+    expectInvalidResourcePercentage("cpu", () -> {
+      parseResourceConfigValue("50% memory, 50% 100%cpu");
+    });
   }
 
   @Test
   public void testDuplicateMemoryDefinitionPercentage() throws Exception {
-    expectInvalidResourcePercentage("memory");
-    parseResourceConfigValue("50% 80% memory, 100%cpu");
+    expectInvalidResourcePercentage("memory", () -> {
+      parseResourceConfigValue("50% 80% memory, 100%cpu");
+    });
   }
 
   @Test
   public void testParseNewStyleDuplicateMemoryDefinitionPercentage()
       throws Exception {
-    expectInvalidResourcePercentageNewStyle("40% 80%");
-    parseResourceConfigValue("vcores = 75%, memory-mb = 40% 80%");
+    expectInvalidResourcePercentageNewStyle("40% 80%", () -> {
+      parseResourceConfigValue("vcores = 75%, memory-mb = 40% 80%");
+    });
   }
 
   @Test
   public void testParseNewStyleDuplicateVcoresDefinitionPercentage()
-      throws Exception {
-    expectInvalidResourcePercentageNewStyle("75% 65%");
-    parseResourceConfigValue("vcores = 75% 65%, memory-mb = 40%");
+          throws Exception {
+    expectInvalidResourcePercentageNewStyle("75% 65%", () -> {
+      parseResourceConfigValue("vcores = 75% 65%, memory-mb = 40%");
+    });
   }
 
   @Test
   public void testMemoryPercentageNegativeValue() throws Exception {
-    expectNegativePercentageOldStyle();
-    parseResourceConfigValue("-10% memory, 50% cpu");
+    expectNegativePercentageOldStyle(() -> {
+      parseResourceConfigValue("-10% memory, 50% cpu");
+    });
   }
 
   @Test
   public void testCpuPercentageNegativeValue() throws Exception {
-    expectNegativePercentageOldStyle();
-    parseResourceConfigValue("10% memory, -10% cpu");
+    expectNegativePercentageOldStyle(() -> {
+      parseResourceConfigValue("10% memory, -10% cpu");
+    });
   }
 
   @Test
   public void testMemoryAndCpuPercentageNegativeValue() throws Exception {
-    expectNegativePercentageOldStyle();
-    parseResourceConfigValue("-20% memory, -10% cpu");
+    expectNegativePercentageOldStyle(() -> {
+      parseResourceConfigValue("-20% memory, -10% cpu");
+    });
   }
 
   @Test
   public void testCpuPercentageMemoryAbsoluteCpuNegative() throws Exception {
-    expectMissingResource("memory");
-    parseResourceConfigValue("-50% cpu, 1024 mb");
+    expectMissingResource("memory", () -> {
+      parseResourceConfigValue("-50% cpu, 1024 mb");
+    });
   }
 
   @Test
   public void testCpuPercentageMemoryAbsoluteMemoryNegative() throws Exception {
-    expectMissingResource("memory");
-    parseResourceConfigValue("50% cpu, -1024 mb");
+    expectMissingResource("memory", () -> {
+      parseResourceConfigValue("50% cpu, -1024 mb");
+    });
   }
 
   @Test
   public void testMemoryPercentageCpuAbsoluteCpuNegative() throws Exception {
-    expectMissingResource("cpu");
-    parseResourceConfigValue("50% memory, -2 vcores");
+    expectMissingResource("cpu", () -> {
+      parseResourceConfigValue("50% memory, -2 vcores");
+    });
   }
 
   @Test
   public void testMemoryPercentageCpuAbsoluteMemoryNegative() throws Exception {
-    expectNegativePercentageOldStyle();
-    parseResourceConfigValue("-50% memory, 2 vcores");
+    expectNegativePercentageOldStyle(() -> {
+      parseResourceConfigValue("-50% memory, 2 vcores");
+    });
   }
 
   @Test
   public void testAbsoluteVcoresNegative() throws Exception {
-    expectNegativeValueOfResource("vcores");
-    parseResourceConfigValue("-2 vcores,5120 mb");
+    expectNegativeValueOfResource("vcores", () -> {
+      parseResourceConfigValue("-2 vcores,5120 mb");
+    });
   }
 
   @Test
   public void testAbsoluteMemoryNegative() throws Exception {
-    expectNegativeValueOfResource("memory");
-    parseResourceConfigValue("2 vcores,-5120 mb");
+    expectNegativeValueOfResource("memory", () -> {
+      parseResourceConfigValue("2 vcores,-5120 mb");
+    });
   }
 
   @Test
   public void testAbsoluteVcoresNegativeWithSpaces() throws Exception {
-    expectNegativeValueOfResource("vcores");
-    parseResourceConfigValue("-2 vcores, 5120 mb");
+    expectNegativeValueOfResource("vcores", () -> {
+      parseResourceConfigValue("-2 vcores, 5120 mb");
+    });
   }
 
   @Test
   public void testAbsoluteMemoryNegativeWithSpaces() throws Exception {
-    expectNegativeValueOfResource("memory");
-    parseResourceConfigValue("2 vcores, -5120 mb");
+    expectNegativeValueOfResource("memory", () -> {
+      parseResourceConfigValue("2 vcores, -5120 mb");
+    });
   }
 
   @Test
   public void testAbsoluteVcoresNegativeWithMoreSpaces() throws Exception {
-    expectNegativeValueOfResource("vcores");
-    parseResourceConfigValue("5120mb   mb, -2    vcores");
+    expectNegativeValueOfResource("vcores", () -> {
+      parseResourceConfigValue("5120mb   mb, -2    vcores");
+    });
   }
 
   @Test
   public void testAbsoluteMemoryNegativeWithMoreSpaces() throws Exception {
-    expectNegativeValueOfResource("memory");
-    parseResourceConfigValue("-5120mb   mb, 2    vcores");
+    expectNegativeValueOfResource("memory", () -> {
+      parseResourceConfigValue("-5120mb   mb, 2    vcores");
+    });
   }
 
   @Test
   public void testAbsoluteVcoresNegativeFractional() throws Exception {
-    expectNegativeValueOfResource("vcores");
-    parseResourceConfigValue("  5120.3 mb, -2.35 vcores  ");
+    expectNegativeValueOfResource("vcores", () -> {
+      parseResourceConfigValue("  5120.3 mb, -2.35 vcores  ");
+    });
   }
 
   @Test
   public void testAbsoluteMemoryNegativeFractional() throws Exception {
-    expectNegativeValueOfResource("memory");
-    parseResourceConfigValue("  -5120.3 mb, 2.35 vcores  ");
+    expectNegativeValueOfResource("memory", () -> {
+      parseResourceConfigValue("  -5120.3 mb, 2.35 vcores  ");
+    });
   }
 
   @Test
@@ -455,136 +491,155 @@ public class TestFairSchedulerConfiguration {
   @Test
   public void testOldStyleResourcesSeparatedBySpacesInvalid() throws Exception {
     String value = "2 vcores 5120 mb 555 mb";
-    expectUnparsableResource(value);
-    parseResourceConfigValue(value);
+    expectUnparsableResource(value, () -> {
+      parseResourceConfigValue(value);
+    });
   }
 
   @Test
   public void testOldStyleResourcesSeparatedBySpacesInvalidUppercaseUnits()
-      throws Exception {
+          throws Exception {
     String value = "2 vcores 5120 MB 555 GB";
-    expectUnparsableResource(value);
-    parseResourceConfigValue(value);
+    expectUnparsableResource(value, () -> {
+      parseResourceConfigValue(value);
+    });
   }
 
   @Test
   public void testParseNewStyleResourceMemoryNegative() throws Exception {
-    expectNegativeValueOfResource("memory");
-    parseResourceConfigValue("memory-mb=-5120,vcores=2");
+    expectNegativeValueOfResource("memory", () -> {
+      parseResourceConfigValue("memory-mb=-5120,vcores=2");
+    });
   }
 
   @Test
   public void testParseNewStyleResourceVcoresNegative() throws Exception {
-    expectNegativeValueOfResource("vcores");
-    parseResourceConfigValue("memory-mb=5120,vcores=-2");
+    expectNegativeValueOfResource("vcores", () -> {
+      parseResourceConfigValue("memory-mb=5120,vcores=-2");
+    });
   }
 
   @Test
   public void testParseNewStyleResourceMemoryNegativeWithSpaces()
       throws Exception {
-    expectNegativeValueOfResource("memory");
-    parseResourceConfigValue("memory-mb=-5120, vcores=2");
+    expectNegativeValueOfResource("memory", () -> {
+      parseResourceConfigValue("memory-mb=-5120, vcores=2");
+    });
   }
 
   @Test
   public void testParseNewStyleResourceVcoresNegativeWithSpaces()
       throws Exception {
-    expectNegativeValueOfResource("vcores");
-    parseResourceConfigValue("memory-mb=5120, vcores=-2");
+    expectNegativeValueOfResource("vcores", () -> {
+      parseResourceConfigValue("memory-mb=5120, vcores=-2");
+    });
   }
 
   @Test
   public void testParseNewStyleResourceMemoryNegativeWithMoreSpaces()
       throws Exception {
-    expectNegativeValueOfResource("memory");
-    parseResourceConfigValue(" vcores = 2 ,  memory-mb = -5120 ");
+    expectNegativeValueOfResource("memory", () -> {
+      parseResourceConfigValue(" vcores = 2 ,  memory-mb = -5120 ");
+    });
   }
 
   @Test
   public void testParseNewStyleResourceVcoresNegativeWithMoreSpaces()
       throws Exception {
-    expectNegativeValueOfResource("vcores");
-    parseResourceConfigValue(" vcores = -2 ,  memory-mb = 5120 ");
+    expectNegativeValueOfResource("vcores", () -> {
+      parseResourceConfigValue(" vcores = -2 ,  memory-mb = 5120 ");
+    });
   }
 
   @Test
   public void testParseNewStyleResourceWithCustomResourceMemoryNegative()
       throws Exception {
-    expectNegativeValueOfResource("memory");
-    parseResourceConfigValue("vcores=2,memory-mb=-5120,test1=4");
+    expectNegativeValueOfResource("memory", () -> {
+      parseResourceConfigValue("vcores=2,memory-mb=-5120,test1=4");
+    });
   }
 
   @Test
   public void testParseNewStyleResourceWithCustomResourceVcoresNegative()
       throws Exception {
-    expectNegativeValueOfResource("vcores");
-    parseResourceConfigValue("vcores=-2,memory-mb=-5120,test1=4");
+    expectNegativeValueOfResource("vcores", () -> {
+      parseResourceConfigValue("vcores=-2,memory-mb=-5120,test1=4");
+    });
   }
 
   @Test
   public void testParseNewStyleResourceWithCustomResourceNegative()
       throws Exception {
-    expectNegativeValueOfResource("test1");
-    parseResourceConfigValue("vcores=2,memory-mb=5120,test1=-4");
+    expectNegativeValueOfResource("test1", () -> {
+      parseResourceConfigValue("vcores=2,memory-mb=5120,test1=-4");
+    });
   }
 
   @Test
   public void testParseNewStyleResourceWithCustomResourceNegativeWithSpaces()
       throws Exception {
-    expectNegativeValueOfResource("test1");
-    parseResourceConfigValue(" vcores = 2 , memory-mb = 5120 , test1 = -4 ");
+    expectNegativeValueOfResource("test1", () -> {
+      parseResourceConfigValue(" vcores = 2 , memory-mb = 5120 , test1 = -4 ");
+    });
   }
 
   @Test
   public void testParseNewStyleResourceWithPercentagesVcoresNegative() throws
       Exception {
-    expectNegativePercentageNewStyle();
-    parseResourceConfigValue("vcores=-75%,memory-mb=40%");
+    expectNegativePercentageNewStyle(() -> {
+      parseResourceConfigValue("vcores=-75%,memory-mb=40%");
+    });
   }
 
   @Test
   public void testParseNewStyleResourceWithPercentagesMemoryNegative() throws
-      Exception {
-    expectNegativePercentageNewStyle();
-    parseResourceConfigValue("vcores=75%,memory-mb=-40%");
+          Exception {
+    expectNegativePercentageNewStyle(() -> {
+      parseResourceConfigValue("vcores=75%,memory-mb=-40%");
+    });
   }
 
   @Test
   public void testParseNewStyleResourceWithPercentagesVcoresNegativeWithSpaces()
       throws Exception {
-    expectNegativePercentageNewStyle();
-    parseResourceConfigValue("vcores=-75%, memory-mb=40%");
+    expectNegativePercentageNewStyle(() -> {
+      parseResourceConfigValue("vcores=-75%, memory-mb=40%");
+    });
   }
 
   @Test
   public void testParseNewStyleResourceWithPercentagesMemoryNegativeWithSpaces()
       throws Exception {
-    expectNegativePercentageNewStyle();
-    parseResourceConfigValue("vcores=75%, memory-mb=-40%");
+    expectNegativePercentageNewStyle(() -> {
+      parseResourceConfigValue("vcores=75%, memory-mb=-40%");
+    });
   }
 
   @Test
   public void
   testParseNewStyleResourceWithPercentagesVcoresNegativeWithMoreSpaces()
       throws Exception {
-    expectNegativePercentageNewStyle();
-    parseResourceConfigValue("vcores = -75%, memory-mb = 40%");
+    expectNegativePercentageNewStyle(() -> {
+      parseResourceConfigValue("vcores = -75%, memory-mb = 40%");
+    });
   }
 
   @Test
   public void
   testParseNewStyleResourceWithPercentagesMemoryNegativeWithMoreSpaces()
       throws Exception {
-    expectNegativePercentageNewStyle();
-    parseResourceConfigValue("vcores = 75%, memory-mb = -40%");
+    expectNegativePercentageNewStyle(() -> {
+      parseResourceConfigValue("vcores = 75%, memory-mb = -40%");
+    });
   }
 
   @Test
   public void
   testParseNewStyleResourceWithPercentagesCustomResourceNegativeWithSpaces()
       throws Exception {
-    expectNegativeValueOfResource("test1");
-    parseResourceConfigValue(" vcores = 2 , memory-mb = 5120 , test1 = -4 ");
+    expectNegativeValueOfResource("test1", () -> {
+      parseResourceConfigValue(" vcores = 2 , memory-mb = 5120 , test1 = -4 ");
+    });
   }
 
   @Test
@@ -638,13 +693,15 @@ public class TestFairSchedulerConfiguration {
           .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 +819,15 @@ public class TestFairSchedulerConfiguration {
           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",
+      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())));
+            "yarn.resource-types.memory-mb.increment-allocation=13 is " +
+            "overriding the yarn.scheduler.increment-allocation-mb=7 " +
+            "property").equals(e.getMessage())),
+            "Warning message is not logged when specifying memory " +
+            "increment via multiple properties");
     } finally {
       logger.removeAppender(testAppender);
     }
@@ -788,14 +846,15 @@ public class TestFairSchedulerConfiguration {
           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",
+      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())));
+            "yarn.resource-types.vcores.increment-allocation=13 is " +
+            "overriding the yarn.scheduler.increment-allocation-vcores=7 " +
+            "property").equals(e.getMessage())),
+            "Warning message is not logged when specifying CPU vCores " +
+            "increment via multiple properties");
     } finally {
       logger.removeAppender(testAppender);
     }

+ 6 - 6
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
 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();

+ 76 - 57
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

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
+import org.apache.hadoop.test.TestName;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
@@ -30,16 +31,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.allocationfi
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.allocationfile.AllocationFileWriter;
 import org.apache.hadoop.yarn.util.ControlledClock;
 import org.apache.hadoop.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.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+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.extension.RegisterExtension;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 
 import java.io.File;
 import java.io.IOException;
@@ -51,7 +49,6 @@ import java.util.List;
 /**
  * Tests to verify fairshare and minshare preemption, using parameterization.
  */
-@RunWith(Parameterized.class)
 public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
   private static final File ALLOC_FILE = new File(TEST_DIR, "test-queues");
   private static final int GB = 1024;
@@ -64,8 +61,8 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
   // Node Capacity = NODE_CAPACITY_MULTIPLE * (1 GB or 1 vcore)
   private static final int NODE_CAPACITY_MULTIPLE = 4;
 
-  private final boolean fairsharePreemption;
-  private final boolean drf;
+  private boolean fairsharePreemption;
+  private boolean drf;
 
   // App that takes up the entire cluster
   private FSAppAttempt greedyApp;
@@ -73,10 +70,9 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
   // Starving app that is expected to instigate preemption
   private FSAppAttempt starvingApp;
 
-  @Rule
-  public TestName testName = new TestName();
+  @RegisterExtension
+  private TestName testName = new TestName();
 
-  @Parameterized.Parameters(name = "{0}")
   public static Collection<Object[]> getParameters() {
     return Arrays.asList(new Object[][] {
         {"MinSharePreemption", 0},
@@ -86,14 +82,14 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
     });
   }
 
-  public TestFairSchedulerPreemption(String name, int mode)
+  private void initTestFairSchedulerPreemption(String name, int mode)
       throws IOException {
     fairsharePreemption = (mode > 1); // 2 and 3
     drf = (mode % 2 == 1); // 1 and 3
     writeAllocFile();
+    setup();
   }
 
-  @Before
   public void setup() throws IOException {
     createConfiguration();
     conf.set(FairSchedulerConfiguration.ALLOCATION_FILE,
@@ -109,7 +105,7 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
     setupCluster();
   }
 
-  @After
+  @AfterEach
   public void teardown() {
     ALLOC_FILE.delete();
     conf = null;
@@ -190,8 +186,8 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
     }
     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 +294,30 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
     }
 
     // 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) {
@@ -344,8 +340,10 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
     assertEquals(8, greedyApp.getLiveContainers().size());
   }
 
-  @Test
-  public void testPreemptionWithinSameLeafQueue() throws Exception {
+  @ParameterizedTest(name = "{0}")
+  @MethodSource("getParameters")
+  public void testPreemptionWithinSameLeafQueue(String name, int mode) throws Exception {
+    initTestFairSchedulerPreemption(name, mode);
     String queue = "root.preemptable.child-1";
     submitApps(queue, queue);
     if (fairsharePreemption) {
@@ -355,20 +353,26 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
     }
   }
 
-  @Test
-  public void testPreemptionBetweenTwoSiblingLeafQueues() throws Exception {
+  @ParameterizedTest(name = "{0}")
+  @MethodSource("getParameters")
+  public void testPreemptionBetweenTwoSiblingLeafQueues(String name, int mode) throws Exception {
+    initTestFairSchedulerPreemption(name, mode);
     submitApps("root.preemptable.child-1", "root.preemptable.child-2");
     verifyPreemption(2, 4);
   }
 
-  @Test
-  public void testPreemptionBetweenNonSiblingQueues() throws Exception {
+  @ParameterizedTest(name = "{0}")
+  @MethodSource("getParameters")
+  public void testPreemptionBetweenNonSiblingQueues(String name, int mode) throws Exception {
+    initTestFairSchedulerPreemption(name, mode);
     submitApps("root.preemptable.child-1", "root.nonpreemptable.child-1");
     verifyPreemption(2, 4);
   }
 
-  @Test
-  public void testNoPreemptionFromDisallowedQueue() throws Exception {
+  @ParameterizedTest(name = "{0}")
+  @MethodSource("getParameters")
+  public void testNoPreemptionFromDisallowedQueue(String name, int mode) throws Exception {
+    initTestFairSchedulerPreemption(name, mode);
     submitApps("root.nonpreemptable.child-1", "root.preemptable.child-1");
     verifyNoPreemption();
   }
@@ -392,8 +396,10 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
     }
   }
 
-  @Test
-  public void testPreemptionSelectNonAMContainer() throws Exception {
+  @ParameterizedTest(name = "{0}")
+  @MethodSource("getParameters")
+  public void testPreemptionSelectNonAMContainer(String name, int mode) throws Exception {
+    initTestFairSchedulerPreemption(name, mode);
     takeAllResources("root.preemptable.child-1");
     setNumAMContainersPerNode(2);
     preemptHalfResources("root.preemptable.child-2");
@@ -406,12 +412,14 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
     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
-  public void testAppNotPreemptedBelowFairShare() throws Exception {
+  @ParameterizedTest(name = "{0}")
+  @MethodSource("getParameters")
+  public void testAppNotPreemptedBelowFairShare(String name, int mode) throws Exception {
+    initTestFairSchedulerPreemption(name, mode);
     takeAllResources("root.preemptable.child-1");
     tryPreemptMoreThanFairShare("root.preemptable.child-2");
   }
@@ -426,13 +434,18 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
     verifyPreemption(1, 5);
   }
 
-  @Test
-  public void testDisableAMPreemption() {
+  @ParameterizedTest(name = "{0}")
+  @MethodSource("getParameters")
+  public void testDisableAMPreemption(String name, int mode) throws IOException {
+    initTestFairSchedulerPreemption(name, mode);
     testDisableAMPreemption(false);
   }
 
-  @Test
-  public void testDisableAMPreemptionGlobally() {
+  @ParameterizedTest(name = "{0}")
+  @MethodSource("getParameters")
+  public void testDisableAMPreemptionGlobally(String name, int mode)
+      throws IOException {
+    initTestFairSchedulerPreemption(name, mode);
     testDisableAMPreemption(true);
   }
 
@@ -449,9 +462,11 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
     assertFalse(greedyApp.canContainerBePreempted(container, null));
   }
 
-  @Test
-  public void testPreemptionBetweenSiblingQueuesWithParentAtFairShare()
-      throws InterruptedException {
+  @ParameterizedTest(name = "{0}")
+  @MethodSource("getParameters")
+  public void testPreemptionBetweenSiblingQueuesWithParentAtFairShare(
+      String name, int mode) throws InterruptedException, IOException {
+    initTestFairSchedulerPreemption(name, mode);
     // Run this test only for fairshare preemption
     if (!fairsharePreemption) {
       return;
@@ -472,9 +487,11 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
   /* It tests the case that there is less-AM-container solution in the
    * remaining nodes.
    */
-  @Test
-  public void testRelaxLocalityPreemptionWithLessAMInRemainingNodes()
-      throws Exception {
+  @ParameterizedTest(name = "{0}")
+  @MethodSource("getParameters")
+  public void testRelaxLocalityPreemptionWithLessAMInRemainingNodes(
+      String name, int mode) throws Exception {
+    initTestFairSchedulerPreemption(name, mode);
     takeAllResources("root.preemptable.child-1");
     RMNode node1 = rmNodes.get(0);
     setAllAMContainersOnNode(node1.getNodeID());
@@ -487,9 +504,11 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
   /* It tests the case that there is no less-AM-container solution in the
    * remaining nodes.
    */
-  @Test
-  public void testRelaxLocalityPreemptionWithNoLessAMInRemainingNodes()
-      throws Exception {
+  @ParameterizedTest(name = "{0}")
+  @MethodSource("getParameters")
+  public void testRelaxLocalityPreemptionWithNoLessAMInRemainingNodes(
+      String name, int mode) throws Exception {
+    initTestFairSchedulerPreemption(name, mode);
     takeAllResources("root.preemptable.child-1");
     RMNode node1 = rmNodes.get(0);
     setNumAMContainersOnNode(3, node1.getNodeID());

+ 4 - 4
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}.

+ 6 - 6
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.conf.YarnConfiguration;
 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 java.util.Map;
 
 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 class TestFairSchedulerWithMultiResourceTypes
     scheduler.setRMContext(rmContext);
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
     if (scheduler != null) {
       scheduler.stop();

+ 4 - 4
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.api.records.ApplicationId;
 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();

+ 147 - 146
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,8 +17,14 @@
 */
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 import java.util.Collections;
 import java.util.Set;
@@ -32,9 +38,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManage
 import org.apache.hadoop.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.mockito.Mockito;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet;
 
@@ -46,7 +51,7 @@ public class TestQueueManager {
   private QueueManager queueManager;
   private FairScheduler scheduler;
 
-  @Before
+  @BeforeEach
   public void setUp() {
     PlacementManager placementManager = new PlacementManager();
     FairSchedulerConfiguration conf = new FairSchedulerConfiguration();
@@ -194,10 +199,9 @@ public class TestQueueManager {
   public void testCreateLeafQueue() {
     FSQueue q1 = queueManager.createQueue("root.queue1", FSQueueType.LEAF);
 
-    assertNotNull("Leaf queue root.queue1 was not created",
-        queueManager.getLeafQueue("root.queue1", false));
-    assertEquals("createQueue() returned wrong queue",
-        "root.queue1", q1.getName());
+    assertNotNull(queueManager.getLeafQueue("root.queue1", false),
+        "Leaf queue root.queue1 was not created");
+    assertEquals("root.queue1", q1.getName(), "createQueue() returned wrong queue");
   }
 
   /**
@@ -208,12 +212,12 @@ public class TestQueueManager {
     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));
-    assertEquals("createQueue() returned wrong queue",
-        "root.queue1.queue2", q2.getName());
+    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("root.queue1.queue2", q2.getName(),
+        "createQueue() returned wrong queue");
   }
 
   /**
@@ -228,38 +232,38 @@ public class TestQueueManager {
         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));
-    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());
+    assertNotNull(queueManager.getLeafQueue("root.test.childC", false),
+        "Leaf queue root.test.childC was not created");
+    assertEquals(q1.getName(), "root.test.childC", "createQueue() returned wrong queue");
+    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));
-    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());
+    assertNotNull(queueManager.getParentQueue("root.test.childD", false),
+        "Leaf queue root.test.childD was not created");
+    assertEquals("root.test.childD", q2.getName(),
+        "createQueue() returned wrong queue");
+    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 +273,17 @@ public class TestQueueManager {
   public void testCreateLeafQueueWithDefaults() {
     FSQueue q1 = queueManager.createQueue("root.queue1", FSQueueType.LEAF);
 
-    assertNotNull("Leaf queue root.queue1 was not created",
-        queueManager.getLeafQueue("root.queue1", false));
-    assertEquals("createQueue() returned wrong queue",
-        "root.queue1", q1.getName());
+    assertNotNull(queueManager.getLeafQueue("root.queue1", false),
+        "Leaf queue root.queue1 was not created");
+    assertEquals("root.queue1", q1.getName(), "createQueue() returned wrong queue");
 
     // 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,10 +293,10 @@ public class TestQueueManager {
   public void testCreateParentQueue() {
     FSQueue q1 = queueManager.createQueue("root.queue1", FSQueueType.PARENT);
 
-    assertNotNull("Parent queue root.queue1 was not created",
-        queueManager.getParentQueue("root.queue1", false));
-    assertEquals("createQueue() returned wrong queue",
-        "root.queue1", q1.getName());
+    assertNotNull(queueManager.getParentQueue("root.queue1", false),
+        "Parent queue root.queue1 was not created");
+    assertEquals("root.queue1", q1.getName(),
+        "createQueue() returned wrong queue");
   }
 
   /**
@@ -305,12 +307,12 @@ public class TestQueueManager {
     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));
-    assertEquals("createQueue() returned wrong queue",
-        "root.queue1.queue2", q2.getName());
+    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("root.queue1.queue2", q2.getName(),
+        "createQueue() returned wrong queue");
   }
 
   /**
@@ -321,11 +323,10 @@ public class TestQueueManager {
     FSLeafQueue q1 = queueManager.getLeafQueue("root.test.childB.dynamic1",
         true);
 
-    assertNotNull("Queue root.test.childB.dynamic1 was not created", q1);
-    assertEquals("createQueue() returned wrong queue",
-        "root.test.childB.dynamic1", q1.getName());
-    assertTrue("root.test.childB.dynamic1 is not a dynamic queue",
-        q1.isDynamic());
+    assertNotNull(q1, "Queue root.test.childB.dynamic1 was not created");
+    assertEquals("root.test.childB.dynamic1", q1.getName(),
+        "createQueue() returned wrong queue");
+    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 +336,7 @@ public class TestQueueManager {
     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 +345,9 @@ public class TestQueueManager {
     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 +357,22 @@ public class TestQueueManager {
   public void testRemovalOfDynamicParentQueue() {
     FSQueue q1 = queueManager.getLeafQueue("root.parent1.dynamic1", true);
 
-    assertNotNull("Queue root.parent1.dynamic1 was not created", q1);
-    assertEquals("createQueue() returned wrong queue",
-        "root.parent1.dynamic1", q1.getName());
-    assertTrue("root.parent1.dynamic1 is not a dynamic queue", q1.isDynamic());
+    assertNotNull(q1, "Queue root.parent1.dynamic1 was not created");
+    assertEquals("root.parent1.dynamic1", q1.getName(),
+        "createQueue() returned wrong queue");
+    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 +382,10 @@ public class TestQueueManager {
   public void testNonEmptyDynamicQueueBecomingStaticQueue() {
     FSLeafQueue q1 = queueManager.getLeafQueue("root.leaf1", true);
 
-    assertNotNull("Queue root.leaf1 was not created", q1);
-    assertEquals("createQueue() returned wrong queue",
-        "root.leaf1", q1.getName());
-    assertTrue("root.leaf1 is not a dynamic queue", q1.isDynamic());
+    assertNotNull(q1, "Queue root.leaf1 was not created");
+    assertEquals("root.leaf1", q1.getName(),
+        "createQueue() returned wrong queue");
+    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 +395,7 @@ public class TestQueueManager {
     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 +403,7 @@ public class TestQueueManager {
     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 +412,8 @@ public class TestQueueManager {
     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 +423,10 @@ public class TestQueueManager {
   public void testNonEmptyStaticQueueBecomingDynamicQueue() {
     FSLeafQueue q1 = queueManager.getLeafQueue("root.test.childA", false);
 
-    assertNotNull("Queue root.test.childA does not exist", q1);
-    assertEquals("createQueue() returned wrong queue",
-        "root.test.childA", q1.getName());
-    assertFalse("root.test.childA is not a static queue", q1.isDynamic());
+    assertNotNull(q1, "Queue root.test.childA does not exist");
+    assertEquals("root.test.childA", q1.getName(),
+        "createQueue() returned wrong queue");
+    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 +437,8 @@ public class TestQueueManager {
     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 +450,8 @@ public class TestQueueManager {
     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 +461,10 @@ public class TestQueueManager {
     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 +474,16 @@ public class TestQueueManager {
   public void testRemovalOfChildlessParentQueue() {
     FSParentQueue q1 = queueManager.getParentQueue("root.test.childB", false);
 
-    assertNotNull("Queue root.test.childB was not created", q1);
-    assertEquals("createQueue() returned wrong queue",
-        "root.test.childB", q1.getName());
-    assertFalse("root.test.childB is a dynamic queue", q1.isDynamic());
+    assertNotNull(q1, "Queue root.test.childB was not created");
+    assertEquals("root.test.childB", q1.getName(),
+        "createQueue() returned wrong queue");
+    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 +497,7 @@ public class TestQueueManager {
     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 +507,14 @@ public class TestQueueManager {
   @Test
   public void testQueueTypeChange() {
     FSQueue q1 = queueManager.getLeafQueue("root.parent1.leaf1", true);
-    assertNotNull("Queue root.parent1.leaf1 was not created", q1);
-    assertEquals("createQueue() returned wrong queue",
-        "root.parent1.leaf1", q1.getName());
-    assertTrue("root.parent1.leaf1 is not a dynamic queue", q1.isDynamic());
+    assertNotNull(q1, "Queue root.parent1.leaf1 was not created");
+    assertEquals("root.parent1.leaf1", q1.getName(),
+        "createQueue() returned wrong queue");
+    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 +525,9 @@ public class TestQueueManager {
     // 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 +541,9 @@ public class TestQueueManager {
     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,27 +552,27 @@ public class TestQueueManager {
   @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);
     ActiveUsersManager activeUsersManager =
-        Mockito.mock(ActiveUsersManager.class);
-    RMContext rmContext = Mockito.mock(RMContext.class);
+        mock(ActiveUsersManager.class);
+    RMContext rmContext = mock(RMContext.class);
     doReturn(scheduler.getConfig()).when(rmContext).getYarnConfiguration();
 
     // the appAttempt is created
@@ -581,19 +582,19 @@ public class TestQueueManager {
     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 +609,20 @@ public class TestQueueManager {
     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 +632,14 @@ public class TestQueueManager {
 
     // 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 +648,10 @@ public class TestQueueManager {
     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 +661,8 @@ public class TestQueueManager {
   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 +672,16 @@ public class TestQueueManager {
     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 +692,12 @@ public class TestQueueManager {
   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 +707,19 @@ public class TestQueueManager {
     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");
   }
 }

+ 6 - 6
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 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair
     .allocationfile.AllocationFileQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair
     .allocationfile.AllocationFileWriter;
-import org.junit.After;
-import org.junit.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 @@ import static org.junit.Assert.assertEquals;
 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 class TestQueueManagerRealScheduler extends FairSchedulerTestBase {
     scheduler = (FairScheduler) resourceManager.getResourceScheduler();
   }
 
-  @After
+  @AfterEach
   public void teardown() {
     ALLOC_FILE.deleteOnExit();
     if (resourceManager != null) {

+ 18 - 14
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,11 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
-import static org.junit.Assert.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -44,10 +48,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementRule;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.UserPlacementRule;
 import org.apache.hadoop.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 +70,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 +94,7 @@ public class TestQueuePlacementPolicy {
     when(scheduler.getQueueManager()).thenReturn(queueManager);
   }
 
-  @After
+  @AfterEach
   public void cleanTest() {
     placementManager = null;
     queueManager = null;
@@ -155,7 +159,7 @@ public class TestQueuePlacementPolicy {
     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 +269,8 @@ public class TestQueuePlacementPolicy {
     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 +378,7 @@ public class TestQueuePlacementPolicy {
     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 +524,7 @@ public class TestQueuePlacementPolicy {
     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 +646,7 @@ public class TestQueuePlacementPolicy {
   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 {

+ 52 - 52
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
 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.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.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.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 class TestSchedulingPolicy {
     // 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 @@ public class TestSchedulingPolicy {
       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));
+        assertTrue(checkTransitivity(genSchedulable),
+            "The comparator must ensure transitivity");
         return;
       }
 
@@ -192,7 +192,7 @@ public class TestSchedulingPolicy {
     private boolean checkTransitivity(
         Collection<Schedulable> schedulableObjs) {
 
-      Assert.assertEquals(3, schedulableObjs.size());
+      assertEquals(3, schedulableObjs.size());
       Schedulable[] copy = schedulableObjs.toArray(new Schedulable[3]);
 
       if (fairShareComparator.compare(copy[0], copy[1]) > 0) {
@@ -335,14 +335,14 @@ public class TestSchedulingPolicy {
     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,15 @@ public class TestSchedulingPolicy {
 
     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 +389,13 @@ public class TestSchedulingPolicy {
     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 class TestSchedulingPolicy {
     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 class TestSchedulingPolicy {
     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 class TestSchedulingPolicy {
     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 class TestSchedulingPolicy {
     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.");
   }
 }

+ 27 - 26
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.rmnode.RMNode;
 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 class TestSchedulingUpdate extends FairSchedulerTestBase {
     return conf;
   }
 
-  @Before
+  @BeforeEach
   public void setup() {
     conf = createConfiguration();
     resourceManager = new MockRM(conf);
@@ -58,7 +59,7 @@ public class TestSchedulingUpdate extends FairSchedulerTestBase {
     scheduler = (FairScheduler) resourceManager.getResourceScheduler();
   }
 
-  @After
+  @AfterEach
   public void teardown() {
     if (resourceManager != null) {
       resourceManager.stop();
@@ -66,7 +67,8 @@ public class TestSchedulingUpdate extends FairSchedulerTestBase {
     }
   }
 
-  @Test (timeout = 3000)
+  @Test
+  @Timeout(value = 3)
   public void testSchedulingUpdateOnNodeJoinLeave() throws InterruptedException {
 
     verifyNoCalls();
@@ -102,34 +104,33 @@ public class TestSchedulingUpdate extends FairSchedulerTestBase {
       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");
   }
 }

+ 15 - 16
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.rmnode.RMNode;
 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 class TestVisitedResourceRequestTracker {
         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 class TestVisitedResourceRequestTracker {
         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,13 @@ public class TestVisitedResourceRequestTracker {
         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");
   }
 }

+ 2 - 2
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.IOException;
 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 class FSConfigConverterTestCommons {
       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() {

+ 14 - 10
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,13 +21,15 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.converter;
 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.runner.RunWith;
-import org.mockito.junit.MockitoJUnitRunner;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.junit.jupiter.MockitoExtension;
 
-@RunWith(MockitoJUnitRunner.class)
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+@ExtendWith(MockitoExtension.class)
 public class TestConvertedConfigValidator {
   private static final String CONFIG_DIR_PASSES =
       new File("src/test/resources/cs-validation-pass").getAbsolutePath();
@@ -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 class TestConvertedConfigValidator {
     // 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);
+    });
   }
 }

+ 73 - 85
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,12 +17,13 @@
 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;
+import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
 
 import java.io.File;
@@ -34,17 +35,14 @@ import org.apache.commons.io.FileUtils;
 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.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
 import org.mockito.ArgumentCaptor;
 import org.mockito.ArgumentMatchers;
 import org.mockito.Mock;
-import org.mockito.Mockito;
-import org.mockito.junit.MockitoJUnitRunner;
+import org.mockito.junit.jupiter.MockitoExtension;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -52,14 +50,11 @@ import org.slf4j.LoggerFactory;
  * Unit tests for FSConfigToCSConfigArgumentHandler.
  *
  */
-@RunWith(MockitoJUnitRunner.class)
+@ExtendWith(MockitoExtension.class)
 public class TestFSConfigToCSConfigArgumentHandler {
   private static final Logger LOG =
       LoggerFactory.getLogger(TestFSConfigToCSConfigArgumentHandler.class);
 
-  @Rule
-  public ExpectedException expectedException = ExpectedException.none();
-
   @Mock
   private FSConfigToCSConfigConverter mockConverter;
 
@@ -71,7 +66,7 @@ public class TestFSConfigToCSConfigArgumentHandler {
 
   private FSConfigConverterTestCommons fsTestCommons;
 
-  @Before
+  @BeforeEach
   public void setUp() throws IOException {
     fsTestCommons = new FSConfigConverterTestCommons();
     fsTestCommons.setUp();
@@ -79,7 +74,7 @@ public class TestFSConfigToCSConfigArgumentHandler {
     conversionOptions = new ConversionOptions(dryRunResultHolder, false);
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
     QueueMetrics.clearQueueMetrics();
     fsTestCommons.tearDown();
@@ -154,10 +149,10 @@ public class TestFSConfigToCSConfigArgumentHandler {
         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 +175,11 @@ public class TestFSConfigToCSConfigArgumentHandler {
         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,17 +218,17 @@ public class TestFSConfigToCSConfigArgumentHandler {
         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
   public void testVerificationException() throws Exception {
     setupFSConfigConversionFiles(true);
-    ConversionOptions mockOptions = Mockito.mock(ConversionOptions.class);
+    ConversionOptions mockOptions = mock(ConversionOptions.class);
     FSConfigToCSConfigArgumentHandler argumentHandler =
         new FSConfigToCSConfigArgumentHandler(mockOptions, mockValidator);
     argumentHandler.setConverterSupplier(this::getMockConverter);
@@ -324,15 +319,13 @@ public class TestFSConfigToCSConfigArgumentHandler {
     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 +349,13 @@ public class TestFSConfigToCSConfigArgumentHandler {
     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 +380,14 @@ public class TestFSConfigToCSConfigArgumentHandler {
     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("Cluster resource", "vcores=20, memory-mb=240",
-        params.getClusterResource());
-    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");
+    assertEquals("vcores=20, memory-mb=240", params.getClusterResource(), "Cluster resource");
+    assertTrue(params.isConsole(), "Console mode");
   }
 
   @Test
@@ -413,13 +400,14 @@ public class TestFSConfigToCSConfigArgumentHandler {
     FSConfigToCSConfigArgumentHandler argumentHandler =
         createArgumentHandler();
 
-    Mockito.doThrow(UnsupportedPropertyException.class)
+    doThrow(UnsupportedPropertyException.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("Unsupported property/setting encountered"));
+    assertEquals(-1, retVal, "Return value");
+    assertTrue(fsTestCommons.getErrContent()
+        .toString().contains("Unsupported property/setting encountered"),
+        "Error content missing");
   }
 
   @Test
@@ -432,12 +420,13 @@ public class TestFSConfigToCSConfigArgumentHandler {
     FSConfigToCSConfigArgumentHandler argumentHandler =
         createArgumentHandler();
 
-    Mockito.doThrow(ConversionException.class).when(mockConverter)
+    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
@@ -479,15 +468,14 @@ public class TestFSConfigToCSConfigArgumentHandler {
         new FSConfigToCSConfigArgumentHandler(conversionOptions, mockValidator);
     argumentHandler.setConverterSupplier(this::getMockConverter);
 
-    Mockito.doThrow(exception).when(mockConverter)
+    doThrow(exception).when(mockConverter)
       .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 +493,8 @@ public class TestFSConfigToCSConfigArgumentHandler {
 
     argumentHandler.parseAndConvert(args);
 
-    assertTrue("-t switch had no effect",
-        conversionOptions.isNoRuleTerminalCheck());
+    assertTrue(conversionOptions.isNoRuleTerminalCheck(),
+        "-t switch had no effect");
   }
 
   @Test
@@ -523,8 +511,8 @@ public class TestFSConfigToCSConfigArgumentHandler {
 
     argumentHandler.parseAndConvert(args);
 
-    assertFalse("No terminal rule check was enabled",
-        conversionOptions.isNoRuleTerminalCheck());
+    assertFalse(conversionOptions.isNoRuleTerminalCheck(),
+        "No terminal rule check was enabled");
   }
 
   @Test
@@ -539,7 +527,7 @@ public class TestFSConfigToCSConfigArgumentHandler {
         "-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 +549,7 @@ public class TestFSConfigToCSConfigArgumentHandler {
           "-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 +619,11 @@ public class TestFSConfigToCSConfigArgumentHandler {
     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 +684,8 @@ public class TestFSConfigToCSConfigArgumentHandler {
             "-a");
     argumentHandler.parseAndConvert(args);
 
-    assertTrue("-a switch had no effect",
-            conversionOptions.isEnableAsyncScheduler());
+    assertTrue(conversionOptions.isEnableAsyncScheduler(),
+        "-a switch had no effect");
   }
 
   @Test
@@ -711,8 +699,8 @@ public class TestFSConfigToCSConfigArgumentHandler {
             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 +737,6 @@ public class TestFSConfigToCSConfigArgumentHandler {
     verify(mockConverter).convert(captor.capture());
     FSConfigToCSConfigConverterParams params = captor.getValue();
 
-    assertEquals("Use percentages", enabled, params.isUsePercentages());
+    assertEquals(enabled, params.isUsePercentages(), "Use percentages");
   }
 }

+ 185 - 204
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,11 +31,14 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.conve
 
 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.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.Mockito.anyBoolean;
+import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.verify;
 
 import java.io.ByteArrayInputStream;
@@ -55,15 +58,12 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePat
 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.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
 import org.mockito.Mock;
-import org.mockito.Mockito;
-import org.mockito.junit.MockitoJUnitRunner;
+import org.mockito.junit.jupiter.MockitoExtension;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 
@@ -72,7 +72,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
  * Unit tests for FSConfigToCSConfigConverter.
  *
  */
-@RunWith(MockitoJUnitRunner.class)
+@ExtendWith(MockitoExtension.class)
 public class TestFSConfigToCSConfigConverter {
   private static final String CLUSTER_RESOURCE_STRING =
       "vcores=20, memory-mb=240";
@@ -104,8 +104,6 @@ public class TestFSConfigToCSConfigConverter {
   private FSConfigToCSConfigConverter converter;
   private Configuration config;
 
-  @Rule
-  public ExpectedException expectedException = ExpectedException.none();
   private FSConfigConverterTestCommons converterTestCommons;
 
   private static String prepareFileName(String f) {
@@ -136,7 +134,7 @@ public class TestFSConfigToCSConfigConverter {
     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 +145,7 @@ public class TestFSConfigToCSConfigConverter {
     converterTestCommons.setUp();
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
     converterTestCommons.tearDown();
   }
@@ -183,17 +181,17 @@ public class TestFSConfigToCSConfigConverter {
     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 +200,21 @@ public class TestFSConfigToCSConfigConverter {
 
     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 +229,20 @@ public class TestFSConfigToCSConfigConverter {
     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
@@ -254,34 +252,34 @@ public class TestFSConfigToCSConfigConverter {
     CapacitySchedulerConfiguration conf = converter.getCapacitySchedulerConfig();
 
     // root
-    assertEquals("root submit ACL", "alice,bob,joe,john hadoop_users",
-        conf.getAcl(ROOT, QueueACL.SUBMIT_APPLICATIONS).getAclString());
-    assertEquals("root admin ACL", "alice,bob,joe,john hadoop_users",
-        conf.getAcl(ROOT, QueueACL.ADMINISTER_QUEUE).getAclString());
+    assertEquals("alice,bob,joe,john hadoop_users",
+        conf.getAcl(ROOT, QueueACL.SUBMIT_APPLICATIONS).getAclString(), "root submit ACL");
+    assertEquals("alice,bob,joe,john hadoop_users",
+        conf.getAcl(ROOT, QueueACL.ADMINISTER_QUEUE).getAclString(), "root admin ACL");
 
     // root.admins.bob
-    assertEquals("root.admins.bob submit ACL", "bob ",
-        conf.getAcl(ADMINS_BOB, QueueACL.SUBMIT_APPLICATIONS).getAclString());
-    assertEquals("root.admins.bob admin ACL", "bob ",
-        conf.getAcl(ADMINS_BOB, QueueACL.ADMINISTER_QUEUE).getAclString());
+    assertEquals("bob ", conf.getAcl(ADMINS_BOB, QueueACL.SUBMIT_APPLICATIONS).getAclString(),
+        "root.admins.bob submit ACL");
+    assertEquals("bob ", conf.getAcl(ADMINS_BOB, QueueACL.ADMINISTER_QUEUE).getAclString(),
+        "root.admins.bob admin ACL");
 
     // root.admins.alice
-    assertEquals("root.admins.alice submit ACL", "alice ",
-        conf.getAcl(ADMINS_ALICE, QueueACL.SUBMIT_APPLICATIONS).getAclString());
-    assertEquals("root.admins.alice admin ACL", "alice ",
-        conf.getAcl(ADMINS_ALICE, QueueACL.ADMINISTER_QUEUE).getAclString());
+    assertEquals("alice ", conf.getAcl(ADMINS_ALICE, QueueACL.SUBMIT_APPLICATIONS).getAclString(),
+        "root.admins.alice submit ACL");
+    assertEquals("alice ", conf.getAcl(ADMINS_ALICE, QueueACL.ADMINISTER_QUEUE).getAclString(),
+        "root.admins.alice admin ACL");
 
     // root.users.john
-    assertEquals("root.users.john submit ACL", "*",
-        conf.getAcl(USERS_JOHN, QueueACL.SUBMIT_APPLICATIONS).getAclString());
-    assertEquals("root.users.john admin ACL", "*",
-        conf.getAcl(USERS_JOHN, QueueACL.ADMINISTER_QUEUE).getAclString());
+    assertEquals("*", conf.getAcl(USERS_JOHN, QueueACL.SUBMIT_APPLICATIONS).getAclString(),
+        "root.users.john submit ACL");
+    assertEquals("*", conf.getAcl(USERS_JOHN, QueueACL.ADMINISTER_QUEUE).getAclString(),
+        "root.users.john admin ACL");
 
     // root.users.joe
-    assertEquals("root.users.joe submit ACL", "joe ",
-        conf.getAcl(USERS_JOE, QueueACL.SUBMIT_APPLICATIONS).getAclString());
-    assertEquals("root.users.joe admin ACL", "joe ",
-        conf.getAcl(USERS_JOE, QueueACL.ADMINISTER_QUEUE).getAclString());
+    assertEquals("joe ", conf.getAcl(USERS_JOE, QueueACL.SUBMIT_APPLICATIONS).getAclString(),
+        "root.users.joe submit ACL");
+    assertEquals("joe ", conf.getAcl(USERS_JOE, QueueACL.ADMINISTER_QUEUE).getAclString(),
+        "root.users.joe admin ACL");
   }
 
   @Test
@@ -290,8 +288,7 @@ public class TestFSConfigToCSConfigConverter {
 
     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 +297,8 @@ public class TestFSConfigToCSConfigConverter {
 
     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 +307,8 @@ public class TestFSConfigToCSConfigConverter {
 
     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,40 +317,40 @@ public class TestFSConfigToCSConfigConverter {
 
     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
   public void testQueueMaxChildCapacityNotSupported() throws Exception {
-    expectedException.expect(UnsupportedPropertyException.class);
-    expectedException.expectMessage("test");
-
-    Mockito.doThrow(new UnsupportedPropertyException("test"))
-      .when(ruleHandler).handleMaxChildCapacity();
-
-    converter.convert(config);
+    UnsupportedPropertyException exception =
+        assertThrows(UnsupportedPropertyException.class, () -> {
+          doThrow(new UnsupportedPropertyException("test"))
+              .when(ruleHandler).handleMaxChildCapacity();
+          converter.convert(config);
+        });
+    assertThat(exception.getMessage()).contains("test");
   }
 
   @Test
   public void testReservationSystemNotSupported() throws Exception {
-    expectedException.expect(UnsupportedPropertyException.class);
-    expectedException.expectMessage("maxCapacity");
-
-    Mockito.doThrow(new UnsupportedPropertyException("maxCapacity"))
-      .when(ruleHandler).handleMaxChildCapacity();
-    config.setBoolean(YarnConfiguration.RM_RESERVATION_SYSTEM_ENABLE, true);
-
-    converter.convert(config);
+    UnsupportedPropertyException exception =
+        assertThrows(UnsupportedPropertyException.class, () -> {
+          doThrow(new UnsupportedPropertyException("maxCapacity"))
+              .when(ruleHandler).handleMaxChildCapacity();
+          config.setBoolean(YarnConfiguration.RM_RESERVATION_SYSTEM_ENABLE, true);
+          converter.convert(config);
+        });
+    assertThat(exception.getMessage()).contains("maxCapacity");
   }
 
   @Test
@@ -362,8 +359,8 @@ public class TestFSConfigToCSConfigConverter {
         .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,34 +370,34 @@ public class TestFSConfigToCSConfigConverter {
             .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
   public void testConvertFSConfigurationClusterResourceInvalid()
       throws Exception {
-    FSConfigToCSConfigConverterParams params = createDefaultParamsBuilder()
-            .withClusterResource("vcores=20, memory-mb=240G")
-            .build();
-
-    expectedException.expect(ConversionException.class);
-    expectedException.expectMessage("Error while parsing resource");
-
-    converter.convert(params);
+    ConversionException exception =
+        assertThrows(ConversionException.class, () -> {
+          FSConfigToCSConfigConverterParams params = createDefaultParamsBuilder()
+              .withClusterResource("vcores=20, memory-mb=240G")
+              .build();
+          converter.convert(params);
+        });
+    assertThat(exception.getMessage()).contains("Error while parsing resource");
   }
 
   @Test
   public void testConvertFSConfigurationClusterResourceInvalid2()
       throws Exception {
-    FSConfigToCSConfigConverterParams params = createDefaultParamsBuilder()
-        .withClusterResource("vcores=20, memmmm=240")
-        .build();
-
-    expectedException.expect(ConversionException.class);
-    expectedException.expectMessage("Error while parsing resource");
-
-    converter.convert(params);
+    ConversionException exception =
+        assertThrows(ConversionException.class, () -> {
+          FSConfigToCSConfigConverterParams params = createDefaultParamsBuilder()
+              .withClusterResource("vcores=20, memmmm=240")
+              .build();
+          converter.convert(params);
+        });
+    assertThat(exception.getMessage()).contains("Error while parsing resource");
   }
 
   @Test
@@ -426,16 +423,11 @@ public class TestFSConfigToCSConfigConverter {
     Map<String, FSConfigToCSConfigRuleHandler.RuleAction> 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,44 +447,33 @@ public class TestFSConfigToCSConfigConverter {
     Map<String, FSConfigToCSConfigRuleHandler.RuleAction> 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
   public void testConvertFSConfigurationUndefinedYarnSiteConfig()
       throws Exception {
-    FSConfigToCSConfigConverterParams params =
-        FSConfigToCSConfigConverterParams.Builder.create()
-            .withYarnSiteXmlConfig(null)
-            .withOutputDirectory(FSConfigConverterTestCommons.OUTPUT_DIR)
-            .build();
-
-    expectedException.expect(PreconditionException.class);
-    expectedException.expectMessage(
-        "yarn-site.xml configuration is not defined");
-
-    converter.convert(params);
+    PreconditionException exception =
+        assertThrows(PreconditionException.class, () -> {
+          FSConfigToCSConfigConverterParams params =
+              FSConfigToCSConfigConverterParams.Builder.create()
+              .withYarnSiteXmlConfig(null)
+              .withOutputDirectory(FSConfigConverterTestCommons.OUTPUT_DIR)
+              .build();
+          converter.convert(params);
+        });
+    assertThat(exception.getMessage()).contains("yarn-site.xml configuration is not defined");
   }
 
   @Test
@@ -510,65 +491,66 @@ public class TestFSConfigToCSConfigConverter {
 
     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
   public void testFairSchedulerXmlIsNotDefinedNeitherDirectlyNorInYarnSiteXml()
       throws Exception {
-    FSConfigToCSConfigConverterParams params =
-        createParamsBuilder(YARN_SITE_XML_NO_REF_TO_FS_XML)
-        .withClusterResource(CLUSTER_RESOURCE_STRING)
-        .build();
-
-    expectedException.expect(PreconditionException.class);
-    expectedException.expectMessage("fair-scheduler.xml is not defined");
-    converter.convert(params);
+    PreconditionException exception =
+        assertThrows(PreconditionException.class, () -> {
+          FSConfigToCSConfigConverterParams params =
+              createParamsBuilder(YARN_SITE_XML_NO_REF_TO_FS_XML)
+              .withClusterResource(CLUSTER_RESOURCE_STRING)
+              .build();
+          converter.convert(params);
+        });
+    assertThat(exception.getMessage()).contains("fair-scheduler.xml is not defined");
   }
 
   @Test
   public void testInvalidFairSchedulerXml() throws Exception {
-    FSConfigToCSConfigConverterParams params = createDefaultParamsBuilder()
-        .withClusterResource(CLUSTER_RESOURCE_STRING)
-        .withFairSchedulerXmlConfig(FAIR_SCHEDULER_XML_INVALID)
-        .build();
-
-    expectedException.expect(RuntimeException.class);
-    converter.convert(params);
+    assertThrows(RuntimeException.class, () -> {
+      FSConfigToCSConfigConverterParams params = createDefaultParamsBuilder()
+          .withClusterResource(CLUSTER_RESOURCE_STRING)
+          .withFairSchedulerXmlConfig(FAIR_SCHEDULER_XML_INVALID)
+          .build();
+      converter.convert(params);
+    });
   }
 
   @Test
   public void testInvalidYarnSiteXml() throws Exception {
-    FSConfigToCSConfigConverterParams params =
-        createParamsBuilder(YARN_SITE_XML_INVALID)
-        .withClusterResource(CLUSTER_RESOURCE_STRING)
-        .build();
-
-    expectedException.expect(RuntimeException.class);
-    converter.convert(params);
+    assertThrows(RuntimeException.class, () -> {
+      FSConfigToCSConfigConverterParams params =
+          createParamsBuilder(YARN_SITE_XML_INVALID)
+          .withClusterResource(CLUSTER_RESOURCE_STRING)
+          .build();
+      converter.convert(params);
+    });
   }
 
   @Test
   public void testConversionWithInvalidPlacementRules() throws Exception {
-    config = new Configuration(false);
-    config.set(FairSchedulerConfiguration.ALLOCATION_FILE,
-        FS_INVALID_PLACEMENT_RULES_XML);
-    config.setBoolean(FairSchedulerConfiguration.MIGRATION_MODE, true);
-    expectedException.expect(ServiceStateException.class);
-
-    converter.convert(config);
+    assertThrows(ServiceStateException.class, () -> {
+      config = new Configuration(false);
+      config.set(FairSchedulerConfiguration.ALLOCATION_FILE,
+          FS_INVALID_PLACEMENT_RULES_XML);
+      config.setBoolean(FairSchedulerConfiguration.MIGRATION_MODE, true);
+      converter.convert(config);
+    });
   }
 
   @Test
@@ -601,9 +583,9 @@ public class TestFSConfigToCSConfigConverter {
 
     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 +599,10 @@ public class TestFSConfigToCSConfigConverter {
 
     Configuration convertedConfig = converter.getYarnSiteConfig();
 
-    assertEquals("Resource calculator type", DominantResourceCalculator.class,
+    assertEquals(DominantResourceCalculator.class,
         convertedConfig.getClass(
-            CapacitySchedulerConfiguration.RESOURCE_CALCULATOR_CLASS, null));
+        CapacitySchedulerConfiguration.RESOURCE_CALCULATOR_CLASS, null),
+        "Resource calculator type");
   }
 
   @Test
@@ -664,10 +647,10 @@ public class TestFSConfigToCSConfigConverter {
 
     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 +692,14 @@ public class TestFSConfigToCSConfigConverter {
   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 +711,8 @@ public class TestFSConfigToCSConfigConverter {
         .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 {

+ 30 - 30
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.conve
 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.Arrays;
 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 class TestFSConfigToCSConfigConverterMain {
     FSConfigToCSConfigConverterMain.setExit(exitFunc);
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     QueueMetrics.clearQueueMetrics();
     FSConfigToCSConfigConverterMain.setExit(System::exit);
@@ -108,9 +108,9 @@ public class TestFSConfigToCSConfigConverterMain {
     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 class TestFSConfigToCSConfigConverterMain {
         "-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 class TestFSConfigToCSConfigConverterMain {
     FSConfigToCSConfigConverterMain.main(new String[] {"-h"});
 
     verifyHelpText();
-    assertEquals("Exit code", 0, exitFunc.exitCode);
+    assertEquals(0, exitFunc.exitCode, "Exit code");
   }
 
   @Test
@@ -148,7 +148,7 @@ public class TestFSConfigToCSConfigConverterMain {
     FSConfigToCSConfigConverterMain.main(new String[] {"--help"});
 
     verifyHelpText();
-    assertEquals("Exit code", 0, exitFunc.exitCode);
+    assertEquals(0, exitFunc.exitCode, "Exit code");
   }
 
   @Test
@@ -156,7 +156,7 @@ public class TestFSConfigToCSConfigConverterMain {
     FSConfigToCSConfigConverterMain.main(new String[] {});
 
     verifyHelpText();
-    assertEquals("Exit code", 0, exitFunc.exitCode);
+    assertEquals(0, exitFunc.exitCode, "Exit code");
   }
 
   @Test
@@ -173,13 +173,13 @@ public class TestFSConfigToCSConfigConverterMain {
         "--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 class TestFSConfigToCSConfigConverterMain {
         "--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")

+ 24 - 21
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,15 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.conve
 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.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 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 +46,7 @@ public class TestFSConfigToCSConfigRuleHandler {
   private FSConfigToCSConfigRuleHandler ruleHandler;
   private DryRunResultHolder dryRunResultHolder;
 
-  @Before
+  @BeforeEach
   public void setup() {
     dryRunResultHolder = new DryRunResultHolder();
   }
@@ -123,15 +124,17 @@ public class TestFSConfigToCSConfigRuleHandler {
     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 +147,10 @@ public class TestFSConfigToCSConfigRuleHandler {
     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 +165,10 @@ public class TestFSConfigToCSConfigRuleHandler {
     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 +186,8 @@ public class TestFSConfigToCSConfigRuleHandler {
       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

+ 150 - 164
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,10 +18,13 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.converter;
 
 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.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 
@@ -42,21 +45,18 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePat
 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.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
 import org.mockito.Mock;
-import org.mockito.Mockito;
-import org.mockito.junit.MockitoJUnitRunner;
+import org.mockito.junit.jupiter.MockitoExtension;
 
 /**
  * Unit tests for FSQueueConverter.
  *
  */
-@RunWith(MockitoJUnitRunner.class)
+@ExtendWith(MockitoExtension.class)
 public class TestFSQueueConverter {
   private static final float MAX_AM_SHARE_DEFAULT = 0.16f;
   private static final int MAX_APPS_DEFAULT = 15;
@@ -108,10 +108,7 @@ public class TestFSQueueConverter {
   @Mock
   private FSConfigToCSConfigRuleHandler ruleHandler;
 
-  @Rule
-  public ExpectedException expectedException = ExpectedException.none();
-
-  @Before
+  @BeforeEach
   public void setup() {
     yarnConfig = new Configuration(false);
     yarnConfig.set(FairSchedulerConfiguration.ALLOCATION_FILE,
@@ -129,7 +126,7 @@ public class TestFSQueueConverter {
     rootQueue = fs.getQueueManager().getRootQueue();
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws IOException {
     if (fs != null) {
       fs.close();
@@ -167,20 +164,19 @@ public class TestFSQueueConverter {
     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<String> leafs = Sets.difference(ALL_QUEUES,
         Sets.newHashSet("root",
@@ -191,8 +187,8 @@ public class TestFSQueueConverter {
 
     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 +200,23 @@ public class TestFSQueueConverter {
     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<String> 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",
-          0.1f, csConfig
-              .getMaximumApplicationMasterResourcePerQueuePercent(new QueuePath(queue)), 0.0f);
+      assertEquals(0.1f,
+          csConfig.getMaximumApplicationMasterResourcePerQueuePercent(new QueuePath(queue)), 0.0f,
+          "Key " + key + " has different value");
     }
   }
 
@@ -230,17 +226,17 @@ public class TestFSQueueConverter {
 
     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<String> 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",
-          DEFAULT_MAX_PARALLEL_APPLICATIONS, csConfig
-              .getMaxParallelAppsForQueue(new QueuePath(queue)), 0);
+      assertEquals(DEFAULT_MAX_PARALLEL_APPLICATIONS,
+          csConfig.getMaxParallelAppsForQueue(new QueuePath(queue)), 0,
+          "Key " + key + " has different value");
     }
   }
 
@@ -251,30 +247,30 @@ public class TestFSQueueConverter {
     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<String> 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",
-          -1.0, csConfig
-              .getQueueMaximumAllocationVcores(new QueuePath(queue)), 0.0f);
+      assertEquals(-1.0,
+          csConfig.getQueueMaximumAllocationVcores(new QueuePath(queue)), 0.0f,
+          "Key " + key + " has different value");
 
       key = PREFIX + queue + ".maximum-allocation-mb";
-      assertEquals("Key " + key + " has different value",
-          -1.0, csConfig
-              .getQueueMaximumAllocationMb(new QueuePath(queue)), 0.0f);
+      assertEquals(-1.0,
+          csConfig.getQueueMaximumAllocationMb(new QueuePath(queue)), 0.0f,
+          "Key " + key + " has different value");
     }
   }
 
@@ -284,20 +280,18 @@ public class TestFSQueueConverter {
 
     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<String> 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 +303,8 @@ public class TestFSQueueConverter {
 
     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 +315,32 @@ public class TestFSQueueConverter {
     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(75.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 +350,32 @@ public class TestFSQueueConverter {
     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 +384,16 @@ public class TestFSQueueConverter {
 
     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 +406,8 @@ public class TestFSQueueConverter {
 
     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 +423,11 @@ public class TestFSQueueConverter {
 
     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 +438,8 @@ public class TestFSQueueConverter {
 
     for (String queue : ALL_QUEUES) {
       key = PREFIX + queue + ".maximum-capacity";
-      assertEquals("Key " + key + " has different value",
-          100.0, csConfig
-              .getNonLabeledQueueMaximumCapacity(new QueuePath(queue)), 0.0f);
+      assertEquals(100.0, csConfig.getNonLabeledQueueMaximumCapacity(new QueuePath(queue)), 0.0f,
+          "Key " + key + " has different value");
     }
     verify(ruleHandler, times(3)).handleMaxResources();
   }
@@ -473,8 +463,8 @@ public class TestFSQueueConverter {
 
     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 +476,7 @@ public class TestFSQueueConverter {
 
     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 +488,7 @@ public class TestFSQueueConverter {
 
     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");
     }
   }
 
@@ -518,26 +506,26 @@ public class TestFSQueueConverter {
 
     converter.convertQueueHierarchy(rootQueue);
     // root
-    assertEquals("root ordering policy", "fifo",
-        csConfig.getAppOrderingPolicy(ROOT).getConfigName());
-    assertEquals("root.default ordering policy", "fair",
-        csConfig.getAppOrderingPolicy(DEFAULT).getConfigName());
-    assertEquals("root.admins ordering policy", "fifo",
-        csConfig.getAppOrderingPolicy(ADMINS).getConfigName());
-    assertEquals("root.users ordering policy", "fifo",
-        csConfig.getAppOrderingPolicy(USERS).getConfigName());
+    assertEquals("fifo", csConfig.getAppOrderingPolicy(ROOT).getConfigName(),
+        "root ordering policy");
+    assertEquals("fair", csConfig.getAppOrderingPolicy(DEFAULT).getConfigName(),
+        "root.default ordering policy");
+    assertEquals("fifo", csConfig.getAppOrderingPolicy(ADMINS).getConfigName(),
+        "root.admins ordering policy");
+    assertEquals("fifo", csConfig.getAppOrderingPolicy(USERS).getConfigName(),
+        "root.users ordering policy");
 
     // root.users
-    assertEquals("root.users.joe ordering policy", "fair",
-        csConfig.getAppOrderingPolicy(USERS_JOE).getConfigName());
-    assertEquals("root.users.john ordering policy", "fifo",
-        csConfig.getAppOrderingPolicy(USERS_JOHN).getConfigName());
+    assertEquals("fair", csConfig.getAppOrderingPolicy(USERS_JOE).getConfigName(),
+        "root.users.joe ordering policy");
+    assertEquals("fifo", csConfig.getAppOrderingPolicy(USERS_JOHN).getConfigName(),
+        "root.users.john ordering policy");
 
     // root.admins
-    assertEquals("root.admins.alice ordering policy", "fifo",
-        csConfig.getAppOrderingPolicy(ADMINS_ALICE).getConfigName());
-    assertEquals("root.admins.bob ordering policy", "fair",
-        csConfig.getAppOrderingPolicy(ADMINS_BOB).getConfigName());
+    assertEquals("fifo", csConfig.getAppOrderingPolicy(ADMINS_ALICE).getConfigName(),
+        "root.admins.alice ordering policy");
+    assertEquals("fair", csConfig.getAppOrderingPolicy(ADMINS_BOB).getConfigName(),
+        "root.admins.bob ordering policy");
   }
 
   @Test
@@ -559,27 +547,25 @@ public class TestFSQueueConverter {
 
   @Test
   public void testQueueMaxChildCapacityNotSupported() {
-    converter = builder.build();
-    expectedException.expect(UnsupportedPropertyException.class);
-    expectedException.expectMessage("test");
-
-    Mockito.doThrow(new UnsupportedPropertyException("test"))
-      .when(ruleHandler).handleMaxChildCapacity();
-
-    converter.convertQueueHierarchy(rootQueue);
+    UnsupportedPropertyException exception =
+        assertThrows(UnsupportedPropertyException.class, () -> {
+          converter = builder.build();
+          doThrow(new UnsupportedPropertyException("test"))
+              .when(ruleHandler).handleMaxChildCapacity();
+          converter.convertQueueHierarchy(rootQueue);
+        });
+    assertThat(exception.getMessage()).contains("test");
   }
 
   @Test
   public void testReservationSystemNotSupported() {
-    converter = builder.build();
-    expectedException.expect(UnsupportedPropertyException.class);
-    expectedException.expectMessage("maxCapacity");
-
-    Mockito.doThrow(new UnsupportedPropertyException("maxCapacity"))
-      .when(ruleHandler).handleMaxChildCapacity();
-    yarnConfig.setBoolean(YarnConfiguration.RM_RESERVATION_SYSTEM_ENABLE,
-        true);
-
-    converter.convertQueueHierarchy(rootQueue);
+    assertThrows(UnsupportedPropertyException.class, () -> {
+      converter = builder.build();
+      doThrow(new UnsupportedPropertyException("maxCapacity"))
+           .when(ruleHandler).handleMaxChildCapacity();
+      yarnConfig.setBoolean(YarnConfiguration.RM_RESERVATION_SYSTEM_ENABLE,
+           true);
+      converter.convertQueueHierarchy(rootQueue);
+    });
   }
 }

+ 96 - 116
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.capacity.QueueCon
 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 class TestFSYarnSiteConverter {
     converter.convertSiteProperties(yarnConfig, yarnConvertedConfig, false,
         true, false, null);
 
-    assertTrue("Cont. scheduling", yarnConvertedConfig.getBoolean(
-        CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_ENABLE, false));
-    assertEquals("Scheduling interval", 666,
-        yarnConvertedConfig.getInt(
-            "yarn.scheduler.capacity.schedule-asynchronously" +
-                ".scheduling-interval-ms", -1));
+    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");
   }
 
   @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.
-            get(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES)
-            .contains(QueueConfigurationAutoRefreshPolicy.
-                class.getCanonicalName()));
+    assertTrue(true, yarnConvertedConfig.get(YarnConfiguration.
+        RM_SCHEDULER_ENABLE_MONITORS));
+    assertTrue(yarnConvertedConfig.
+        get(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES)
+        .contains(QueueConfigurationAutoRefreshPolicy.
+        class.getCanonicalName()),
+        "Scheduling Policies contains queue conf auto refresh");
   }
 
   @Test
@@ -94,28 +94,21 @@ public class TestFSYarnSiteConverter {
     converter.convertSiteProperties(yarnConfig, yarnConvertedConfig, false,
         false, false, null);
 
-    assertTrue("Preemption enabled",
-        yarnConvertedConfig.getBoolean(
-            YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS,
-            false));
-    assertEquals("Wait time before kill", 123,
-        yarnConvertedConfig.getInt(
-            CapacitySchedulerConfiguration.PREEMPTION_WAIT_TIME_BEFORE_KILL,
-              -1));
-    assertEquals("Starvation check wait time", 321,
-        yarnConvertedConfig.getInt(
-            CapacitySchedulerConfiguration.PREEMPTION_MONITORING_INTERVAL,
-              -1));
-
-    assertFalse("Observe_only should be false",
-        yarnConvertedConfig.getBoolean(CapacitySchedulerConfiguration.
-                PREEMPTION_OBSERVE_ONLY, false));
-
-    assertTrue("Should contain ProportionalCapacityPreemptionPolicy.",
-        yarnConvertedConfig.
-            get(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES).
-        contains(ProportionalCapacityPreemptionPolicy.
-        class.getCanonicalName()));
+    assertTrue(yarnConvertedConfig.
+        getBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS, false), "Preemption enabled");
+    assertEquals(123, yarnConvertedConfig.getInt(
+        CapacitySchedulerConfiguration.PREEMPTION_WAIT_TIME_BEFORE_KILL, -1),
+        "Wait time before kill");
+    assertEquals(321, yarnConvertedConfig.getInt(
+        CapacitySchedulerConfiguration.PREEMPTION_MONITORING_INTERVAL, -1),
+        "Starvation check wait time");
+
+    assertFalse(yarnConvertedConfig.getBoolean(CapacitySchedulerConfiguration.
+        PREEMPTION_OBSERVE_ONLY, false), "Observe_only should be false");
+
+    assertTrue(yarnConvertedConfig.get(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES).
+        contains(ProportionalCapacityPreemptionPolicy.class.getCanonicalName()),
+        "Should contain ProportionalCapacityPreemptionPolicy.");
   }
 
   @Test
@@ -125,22 +118,19 @@ public class TestFSYarnSiteConverter {
     converter.convertSiteProperties(yarnConfig, yarnConvertedConfig, false,
         false, false,  null);
 
-    assertFalse("Should not contain ProportionalCapacityPreemptionPolicy.",
-        yarnConvertedConfig.
-            get(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES).
-            contains(ProportionalCapacityPreemptionPolicy.
-                class.getCanonicalName()));
+    assertFalse(yarnConvertedConfig.get(
+        YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES).
+        contains(ProportionalCapacityPreemptionPolicy.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.
-            get(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES).
-            contains(ProportionalCapacityPreemptionPolicy.
-                class.getCanonicalName()));
+    assertFalse(yarnConvertedConfig.get(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES).
+        contains(ProportionalCapacityPreemptionPolicy.class.getCanonicalName()),
+        "Should not contain ProportionalCapacityPreemptionPolicy.");
   }
 
   @Test
@@ -150,10 +140,9 @@ public class TestFSYarnSiteConverter {
     converter.convertSiteProperties(yarnConfig, yarnConvertedConfig, false,
         false, false, null);
 
-    assertTrue("Assign multiple",
-        yarnConvertedConfig.getBoolean(
-            CapacitySchedulerConfiguration.ASSIGN_MULTIPLE_ENABLED,
-            false));
+    assertTrue(yarnConvertedConfig.getBoolean(
+        CapacitySchedulerConfiguration.ASSIGN_MULTIPLE_ENABLED, false),
+        "Assign multiple");
   }
 
   @Test
@@ -163,9 +152,9 @@ public class TestFSYarnSiteConverter {
     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
@@ -178,12 +167,12 @@ public class TestFSYarnSiteConverter {
     converter.convertSiteProperties(yarnConfig, yarnConvertedConfig, false,
         false, false, null);
 
-    assertEquals("Locality threshold node", "123.123",
-        yarnConvertedConfig.get(
-            CapacitySchedulerConfiguration.NODE_LOCALITY_DELAY));
-    assertEquals("Locality threshold rack", "321.321",
-        yarnConvertedConfig.get(
-            CapacitySchedulerConfiguration.RACK_LOCALITY_ADDITIONAL_DELAY));
+    assertEquals("123.123",
+        yarnConvertedConfig.get(CapacitySchedulerConfiguration.NODE_LOCALITY_DELAY),
+        "Locality threshold node");
+    assertEquals("321.321", yarnConvertedConfig.get(
+        CapacitySchedulerConfiguration.RACK_LOCALITY_ADDITIONAL_DELAY),
+        "Locality threshold rack");
   }
 
   @Test
@@ -191,9 +180,9 @@ public class TestFSYarnSiteConverter {
     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 +190,11 @@ public class TestFSYarnSiteConverter {
     converter.convertSiteProperties(yarnConfig, yarnConvertedConfig, false,
         false, false, null);
 
-    assertEquals("Resource calculator type", DefaultResourceCalculator.class,
+    assertEquals(DefaultResourceCalculator.class,
         yarnConvertedConfig.getClass(
-            CapacitySchedulerConfiguration.RESOURCE_CALCULATOR_CLASS,
-            CapacitySchedulerConfiguration.DEFAULT_RESOURCE_CALCULATOR_CLASS));
+        CapacitySchedulerConfiguration.RESOURCE_CALCULATOR_CLASS,
+        CapacitySchedulerConfiguration.DEFAULT_RESOURCE_CALCULATOR_CLASS),
+        "Resource calculator type");
   }
 
   @Test
@@ -212,9 +202,10 @@ public class TestFSYarnSiteConverter {
     converter.convertSiteProperties(yarnConfig, yarnConvertedConfig, true,
             true, false, null);
 
-    assertTrue("Asynchronous scheduling", yarnConvertedConfig.getBoolean(
-                    CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_ENABLE,
-            CapacitySchedulerConfiguration.DEFAULT_SCHEDULE_ASYNCHRONOUSLY_ENABLE));
+    assertTrue(yarnConvertedConfig.getBoolean(
+        CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_ENABLE,
+        CapacitySchedulerConfiguration.DEFAULT_SCHEDULE_ASYNCHRONOUSLY_ENABLE),
+        "Asynchronous scheduling");
   }
 
   @Test
@@ -222,21 +213,19 @@ public class TestFSYarnSiteConverter {
     converter.convertSiteProperties(yarnConfig, yarnConvertedConfig, false,
             false, false, null);
 
-    assertFalse("Asynchronous scheduling", yarnConvertedConfig.getBoolean(
-            CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_ENABLE,
-            false));
+    assertFalse(yarnConvertedConfig.getBoolean(
+        CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_ENABLE,
+        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.
-            get(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES)
-            .contains(DELETION_POLICY_CLASS));
+    assertTrue(true, yarnConvertedConfig.get(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS));
+    assertTrue(yarnConvertedConfig.get(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES)
+        .contains(DELETION_POLICY_CLASS),
+        "Scheduling Policies contain auto deletion policy");
 
     // Test when policy has existed.
     yarnConvertedConfig.
@@ -244,17 +233,15 @@ public class TestFSYarnSiteConverter {
         "testPolicy");
     converter.convertSiteProperties(yarnConfig, yarnConvertedConfig, false,
         false, false, null);
-    assertTrue("Scheduling Policies contain auto deletion policy",
-        yarnConvertedConfig.
-            get(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES)
-            .contains(DELETION_POLICY_CLASS));
-
-    assertEquals("Auto deletion policy expired time should be 10s",
-        10, yarnConvertedConfig.
-            getLong(CapacitySchedulerConfiguration.
-                    AUTO_CREATE_CHILD_QUEUE_EXPIRED_TIME,
-                CapacitySchedulerConfiguration.
-                    DEFAULT_AUTO_CREATE_CHILD_QUEUE_EXPIRED_TIME));
+    assertTrue(yarnConvertedConfig.get(
+        YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES).contains(
+        DELETION_POLICY_CLASS),
+        "Scheduling Policies contain auto deletion policy");
+
+    assertEquals(10, yarnConvertedConfig.
+        getLong(CapacitySchedulerConfiguration.AUTO_CREATE_CHILD_QUEUE_EXPIRED_TIME,
+        CapacitySchedulerConfiguration.DEFAULT_AUTO_CREATE_CHILD_QUEUE_EXPIRED_TIME),
+        "Auto deletion policy expired time should be 10s");
   }
 
   @Test
@@ -264,35 +251,28 @@ public class TestFSYarnSiteConverter {
     // 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.
-            get(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES) == null ||
-            !yarnConvertedConfig.
-            get(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES).
-                contains(DELETION_POLICY_CLASS));
+    assertTrue(yarnConvertedConfig.
+        get(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES) == null ||
+        !yarnConvertedConfig.get(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES).
+        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.
-            get(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES)
-            .contains(DELETION_POLICY_CLASS));
-
-    assertNotEquals("Auto deletion policy expired time should not " +
-            "be set in percentage mode",
-        10, yarnConvertedConfig.
-            getLong(CapacitySchedulerConfiguration.
-                    AUTO_CREATE_CHILD_QUEUE_EXPIRED_TIME,
-                CapacitySchedulerConfiguration.
-                    DEFAULT_AUTO_CREATE_CHILD_QUEUE_EXPIRED_TIME));
+    assertFalse(yarnConvertedConfig.
+        get(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES).contains(
+        DELETION_POLICY_CLASS), "Scheduling Policies should not " +
+        "contain auto deletion policy in percentage mode");
+
+    assertNotEquals(10, yarnConvertedConfig.
+        getLong(CapacitySchedulerConfiguration.AUTO_CREATE_CHILD_QUEUE_EXPIRED_TIME,
+        CapacitySchedulerConfiguration.DEFAULT_AUTO_CREATE_CHILD_QUEUE_EXPIRED_TIME),
+        "Auto deletion policy expired time should not be set in percentage mode");
 
   }
 }

+ 43 - 39
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,8 @@
 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.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
@@ -47,17 +48,17 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.placemen
 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.runner.RunWith;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
 import org.mockito.Mock;
-import org.mockito.junit.MockitoJUnitRunner;
+import org.mockito.junit.jupiter.MockitoExtension;
 
 /**
  * Unit tests for QueuePlacementConverter.
  *
  */
-@RunWith(MockitoJUnitRunner.class)
+@ExtendWith(MockitoExtension.class)
 public class TestQueuePlacementConverter {
   private static final String DEFAULT_QUEUE = "root.default";
 
@@ -71,7 +72,7 @@ public class TestQueuePlacementConverter {
 
   private CapacitySchedulerConfiguration csConf;
 
-  @Before
+  @BeforeEach
   public void setup() {
     this.converter = new QueuePlacementConverter();
     this.csConf = new CapacitySchedulerConfiguration(
@@ -84,7 +85,7 @@ public class TestQueuePlacementConverter {
     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 +96,7 @@ public class TestQueuePlacementConverter {
     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 +108,7 @@ public class TestQueuePlacementConverter {
 
     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 +120,7 @@ public class TestQueuePlacementConverter {
 
     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 +133,7 @@ public class TestQueuePlacementConverter {
 
     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 +147,20 @@ public class TestQueuePlacementConverter {
 
     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 +170,7 @@ public class TestQueuePlacementConverter {
 
     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 +184,7 @@ public class TestQueuePlacementConverter {
 
     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 +199,7 @@ public class TestQueuePlacementConverter {
 
     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 +215,25 @@ public class TestQueuePlacementConverter {
 
     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());
+    assertEquals("root.abc", rule.getParentQueue(), "Parent path");
     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 +247,7 @@ public class TestQueuePlacementConverter {
 
     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 +362,7 @@ public class TestQueuePlacementConverter {
     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(
@@ -410,7 +415,7 @@ public class TestQueuePlacementConverter {
     MappingRulesDescription desc = convertInWeightMode();
     Rule rule = desc.getRules().get(0);
 
-    assertEquals("Parent queue", "root", rule.getParentQueue());
+    assertEquals("root", rule.getParentQueue(), "Parent queue");
   }
 
   @Test
@@ -493,11 +498,10 @@ public class TestQueuePlacementConverter {
   }
 
   private void verifyRule(Rule rule, Policy expectedPolicy) {
-    assertEquals("Policy type", expectedPolicy, rule.getPolicy());
-    assertEquals("Match string", "*", rule.getMatches());
-    assertEquals("Fallback result",
-        FallbackResult.SKIP, rule.getFallbackResult());
-    assertEquals("Type", Type.USER, rule.getType());
+    assertEquals(expectedPolicy, rule.getPolicy(), "Policy type");
+    assertEquals("*", rule.getMatches(), "Match string");
+    assertEquals(FallbackResult.SKIP, rule.getFallbackResult(), "Fallback result");
+    assertEquals(Type.USER, rule.getType(), "Type");
   }
 
   private class TestPlacementRule extends FSPlacementRule {

+ 46 - 55
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.conf.Configuration;
 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 class TestWeightToPercentageConverter
     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,7 @@ public class TestWeightToPercentageConverter
     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 +75,14 @@ public class TestWeightToPercentageConverter
 
     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 +91,14 @@ public class TestWeightToPercentageConverter
 
     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 +107,14 @@ public class TestWeightToPercentageConverter
 
     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 +123,14 @@ public class TestWeightToPercentageConverter
 
     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 +161,9 @@ public class TestWeightToPercentageConverter
         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");
   }
 }

+ 18 - 31
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.conf.Configuration;
 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,8 @@ public class TestWeightToWeightConverter extends WeightConverterTestBase {
     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 +61,9 @@ public class TestWeightToWeightConverter extends WeightConverterTestBase {
     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 +72,11 @@ public class TestWeightToWeightConverter extends WeightConverterTestBase {
 
     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 +84,7 @@ public class TestWeightToWeightConverter extends WeightConverterTestBase {
     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 +92,7 @@ public class TestWeightToWeightConverter extends WeightConverterTestBase {
     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");
   }
 }

+ 92 - 91
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,10 @@
  */
 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.assertArrayEquals;
+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 +42,15 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.Dom
 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.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 class TestDominantResourceFairnessPolicy {
     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 class TestDominantResourceFairnessPolicy {
     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 class TestDominantResourceFairnessPolicy {
     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 class TestDominantResourceFairnessPolicy {
     // 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 class TestDominantResourceFairnessPolicy {
     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);
+      assertArrayEquals(
+          expected1[i], ratios1[i], 0.00001f, "The shares array was not sorted into the "
+          + "expected order: incorrect inner array encountered");
+      assertArrayEquals(
+          expected2[i], ratios2[i], 0.00001f, "The shares array was not sorted into the "
+          + "expected order: incorrect inner array encountered");
     }
   }
 
@@ -232,17 +232,16 @@ public class TestDominantResourceFairnessPolicy {
     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 +256,14 @@ public class TestDominantResourceFairnessPolicy {
         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 class TestDominantResourceFairnessPolicy {
 
     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 class TestDominantResourceFairnessPolicy {
         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,94 @@ public class TestDominantResourceFairnessPolicy {
 
     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

+ 10 - 6
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 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies;
 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 @@ import java.util.Collection;
 public class TestEmptyQueues {
   private Collection<? extends Schedulable> schedulables;
 
-  @Before
+  @BeforeEach
   public void setup() {
     schedulables = new ArrayList<Schedulable>();
   }
@@ -39,17 +40,20 @@ public class TestEmptyQueues {
     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));

+ 109 - 93
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,10 +20,13 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo;
 
 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.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
@@ -114,11 +117,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManag
 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.mockito.Mockito;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 public class TestFifoScheduler {
   private static final Logger LOG =
@@ -133,7 +135,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 +143,7 @@ public class TestFifoScheduler {
     resourceManager = new MockRM(conf);
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     resourceManager.stop();
   }
@@ -179,14 +181,16 @@ public class TestFifoScheduler {
     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);
+    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 +227,12 @@ public class TestFifoScheduler {
     scheduler.handle(attemptEvent2);
 
     int afterAppsSubmitted = metrics.getAppsSubmitted();
-    Assert.assertEquals(1, afterAppsSubmitted - beforeAppsSubmitted);
+    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 +299,21 @@ public class TestFifoScheduler {
     NodeUpdateSchedulerEvent node0Update = new NodeUpdateSchedulerEvent(node0);
 
     // Before the node update event, there are 3 local requests outstanding
-    Assert.assertEquals(3, nodeLocal.getNumContainers());
+    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());
+    assertEquals(0, nodeLocal.getNumContainers());
     //Also check that the containers were scheduled
     SchedulerAppReport info = scheduler.getSchedulerAppInfo(appAttemptId);
-    Assert.assertEquals(3, info.getLiveContainers().size());
+    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 +362,7 @@ public class TestFifoScheduler {
     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);
+    assertEquals(0.0f, queueInfo.getCurrentCapacity(), 0.0f);
     
     int _appId = 1;
     int _appAttemptId = 1;
@@ -389,20 +395,20 @@ public class TestFifoScheduler {
         null, null, NULL_UPDATE_REQUESTS);
 
     // Before the node update event, there are one local request
-    Assert.assertEquals(1, nodeLocal.getNumContainers());
+    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());
+    assertEquals(0, nodeLocal.getNumContainers());
     // Also check that one container was scheduled
     SchedulerAppReport info = scheduler.getSchedulerAppInfo(appAttemptId);
-    Assert.assertEquals(1, info.getLiveContainers().size());
+    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);
+    assertEquals(1.0f, queueInfo.getCurrentCapacity(), 0.0f);
   }
   
 //  @Test
@@ -627,7 +633,7 @@ public class TestFifoScheduler {
     assertTrue(appsInDefault.contains(application_1.getApplicationAttemptId()));
     assertEquals(2, appsInDefault.size());
     
-    Assert.assertNull(scheduler.getAppsInQueue("someotherqueue"));
+    assertNull(scheduler.getAppsInQueue("someotherqueue"));
   }
 
   @Test
@@ -644,7 +650,8 @@ public class TestFifoScheduler {
       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 +663,13 @@ public class TestFifoScheduler {
           + " 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 +695,14 @@ public class TestFifoScheduler {
       // kick the schedule
       nm1.nodeHeartbeat(true);
     } catch (NullPointerException e) {
-      Assert.fail("NPE when allocating container on node but "
+      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 +718,7 @@ public class TestFifoScheduler {
     am1.registerAppAttempt();
     SchedulerNodeReport report_nm1 =
         rm.getResourceScheduler().getNodeReport(nm1.getNodeId());
-    Assert.assertEquals(2 * GB, report_nm1.getUsedResource().getMemorySize());
+    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 +728,7 @@ public class TestFifoScheduler {
     am2.registerAppAttempt();
     SchedulerNodeReport report_nm2 =
         rm.getResourceScheduler().getNodeReport(nm2.getNodeId());
-    Assert.assertEquals(2 * GB, report_nm2.getUsedResource().getMemorySize());
+    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 +753,25 @@ public class TestFifoScheduler {
     nm2.nodeHeartbeat(true);
 
     List<Container> 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());
+    assertEquals(1, allocated1.size());
+    assertEquals(1 * GB, allocated1.get(0).getResource().getMemorySize());
+    assertEquals(nm1.getNodeId(), allocated1.get(0).getNodeId());
 
     List<Container> 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());
+    assertEquals(1, allocated2.size());
+    assertEquals(3 * GB, allocated2.get(0).getResource().getMemorySize());
+    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());
+    assertEquals(0, report_nm1.getAvailableResource().getMemorySize());
+    assertEquals(2 * GB, report_nm2.getAvailableResource().getMemorySize());
 
-    Assert.assertEquals(6 * GB, report_nm1.getUsedResource().getMemorySize());
-    Assert.assertEquals(2 * GB, report_nm2.getUsedResource().getMemorySize());
+    assertEquals(6 * GB, report_nm1.getUsedResource().getMemorySize());
+    assertEquals(2 * GB, report_nm2.getUsedResource().getMemorySize());
 
     Container c1 = allocated1.get(0);
-    Assert.assertEquals(GB, c1.getResource().getMemorySize());
+    assertEquals(GB, c1.getResource().getMemorySize());
     ContainerStatus containerStatus =
         BuilderUtils.newContainerStatus(c1.getId(), ContainerState.COMPLETE,
             "", 0, c1.getResource());
@@ -773,16 +782,17 @@ public class TestFifoScheduler {
           + waitCount + " times already..");
       Thread.sleep(1000);
     }
-    Assert.assertEquals(1, attempt1.getJustFinishedContainers().size());
-    Assert.assertEquals(1, am1.schedule().getCompletedContainersStatuses()
+    assertEquals(1, attempt1.getJustFinishedContainers().size());
+    assertEquals(1, am1.schedule().getCompletedContainersStatuses()
         .size());
     report_nm1 = rm.getResourceScheduler().getNodeReport(nm1.getNodeId());
-    Assert.assertEquals(5 * GB, report_nm1.getUsedResource().getMemorySize());
+    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 +813,7 @@ public class TestFifoScheduler {
     try {
       scheduler.handle(updateEvent);
     } catch (NullPointerException e) {
-      Assert.fail();
+      fail();
     }
 
     ApplicationAttemptId attId = ApplicationAttemptId.newInstance(appId, 1);
@@ -834,19 +844,21 @@ public class TestFifoScheduler {
     int checkAlloc =
         conf.getInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
             YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB);
-    Assert.assertEquals(checkAlloc, report_nm1.getUsedResource().getMemorySize());
+    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 +870,8 @@ public class TestFifoScheduler {
     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 +893,7 @@ public class TestFifoScheduler {
     fs.handle(new NodeAddedSchedulerEvent(n1));
     fs.handle(new NodeAddedSchedulerEvent(n2));
     fs.handle(new NodeUpdateSchedulerEvent(n1));
-    Assert.assertEquals(6 * GB, fs.getRootQueueMetrics().getAvailableMB());
+    assertEquals(6 * GB, fs.getRootQueueMetrics().getAvailableMB());
 
     // reconnect n1 with downgraded memory
     n1 =
@@ -889,11 +902,12 @@ public class TestFifoScheduler {
     fs.handle(new NodeAddedSchedulerEvent(n1));
     fs.handle(new NodeUpdateSchedulerEvent(n1));
 
-    Assert.assertEquals(4 * GB, fs.getRootQueueMetrics().getAvailableMB());
+    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 +988,18 @@ public class TestFifoScheduler {
         fs.allocate(appAttemptId1, emptyAsk, null, emptyId,
             null, null, NULL_UPDATE_REQUESTS);
 
-    Assert.assertEquals("allocation1", 0, allocation1.getContainers().size());
+    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());
+    assertEquals(1, allocation2.getContainers().size(), "allocation2");
     List<Container> containerList = allocation2.getContainers();
     for (Container container : containerList) {
-      Assert.assertEquals("Container is allocated on n4",
-          container.getNodeId(), n4.getNodeID());
+      assertEquals(container.getNodeId(), n4.getNodeID(),
+          "Container is allocated on n4");
     }
 
     // Ask for a 1 GB container again for app 1
@@ -1002,38 +1016,39 @@ public class TestFifoScheduler {
     Allocation allocation3 =
         fs.allocate(appAttemptId1, emptyAsk, null, emptyId,
             null, null, NULL_UPDATE_REQUESTS);
-    Assert.assertEquals("allocation3", 0, allocation3.getContainers().size());
+    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());
+    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());
+    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());
+    assertEquals(1, allocation6.getContainers().size(), "allocation6");
 
     containerList = allocation6.getContainers();
     for (Container container : containerList) {
-      Assert.assertEquals("Container is allocated on n4",
-          container.getNodeId(), n4.getNodeID());
+      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 +1112,20 @@ public class TestFifoScheduler {
     Allocation allocation1 =
         fs.allocate(appAttemptId1, emptyAsk, null, emptyId,
             null, null, NULL_UPDATE_REQUESTS);
-    Assert.assertEquals("Allocation headroom", 1 * GB, allocation1
-        .getResourceLimit().getMemorySize());
+    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());
+    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 +1142,8 @@ public class TestFifoScheduler {
     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());
+    assertEquals(2 * GB, report_nm1.getUsedResource().getMemorySize());
+    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 +1158,18 @@ public class TestFifoScheduler {
     }
 
     List<Container> 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());
+    assertEquals(1, allocated1.size());
+    assertEquals(2 * GB, allocated1.get(0).getResource().getMemorySize());
+    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());
+    assertEquals(0, report_nm1.getAvailableResource().getMemorySize());
+    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());
+    assertEquals(2 * GB, c1.getResource().getMemorySize());
 
     // update node resource to 2 GB, so resource is over-consumed.
     Map<NodeId, ResourceOption> nodeResourceMap =
@@ -1178,8 +1194,8 @@ public class TestFifoScheduler {
     // 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());
+    assertEquals(4 * GB, report_nm1.getUsedResource().getMemorySize());
+    assertEquals(-2 * GB, report_nm1.getAvailableResource().getMemorySize());
 
     // Check container can complete successfully in case of resource
     // over-commitment.
@@ -1193,13 +1209,13 @@ public class TestFifoScheduler {
           + waitCount + " times already..");
       Thread.sleep(100);
     }
-    Assert.assertEquals(1, attempt1.getJustFinishedContainers().size());
-    Assert.assertEquals(1, am1.schedule().getCompletedContainersStatuses()
+    assertEquals(1, attempt1.getJustFinishedContainers().size());
+    assertEquals(1, am1.schedule().getCompletedContainersStatuses()
         .size());
     report_nm1 = rm.getResourceScheduler().getNodeReport(nm1.getNodeId());
-    Assert.assertEquals(2 * GB, report_nm1.getUsedResource().getMemorySize());
+    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());
+    assertEquals(0 * GB, report_nm1.getAvailableResource().getMemorySize());
     rm.stop();
   }
 
@@ -1223,7 +1239,7 @@ public class TestFifoScheduler {
         .getNodeTracker().removeNode(nm.getNodeId());
     // Kick off another heartbeat with the node state mocked to decommissioning
     RMNode spyNode =
-        Mockito.spy(resourceManager.getRMContext().getRMNodes()
+        spy(resourceManager.getRMContext().getRMNodes()
             .get(nm.getNodeId()));
     when(spyNode.getState()).thenReturn(NodeState.DECOMMISSIONING);
     resourceManager.getResourceScheduler().handle(
@@ -1234,7 +1250,7 @@ public class TestFifoScheduler {
   public void testResourceUpdateDecommissioningNode() throws Exception {
     // Mock the RMNodeResourceUpdate event handler to update SchedulerNode
     // to have 0 available resource
-    RMContext spyContext = Mockito.spy(resourceManager.getRMContext());
+    RMContext spyContext = spy(resourceManager.getRMContext());
     Dispatcher mockDispatcher = mock(AsyncDispatcher.class);
     when(mockDispatcher.getEventHandler()).thenReturn(new EventHandler<Event>() {
       @Override
@@ -1249,7 +1265,7 @@ public class TestFifoScheduler {
         }
       }
     });
-    Mockito.doReturn(mockDispatcher).when(spyContext).getDispatcher();
+    doReturn(mockDispatcher).when(spyContext).getDispatcher();
     ((FifoScheduler) resourceManager.getResourceScheduler())
         .setRMContext(spyContext);
     ((AsyncDispatcher) mockDispatcher).start();
@@ -1290,7 +1306,7 @@ public class TestFifoScheduler {
     // Kick off another heartbeat with the node state mocked to decommissioning
     // This should update the schedulernodes to have 0 available resource
     RMNode spyNode =
-        Mockito.spy(resourceManager.getRMContext().getRMNodes()
+        spy(resourceManager.getRMContext().getRMNodes()
             .get(nm_0.getNodeId()));
     when(spyNode.getState()).thenReturn(NodeState.DECOMMISSIONING);
     resourceManager.getResourceScheduler().handle(
@@ -1300,7 +1316,7 @@ public class TestFifoScheduler {
     application_0.schedule();
 
     // Check the used resource is 1 GB 1 core
-    // Assert.assertEquals(1 * GB, nm_0.getUsed().getMemory());
+    // assertEquals(1 * GB, nm_0.getUsed().getMemory());
     Resource usedResource =
         resourceManager.getResourceScheduler()
             .getSchedulerNode(nm_0.getNodeId()).getAllocatedResource();
@@ -1329,12 +1345,12 @@ public class TestFifoScheduler {
 
   private void checkApplicationResourceUsage(int expected, 
       Application application) {
-    Assert.assertEquals(expected, application.getUsedResources().getMemorySize());
+    assertEquals(expected, application.getUsedResources().getMemorySize());
   }
   
   private void checkNodeResourceUsage(int expected,
       org.apache.hadoop.yarn.server.resourcemanager.NodeManager node) {
-    Assert.assertEquals(expected, node.getUsed().getMemorySize());
+    assertEquals(expected, node.getUsed().getMemorySize());
     node.checkResourceUsage();
   }
 

+ 29 - 34
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,19 +33,23 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.Invali
 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.mockito.Mockito;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 import java.util.HashSet;
 import java.util.Set;
 import java.util.function.LongBinaryOperator;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.atLeast;
+import static org.mockito.Mockito.eq;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
@@ -59,7 +63,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,8 +112,7 @@ public class TestSingleConstraintAppPlacementAllocator {
       // Expected
       return;
     }
-    Assert.fail(
-        "Expect failure for schedulingRequest=" + schedulingRequest.toString());
+    fail("Expect failure for schedulingRequest=" + schedulingRequest.toString());
   }
 
   @Test
@@ -126,7 +129,7 @@ public class TestSingleConstraintAppPlacementAllocator {
             .build()).resourceSizing(
             ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
         .build());
-    Assert.assertEquals("", allocator.getTargetNodePartition());
+    assertEquals("", allocator.getTargetNodePartition());
 
     // Valid (with partition)
     assertValidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
@@ -140,7 +143,7 @@ public class TestSingleConstraintAppPlacementAllocator {
             .build()).resourceSizing(
             ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
         .build());
-    Assert.assertEquals("x", allocator.getTargetNodePartition());
+    assertEquals("x", allocator.getTargetNodePartition());
 
     // Valid (without specifying node partition)
     assertValidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
@@ -154,7 +157,7 @@ public class TestSingleConstraintAppPlacementAllocator {
             ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
         .build());
     // Node partition is unspecified, use the default node label expression y
-    Assert.assertEquals("y", allocator.getTargetNodePartition());
+    assertEquals("y", allocator.getTargetNodePartition());
 
     // Valid (with application Id target)
     assertValidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
@@ -168,7 +171,7 @@ public class TestSingleConstraintAppPlacementAllocator {
             ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
         .build());
     // Allocation tags should not include application Id
-    Assert.assertEquals("y", allocator.getTargetNodePartition());
+    assertEquals("y", allocator.getTargetNodePartition());
 
     // Invalid (without sizing)
     assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
@@ -279,7 +282,7 @@ public class TestSingleConstraintAppPlacementAllocator {
             ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
         .build();
     allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest, true);
-    Assert.assertEquals(existingNumAllocations + 1,
+    assertEquals(existingNumAllocations + 1,
         allocator.getSchedulingRequest().getResourceSizing()
             .getNumAllocations());
   }
@@ -301,7 +304,7 @@ public class TestSingleConstraintAppPlacementAllocator {
     allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest, false);
     allocator.canAllocate(NodeType.NODE_LOCAL,
         TestUtils.getMockNode("host1", "/rack1", 123, 1024));
-    verify(spyAllocationTagsManager, Mockito.times(1)).getNodeCardinalityByOp(
+    verify(spyAllocationTagsManager, times(1)).getNodeCardinalityByOp(
         eq(NodeId.fromString("host1:123")), any(AllocationTags.class),
         any(LongBinaryOperator.class));
 
@@ -322,7 +325,7 @@ public class TestSingleConstraintAppPlacementAllocator {
     allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest, false);
     allocator.canAllocate(NodeType.NODE_LOCAL,
         TestUtils.getMockNode("host1", "/rack1", 123, 1024));
-    verify(spyAllocationTagsManager, Mockito.atLeast(1)).getNodeCardinalityByOp(
+    verify(spyAllocationTagsManager, atLeast(1)).getNodeCardinalityByOp(
         eq(NodeId.fromString("host1:123")), any(AllocationTags.class),
         any(LongBinaryOperator.class));
 
@@ -330,13 +333,13 @@ public class TestSingleConstraintAppPlacementAllocator {
     when(node1.getPartition()).thenReturn("x");
     when(node1.getNodeID()).thenReturn(NodeId.fromString("host1:123"));
 
-    Assert.assertTrue(allocator
+    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
+    assertFalse(allocator
         .precheckNode(node2, SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY));
   }
 
@@ -363,7 +366,7 @@ public class TestSingleConstraintAppPlacementAllocator {
     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);
+    assertTrue(result, "Allocation should be success for java=1.8");
 
     // 2. verify python!=3 validation
     SchedulingRequest schedulingRequest2 =
@@ -387,8 +390,7 @@ public class TestSingleConstraintAppPlacementAllocator {
     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);
+    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 +402,7 @@ public class TestSingleConstraintAppPlacementAllocator {
     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);
+    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,7 @@ public class TestSingleConstraintAppPlacementAllocator {
     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);
+    assertFalse(result, "Allocation should fail as python=3 exist in node");
   }
 
   @Test
@@ -451,8 +450,7 @@ public class TestSingleConstraintAppPlacementAllocator {
     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);
+    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 +465,7 @@ public class TestSingleConstraintAppPlacementAllocator {
     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);
+    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 +498,7 @@ public class TestSingleConstraintAppPlacementAllocator {
     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);
+    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 +513,7 @@ public class TestSingleConstraintAppPlacementAllocator {
     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);
+    assertFalse(result,
+        "Allocation should fail as java=1.8 doesnt exist in node");
   }
 }

+ 28 - 33
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,14 +33,15 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 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.Test;
 
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
 public class TestFairOrderingPolicy {
 
   final static int GB = 1024;
@@ -52,14 +53,13 @@ public class TestFairOrderingPolicy {
     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);
+    assertTrue(policy.getComparator().compare(r1, r2) > 0);
   }
 
   @Test
@@ -71,8 +71,7 @@ public class TestFairOrderingPolicy {
     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 +85,7 @@ public class TestFairOrderingPolicy {
       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 +95,13 @@ public class TestFairOrderingPolicy {
 
     //More demand and consumption, but not enough more demand to overcome
     //additional consumption
-    Assert.assertTrue(policy.getComparator().compare(r1, r2) < 0);
+    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);
+    assertTrue(policy.getComparator().compare(r1, r2) > 0);
   }
 
   @Test
@@ -183,8 +181,8 @@ public class TestFairOrderingPolicy {
     // Get LeafQueue
     LeafQueue lq = (LeafQueue) cs.getQueue("default");
     OrderingPolicy<FiCaSchedulerApp> policy = lq.getOrderingPolicy();
-    Assert.assertTrue(policy instanceof FairOrderingPolicy);
-    Assert.assertTrue(((FairOrderingPolicy<FiCaSchedulerApp>)policy).getSizeBasedWeight());
+    assertTrue(policy instanceof FairOrderingPolicy);
+    assertTrue(((FairOrderingPolicy<FiCaSchedulerApp>)policy).getSizeBasedWeight());
 
     rm.registerNode("h1:1234", 10 * GB);
 
@@ -226,21 +224,20 @@ public class TestFairOrderingPolicy {
             .build();
     MockRMAppSubmitter.submit(rm, data);
 
-    Assert.assertEquals(1, lq.getNumActiveApplications());
-    Assert.assertEquals(3, lq.getNumPendingApplications());
+    assertEquals(1, lq.getNumActiveApplications());
+    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());
+    assertEquals(1, lq.getNumActiveApplications());
+    assertEquals(3, lq.getNumPendingApplications());
   }
 
   public void checkIds(Iterator<MockSchedulableEntity> si,
       String[] ids) {
     for (int i = 0;i < ids.length;i++) {
-      Assert.assertEquals(si.next().getId(),
-        ids[i]);
+      assertEquals(si.next().getId(), ids[i]);
     }
   }
 
@@ -264,8 +261,8 @@ public class TestFairOrderingPolicy {
         .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 +275,7 @@ public class TestFairOrderingPolicy {
     AbstractComparatorOrderingPolicy
         .updateSchedulingResourceUsage(r2.getSchedulingResourceUsage());
 
-    Assert.assertTrue(policy.getComparator().compare(r1, r2) < 0);
+    assertTrue(policy.getComparator().compare(r1, r2) < 0);
   }
 
   @Test
@@ -293,20 +290,19 @@ public class TestFairOrderingPolicy {
     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);
+    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);
+    assertTrue(policy.getComparator().compare(r1, r2) > 0);
   }
 
   @Test
@@ -325,8 +321,7 @@ public class TestFairOrderingPolicy {
         .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 +332,7 @@ public class TestFairOrderingPolicy {
     AbstractComparatorOrderingPolicy
         .updateSchedulingResourceUsage(r2.getSchedulingResourceUsage());
 
-    Assert.assertTrue(policy.getComparator().compare(r1, r2) > 0);
+    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 +342,8 @@ public class TestFairOrderingPolicy {
     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 +383,6 @@ public class TestFairOrderingPolicy {
       policy.removeSchedulableEntity(entities.get(i));
     }
 
-    Assert.assertEquals(5, policy.getNumSchedulableEntities());
+    assertEquals(5, policy.getNumSchedulableEntities());
   }
 }

+ 30 - 29
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,15 @@
 
 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.Test;
 
 public
 class TestFifoOrderingPolicy {
@@ -39,17 +38,19 @@ class 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));
+    assertEquals(-1, policy.getComparator().compare(r1, r2),
+        "The rhs entity has a larger serial, the comparator return " +
+        "value should be -1.");
   }
   
   @Test
@@ -91,32 +92,32 @@ class TestFifoOrderingPolicy {
     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);
+    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);
+    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 +131,20 @@ class TestFifoOrderingPolicy {
     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);
+    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);
+    assertTrue(policy.getComparator().compare(r1, r2) > 0,
+        "r2 was started before r1, the comparator should return a positive value.");
   }
 }

+ 3 - 3
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,10 +20,10 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy;
 
 import java.util.*;
 
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 public class TestFifoOrderingPolicyForPendingApps {
 
@@ -86,7 +86,7 @@ public class TestFifoOrderingPolicyForPendingApps {
   public void checkSerials(Iterator<MockSchedulableEntity> si,
       long[] serials) {
     for (int i = 0; i < serials.length; i++) {
-      Assert.assertEquals(si.next().getSerial(), serials[i]);
+      assertEquals(si.next().getSerial(), serials[i]);
     }
   }
 }

+ 24 - 22
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

@@ -18,13 +18,15 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+
 import java.util.Arrays;
 import java.util.Collections;
 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.Test;
 
 
 /**
@@ -55,8 +57,8 @@ public class TestFifoOrderingPolicyWithExclusivePartitions {
 
     policy.addSchedulableEntity(p1);
     policy.addAllSchedulableEntities(Arrays.asList(p2, r1, r2));
-    Assert.assertEquals(4, policy.getNumSchedulableEntities());
-    Assert.assertEquals(4, policy.getSchedulableEntities().size());
+    assertEquals(4, policy.getNumSchedulableEntities());
+    assertEquals(4, policy.getSchedulableEntities().size());
     IteratorSelector sel = new IteratorSelector();
     // Should behave like FifoOrderingPolicy, regardless of partition
     verifyAssignmentIteratorOrder(policy,
@@ -68,8 +70,8 @@ public class TestFifoOrderingPolicyWithExclusivePartitions {
 
     policy.removeSchedulableEntity(p2);
     policy.removeSchedulableEntity(r2);
-    Assert.assertEquals(2, policy.getNumSchedulableEntities());
-    Assert.assertEquals(2, policy.getSchedulableEntities().size());
+    assertEquals(2, policy.getNumSchedulableEntities());
+    assertEquals(2, policy.getSchedulableEntities().size());
     verifyAssignmentIteratorOrder(policy,
         IteratorSelector.EMPTY_ITERATOR_SELECTOR, "r1", "p1");
     verifyPreemptionIteratorOrder(policy, "p1", "r1");
@@ -105,8 +107,8 @@ public class TestFifoOrderingPolicyWithExclusivePartitions {
     r3.setId("r3");
 
     policy.addSchedulableEntity(r1);
-    Assert.assertEquals(1, policy.getNumSchedulableEntities());
-    Assert.assertEquals("r1", policy.getSchedulableEntities()
+    assertEquals(1, policy.getNumSchedulableEntities());
+    assertEquals("r1", policy.getSchedulableEntities()
         .iterator().next().getId());
     verifyAssignmentIteratorOrder(policy,
         IteratorSelector.EMPTY_ITERATOR_SELECTOR, "r1");
@@ -115,8 +117,8 @@ public class TestFifoOrderingPolicyWithExclusivePartitions {
     List<MockSchedulableEntity> entities = Arrays.asList(r2, r3, p1, p2);
     policy.addAllSchedulableEntities(entities);
     policy.addSchedulableEntity(p3);
-    Assert.assertEquals(6, policy.getNumSchedulableEntities());
-    Assert.assertEquals(6, policy.getSchedulableEntities().size());
+    assertEquals(6, policy.getNumSchedulableEntities());
+    assertEquals(6, policy.getSchedulableEntities().size());
     // Assignment iterator should return non-PARTITION entities,
     // in order based on FifoOrderingPolicy
     verifyAssignmentIteratorOrder(policy,
@@ -135,8 +137,8 @@ public class TestFifoOrderingPolicyWithExclusivePartitions {
 
     policy.removeSchedulableEntity(p2);
     policy.removeSchedulableEntity(r2);
-    Assert.assertEquals(4, policy.getNumSchedulableEntities());
-    Assert.assertEquals(4, policy.getSchedulableEntities().size());
+    assertEquals(4, policy.getNumSchedulableEntities());
+    assertEquals(4, policy.getSchedulableEntities().size());
     verifyAssignmentIteratorOrder(policy,
         IteratorSelector.EMPTY_ITERATOR_SELECTOR, "r3", "r1");
     verifyPreemptionIteratorOrder(policy, "r1", "p3", "p1", "r3");
@@ -146,8 +148,8 @@ public class TestFifoOrderingPolicyWithExclusivePartitions {
 
     policy.removeSchedulableEntity(p1);
     policy.removeSchedulableEntity(p3);
-    Assert.assertEquals(2, policy.getNumSchedulableEntities());
-    Assert.assertEquals(2, policy.getSchedulableEntities().size());
+    assertEquals(2, policy.getNumSchedulableEntities());
+    assertEquals(2, policy.getSchedulableEntities().size());
     verifyAssignmentIteratorOrder(policy,
         IteratorSelector.EMPTY_ITERATOR_SELECTOR, "r3", "r1");
     verifyPreemptionIteratorOrder(policy, "r1", "r3");
@@ -187,8 +189,8 @@ public class TestFifoOrderingPolicyWithExclusivePartitions {
     s2.setId("s2");
 
     policy.addAllSchedulableEntities(Arrays.asList(s1, s2, r1));
-    Assert.assertEquals(3, policy.getNumSchedulableEntities());
-    Assert.assertEquals(3, policy.getSchedulableEntities().size());
+    assertEquals(3, policy.getNumSchedulableEntities());
+    assertEquals(3, policy.getSchedulableEntities().size());
     IteratorSelector sel = new IteratorSelector();
     // assignment iterator returns only default (non-partitioned) entities
     verifyAssignmentIteratorOrder(policy,
@@ -198,8 +200,8 @@ public class TestFifoOrderingPolicyWithExclusivePartitions {
     verifyAssignmentIteratorOrder(policy, sel, "r1");
 
     policy.addAllSchedulableEntities(Arrays.asList(r2, p1, p2));
-    Assert.assertEquals(6, policy.getNumSchedulableEntities());
-    Assert.assertEquals(6, policy.getSchedulableEntities().size());
+    assertEquals(6, policy.getNumSchedulableEntities());
+    assertEquals(6, policy.getSchedulableEntities().size());
     verifyAssignmentIteratorOrder(policy,
         IteratorSelector.EMPTY_ITERATOR_SELECTOR, "s2", "s1");
     sel.setPartition(PARTITION);
@@ -211,8 +213,8 @@ public class TestFifoOrderingPolicyWithExclusivePartitions {
     policy.removeSchedulableEntity(p2);
     policy.removeSchedulableEntity(r1);
     policy.removeSchedulableEntity(r2);
-    Assert.assertEquals(3, policy.getNumSchedulableEntities());
-    Assert.assertEquals(3, policy.getSchedulableEntities().size());
+    assertEquals(3, policy.getNumSchedulableEntities());
+    assertEquals(3, policy.getSchedulableEntities().size());
     verifyAssignmentIteratorOrder(policy,
         IteratorSelector.EMPTY_ITERATOR_SELECTOR, "s2", "s1");
     sel.setPartition(PARTITION);
@@ -237,8 +239,8 @@ public class TestFifoOrderingPolicyWithExclusivePartitions {
   private void verifyIteratorOrder(Iterator<MockSchedulableEntity> itr,
       String... ids) {
     for (String id : ids) {
-      Assert.assertEquals(id, itr.next().getId());
+      assertEquals(id, itr.next().getId());
     }
-    Assert.assertFalse(itr.hasNext());
+    assertFalse(itr.hasNext());
   }
 }

Деякі файли не було показано, через те що забагато файлів було змінено