浏览代码

YARN-11262. [JDK17] Upgrade JUnit from 4 to 5 in hadoop-yarn-server-resourcemanager Part2. (#7484)

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 月之前
父节点
当前提交
baf393d6c6
共有 53 个文件被更改,包括 2205 次插入1879 次删除
  1. 5 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/TestAppNameMappingPlacementRule.java
  2. 54 56
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/TestFairQueuePlacementUtils.java
  3. 14 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/TestPlacementFactory.java
  4. 12 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/TestPlacementManager.java
  5. 26 26
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/TestPlacementRuleFS.java
  6. 8 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/TestUserGroupMappingPlacementRule.java
  7. 40 36
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/csmappingrule/TestCSMappingPlacementRule.java
  8. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/csmappingrule/TestMappingRule.java
  9. 5 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/csmappingrule/TestMappingRuleActions.java
  10. 32 34
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/csmappingrule/TestMappingRuleMatchers.java
  11. 5 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/csmappingrule/TestMappingRuleValidationContextImpl.java
  12. 42 28
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/csmappingrule/TestVariableContext.java
  13. 4 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/preprocessor/TestContextProcessor.java
  14. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/preprocessor/TestNodeLabelProcessor.java
  15. 5 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/preprocessor/TestQueueProcessor.java
  16. 6 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/preprocessor/TestTagAddProcessor.java
  17. 70 69
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java
  18. 50 39
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestCheckRemoveZKNodeRMStateStore.java
  19. 28 18
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java
  20. 30 17
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestLeveldbRMStateStore.java
  21. 6 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestMemoryRMStateStore.java
  22. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestProtos.java
  23. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestRMStateStoreUtils.java
  24. 172 174
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java
  25. 5 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStorePerf.java
  26. 16 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStoreZKClientConnections.java
  27. 5 13
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/BaseSharingPolicyTest.java
  28. 14 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
  29. 20 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityOverTimePolicy.java
  30. 13 18
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacitySchedulerPlanFollower.java
  31. 11 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestFairSchedulerPlanFollower.java
  32. 104 98
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java
  33. 25 22
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryReservationAllocation.java
  34. 20 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestNoOverCommitPolicy.java
  35. 38 35
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.java
  36. 64 63
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
  37. 130 137
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationInputValidator.java
  38. 11 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationSystemUtil.java
  39. 24 24
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestSchedulerPlanFollowerBase.java
  40. 165 149
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestAlignedPlanner.java
  41. 154 133
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestGreedyReservationAgent.java
  42. 17 14
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestReservationAgents.java
  43. 31 32
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java
  44. 29 26
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resource/TestResourceProfiles.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/resourcetracker/TestNMExpiry.java
  46. 13 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestRMNMRPCResponseId.java
  47. 90 80
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestApplicationLifetimeMonitor.java
  48. 25 22
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestNodesListManager.java
  49. 200 139
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
  50. 8 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestAMLivelinessMonitor.java
  51. 13 18
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptImplDiagnostics.java
  52. 297 180
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
  53. 31 25
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/TestAppNameMappingPlacementRule.java

@@ -30,14 +30,14 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.SimpleGroupsMapping;
 import org.apache.hadoop.yarn.util.Records;
-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.io.IOException;
 import java.util.Collections;
 
 import static org.apache.hadoop.yarn.server.resourcemanager.placement.FairQueuePlacementUtils.DOT;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.mockito.ArgumentMatchers.isNull;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -55,7 +55,7 @@ public class TestAppNameMappingPlacementRule {
 
   private YarnConfiguration conf = new YarnConfiguration();
 
-  @Before
+  @BeforeEach
   public void setup() {
     conf.setClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
         SimpleGroupsMapping.class, GroupMappingServiceProvider.class);
@@ -108,7 +108,7 @@ public class TestAppNameMappingPlacementRule {
     asc.setApplicationName(appName);
     ApplicationPlacementContext ctx = engine.getPlacementForApp(asc,
         user);
-    Assert.assertEquals(expectedQueue,
+    assertEquals(expectedQueue,
         ctx != null ? ctx.getQueue() : inputQueue);
   }
 

+ 54 - 56
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/TestFairQueuePlacementUtils.java

@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.placement;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import static org.apache.hadoop.yarn.server.resourcemanager.placement.FairQueuePlacementUtils.DOT;
 import static org.apache.hadoop.yarn.server.resourcemanager.placement.FairQueuePlacementUtils.DOT_REPLACEMENT;
@@ -26,11 +26,11 @@ import static org.apache.hadoop.yarn.server.resourcemanager.placement.FairQueueP
 import static org.apache.hadoop.yarn.server.resourcemanager.placement.FairQueuePlacementUtils.assureRoot;
 import static org.apache.hadoop.yarn.server.resourcemanager.placement.FairQueuePlacementUtils.cleanName;
 import static org.apache.hadoop.yarn.server.resourcemanager.placement.FairQueuePlacementUtils.isValidQueueName;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotEquals;
-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.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 /**
  * Tests of the utility methods from {@link FairQueuePlacementUtils}.
@@ -50,27 +50,27 @@ public class TestFairQueuePlacementUtils {
     final String unTrimmed = " .invalid. "; // not really a valid queue
 
     String cleaned = cleanName(clean);
-    assertEquals("Name was changed and it should not", clean, cleaned);
+    assertEquals(clean, cleaned, "Name was changed and it should not");
     cleaned = cleanName(dotted);
-    assertFalse("Cleaned name contains dots and it should not",
-        cleaned.contains(DOT));
+    assertFalse(cleaned.contains(DOT),
+        "Cleaned name contains dots and it should not");
     cleaned = cleanName(multiDot);
-    assertFalse("Cleaned name contains dots and it should not",
-        cleaned.contains(DOT));
-    assertNotEquals("Multi dot failed: wrong replacements found",
-        cleaned.indexOf(DOT_REPLACEMENT),
-        cleaned.lastIndexOf(DOT_REPLACEMENT));
+    assertFalse(cleaned.contains(DOT),
+        "Cleaned name contains dots and it should not");
+    assertNotEquals(cleaned.indexOf(DOT_REPLACEMENT),
+        cleaned.lastIndexOf(DOT_REPLACEMENT),
+        "Multi dot failed: wrong replacements found");
     cleaned = cleanName(seqDot);
-    assertFalse("Cleaned name contains dots and it should not",
-        cleaned.contains(DOT));
-    assertNotEquals("Sequential dot failed: wrong replacements found",
-        cleaned.indexOf(DOT_REPLACEMENT),
-        cleaned.lastIndexOf(DOT_REPLACEMENT));
+    assertFalse(cleaned.contains(DOT),
+        "Cleaned name contains dots and it should not");
+    assertNotEquals(cleaned.indexOf(DOT_REPLACEMENT),
+        cleaned.lastIndexOf(DOT_REPLACEMENT),
+        "Sequential dot failed: wrong replacements found");
     cleaned = cleanName(unTrimmed);
-    assertTrue("Trimming start failed: space not removed or dot not replaced",
-        cleaned.startsWith(DOT_REPLACEMENT));
-    assertTrue("Trimming end failed: space not removed or dot not replaced",
-        cleaned.endsWith(DOT_REPLACEMENT));
+    assertTrue(cleaned.startsWith(DOT_REPLACEMENT),
+        "Trimming start failed: space not removed or dot not replaced");
+    assertTrue(cleaned.endsWith(DOT_REPLACEMENT),
+        "Trimming end failed: space not removed or dot not replaced");
   }
 
   @Test
@@ -82,23 +82,22 @@ public class TestFairQueuePlacementUtils {
     final String alreadyRoot = "root.base";
 
     String rooted = assureRoot(queueName);
-    assertTrue("Queue should have root prefix (base)",
-        rooted.startsWith(ROOT_QUEUE + DOT));
+    assertTrue(rooted.startsWith(ROOT_QUEUE + DOT),
+        "Queue should have root prefix (base)");
     rooted = assureRoot(rootOnly);
-    assertEquals("'root' queue should not have root prefix (root)",
-        rootOnly, rooted);
+    assertEquals(rootOnly, rooted,
+        "'root' queue should not have root prefix (root)");
     rooted = assureRoot(rootNoDot);
-    assertTrue("Queue should have root prefix (rootbase)",
-        rooted.startsWith(ROOT_QUEUE + DOT));
-    assertEquals("'root' queue base was replaced and not prefixed", 5,
-        rooted.lastIndexOf(ROOT_QUEUE));
+    assertTrue(rooted.startsWith(ROOT_QUEUE + DOT),
+        "Queue should have root prefix (rootbase)");
+    assertEquals(5, rooted.lastIndexOf(ROOT_QUEUE),
+        "'root' queue base was replaced and not prefixed");
     rooted = assureRoot(alreadyRoot);
-    assertEquals("Root prefixed queue changed and it should not (root.base)",
-        rooted, alreadyRoot);
-    assertNull("Null queue did not return null queue",
-        assureRoot(null));
-    assertEquals("Empty queue did not return empty name", "",
-        assureRoot(""));
+    assertEquals(rooted, alreadyRoot,
+        "Root prefixed queue changed and it should not (root.base)");
+    assertNull(assureRoot(null), "Null queue did not return null queue");
+    assertEquals("", assureRoot(""),
+        "Empty queue did not return empty name");
   }
 
   @Test
@@ -113,25 +112,24 @@ public class TestFairQueuePlacementUtils {
     final String endSpace = "invalid ";
     final String unicodeSpace = "\u00A0invalid";
 
-    assertFalse("'null' queue was not marked as invalid",
-        isValidQueueName(null));
-    assertTrue("empty queue was not tagged valid", isValidQueueName(""));
-    assertTrue("Simple queue name was not tagged valid (valid)",
-        isValidQueueName(valid));
-    assertTrue("Root only queue was not tagged valid (root)",
-        isValidQueueName(rootOnly));
-    assertTrue("Root prefixed queue was not tagged valid (root.valid)",
-        isValidQueueName(validRooted));
-    assertFalse("Queue starting with dot was not tagged invalid (.invalid)",
-        isValidQueueName(startDot));
-    assertFalse("Queue ending with dot was not tagged invalid (invalid.)",
-        isValidQueueName(endDot));
-    assertFalse("Queue starting with space was not tagged invalid ( invalid)",
-        isValidQueueName(startSpace));
-    assertFalse("Queue ending with space was not tagged invalid (invalid )",
-        isValidQueueName(endSpace));
+    assertFalse(isValidQueueName(null), "'null' queue was not marked as invalid");
+    assertTrue(isValidQueueName(""), "empty queue was not tagged valid");
+    assertTrue(isValidQueueName(valid),
+        "Simple queue name was not tagged valid (valid)");
+    assertTrue(isValidQueueName(rootOnly),
+        "Root only queue was not tagged valid (root)");
+    assertTrue(isValidQueueName(validRooted),
+        "Root prefixed queue was not tagged valid (root.valid)");
+    assertFalse(isValidQueueName(startDot),
+        "Queue starting with dot was not tagged invalid (.invalid)");
+    assertFalse(isValidQueueName(endDot),
+        "Queue ending with dot was not tagged invalid (invalid.)");
+    assertFalse(isValidQueueName(startSpace),
+        "Queue starting with space was not tagged invalid ( invalid)");
+    assertFalse(isValidQueueName(endSpace),
+        "Queue ending with space was not tagged invalid (invalid )");
     // just one for sanity check extensive tests are in the scheduler utils
-    assertFalse("Queue with unicode space was not tagged as invalid (unicode)",
-        isValidQueueName(unicodeSpace));
+    assertFalse(isValidQueueName(unicodeSpace),
+        "Queue with unicode space was not tagged as invalid (unicode)");
   }
 }

+ 14 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/TestPlacementFactory.java

@@ -18,11 +18,12 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.placement;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-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.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
 
 /**
  * Test for the {@link PlacementFactory}.
@@ -34,10 +35,12 @@ public class TestPlacementFactory {
    *
    * @throws ClassNotFoundException
    */
-  @Test(expected = ClassNotFoundException.class)
+  @Test
   public void testGetNonExistRuleText() throws ClassNotFoundException {
-    final String nonExist = "my.placement.Rule";
-    PlacementFactory.getPlacementRule(nonExist, null);
+    assertThrows(ClassNotFoundException.class, ()->{
+      final String nonExist = "my.placement.Rule";
+      PlacementFactory.getPlacementRule(nonExist, null);
+    });
   }
 
   /**
@@ -53,8 +56,8 @@ public class TestPlacementFactory {
     } catch (ClassNotFoundException cnfe) {
       fail("Class should have been found");
     }
-    assertNotNull("Rule object is null", rule);
-    assertEquals("Names not equal", rule.getName(), exists);
+    assertNotNull(rule, "Rule object is null");
+    assertEquals(rule.getName(), exists, "Names not equal");
   }
 
   /**
@@ -65,11 +68,11 @@ public class TestPlacementFactory {
   public void testGetRuleClass() {
     PlacementRule rule = PlacementFactory.getPlacementRule(
         DefaultPlacementRule.class, null);
-    assertNotNull("Rule object is null", rule);
+    assertNotNull(rule, "Rule object is null");
     // Should take anything as the second object: ignores unknown types in the
     // default implementation.
     rule = PlacementFactory.getPlacementRule(
         DefaultPlacementRule.class, "");
-    assertNotNull("Rule object is null", rule);
+    assertNotNull(rule, "Rule object is null");
   }
 }

+ 12 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/TestPlacementManager.java

@@ -28,16 +28,18 @@ 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.apache.hadoop.yarn.util.Records;
-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 java.util.ArrayList;
 import java.util.List;
 
 import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.DOT;
 import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestCapacitySchedulerAutoCreatedQueueBase.setupQueueConfiguration;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
 
 public class TestPlacementManager {
 
@@ -56,7 +58,7 @@ public class TestPlacementManager {
     return parentQueue + DOT + leafQueue;
   }
 
-  @Before
+  @BeforeEach
   public void setup() {
     conf = new CapacitySchedulerConfiguration();
     setupQueueConfiguration(conf);
@@ -96,8 +98,7 @@ public class TestPlacementManager {
     asc.setQueue(YarnConfiguration.DEFAULT_QUEUE_NAME);
     asc.setApplicationName(APP_NAME);
 
-    Assert.assertNull("Placement should be null",
-        pm.placeApplication(asc, USER2));
+    assertNull(pm.placeApplication(asc, USER2), "Placement should be null");
     QueueMapping queueMappingEntity = QueueMapping.QueueMappingBuilder.create()
       .type(MappingType.APPLICATION)
       .source(APP_NAME)
@@ -112,7 +113,7 @@ public class TestPlacementManager {
     queuePlacementRules.add(anRule);
     pm.updateRules(queuePlacementRules);
     ApplicationPlacementContext pc = pm.placeApplication(asc, USER2);
-    Assert.assertNotNull(pc);
+    assertNotNull(pc);
   }
 
   @Test
@@ -135,13 +136,13 @@ public class TestPlacementManager {
 
     // As we are setting placement rule, It shouldn't update default
     // placement rule ie user-group. Number of placement rules should be 1.
-    Assert.assertEquals(1, pm.getPlacementRules().size());
+    assertEquals(1, pm.getPlacementRules().size());
     // Verifying if placement rule set is same as the one we configured
-    Assert.assertEquals(ugRule.getName(),
+    assertEquals(ugRule.getName(),
         pm.getPlacementRules().get(0).getName());
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
     if (null != mockRM) {
       mockRM.stop();

+ 26 - 26
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/TestPlacementRuleFS.java

@@ -23,9 +23,9 @@ import org.apache.hadoop.util.XMLUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.QueueManager;
-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.w3c.dom.Document;
 import org.w3c.dom.Element;
 
@@ -37,10 +37,10 @@ import java.util.ArrayList;
 import java.util.List;
 
 import static org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementFactory.getPlacementRule;
-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.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.when;
 
@@ -75,7 +75,7 @@ public class TestPlacementRuleFS {
   private QueueManager queueManager;
 
 
-  @Before
+  @BeforeEach
   public void initTest() {
     scheduler = mock(FairScheduler.class);
     // needed for all rules that rely on group info
@@ -85,7 +85,7 @@ public class TestPlacementRuleFS {
     when(scheduler.getQueueManager()).thenReturn(queueManager);
   }
 
-  @After
+  @AfterEach
   public void cleanTest() {
     queueManager = null;
     scheduler = null;
@@ -126,26 +126,26 @@ public class TestPlacementRuleFS {
   private void ruleCreateNoConfig(Class <? extends PlacementRule> ruleClass) {
     PlacementRule rule = getPlacementRule(ruleClass, null);
     String name = ruleClass.getName();
-    assertNotNull("Rule object should not be null for " + name, rule);
+    assertNotNull(rule, "Rule object should not be null for " + name);
   }
 
   private void ruleCreateWrongObject(
       Class <? extends PlacementRule> ruleClass) {
     PlacementRule rule = getPlacementRule(ruleClass, "a string object");
     String name = ruleClass.getName();
-    assertNotNull("Rule object should not be null for " + name, rule);
+    assertNotNull(rule, "Rule object should not be null for " + name);
   }
 
   private void ruleCreateBoolean(Class <? extends PlacementRule> ruleClass) {
     PlacementRule rule = getPlacementRule(ruleClass, true);
     String name = ruleClass.getName();
-    assertNotNull("Rule object should not be null for " + name, rule);
-    assertTrue("Create flag was not set to true on " + name,
-        getCreateFlag(rule));
+    assertNotNull(rule, "Rule object should not be null for " + name);
+    assertTrue(getCreateFlag(rule),
+        "Create flag was not set to true on " + name);
     rule = getPlacementRule(ruleClass, false);
-    assertNotNull("Rule object should not be null for " + name, rule);
-    assertFalse("Create flag was not set to false on " + name,
-        getCreateFlag(rule));
+    assertNotNull(rule, "Rule object should not be null for " + name);
+    assertFalse(getCreateFlag(rule),
+        "Create flag was not set to false on " + name);
   }
 
   private void ruleCreateElement(Class <? extends PlacementRule> ruleClass) {
@@ -153,21 +153,21 @@ public class TestPlacementRuleFS {
     Element conf = createConf(str);
     PlacementRule rule = getPlacementRule(ruleClass, conf);
     String name = ruleClass.getName();
-    assertNotNull("Rule object should not be null for " + name, rule);
-    assertTrue("Create flag was not set to true on " + name,
-        getCreateFlag(rule));
+    assertNotNull(rule, "Rule object should not be null for " + name);
+    assertTrue(getCreateFlag(rule),
+        "Create flag was not set to true on " + name);
     str = "<rule name='not used' create=\"false\" />";
     conf = createConf(str);
     rule = getPlacementRule(ruleClass, conf);
-    assertNotNull("Rule object should not be null for " + name, rule);
-    assertFalse("Create flag was not set to false on " + name,
-        getCreateFlag(rule));
+    assertNotNull(rule, "Rule object should not be null for " + name);
+    assertFalse(getCreateFlag(rule),
+        "Create flag was not set to false on " + name);
   }
 
   private void ruleInit(Class <? extends PlacementRule> ruleClass) {
     PlacementRule rule = getPlacementRule(ruleClass, null);
     String name = ruleClass.getName();
-    assertNotNull("Rule object should not be null for " + name, rule);
+    assertNotNull(rule, "Rule object should not be null for " + name);
     try {
       rule.initialize(scheduler);
     } catch (IOException ioe) {
@@ -183,8 +183,8 @@ public class TestPlacementRuleFS {
     } catch (IOException ioe) {
       exceptionThrown = true;
     }
-    assertTrue("Initialize with parent rule should have thrown exception " +
-            name, exceptionThrown);
+    assertTrue(exceptionThrown,
+        "Initialize with parent rule should have thrown exception " + name);
   }
 
   private Element createConf(String str) {

+ 8 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/TestUserGroupMappingPlacementRule.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.placement;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.isNull;
@@ -42,14 +43,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.PrimaryGroupMapping;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.SimpleGroupsMapping;
 import org.apache.hadoop.yarn.util.Records;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 public class TestUserGroupMappingPlacementRule {
   private final YarnConfiguration conf = new YarnConfiguration();
 
-  @Before
+  @BeforeEach
   public void setup() {
     conf.setClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
         SimpleGroupsMapping.class, GroupMappingServiceProvider.class);
@@ -94,11 +94,11 @@ public class TestUserGroupMappingPlacementRule {
         ApplicationSubmissionContext.class);
     asc.setQueue(inputQueue);
     ApplicationPlacementContext ctx = engine.getPlacementForApp(asc, inputUser);
-    Assert.assertEquals("Queue", expectedQueue,
-        ctx != null ? ctx.getQueue() : inputQueue);
+    assertEquals(expectedQueue,
+        ctx != null ? ctx.getQueue() : inputQueue, "Queue");
     if (ctx != null && expectedParentQueue != null) {
-      Assert.assertEquals("Parent Queue", expectedParentQueue,
-          ctx.getParentQueue());
+      assertEquals(expectedParentQueue,
+          ctx.getParentQueue(), "Parent Queue");
     }
   }
 

+ 40 - 36
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/csmappingrule/TestCSMappingPlacementRule.java

@@ -33,9 +33,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueManager;
 import org.apache.hadoop.yarn.util.Records;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -43,17 +42,19 @@ import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileWriter;
 import java.io.IOException;
+import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import static junit.framework.TestCase.assertEquals;
-import static junit.framework.TestCase.assertNotNull;
-import static junit.framework.TestCase.assertNull;
-import static junit.framework.TestCase.assertTrue;
-import static junit.framework.TestCase.fail;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_GROUP_MAPPING;
+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.isNull;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -63,8 +64,8 @@ public class TestCSMappingPlacementRule {
   private static final Logger LOG = LoggerFactory
       .getLogger(TestCSMappingPlacementRule.class);
 
-  @Rule
-  public TemporaryFolder folder = new TemporaryFolder();
+  @TempDir
+  private Path folder;
   
   private Map<String, Set<String>> userGroups =
       ImmutableMap.<String, Set<String>>builder()
@@ -185,11 +186,11 @@ public class TestCSMappingPlacementRule {
       ApplicationSubmissionContext asc, String user, String expectedQueue) {
     try {
       ApplicationPlacementContext apc = engine.getPlacementForApp(asc, user);
-      assertNotNull(message, apc);
+      assertNotNull(apc, message);
       String queue = apc.getParentQueue() == null ? "" :
           (apc.getParentQueue() + DOT);
       queue += apc.getQueue();
-      assertEquals(message, expectedQueue,  queue);
+      assertEquals(expectedQueue,  queue, message);
     } catch (YarnException e) {
       LOG.error(message, e);
       fail(message);
@@ -199,7 +200,7 @@ public class TestCSMappingPlacementRule {
   private void assertNullResult(String message, CSMappingPlacementRule engine,
                         ApplicationSubmissionContext asc, String user) {
     try {
-      assertNull(message, engine.getPlacementForApp(asc, user));
+      assertNull(engine.getPlacementForApp(asc, user), message);
     } catch (YarnException e) {
       LOG.error(message, e);
       fail(message);
@@ -628,15 +629,14 @@ public class TestCSMappingPlacementRule {
 
 
   void assertConfigTestResult(List<MappingRule> rules) {
-    assertEquals("We only specified one rule", 1, rules.size());
+    assertEquals(1, rules.size(), "We only specified one rule");
     MappingRule rule = rules.get(0);
     String ruleStr = rule.toString();
-    assertTrue("Rule's matcher variable should be %user",
-        ruleStr.contains("variable='%user'"));
-    assertTrue("Rule's match value should be bob",
-        ruleStr.contains("value='bob'"));
-    assertTrue("Rule's action should be place to queue", ruleStr.contains(
-        "action=PlaceToQueueAction{queueName='%primary_group'"));
+    assertTrue(ruleStr.contains("variable='%user'"),
+        "Rule's matcher variable should be %user");
+    assertTrue(ruleStr.contains("value='bob'"), "Rule's match value should be bob");
+    assertTrue(ruleStr.contains("action=PlaceToQueueAction{queueName='%primary_group'"),
+        "Rule's action should be place to queue");
   }
 
   @Test
@@ -677,27 +677,31 @@ public class TestCSMappingPlacementRule {
     conf.set(CapacitySchedulerConfiguration.MAPPING_RULE_JSON, "");
 
     List<MappingRule> rules = conf.getMappingRules();
-    assertEquals("We expect no rules", 0, rules.size());
+    assertEquals(0, rules.size(), "We expect no rules");
   }
 
-  @Test(expected = IOException.class)
+  @Test
   public void testInvalidJSONConfiguration() throws IOException {
-    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
-    conf.set(CapacitySchedulerConfiguration.MAPPING_RULE_FORMAT,
-        CapacitySchedulerConfiguration.MAPPING_RULE_FORMAT_JSON);
-    conf.set(CapacitySchedulerConfiguration.MAPPING_RULE_JSON,
-        "I'm a bad JSON, since I'm not a JSON.");
-    List<MappingRule> rules = conf.getMappingRules();
+    assertThrows(IOException.class, ()->{
+      CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+      conf.set(CapacitySchedulerConfiguration.MAPPING_RULE_FORMAT,
+              CapacitySchedulerConfiguration.MAPPING_RULE_FORMAT_JSON);
+      conf.set(CapacitySchedulerConfiguration.MAPPING_RULE_JSON,
+              "I'm a bad JSON, since I'm not a JSON.");
+      List<MappingRule> rules = conf.getMappingRules();
+    });
   }
 
-  @Test(expected = IOException.class)
+  @Test
   public void testMissingJSONFileConfiguration() throws IOException {
-    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
-    conf.set(CapacitySchedulerConfiguration.MAPPING_RULE_FORMAT,
-        CapacitySchedulerConfiguration.MAPPING_RULE_FORMAT_JSON);
-    conf.set(CapacitySchedulerConfiguration.MAPPING_RULE_JSON_FILE,
-        "/dev/null/nofile");
-    List<MappingRule> rules = conf.getMappingRules();
+    assertThrows(IOException.class, ()->{
+      CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+      conf.set(CapacitySchedulerConfiguration.MAPPING_RULE_FORMAT,
+              CapacitySchedulerConfiguration.MAPPING_RULE_FORMAT_JSON);
+      conf.set(CapacitySchedulerConfiguration.MAPPING_RULE_JSON_FILE,
+              "/dev/null/nofile");
+      List<MappingRule> rules = conf.getMappingRules();
+    });
   }
 
   @Test
@@ -706,7 +710,7 @@ public class TestCSMappingPlacementRule {
     conf.set(CapacitySchedulerConfiguration.MAPPING_RULE_FORMAT,
         CapacitySchedulerConfiguration.MAPPING_RULE_FORMAT_JSON);
 
-    File jsonFile = folder.newFile("testJSONFileConfiguration.json");
+    File jsonFile = folder.resolve("testJSONFileConfiguration.json").toFile();
 
     BufferedWriter writer = new BufferedWriter(new FileWriter(jsonFile));
     try {

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/csmappingrule/TestMappingRule.java

@@ -18,13 +18,13 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.placement.csmappingrule;
 
-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 org.apache.hadoop.util.Sets;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.VariableContext;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 public class TestMappingRule {
   VariableContext setupVariables(

+ 5 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/csmappingrule/TestMappingRuleActions.java

@@ -18,9 +18,12 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.placement.csmappingrule;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.VariableContext;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
-import static org.junit.Assert.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 public class TestMappingRuleActions {
   void assertRejectResult(MappingRuleResult result) {

+ 32 - 34
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/csmappingrule/TestMappingRuleMatchers.java

@@ -18,15 +18,17 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.placement.csmappingrule;
 
-import junit.framework.TestCase;
+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.apache.hadoop.util.Sets;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.VariableContext;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.util.HashMap;
 import java.util.Map;
 
-public class TestMappingRuleMatchers extends TestCase {
+public class TestMappingRuleMatchers  {
 
   @Test
   public void testCatchAll() {
@@ -64,12 +66,12 @@ public class TestMappingRuleMatchers extends TestCase {
         new MappingRuleMatchers.VariableMatcher("%custom", "Matching string"));
 
     matchers.forEach((matcherName, matcher) -> {
-      assertTrue(matcherName + " with matchingContext should match",
-          matcher.match(matchingContext));
-      assertFalse(matcherName + " with mismatchingContext shouldn't match",
-          matcher.match(mismatchingContext));
-      assertFalse(matcherName + " with emptyContext shouldn't match",
-          matcher.match(emptyContext));
+      assertTrue(matcher.match(matchingContext),
+          matcherName + " with matchingContext should match");
+      assertFalse(matcher.match(mismatchingContext),
+          matcherName + " with mismatchingContext shouldn't match");
+      assertFalse(matcher.match(emptyContext),
+          matcherName + " with emptyContext shouldn't match");
     });
   }
 
@@ -106,31 +108,27 @@ public class TestMappingRuleMatchers extends TestCase {
     MappingRuleMatcher userStatic =
         new MappingRuleMatchers.VariableMatcher("%user", "bob");
 
-    assertTrue("%custom should match %user in matching context",
-        customUser.match(matchingContext));
-    assertTrue("%user should match %custom in matching context",
-        userCustom.match(matchingContext));
-    assertTrue("%user (bob) should match %cus%tom (b + ob) in matching context",
-        userCusTom.match(matchingContext));
-    assertTrue("%user should match %user in any context",
-        userUser.match(matchingContext));
-    assertTrue("%user (bob) should match bob in in matching context",
-        userStatic.match(matchingContext));
-
-    assertFalse(
-        "%custom (bob) should NOT match %user (dave) in mismatching context",
-        customUser.match(mismatchingContext));
-    assertFalse(
-        "%user (dave) should NOT match %custom (bob) in mismatching context",
-        userCustom.match(mismatchingContext));
-    assertFalse(
-        "%user (dave) should NOT match %cus%tom (b+ob) in mismatching context",
-        userCusTom.match(mismatchingContext));
-    assertTrue("%user should match %user in any context",
-        userUser.match(mismatchingContext));
-    assertFalse(
-        "%user (dave) should NOT match match bob in in matching context",
-        userStatic.match(mismatchingContext));
+    assertTrue(customUser.match(matchingContext),
+        "%custom should match %user in matching context");
+    assertTrue(userCustom.match(matchingContext),
+        "%user should match %custom in matching context");
+    assertTrue(userCusTom.match(matchingContext),
+        "%user (bob) should match %cus%tom (b + ob) in matching context");
+    assertTrue(userUser.match(matchingContext),
+        "%user should match %user in any context");
+    assertTrue(userStatic.match(matchingContext),
+        "%user (bob) should match bob in in matching context");
+
+    assertFalse(customUser.match(mismatchingContext),
+        "%custom (bob) should NOT match %user (dave) in mismatching context");
+    assertFalse(userCustom.match(mismatchingContext),
+        "%user (dave) should NOT match %custom (bob) in mismatching context");
+    assertFalse(userCusTom.match(mismatchingContext),
+        "%user (dave) should NOT match %cus%tom (b+ob) in mismatching context");
+    assertTrue(userUser.match(mismatchingContext),
+        "%user should match %user in any context");
+    assertFalse(userStatic.match(mismatchingContext),
+        "%user (dave) should NOT match match bob in in matching context");
   }
 
   @Test

+ 5 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/csmappingrule/TestMappingRuleValidationContextImpl.java

@@ -21,9 +21,12 @@ package org.apache.hadoop.yarn.server.resourcemanager.placement.csmappingrule;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.MockQueueHierarchyBuilder;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueManager;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
-import static org.junit.Assert.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.ArgumentMatchers.isNull;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;

+ 42 - 28
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/csmappingrule/TestVariableContext.java

@@ -20,13 +20,19 @@ package org.apache.hadoop.yarn.server.resourcemanager.placement.csmappingrule;
 
 import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.VariableContext;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Set;
 
-import static org.junit.Assert.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 public class TestVariableContext {
 
@@ -49,32 +55,38 @@ public class TestVariableContext {
     assertEquals("", variables.get("%empty"));
   }
 
-  @Test(expected = IllegalStateException.class)
+  @Test
   public void testImmutablesCanOnlySetOnceFromSet() {
-    VariableContext variables = new VariableContext();
-    ImmutableSet<String> immutables =
-        ImmutableSet.of("%user", "%primary_group", "%secondary_group");
-
-    variables.setImmutables(immutables);
-    variables.setImmutables(immutables);
+    assertThrows(IllegalStateException.class, () -> {
+      VariableContext variables = new VariableContext();
+      ImmutableSet<String> immutables =
+          ImmutableSet.of("%user", "%primary_group", "%secondary_group");
+
+      variables.setImmutables(immutables);
+      variables.setImmutables(immutables);
+    });
   }
 
-  @Test(expected = IllegalStateException.class)
+  @Test
   public void testImmutablesCanOnlySetOnceFromArray() {
-    VariableContext variables = new VariableContext();
+    assertThrows(IllegalStateException.class, () -> {
+      VariableContext variables = new VariableContext();
 
-    variables.setImmutables("%user", "%primary_group", "%secondary_group");
-    variables.setImmutables("%user", "%primary_group", "%secondary_group");
+      variables.setImmutables("%user", "%primary_group", "%secondary_group");
+      variables.setImmutables("%user", "%primary_group", "%secondary_group");
+    });
   }
 
-  @Test(expected = IllegalStateException.class)
+  @Test
   public void testImmutablesCanOnlySetOnceFromSetAndArray() {
-    VariableContext variables = new VariableContext();
-    ImmutableSet<String> immutables =
-        ImmutableSet.of("%user", "%primary_group", "%secondary_group");
-
-    variables.setImmutables(immutables);
-    variables.setImmutables("%user", "%primary_group", "%secondary_group");
+    assertThrows(IllegalStateException.class, () -> {
+      VariableContext variables = new VariableContext();
+      ImmutableSet<String> immutables =
+          ImmutableSet.of("%user", "%primary_group", "%secondary_group");
+
+      variables.setImmutables(immutables);
+      variables.setImmutables("%user", "%primary_group", "%secondary_group");
+    });
   }
 
   @Test
@@ -87,15 +99,17 @@ public class TestVariableContext {
     variables.put("%user", "bob");
   }
 
-  @Test(expected = IllegalStateException.class)
+  @Test
   public void testImmutableVariableProtection() {
-    VariableContext variables = new VariableContext();
-    ImmutableSet<String> immutables =
-        ImmutableSet.of("%user", "%primary_group", "%secondary_group");
-
-    variables.setImmutables(immutables);
-    variables.put("%user", "bob");
-    variables.put("%user", "bob");
+    assertThrows(IllegalStateException.class, () -> {
+      VariableContext variables = new VariableContext();
+      ImmutableSet<String> immutables =
+          ImmutableSet.of("%user", "%primary_group", "%secondary_group");
+
+      variables.setImmutables(immutables);
+      variables.put("%user", "bob");
+      variables.put("%user", "bob");
+    });
   }
 
   @Test

+ 4 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/preprocessor/TestContextProcessor.java

@@ -24,9 +24,11 @@ import java.util.Map;
 import java.util.Set;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
 
 
 /**

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/preprocessor/TestNodeLabelProcessor.java

@@ -21,7 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.preprocessor;
 
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import static org.mockito.Mockito.*;
 

+ 5 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/preprocessor/TestQueueProcessor.java

@@ -20,9 +20,12 @@ package org.apache.hadoop.yarn.server.resourcemanager.preprocessor;
 
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 
 /**

+ 6 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/preprocessor/TestTagAddProcessor.java

@@ -22,8 +22,12 @@ import java.util.HashSet;
 import java.util.Set;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.junit.Test;
-import static org.mockito.Mockito.*;
+import org.junit.jupiter.api.Test;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 
 /**

+ 70 - 69
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java

@@ -18,12 +18,14 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.recovery;
 
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.when;
@@ -86,7 +88,6 @@ import org.apache.hadoop.yarn.server.security.MasterKeyData;
 import org.apache.hadoop.yarn.server.webproxy.ProxyCA;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
-import org.junit.Assert;
 
 public class RMStateStoreTestBase {
 
@@ -484,13 +485,13 @@ public class RMStateStoreTestBase {
 
     RMDTSecretManagerState secretManagerState =
         store.loadState().getRMDTSecretManagerState();
-    Assert.assertEquals(token1, secretManagerState.getTokenState());
-    Assert.assertEquals(keySet, secretManagerState.getMasterKeyState());
-    Assert.assertEquals(sequenceNumber,
+    assertEquals(token1, secretManagerState.getTokenState());
+    assertEquals(keySet, secretManagerState.getMasterKeyState());
+    assertEquals(sequenceNumber,
         secretManagerState.getDTSequenceNumber());
     RMDelegationTokenIdentifier tokenAfterStore =
         secretManagerState.getTokenState().keySet().iterator().next();
-    Assert.assertTrue(Arrays.equals(tokenBeforeStore,
+    assertTrue(Arrays.equals(tokenBeforeStore,
       tokenAfterStore.getBytes()));
 
     // update RM delegation token;
@@ -500,9 +501,9 @@ public class RMStateStoreTestBase {
 
     RMDTSecretManagerState updateSecretManagerState =
         store.loadState().getRMDTSecretManagerState();
-    Assert.assertEquals(token1, updateSecretManagerState.getTokenState());
-    Assert.assertEquals(keySet, updateSecretManagerState.getMasterKeyState());
-    Assert.assertEquals(sequenceNumber,
+    assertEquals(token1, updateSecretManagerState.getTokenState());
+    assertEquals(keySet, updateSecretManagerState.getMasterKeyState());
+    assertEquals(sequenceNumber,
         updateSecretManagerState.getDTSequenceNumber());
 
     // check to delete delegationKey
@@ -510,9 +511,9 @@ public class RMStateStoreTestBase {
     keySet.clear();
     RMDTSecretManagerState noKeySecretManagerState =
         store.loadState().getRMDTSecretManagerState();
-    Assert.assertEquals(token1, noKeySecretManagerState.getTokenState());
-    Assert.assertEquals(keySet, noKeySecretManagerState.getMasterKeyState());
-    Assert.assertEquals(sequenceNumber,
+    assertEquals(token1, noKeySecretManagerState.getTokenState());
+    assertEquals(keySet, noKeySecretManagerState.getMasterKeyState());
+    assertEquals(sequenceNumber,
         noKeySecretManagerState.getDTSequenceNumber());
 
     // check to delete delegationToken
@@ -520,11 +521,11 @@ public class RMStateStoreTestBase {
     RMDTSecretManagerState noKeyAndTokenSecretManagerState =
         store.loadState().getRMDTSecretManagerState();
     token1.clear();
-    Assert.assertEquals(token1,
+    assertEquals(token1,
         noKeyAndTokenSecretManagerState.getTokenState());
-    Assert.assertEquals(keySet,
+    assertEquals(keySet,
         noKeyAndTokenSecretManagerState.getMasterKeyState());
-    Assert.assertEquals(sequenceNumber,
+    assertEquals(sequenceNumber,
         noKeySecretManagerState.getDTSequenceNumber());
     store.close();
 
@@ -547,17 +548,17 @@ public class RMStateStoreTestBase {
     // default version
     Version defaultVersion = stateStoreHelper.getCurrentVersion();
     store.checkVersion();
-    Assert.assertEquals(defaultVersion, store.loadVersion());
+    assertEquals(defaultVersion, store.loadVersion());
 
     // compatible version
     Version compatibleVersion =
         Version.newInstance(defaultVersion.getMajorVersion(),
           defaultVersion.getMinorVersion() + 2);
     stateStoreHelper.writeVersion(compatibleVersion);
-    Assert.assertEquals(compatibleVersion, store.loadVersion());
+    assertEquals(compatibleVersion, store.loadVersion());
     store.checkVersion();
     // overwrite the compatible version
-    Assert.assertEquals(defaultVersion, store.loadVersion());
+    assertEquals(defaultVersion, store.loadVersion());
 
     // incompatible version
     Version incompatibleVersion =
@@ -566,9 +567,9 @@ public class RMStateStoreTestBase {
     stateStoreHelper.writeVersion(incompatibleVersion);
     try {
       store.checkVersion();
-      Assert.fail("Invalid version, should fail.");
+      fail("Invalid version, should fail.");
     } catch (Throwable t) {
-      Assert.assertTrue(t instanceof RMStateVersionIncompatibleException);
+      assertTrue(t instanceof RMStateVersionIncompatibleException);
     }
   }
   
@@ -578,13 +579,13 @@ public class RMStateStoreTestBase {
     store.setRMDispatcher(new TestDispatcher());
     
     long firstTimeEpoch = store.getAndIncrementEpoch();
-    Assert.assertEquals(epoch, firstTimeEpoch);
+    assertEquals(epoch, firstTimeEpoch);
     
     long secondTimeEpoch = store.getAndIncrementEpoch();
-    Assert.assertEquals(epoch + 1, secondTimeEpoch);
+    assertEquals(epoch + 1, secondTimeEpoch);
     
     long thirdTimeEpoch = store.getAndIncrementEpoch();
-    Assert.assertEquals(epoch + 2, thirdTimeEpoch);
+    assertEquals(epoch + 2, thirdTimeEpoch);
 
     for (int i = 0; i < epochRange; ++i) {
       store.getAndIncrementEpoch();
@@ -592,7 +593,7 @@ public class RMStateStoreTestBase {
     long wrappedEpoch = store.getAndIncrementEpoch();
     // Epoch should have wrapped around and then incremented once for a total
     // of + 3
-    Assert.assertEquals(epoch + 3, wrappedEpoch);
+    assertEquals(epoch + 3, wrappedEpoch);
   }
 
   public void testAppDeletion(RMStateStoreHelper stateStoreHelper)
@@ -625,7 +626,7 @@ public class RMStateStoreTestBase {
       appList.add(app);
     }
 
-    Assert.assertEquals(numApps, appList.size());
+    assertEquals(numApps, appList.size());
     for (RMApp app : appList) {
       // wait for app to be stored.
       while (true) {
@@ -646,7 +647,7 @@ public class RMStateStoreTestBase {
     store.deleteStore();
     // verify apps deleted
     for (RMApp app : appList) {
-      Assert.assertFalse(stateStoreHelper.appExists(app));
+      assertFalse(stateStoreHelper.appExists(app));
     }
   }
 
@@ -659,10 +660,10 @@ public class RMStateStoreTestBase {
 
     RMApp rmApp1 = appList.get(0);
     store.removeApplication(rmApp1.getApplicationId());
-    Assert.assertFalse(stateStoreHelper.appExists(rmApp1));
+    assertFalse(stateStoreHelper.appExists(rmApp1));
 
     RMApp rmApp2 = appList.get(1);
-    Assert.assertTrue(stateStoreHelper.appExists(rmApp2));
+    assertTrue(stateStoreHelper.appExists(rmApp2));
   }
 
   public void testRemoveAttempt(RMStateStoreHelper stateStoreHelper)
@@ -685,8 +686,8 @@ public class RMStateStoreTestBase {
         ContainerId.newContainerId(attemptId2, 1).toString(),
         null, null, dispatcher);
     store.removeApplicationAttemptInternal(attemptId1);
-    Assert.assertFalse(stateStoreHelper.attemptExists(attempt1));
-    Assert.assertTrue(stateStoreHelper.attemptExists(attempt2));
+    assertFalse(stateStoreHelper.attemptExists(attempt1));
+    assertTrue(stateStoreHelper.attemptExists(attempt2));
 
     // let things settle down
     Thread.sleep(1000);
@@ -742,10 +743,10 @@ public class RMStateStoreTestBase {
     when(rmContext.getStateStore()).thenReturn(store);
     store.setRMDispatcher(dispatcher);
     RMState state = store.loadState();
-    Assert.assertNotNull(state.getAMRMTokenSecretManagerState());
-    Assert.assertEquals(firstMasterKeyData.getMasterKey(), state
+    assertNotNull(state.getAMRMTokenSecretManagerState());
+    assertEquals(firstMasterKeyData.getMasterKey(), state
       .getAMRMTokenSecretManagerState().getCurrentMasterKey());
-    Assert.assertNull(state
+    assertNull(state
       .getAMRMTokenSecretManagerState().getNextMasterKey());
 
     //create and save the second masterkey
@@ -762,18 +763,18 @@ public class RMStateStoreTestBase {
     when(rmContext.getStateStore()).thenReturn(store);
     store.setRMDispatcher(dispatcher);
     RMState state_2 = store.loadState();
-    Assert.assertNotNull(state_2.getAMRMTokenSecretManagerState());
-    Assert.assertEquals(firstMasterKeyData.getMasterKey(), state_2
+    assertNotNull(state_2.getAMRMTokenSecretManagerState());
+    assertEquals(firstMasterKeyData.getMasterKey(), state_2
       .getAMRMTokenSecretManagerState().getCurrentMasterKey());
-    Assert.assertEquals(secondMasterKeyData.getMasterKey(), state_2
+    assertEquals(secondMasterKeyData.getMasterKey(), state_2
       .getAMRMTokenSecretManagerState().getNextMasterKey());
 
     // re-create the masterKeyData based on the recovered masterkey
     // should have the same secretKey
     appTokenMgr.recover(state_2);
-    Assert.assertEquals(appTokenMgr.getCurrnetMasterKeyData().getSecretKey(),
+    assertEquals(appTokenMgr.getCurrnetMasterKeyData().getSecretKey(),
       firstMasterKeyData.getSecretKey());
-    Assert.assertEquals(appTokenMgr.getNextMasterKeyData().getSecretKey(),
+    assertEquals(appTokenMgr.getNextMasterKeyData().getSecretKey(),
       secondMasterKeyData.getSecretKey());
 
     store.close();
@@ -814,7 +815,7 @@ public class RMStateStoreTestBase {
     RMState state = store.loadState();
     Map<String, Map<ReservationId, ReservationAllocationStateProto>>
       reservationState = state.getReservationState();
-    Assert.assertNotNull(reservationState);
+    assertNotNull(reservationState);
 
     // 2. Store single reservation and verify
     String reservationIdName = r1.toString();
@@ -868,13 +869,13 @@ public class RMStateStoreTestBase {
     store.setRMDispatcher(dispatcher);
     state = store.loadState();
     reservationState = state.getReservationState();
-    Assert.assertNotNull(reservationState);
+    assertNotNull(reservationState);
     reservations = reservationState.get(planName);
-    Assert.assertNotNull(reservations);
+    assertNotNull(reservations);
     ReservationAllocationStateProto storedReservationAllocation =
         reservations.get(r1);
-    Assert.assertNull("Removed reservation should not be available in store",
-        storedReservationAllocation);
+    assertNull(storedReservationAllocation,
+        "Removed reservation should not be available in store");
 
     storedReservationAllocation = reservations.get(r2);
     assertAllocationStateEqual(
@@ -890,9 +891,9 @@ public class RMStateStoreTestBase {
     store.setRMDispatcher(dispatcher);
     state = store.loadState();
     reservationState = state.getReservationState();
-    Assert.assertNotNull(reservationState);
+    assertNotNull(reservationState);
     reservations = reservationState.get(planName);
-    Assert.assertNull(reservations);
+    assertNull(reservations);
   }
 
   public void testProxyCA(
@@ -908,22 +909,22 @@ public class RMStateStoreTestBase {
 
     RMStateStore.ProxyCAState proxyCAState =
         store.loadState().getProxyCAState();
-    Assert.assertEquals(originalProxyCA.getCaCert(), proxyCAState.getCaCert());
-    Assert.assertEquals(originalProxyCA.getCaKeyPair().getPrivate(),
+    assertEquals(originalProxyCA.getCaCert(), proxyCAState.getCaCert());
+    assertEquals(originalProxyCA.getCaKeyPair().getPrivate(),
         proxyCAState.getCaPrivateKey());
 
     // Try replacing with a different ProxyCA
     ProxyCA newProxyCA = new ProxyCA();
     newProxyCA.init();
-    Assert.assertNotEquals(originalProxyCA.getCaCert(), newProxyCA.getCaCert());
-    Assert.assertNotEquals(originalProxyCA.getCaKeyPair().getPrivate(),
+    assertNotEquals(originalProxyCA.getCaCert(), newProxyCA.getCaCert());
+    assertNotEquals(originalProxyCA.getCaKeyPair().getPrivate(),
         newProxyCA.getCaKeyPair().getPrivate());
     store.storeProxyCACert(newProxyCA.getCaCert(),
         newProxyCA.getCaKeyPair().getPrivate());
 
     proxyCAState = store.loadState().getProxyCAState();
-    Assert.assertEquals(newProxyCA.getCaCert(), proxyCAState.getCaCert());
-    Assert.assertEquals(newProxyCA.getCaKeyPair().getPrivate(),
+    assertEquals(newProxyCA.getCaCert(), proxyCAState.getCaCert());
+    assertEquals(newProxyCA.getCaKeyPair().getPrivate(),
         proxyCAState.getCaPrivateKey());
   }
 
@@ -938,13 +939,13 @@ public class RMStateStoreTestBase {
     RMState state = store.loadState();
     Map<String, Map<ReservationId, ReservationAllocationStateProto>>
         reservationState = state.getReservationState();
-    Assert.assertNotNull(reservationState);
+    assertNotNull(reservationState);
     Map<ReservationId, ReservationAllocationStateProto> reservations =
         reservationState.get(planName);
-    Assert.assertNotNull(reservations);
+    assertNotNull(reservations);
     ReservationAllocationStateProto storedReservationAllocation =
         reservations.get(r1);
-    Assert.assertNotNull(storedReservationAllocation);
+    assertNotNull(storedReservationAllocation);
 
     assertAllocationStateEqual(
         allocationStateProto, storedReservationAllocation);
@@ -955,12 +956,12 @@ public class RMStateStoreTestBase {
       ReservationAllocationStateProto expected,
       ReservationAllocationStateProto actual) {
 
-    Assert.assertEquals(
+    assertEquals(
         expected.getAcceptanceTime(), actual.getAcceptanceTime());
-    Assert.assertEquals(expected.getStartTime(), actual.getStartTime());
-    Assert.assertEquals(expected.getEndTime(), actual.getEndTime());
-    Assert.assertEquals(expected.getContainsGangs(), actual.getContainsGangs());
-    Assert.assertEquals(expected.getUser(), actual.getUser());
+    assertEquals(expected.getStartTime(), actual.getStartTime());
+    assertEquals(expected.getEndTime(), actual.getEndTime());
+    assertEquals(expected.getContainsGangs(), actual.getContainsGangs());
+    assertEquals(expected.getUser(), actual.getUser());
     assertEquals(
         expected.getReservationDefinition(), actual.getReservationDefinition());
     assertEquals(expected.getAllocationRequestsList(),
@@ -970,12 +971,12 @@ public class RMStateStoreTestBase {
   void assertAllocationStateEqual(
       ReservationAllocation expected,
       ReservationAllocationStateProto actual) {
-    Assert.assertEquals(
+    assertEquals(
         expected.getAcceptanceTime(), actual.getAcceptanceTime());
-    Assert.assertEquals(expected.getStartTime(), actual.getStartTime());
-    Assert.assertEquals(expected.getEndTime(), actual.getEndTime());
-    Assert.assertEquals(expected.containsGangs(), actual.getContainsGangs());
-    Assert.assertEquals(expected.getUser(), actual.getUser());
+    assertEquals(expected.getStartTime(), actual.getStartTime());
+    assertEquals(expected.getEndTime(), actual.getEndTime());
+    assertEquals(expected.containsGangs(), actual.getContainsGangs());
+    assertEquals(expected.getUser(), actual.getUser());
     assertEquals(
         expected.getReservationDefinition(),
         ReservationSystemUtil.convertFromProtoFormat(

+ 50 - 39
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestCheckRemoveZKNodeRMStateStore.java

@@ -44,16 +44,26 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.Applicatio
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.zookeeper.KeeperException;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 
-import org.mockito.Mockito;
-import org.junit.Assert;
+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.anyList;
+import static org.mockito.Mockito.anyString;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 public class TestCheckRemoveZKNodeRMStateStore extends RMStateStoreTestBase {
 
@@ -78,13 +88,13 @@ public class TestCheckRemoveZKNodeRMStateStore extends RMStateStoreTestBase {
     return curatorFramework;
   }
 
-  @Before
+  @BeforeEach
   public void setupCurator() throws Exception {
     curatorTestingServer = setupCuratorServer();
     curatorFramework = setupCuratorFramework(curatorTestingServer);
   }
 
-  @After
+  @AfterEach
   public void cleanupCuratorServer() throws IOException {
     curatorFramework.close();
     curatorTestingServer.stop();
@@ -101,31 +111,31 @@ public class TestCheckRemoveZKNodeRMStateStore extends RMStateStoreTestBase {
       private ZKCuratorManager zkCuratorManager;
       TestZKRMStateStoreInternal(Configuration conf, String workingZnode)
           throws Exception {
-        resourceManager = Mockito.mock(ResourceManager.class);
-        zkCuratorManager = Mockito.mock(ZKCuratorManager.class, Mockito.RETURNS_DEEP_STUBS);
+        resourceManager = mock(ResourceManager.class);
+        zkCuratorManager = mock(ZKCuratorManager.class, RETURNS_DEEP_STUBS);
 
-        Mockito.when(resourceManager.getZKManager()).thenReturn(zkCuratorManager);
-        Mockito.when(resourceManager.createAndStartZKManager(conf)).thenReturn(zkCuratorManager);
-        Mockito.when(zkCuratorManager.exists(getAppNode("application_1708333280_0001")))
+        when(resourceManager.getZKManager()).thenReturn(zkCuratorManager);
+        when(resourceManager.createAndStartZKManager(conf)).thenReturn(zkCuratorManager);
+        when(zkCuratorManager.exists(getAppNode("application_1708333280_0001")))
                 .thenReturn(true);
-        Mockito.when(zkCuratorManager.exists(getAppNode("application_1708334188_0001")))
+        when(zkCuratorManager.exists(getAppNode("application_1708334188_0001")))
                 .thenReturn(true).thenReturn(false);
-        Mockito.when(zkCuratorManager.exists(getDelegationTokenNode(0, 0)))
+        when(zkCuratorManager.exists(getDelegationTokenNode(0, 0)))
                 .thenReturn(true).thenReturn(false);
-        Mockito.when(zkCuratorManager.exists(getAppNode("application_1709705779_0001")))
+        when(zkCuratorManager.exists(getAppNode("application_1709705779_0001")))
                 .thenReturn(true);
-        Mockito.when(zkCuratorManager.exists(getAttemptNode("application_1709705779_0001",
+        when(zkCuratorManager.exists(getAttemptNode("application_1709705779_0001",
                         "appattempt_1709705779_0001_000001")))
                 .thenReturn(true);
-        Mockito.doThrow(new KeeperException.NoNodeException()).when(zkCuratorManager)
-                .safeDelete(Mockito.anyString(), Mockito.anyList(), Mockito.anyString());
+        doThrow(new KeeperException.NoNodeException()).when(zkCuratorManager)
+                .safeDelete(anyString(), anyList(), anyString());
 
         setResourceManager(resourceManager);
         init(conf);
         dispatcher.disableExitOnDispatchException();
         start();
 
-        Assert.assertTrue(znodeWorkingPath.equals(workingZnode));
+        assertTrue(znodeWorkingPath.equals(workingZnode));
       }
 
       private String getVersionNode() {
@@ -236,7 +246,8 @@ public class TestCheckRemoveZKNodeRMStateStore extends RMStateStoreTestBase {
     }
   }
 
-  @Test (timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testSafeDeleteZKNode() throws Exception  {
     TestZKRMStateStoreTester zkTester = new TestZKRMStateStoreTester();
     testRemoveAttempt(zkTester);
@@ -262,12 +273,12 @@ public class TestCheckRemoveZKNodeRMStateStore extends RMStateStoreTestBase {
     try {
       store.removeApplicationAttemptInternal(attemptIdRemoved);
     } catch (KeeperException.NoNodeException nne) {
-      Assert.fail("NoNodeException should not happen.");
+      fail("NoNodeException should not happen.");
     }
 
     // The verification method safeDelete is called once.
-    Mockito.verify(store.resourceManager.getZKManager(), Mockito.times(1))
-            .safeDelete(Mockito.anyString(), Mockito.anyList(), Mockito.anyString());
+    verify(store.resourceManager.getZKManager(), times(1))
+            .safeDelete(anyString(), anyList(), anyString());
 
     store.close();
   }
@@ -296,7 +307,7 @@ public class TestCheckRemoveZKNodeRMStateStore extends RMStateStoreTestBase {
       // The occurrence of NoNodeException is induced by calling the safeDelete method.
       store.removeApplicationStateInternal(appStateRemoved);
     } catch (KeeperException.NoNodeException nne) {
-      Assert.fail("NoNodeException should not happen.");
+      fail("NoNodeException should not happen.");
     }
 
     store.close();
@@ -312,12 +323,12 @@ public class TestCheckRemoveZKNodeRMStateStore extends RMStateStoreTestBase {
     try {
       store.removeRMDelegationTokenState(tokenIdRemoved);
     } catch (KeeperException.NoNodeException nne) {
-      Assert.fail("NoNodeException should not happen.");
+      fail("NoNodeException should not happen.");
     }
 
     // The verification method safeDelete is called once.
-    Mockito.verify(store.resourceManager.getZKManager(), Mockito.times(1))
-            .safeDelete(Mockito.anyString(), Mockito.anyList(), Mockito.anyString());
+    verify(store.resourceManager.getZKManager(), times(1))
+            .safeDelete(anyString(), anyList(), anyString());
 
     store.close();
   }
@@ -332,12 +343,12 @@ public class TestCheckRemoveZKNodeRMStateStore extends RMStateStoreTestBase {
     try {
       store.removeRMDTMasterKeyState(keyRemoved);
     } catch (KeeperException.NoNodeException nne) {
-      Assert.fail("NoNodeException should not happen.");
+      fail("NoNodeException should not happen.");
     }
 
     // The verification method safeDelete is called once.
-    Mockito.verify(store.resourceManager.getZKManager(), Mockito.times(1))
-            .safeDelete(Mockito.anyString(), Mockito.anyList(), Mockito.anyString());
+    verify(store.resourceManager.getZKManager(), times(1))
+            .safeDelete(anyString(), anyList(), anyString());
 
     store.close();
   }
@@ -353,12 +364,12 @@ public class TestCheckRemoveZKNodeRMStateStore extends RMStateStoreTestBase {
     try {
       store.removeReservationState(planName, reservationIdRemoved.toString());
     } catch (KeeperException.NoNodeException nne) {
-      Assert.fail("NoNodeException should not happen.");
+      fail("NoNodeException should not happen.");
     }
 
     // The verification method safeDelete is called once.
-    Mockito.verify(store.resourceManager.getZKManager(), Mockito.times(1))
-            .safeDelete(Mockito.anyString(), Mockito.anyList(), Mockito.anyString());
+    verify(store.resourceManager.getZKManager(), times(1))
+            .safeDelete(anyString(), anyList(), anyString());
 
     store.close();
   }
@@ -376,11 +387,11 @@ public class TestCheckRemoveZKNodeRMStateStore extends RMStateStoreTestBase {
 
     // Transition to active.
     rm.getRMContext().getRMAdminService().transitionToActive(req);
-    Assert.assertEquals("RM with ZKStore didn't start",
-            Service.STATE.STARTED, rm.getServiceState());
-    Assert.assertEquals("RM should be Active",
-            HAServiceProtocol.HAServiceState.ACTIVE,
-            rm.getRMContext().getRMAdminService().getServiceStatus().getState());
+    assertEquals(Service.STATE.STARTED, rm.getServiceState(),
+        "RM with ZKStore didn't start");
+    assertEquals(HAServiceProtocol.HAServiceState.ACTIVE,
+        rm.getRMContext().getRMAdminService().getServiceStatus().getState(),
+        "RM should be Active");
 
     // Simulate throw NodeExistsException
     ZKRMStateStore zKStore = (ZKRMStateStore) rm.getRMContext().getStateStore();
@@ -397,7 +408,7 @@ public class TestCheckRemoveZKNodeRMStateStore extends RMStateStoreTestBase {
     try {
       zKStore.removeApplicationAttemptInternal(attemptIdRemoved);
     } catch (Exception e) {
-      Assert.assertTrue(e instanceof KeeperException.NodeExistsException);
+      assertTrue(e instanceof KeeperException.NodeExistsException);
     }
 
     rm.close();

+ 28 - 18
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java

@@ -19,7 +19,12 @@
 package org.apache.hadoop.yarn.server.resourcemanager.recovery;
 
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
@@ -50,8 +55,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.rmapp.attempt.RMAppAttempt;
 
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 public class TestFSRMStateStore extends RMStateStoreTestBase {
 
@@ -71,11 +76,11 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
 
       TestFileSystemRMStore(Configuration conf) throws Exception {
         init(conf);
-        Assert.assertNull(fs);
+        assertNull(fs);
         assertTrue(workingDirPathURI.equals(fsWorkingPath));
         dispatcher.disableExitOnDispatchException();
         start();
-        Assert.assertNotNull(fs);
+        assertNotNull(fs);
       }
 
       public Path getVersionNode() {
@@ -128,11 +133,11 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
       this.store = new TestFileSystemRMStore(conf);
       assertThat(store.getNumRetries()).isEqualTo(8);
       assertThat(store.getRetryInterval()).isEqualTo(900L);
-      Assert.assertTrue(store.fs.getConf() == store.fsConf);
+      assertTrue(store.fs.getConf() == store.fsConf);
       FileSystem previousFs = store.fs;
       store.startInternal();
-      Assert.assertTrue(store.fs != previousFs);
-      Assert.assertTrue(store.fs.getConf() == store.fsConf);
+      assertTrue(store.fs != previousFs);
+      assertTrue(store.fs.getConf() == store.fsConf);
       return store;
     }
 
@@ -172,7 +177,8 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
     }
   }
 
-  @Test(timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testFSRMStateStore() throws Exception {
     HdfsConfiguration conf = new HdfsConfiguration();
     MiniDFSCluster cluster =
@@ -196,7 +202,7 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
       fsOut.close();
 
       testRMAppStateStore(fsTester);
-      Assert.assertFalse(fsTester.workingDirPathURI
+      assertFalse(fsTester.workingDirPathURI
               .getFileSystem(conf).exists(tempAppAttemptFile));
       testRMDTSecretManagerStateStore(fsTester);
       testCheckVersion(fsTester);
@@ -213,7 +219,8 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
     }
   }
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testHDFSRMStateStore() throws Exception {
     final HdfsConfiguration conf = new HdfsConfiguration();
     UserGroupInformation yarnAdmin =
@@ -293,11 +300,12 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
         try {
           LOG.warn("\n\n ##Testing path [" + p + "]\n\n");
           fs.open(p);
-          Assert.fail("Super user should not be able to read ["+ UserGroupInformation.getCurrentUser() + "] [" + p.getName() + "]");
+          fail("Super user should not be able to read ["+
+              UserGroupInformation.getCurrentUser() + "] [" + p.getName() + "]");
         } catch (AccessControlException e) {
-          Assert.assertTrue(e.getMessage().contains("superuser is not allowed to perform this operation"));
+          assertTrue(e.getMessage().contains("superuser is not allowed to perform this operation"));
         } catch (Exception e) {
-          Assert.fail("Should get an AccessControlException here");
+          fail("Should get an AccessControlException here");
         }
       }
       if (stat.isDirectory()) {
@@ -310,7 +318,8 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
 
   }
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testCheckMajorVersionChange() throws Exception {
     HdfsConfiguration conf = new HdfsConfiguration();
     MiniDFSCluster cluster =
@@ -355,7 +364,7 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
       RMStateStore store = fsTester.getRMStateStore();
       Version defaultVersion = fsTester.getCurrentVersion();
       store.checkVersion();
-      Assert.assertEquals(defaultVersion, store.loadVersion());
+      assertEquals(defaultVersion, store.loadVersion());
     } finally {
       cluster.shutdown();
     }
@@ -391,7 +400,8 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
             nodeCreatePath.getName() + ".new"));
   }
 
-  @Test (timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testFSRMStateStoreClientRetry() throws Exception {
     HdfsConfiguration conf = new HdfsConfiguration();
     MiniDFSCluster cluster =
@@ -419,7 +429,7 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
       clientThread.start();
       cluster.restartNameNode();
       clientThread.join();
-      Assert.assertFalse(assertionFailedInThread.get());
+      assertFalse(assertionFailedInThread.get());
     } finally {
       cluster.shutdown();
     }

+ 30 - 17
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestLeveldbRMStateStore.java

@@ -36,9 +36,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.fusesource.leveldbjni.JniDBFactory;
 import org.iq80.leveldb.DB;
 import org.iq80.leveldb.Options;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 public class TestLeveldbRMStateStore extends RMStateStoreTestBase {
 
@@ -50,14 +51,14 @@ public class TestLeveldbRMStateStore extends RMStateStoreTestBase {
   private YarnConfiguration conf;
   private LeveldbRMStateStore stateStore = null;
 
-  @Before
+  @BeforeEach
   public void setup() throws IOException {
     FileUtil.fullyDelete(TEST_DIR);
     conf = new YarnConfiguration();
     conf.set(YarnConfiguration.RM_LEVELDB_STORE_PATH, TEST_DIR.toString());
   }
 
-  @After
+  @AfterEach
   public void cleanup() throws IOException {
     if (stateStore != null) {
       stateStore.close();
@@ -65,25 +66,29 @@ public class TestLeveldbRMStateStore extends RMStateStoreTestBase {
     FileUtil.fullyDelete(TEST_DIR);
   }
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testApps() throws Exception {
     LeveldbStateStoreTester tester = new LeveldbStateStoreTester();
     testRMAppStateStore(tester);
   }
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testClientTokens() throws Exception {
     LeveldbStateStoreTester tester = new LeveldbStateStoreTester();
     testRMDTSecretManagerStateStore(tester);
   }
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testVersion() throws Exception {
     LeveldbStateStoreTester tester = new LeveldbStateStoreTester();
     testCheckVersion(tester);
   }
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testEpoch() throws Exception {
     conf.setLong(YarnConfiguration.RM_EPOCH, epoch);
     conf.setLong(YarnConfiguration.RM_EPOCH_RANGE, getEpochRange());
@@ -91,49 +96,57 @@ public class TestLeveldbRMStateStore extends RMStateStoreTestBase {
     testEpoch(tester);
   }
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testAppDeletion() throws Exception {
     LeveldbStateStoreTester tester = new LeveldbStateStoreTester();
     testAppDeletion(tester);
   }
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testDeleteStore() throws Exception {
     LeveldbStateStoreTester tester = new LeveldbStateStoreTester();
     testDeleteStore(tester);
   }
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testRemoveApplication() throws Exception {
     LeveldbStateStoreTester tester = new LeveldbStateStoreTester();
     testRemoveApplication(tester);
   }
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testRemoveAttempt() throws Exception {
     LeveldbStateStoreTester tester = new LeveldbStateStoreTester();
     testRemoveAttempt(tester);
   }
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testAMTokens() throws Exception {
     LeveldbStateStoreTester tester = new LeveldbStateStoreTester();
     testAMRMTokenSecretManagerStateStore(tester);
   }
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testReservation() throws Exception {
     LeveldbStateStoreTester tester = new LeveldbStateStoreTester();
     testReservationStateStore(tester);
   }
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testProxyCA() throws Exception {
     LeveldbStateStoreTester tester = new LeveldbStateStoreTester();
     testProxyCA(tester);
   }
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testCompactionCycle() {
     final DB mockdb = mock(DB.class);
     conf.setLong(YarnConfiguration.RM_LEVELDB_COMPACTION_INTERVAL_SECS, 1);

+ 6 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestMemoryRMStateStore.java

@@ -22,9 +22,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.Mockito.mock;
 
 public class TestMemoryRMStateStore {
@@ -47,8 +47,8 @@ public class TestMemoryRMStateStore {
     RMDelegationTokenIdentifier mockTokenId =
         mock(RMDelegationTokenIdentifier.class);
     store.removeRMDelegationToken(mockTokenId);
-    assertTrue("RMStateStore should have been in fenced state",
-        store.isFencedState());
+    assertTrue(store.isFencedState(),
+        "RMStateStore should have been in fenced state");
     store = new MemoryRMStateStore() {
       @Override
       public synchronized void removeRMDelegationToken(
@@ -61,7 +61,7 @@ public class TestMemoryRMStateStore {
     store.setResourceManager(mockRM);
     store.setRMDispatcher(new RMStateStoreTestBase.TestDispatcher());
     store.removeRMDelegationToken(mockTokenId);
-    assertTrue("RMStateStore should have been in fenced state",
-        store.isFencedState());
+    assertTrue(store.isFencedState(),
+        "RMStateStore should have been in fenced state");
   }
 }

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestProtos.java

@@ -18,7 +18,8 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.recovery;
 
-import static org.junit.Assert.*;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -27,8 +28,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateResponsePBImpl
 import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerRecoveryProtos.EpochProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateResponseProto;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 /**
  * Simple test to verify the protos generated are valid
@@ -39,7 +39,7 @@ public class TestProtos {
   public void testProtoCanBePrinted() throws Exception {
     EpochProto proto = EpochProto.newBuilder().setEpoch(100).build();
     String protoString = proto.toString();
-    Assert.assertNotNull(protoString);
+    assertNotNull(protoString);
   }
 
   @Test

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestRMStateStoreUtils.java

@@ -23,9 +23,9 @@ import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.RMDelegationTokenIdentifierData;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 public class TestRMStateStoreUtils {
 
@@ -64,8 +64,8 @@ public class TestRMStateStoreUtils {
 
     RMDelegationTokenIdentifierData identifierData
         = RMStateStoreUtils.readRMDelegationTokenIdentifierData(inBuf);
-    assertEquals("Found unexpected data still in the InputStream",
-        -1, inBuf.read());
+    assertEquals(-1, inBuf.read(),
+        "Found unexpected data still in the InputStream");
 
     RMDelegationTokenIdentifier identifier
         = identifierData.getTokenIdentifier();

+ 172 - 174
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java

@@ -69,9 +69,10 @@ import org.apache.hadoop.yarn.server.security.MasterKeyData;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooDefs.Perms;
 import org.apache.zookeeper.data.ACL;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import org.apache.hadoop.thirdparty.com.google.common.base.Joiner;
 import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap;
@@ -80,12 +81,12 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.assertArrayEquals;
-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.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.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;
@@ -125,13 +126,13 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
     return curatorFramework;
   }
 
-  @Before
+  @BeforeEach
   public void setupCurator() throws Exception {
     curatorTestingServer = setupCuratorServer();
     curatorFramework = setupCuratorFramework(curatorTestingServer);
   }
 
-  @After
+  @AfterEach
   public void cleanupCuratorServer() throws IOException {
     curatorFramework.close();
     curatorTestingServer.stop();
@@ -280,7 +281,8 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
     }
   }
 
-  @Test (timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testZKRMStateStoreRealZK() throws Exception {
     TestZKRMStateStoreTester zkTester = new TestZKRMStateStoreTester();
     testRMAppStateStore(zkTester);
@@ -333,7 +335,8 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
     ;
   }
 
-  @Test (timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testCheckMajorVersionChange() throws Exception {
     TestZKRMStateStoreTester zkTester = new TestZKRMStateStoreTester() {
       Version VERSION_INFO = Version.newInstance(Integer.MAX_VALUE, 0);
@@ -376,8 +379,8 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
     RMStateStore store = zkTester.getRMStateStore();
     Version defaultVersion = zkTester.getCurrentVersion();
     store.checkVersion();
-    assertEquals("Store had wrong version",
-        defaultVersion, store.loadVersion());
+    assertEquals(
+        defaultVersion, store.loadVersion(), "Store had wrong version");
   }
 
   public static Configuration createHARMConf(String rmIds, String rmId,
@@ -487,22 +490,22 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
     ResourceManager rm1 = new MockRM(conf1);
     rm1.start();
     rm1.getRMContext().getRMAdminService().transitionToActive(req);
-    assertEquals("RM with ZKStore didn't start",
-        Service.STATE.STARTED, rm1.getServiceState());
-    assertEquals("RM should be Active",
-        HAServiceProtocol.HAServiceState.ACTIVE,
-        rm1.getRMContext().getRMAdminService().getServiceStatus().getState());
+    assertEquals(Service.STATE.STARTED, rm1.getServiceState(),
+        "RM with ZKStore didn't start");
+    assertEquals(HAServiceProtocol.HAServiceState.ACTIVE,
+        rm1.getRMContext().getRMAdminService().getServiceStatus().getState(),
+        "RM should be Active");
 
     Configuration conf2 =
         createHARMConf("rm1,rm2", "rm2", 5678, false, curatorTestingServer);
     ResourceManager rm2 = new MockRM(conf2);
     rm2.start();
     rm2.getRMContext().getRMAdminService().transitionToActive(req);
-    assertEquals("RM with ZKStore didn't start",
-        Service.STATE.STARTED, rm2.getServiceState());
-    assertEquals("RM should be Active",
-        HAServiceProtocol.HAServiceState.ACTIVE,
-        rm2.getRMContext().getRMAdminService().getServiceStatus().getState());
+    assertEquals(Service.STATE.STARTED, rm2.getServiceState(),
+        "RM with ZKStore didn't start");
+    assertEquals(HAServiceProtocol.HAServiceState.ACTIVE,
+        rm2.getRMContext().getRMAdminService().getServiceStatus().getState(),
+        "RM should be Active");
 
     for (int i = 0; i < ZK_TIMEOUT_MS / 50; i++) {
       if (HAServiceProtocol.HAServiceState.ACTIVE ==
@@ -510,12 +513,12 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
         Thread.sleep(100);
       }
     }
-    assertEquals("RM should have been fenced",
-        HAServiceProtocol.HAServiceState.STANDBY,
-        rm1.getRMContext().getRMAdminService().getServiceStatus().getState());
-    assertEquals("RM should be Active",
-        HAServiceProtocol.HAServiceState.ACTIVE,
-        rm2.getRMContext().getRMAdminService().getServiceStatus().getState());
+    assertEquals(HAServiceProtocol.HAServiceState.STANDBY,
+        rm1.getRMContext().getRMAdminService().getServiceStatus().getState(),
+        "RM should have been fenced");
+    assertEquals(HAServiceProtocol.HAServiceState.ACTIVE,
+        rm2.getRMContext().getRMAdminService().getServiceStatus().getState(),
+        "RM should be Active");
     rm1.close();
     rm2.close();
   }
@@ -527,8 +530,8 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
 
     // Move state to FENCED from ACTIVE
     store.updateFencedState();
-    assertEquals("RMStateStore should have been in fenced state",
-            true, store.isFencedState());    
+    assertEquals(true, store.isFencedState(),
+        "RMStateStore should have been in fenced state");
 
     long submitTime = System.currentTimeMillis();
     long startTime = submitTime + 1000;
@@ -542,8 +545,8 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
     when(mockApp.getApplicationSubmissionContext()).thenReturn(context);
     when(mockApp.getUser()).thenReturn("test");
     store.storeNewApplication(mockApp);
-    assertEquals("RMStateStore should have been in fenced state",
-            true, store.isFencedState());
+    assertEquals(true, store.isFencedState(),
+        "RMStateStore should have been in fenced state");
 
     // Add a new attempt
     ClientToAMTokenSecretManagerInRM clientToAMTokenMgr =
@@ -566,8 +569,8 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
     when(mockRmAppAttemptMetrics.getAggregateAppResourceUsage())
         .thenReturn(new AggregateAppResourceUsage(new HashMap<>()));
     store.storeNewApplicationAttempt(mockAttempt);
-    assertEquals("RMStateStore should have been in fenced state",
-            true, store.isFencedState());
+    assertEquals(true, store.isFencedState(),
+        "RMStateStore should have been in fenced state");
 
     long finishTime = submitTime + 1000;
     // Update attempt
@@ -578,20 +581,20 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
             "test", FinalApplicationStatus.SUCCEEDED, 100, 
             finishTime, new HashMap<>(), new HashMap<>(), 0);
     store.updateApplicationAttemptState(newAttemptState);
-    assertEquals("RMStateStore should have been in fenced state",
-            true, store.isFencedState());
+    assertEquals(true, store.isFencedState(),
+        "RMStateStore should have been in fenced state");
 
     // Update app
     ApplicationStateData appState = ApplicationStateData.newInstance(submitTime, 
             startTime, context, "test");
     store.updateApplicationState(appState);
-    assertEquals("RMStateStore should have been in fenced state",
-            true, store.isFencedState());
+    assertEquals(true, store.isFencedState(),
+        "RMStateStore should have been in fenced state");
 
     // Remove app
     store.removeApplication(mockApp);
-    assertEquals("RMStateStore should have been in fenced state",
-            true, store.isFencedState());
+    assertEquals(true, store.isFencedState(),
+        "RMStateStore should have been in fenced state");
 
     // store RM delegation token;
     RMDelegationTokenIdentifier dtId1 =
@@ -599,43 +602,40 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
             new Text("renewer1"), new Text("realuser1"));
     Long renewDate1 = new Long(System.currentTimeMillis()); 
     dtId1.setSequenceNumber(1111);
-    assertFalse("Token " + dtId1
-        + " should not exist but was found in ZooKeeper",
-        zkTester.delegationTokenExists(dtId1, 0));
+    assertFalse(zkTester.delegationTokenExists(dtId1, 0),
+        "Token " + dtId1 + " should not exist but was found in ZooKeeper");
     store.storeRMDelegationToken(dtId1, renewDate1);
-    assertFalse("Token " + dtId1
-        + " should not exist but was found in ZooKeeper",
-        zkTester.delegationTokenExists(dtId1, 0));
-    assertEquals("RMStateStore should have been in fenced state", true,
-        store.isFencedState());
+    assertFalse(zkTester.delegationTokenExists(dtId1, 0),
+        "Token " + dtId1 + " should not exist but was found in ZooKeeper");
+    assertEquals(true, store.isFencedState(),
+        "RMStateStore should have been in fenced state");
 
     store.updateRMDelegationToken(dtId1, renewDate1);
-    assertFalse("Token " + dtId1
-        + " should not exist but was found in ZooKeeper",
-        zkTester.delegationTokenExists(dtId1, 0));
-    assertEquals("RMStateStore should have been in fenced state", true,
-        store.isFencedState());
+    assertFalse(zkTester.delegationTokenExists(dtId1, 0), "Token " + dtId1
+        + " should not exist but was found in ZooKeeper");
+    assertEquals(true, store.isFencedState(),
+        "RMStateStore should have been in fenced state");
 
     // remove delegation key;
     store.removeRMDelegationToken(dtId1);
-    assertEquals("RMStateStore should have been in fenced state", true,
-        store.isFencedState());
+    assertEquals(true, store.isFencedState(),
+        "RMStateStore should have been in fenced state");
 
     // store delegation master key;
     DelegationKey key = new DelegationKey(1234, 4321, "keyBytes".getBytes());
     store.storeRMDTMasterKey(key);
-    assertEquals("RMStateStore should have been in fenced state", true,
-        store.isFencedState());
+    assertEquals(true, store.isFencedState(),
+        "RMStateStore should have been in fenced state");
 
     // remove delegation master key;
     store.removeRMDTMasterKey(key);
-    assertEquals("RMStateStore should have been in fenced state", true,
-        store.isFencedState());
+    assertEquals(true, store.isFencedState(),
+        "RMStateStore should have been in fenced state");
 
     // store or update AMRMToken;
     store.storeOrUpdateAMRMTokenSecretManager(null, false);
-    assertEquals("RMStateStore should have been in fenced state", true,
-        store.isFencedState());
+    assertEquals(true, store.isFencedState(),
+        "RMStateStore should have been in fenced state");
 
     store.close();
   }
@@ -707,19 +707,20 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
       ApplicationId appId, String user, long submitTime, long startTime,
       RMAppState state, long finishTime, String diagnostics) {
     // Check if app is loaded correctly
-    assertNotNull("App " + appId + " should have been loaded.", appState);
-    assertEquals("App submit time in app state", submitTime,
-        appState.getSubmitTime());
-    assertEquals("App start time in app state", startTime,
-        appState.getStartTime());
-    assertEquals("App ID in app state", appId,
-        appState.getApplicationSubmissionContext().getApplicationId());
-    assertEquals("App state", state, appState.getState());
-    assertEquals("Finish time in app state", finishTime,
-        appState.getFinishTime());
-    assertEquals("User in app state", user, appState.getUser());
-    assertEquals("Diagnostics in app state", diagnostics,
-        appState.getDiagnostics());
+    assertNotNull(appState, "App " + appId + " should have been loaded.");
+    assertEquals(submitTime,
+        appState.getSubmitTime(), "App submit time in app state");
+    assertEquals(startTime,
+        appState.getStartTime(), "App start time in app state");
+    assertEquals(appId,
+        appState.getApplicationSubmissionContext().getApplicationId(),
+        "App ID in app state");
+    assertEquals(state, appState.getState(), "App state");
+    assertEquals(finishTime,
+        appState.getFinishTime(), "Finish time in app state");
+    assertEquals(user, appState.getUser(), "User in app state");
+    assertEquals(diagnostics,
+        appState.getDiagnostics(), "Diagnostics in app state");
   }
 
   private static void verifyLoadedApp(RMState rmState,
@@ -737,14 +738,14 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
     Map<ApplicationId, ApplicationStateData> rmAppState =
         rmState.getApplicationState();
     ApplicationStateData appState = rmAppState.get(appId);
-    assertNotNull(appId + " is not there in loaded apps", appState);
+    assertNotNull(appState, appId + " is not there in loaded apps");
     verifyLoadedApp(appState, appId, "test", submitTime, startTime,
         isFinished ? RMAppState.FINISHED : null, finishTime,
         isFinished ? "appDiagnostics" : "");
     // Check attempt state.
     if (attempts != null) {
-      assertEquals("Attempts loaded for app " + appId, attempts.size(),
-          appState.attempts.size());
+      assertEquals(attempts.size(),
+          appState.attempts.size(), "Attempts loaded for app " + appId);
       if (finalStatuses != null && amExitStatuses != null) {
         for (int i = 0; i < attempts.size(); i++) {
           if (finalStatuses.get(i) != null) {
@@ -757,8 +758,8 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
         }
       }
     } else {
-      assertEquals(
-          "Attempts loaded for app " + appId, 0, appState.attempts.size());
+      assertEquals(0, appState.attempts.size(),
+          "Attempts loaded for app " + appId);
     }
   }
 
@@ -778,28 +779,29 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
       int amExitStatus, FinalApplicationStatus finalStatus) {
     ApplicationAttemptStateData attemptState = appState.getAttempt(attemptId);
     // Check if attempt is loaded correctly
-    assertNotNull(
-        "Attempt " + attemptId + " should have been loaded.", attemptState);
-    assertEquals("Attempt Id in attempt state",
-        attemptId, attemptState.getAttemptId());
-    assertEquals("Master Container Id in attempt state",
-        masterContainerId, attemptState.getMasterContainer().getId());
+    assertNotNull(attemptState,
+        "Attempt " + attemptId + " should have been loaded.");
+    assertEquals(attemptId, attemptState.getAttemptId(),
+        "Attempt Id in attempt state");
+    assertEquals(masterContainerId, attemptState.getMasterContainer().getId(),
+        "Master Container Id in attempt state");
     if (null != clientTokenKey) {
-      assertArrayEquals("Client token key in attempt state",
-          clientTokenKey.getEncoded(), attemptState.getAppAttemptTokens().
-          getSecretKey(RMStateStore.AM_CLIENT_TOKEN_MASTER_KEY_NAME));
+      assertArrayEquals(clientTokenKey.getEncoded(),
+          attemptState.getAppAttemptTokens().
+          getSecretKey(RMStateStore.AM_CLIENT_TOKEN_MASTER_KEY_NAME),
+          "Client token key in attempt state");
     }
-    assertEquals("Attempt state", state, attemptState.getState());
-    assertEquals("Finish time in attempt state", finishTime,
-        attemptState.getFinishTime());
-    assertEquals("Diagnostics in attempt state", diagnostics,
-        attemptState.getDiagnostics());
-    assertEquals("AM Container exit status in attempt state", amExitStatus,
-        attemptState.getAMContainerExitStatus());
-    assertEquals("Final app status in attempt state", finalStatus,
-        attemptState.getFinalApplicationStatus());
-    assertEquals("Tracking URL in attempt state", trackingURL,
-        attemptState.getFinalTrackingUrl());
+    assertEquals(state, attemptState.getState(), "Attempt state");
+    assertEquals(finishTime, attemptState.getFinishTime(),
+        "Finish time in attempt state");
+    assertEquals(diagnostics, attemptState.getDiagnostics(),
+        "Diagnostics in attempt state");
+    assertEquals(amExitStatus, attemptState.getAMContainerExitStatus(),
+        "AM Container exit status in attempt state");
+    assertEquals(finalStatus, attemptState.getFinalApplicationStatus(),
+        "Final app status in attempt state");
+    assertEquals(trackingURL, attemptState.getFinalTrackingUrl(),
+        "Tracking URL in attempt state");
   }
 
   private static ApplicationStateData createAppState(
@@ -931,8 +933,9 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
         ZKRMStateStore.ROOT_ZNODE_NAME, ZKRMStateStore.RM_APP_ROOT,
         ZKRMStateStore.RM_APP_ROOT_HIERARCHIES, String.valueOf(splitIndex),
         appParent, appPath);
-    assertTrue("Application with id " + appIdStr + " does not exist as per " +
-        "split in state store.", ((ZKRMStateStore)store).exists(path));
+    assertTrue(((ZKRMStateStore)store).exists(path),
+        "Application with id " + appIdStr + " does not exist as per " +
+        "split in state store.");
   }
 
   private static void verifyAppInHierarchicalPath(RMStateStore store,
@@ -946,8 +949,8 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
     } else {
       path = createPath(path, appId);
     }
-    assertTrue(appId + " should exist in path " + path,
-        ((ZKRMStateStore)store).exists(createPath(path)));
+    assertTrue(((ZKRMStateStore)store).exists(createPath(path)),
+        appId + " should exist in path " + path);
   }
 
   private static void assertHierarchicalPaths(RMStateStore store,
@@ -959,9 +962,8 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
         path = createPath(path, ZKRMStateStore.RM_APP_ROOT_HIERARCHIES,
             String.valueOf(entry.getKey()));
       }
-      assertEquals("Number of childrens for path " + path,
-          (int) entry.getValue(),
-          ((ZKRMStateStore)store).getChildren(path).size());
+      assertEquals((int) entry.getValue(), ((ZKRMStateStore)store).getChildren(path).size(),
+          "Number of childrens for path " + path);
     }
   }
 
@@ -1056,7 +1058,7 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
         FinalApplicationStatus.SUCCEEDED));
 
     // assert store is in expected state after everything is cleaned
-    assertTrue("Store is not in expected state", zkTester.isFinalStateValid());
+    assertTrue(zkTester.isFinalStateValid(), "Store is not in expected state");
     store.close();
   }
 
@@ -1128,8 +1130,8 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
     store = zkTester.getRMStateStore(createConfForAppNodeSplit(0));
     store.setRMDispatcher(dispatcher);
     state = store.loadState();
-    assertEquals("Number of Apps loaded should be 4.", 4,
-        state.getApplicationState().size());
+    assertEquals(4, state.getApplicationState().size(),
+        "Number of Apps loaded should be 4.");
     verifyLoadedApp(state, appId1, submitTime, startTime, 1234, true,
         Lists.newArrayList(attemptId1, attemptId2), Lists.newArrayList(-1000,
         100), Lists.newArrayList(null, FinalApplicationStatus.SUCCEEDED));
@@ -1144,8 +1146,8 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
     store = zkTester.getRMStateStore(createConfForAppNodeSplit(3));
     store.setRMDispatcher(dispatcher);
     state = store.loadState();
-    assertEquals("Number of apps loaded should be 5.", 5,
-        state.getApplicationState().size());
+    assertEquals(5, state.getApplicationState().size(),
+        "Number of apps loaded should be 5.");
     verifyLoadedApp(state, dummyAppId, submitTime, startTime, 1234, true,
         Lists.newArrayList(dummyAttemptId), Lists.newArrayList(111),
         Lists.newArrayList(FinalApplicationStatus.SUCCEEDED));
@@ -1192,8 +1194,8 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
     store = zkTester.getRMStateStore(createConfForAppNodeSplit(3));
     store.setRMDispatcher(dispatcher);
     state = store.loadState();
-    assertEquals("Number of apps loaded should be 0.", 0,
-        state.getApplicationState().size());
+    assertEquals(0, state.getApplicationState().size(),
+        "Number of apps loaded should be 0.");
     // Close the state store.
     store.close();
   }
@@ -1213,16 +1215,15 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
       int sequenceNumber) throws Exception {
     RMStateStore.RMDTSecretManagerState secretManagerState =
         zkTester.store.loadState().getRMDTSecretManagerState();
-    assertEquals("Unexpected token state",
-        tokensWithRenewal, secretManagerState.getTokenState());
-    assertEquals("Unexpected sequence number", sequenceNumber,
-        secretManagerState.getDTSequenceNumber());
+    assertEquals(tokensWithRenewal, secretManagerState.getTokenState(),
+        "Unexpected token state");
+    assertEquals(sequenceNumber, secretManagerState.getDTSequenceNumber(),
+        "Unexpected sequence number");
     for (Map.Entry<RMDelegationTokenIdentifier, Integer> tokenEntry
         : tokensWithIndex.entrySet()) {
-      assertTrue("Expected to find token " + tokenEntry.getKey()
-          + " in zookeeper but did not",
-          zkTester.delegationTokenExists(tokenEntry.getKey(),
-          tokenEntry.getValue()));
+      assertTrue(zkTester.delegationTokenExists(tokenEntry.getKey(),
+          tokenEntry.getValue()), "Expected to find token " + tokenEntry.getKey()
+          + " in zookeeper but did not");
     }
   }
 
@@ -1233,12 +1234,12 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
         zkTester.store.loadState().getRMDTSecretManagerState();
     Map<RMDelegationTokenIdentifier, Long> tokenState =
         secretManagerState.getTokenState();
-    assertTrue("token state does not contain " + token,
-        tokenState.containsKey(token));
-    assertTrue("token state does not contain a token with renewal " + renewDate,
-        tokenState.containsValue(renewDate));
-    assertTrue("Token " + token + "should exist but was not found in ZooKeeper",
-        zkTester.delegationTokenExists(token, index));
+    assertTrue(tokenState.containsKey(token),
+        "token state does not contain " + token);
+    assertTrue(tokenState.containsValue(renewDate),
+        "token state does not contain a token with renewal " + renewDate);
+    assertTrue(zkTester.delegationTokenExists(token, index),
+        "Token " + token + "should exist but was not found in ZooKeeper");
   }
 
   private RMDelegationTokenIdentifier storeUpdateAndVerifyDelegationToken(
@@ -1250,8 +1251,8 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
     RMDelegationTokenIdentifier token =
         new RMDelegationTokenIdentifier(new Text("owner"),
             new Text("renewer"), new Text("realuser"));
-    assertFalse("Token should not exist but was found in ZooKeeper",
-        zkTester.delegationTokenExists(token, split));
+    assertFalse(zkTester.delegationTokenExists(token, split),
+        "Token should not exist but was found in ZooKeeper");
     token.setSequenceNumber(sequenceNumber);
     Long renewDate = System.currentTimeMillis();
     zkTester.store.storeRMDelegationToken(token, renewDate);
@@ -1281,28 +1282,28 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
     // Valid values
     TestZKRMStateStoreTester zkTester = new TestZKRMStateStoreTester();
     zkTester.getRMStateStore(createConfForDelegationTokenNodeSplit(0)).close();
-    assertEquals("Incorrect split index",
-        0, zkTester.getDelegationTokenNodeSplitIndex());
+    assertEquals(0, zkTester.getDelegationTokenNodeSplitIndex(),
+        "Incorrect split index");
     zkTester.getRMStateStore(createConfForDelegationTokenNodeSplit(1)).close();
-    assertEquals("Incorrect split index",
-        1, zkTester.getDelegationTokenNodeSplitIndex());
+    assertEquals(1, zkTester.getDelegationTokenNodeSplitIndex(),
+        "Incorrect split index");
     zkTester.getRMStateStore(createConfForDelegationTokenNodeSplit(2)).close();
-    assertEquals("Incorrect split index",
-        2, zkTester.getDelegationTokenNodeSplitIndex());
+    assertEquals(2, zkTester.getDelegationTokenNodeSplitIndex(),
+        "Incorrect split index");
     zkTester.getRMStateStore(createConfForDelegationTokenNodeSplit(3)).close();
-    assertEquals("Incorrect split index",
-        3, zkTester.getDelegationTokenNodeSplitIndex());
+    assertEquals(3, zkTester.getDelegationTokenNodeSplitIndex(),
+        "Incorrect split index");
     zkTester.getRMStateStore(createConfForDelegationTokenNodeSplit(4)).close();
-    assertEquals("Incorrect split index",
-        4, zkTester.getDelegationTokenNodeSplitIndex());
+    assertEquals(4, zkTester.getDelegationTokenNodeSplitIndex(),
+        "Incorrect split index");
 
     // Invalid values --> override to 0
     zkTester.getRMStateStore(createConfForDelegationTokenNodeSplit(-1)).close();
-    assertEquals("Incorrect split index",
-        0, zkTester.getDelegationTokenNodeSplitIndex());
+    assertEquals(0, zkTester.getDelegationTokenNodeSplitIndex(),
+        "Incorrect split index");
     zkTester.getRMStateStore(createConfForDelegationTokenNodeSplit(5)).close();
-    assertEquals("Incorrect split index",
-        0, zkTester.getDelegationTokenNodeSplitIndex());
+    assertEquals(0, zkTester.getDelegationTokenNodeSplitIndex(),
+        "Incorrect split index");
   }
 
   @Test
@@ -1348,12 +1349,12 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
         store.loadState().getRMDTSecretManagerState();
     tokensWithRenewal.clear();
     tokensWithIndex.clear();
-    assertEquals("Unexpected token state",
-        tokensWithRenewal, state.getTokenState());
-    assertEquals("Unexpected sequence number",
-        sequenceNumber, state.getDTSequenceNumber());
-    assertFalse("Token should not exist but was found in ZooKeeper",
-        zkTester.delegationTokenExists(token, split));
+    assertEquals(tokensWithRenewal, state.getTokenState(),
+        "Unexpected token state");
+    assertEquals(sequenceNumber, state.getDTSequenceNumber(),
+        "Unexpected sequence number");
+    assertFalse(zkTester.delegationTokenExists(token, split),
+        "Token should not exist but was found in ZooKeeper");
     store.close();
   }
 
@@ -1375,8 +1376,8 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
               new Text("renewer" + i), new Text("realuser" + i));
       sequenceNumber = i;
       token.setSequenceNumber(sequenceNumber);
-      assertFalse("Token should not exist but was found in ZooKeeper",
-          zkTester.delegationTokenExists(token, 1));
+      assertFalse(zkTester.delegationTokenExists(token, 1),
+          "Token should not exist but was found in ZooKeeper");
       Long renewDate = System.currentTimeMillis();
       store.storeRMDelegationToken(token, renewDate);
       modifyRMDelegationTokenState();
@@ -1419,9 +1420,8 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
     verifyDelegationTokensStateStore(zkTester, tokensWithRenewal,
         tokensWithIndex, sequenceNumber);
     for (RMDelegationTokenIdentifier token : tokensToDelete) {
-      assertFalse("Token " + token
-              + " should not exist but was found in ZooKeeper",
-          zkTester.delegationTokenExists(token, 1));
+      assertFalse(zkTester.delegationTokenExists(token, 1),
+          "Token " + token + " should not exist but was found in ZooKeeper");
     }
     store.close();
   }
@@ -1495,15 +1495,12 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
     tokensWithIndex.clear();
     verifyDelegationTokensStateStore(
         zkTester, tokensWithRenewal, tokensWithIndex, sequenceNumber);
-    assertFalse("Token " + token1
-            + " should not exist but was found in ZooKeeper",
-        zkTester.delegationTokenExists(token1, 1));
-    assertFalse("Token " + token1
-            + " should not exist but was found in ZooKeeper",
-        zkTester.delegationTokenExists(token2, 2));
-    assertFalse("Token " + token1
-            + " should not exist but was found in ZooKeeper",
-        zkTester.delegationTokenExists(token0, 0));
+    assertFalse(zkTester.delegationTokenExists(token1, 1),
+        "Token " + token1 + " should not exist but was found in ZooKeeper");
+    assertFalse(zkTester.delegationTokenExists(token2, 2), "Token " + token1 +
+        " should not exist but was found in ZooKeeper");
+    assertFalse(zkTester.delegationTokenExists(token0, 0), "Token " + token1 +
+        " should not exist but was found in ZooKeeper");
     // Store a token with index 3
     sequenceNumber++;
     storeUpdateAndVerifyDelegationToken(zkTester, tokensWithRenewal,
@@ -1550,10 +1547,11 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
     ctx = rmState.getApplicationState().get(appId)
         .getApplicationSubmissionContext();
 
-    assertEquals("ApplicationSchedulingPropertiesMap should not have been "
+    assertEquals(schedulingPropertiesMap,
+        ctx.getApplicationSchedulingPropertiesMap(),
+        "ApplicationSchedulingPropertiesMap should not have been "
         + "pruned from the application submission context before the "
-        + "FINISHED state",
-        schedulingPropertiesMap, ctx.getApplicationSchedulingPropertiesMap());
+        + "FINISHED state");
 
     appState.setState(RMAppState.FINISHED);
     store.handleStoreEvent(new RMStateUpdateAppEvent(appState, false, null));
@@ -1566,9 +1564,9 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
     assertEquals("a_queue", ctx.getQueue());
     assertNotNull(ctx.getAMContainerSpec());
     assertEquals(17L, ctx.getResource().getMemorySize());
-    assertEquals("ApplicationSchedulingPropertiesMap should have been pruned"
-        + " from the application submission context when in FINISHED STATE",
-        Collections.emptyMap(), ctx.getApplicationSchedulingPropertiesMap());
+    assertEquals(Collections.emptyMap(), ctx.getApplicationSchedulingPropertiesMap(),
+        "ApplicationSchedulingPropertiesMap should have been pruned"
+        + " from the application submission context when in FINISHED STATE");
     store.close();
   }
 
@@ -1589,8 +1587,8 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
     Thread.sleep(110);
 
     opDurations.getMetrics(collector, true);
-    assertEquals("Incorrect number of perf metrics", 1,
-        collector.getRecords().size());
+    assertEquals(1, collector.getRecords().size(),
+        "Incorrect number of perf metrics");
     MetricsRecord record = collector.getRecords().get(0);
     MetricsRecords.assertTag(record,
         ZKRMStateStoreOpDurations.RECORD_INFO.name(),

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStorePerf.java

@@ -45,9 +45,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.Applicatio
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
-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;
 
@@ -78,12 +78,12 @@ public class TestZKRMStateStorePerf extends RMStateStoreTestBase
   private ClientToAMTokenSecretManagerInRM clientToAMTokenMgr;
   private TestingServer curatorTestingServer;
 
-  @Before
+  @BeforeEach
   public void setUpZKServer() throws Exception {
     curatorTestingServer = new TestingServer();
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     if (store != null) {
       store.stop();

+ 16 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStoreZKClientConnections.java

@@ -29,16 +29,17 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStoreTestBa
 import org.apache.hadoop.util.ZKUtil;
 
 import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
-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.security.NoSuchAlgorithmException;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 public class TestZKRMStateStoreZKClientConnections {
   private static final Logger LOG =
@@ -60,13 +61,13 @@ public class TestZKRMStateStoreZKClientConnections {
 
   private TestingServer testingServer;
 
-  @Before
+  @BeforeEach
   public void setupZKServer() throws Exception {
     testingServer = new TestingServer();
     testingServer.start();
   }
 
-  @After
+  @AfterEach
   public void cleanupZKServer() throws Exception {
     testingServer.stop();
   }
@@ -96,7 +97,8 @@ public class TestZKRMStateStoreZKClientConnections {
     }
   }
 
-  @Test (timeout = 20000)
+  @Test
+  @Timeout(value = 20)
   public void testZKClientRetry() throws Exception {
     TestZKClient zkClientTester = new TestZKClient();
     final String path = "/test";
@@ -124,10 +126,11 @@ public class TestZKRMStateStoreZKClientConnections {
     Thread.sleep(2000);
     testingServer.start();
     clientThread.join();
-    Assert.assertFalse(assertionFailedInThread.get());
+    assertFalse(assertionFailedInThread.get());
   }
 
-  @Test(timeout = 20000)
+  @Test
+  @Timeout(value = 20)
   public void testSetZKAcl() {
     TestZKClient zkClientTester = new TestZKClient();
     YarnConfiguration conf = new YarnConfiguration();
@@ -140,7 +143,8 @@ public class TestZKRMStateStoreZKClientConnections {
     }
   }
 
-  @Test(timeout = 20000)
+  @Test
+  @Timeout(value = 20)
   public void testInvalidZKAclConfiguration() {
     TestZKClient zkClientTester = new TestZKClient();
     YarnConfiguration conf = new YarnConfiguration();

+ 5 - 13
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/BaseSharingPolicyTest.java

@@ -17,8 +17,8 @@
  *******************************************************************************/
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
-import static junit.framework.TestCase.fail;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.Mockito.mock;
 
 import java.io.IOException;
@@ -33,33 +33,25 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
-import org.junit.Before;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.BeforeEach;
 
 import net.jcip.annotations.NotThreadSafe;
 
 /**
  * This class is a base test for {@code SharingPolicy} implementors.
  */
-@RunWith(value = Parameterized.class)
 @NotThreadSafe
 @SuppressWarnings("VisibilityModifier")
 public abstract class BaseSharingPolicyTest {
 
-  @Parameterized.Parameter(value = 0)
   public long duration;
 
-  @Parameterized.Parameter(value = 1)
   public double height;
 
-  @Parameterized.Parameter(value = 2)
   public int numSubmissions;
 
-  @Parameterized.Parameter(value = 3)
   public String recurrenceExpression;
 
-  @Parameterized.Parameter(value = 4)
   public Class expectedError;
 
   private long step;
@@ -75,7 +67,7 @@ public abstract class BaseSharingPolicyTest {
 
   protected ReservationSchedulerConfiguration conf;
 
-  @Before
+  @BeforeEach
   public void setup() {
     // 1 sec step
     step = 1000L;
@@ -138,7 +130,7 @@ public abstract class BaseSharingPolicyTest {
                 "dedicated", rstart, rend, reservationIntervalResourceMap, res,
                 minAlloc);
 
-        assertTrue(plan.toString(), plan.addReservation(resAlloc, false));
+        assertTrue(plan.addReservation(resAlloc, false), plan.toString());
       }
       // fail if error was expected
       if (expectedError != null) {

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

@@ -17,6 +17,10 @@
  *****************************************************************************/
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anySet;
 import static org.mockito.Mockito.doReturn;
@@ -65,8 +69,7 @@ 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.Assert;
-import org.mockito.Mockito;
+import org.junit.jupiter.api.Assertions;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
@@ -112,13 +115,13 @@ public class ReservationSystemTestUtil {
   public static void validateReservationQueue(
       AbstractReservationSystem reservationSystem, String planQName) {
     Plan plan = reservationSystem.getPlan(planQName);
-    Assert.assertNotNull(plan);
-    Assert.assertTrue(plan instanceof InMemoryPlan);
-    Assert.assertEquals(planQName, plan.getQueueName());
-    Assert.assertEquals(8192, plan.getTotalCapacity().getMemorySize());
-    Assert.assertTrue(
+    assertNotNull(plan);
+    assertTrue(plan instanceof InMemoryPlan);
+    assertEquals(planQName, plan.getQueueName());
+    assertEquals(8192, plan.getTotalCapacity().getMemorySize());
+    assertTrue(
         plan.getReservationAgent() instanceof AlignedPlannerWithGreedy);
-    Assert
+    Assertions
         .assertTrue(plan.getSharingPolicy() instanceof CapacityOverTimePolicy);
   }
 
@@ -247,7 +250,7 @@ public class ReservationSystemTestUtil {
     CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
     setupQueueConfiguration(conf);
 
-    CapacityScheduler cs = Mockito.spy(new CapacityScheduler());
+    CapacityScheduler cs = spy(new CapacityScheduler());
     cs.setConf(new YarnConfiguration());
 
     RMContext mockRmContext = createRMContext(conf);
@@ -256,7 +259,7 @@ public class ReservationSystemTestUtil {
     try {
       cs.serviceInit(conf);
     } catch (Exception e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
 
     initializeRMContext(numContainers, cs, mockRmContext);
@@ -273,7 +276,7 @@ public class ReservationSystemTestUtil {
   }
 
   public static RMContext createRMContext(Configuration conf) {
-    RMContext mockRmContext = Mockito.spy(new RMContextImpl(null, null, null,
+    RMContext mockRmContext = spy(new RMContextImpl(null, null, null,
         null, null, null, new RMContainerTokenSecretManager(conf),
         new NMTokenSecretManagerInRM(conf),
         new ClientToAMTokenSecretManagerInRM(), null));

+ 20 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityOverTimePolicy.java

@@ -25,14 +25,12 @@ import net.jcip.annotations.NotThreadSafe;
 
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningQuotaException;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 
 /**
  * This class tests the {@code CapacityOvertimePolicy} sharing policy.
  */
-@RunWith(value = Parameterized.class)
 @NotThreadSafe
 @SuppressWarnings("VisibilityModifier")
 public class TestCapacityOverTimePolicy extends BaseSharingPolicyTest {
@@ -43,8 +41,16 @@ public class TestCapacityOverTimePolicy extends BaseSharingPolicyTest {
   final static String TWOHOURPERIOD = "7200000";
   final static String ONEDAYPERIOD = "86400000";
 
-  @Parameterized.Parameters(name = "Duration {0}, height {1}," +
-          " numSubmission {2}, periodic {3})")
+  public void initTestCapacityOverTimePolicy(long pDuration,
+      double pHeight, int pNumSubmissions, String pRecurrenceExpression, Class pExpectedError) {
+    this.duration = pDuration;
+    this.height = pHeight;
+    this.numSubmissions = pNumSubmissions;
+    this.recurrenceExpression = pRecurrenceExpression;
+    this.expectedError = pExpectedError;
+    super.setup();
+  }
+
   public static Collection<Object[]> data() {
     return Arrays.asList(new Object[][] {
 
@@ -131,8 +137,14 @@ public class TestCapacityOverTimePolicy extends BaseSharingPolicyTest {
     return policy;
   }
 
-  @Test
-  public void testAllocation() throws IOException, PlanningException {
+  @ParameterizedTest(name = "Duration {0}, height {1}," +
+      " numSubmission {2}, periodic {3})")
+  @MethodSource("data")
+  public void testAllocation(long pDuration, double pHeight, int pNumSubmissions,
+      String pRecurrenceExpression, Class pExpectedError)
+      throws IOException, PlanningException {
+    initTestCapacityOverTimePolicy(pDuration, pHeight, pNumSubmissions,
+        pRecurrenceExpression, pExpectedError);
     runTest();
   }
 

+ 13 - 18
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacitySchedulerPlanFollower.java

@@ -17,9 +17,10 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
-import static org.junit.Assert.assertNotNull;
-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.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
@@ -50,12 +51,9 @@ import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.util.Clock;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.mockito.ArgumentMatchers;
 import org.mockito.Mockito;
 
@@ -67,10 +65,7 @@ public class TestCapacitySchedulerPlanFollower extends
   private CapacitySchedulerContext csContext;
   private CapacityScheduler cs;
 
-  @Rule
-  public TestName name = new TestName();
-
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     CapacityScheduler spyCs = new CapacityScheduler();
     cs = spy(spyCs);
@@ -161,8 +156,8 @@ public class TestCapacitySchedulerPlanFollower extends
   @Override
   protected void checkDefaultQueueBeforePlanFollowerRun(){
     Queue defQ = getDefaultQueue();
-    Assert.assertEquals(0, getNumberOfApplications(defQ));
-    Assert.assertNotNull(defQ);
+    assertEquals(0, getNumberOfApplications(defQ));
+    assertNotNull(defQ);
   }
 
   @Override
@@ -196,8 +191,8 @@ public class TestCapacitySchedulerPlanFollower extends
       double expectedCapacity, double expectedMaxCapacity) {
     CSQueue q = cs.getQueue(r2.toString());
     assertNotNull(q);
-    Assert.assertEquals(expectedCapacity, q.getCapacity(), 0.01);
-    Assert.assertEquals(expectedMaxCapacity, q.getMaximumCapacity(), 1.0);
+    assertEquals(expectedCapacity, q.getCapacity(), 0.01);
+    assertEquals(expectedMaxCapacity, q.getMaximumCapacity(), 1.0);
   }
 
   @Override
@@ -211,7 +206,7 @@ public class TestCapacitySchedulerPlanFollower extends
     return new ApplicationACLsManager(conf);
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     if (scheduler != null) {
       cs.stop();

+ 11 - 16
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestFairSchedulerPlanFollower.java

@@ -17,9 +17,10 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
-import static org.junit.Assert.assertNotNull;
-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.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
@@ -50,12 +51,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedule
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedulerTestBase;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.util.Clock;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.mockito.ArgumentMatchers;
 import org.mockito.Mockito;
 
@@ -70,9 +68,6 @@ public class TestFairSchedulerPlanFollower extends
   private Configuration conf;
   private FairSchedulerTestBase testHelper = new FairSchedulerTestBase();
 
-  @Rule
-  public TestName name = new TestName();
-
   protected Configuration createConfiguration() {
     Configuration conf = testHelper.createConfiguration();
     conf.setClass(YarnConfiguration.RM_SCHEDULER, FairScheduler.class,
@@ -81,7 +76,7 @@ public class TestFairSchedulerPlanFollower extends
     return conf;
   }
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     conf = createConfiguration();
     ReservationSystemTestUtil.setupFSAllocationFile(ALLOC_FILE);
@@ -132,7 +127,7 @@ public class TestFairSchedulerPlanFollower extends
 
   @Override
   protected void checkDefaultQueueBeforePlanFollowerRun() {
-    Assert.assertNull(getDefaultQueue());
+    assertNull(getDefaultQueue());
   }
   @Override
   protected void verifyCapacity(Queue defQ) {
@@ -172,7 +167,7 @@ public class TestFairSchedulerPlanFollower extends
             false);
     assertNotNull(q);
     // For now we are setting both to same weight
-    Assert.assertEquals(expectedCapacity, q.getWeight(), 0.01);
+    assertEquals(expectedCapacity, q.getWeight(), 0.01);
   }
 
   @Override
@@ -192,7 +187,7 @@ public class TestFairSchedulerPlanFollower extends
     return new ApplicationACLsManager(conf);
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     if (scheduler != null) {
       fs.stop();

+ 104 - 98
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryPlan.java

@@ -17,7 +17,12 @@
  *******************************************************************************/
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.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;
 
@@ -46,10 +51,9 @@ import org.apache.hadoop.util.Clock;
 import org.apache.hadoop.util.UTCClock;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 /**
  * Testing the class {@link InMemoryPlan}.
@@ -72,7 +76,7 @@ public class TestInMemoryPlan {
   private RMContext context;
   private long maxPeriodicity;
 
-  @Before
+  @BeforeEach
   public void setUp() throws PlanningException {
     resCalc = new DefaultResourceCalculator();
     minAlloc = Resource.newInstance(1024, 1);
@@ -89,7 +93,7 @@ public class TestInMemoryPlan {
     context = ReservationSystemTestUtil.createMockRMContext();
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
     resCalc = null;
     minAlloc = null;
@@ -112,26 +116,28 @@ public class TestInMemoryPlan {
     int start = 100;
     ReservationAllocation rAllocation =
         createReservationAllocation(reservationID, start, alloc);
-    Assert.assertNull(plan.getReservationById(reservationID));
+    assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
     } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
     checkAllocation(plan, alloc, start, 0);
   }
 
-  @Test(expected = PlanningException.class)
+  @Test
   public void testOutOfRange() throws PlanningException {
-    maxPeriodicity = 100;
-    Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
-        resCalc, minAlloc, maxAlloc, planName, replanner, true, maxPeriodicity,
-        context, new UTCClock());
-
-    // we expect the plan to complaint as the range 330-150 > 50
-    RLESparseResourceAllocation availableBefore =
-        plan.getAvailableResourceOverTime(user, null, 150, 330, 50);
+    assertThrows(PlanningException.class, () -> {
+      maxPeriodicity = 100;
+      Plan plan = new InMemoryPlan(queueMetrics, policy, agent, totalCapacity, 1L,
+          resCalc, minAlloc, maxAlloc, planName, replanner, true, maxPeriodicity,
+          context, new UTCClock());
+
+      // we expect the plan to complaint as the range 330-150 > 50
+      RLESparseResourceAllocation availableBefore =
+          plan.getAvailableResourceOverTime(user, null, 150, 330, 50);
+    });
   }
 
   @Test
@@ -151,11 +157,11 @@ public class TestInMemoryPlan {
         reservationID, start, alloc, String.valueOf(period));
     // use periodicity of 1hr
     rAllocation.setPeriodicity(period);
-    Assert.assertNull(plan.getReservationById(reservationID));
+    assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
     } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
     checkAllocation(plan, alloc, start, period);
@@ -192,10 +198,10 @@ public class TestInMemoryPlan {
         long y = 0;
         Resource res = Resource.newInstance(1024 * (alloc[i]), (alloc[i]));
         while (y <= end * 2) {
-          assertEquals("At time: " + start + i + y, res,
-              plan.getTotalCommittedResources(start + i + y));
-          assertEquals(" At time: " + (start + i + y), res,
-              userCons.getCapacityAtTime(start + i + y));
+          assertEquals(res, plan.getTotalCommittedResources(start + i + y),
+              "At time: " + start + i + y);
+          assertEquals(res, userCons.getCapacityAtTime(start + i + y),
+              " At time: " + (start + i + y));
           y = y + periodicity;
         }
       }
@@ -212,11 +218,11 @@ public class TestInMemoryPlan {
     int start = 100;
     ReservationAllocation rAllocation =
         createReservationAllocation(reservationID, start, alloc);
-    Assert.assertNull(plan.getReservationById(reservationID));
+    assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
     } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
   }
 
@@ -231,11 +237,11 @@ public class TestInMemoryPlan {
     int start = 100;
     ReservationAllocation rAllocation =
         createReservationAllocation(reservationID, start, alloc);
-    Assert.assertNull(plan.getReservationById(reservationID));
+    assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
     } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
     checkAllocation(plan, alloc, start, 0);
@@ -243,11 +249,11 @@ public class TestInMemoryPlan {
     // Try to add it again
     try {
       plan.addReservation(rAllocation, false);
-      Assert.fail("Add should fail as it already exists");
+      fail("Add should fail as it already exists");
     } catch (IllegalArgumentException e) {
-      Assert.assertTrue(e.getMessage().endsWith("already exists"));
+      assertTrue(e.getMessage().endsWith("already exists"));
     } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
   }
@@ -263,11 +269,11 @@ public class TestInMemoryPlan {
     int start = 100;
     ReservationAllocation rAllocation =
         createReservationAllocation(reservationID, start, alloc);
-    Assert.assertNull(plan.getReservationById(reservationID));
+    assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
     } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
 
@@ -288,7 +294,7 @@ public class TestInMemoryPlan {
     try {
       plan.updateReservation(rAllocation);
     } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
 
@@ -319,11 +325,11 @@ public class TestInMemoryPlan {
     rAllocation.getReservationDefinition()
         .setRecurrenceExpression(String.valueOf(period));
     rAllocation.setPeriodicity(period);
-    Assert.assertNull(plan.getReservationById(reservationID));
+    assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
     } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
     System.out.println(plan.toString());
     doAssertions(plan, rAllocation);
@@ -340,7 +346,7 @@ public class TestInMemoryPlan {
     try {
       plan.updateReservation(rAllocation);
     } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
     checkAllocation(plan, updatedAlloc, start, period);
@@ -357,16 +363,16 @@ public class TestInMemoryPlan {
     int start = 100;
     ReservationAllocation rAllocation =
         createReservationAllocation(reservationID, start, alloc);
-    Assert.assertNull(plan.getReservationById(reservationID));
+    assertNull(plan.getReservationById(reservationID));
     try {
       plan.updateReservation(rAllocation);
-      Assert.fail("Update should fail as it does not exist in the plan");
+      fail("Update should fail as it does not exist in the plan");
     } catch (IllegalArgumentException e) {
-      Assert.assertTrue(e.getMessage().endsWith("does not exist in the plan"));
+      assertTrue(e.getMessage().endsWith("does not exist in the plan"));
     } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
-    Assert.assertNull(plan.getReservationById(reservationID));
+    assertNull(plan.getReservationById(reservationID));
   }
 
   @Test
@@ -380,11 +386,11 @@ public class TestInMemoryPlan {
     int start = 100;
     ReservationAllocation rAllocation =
         createReservationAllocation(reservationID, start, alloc, true);
-    Assert.assertNull(plan.getReservationById(reservationID));
+    assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
     } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
 
@@ -404,9 +410,9 @@ public class TestInMemoryPlan {
     try {
       plan.deleteReservation(reservationID);
     } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
-    Assert.assertNull(plan.getReservationById(reservationID));
+    assertNull(plan.getReservationById(reservationID));
     userCons =
         plan.getConsumptionForUserOverTime(user, start, start + alloc.length);
     for (int i = 0; i < alloc.length; i++) {
@@ -433,11 +439,11 @@ public class TestInMemoryPlan {
     rAllocation.getReservationDefinition()
         .setRecurrenceExpression(String.valueOf(period));
     rAllocation.setPeriodicity(period);
-    Assert.assertNull(plan.getReservationById(reservationID));
+    assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
     } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
     System.out.println(plan.toString());
     doAssertions(plan, rAllocation);
@@ -447,9 +453,9 @@ public class TestInMemoryPlan {
     try {
       plan.deleteReservation(reservationID);
     } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
-    Assert.assertNull(plan.getReservationById(reservationID));
+    assertNull(plan.getReservationById(reservationID));
     System.out.print(plan);
     checkAllocation(plan, new int[] { 0, 0 }, start, period);
   }
@@ -461,16 +467,16 @@ public class TestInMemoryPlan {
     ReservationId reservationID =
         ReservationSystemTestUtil.getNewReservationId();
     // Try to delete a reservation without adding
-    Assert.assertNull(plan.getReservationById(reservationID));
+    assertNull(plan.getReservationById(reservationID));
     try {
       plan.deleteReservation(reservationID);
-      Assert.fail("Delete should fail as it does not exist in the plan");
+      fail("Delete should fail as it does not exist in the plan");
     } catch (IllegalArgumentException e) {
-      Assert.assertTrue(e.getMessage().endsWith("does not exist in the plan"));
+      assertTrue(e.getMessage().endsWith("does not exist in the plan"));
     } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
-    Assert.assertNull(plan.getReservationById(reservationID));
+    assertNull(plan.getReservationById(reservationID));
   }
 
   @Test
@@ -486,11 +492,11 @@ public class TestInMemoryPlan {
     int start = 100;
     ReservationAllocation rAllocation =
         createReservationAllocation(reservationID1, start, alloc1);
-    Assert.assertNull(plan.getReservationById(reservationID1));
+    assertNull(plan.getReservationById(reservationID1));
     try {
       plan.addReservation(rAllocation, false);
     } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
     doAssertions(plan, rAllocation);
     checkAllocation(plan, alloc1, start, 0);
@@ -501,13 +507,13 @@ public class TestInMemoryPlan {
     int[] alloc2 = { 0, 5, 10, 5, 0 };
     rAllocation =
         createReservationAllocation(reservationID2, start, alloc2, true);
-    Assert.assertNull(plan.getReservationById(reservationID2));
+    assertNull(plan.getReservationById(reservationID2));
     try {
       plan.addReservation(rAllocation, false);
     } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
-    Assert.assertNotNull(plan.getReservationById(reservationID2));
+    assertNotNull(plan.getReservationById(reservationID2));
 
     RLESparseResourceAllocation userCons =
         plan.getConsumptionForUserOverTime(user, start, start + alloc2.length);
@@ -531,10 +537,10 @@ public class TestInMemoryPlan {
       // archival window
       plan.archiveCompletedReservations(clock.getTime());
     } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
-    Assert.assertNotNull(plan.getReservationById(reservationID1));
-    Assert.assertNull(plan.getReservationById(reservationID2));
+    assertNotNull(plan.getReservationById(reservationID1));
+    assertNull(plan.getReservationById(reservationID2));
     checkAllocation(plan, alloc1, start, 0);
 
     when(clock.getTime()).thenReturn(107L);
@@ -543,13 +549,13 @@ public class TestInMemoryPlan {
       // window
       plan.archiveCompletedReservations(clock.getTime());
     } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
 
     userCons =
         plan.getConsumptionForUserOverTime(user, start, start + alloc1.length);
 
-    Assert.assertNull(plan.getReservationById(reservationID1));
+    assertNull(plan.getReservationById(reservationID1));
     for (int i = 0; i < alloc1.length; i++) {
       assertEquals(Resource.newInstance(0, 0),
           plan.getTotalCommittedResources(start + i));
@@ -568,26 +574,26 @@ public class TestInMemoryPlan {
     int start = 100;
     ReservationAllocation rAllocation =
         createReservationAllocation(reservationID, start, alloc);
-    Assert.assertNull(plan.getReservationById(reservationID));
+    assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
     } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
 
     // Verify that get by reservation id works.
     Set<ReservationAllocation> rAllocations =
         plan.getReservations(reservationID, null, "");
-    Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation
+    assertTrue(rAllocations.size() == 1);
+    assertTrue(rAllocation
         .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by reservation id works even when time range
     // and user is invalid.
     ReservationInterval interval = new ReservationInterval(0, 0);
     rAllocations = plan.getReservations(reservationID, interval, "invalid");
-    Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation
+    assertTrue(rAllocations.size() == 1);
+    assertTrue(rAllocation
         .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
   }
 
@@ -601,11 +607,11 @@ public class TestInMemoryPlan {
     int start = 100;
     ReservationAllocation rAllocation =
         createReservationAllocation(reservationID, start, alloc);
-    Assert.assertNull(plan.getReservationById(reservationID));
+    assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
     } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
 
     // If reservationId is null, then nothing is returned.
@@ -613,7 +619,7 @@ public class TestInMemoryPlan {
         ReservationSystemTestUtil.getNewReservationId();
     Set<ReservationAllocation> rAllocations =
         plan.getReservations(invalidReservationID, null, "");
-    Assert.assertTrue(rAllocations.size() == 0);
+    assertTrue(rAllocations.size() == 0);
   }
 
   @Test
@@ -626,11 +632,11 @@ public class TestInMemoryPlan {
     int start = 100;
     ReservationAllocation rAllocation =
         createReservationAllocation(reservationID, start, alloc);
-    Assert.assertNull(plan.getReservationById(reservationID));
+    assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
     } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
 
     // Verify that get by time interval works if the selection interval
@@ -639,8 +645,8 @@ public class TestInMemoryPlan {
         rAllocation.getStartTime(), rAllocation.getEndTime());
     Set<ReservationAllocation> rAllocations =
         plan.getReservations(null, interval, "");
-    Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation
+    assertTrue(rAllocations.size() == 1);
+    assertTrue(rAllocation
         .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by time interval works if the selection interval
@@ -650,8 +656,8 @@ public class TestInMemoryPlan {
         rAllocation.getStartTime() + duration * (long) 0.3,
         rAllocation.getEndTime() - duration * (long) 0.3);
     rAllocations = plan.getReservations(null, interval, "");
-    Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation
+    assertTrue(rAllocations.size() == 1);
+    assertTrue(rAllocation
         .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by time interval selects 1 allocation if the end
@@ -659,8 +665,8 @@ public class TestInMemoryPlan {
     // allocation.
     interval = new ReservationInterval(0, rAllocation.getStartTime());
     rAllocations = plan.getReservations(null, interval, "");
-    Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation
+    assertTrue(rAllocations.size() == 1);
+    assertTrue(rAllocation
         .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
 
     // Verify that get by time interval selects no reservations if the start
@@ -668,13 +674,13 @@ public class TestInMemoryPlan {
     interval =
         new ReservationInterval(rAllocation.getEndTime(), Long.MAX_VALUE);
     rAllocations = plan.getReservations(null, interval, "");
-    Assert.assertTrue(rAllocations.size() == 0);
+    assertTrue(rAllocations.size() == 0);
 
     // Verify that get by time interval selects no reservations if the
     // selection interval and allocation interval do not overlap.
     interval = new ReservationInterval(0, rAllocation.getStartTime() / 2);
     rAllocations = plan.getReservations(null, interval, "");
-    Assert.assertTrue(rAllocations.size() == 0);
+    assertTrue(rAllocations.size() == 0);
   }
 
   @Test
@@ -687,17 +693,17 @@ public class TestInMemoryPlan {
     int start = 100;
     ReservationAllocation rAllocation =
         createReservationAllocation(reservationID, start, alloc);
-    Assert.assertNull(plan.getReservationById(reservationID));
+    assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
     } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
 
     Set<ReservationAllocation> rAllocations =
         plan.getReservationsAtTime(rAllocation.getStartTime());
-    Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation
+    assertTrue(rAllocations.size() == 1);
+    assertTrue(rAllocation
         .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
   }
 
@@ -894,19 +900,19 @@ public class TestInMemoryPlan {
     int start = 100;
     ReservationAllocation rAllocation =
         createReservationAllocation(reservationID, start, alloc);
-    Assert.assertNull(plan.getReservationById(reservationID));
+    assertNull(plan.getReservationById(reservationID));
     try {
       plan.addReservation(rAllocation, false);
     } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
 
     // Verify that getReservations defaults to getting all reservations if no
     // reservationID, time interval, and user is provided,
     Set<ReservationAllocation> rAllocations =
         plan.getReservations(null, null, "");
-    Assert.assertTrue(rAllocations.size() == 1);
-    Assert.assertTrue(rAllocation
+    assertTrue(rAllocations.size() == 1);
+    assertTrue(rAllocation
         .compareTo((ReservationAllocation) rAllocations.toArray()[0]) == 0);
   }
 
@@ -919,7 +925,7 @@ public class TestInMemoryPlan {
     ReservationInterval interval = new ReservationInterval(0, Long.MAX_VALUE);
     Set<ReservationAllocation> rAllocations =
         plan.getReservations(null, interval, "");
-    Assert.assertTrue(rAllocations.size() == 0);
+    assertTrue(rAllocations.size() == 0);
   }
 
   private void testPositiveGetRecurringReservationsHelper(long reservationStart,
@@ -971,20 +977,20 @@ public class TestInMemoryPlan {
 
     rAllocation.setPeriodicity(period);
 
-    Assert.assertNull(plan.getReservationById(reservation));
+    assertNull(plan.getReservationById(reservation));
     try {
       plan.addReservation(rAllocation, false);
     } catch (PlanningException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
     return reservation;
   }
 
   private void doAssertions(Plan plan, ReservationAllocation rAllocation) {
     ReservationId reservationID = rAllocation.getReservationId();
-    Assert.assertNotNull(plan.getReservationById(reservationID));
+    assertNotNull(plan.getReservationById(reservationID));
     assertEquals(rAllocation, plan.getReservationById(reservationID));
-    Assert.assertTrue(((InMemoryPlan) plan).getAllReservations().size() == 1);
+    assertTrue(((InMemoryPlan) plan).getAllReservations().size() == 1);
     if (rAllocation.getPeriodicity() <= 0) {
       assertEquals(rAllocation.getEndTime(), plan.getLastEndTime());
     }
@@ -993,7 +999,7 @@ public class TestInMemoryPlan {
     assertEquals(maxAlloc, plan.getMaximumAllocation());
     assertEquals(resCalc, plan.getResourceCalculator());
     assertEquals(planName, plan.getQueueName());
-    Assert.assertTrue(plan.getMoveOnExpiry());
+    assertTrue(plan.getMoveOnExpiry());
   }
 
   private ReservationDefinition createSimpleReservationDefinition(long arrival,

+ 25 - 22
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestInMemoryReservationAllocation.java

@@ -17,6 +17,10 @@
  *******************************************************************************/
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
+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.util.HashMap;
 import java.util.Map;
 import java.util.Random;
@@ -27,10 +31,9 @@ import org.apache.hadoop.yarn.api.records.ReservationRequest;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
-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 TestInMemoryReservationAllocation {
 
@@ -41,13 +44,13 @@ public class TestInMemoryReservationAllocation {
 
   private Random rand = new Random();
 
-  @Before
+  @BeforeEach
   public void setUp() {
     resCalc = new DefaultResourceCalculator();
     minAlloc = Resource.newInstance(1, 1);
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
     user = null;
     planName = null;
@@ -71,9 +74,9 @@ public class TestInMemoryReservationAllocation {
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
             start, start + alloc.length + 1, allocations, resCalc, minAlloc);
     doAssertions(rAllocation, reservationID, rDef, allocations, start, alloc);
-    Assert.assertFalse(rAllocation.containsGangs());
+    assertFalse(rAllocation.containsGangs());
     for (int i = 0; i < alloc.length; i++) {
-      Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
+      assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
           rAllocation.getResourcesAtTime(start + i));
     }
   }
@@ -94,9 +97,9 @@ public class TestInMemoryReservationAllocation {
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
             start, start + alloc.length + 1, allocations, resCalc, minAlloc);
     doAssertions(rAllocation, reservationID, rDef, allocations, start, alloc);
-    Assert.assertFalse(rAllocation.containsGangs());
+    assertFalse(rAllocation.containsGangs());
     for (int i = 0; i < alloc.length; i++) {
-      Assert.assertEquals(
+      assertEquals(
           Resource.newInstance(1024 * (alloc[i] + i), (alloc[i] + i)),
           rAllocation.getResourcesAtTime(start + i));
     }
@@ -118,9 +121,9 @@ public class TestInMemoryReservationAllocation {
         new InMemoryReservationAllocation(reservationID, rDef, user, planName,
             start, start + alloc.length + 1, allocations, resCalc, minAlloc);
     doAssertions(rAllocation, reservationID, rDef, allocations, start, alloc);
-    Assert.assertFalse(rAllocation.containsGangs());
+    assertFalse(rAllocation.containsGangs());
     for (int i = 0; i < alloc.length; i++) {
-      Assert.assertEquals(
+      assertEquals(
           Resource.newInstance(1024 * (alloc[i] + i), (alloc[i] + i)),
           rAllocation.getResourcesAtTime(start + i));
     }
@@ -143,7 +146,7 @@ public class TestInMemoryReservationAllocation {
             start, start + alloc.length + 1, allocations, resCalc, minAlloc);
     doAssertions(rAllocation, reservationID, rDef, allocations, (int) start,
         alloc);
-    Assert.assertFalse(rAllocation.containsGangs());
+    assertFalse(rAllocation.containsGangs());
   }
 
   @Test
@@ -164,9 +167,9 @@ public class TestInMemoryReservationAllocation {
             start, start + alloc.length + 1, allocations, resCalc, minAlloc,
             isGang);
     doAssertions(rAllocation, reservationID, rDef, allocations, start, alloc);
-    Assert.assertTrue(rAllocation.containsGangs());
+    assertTrue(rAllocation.containsGangs());
     for (int i = 0; i < alloc.length; i++) {
-      Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
+      assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
           rAllocation.getResourcesAtTime(start + i));
     }
   }
@@ -175,13 +178,13 @@ public class TestInMemoryReservationAllocation {
       ReservationId reservationID, ReservationDefinition rDef,
       Map<ReservationInterval, Resource> allocations, int start,
       int[] alloc) {
-    Assert.assertEquals(reservationID, rAllocation.getReservationId());
-    Assert.assertEquals(rDef, rAllocation.getReservationDefinition());
-    Assert.assertEquals(allocations, rAllocation.getAllocationRequests());
-    Assert.assertEquals(user, rAllocation.getUser());
-    Assert.assertEquals(planName, rAllocation.getPlanName());
-    Assert.assertEquals(start, rAllocation.getStartTime());
-    Assert.assertEquals(start + alloc.length + 1, rAllocation.getEndTime());
+    assertEquals(reservationID, rAllocation.getReservationId());
+    assertEquals(rDef, rAllocation.getReservationDefinition());
+    assertEquals(allocations, rAllocation.getAllocationRequests());
+    assertEquals(user, rAllocation.getUser());
+    assertEquals(planName, rAllocation.getPlanName());
+    assertEquals(start, rAllocation.getStartTime());
+    assertEquals(start + alloc.length + 1, rAllocation.getEndTime());
   }
 
   private Map<ReservationInterval, Resource> generateAllocation(

+ 20 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestNoOverCommitPolicy.java

@@ -25,14 +25,12 @@ import net.jcip.annotations.NotThreadSafe;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.ResourceOverCommitException;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 
 /**
  * This clas tests {@code NoOverCommitPolicy} sharing policy.
  */
-@RunWith(value = Parameterized.class)
 @NotThreadSafe
 @SuppressWarnings("VisibilityModifier")
 public class TestNoOverCommitPolicy extends BaseSharingPolicyTest {
@@ -40,8 +38,16 @@ public class TestNoOverCommitPolicy extends BaseSharingPolicyTest {
   final static long ONEHOUR = 3600 * 1000;
   final static String TWOHOURPERIOD = "7200000";
 
-  @Parameterized.Parameters(name = "Duration {0}, height {1}," +
-          " submissions {2}, periodic {3})")
+  public void initTestNoOverCommitPolicy(long pDuration,
+      double pHeight, int pNumSubmissions, String pRecurrenceExpression, Class pExpectedError) {
+    this.duration = pDuration;
+    this.height = pHeight;
+    this.numSubmissions = pNumSubmissions;
+    this.recurrenceExpression = pRecurrenceExpression;
+    this.expectedError = pExpectedError;
+    super.setup();
+  }
+
   public static Collection<Object[]> data() {
     return Arrays.asList(new Object[][] {
 
@@ -78,8 +84,14 @@ public class TestNoOverCommitPolicy extends BaseSharingPolicyTest {
     return policy;
   }
 
-  @Test
-  public void testAllocation() throws IOException, PlanningException {
+  @ParameterizedTest(name = "Duration {0}, height {1}," +
+      " numSubmission {2}, periodic {3})")
+  @MethodSource("data")
+  public void testAllocation(long pDuration,
+      double pHeight, int pNumSubmissions, String pRecurrenceExpression, Class pExpectedError)
+      throws IOException, PlanningException {
+    initTestNoOverCommitPolicy(pDuration, pHeight, pNumSubmissions,
+        pRecurrenceExpression, pExpectedError);
     runTest();
   }
 

+ 38 - 35
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestPeriodicRLESparseResourceAllocation.java

@@ -18,10 +18,13 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
+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.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -43,13 +46,13 @@ public class TestPeriodicRLESparseResourceAllocation {
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 20L);
     LOG.info(periodicVector.toString());
-    Assert.assertEquals(Resource.newInstance(5, 5),
+    assertEquals(Resource.newInstance(5, 5),
         periodicVector.getCapacityAtTime(10L));
-    Assert.assertEquals(Resource.newInstance(10, 10),
+    assertEquals(Resource.newInstance(10, 10),
         periodicVector.getCapacityAtTime(20L));
-    Assert.assertEquals(Resource.newInstance(7, 7),
+    assertEquals(Resource.newInstance(7, 7),
         periodicVector.getCapacityAtTime(27L));
-    Assert.assertEquals(Resource.newInstance(5, 5),
+    assertEquals(Resource.newInstance(5, 5),
         periodicVector.getCapacityAtTime(50L));
   }
 
@@ -62,15 +65,15 @@ public class TestPeriodicRLESparseResourceAllocation {
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 8L);
     LOG.info(periodicVector.toString());
-    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(0, 1),
+    assertEquals(periodicVector.getMaximumPeriodicCapacity(0, 1),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(8, 2),
+    assertEquals(periodicVector.getMaximumPeriodicCapacity(8, 2),
         Resource.newInstance(7, 7));
-    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(16, 3),
+    assertEquals(periodicVector.getMaximumPeriodicCapacity(16, 3),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(17, 4),
+    assertEquals(periodicVector.getMaximumPeriodicCapacity(17, 4),
         Resource.newInstance(5, 5));
-    Assert.assertEquals(periodicVector.getMaximumPeriodicCapacity(32, 5),
+    assertEquals(periodicVector.getMaximumPeriodicCapacity(32, 5),
         Resource.newInstance(4, 4));
   }
 
@@ -93,23 +96,23 @@ public class TestPeriodicRLESparseResourceAllocation {
             Resource.newInstance(100 * 1024, 100), periodic, nonPeriodic,
             RLESparseResourceAllocation.RLEOperator.add, 2, 25);
 
-    Assert.assertEquals(Resource.newInstance(5, 5),
+    assertEquals(Resource.newInstance(5, 5),
         merged.getCapacityAtTime(2L));
-    Assert.assertEquals(Resource.newInstance(0, 0),
+    assertEquals(Resource.newInstance(0, 0),
         merged.getCapacityAtTime(3L));
-    Assert.assertEquals(Resource.newInstance(2, 2),
+    assertEquals(Resource.newInstance(2, 2),
         merged.getCapacityAtTime(11L));
-    Assert.assertEquals(Resource.newInstance(15, 15),
+    assertEquals(Resource.newInstance(15, 15),
         merged.getCapacityAtTime(12L));
-    Assert.assertEquals(Resource.newInstance(10, 10),
+    assertEquals(Resource.newInstance(10, 10),
         merged.getCapacityAtTime(13L));
-    Assert.assertEquals(Resource.newInstance(0, 0),
+    assertEquals(Resource.newInstance(0, 0),
         merged.getCapacityAtTime(14L));
-    Assert.assertEquals(Resource.newInstance(2, 2),
+    assertEquals(Resource.newInstance(2, 2),
         merged.getCapacityAtTime(21L));
-    Assert.assertEquals(Resource.newInstance(5, 5),
+    assertEquals(Resource.newInstance(5, 5),
         merged.getCapacityAtTime(22L));
-    Assert.assertEquals(Resource.newInstance(0, 0),
+    assertEquals(Resource.newInstance(0, 0),
         merged.getCapacityAtTime(23L));
   }
 
@@ -123,15 +126,15 @@ public class TestPeriodicRLESparseResourceAllocation {
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 10L);
     ReservationInterval interval = new ReservationInterval(5L, 10L);
     periodicVector.addInterval(interval, Resource.newInstance(8, 8));
-    Assert.assertEquals(Resource.newInstance(8, 8),
+    assertEquals(Resource.newInstance(8, 8),
         periodicVector.getCapacityAtTime(5L));
-    Assert.assertEquals(Resource.newInstance(8, 8),
+    assertEquals(Resource.newInstance(8, 8),
         periodicVector.getCapacityAtTime(9L));
-    Assert.assertEquals(Resource.newInstance(0, 0),
+    assertEquals(Resource.newInstance(0, 0),
         periodicVector.getCapacityAtTime(10L));
-    Assert.assertEquals(Resource.newInstance(0, 0),
+    assertEquals(Resource.newInstance(0, 0),
         periodicVector.getCapacityAtTime(0L));
-    // Assert.assertFalse(periodicVector.addInterval(
+    // assertFalse(periodicVector.addInterval(
     // new ReservationInterval(7L, 12L), Resource.newInstance(8, 8)));
   }
 
@@ -143,29 +146,29 @@ public class TestPeriodicRLESparseResourceAllocation {
     PeriodicRLESparseResourceAllocation periodicVector =
         new PeriodicRLESparseResourceAllocation(rleSparseVector, 10L);
     ReservationInterval interval = new ReservationInterval(3L, 7L);
-    Assert.assertTrue(
+    assertTrue(
         periodicVector.removeInterval(interval, Resource.newInstance(3, 3)));
-    Assert.assertEquals(Resource.newInstance(2, 2),
+    assertEquals(Resource.newInstance(2, 2),
         periodicVector.getCapacityAtTime(1L));
-    Assert.assertEquals(Resource.newInstance(2, 2),
+    assertEquals(Resource.newInstance(2, 2),
         periodicVector.getCapacityAtTime(2L));
-    Assert.assertEquals(Resource.newInstance(2, 2),
+    assertEquals(Resource.newInstance(2, 2),
         periodicVector.getCapacityAtTime(3L));
-    Assert.assertEquals(Resource.newInstance(2, 2),
+    assertEquals(Resource.newInstance(2, 2),
         periodicVector.getCapacityAtTime(4L));
-    Assert.assertEquals(Resource.newInstance(0, 0),
+    assertEquals(Resource.newInstance(0, 0),
         periodicVector.getCapacityAtTime(5L));
-    Assert.assertEquals(Resource.newInstance(0, 0),
+    assertEquals(Resource.newInstance(0, 0),
         periodicVector.getCapacityAtTime(6L));
-    Assert.assertEquals(Resource.newInstance(4, 4),
+    assertEquals(Resource.newInstance(4, 4),
         periodicVector.getCapacityAtTime(7L));
 
     // invalid interval
-    Assert.assertFalse(periodicVector.removeInterval(
+    assertFalse(periodicVector.removeInterval(
         new ReservationInterval(7L, 12L), Resource.newInstance(1, 1)));
 
     // invalid capacity
-    Assert.assertFalse(periodicVector.removeInterval(
+    assertFalse(periodicVector.removeInterval(
         new ReservationInterval(2L, 4L), Resource.newInstance(8, 8)));
 
   }

+ 64 - 63
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java

@@ -17,9 +17,11 @@
  *******************************************************************************/
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-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.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -34,9 +36,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.RLESparseResour
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
-import org.junit.Assert;
-import org.junit.Ignore;
-import org.junit.Test;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -251,7 +252,7 @@ public class TestRLESparseResourceAllocation {
   }
 
   @Test
-  @Ignore
+  @Disabled
   public void testMergeSpeed() throws PlanningException {
 
     for (int j = 0; j < 100; j++) {
@@ -322,26 +323,26 @@ public class TestRLESparseResourceAllocation {
       rleSparseVector.addInterval(ip.getKey(), ip.getValue());
     }
     LOG.info(rleSparseVector.toString());
-    Assert.assertFalse(rleSparseVector.isEmpty());
-    Assert.assertEquals(Resource.newInstance(0, 0),
+    assertFalse(rleSparseVector.isEmpty());
+    assertEquals(Resource.newInstance(0, 0),
         rleSparseVector.getCapacityAtTime(99));
-    Assert.assertEquals(Resource.newInstance(0, 0),
+    assertEquals(Resource.newInstance(0, 0),
         rleSparseVector.getCapacityAtTime(start + alloc.length + 1));
     for (int i = 0; i < alloc.length; i++) {
-      Assert.assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
+      assertEquals(Resource.newInstance(1024 * (alloc[i]), (alloc[i])),
           rleSparseVector.getCapacityAtTime(start + i));
     }
-    Assert.assertEquals(Resource.newInstance(0, 0),
+    assertEquals(Resource.newInstance(0, 0),
         rleSparseVector.getCapacityAtTime(start + alloc.length + 2));
     for (Entry<ReservationInterval, Resource> ip : inputs) {
       rleSparseVector.removeInterval(ip.getKey(), ip.getValue());
     }
     LOG.info(rleSparseVector.toString());
     for (int i = 0; i < alloc.length; i++) {
-      Assert.assertEquals(Resource.newInstance(0, 0),
+      assertEquals(Resource.newInstance(0, 0),
           rleSparseVector.getCapacityAtTime(start + i));
     }
-    Assert.assertTrue(rleSparseVector.isEmpty());
+    assertTrue(rleSparseVector.isEmpty());
   }
 
   @Test
@@ -374,28 +375,28 @@ public class TestRLESparseResourceAllocation {
     // Current bug prevents this to pass. The RLESparseResourceAllocation
     // does not handle removal of "partial"
     // allocations correctly.
-    Assert.assertEquals(102400,
+    assertEquals(102400,
         rleSparseVector.getCapacityAtTime(10).getMemorySize());
-    Assert.assertEquals(0,
+    assertEquals(0,
         rleSparseVector.getCapacityAtTime(13).getMemorySize());
-    Assert.assertEquals(0,
+    assertEquals(0,
         rleSparseVector.getCapacityAtTime(19).getMemorySize());
-    Assert.assertEquals(102400,
+    assertEquals(102400,
         rleSparseVector.getCapacityAtTime(21).getMemorySize());
-    Assert.assertEquals(2 * 102400,
+    assertEquals(2 * 102400,
         rleSparseVector.getCapacityAtTime(26).getMemorySize());
 
     ReservationInterval riRemove2 = new ReservationInterval(9, 13);
     rleSparseVector.removeInterval(riRemove2, rr);
     LOG.info(rleSparseVector.toString());
 
-    Assert.assertEquals(0,
+    assertEquals(0,
         rleSparseVector.getCapacityAtTime(11).getMemorySize());
-    Assert.assertEquals(-102400,
+    assertEquals(-102400,
         rleSparseVector.getCapacityAtTime(9).getMemorySize());
-    Assert.assertEquals(0,
+    assertEquals(0,
         rleSparseVector.getCapacityAtTime(13).getMemorySize());
-    Assert.assertEquals(102400,
+    assertEquals(102400,
         rleSparseVector.getCapacityAtTime(20).getMemorySize());
 
   }
@@ -414,27 +415,27 @@ public class TestRLESparseResourceAllocation {
       rleSparseVector.addInterval(ip.getKey(), ip.getValue());
     }
     LOG.info(rleSparseVector.toString());
-    Assert.assertFalse(rleSparseVector.isEmpty());
-    Assert.assertEquals(Resource.newInstance(0, 0),
+    assertFalse(rleSparseVector.isEmpty());
+    assertEquals(Resource.newInstance(0, 0),
         rleSparseVector.getCapacityAtTime(99));
-    Assert.assertEquals(Resource.newInstance(0, 0),
+    assertEquals(Resource.newInstance(0, 0),
         rleSparseVector.getCapacityAtTime(start + alloc.length + 1));
     for (int i = 0; i < alloc.length; i++) {
-      Assert.assertEquals(
+      assertEquals(
           Resource.newInstance(1024 * (alloc[i] + i), (alloc[i] + i)),
           rleSparseVector.getCapacityAtTime(start + i));
     }
-    Assert.assertEquals(Resource.newInstance(0, 0),
+    assertEquals(Resource.newInstance(0, 0),
         rleSparseVector.getCapacityAtTime(start + alloc.length + 2));
     for (Entry<ReservationInterval, Resource> ip : inputs) {
       rleSparseVector.removeInterval(ip.getKey(), ip.getValue());
     }
     LOG.info(rleSparseVector.toString());
     for (int i = 0; i < alloc.length; i++) {
-      Assert.assertEquals(Resource.newInstance(0, 0),
+      assertEquals(Resource.newInstance(0, 0),
           rleSparseVector.getCapacityAtTime(start + i));
     }
-    Assert.assertTrue(rleSparseVector.isEmpty());
+    assertTrue(rleSparseVector.isEmpty());
   }
 
   @Test
@@ -451,27 +452,27 @@ public class TestRLESparseResourceAllocation {
       rleSparseVector.addInterval(ip.getKey(), ip.getValue());
     }
     LOG.info(rleSparseVector.toString());
-    Assert.assertFalse(rleSparseVector.isEmpty());
-    Assert.assertEquals(Resource.newInstance(0, 0),
+    assertFalse(rleSparseVector.isEmpty());
+    assertEquals(Resource.newInstance(0, 0),
         rleSparseVector.getCapacityAtTime(99));
-    Assert.assertEquals(Resource.newInstance(0, 0),
+    assertEquals(Resource.newInstance(0, 0),
         rleSparseVector.getCapacityAtTime(start + alloc.length + 1));
     for (int i = 0; i < alloc.length; i++) {
-      Assert.assertEquals(
+      assertEquals(
           Resource.newInstance(1024 * (alloc[i] + i), (alloc[i] + i)),
           rleSparseVector.getCapacityAtTime(start + i));
     }
-    Assert.assertEquals(Resource.newInstance(0, 0),
+    assertEquals(Resource.newInstance(0, 0),
         rleSparseVector.getCapacityAtTime(start + alloc.length + 2));
     for (Entry<ReservationInterval, Resource> ip : inputs) {
       rleSparseVector.removeInterval(ip.getKey(), ip.getValue());
     }
     LOG.info(rleSparseVector.toString());
     for (int i = 0; i < alloc.length; i++) {
-      Assert.assertEquals(Resource.newInstance(0, 0),
+      assertEquals(Resource.newInstance(0, 0),
           rleSparseVector.getCapacityAtTime(start + i));
     }
-    Assert.assertTrue(rleSparseVector.isEmpty());
+    assertTrue(rleSparseVector.isEmpty());
   }
 
   @Test
@@ -482,9 +483,9 @@ public class TestRLESparseResourceAllocation {
     rleSparseVector.addInterval(new ReservationInterval(0, Long.MAX_VALUE),
         Resource.newInstance(0, 0));
     LOG.info(rleSparseVector.toString());
-    Assert.assertEquals(Resource.newInstance(0, 0),
+    assertEquals(Resource.newInstance(0, 0),
         rleSparseVector.getCapacityAtTime(new Random().nextLong()));
-    Assert.assertTrue(rleSparseVector.isEmpty());
+    assertTrue(rleSparseVector.isEmpty());
   }
 
   @Test
@@ -496,7 +497,7 @@ public class TestRLESparseResourceAllocation {
 
     // Check empty
     mapAllocations = rleSparseVector.toIntervalMap();
-    Assert.assertTrue(mapAllocations.isEmpty());
+    assertTrue(mapAllocations.isEmpty());
 
     // Check full
     int[] alloc = { 0, 5, 10, 10, 5, 0, 5, 0 };
@@ -507,28 +508,28 @@ public class TestRLESparseResourceAllocation {
       rleSparseVector.addInterval(ip.getKey(), ip.getValue());
     }
     mapAllocations = rleSparseVector.toIntervalMap();
-    Assert.assertTrue(mapAllocations.size() == 5);
+    assertTrue(mapAllocations.size() == 5);
     for (Entry<ReservationInterval, Resource> entry : mapAllocations
         .entrySet()) {
       ReservationInterval interval = entry.getKey();
       Resource resource = entry.getValue();
       if (interval.getStartTime() == 101L) {
-        Assert.assertTrue(interval.getEndTime() == 102L);
-        Assert.assertEquals(resource, Resource.newInstance(5 * 1024, 5));
+        assertTrue(interval.getEndTime() == 102L);
+        assertEquals(resource, Resource.newInstance(5 * 1024, 5));
       } else if (interval.getStartTime() == 102L) {
-        Assert.assertTrue(interval.getEndTime() == 104L);
-        Assert.assertEquals(resource, Resource.newInstance(10 * 1024, 10));
+        assertTrue(interval.getEndTime() == 104L);
+        assertEquals(resource, Resource.newInstance(10 * 1024, 10));
       } else if (interval.getStartTime() == 104L) {
-        Assert.assertTrue(interval.getEndTime() == 105L);
-        Assert.assertEquals(resource, Resource.newInstance(5 * 1024, 5));
+        assertTrue(interval.getEndTime() == 105L);
+        assertEquals(resource, Resource.newInstance(5 * 1024, 5));
       } else if (interval.getStartTime() == 105L) {
-        Assert.assertTrue(interval.getEndTime() == 106L);
-        Assert.assertEquals(resource, Resource.newInstance(0 * 1024, 0));
+        assertTrue(interval.getEndTime() == 106L);
+        assertEquals(resource, Resource.newInstance(0 * 1024, 0));
       } else if (interval.getStartTime() == 106L) {
-        Assert.assertTrue(interval.getEndTime() == 107L);
-        Assert.assertEquals(resource, Resource.newInstance(5 * 1024, 5));
+        assertTrue(interval.getEndTime() == 107L);
+        assertEquals(resource, Resource.newInstance(5 * 1024, 5));
       } else {
-        Assert.fail();
+        fail();
       }
     }
   }
@@ -540,21 +541,21 @@ public class TestRLESparseResourceAllocation {
     RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
         .generateRLESparseResourceAllocation(alloc, timeSteps);
     LOG.info(rleSparseVector.toString());
-    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 1),
+    assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 1),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 2),
+    assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 2),
         Resource.newInstance(7, 7));
-    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 3),
+    assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 3),
         Resource.newInstance(10, 10));
-    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 4),
+    assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 4),
         Resource.newInstance(3, 3));
-    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 5),
+    assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 5),
         Resource.newInstance(4, 4));
-    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 5),
+    assertEquals(rleSparseVector.getMaximumPeriodicCapacity(0, 5),
         Resource.newInstance(4, 4));
-    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(7, 5),
+    assertEquals(rleSparseVector.getMaximumPeriodicCapacity(7, 5),
         Resource.newInstance(8, 8));
-    Assert.assertEquals(rleSparseVector.getMaximumPeriodicCapacity(10, 3),
+    assertEquals(rleSparseVector.getMaximumPeriodicCapacity(10, 3),
         Resource.newInstance(0, 0));
   }
 
@@ -565,11 +566,11 @@ public class TestRLESparseResourceAllocation {
     RLESparseResourceAllocation rleSparseVector = ReservationSystemTestUtil
         .generateRLESparseResourceAllocation(alloc, timeSteps);
     LOG.info(rleSparseVector.toString());
-    Assert.assertEquals(rleSparseVector.getMinimumCapacityInInterval(
+    assertEquals(rleSparseVector.getMinimumCapacityInInterval(
         new ReservationInterval(1L, 3L)), Resource.newInstance(5, 5));
-    Assert.assertEquals(rleSparseVector.getMinimumCapacityInInterval(
+    assertEquals(rleSparseVector.getMinimumCapacityInInterval(
         new ReservationInterval(2L, 5L)), Resource.newInstance(3, 3));
-    Assert.assertEquals(rleSparseVector.getMinimumCapacityInInterval(
+    assertEquals(rleSparseVector.getMinimumCapacityInInterval(
         new ReservationInterval(1L, 7L)), Resource.newInstance(0, 0));
   }
 

+ 130 - 137
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationInputValidator.java

@@ -17,6 +17,11 @@
  *******************************************************************************/
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
+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.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -49,10 +54,9 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.util.Clock;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
-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 TestReservationInputValidator {
 
@@ -68,7 +72,7 @@ public class TestReservationInputValidator {
 
   private ReservationInputValidator rrValidator;
 
-  @Before
+  @BeforeEach
   public void setUp() {
     clock = mock(Clock.class);
     plan = mock(Plan.class);
@@ -87,7 +91,7 @@ public class TestReservationInputValidator {
     when(rSystem.getPlan(PLAN_NAME)).thenReturn(plan);
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
     rrValidator = null;
     clock = null;
@@ -104,9 +108,9 @@ public class TestReservationInputValidator {
           rrValidator.validateReservationSubmissionRequest(rSystem, request,
               ReservationSystemTestUtil.getNewReservationId());
     } catch (YarnException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
-    Assert.assertNotNull(plan);
+    assertNotNull(plan);
   }
 
   @Test
@@ -118,11 +122,11 @@ public class TestReservationInputValidator {
       plan =
           rrValidator.validateReservationSubmissionRequest(rSystem, request,
               ReservationSystemTestUtil.getNewReservationId());
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert.assertEquals("The queue is not specified. Please try again with a "
+      assertEquals("The queue is not specified. Please try again with a "
           + "valid reservable queue.", message);
       LOG.info(message);
     }
@@ -138,13 +142,12 @@ public class TestReservationInputValidator {
       plan =
           rrValidator.validateReservationSubmissionRequest(rSystem, request,
               ReservationSystemTestUtil.getNewReservationId());
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert
-          .assertTrue(message
-              .endsWith(" is not managed by reservation system. Please try again with a valid reservable queue."));
+      assertTrue(message.endsWith(" is not managed by reservation system. " +
+          "Please try again with a valid reservable queue."));
       LOG.info(message);
     }
   }
@@ -159,11 +162,11 @@ public class TestReservationInputValidator {
       plan =
           rrValidator.validateReservationSubmissionRequest(rSystem, request,
               ReservationSystemTestUtil.getNewReservationId());
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert.assertEquals("Missing reservation definition. Please try again by "
+      assertEquals("Missing reservation definition. Please try again by "
           + "specifying a reservation definition.", message);
       LOG.info(message);
     }
@@ -178,11 +181,11 @@ public class TestReservationInputValidator {
       plan =
           rrValidator.validateReservationSubmissionRequest(rSystem, request,
               ReservationSystemTestUtil.getNewReservationId());
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert.assertTrue(message
+      assertTrue(message
           .startsWith("The specified deadline: 0 is the past"));
       LOG.info(message);
     }
@@ -197,11 +200,11 @@ public class TestReservationInputValidator {
       plan =
           rrValidator.validateReservationSubmissionRequest(rSystem, request,
               ReservationSystemTestUtil.getNewReservationId());
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert.assertTrue(message
+      assertTrue(message
           .startsWith("No resources have been specified to reserve"));
       LOG.info(message);
     }
@@ -216,11 +219,11 @@ public class TestReservationInputValidator {
       plan =
           rrValidator.validateReservationSubmissionRequest(rSystem, request,
               ReservationSystemTestUtil.getNewReservationId());
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert.assertTrue(message
+      assertTrue(message
           .startsWith("No resources have been specified to reserve"));
       LOG.info(message);
     }
@@ -235,14 +238,13 @@ public class TestReservationInputValidator {
       plan =
           rrValidator.validateReservationSubmissionRequest(rSystem, request,
               ReservationSystemTestUtil.getNewReservationId());
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert.assertTrue(message.startsWith("The time difference"));
-      Assert
-          .assertTrue(message
-              .contains("must  be greater or equal to the minimum resource duration"));
+      assertTrue(message.startsWith("The time difference"));
+      assertTrue(message
+          .contains("must  be greater or equal to the minimum resource duration"));
       LOG.info(message);
     }
   }
@@ -258,13 +260,13 @@ public class TestReservationInputValidator {
       plan =
           rrValidator.validateReservationSubmissionRequest(rSystem, request,
               ReservationSystemTestUtil.getNewReservationId());
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert.assertTrue(message.startsWith(
+      assertTrue(message.startsWith(
           "The size of the largest gang in the reservation definition"));
-      Assert.assertTrue(message.contains(
+      assertTrue(message.contains(
           "exceed the capacity available "));
       LOG.info(message);
     }
@@ -280,9 +282,9 @@ public class TestReservationInputValidator {
           rrValidator.validateReservationSubmissionRequest(rSystem, request,
               ReservationSystemTestUtil.getNewReservationId());
     } catch (YarnException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
-    Assert.assertNotNull(plan);
+    assertNotNull(plan);
   }
 
   @Test
@@ -294,11 +296,11 @@ public class TestReservationInputValidator {
       plan =
           rrValidator.validateReservationSubmissionRequest(rSystem, request,
               ReservationSystemTestUtil.getNewReservationId());
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert.assertTrue(message
+      assertTrue(message
           .startsWith("Negative Period : "));
       LOG.info(message);
     }
@@ -315,11 +317,11 @@ public class TestReservationInputValidator {
     try {
       plan = rrValidator.validateReservationSubmissionRequest(rSystem, request,
           ReservationSystemTestUtil.getNewReservationId());
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert.assertTrue(message.startsWith("The maximum periodicity:"));
+      assertTrue(message.startsWith("The maximum periodicity:"));
       LOG.info(message);
     }
   }
@@ -334,11 +336,11 @@ public class TestReservationInputValidator {
       plan =
           rrValidator.validateReservationSubmissionRequest(rSystem, request,
               ReservationSystemTestUtil.getNewReservationId());
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert.assertTrue(message
+      assertTrue(message
           .startsWith("Invalid period "));
       LOG.info(message);
     }
@@ -351,11 +353,11 @@ public class TestReservationInputValidator {
       plan =
           rrValidator.validateReservationSubmissionRequest(rSystem, request,
               ReservationSystemTestUtil.getNewReservationId());
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert.assertTrue(message
+      assertTrue(message
           .startsWith("Duration of the requested reservation:"));
       LOG.info(message);
     }
@@ -369,9 +371,9 @@ public class TestReservationInputValidator {
     try {
       plan = rrValidator.validateReservationUpdateRequest(rSystem, request);
     } catch (YarnException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
-    Assert.assertNotNull(plan);
+    assertNotNull(plan);
   }
 
   @Test
@@ -380,13 +382,12 @@ public class TestReservationInputValidator {
     Plan plan = null;
     try {
       plan = rrValidator.validateReservationUpdateRequest(rSystem, request);
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert
-          .assertTrue(message
-              .startsWith("Missing reservation id. Please try again by specifying a reservation id."));
+      assertTrue(message
+          .startsWith("Missing reservation id. Please try again by specifying a reservation id."));
       LOG.info(message);
     }
   }
@@ -400,15 +401,13 @@ public class TestReservationInputValidator {
     Plan plan = null;
     try {
       plan = rrValidator.validateReservationUpdateRequest(rSystem, request);
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert
-          .assertTrue(message.equals(MessageFormat
-              .format(
-                  "The specified reservation with ID: {0} is unknown. Please try again with a valid reservation.",
-                  rId)));
+      assertTrue(message.equals(MessageFormat
+          .format("The specified reservation with ID: {0} is unknown. " +
+          "Please try again with a valid reservation.", rId)));
       LOG.info(message);
     }
   }
@@ -421,13 +420,13 @@ public class TestReservationInputValidator {
     Plan plan = null;
     try {
       plan = rrValidator.validateReservationUpdateRequest(rSystem, request);
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert
-          .assertTrue(message
-              .endsWith(" is not associated with any valid plan. Please try again with a valid reservation."));
+      assertTrue(message
+          .endsWith(" is not associated with any valid plan. " +
+          "Please try again with a valid reservation."));
       LOG.info(message);
     }
   }
@@ -439,13 +438,12 @@ public class TestReservationInputValidator {
     Plan plan = null;
     try {
       plan = rrValidator.validateReservationUpdateRequest(rSystem, request);
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert
-          .assertTrue(message
-              .startsWith("Missing reservation definition. Please try again by specifying a reservation definition."));
+      assertTrue(message.startsWith("Missing reservation definition. " +
+          "Please try again by specifying a reservation definition."));
       LOG.info(message);
     }
   }
@@ -457,11 +455,11 @@ public class TestReservationInputValidator {
     Plan plan = null;
     try {
       plan = rrValidator.validateReservationUpdateRequest(rSystem, request);
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert.assertTrue(message
+      assertTrue(message
           .startsWith("The specified deadline: 0 is the past"));
       LOG.info(message);
     }
@@ -474,11 +472,11 @@ public class TestReservationInputValidator {
     Plan plan = null;
     try {
       plan = rrValidator.validateReservationUpdateRequest(rSystem, request);
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert.assertTrue(message
+      assertTrue(message
           .startsWith("No resources have been specified to reserve"));
       LOG.info(message);
     }
@@ -491,11 +489,11 @@ public class TestReservationInputValidator {
     Plan plan = null;
     try {
       plan = rrValidator.validateReservationUpdateRequest(rSystem, request);
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert.assertTrue(message
+      assertTrue(message
           .startsWith("No resources have been specified to reserve"));
       LOG.info(message);
     }
@@ -508,13 +506,12 @@ public class TestReservationInputValidator {
     Plan plan = null;
     try {
       plan = rrValidator.validateReservationUpdateRequest(rSystem, request);
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert
-          .assertTrue(message
-              .contains("must  be greater or equal to the minimum resource duration"));
+      assertTrue(message.contains("must  be greater or " +
+          "equal to the minimum resource duration"));
       LOG.info(message);
     }
   }
@@ -528,13 +525,13 @@ public class TestReservationInputValidator {
     Plan plan = null;
     try {
       plan = rrValidator.validateReservationUpdateRequest(rSystem, request);
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert.assertTrue(message.startsWith(
+      assertTrue(message.startsWith(
           "The size of the largest gang in the reservation definition"));
-      Assert.assertTrue(message.contains(
+      assertTrue(message.contains(
           "exceed the capacity available "));
       LOG.info(message);
     }
@@ -549,9 +546,9 @@ public class TestReservationInputValidator {
       plan =
           rrValidator.validateReservationUpdateRequest(rSystem, request);
     } catch (YarnException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
-    Assert.assertNotNull(plan);
+    assertNotNull(plan);
   }
 
   @Test
@@ -562,11 +559,11 @@ public class TestReservationInputValidator {
     try {
       plan =
           rrValidator.validateReservationUpdateRequest(rSystem, request);
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert.assertTrue(message
+      assertTrue(message
           .startsWith("Negative Period : "));
       LOG.info(message);
     }
@@ -581,11 +578,11 @@ public class TestReservationInputValidator {
     try {
       plan =
           rrValidator.validateReservationUpdateRequest(rSystem, request);
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert.assertTrue(message
+      assertTrue(message
           .startsWith("Invalid period "));
       LOG.info(message);
     }
@@ -597,11 +594,11 @@ public class TestReservationInputValidator {
     try {
       plan =
           rrValidator.validateReservationUpdateRequest(rSystem, request);
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert.assertTrue(message
+      assertTrue(message
           .startsWith("Duration of the requested reservation:"));
       LOG.info(message);
     }
@@ -619,9 +616,9 @@ public class TestReservationInputValidator {
     try {
       plan = rrValidator.validateReservationDeleteRequest(rSystem, request);
     } catch (YarnException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
-    Assert.assertNotNull(plan);
+    assertNotNull(plan);
   }
 
   @Test
@@ -630,13 +627,12 @@ public class TestReservationInputValidator {
     Plan plan = null;
     try {
       plan = rrValidator.validateReservationDeleteRequest(rSystem, request);
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert
-          .assertTrue(message
-              .startsWith("Missing reservation id. Please try again by specifying a reservation id."));
+      assertTrue(message.startsWith("Missing reservation id. " +
+          "Please try again by specifying a reservation id."));
       LOG.info(message);
     }
   }
@@ -650,15 +646,13 @@ public class TestReservationInputValidator {
     Plan plan = null;
     try {
       plan = rrValidator.validateReservationDeleteRequest(rSystem, request);
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert
-          .assertTrue(message.equals(MessageFormat
-              .format(
-                  "The specified reservation with ID: {0} is unknown. Please try again with a valid reservation.",
-                  rId)));
+      assertTrue(message.equals(MessageFormat
+          .format("The specified reservation with ID: {0} is unknown. " +
+          "Please try again with a valid reservation.", rId)));
       LOG.info(message);
     }
   }
@@ -673,13 +667,12 @@ public class TestReservationInputValidator {
     Plan plan = null;
     try {
       plan = rrValidator.validateReservationDeleteRequest(rSystem, request);
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert
-          .assertTrue(message
-              .endsWith(" is not associated with any valid plan. Please try again with a valid reservation."));
+      assertTrue(message.endsWith(" is not associated with any valid plan. " +
+          "Please try again with a valid reservation."));
       LOG.info(message);
     }
   }
@@ -696,9 +689,9 @@ public class TestReservationInputValidator {
     try {
       plan = rrValidator.validateReservationListRequest(rSystem, request);
     } catch (YarnException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
-    Assert.assertNotNull(plan);
+    assertNotNull(plan);
   }
 
   @Test
@@ -715,9 +708,9 @@ public class TestReservationInputValidator {
     try {
       plan = rrValidator.validateReservationListRequest(rSystem, request);
     } catch (YarnException e) {
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
-    Assert.assertNotNull(plan);
+    assertNotNull(plan);
   }
 
   @Test
@@ -731,11 +724,11 @@ public class TestReservationInputValidator {
     Plan plan = null;
     try {
       plan = rrValidator.validateReservationListRequest(rSystem, request);
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert.assertTrue(message.equals("The specified end time must be " +
+      assertTrue(message.equals("The specified end time must be " +
               "greater than the specified start time."));
       LOG.info(message);
     }
@@ -748,11 +741,11 @@ public class TestReservationInputValidator {
     Plan plan = null;
     try {
       plan = rrValidator.validateReservationListRequest(rSystem, request);
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert.assertTrue(message.equals(
+      assertTrue(message.equals(
           "The queue is not specified. Please try again with a valid " +
                                       "reservable queue."));
       LOG.info(message);
@@ -768,11 +761,11 @@ public class TestReservationInputValidator {
     Plan plan = null;
     try {
       plan = rrValidator.validateReservationListRequest(rSystem, request);
-      Assert.fail();
+      fail();
     } catch (YarnException e) {
-      Assert.assertNull(plan);
+      assertNull(plan);
       String message = e.getMessage();
-      Assert.assertTrue(message.equals(
+      assertTrue(message.equals(
               "The specified queue: " + ReservationSystemTestUtil.reservationQ
             + " is not managed by reservation system."
             + " Please try again with a valid reservable queue."

+ 11 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationSystemUtil.java

@@ -18,6 +18,9 @@
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
 import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.Mockito.mock;
 
 import org.apache.hadoop.yarn.api.records.ReservationAllocationState;
@@ -26,8 +29,7 @@ import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.util.Collections;
 import java.util.Date;
@@ -53,7 +55,7 @@ public class TestReservationSystemUtil {
     assertThat(infoList).hasSize(1);
     assertThat(infoList.get(0).getReservationId().toString()).isEqualTo(
         id.toString());
-    Assert.assertFalse(infoList.get(0).getResourceAllocationRequests()
+    assertFalse(infoList.get(0).getResourceAllocationRequests()
             .isEmpty());
   }
 
@@ -71,10 +73,10 @@ public class TestReservationSystemUtil {
             .convertAllocationsToReservationInfo(
                     Collections.singleton(allocation), false);
 
-    Assert.assertEquals(infoList.size(), 1);
-    Assert.assertEquals(infoList.get(0).getReservationId().toString(),
+    assertEquals(infoList.size(), 1);
+    assertEquals(infoList.get(0).getReservationId().toString(),
             id.toString());
-    Assert.assertTrue(infoList.get(0).getResourceAllocationRequests()
+    assertTrue(infoList.get(0).getResourceAllocationRequests()
             .isEmpty());
   }
 
@@ -92,10 +94,10 @@ public class TestReservationSystemUtil {
             .convertAllocationsToReservationInfo(
                     Collections.singleton(allocation), false);
 
-    Assert.assertEquals(infoList.size(), 1);
-    Assert.assertEquals(infoList.get(0).getReservationId().toString(),
+    assertEquals(infoList.size(), 1);
+    assertEquals(infoList.get(0).getReservationId().toString(),
             id.toString());
-    Assert.assertTrue(infoList.get(0).getResourceAllocationRequests()
+    assertTrue(infoList.get(0).getResourceAllocationRequests()
             .isEmpty());
   }
 

+ 24 - 24
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestSchedulerPlanFollowerBase.java

@@ -18,9 +18,10 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
-import static org.junit.Assert.assertNotNull;
-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.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.Mockito.when;
 
 import org.apache.hadoop.security.AccessControlException;
@@ -43,7 +44,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedule
 import org.apache.hadoop.util.Clock;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
-import org.junit.Assert;
 
 public abstract class TestSchedulerPlanFollowerBase {
   final static int GB = 1024;
@@ -73,23 +73,23 @@ public abstract class TestSchedulerPlanFollowerBase {
     ReservationDefinition rDef =
         ReservationSystemTestUtil.createSimpleReservationDefinition(
             0, 0 + f1.length + 1, f1.length);
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(r1, rDef, "u3",
-            "dedicated", 0, 0 + f1.length, ReservationSystemTestUtil
-                .generateAllocation(0L, 1L, f1), res, minAlloc), false));
+    assertTrue(plan.addReservation(new InMemoryReservationAllocation(r1, rDef, "u3",
+        "dedicated", 0, 0 + f1.length, ReservationSystemTestUtil
+        .generateAllocation(0L, 1L, f1), res, minAlloc), false),
+         plan.toString());
 
     ReservationId r2 = ReservationId.newInstance(ts, 2);
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(r2, rDef, "u3",
-            "dedicated", 3, 3 + f1.length, ReservationSystemTestUtil
-                .generateAllocation(3L, 1L, f1), res, minAlloc), false));
+    assertTrue(plan.addReservation(new InMemoryReservationAllocation(r2, rDef, "u3",
+        "dedicated", 3, 3 + f1.length, ReservationSystemTestUtil
+        .generateAllocation(3L, 1L, f1), res, minAlloc), false),
+        plan.toString());
 
     ReservationId r3 = ReservationId.newInstance(ts, 3);
     int[] f2 = { 0, 10, 20, 10, 0 };
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(r3, rDef, "u4",
-            "dedicated", 10, 10 + f2.length, ReservationSystemTestUtil
-                .generateAllocation(10L, 1L, f2), res, minAlloc), false));
+    assertTrue(plan.addReservation(new InMemoryReservationAllocation(r3, rDef, "u4",
+        "dedicated", 10, 10 + f2.length, ReservationSystemTestUtil
+        .generateAllocation(10L, 1L, f2), res, minAlloc), false),
+        plan.toString());
 
 
     // default reseration queue should exist before run of PlanFollower AND have
@@ -124,10 +124,10 @@ public abstract class TestSchedulerPlanFollowerBase {
 
     // initial default reservation queue should have no apps after first run
     Queue defQ = getDefaultQueue();
-    Assert.assertEquals(0, getNumberOfApplications(defQ));
+    assertEquals(0, getNumberOfApplications(defQ));
 
     assertReservationQueueExists(r1, 0.1, 0.1);
-    Assert.assertEquals(1, getNumberOfApplications(q));
+    assertEquals(1, getNumberOfApplications(q));
 
     assertReservationQueueDoesNotExist(r2);
     assertReservationQueueDoesNotExist(r3);
@@ -135,9 +135,9 @@ public abstract class TestSchedulerPlanFollowerBase {
     when(mClock.getTime()).thenReturn(3L);
     planFollower.run();
 
-    Assert.assertEquals(0, getNumberOfApplications(defQ));
+    assertEquals(0, getNumberOfApplications(defQ));
     assertReservationQueueExists(r1, 0.1, 0.1);
-    Assert.assertEquals(1, getNumberOfApplications(q));
+    assertEquals(1, getNumberOfApplications(q));
     assertReservationQueueExists(r2, 0.1, 0.1);
     assertReservationQueueDoesNotExist(r3);
 
@@ -147,11 +147,11 @@ public abstract class TestSchedulerPlanFollowerBase {
     q = getReservationQueue(r1.toString());
     if (isMove) {
       // app should have been moved to default reservation queue
-      Assert.assertEquals(1, getNumberOfApplications(defQ));
+      assertEquals(1, getNumberOfApplications(defQ));
       assertNull(q);
     } else {
       // app should be killed
-      Assert.assertEquals(0, getNumberOfApplications(defQ));
+      assertEquals(0, getNumberOfApplications(defQ));
       assertNotNull(q);
       AppAttemptRemovedSchedulerEvent appAttemptRemovedEvent =
           new AppAttemptRemovedSchedulerEvent(appAttemptId_0,
@@ -166,10 +166,10 @@ public abstract class TestSchedulerPlanFollowerBase {
 
     if (isMove) {
       // app should have been moved to default reservation queue
-      Assert.assertEquals(1, getNumberOfApplications(defQ));
+      assertEquals(1, getNumberOfApplications(defQ));
     } else {
       // app should be killed
-      Assert.assertEquals(0, getNumberOfApplications(defQ));
+      assertEquals(0, getNumberOfApplications(defQ));
     }
     assertReservationQueueDoesNotExist(r1);
     assertReservationQueueDoesNotExist(r2);

+ 165 - 149
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestAlignedPlanner.java

@@ -20,9 +20,9 @@ package org.apache.hadoop.yarn.server.resourcemanager.reservation.planning;
 
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.within;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.Mockito.mock;
 
 import java.util.ArrayList;
@@ -56,10 +56,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -67,12 +66,10 @@ import org.slf4j.LoggerFactory;
 /**
  * This class tests the {@code AlignedPlannerWithGreedy} agent.
  */
-@RunWith(value = Parameterized.class)
 @NotThreadSafe
 @SuppressWarnings("VisibilityModifier")
 public class TestAlignedPlanner {
 
-  @Parameterized.Parameter(value = 0)
   public String recurrenceExpression;
 
   final static String NONPERIODIC = "0";
@@ -92,8 +89,6 @@ public class TestAlignedPlanner {
   private Resource clusterCapacity;
   private long step;
 
-
-  @Parameterized.Parameters(name = "Testing: periodicity {0})")
   public static Collection<Object[]> data() {
     return Arrays.asList(new Object[][]{
             {NONPERIODIC},
@@ -102,8 +97,15 @@ public class TestAlignedPlanner {
     });
   }
 
-  @Test
-  public void testSingleReservationAccept() throws PlanningException {
+  public void initTestAlignedPlanner(String pRecurrenceExpression) {
+    this.recurrenceExpression = pRecurrenceExpression;
+  }
+
+  @ParameterizedTest(name = "Testing: periodicity {0}")
+  @MethodSource("data")
+  public void testSingleReservationAccept(String pRecurrenceExpression) throws PlanningException {
+
+    initTestAlignedPlanner(pRecurrenceExpression);
 
     // Prepare basic plan
     int numJobsInScenario = initializeScenario1();
@@ -126,25 +128,26 @@ public class TestAlignedPlanner {
     agentRight.createReservation(reservationID, "u1", plan, rr1);
 
     // CHECK: allocation was accepted
-    assertTrue("Agent-based allocation failed", reservationID != null);
-    assertTrue("Agent-based allocation failed", plan.getAllReservations()
-        .size() == numJobsInScenario + 1);
+    assertTrue(reservationID != null, "Agent-based allocation failed");
+    assertTrue(plan.getAllReservations()
+        .size() == numJobsInScenario + 1, "Agent-based allocation failed");
 
     // Get reservation
     ReservationAllocation alloc1 = plan.getReservationById(reservationID);
 
     // Verify allocation
-    assertTrue(alloc1.toString(),
-        check(alloc1, 10 * step, 20 * step, 10, 2048, 2));
+    assertTrue(check(alloc1, 10 * step, 20 * step,
+        10, 2048, 2), alloc1.toString());
 
     System.out.println("--------AFTER AGENT----------");
     System.out.println(plan.toString());
 
   }
 
-  @Test
-  public void testOrderNoGapImpossible() throws PlanningException {
-
+  @ParameterizedTest(name = "Testing: periodicity {0}")
+  @MethodSource("data")
+  public void testOrderNoGapImpossible(String pRecurrenceExpression) throws PlanningException {
+    initTestAlignedPlanner(pRecurrenceExpression);
     // Prepare basic plan
     int numJobsInScenario = initializeScenario2();
 
@@ -177,14 +180,15 @@ public class TestAlignedPlanner {
     }
 
     // CHECK: allocation was not accepted
-    assertTrue("Agent-based allocation should have failed", plan
-        .getAllReservations().size() == numJobsInScenario);
+    assertTrue(plan.getAllReservations().size() == numJobsInScenario,
+        "Agent-based allocation should have failed");
 
   }
 
-  @Test
-  public void testOrderNoGapImpossible2() throws PlanningException {
-
+  @ParameterizedTest(name = "Testing: periodicity {0}")
+  @MethodSource("data")
+  public void testOrderNoGapImpossible2(String pRecurrenceExpression) throws PlanningException {
+    initTestAlignedPlanner(pRecurrenceExpression);
     // Prepare basic plan
     int numJobsInScenario = initializeScenario2();
 
@@ -217,14 +221,15 @@ public class TestAlignedPlanner {
     }
 
     // CHECK: allocation was not accepted
-    assertTrue("Agent-based allocation should have failed", plan
-        .getAllReservations().size() == numJobsInScenario);
+    assertTrue(plan.getAllReservations().size() == numJobsInScenario,
+        "Agent-based allocation should have failed");
 
   }
 
-  @Test
-  public void testOrderImpossible() throws PlanningException {
-
+  @ParameterizedTest(name = "Testing: periodicity {0}")
+  @MethodSource("data")
+  public void testOrderImpossible(String pRecurrenceExpression) throws PlanningException {
+    initTestAlignedPlanner(pRecurrenceExpression);
     // Prepare basic plan
     int numJobsInScenario = initializeScenario2();
 
@@ -257,14 +262,15 @@ public class TestAlignedPlanner {
     }
 
     // CHECK: allocation was not accepted
-    assertTrue("Agent-based allocation should have failed", plan
-        .getAllReservations().size() == numJobsInScenario);
+    assertTrue(plan.getAllReservations().size() == numJobsInScenario,
+        "Agent-based allocation should have failed");
 
   }
 
-  @Test
-  public void testAnyImpossible() throws PlanningException {
-
+  @ParameterizedTest(name = "Testing: periodicity {0}")
+  @MethodSource("data")
+  public void testAnyImpossible(String pRecurrenceExpression) throws PlanningException {
+    initTestAlignedPlanner(pRecurrenceExpression);
     // Prepare basic plan
     int numJobsInScenario = initializeScenario2();
 
@@ -297,14 +303,15 @@ public class TestAlignedPlanner {
     }
 
     // CHECK: allocation was not accepted
-    assertTrue("Agent-based allocation should have failed", plan
-        .getAllReservations().size() == numJobsInScenario);
+    assertTrue(plan.getAllReservations().size() == numJobsInScenario,
+        "Agent-based allocation should have failed");
 
   }
 
-  @Test
-  public void testAnyAccept() throws PlanningException {
-
+  @ParameterizedTest(name = "Testing: periodicity {0}")
+  @MethodSource("data")
+  public void testAnyAccept(String pRecurrenceExpression) throws PlanningException {
+    initTestAlignedPlanner(pRecurrenceExpression);
     // Prepare basic plan
     int numJobsInScenario = initializeScenario2();
 
@@ -332,22 +339,23 @@ public class TestAlignedPlanner {
     agentRight.createReservation(reservationID, "u1", plan, rr1);
 
     // CHECK: allocation was accepted
-    assertTrue("Agent-based allocation failed", reservationID != null);
-    assertTrue("Agent-based allocation failed", plan.getAllReservations()
-        .size() == numJobsInScenario + 1);
+    assertTrue(reservationID != null, "Agent-based allocation failed");
+    assertTrue(plan.getAllReservations()
+        .size() == numJobsInScenario + 1, "Agent-based allocation failed");
 
     // Get reservation
     ReservationAllocation alloc1 = plan.getReservationById(reservationID);
 
     // Verify allocation
-    assertTrue(alloc1.toString(),
-        check(alloc1, 14 * step, 15 * step, 20, 1024, 1));
+    assertTrue(check(alloc1, 14 * step, 15 * step, 20, 1024, 1),
+        alloc1.toString());
 
   }
 
-  @Test
-  public void testAllAccept() throws PlanningException {
-
+  @ParameterizedTest(name = "Testing: periodicity {0}")
+  @MethodSource("data")
+  public void testAllAccept(String pRecurrenceExpression) throws PlanningException {
+    initTestAlignedPlanner(pRecurrenceExpression);
     // Prepare basic plan
     int numJobsInScenario = initializeScenario2();
 
@@ -375,24 +383,25 @@ public class TestAlignedPlanner {
     agentRight.createReservation(reservationID, "u1", plan, rr1);
 
     // CHECK: allocation was accepted
-    assertTrue("Agent-based allocation failed", reservationID != null);
-    assertTrue("Agent-based allocation failed", plan.getAllReservations()
-        .size() == numJobsInScenario + 1);
+    assertTrue(reservationID != null, "Agent-based allocation failed");
+    assertTrue(plan.getAllReservations()
+        .size() == numJobsInScenario + 1, "Agent-based allocation failed");
 
     // Get reservation
     ReservationAllocation alloc1 = plan.getReservationById(reservationID);
 
     // Verify allocation
-    assertTrue(alloc1.toString(),
-        check(alloc1, 10 * step, 11 * step, 20, 1024, 1));
-    assertTrue(alloc1.toString(),
-        check(alloc1, 14 * step, 15 * step, 20, 1024, 1));
+    assertTrue(check(alloc1, 10 * step, 11 * step, 20, 1024, 1),
+        alloc1.toString());
+    assertTrue(check(alloc1, 14 * step, 15 * step, 20, 1024, 1),
+        alloc1.toString());
 
   }
 
-  @Test
-  public void testAllImpossible() throws PlanningException {
-
+  @ParameterizedTest(name = "Testing: periodicity {0}")
+  @MethodSource("data")
+  public void testAllImpossible(String pRecurrenceExpression) throws PlanningException {
+    initTestAlignedPlanner(pRecurrenceExpression);
     // Prepare basic plan
     int numJobsInScenario = initializeScenario2();
 
@@ -425,14 +434,15 @@ public class TestAlignedPlanner {
     }
 
     // CHECK: allocation was not accepted
-    assertTrue("Agent-based allocation should have failed", plan
-        .getAllReservations().size() == numJobsInScenario);
+    assertTrue(plan.getAllReservations().size() == numJobsInScenario,
+        "Agent-based allocation should have failed");
 
   }
 
-  @Test
-  public void testUpdate() throws PlanningException {
-
+  @ParameterizedTest(name = "Testing: periodicity {0}")
+  @MethodSource("data")
+  public void testUpdate(String pRecurrenceExpression) throws PlanningException {
+    initTestAlignedPlanner(pRecurrenceExpression);
     // Create flexible reservation
     ReservationDefinition rrFlex =
         createReservationDefinition(
@@ -470,22 +480,23 @@ public class TestAlignedPlanner {
     agentRight.updateReservation(flexReservationID, "uFlex", plan, rrFlex);
 
     // CHECK: allocation was accepted
-    assertTrue("Agent-based allocation failed", flexReservationID != null);
-    assertTrue("Agent-based allocation failed", plan.getAllReservations()
-        .size() == 1);
+    assertTrue(flexReservationID != null, "Agent-based allocation failed");
+    assertTrue(plan.getAllReservations()
+        .size() == 1, "Agent-based allocation failed");
 
     // Get reservation
     ReservationAllocation alloc1 = plan.getReservationById(flexReservationID);
 
     // Verify allocation
-    assertTrue(alloc1.toString(),
-        check(alloc1, 10 * step, 14 * step, 50, 1024, 1));
+    assertTrue(check(alloc1, 10 * step, 14 * step,
+        50, 1024, 1), alloc1.toString());
 
   }
 
-  @Test
-  public void testImpossibleDuration() throws PlanningException {
-
+  @ParameterizedTest(name = "Testing: periodicity {0}")
+  @MethodSource("data")
+  public void testImpossibleDuration(String pRecurrenceExpression) throws PlanningException {
+    initTestAlignedPlanner(pRecurrenceExpression);
     // Create reservation
     ReservationDefinition rr1 =
         createReservationDefinition(
@@ -509,14 +520,15 @@ public class TestAlignedPlanner {
     }
 
     // CHECK: allocation was not accepted
-    assertTrue("Agent-based allocation should have failed", plan
-        .getAllReservations().size() == 0);
+    assertTrue(plan.getAllReservations().size() == 0,
+        "Agent-based allocation should have failed");
 
   }
 
-  @Test
-  public void testLoadedDurationIntervals() throws PlanningException {
-
+  @ParameterizedTest(name = "Testing: periodicity {0}")
+  @MethodSource("data")
+  public void testLoadedDurationIntervals(String pRecurrenceExpression) throws PlanningException {
+    initTestAlignedPlanner(pRecurrenceExpression);
     int numJobsInScenario = initializeScenario3();
 
     // Create reservation
@@ -537,25 +549,26 @@ public class TestAlignedPlanner {
     agentRight.createReservation(reservationID, "u1", plan, rr1);
 
     // CHECK: allocation was accepted
-    assertTrue("Agent-based allocation failed", reservationID != null);
-    assertTrue("Agent-based allocation failed", plan.getAllReservations()
-        .size() == numJobsInScenario + 1);
+    assertTrue(reservationID != null, "Agent-based allocation failed");
+    assertTrue(plan.getAllReservations()
+        .size() == numJobsInScenario + 1, "Agent-based allocation failed");
 
     // Get reservation
     ReservationAllocation alloc1 = plan.getReservationById(reservationID);
 
     // Verify allocation
-    assertTrue(alloc1.toString(),
-        check(alloc1, 10 * step, 11 * step, 20, 1024, 1));
-    assertTrue(alloc1.toString(),
-        check(alloc1, 11 * step, 12 * step, 20, 1024, 1));
-    assertTrue(alloc1.toString(),
-        check(alloc1, 12 * step, 13 * step, 40, 1024, 1));
+    assertTrue(check(alloc1, 10 * step, 11 * step, 20, 1024, 1),
+        alloc1.toString());
+    assertTrue(check(alloc1, 11 * step, 12 * step, 20, 1024, 1),
+        alloc1.toString());
+    assertTrue(check(alloc1, 12 * step, 13 * step, 40, 1024, 1),
+        alloc1.toString());
   }
 
-  @Test
-  public void testCostFunction() throws PlanningException {
-
+  @ParameterizedTest(name = "Testing: periodicity {0}")
+  @MethodSource("data")
+  public void testCostFunction(String pRecurrenceExpression) throws PlanningException {
+    initTestAlignedPlanner(pRecurrenceExpression);
     // Create large memory reservation
     ReservationDefinition rr7Mem1Core =
         createReservationDefinition(
@@ -608,16 +621,17 @@ public class TestAlignedPlanner {
     // Get reservation
     ReservationAllocation alloc3 = plan.getReservationById(reservationID3);
 
-    assertTrue(alloc3.toString(),
-        check(alloc3, 10 * step, 11 * step, 0, 1024, 1));
-    assertTrue(alloc3.toString(),
-        check(alloc3, 11 * step, 12 * step, 1, 1024, 1));
+    assertTrue(check(alloc3, 10 * step, 11 * step, 0, 1024, 1),
+        alloc3.toString());
+    assertTrue(check(alloc3, 11 * step, 12 * step, 1, 1024, 1),
+        alloc3.toString());
 
   }
 
-  @Test
-  public void testFromCluster() throws PlanningException {
-
+  @ParameterizedTest(name = "Testing: periodicity {0}")
+  @MethodSource("data")
+  public void testFromCluster(String pRecurrenceExpression) throws PlanningException {
+    initTestAlignedPlanner(pRecurrenceExpression);
     // int numJobsInScenario = initializeScenario3();
 
     List<ReservationDefinition> list = new ArrayList<ReservationDefinition>();
@@ -734,15 +748,16 @@ public class TestAlignedPlanner {
     }
 
     // CHECK: allocation was accepted
-    assertTrue("Agent-based allocation failed", plan.getAllReservations()
-        .size() == list.size());
+    assertTrue(plan.getAllReservations().size() == list.size(),
+        "Agent-based allocation failed");
 
   }
 
-  @Test
-  public void testSingleReservationAcceptAllocateLeft()
+  @ParameterizedTest(name = "Testing: periodicity {0}")
+  @MethodSource("data")
+  public void testSingleReservationAcceptAllocateLeft(String pRecurrenceExpression)
       throws PlanningException {
-
+    initTestAlignedPlanner(pRecurrenceExpression);
     // Create reservation
     ReservationDefinition rr1 =
         createReservationDefinition(
@@ -767,22 +782,23 @@ public class TestAlignedPlanner {
     agentLeft.createReservation(reservationID, "u1", plan, rr1);
 
     // CHECK: allocation was accepted
-    assertTrue("Agent-based allocation failed", reservationID != null);
-    assertTrue("Agent-based allocation failed", plan.getAllReservations()
-        .size() == 1);
+    assertTrue(reservationID != null, "Agent-based allocation failed");
+    assertTrue(plan.getAllReservations().size() == 1,
+        "Agent-based allocation failed");
 
     // Get reservation
     ReservationAllocation alloc1 = plan.getReservationById(reservationID);
 
     // Verify allocation
-    assertTrue(alloc1.toString(),
-        check(alloc1, 10 * step, 30 * step, 20, 1024, 1));
+    assertTrue(check(alloc1, 10 * step, 30 * step, 20, 1024, 1),
+        alloc1.toString());
 
   }
 
-  @Test
-  public void testLeftSucceedsRightFails() throws PlanningException {
-
+  @ParameterizedTest(name = "Testing: periodicity {0}")
+  @MethodSource("data")
+  public void testLeftSucceedsRightFails(String pRecurrenceExpression) throws PlanningException {
+    initTestAlignedPlanner(pRecurrenceExpression);
     // Prepare basic plan
     int numJobsInScenario = initializeScenario2();
 
@@ -820,16 +836,15 @@ public class TestAlignedPlanner {
     agentLeft.createReservation(reservationID1, "u1", plan, rr1);
 
     // CHECK: allocation was accepted
-    assertTrue("Agent-based allocation failed", reservationID1 != null);
-    assertTrue("Agent-based allocation failed", plan.getAllReservations()
-        .size() == numJobsInScenario + 1);
+    assertTrue(reservationID1 != null, "Agent-based allocation failed");
+    assertTrue(plan.getAllReservations().size() == numJobsInScenario + 1,
+        "Agent-based allocation failed");
 
     // Get reservation
     ReservationAllocation alloc1 = plan.getReservationById(reservationID1);
 
     // Verify allocation
-    assertTrue(alloc1.toString(),
-        check(alloc1, 7 * step, 11 * step, 20, 1024, 1));
+    assertTrue(check(alloc1, 7 * step, 11 * step, 20, 1024, 1), alloc1.toString());
 
     // Add second reservation
     ReservationId reservationID2 =
@@ -837,16 +852,16 @@ public class TestAlignedPlanner {
     agentLeft.createReservation(reservationID2, "u2", plan, rr2);
 
     // CHECK: allocation was accepted
-    assertTrue("Agent-based allocation failed", reservationID2 != null);
-    assertTrue("Agent-based allocation failed", plan.getAllReservations()
-        .size() == numJobsInScenario + 2);
+    assertTrue(reservationID2 != null, "Agent-based allocation failed");
+    assertTrue(plan.getAllReservations().size() == numJobsInScenario + 2,
+        "Agent-based allocation failed");
 
     // Get reservation
     ReservationAllocation alloc2 = plan.getReservationById(reservationID2);
 
     // Verify allocation
-    assertTrue(alloc2.toString(),
-        check(alloc2, 14 * step, 16 * step, 100, 1024, 1));
+    assertTrue(check(alloc2, 14 * step, 16 * step, 100, 1024, 1),
+        alloc2.toString());
 
     agentLeft.deleteReservation(reservationID1, "u1", plan);
     agentLeft.deleteReservation(reservationID2, "u2", plan);
@@ -859,9 +874,9 @@ public class TestAlignedPlanner {
     agentRight.createReservation(reservationID3, "u1", plan, rr1);
 
     // CHECK: allocation was accepted
-    assertTrue("Agent-based allocation failed", reservationID3 != null);
-    assertTrue("Agent-based allocation failed", plan.getAllReservations()
-        .size() == numJobsInScenario + 1);
+    assertTrue(reservationID3 != null, "Agent-based allocation failed");
+    assertTrue(plan.getAllReservations().size() == numJobsInScenario + 1,
+        "Agent-based allocation failed");
 
  // Add 2nd reservation
     try {
@@ -875,9 +890,10 @@ public class TestAlignedPlanner {
 
   }
 
-  @Test
-  public void testValidateOrderNoGap() {
-
+  @ParameterizedTest(name = "Testing: periodicity {0}")
+  @MethodSource("data")
+  public void testValidateOrderNoGap(String pRecurrenceExpression) {
+    initTestAlignedPlanner(pRecurrenceExpression);
     //
     // Initialize allocations
     //
@@ -899,51 +915,52 @@ public class TestAlignedPlanner {
     curAlloc.clear();
     curAlloc.put(new ReservationInterval(9 * step, 10 * step),
         Resource.newInstance(1024, 1));
-    assertTrue("validateOrderNoFap() should have succeeded",
-        IterativePlanner.validateOrderNoGap(allocation, curAlloc, false));
+    assertTrue(IterativePlanner.validateOrderNoGap(allocation, curAlloc, false),
+        "validateOrderNoFap() should have succeeded");
 
     // 2. allocateLeft = false, fail when curAlloc has a gap
     curAlloc.put(new ReservationInterval(7 * step, 8 * step),
         Resource.newInstance(1024, 1));
-    assertFalse("validateOrderNoGap() failed to identify a gap in curAlloc",
-        IterativePlanner.validateOrderNoGap(allocation, curAlloc, false));
+    assertFalse(IterativePlanner.validateOrderNoGap(allocation, curAlloc, false),
+        "validateOrderNoGap() failed to identify a gap in curAlloc");
 
     // 3. allocateLeft = false, fail when there is a gap between curAlloc and
     // allocations
     curAlloc.clear();
     curAlloc.put(new ReservationInterval(8 * step, 9 * step),
         Resource.newInstance(1024, 1));
-    assertFalse("validateOrderNoGap() failed to identify a gap between "
-        + "allocations and curAlloc",
-        IterativePlanner.validateOrderNoGap(allocation, curAlloc, false));
+    assertFalse(IterativePlanner.validateOrderNoGap(allocation, curAlloc, false),
+        "validateOrderNoGap() failed to identify a gap between "
+        + "allocations and curAlloc");
 
     // 4. allocateLeft = true, succeed when there is no gap
     curAlloc.clear();
     curAlloc.put(new ReservationInterval(13 * step, 14 * step),
         Resource.newInstance(1024, 1));
-    assertTrue("validateOrderNoFap() should have succeeded",
-        IterativePlanner.validateOrderNoGap(allocation, curAlloc, true));
+    assertTrue(IterativePlanner.validateOrderNoGap(allocation, curAlloc, true),
+        "validateOrderNoFap() should have succeeded");
 
     // 5. allocateLeft = true, fail when there is a gap between curAlloc and
     // allocations
     curAlloc.put(new ReservationInterval(15 * step, 16 * step),
         Resource.newInstance(1024, 1));
-    assertFalse("validateOrderNoGap() failed to identify a gap in curAlloc",
-        IterativePlanner.validateOrderNoGap(allocation, curAlloc, true));
+    assertFalse(IterativePlanner.validateOrderNoGap(allocation, curAlloc, true),
+        "validateOrderNoGap() failed to identify a gap in curAlloc");
 
     // 6. allocateLeft = true, fail when curAlloc has a gap
     curAlloc.clear();
     curAlloc.put(new ReservationInterval(14 * step, 15 * step),
         Resource.newInstance(1024, 1));
-    assertFalse("validateOrderNoGap() failed to identify a gap between "
-        + "allocations and curAlloc",
-        IterativePlanner.validateOrderNoGap(allocation, curAlloc, true));
+    assertFalse(IterativePlanner.validateOrderNoGap(allocation, curAlloc, true),
+        "validateOrderNoGap() failed to identify a gap between "
+        + "allocations and curAlloc");
 
   }
 
-  @Test
-  public void testGetDurationInterval() throws PlanningException {
-
+  @ParameterizedTest(name = "Testing: periodicity {0}")
+  @MethodSource("data")
+  public void testGetDurationInterval(String pRecurrenceExpression) throws PlanningException {
+    initTestAlignedPlanner(pRecurrenceExpression);
     DurationInterval durationInterval = null;
 
     // Create netRLERes:
@@ -1067,7 +1084,7 @@ public class TestAlignedPlanner {
 
   }
 
-  @Before
+  @BeforeEach
   public void setup() throws Exception {
 
     // Initialize random seed
@@ -1165,12 +1182,11 @@ public class TestAlignedPlanner {
     ReservationDefinition rDef =
         ReservationSystemTestUtil.createSimpleReservationDefinition(
             start, start + f.length * step, f.length * step);
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(
-            ReservationSystemTestUtil.getNewReservationId(), rDef,
-            "user_fixed", "dedicated", start, start + f.length * step,
-            ReservationSystemTestUtil.generateAllocation(start, step, f), res,
-            minAlloc), false));
+    assertTrue(plan.addReservation(new InMemoryReservationAllocation(
+        ReservationSystemTestUtil.getNewReservationId(), rDef,
+        "user_fixed", "dedicated", start, start + f.length * step,
+        ReservationSystemTestUtil.generateAllocation(start, step, f), res,
+        minAlloc), false), plan.toString());
 
   }
 

+ 154 - 133
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestGreedyReservationAgent.java

@@ -17,11 +17,11 @@
  *******************************************************************************/
 package org.apache.hadoop.yarn.server.resourcemanager.reservation.planning;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.Mockito.mock;
 
 import java.io.IOException;
@@ -55,22 +55,18 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-@RunWith(Parameterized.class)
+
 @SuppressWarnings("VisibilityModifier")
 public class TestGreedyReservationAgent {
 
 
-  @Parameterized.Parameter(value = 0)
   public boolean allocateLeft;
 
-  @Parameterized.Parameter(value = 1)
   public String recurrenceExpression;
 
   private static final Logger LOG = LoggerFactory
@@ -84,8 +80,6 @@ public class TestGreedyReservationAgent {
   Random rand = new Random();
   long step;
 
-  @Parameterized.Parameters(name = "Testing: allocateLeft {0}," +
-          " recurrenceExpression {1})")
   public static Collection<Object[]> data() {
       return Arrays.asList(new Object[][] {
               {true, "0"},
@@ -97,7 +91,13 @@ public class TestGreedyReservationAgent {
       });
   }
 
-  @Before
+  public void initTestGreedyReservationAgent(boolean pAllocateLeft,
+      String pRecurrenceExpression) throws Exception {
+    this.allocateLeft = pAllocateLeft;
+    this.recurrenceExpression = pRecurrenceExpression;
+    setup();
+  }
+
   public void setup() throws Exception {
 
     long seed = rand.nextLong();
@@ -134,9 +134,12 @@ public class TestGreedyReservationAgent {
   }
 
   @SuppressWarnings("javadoc")
-  @Test
-  public void testSimple() throws PlanningException {
-
+  @ParameterizedTest(name = "Testing: allocateLeft {0}," +
+      " recurrenceExpression {1}")
+  @MethodSource("data")
+  public void testSimple(boolean pAllocateLeft,
+      String pRecurrenceExpression) throws Exception {
+    initTestGreedyReservationAgent(pAllocateLeft, pRecurrenceExpression);
     prepareBasicPlan();
 
     // create a request with a single atomic ask
@@ -154,9 +157,9 @@ public class TestGreedyReservationAgent {
         .getNewReservationId();
     agent.createReservation(reservationID, "u1", plan, rr);
 
-    assertTrue("Agent-based allocation failed", reservationID != null);
-    assertTrue("Agent-based allocation failed", plan.getAllReservations()
-        .size() == 3);
+    assertTrue(reservationID != null, "Agent-based allocation failed");
+    assertTrue(plan.getAllReservations()
+        .size() == 3, "Agent-based allocation failed");
 
     ReservationAllocation cs = plan.getReservationById(reservationID);
 
@@ -167,25 +170,24 @@ public class TestGreedyReservationAgent {
 
     if(allocateLeft){
       for (long i = 5 * step; i < 15 * step; i++) {
-        assertTrue(
-            "Agent-based allocation unexpected",
-            Resources.equals(cs.getResourcesAtTime(i),
-                Resource.newInstance(2048 * 10, 2 * 10)));
+        assertTrue(Resources.equals(cs.getResourcesAtTime(i),
+            Resource.newInstance(2048 * 10, 2 * 10)), "Agent-based allocation unexpected");
       }
     } else {
       for (long i = 10 * step; i < 20 * step; i++) {
-        assertTrue(
-            "Agent-based allocation unexpected",
-            Resources.equals(cs.getResourcesAtTime(i),
-                Resource.newInstance(2048 * 10, 2 * 10)));
+        assertTrue(Resources.equals(cs.getResourcesAtTime(i),
+            Resource.newInstance(2048 * 10, 2 * 10)), "Agent-based allocation unexpected");
       }
     }
   }
 
   @SuppressWarnings("javadoc")
-  @Test
-  public void testSharingPolicyFeedback() throws PlanningException {
-
+  @ParameterizedTest(name = "Testing: allocateLeft {0}," +
+      " recurrenceExpression {1}")
+  @MethodSource("data")
+  public void testSharingPolicyFeedback(boolean pAllocateLeft,
+      String pRecurrenceExpression) throws Exception {
+    initTestGreedyReservationAgent(pAllocateLeft, pRecurrenceExpression);
     prepareBasicPlan();
 
     // let's constraint the instantaneous allocation and see the
@@ -239,9 +241,9 @@ public class TestGreedyReservationAgent {
       // expected
     }
 
-    assertTrue("Agent-based allocation failed", reservationID != null);
-    assertTrue("Agent-based allocation failed", plan.getAllReservations()
-        .size() == 4);
+    assertTrue(reservationID != null, "Agent-based allocation failed");
+    assertTrue(plan.getAllReservations()
+        .size() == 4, "Agent-based allocation failed");
 
     ReservationAllocation cs = plan.getReservationById(reservationID);
     ReservationAllocation cs2 = plan.getReservationById(reservationID2);
@@ -258,36 +260,32 @@ public class TestGreedyReservationAgent {
 
     if (allocateLeft) {
       for (long i = 5 * step; i < 15 * step; i++) {
-        assertTrue(
-            "Agent-based allocation unexpected",
-            Resources.equals(cs.getResourcesAtTime(i),
-                Resource.newInstance(2048 * 20, 2 * 20)));
+        assertTrue(Resources.equals(cs.getResourcesAtTime(i),
+            Resource.newInstance(2048 * 20, 2 * 20)), "Agent-based allocation unexpected");
       }
       for (long i = 15 * step; i < 25 * step; i++) {
         // RR2 is pushed out by the presence of RR
-        assertTrue(
-            "Agent-based allocation unexpected",
-            Resources.equals(cs2.getResourcesAtTime(i),
-                Resource.newInstance(2048 * 20, 2 * 20)));
+        assertTrue(Resources.equals(cs2.getResourcesAtTime(i),
+            Resource.newInstance(2048 * 20, 2 * 20)), "Agent-based allocation unexpected");
       }
     } else {
       for (long i = 90 * step; i < 100 * step; i++) {
-        assertTrue(
-            "Agent-based allocation unexpected",
-            Resources.equals(cs.getResourcesAtTime(i),
-                Resource.newInstance(2048 * 20, 2 * 20)));
+        assertTrue(Resources.equals(cs.getResourcesAtTime(i),
+            Resource.newInstance(2048 * 20, 2 * 20)), "Agent-based allocation unexpected");
       }
       for (long i = 80 * step; i < 90 * step; i++) {
-        assertTrue(
-            "Agent-based allocation unexpected",
-            Resources.equals(cs2.getResourcesAtTime(i),
-                Resource.newInstance(2048 * 20, 2 * 20)));
+        assertTrue(Resources.equals(cs2.getResourcesAtTime(i),
+            Resource.newInstance(2048 * 20, 2 * 20)), "Agent-based allocation unexpected");
       }
     }
   }
 
-  @Test
-  public void testOrder() throws PlanningException {
+  @ParameterizedTest(name = "Testing: allocateLeft {0}," +
+      " recurrenceExpression {1}")
+  @MethodSource("data")
+  public void testOrder(boolean pAllocateLeft,
+      String pRecurrenceExpression) throws Exception {
+    initTestGreedyReservationAgent(pAllocateLeft, pRecurrenceExpression);
     prepareBasicPlan();
 
     // create a completely utilized segment around time 30
@@ -297,12 +295,11 @@ public class TestGreedyReservationAgent {
         ReservationSystemTestUtil.createSimpleReservationDefinition(30 * step,
             30 * step + f.length * step, f.length * step, 1,
             recurrenceExpression);
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(
-            ReservationSystemTestUtil.getNewReservationId(), rDef, "u1",
-            "dedicated", 30 * step, 30 * step + f.length * step,
-            ReservationSystemTestUtil.generateAllocation(30 * step, step, f),
-            res, minAlloc), false));
+    assertTrue(plan.addReservation(new InMemoryReservationAllocation(
+        ReservationSystemTestUtil.getNewReservationId(), rDef, "u1",
+        "dedicated", 30 * step, 30 * step + f.length * step,
+        ReservationSystemTestUtil.generateAllocation(30 * step, step, f),
+        res, minAlloc), false), plan.toString());
 
     // create a chain of 4 RR, mixing gang and non-gang
     ReservationDefinition rr = new ReservationDefinitionPBImpl();
@@ -329,23 +326,23 @@ public class TestGreedyReservationAgent {
     agent.createReservation(reservationID, "u1", plan, rr);
 
     // validate
-    assertTrue("Agent-based allocation failed", reservationID != null);
-    assertTrue("Agent-based allocation failed", plan.getAllReservations()
-        .size() == 4);
+    assertTrue(reservationID != null, "Agent-based allocation failed");
+    assertTrue(plan.getAllReservations()
+        .size() == 4, "Agent-based allocation failed");
 
     ReservationAllocation cs = plan.getReservationById(reservationID);
 
     if (allocateLeft) {
-      assertTrue(cs.toString(), check(cs, 0 * step, 10 * step, 20, 1024, 1));
-      assertTrue(cs.toString(), check(cs, 10 * step, 30 * step, 10, 1024, 1));
-      assertTrue(cs.toString(), check(cs, 32 * step, 42 * step, 20, 1024, 1));
-      assertTrue(cs.toString(), check(cs, 42 * step, 62 * step, 10, 1024, 1));
+      assertTrue(check(cs, 0 * step, 10 * step, 20, 1024, 1), cs.toString());
+      assertTrue(check(cs, 10 * step, 30 * step, 10, 1024, 1), cs.toString());
+      assertTrue(check(cs, 32 * step, 42 * step, 20, 1024, 1), cs.toString());
+      assertTrue(check(cs, 42 * step, 62 * step, 10, 1024, 1), cs.toString());
 
     } else {
-      assertTrue(cs.toString(), check(cs, 0 * step, 10 * step, 20, 1024, 1));
-      assertTrue(cs.toString(), check(cs, 10 * step, 30 * step, 10, 1024, 1));
-      assertTrue(cs.toString(), check(cs, 40 * step, 50 * step, 20, 1024, 1));
-      assertTrue(cs.toString(), check(cs, 50 * step, 70 * step, 10, 1024, 1));
+      assertTrue(check(cs, 0 * step, 10 * step, 20, 1024, 1), cs.toString());
+      assertTrue(check(cs, 10 * step, 30 * step, 10, 1024, 1), cs.toString());
+      assertTrue(check(cs, 40 * step, 50 * step, 20, 1024, 1), cs.toString());
+      assertTrue(check(cs, 50 * step, 70 * step, 10, 1024, 1), cs.toString());
     }
     System.out.println("--------AFTER ORDER ALLOCATION (queue: "
         + reservationID + ")----------");
@@ -354,21 +351,23 @@ public class TestGreedyReservationAgent {
 
   }
 
-  @Test
-  public void testOrderNoGapImpossible() throws PlanningException {
+  @ParameterizedTest(name = "Testing: allocateLeft {0}," +
+      " recurrenceExpression {1}")
+  @MethodSource("data")
+  public void testOrderNoGapImpossible(boolean pAllocateLeft,
+      String pRecurrenceExpression) throws Exception {
+    initTestGreedyReservationAgent(pAllocateLeft, pRecurrenceExpression);
     prepareBasicPlan();
     // create a completely utilized segment at time 30
     int[] f = { 100, 100 };
     ReservationDefinition rDef = ReservationSystemTestUtil
         .createSimpleReservationDefinition(30, 30 * step + f.length * step,
             f.length * step, 1, recurrenceExpression);
-    assertTrue(
-        plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(
-            ReservationSystemTestUtil.getNewReservationId(), rDef, "u1",
-            "dedicated", 30 * step, 30 * step + f.length * step,
-            ReservationSystemTestUtil.generateAllocation(30 * step, step, f),
-            res, minAlloc), false));
+    assertTrue(plan.addReservation(new InMemoryReservationAllocation(
+        ReservationSystemTestUtil.getNewReservationId(), rDef, "u1",
+        "dedicated", 30 * step, 30 * step + f.length * step,
+        ReservationSystemTestUtil.generateAllocation(30 * step, step, f),
+        res, minAlloc), false), plan.toString());
 
     // create a chain of 4 RR, mixing gang and non-gang
     ReservationDefinition rr = new ReservationDefinitionPBImpl();
@@ -401,9 +400,9 @@ public class TestGreedyReservationAgent {
     }
 
     // validate
-    assertFalse("Agent-based allocation should have failed", result);
-    assertTrue("Agent-based allocation should have failed", plan
-        .getAllReservations().size() == 3);
+    assertFalse(result, "Agent-based allocation should have failed");
+    assertTrue(plan.getAllReservations().size() == 3,
+        "Agent-based allocation should have failed");
 
     System.out
         .println("--------AFTER ORDER_NO_GAP IMPOSSIBLE ALLOCATION (queue: "
@@ -413,8 +412,12 @@ public class TestGreedyReservationAgent {
 
   }
 
-  @Test
-  public void testOrderNoGap() throws PlanningException {
+  @ParameterizedTest(name = "Testing: allocateLeft {0}," +
+      " recurrenceExpression {1}")
+  @MethodSource("data")
+  public void testOrderNoGap(boolean pAllocateLeft,
+      String pRecurrenceExpression) throws Exception {
+    initTestGreedyReservationAgent(pAllocateLeft, pRecurrenceExpression);
     prepareBasicPlan();
     // create a chain of 4 RR, mixing gang and non-gang
     ReservationDefinition rr = new ReservationDefinitionPBImpl();
@@ -447,21 +450,25 @@ public class TestGreedyReservationAgent {
     System.out.println(plan.toCumulativeString());
 
     // validate
-    assertTrue("Agent-based allocation failed", reservationID != null);
-    assertTrue("Agent-based allocation failed", plan.getAllReservations()
-        .size() == 3);
+    assertTrue(reservationID != null, "Agent-based allocation failed");
+    assertTrue(plan.getAllReservations()
+        .size() == 3, "Agent-based allocation failed");
 
     ReservationAllocation cs = plan.getReservationById(reservationID);
 
-    assertTrue(cs.toString(), check(cs, 0 * step, 10 * step, 20, 1024, 1));
-    assertTrue(cs.toString(), check(cs, 10 * step, 30 * step, 10, 1024, 1));
-    assertTrue(cs.toString(), check(cs, 30 * step, 40 * step, 20, 1024, 1));
-    assertTrue(cs.toString(), check(cs, 40 * step, 60 * step, 10, 1024, 1));
+    assertTrue(check(cs, 0 * step, 10 * step, 20, 1024, 1), cs.toString());
+    assertTrue(check(cs, 10 * step, 30 * step, 10, 1024, 1), cs.toString());
+    assertTrue(check(cs, 30 * step, 40 * step, 20, 1024, 1), cs.toString());
+    assertTrue(check(cs, 40 * step, 60 * step, 10, 1024, 1), cs.toString());
 
   }
 
-  @Test
-  public void testSingleSliding() throws PlanningException {
+  @ParameterizedTest(name = "Testing: allocateLeft {0}," +
+      " recurrenceExpression {1}")
+  @MethodSource("data")
+  public void testSingleSliding(boolean pAllocateLeft,
+      String pRecurrenceExpression) throws Exception {
+    initTestGreedyReservationAgent(pAllocateLeft, pRecurrenceExpression);
     prepareBasicPlan();
 
     // create a single request for which we need subsequent (tight) packing.
@@ -485,13 +492,13 @@ public class TestGreedyReservationAgent {
     agent.createReservation(reservationID, "u1", plan, rr);
 
     // validate results, we expect the second one to be accepted
-    assertTrue("Agent-based allocation failed", reservationID != null);
-    assertTrue("Agent-based allocation failed", plan.getAllReservations()
-        .size() == 3);
+    assertTrue(reservationID != null, "Agent-based allocation failed");
+    assertTrue(plan.getAllReservations()
+        .size() == 3, "Agent-based allocation failed");
 
     ReservationAllocation cs = plan.getReservationById(reservationID);
 
-    assertTrue(cs.toString(), check(cs, 100 * step, 120 * step, 100, 1024, 1));
+    assertTrue(check(cs, 100 * step, 120 * step, 100, 1024, 1), cs.toString());
 
     System.out.println("--------AFTER packed ALLOCATION (queue: "
         + reservationID + ")----------");
@@ -500,8 +507,12 @@ public class TestGreedyReservationAgent {
 
   }
 
-  @Test
-  public void testAny() throws PlanningException {
+  @ParameterizedTest(name = "Testing: allocateLeft {0}," +
+      " recurrenceExpression {1}")
+  @MethodSource("data")
+  public void testAny(boolean pAllocateLeft,
+      String pRecurrenceExpression) throws Exception {
+    initTestGreedyReservationAgent(pAllocateLeft, pRecurrenceExpression);
     prepareBasicPlan();
     // create an ANY request, with an impossible step (last in list, first
     // considered),
@@ -533,16 +544,16 @@ public class TestGreedyReservationAgent {
     boolean res = agent.createReservation(reservationID, "u1", plan, rr);
 
     // validate results, we expect the second one to be accepted
-    assertTrue("Agent-based allocation failed", res);
-    assertTrue("Agent-based allocation failed", plan.getAllReservations()
-        .size() == 3);
+    assertTrue(res, "Agent-based allocation failed");
+    assertTrue(plan.getAllReservations()
+        .size() == 3, "Agent-based allocation failed");
 
     ReservationAllocation cs = plan.getReservationById(reservationID);
 
     if (allocateLeft) {
-      assertTrue(cs.toString(), check(cs, 100 * step, 110 * step, 5, 1024, 1));
+      assertTrue(check(cs, 100 * step, 110 * step, 5, 1024, 1), cs.toString());
     } else {
-      assertTrue(cs.toString(), check(cs, 110 * step, 120 * step, 20, 1024, 1));
+      assertTrue(check(cs, 110 * step, 120 * step, 20, 1024, 1), cs.toString());
     }
 
     System.out.println("--------AFTER ANY ALLOCATION (queue: " + reservationID
@@ -552,8 +563,12 @@ public class TestGreedyReservationAgent {
 
   }
 
-  @Test
-  public void testAnyImpossible() throws PlanningException {
+  @ParameterizedTest(name = "Testing: allocateLeft {0}," +
+      " recurrenceExpression {1}")
+  @MethodSource("data")
+  public void testAnyImpossible(boolean pAllocateLeft,
+      String pRecurrenceExpression) throws Exception {
+    initTestGreedyReservationAgent(pAllocateLeft, pRecurrenceExpression);
     prepareBasicPlan();
     // create an ANY request, with all impossible alternatives
     ReservationDefinition rr = new ReservationDefinitionPBImpl();
@@ -587,9 +602,9 @@ public class TestGreedyReservationAgent {
       // expected
     }
     // validate results, we expect the second one to be accepted
-    assertFalse("Agent-based allocation should have failed", result);
-    assertTrue("Agent-based allocation should have failed", plan
-        .getAllReservations().size() == 2);
+    assertFalse(result, "Agent-based allocation should have failed");
+    assertTrue(plan.getAllReservations().size() == 2,
+        "Agent-based allocation should have failed");
 
     System.out.println("--------AFTER ANY IMPOSSIBLE ALLOCATION (queue: "
         + reservationID + ")----------");
@@ -598,8 +613,12 @@ public class TestGreedyReservationAgent {
 
   }
 
-  @Test
-  public void testAll() throws PlanningException {
+  @ParameterizedTest(name = "Testing: allocateLeft {0}," +
+      " recurrenceExpression {1}")
+  @MethodSource("data")
+  public void testAll(boolean pAllocateLeft,
+      String pRecurrenceExpression) throws Exception {
+    initTestGreedyReservationAgent(pAllocateLeft, pRecurrenceExpression);
     prepareBasicPlan();
     // create an ALL request
     ReservationDefinition rr = new ReservationDefinitionPBImpl();
@@ -625,18 +644,18 @@ public class TestGreedyReservationAgent {
     agent.createReservation(reservationID, "u1", plan, rr);
 
     // validate results, we expect the second one to be accepted
-    assertTrue("Agent-based allocation failed", reservationID != null);
-    assertTrue("Agent-based allocation failed", plan.getAllReservations()
-        .size() == 3);
+    assertTrue(reservationID != null, "Agent-based allocation failed");
+    assertTrue(plan.getAllReservations()
+        .size() == 3, "Agent-based allocation failed");
 
     ReservationAllocation cs = plan.getReservationById(reservationID);
 
     if (allocateLeft) {
-      assertTrue(cs.toString(), check(cs, 100 * step, 110 * step, 25, 1024, 1));
-      assertTrue(cs.toString(), check(cs, 110 * step, 120 * step, 20, 1024, 1));
+      assertTrue(check(cs, 100 * step, 110 * step, 25, 1024, 1), cs.toString());
+      assertTrue(check(cs, 110 * step, 120 * step, 20, 1024, 1), cs.toString());
     } else {
-      assertTrue(cs.toString(), check(cs, 100 * step, 110 * step, 20, 1024, 1));
-      assertTrue(cs.toString(), check(cs, 110 * step, 120 * step, 25, 1024, 1));
+      assertTrue(check(cs, 100 * step, 110 * step, 20, 1024, 1), cs.toString());
+      assertTrue(check(cs, 110 * step, 120 * step, 25, 1024, 1), cs.toString());
     }
 
     System.out.println("--------AFTER ALL ALLOCATION (queue: " + reservationID
@@ -646,8 +665,12 @@ public class TestGreedyReservationAgent {
 
   }
 
-  @Test
-  public void testAllImpossible() throws PlanningException {
+  @ParameterizedTest(name = "Testing: allocateLeft {0}," +
+      " recurrenceExpression {1}")
+  @MethodSource("data")
+  public void testAllImpossible(boolean pAllocateLeft,
+      String pRecurrenceExpression) throws Exception {
+    initTestGreedyReservationAgent(pAllocateLeft, pRecurrenceExpression);
     prepareBasicPlan();
     // create an ALL request, with an impossible combination, it should be
     // rejected, and allocation remain unchanged
@@ -680,9 +703,9 @@ public class TestGreedyReservationAgent {
     }
 
     // validate results, we expect the second one to be accepted
-    assertFalse("Agent-based allocation failed", result);
-    assertTrue("Agent-based allocation failed", plan.getAllReservations()
-        .size() == 2);
+    assertFalse(result, "Agent-based allocation failed");
+    assertTrue(plan.getAllReservations().size() == 2,
+        "Agent-based allocation failed");
 
     System.out.println("--------AFTER ALL IMPOSSIBLE ALLOCATION (queue: "
         + reservationID + ")----------");
@@ -700,20 +723,18 @@ public class TestGreedyReservationAgent {
     ReservationDefinition rDef =
         ReservationSystemTestUtil.createSimpleReservationDefinition(
             0, 0 + f.length * step, f.length * step);
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(
-            ReservationSystemTestUtil.getNewReservationId(), rDef, "u1",
-            "dedicated", 0L, 0L + f.length * step, ReservationSystemTestUtil
-                .generateAllocation(0, step, f), res, minAlloc), false));
+    assertTrue(plan.addReservation(new InMemoryReservationAllocation(
+        ReservationSystemTestUtil.getNewReservationId(), rDef, "u1",
+        "dedicated", 0L, 0L + f.length * step, ReservationSystemTestUtil
+        .generateAllocation(0, step, f), res, minAlloc), false), plan.toString());
 
     int[] f2 = { 5, 5, 5, 5, 5, 5, 5 };
     Map<ReservationInterval, Resource> alloc =
         ReservationSystemTestUtil.generateAllocation(5000, step, f2);
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(
-            ReservationSystemTestUtil.getNewReservationId(), rDef, "u1",
-            "dedicated", 5000, 5000 + f2.length * step, alloc, res, minAlloc),
-        false));
+    assertTrue(plan.addReservation(new InMemoryReservationAllocation(
+        ReservationSystemTestUtil.getNewReservationId(), rDef, "u1",
+        "dedicated", 5000, 5000 + f2.length * step, alloc, res, minAlloc),
+        false), plan.toString());
 
     System.out.println("--------BEFORE AGENT----------");
     System.out.println(plan.toString());

+ 17 - 14
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestReservationAgents.java

@@ -38,10 +38,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -58,20 +56,15 @@ import static org.mockito.Mockito.mock;
 /**
  * General purpose ReservationAgent tester.
  */
-@RunWith(Parameterized.class)
 @SuppressWarnings("VisibilityModifier")
 public class TestReservationAgents {
 
-  @Parameterized.Parameter(value = 0)
   public Class agentClass;
 
-  @Parameterized.Parameter(value = 1)
   public boolean allocateLeft;
 
-  @Parameterized.Parameter(value = 2)
   public String recurrenceExpression;
 
-  @Parameterized.Parameter(value = 3)
   public int numOfNodes;
 
   private long step;
@@ -87,8 +80,15 @@ public class TestReservationAgents {
   private static final Logger LOG =
       LoggerFactory.getLogger(TestReservationAgents.class);
 
-  @Parameterized.Parameters(name = "Testing: agent {0}, allocateLeft: {1}," +
-          " recurrenceExpression: {2}, numNodes: {3})")
+  public void initTestReservationAgents(Class pAgentClass, boolean pAllocateLeft,
+      String pRecurrenceExpression, int pNumOfNodes) throws Exception {
+    this.agentClass = pAgentClass;
+    this.allocateLeft = pAllocateLeft;
+    this.recurrenceExpression = pRecurrenceExpression;
+    this.numOfNodes = pNumOfNodes;
+    setup();
+  }
+
   public static Collection<Object[]> data() {
     return Arrays.asList(
         new Object[][] {{GreedyReservationAgent.class, true, "0", 100 },
@@ -105,7 +105,6 @@ public class TestReservationAgents {
             {AlignedPlannerWithGreedy.class, false, "86400000", 100 } });
   }
 
-  @Before
   public void setup() throws Exception {
 
     long seed = rand.nextLong();
@@ -141,8 +140,12 @@ public class TestReservationAgents {
         resCalc, minAlloc, maxAlloc, "dedicated", null, true, context);
   }
 
-  @Test
-  public void test() throws Exception {
+  @ParameterizedTest(name = "Testing: agent {0}, allocateLeft: {1}," +
+      " recurrenceExpression: {2}, numNodes: {3}")
+  @MethodSource("data")
+  public void test(Class pAgentClass, boolean pAllocateLeft,
+      String pRecurrenceExpression, int pNumOfNodes) throws Exception {
+    initTestReservationAgents(pAgentClass, pAllocateLeft, pRecurrenceExpression, pNumOfNodes);
 
     long period = Long.parseLong(recurrenceExpression);
     for (int i = 0; i < 1000; i++) {

+ 31 - 32
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java

@@ -17,9 +17,9 @@
  *******************************************************************************/
 package org.apache.hadoop.yarn.server.resourcemanager.reservation.planning;
 
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -48,7 +48,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePat
 import org.apache.hadoop.util.Clock;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 public class TestSimpleCapacityReplanner {
 
@@ -93,47 +93,46 @@ public class TestSimpleCapacityReplanner {
     ReservationDefinition rDef =
         ReservationSystemTestUtil.createSimpleReservationDefinition(
             0, 0 + f5.length, f5.length);
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(r1, rDef, "u3",
-            "dedicated", 0, 0 + f5.length, generateAllocation(0, f5), res,
-            minAlloc), false));
+    assertTrue(plan.addReservation(
+        new InMemoryReservationAllocation(r1, rDef, "u3",
+        "dedicated", 0, 0 + f5.length, generateAllocation(0, f5), res,
+        minAlloc), false), plan.toString());
     when(clock.getTime()).thenReturn(1L);
     ReservationId r2 = ReservationId.newInstance(ts, 2);
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(r2, rDef, "u4",
-            "dedicated", 0, 0 + f5.length, generateAllocation(0, f5), res,
-            minAlloc), false));
+    assertTrue(plan.addReservation(
+        new InMemoryReservationAllocation(r2, rDef, "u4",
+        "dedicated", 0, 0 + f5.length, generateAllocation(0, f5), res,
+        minAlloc), false), plan.toString());
     when(clock.getTime()).thenReturn(2L);
     ReservationId r3 = ReservationId.newInstance(ts, 3);
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(r3, rDef, "u5",
-            "dedicated", 0, 0 + f5.length, generateAllocation(0, f5), res,
-            minAlloc), false));
+    assertTrue(plan.addReservation(
+        new InMemoryReservationAllocation(r3, rDef, "u5",
+        "dedicated", 0, 0 + f5.length, generateAllocation(0, f5), res,
+        minAlloc), false), plan.toString());
     when(clock.getTime()).thenReturn(3L);
     ReservationId r4 = ReservationId.newInstance(ts, 4);
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(r4, rDef, "u6",
-            "dedicated", 0, 0 + f5.length, generateAllocation(0, f5), res,
-            minAlloc), false));
+    assertTrue(plan.addReservation(
+        new InMemoryReservationAllocation(r4, rDef, "u6",
+        "dedicated", 0, 0 + f5.length, generateAllocation(0, f5), res,
+        minAlloc), false), plan.toString());
     when(clock.getTime()).thenReturn(4L);
     ReservationId r5 = ReservationId.newInstance(ts, 5);
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(r5, rDef, "u7",
-            "dedicated", 0, 0 + f5.length, generateAllocation(0, f5), res,
-            minAlloc), false));
+    assertTrue(plan.addReservation(new InMemoryReservationAllocation(r5, rDef, "u7",
+        "dedicated", 0, 0 + f5.length, generateAllocation(0, f5), res,
+        minAlloc), false), plan.toString());
 
     int[] f6 = { 50, 50, 50, 50, 50 };
     ReservationId r6 = ReservationId.newInstance(ts, 6);
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(r6, rDef, "u3",
-            "dedicated", 10, 10 + f6.length, generateAllocation(10, f6), res,
-            minAlloc), false));
+    assertTrue(plan.addReservation(
+        new InMemoryReservationAllocation(r6, rDef, "u3",
+        "dedicated", 10, 10 + f6.length, generateAllocation(10, f6), res,
+        minAlloc), false), plan.toString());
     when(clock.getTime()).thenReturn(6L);
     ReservationId r7 = ReservationId.newInstance(ts, 7);
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(r7, rDef, "u4",
-            "dedicated", 10, 10 + f6.length, generateAllocation(10, f6), res,
-            minAlloc), false));
+    assertTrue(plan.addReservation(
+        new InMemoryReservationAllocation(r7, rDef, "u4",
+        "dedicated", 10, 10 + f6.length, generateAllocation(10, f6), res,
+        minAlloc), false), plan.toString());
 
     // remove some of the resources (requires replanning)
     plan.setTotalCapacity(Resource.newInstance(70 * 1024, 70));

+ 29 - 26
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resource/TestResourceProfiles.java

@@ -18,6 +18,10 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.resource;
 
+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 org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -29,8 +33,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRMAppSubmitter;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import java.io.IOException;
 import java.util.HashMap;
@@ -51,16 +55,14 @@ public class TestResourceProfiles {
     manager.init(conf);
     try {
       manager.getResourceProfiles();
-      Assert
-          .fail("Exception should be thrown as resource profile is not enabled"
-              + " and getResourceProfiles is invoked.");
+      fail("Exception should be thrown as resource profile is not enabled"
+          + " and getResourceProfiles is invoked.");
     } catch (YarnException ie) {
     }
     conf.setBoolean(YarnConfiguration.RM_RESOURCE_PROFILES_ENABLED, true);
     try {
       manager.init(conf);
-      Assert.fail(
-          "Exception should be thrown due to missing resource profiles file");
+      fail("Exception should be thrown due to missing resource profiles file");
     } catch (IOException ie) {
     }
     conf.set(YarnConfiguration.RM_RESOURCE_PROFILES_SOURCE_FILE,
@@ -85,10 +87,10 @@ public class TestResourceProfiles {
     for (Map.Entry<String, Resource> entry : expected.entrySet()) {
       String profile = entry.getKey();
       Resource res = entry.getValue();
-      Assert.assertTrue("Mandatory profile '" + profile + "' missing",
-          profiles.containsKey(profile));
-      Assert.assertEquals("Profile " + profile + "' resources don't match", res,
-          manager.getProfile(profile));
+      assertTrue(profiles.containsKey(profile),
+          "Mandatory profile '" + profile + "' missing");
+      assertEquals(res, manager.getProfile(profile),
+          "Profile " + profile + "' resources don't match");
     }
   }
 
@@ -105,7 +107,7 @@ public class TestResourceProfiles {
       conf.set(YarnConfiguration.RM_RESOURCE_PROFILES_SOURCE_FILE, file);
       try {
         manager.init(conf);
-        Assert.fail("Bad profile '" + file + "' is not valid");
+        fail("Bad profile '" + file + "' is not valid");
       } catch (IOException ie) {
       }
     }
@@ -130,8 +132,8 @@ public class TestResourceProfiles {
     for (Map.Entry<String, Resource> entry : expected.entrySet()) {
       String profile = entry.getKey();
       Resource res = entry.getValue();
-      Assert.assertEquals("Profile " + profile + "' resources don't match", res,
-          manager.getProfile(profile));
+      assertEquals(res, manager.getProfile(profile),
+          "Profile " + profile + "' resources don't match");
     }
   }
 
@@ -148,16 +150,17 @@ public class TestResourceProfiles {
     expected.put("default", Resource.newInstance(2048, 2));
     expected.put("maximum", Resource.newInstance(8192, 4));
 
-    Assert.assertEquals("Profile 'minimum' resources don't match",
-        expected.get("minimum"), manager.getMinimumProfile());
-    Assert.assertEquals("Profile 'default' resources don't match",
-        expected.get("default"), manager.getDefaultProfile());
-    Assert.assertEquals("Profile 'maximum' resources don't match",
-        expected.get("maximum"), manager.getMaximumProfile());
+    assertEquals(expected.get("minimum"), manager.getMinimumProfile(),
+        "Profile 'minimum' resources don't match");
+    assertEquals(expected.get("default"), manager.getDefaultProfile(),
+        "Profile 'default' resources don't match");
+    assertEquals(expected.get("maximum"), manager.getMaximumProfile(),
+        "Profile 'maximum' resources don't match");
 
   }
 
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testResourceProfilesInAMResponse() throws Exception {
     Configuration conf = new Configuration();
     MockRM rm = new MockRM(conf);
@@ -168,7 +171,7 @@ public class TestResourceProfiles {
     RMAppAttempt attempt1 = app1.getCurrentAppAttempt();
     MockAM am1 = rm.sendAMLaunched(attempt1.getAppAttemptId());
     RegisterApplicationMasterResponse resp = am1.registerAppAttempt();
-    Assert.assertEquals(0, resp.getResourceProfiles().size());
+    assertEquals(0, resp.getResourceProfiles().size());
     rm.stop();
     conf.setBoolean(YarnConfiguration.RM_RESOURCE_PROFILES_ENABLED, true);
     conf.set(YarnConfiguration.RM_RESOURCE_PROFILES_SOURCE_FILE,
@@ -181,12 +184,12 @@ public class TestResourceProfiles {
     attempt1 = app1.getCurrentAppAttempt();
     am1 = rm.sendAMLaunched(attempt1.getAppAttemptId());
     resp = am1.registerAppAttempt();
-    Assert.assertEquals(3, resp.getResourceProfiles().size());
-    Assert.assertEquals(Resource.newInstance(1024, 1),
+    assertEquals(3, resp.getResourceProfiles().size());
+    assertEquals(Resource.newInstance(1024, 1),
         resp.getResourceProfiles().get("minimum"));
-    Assert.assertEquals(Resource.newInstance(2048, 2),
+    assertEquals(Resource.newInstance(2048, 2),
         resp.getResourceProfiles().get("default"));
-    Assert.assertEquals(Resource.newInstance(8192, 4),
+    assertEquals(Resource.newInstance(8192, 4),
         resp.getResourceProfiles().get("maximum"));
     rm.stop();
   }

+ 6 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMExpiry.java

@@ -19,9 +19,9 @@
 package org.apache.hadoop.yarn.server.resourcemanager.resourcetracker;
 
 import static org.apache.hadoop.yarn.server.resourcemanager.MockNM.createMockNodeStatus;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
-import org.junit.Assert;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -48,8 +48,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEv
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
 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;
 
 public class TestNMExpiry {
   private static final Logger LOG =
@@ -70,7 +70,7 @@ public class TestNMExpiry {
     }
   }
 
-  @Before
+  @BeforeEach
   public void setUp() {
     Configuration conf = new Configuration();
     // Dispatcher that processes events inline
@@ -164,7 +164,7 @@ public class TestNMExpiry {
         wait(100);
       }
     }
-    Assert.assertEquals(2, ClusterMetrics.getMetrics().getNumLostNMs());
+    assertEquals(2, ClusterMetrics.getMetrics().getNumLostNMs());
 
     request3 = recordFactory
         .newRecordInstance(RegisterNodeManagerRequest.class);
@@ -178,7 +178,7 @@ public class TestNMExpiry {
     /* test to see if hostanme 3 does not expire */
     stopT = false;
     new ThirdNodeHeartBeatThread().start();
-    Assert.assertEquals(2,ClusterMetrics.getMetrics().getNumLostNMs());
+    assertEquals(2, ClusterMetrics.getMetrics().getNumLostNMs());
     stopT = true;
   }
 }

+ 13 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestRMNMRPCResponseId.java

@@ -18,6 +18,9 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.resourcetracker;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
@@ -46,10 +49,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEv
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
 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;
 
 @SuppressWarnings("rawtypes")
 public class TestRMNMRPCResponseId {
@@ -57,7 +59,7 @@ public class TestRMNMRPCResponseId {
   ResourceTrackerService resourceTrackerService;
   private NodeId nodeId;
 
-  @Before
+  @BeforeEach
   public void setUp() {
     Configuration conf = new Configuration();
     // Dispatcher that processes events inline
@@ -86,7 +88,7 @@ public class TestRMNMRPCResponseId {
     resourceTrackerService.init(conf);
   }
   
-  @After
+  @AfterEach
   public void tearDown() {
     /* do nothing */
   }
@@ -121,20 +123,20 @@ public class TestRMNMRPCResponseId {
     nodeStatus.setResponseId(0);
     NodeHeartbeatResponse response = resourceTrackerService.nodeHeartbeat(
         nodeHeartBeatRequest);
-    Assert.assertTrue(response.getResponseId() == 1);
+    assertTrue(response.getResponseId() == 1);
 
     nodeStatus.setResponseId(response.getResponseId());
     response = resourceTrackerService.nodeHeartbeat(nodeHeartBeatRequest);
-    Assert.assertTrue(response.getResponseId() == 2);   
+    assertTrue(response.getResponseId() == 2);
 
     /* try calling with less response id */
     response = resourceTrackerService.nodeHeartbeat(nodeHeartBeatRequest);
-    Assert.assertTrue(response.getResponseId() == 2);
+    assertTrue(response.getResponseId() == 2);
 
     nodeStatus.setResponseId(0);
     response = resourceTrackerService.nodeHeartbeat(nodeHeartBeatRequest);
-    Assert.assertTrue(NodeAction.RESYNC.equals(response.getNodeAction()));
-    Assert.assertEquals("Too far behind rm response id:2 nm response id:0",
+    assertTrue(NodeAction.RESYNC.equals(response.getNodeAction()));
+    assertEquals("Too far behind rm response id:2 nm response id:0",
       response.getDiagnosticsMessage());
   }
 }

+ 90 - 80
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestApplicationLifetimeMonitor.java

@@ -18,8 +18,9 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.rmapp;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -63,17 +64,14 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
 import org.apache.hadoop.yarn.util.Times;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 import org.slf4j.event.Level;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.Timeout;
 
 /**
  * Test class for application life time monitor feature test.
  */
-@RunWith(Parameterized.class)
 public class TestApplicationLifetimeMonitor {
   private final long maxLifetime = 30L;
   private static final QueuePath ROOT = new QueuePath(CapacitySchedulerConfiguration.ROOT);
@@ -83,7 +81,6 @@ public class TestApplicationLifetimeMonitor {
 
   private YarnConfiguration conf;
 
-  @Parameterized.Parameters
   public static Collection<Object[]> data() {
     Collection<Object[]> params = new ArrayList<Object[]>();
     params.add(new Object[]{CapacityScheduler.class});
@@ -93,11 +90,12 @@ public class TestApplicationLifetimeMonitor {
 
   private Class scheduler;
 
-  public TestApplicationLifetimeMonitor(Class schedulerParameter) {
+  private void initTestApplicationLifetimeMonitor(Class schedulerParameter)
+      throws IOException {
     scheduler = schedulerParameter;
+    setup();
   }
 
-  @Before
   public void setup() throws IOException {
     if (scheduler.equals(CapacityScheduler.class)) {
       // Since there is limited lifetime monitoring support in fair scheduler
@@ -118,9 +116,12 @@ public class TestApplicationLifetimeMonitor {
         3000L);
   }
 
-  @Test(timeout = 60000)
-  public void testApplicationLifetimeMonitor()
+  @Timeout(value = 60)
+  @ParameterizedTest
+  @MethodSource("data")
+  public void testApplicationLifetimeMonitor(Class schedulerParameter)
       throws Exception {
+    initTestApplicationLifetimeMonitor(schedulerParameter);
     MockRM rm = null;
     try {
       rm = new MockRM(conf);
@@ -167,8 +168,8 @@ public class TestApplicationLifetimeMonitor {
           rm.sendAMLaunched(app1.getCurrentAppAttempt().getAppAttemptId());
       am1.registerAppAttempt();
       rm.waitForState(app1.getApplicationId(), RMAppState.KILLED);
-      Assert.assertTrue("Application killed before lifetime value",
-          (System.currentTimeMillis() - app1.getSubmitTime()) > 10000);
+      assertTrue((System.currentTimeMillis() - app1.getSubmitTime()) > 10000,
+          "Application killed before lifetime value");
 
       Map<ApplicationTimeoutType, String> updateTimeout =
           new HashMap<ApplicationTimeoutType, String>();
@@ -196,8 +197,8 @@ public class TestApplicationLifetimeMonitor {
       long afterUpdate =
           applicationTimeouts.get(ApplicationTimeoutType.LIFETIME);
 
-      Assert.assertTrue("Application lifetime value not updated",
-          afterUpdate > beforeUpdate);
+      assertTrue(afterUpdate > beforeUpdate,
+          "Application lifetime value not updated");
 
       // verify for application report.
       RecordFactory recordFactory =
@@ -208,17 +209,16 @@ public class TestApplicationLifetimeMonitor {
       Map<ApplicationTimeoutType, ApplicationTimeout> appTimeouts = rm
           .getRMContext().getClientRMService().getApplicationReport(appRequest)
           .getApplicationReport().getApplicationTimeouts();
-      Assert.assertTrue("Application Timeout are empty.",
-          !appTimeouts.isEmpty());
+      assertTrue(!appTimeouts.isEmpty(), "Application Timeout are empty.");
       ApplicationTimeout timeout =
           appTimeouts.get(ApplicationTimeoutType.LIFETIME);
-      Assert.assertTrue("Application remaining time is incorrect",
-          timeout.getRemainingTime() > 0);
+      assertTrue(timeout.getRemainingTime() > 0,
+          "Application remaining time is incorrect");
 
       rm.waitForState(app2.getApplicationId(), RMAppState.KILLED);
       // verify for app killed with updated lifetime
-      Assert.assertTrue("Application killed before lifetime value",
-          app2.getFinishTime() > afterUpdate);
+      assertTrue(app2.getFinishTime() > afterUpdate,
+          "Application killed before lifetime value");
 
       if (scheduler.equals(CapacityScheduler.class)) {
         // Supported only on capacity scheduler
@@ -228,19 +228,21 @@ public class TestApplicationLifetimeMonitor {
         // so killed after queue max lifetime.
         rm.waitForState(app4.getApplicationId(), RMAppState.KILLED);
         long totalTimeRun = app4.getFinishTime() - app4.getSubmitTime();
-        Assert.assertTrue("Application killed before lifetime value",
-            totalTimeRun > (maxLifetime * 1000));
-        Assert.assertTrue(
-            "Application killed before lifetime value " + totalTimeRun,
-            totalTimeRun < ((maxLifetime + 10L) * 1000));
+        assertTrue(totalTimeRun > (maxLifetime * 1000),
+            "Application killed before lifetime value");
+        assertTrue(totalTimeRun < ((maxLifetime + 10L) * 1000),
+            "Application killed before lifetime value " + totalTimeRun);
       }
     } finally {
       stopRM(rm);
     }
   }
 
-  @Test(timeout = 180000)
-  public void testApplicationLifetimeOnRMRestart() throws Exception {
+  @Timeout(value = 180)
+  @ParameterizedTest
+  @MethodSource("data")
+  public void testApplicationLifetimeOnRMRestart(Class schedulerParameter) throws Exception {
+    initTestApplicationLifetimeMonitor(schedulerParameter);
     conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
     conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED,
         true);
@@ -310,14 +312,16 @@ public class TestApplicationLifetimeMonitor {
 
     // wait for app life time and application to be in killed state.
     rm2.waitForState(recoveredApp1.getApplicationId(), RMAppState.KILLED);
-    Assert.assertTrue("Application killed before lifetime value",
-        recoveredApp1.getFinishTime() > (recoveredApp1.getSubmitTime()
-            + appLifetime * 1000));
+    assertTrue(recoveredApp1.getFinishTime() > (recoveredApp1.getSubmitTime()
+        + appLifetime * 1000), "Application killed before lifetime value");
   }
 
-  @Test(timeout = 60000)
-  public void testUpdateApplicationTimeoutForStateStoreUpdateFail()
+  @Timeout(value = 60)
+  @ParameterizedTest
+  @MethodSource("data")
+  public void testUpdateApplicationTimeoutForStateStoreUpdateFail(Class schedulerParameter)
       throws Exception {
+    initTestApplicationLifetimeMonitor(schedulerParameter);
     MockRM rm1 = null;
     try {
       MemoryRMStateStore memStore = new MemoryRMStateStore() {
@@ -375,8 +379,8 @@ public class TestApplicationLifetimeMonitor {
         fail("Update application should fail.");
       } catch (YarnException e) {
         // expected
-        assertTrue("State-store exception does not containe appId",
-            e.getMessage().contains(app1.getApplicationId().toString()));
+        assertTrue(e.getMessage().contains(app1.getApplicationId().toString()),
+            "State-store exception does not containe appId");
       }
 
       applicationTimeouts = app1.getApplicationTimeouts();
@@ -384,19 +388,22 @@ public class TestApplicationLifetimeMonitor {
       long afterUpdate =
           applicationTimeouts.get(ApplicationTimeoutType.LIFETIME);
 
-      Assert.assertEquals("Application timeout is updated", beforeUpdate,
-          afterUpdate);
+      assertEquals(beforeUpdate, afterUpdate,
+          "Application timeout is updated");
       rm1.waitForState(app1.getApplicationId(), RMAppState.KILLED);
       // verify for app killed with updated lifetime
-      Assert.assertTrue("Application killed before lifetime value",
-          app1.getFinishTime() > afterUpdate);
+      assertTrue(app1.getFinishTime() > afterUpdate,
+          "Application killed before lifetime value");
     } finally {
       stopRM(rm1);
     }
   }
 
-  @Test(timeout = 120000)
-  public void testInheritAppLifetimeFromParentQueue() throws Exception {
+  @Timeout(value = 120)
+  @ParameterizedTest
+  @MethodSource("data")
+  public void testInheritAppLifetimeFromParentQueue(Class schedulerParameter) throws Exception {
+    initTestApplicationLifetimeMonitor(schedulerParameter);
     YarnConfiguration yarnConf = conf;
     long maxRootLifetime = 20L;
     long defaultRootLifetime = 10L;
@@ -436,29 +443,31 @@ public class TestApplicationLifetimeMonitor {
         rm.waitForState(app1.getApplicationId(), RMAppState.KILLED);
         long totalTimeRun = app1.getFinishTime() - app1.getSubmitTime();
         // Child queue should have inherited parent max and default lifetimes.
-        Assert.assertEquals("Child queue max lifetime should have overridden"
-            + " parent value",
-            maxRootLifetime,
-            csched.getQueue(CQ1).getMaximumApplicationLifetime());
-        Assert.assertEquals("Child queue default lifetime should have"
-            + "  overridden parent value",
-            defaultRootLifetime,
-            csched.getQueue(CQ1).getDefaultApplicationLifetime());
+        assertEquals(maxRootLifetime,
+            csched.getQueue(CQ1).getMaximumApplicationLifetime(),
+            "Child queue max lifetime should have overridden"
+            + " parent value");
+        assertEquals(defaultRootLifetime,
+            csched.getQueue(CQ1).getDefaultApplicationLifetime(),
+            "Child queue default lifetime should have"
+            + "  overridden parent value");
         // app1 (run in the 'child1' queue) should have run longer than the
         // default lifetime but less than the max lifetime.
-        Assert.assertTrue("Application killed before default lifetime value",
-            totalTimeRun > (defaultRootLifetime * 1000));
-        Assert.assertTrue(
-            "Application killed after max lifetime value " + totalTimeRun,
-            totalTimeRun < (maxRootLifetime * 1000));
+        assertTrue(totalTimeRun > (defaultRootLifetime * 1000),
+            "Application killed before default lifetime value");
+        assertTrue(totalTimeRun < (maxRootLifetime * 1000),
+            "Application killed after max lifetime value " + totalTimeRun);
       }
     } finally {
       stopRM(rm);
     }
   }
 
-  @Test(timeout = 120000)
-  public void testOverrideParentQueueMaxAppLifetime() throws Exception {
+  @Timeout(value = 120)
+  @ParameterizedTest
+  @MethodSource("data")
+  public void testOverrideParentQueueMaxAppLifetime(Class schedulerParameter) throws Exception {
+    initTestApplicationLifetimeMonitor(schedulerParameter);
     YarnConfiguration yarnConf = conf;
     long maxRootLifetime = 20L;
     long maxChildLifetime = 40L;
@@ -501,23 +510,26 @@ public class TestApplicationLifetimeMonitor {
         rm.waitForState(app1.getApplicationId(), RMAppState.KILLED);
         long totalTimeRun = app1.getFinishTime() - app1.getSubmitTime();
         // Child queue's max lifetime can override parent's and be larger.
-        Assert.assertTrue("Application killed before default lifetime value",
-            (maxRootLifetime < maxChildLifetime)
-              && (totalTimeRun > (maxChildLifetime * 1000)));
-        Assert.assertEquals("Root queue max lifetime property set incorrectly",
-            maxRootLifetime,
-            csched.getRootQueue().getMaximumApplicationLifetime());
-        Assert.assertEquals("Child queue max lifetime should have overridden"
-            + " parent value", maxChildLifetime,
-            csched.getQueue(CQ1).getMaximumApplicationLifetime());
+        assertTrue((maxRootLifetime < maxChildLifetime) &&
+            (totalTimeRun > (maxChildLifetime * 1000)),
+            "Application killed before default lifetime value");
+        assertEquals(maxRootLifetime, csched.getRootQueue().getMaximumApplicationLifetime(),
+            "Root queue max lifetime property set incorrectly");
+        assertEquals(maxChildLifetime, csched.getQueue(CQ1).getMaximumApplicationLifetime(),
+            "Child queue max lifetime should have overridden"
+            + " parent value");
       }
     } finally {
       stopRM(rm);
     }
   }
 
-  @Test(timeout = 120000)
-  public void testOverrideParentQueueDefaultAppLifetime() throws Exception {
+  @Timeout(value = 120)
+  @ParameterizedTest
+  @MethodSource("data")
+  public void testOverrideParentQueueDefaultAppLifetime(
+      Class schedulerParameter) throws Exception {
+    initTestApplicationLifetimeMonitor(schedulerParameter);
     YarnConfiguration yarnConf = conf;
     long maxRootLifetime = -1L;
     long maxChildLifetime = -1L;
@@ -560,19 +572,17 @@ public class TestApplicationLifetimeMonitor {
         long totalTimeRun = app1.getFinishTime() - app1.getSubmitTime();
         // app1 (run in 'child1' queue) should have overridden the parent's
         // default lifetime.
-        Assert.assertTrue("Application killed before default lifetime value",
-            totalTimeRun > (defaultChildLifetime * 1000));
+        assertTrue(totalTimeRun > (defaultChildLifetime * 1000),
+            "Application killed before default lifetime value");
         // Root and child queue's max lifetime should be -1.
-        Assert.assertEquals("Root queue max lifetime property set incorrectly",
-            maxRootLifetime,
-            csched.getRootQueue().getMaximumApplicationLifetime());
-        Assert.assertEquals("Child queue max lifetime property set incorrectly",
-            maxChildLifetime,
-            csched.getQueue(CQ1).getMaximumApplicationLifetime());
+        assertEquals(maxRootLifetime, csched.getRootQueue().getMaximumApplicationLifetime(),
+            "Root queue max lifetime property set incorrectly");
+        assertEquals(maxChildLifetime, csched.getQueue(CQ1).getMaximumApplicationLifetime(),
+            "Child queue max lifetime property set incorrectly");
         // 'child1' queue's default lifetime should have overridden parent's.
-        Assert.assertEquals("Child queue default lifetime should have"
-            + " overridden parent value", defaultChildLifetime,
-            csched.getQueue(CQ1).getDefaultApplicationLifetime());
+        assertEquals(defaultChildLifetime, csched.getQueue(CQ1).getDefaultApplicationLifetime(),
+            "Child queue default lifetime should have"
+            + " overridden parent value");
       }
     } finally {
       stopRM(rm);

+ 25 - 22
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestNodesListManager.java

@@ -18,6 +18,10 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.rmapp;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.ArgumentMatchers.argThat;
 import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.spy;
@@ -44,15 +48,16 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptS
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.util.ControlledClock;
 import org.slf4j.event.Level;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.mockito.ArgumentMatcher;
 
 public class TestNodesListManager {
   private boolean isRMAppEvent;
   private boolean isNodesListEvent;
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testNodeUsableEvent() throws Exception {
     GenericTestUtils.setRootLogLevel(Level.DEBUG);
     final Dispatcher dispatcher = getDispatcher();
@@ -87,10 +92,8 @@ public class TestNodesListManager {
     // Should not have RMAppNodeUpdateEvent to AsyncDispatcher.
     dispatcher.getEventHandler().handle(new NodesListManagerEvent(
         NodesListManagerEventType.NODE_USABLE, rmnode));
-    Assert.assertFalse("Got unexpected RM app event",
-        getIsRMAppEvent());
-    Assert.assertTrue("Received no NodesListManagerEvent",
-        getIsNodesListEvent());
+    assertFalse(getIsRMAppEvent(), "Got unexpected RM app event");
+    assertTrue(getIsNodesListEvent(), "Received no NodesListManagerEvent");
   }
 
   @Test
@@ -104,28 +107,28 @@ public class TestNodesListManager {
     resolver.init(new YarnConfiguration());
     resolver.start();
     resolver.addToCache("testCachedResolverHost1", "1.1.1.1");
-    Assert.assertEquals("1.1.1.1",
+    assertEquals("1.1.1.1",
         resolver.resolve("testCachedResolverHost1"));
 
     resolver.addToCache("testCachedResolverHost2", "1.1.1.2");
-    Assert.assertEquals("1.1.1.1",
+    assertEquals("1.1.1.1",
         resolver.resolve("testCachedResolverHost1"));
-    Assert.assertEquals("1.1.1.2",
+    assertEquals("1.1.1.2",
         resolver.resolve("testCachedResolverHost2"));
 
     // test removeFromCache
     resolver.removeFromCache("testCachedResolverHost1");
-    Assert.assertNotEquals("1.1.1.1",
+    assertNotEquals("1.1.1.1",
         resolver.resolve("testCachedResolverHost1"));
-    Assert.assertEquals("1.1.1.2",
+    assertEquals("1.1.1.2",
         resolver.resolve("testCachedResolverHost2"));
 
     // test expiry
     clock.tickMsec(CACHE_EXPIRY_INTERVAL_SECS * 1000 + 1);
     resolver.getExpireChecker().run();
-    Assert.assertNotEquals("1.1.1.1",
+    assertNotEquals("1.1.1.1",
         resolver.resolve("testCachedResolverHost1"));
-    Assert.assertNotEquals("1.1.1.2",
+    assertNotEquals("1.1.1.2",
         resolver.resolve("testCachedResolverHost2"));
   }
 
@@ -140,8 +143,8 @@ public class TestNodesListManager {
     NodesListManager nodesListManager = rm.getNodesListManager();
 
     NodesListManager.Resolver resolver = nodesListManager.getResolver();
-    Assert.assertTrue("default resolver should be DirectResolver",
-        resolver instanceof NodesListManager.DirectResolver);
+    assertTrue(resolver instanceof NodesListManager.DirectResolver,
+        "default resolver should be DirectResolver");
   }
 
   @Test
@@ -162,9 +165,9 @@ public class TestNodesListManager {
 
     resolver.addToCache("testCachedResolverHost1", "1.1.1.1");
     resolver.addToCache("testCachedResolverHost2", "1.1.1.2");
-    Assert.assertEquals("1.1.1.1",
+    assertEquals("1.1.1.1",
         resolver.resolve("testCachedResolverHost1"));
-    Assert.assertEquals("1.1.1.2",
+    assertEquals("1.1.1.2",
         resolver.resolve("testCachedResolverHost2"));
 
     RMNode rmnode1 = MockNodes.newNodeInfo(1, Resource.newInstance(28000, 8),
@@ -175,17 +178,17 @@ public class TestNodesListManager {
     nodesListManager.handle(
         new NodesListManagerEvent(NodesListManagerEventType.NODE_USABLE,
             rmnode1));
-    Assert.assertNotEquals("1.1.1.1",
+    assertNotEquals("1.1.1.1",
         resolver.resolve("testCachedResolverHost1"));
-    Assert.assertEquals("1.1.1.2",
+    assertEquals("1.1.1.2",
         resolver.resolve("testCachedResolverHost2"));
 
     nodesListManager.handle(
         new NodesListManagerEvent(NodesListManagerEventType.NODE_USABLE,
             rmnode2));
-    Assert.assertNotEquals("1.1.1.1",
+    assertNotEquals("1.1.1.1",
         resolver.resolve("testCachedResolverHost1"));
-    Assert.assertNotEquals("1.1.1.2",
+    assertNotEquals("1.1.1.2",
         resolver.resolve("testCachedResolverHost2"));
 
   }

+ 200 - 139
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java

@@ -88,11 +88,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineC
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.Records;
 
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.Timeout;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 import org.mockito.ArgumentCaptor;
 
 import org.slf4j.Logger;
@@ -110,9 +108,12 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import static org.assertj.core.api.Assertions.assertThat;
-import static junit.framework.TestCase.assertTrue;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyLong;
 import static org.mockito.ArgumentMatchers.eq;
@@ -124,7 +125,6 @@ import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 
 
-@RunWith(value = Parameterized.class)
 public class TestRMAppTransitions {
   static final Logger LOG =
       LoggerFactory.getLogger(TestRMAppTransitions.class);
@@ -215,7 +215,6 @@ public class TestRMAppTransitions {
     }
   }  
 
-  @Parameterized.Parameters
   public static Collection<Object[]> getTestParameters() {
     return Arrays.asList(new Object[][] {
         { Boolean.FALSE },
@@ -223,11 +222,12 @@ public class TestRMAppTransitions {
     });
   }
 
-  public TestRMAppTransitions(boolean isSecurityEnabled) {
-    this.isSecurityEnabled = isSecurityEnabled;
+  private void initTestRMAppTransitions(boolean pIsSecurityEnabled)
+      throws Exception {
+    this.isSecurityEnabled = pIsSecurityEnabled;
+    setUp();
   }
-  
-  @Before
+
   public void setUp() throws Exception {
     conf = new YarnConfiguration();
     AuthenticationMethod authMethod = AuthenticationMethod.SIMPLE;
@@ -429,52 +429,49 @@ public class TestRMAppTransitions {
   // Test expected newly created app state
   private static void testAppStartState(ApplicationId applicationId, 
       String user, String name, String queue, RMApp application) {
-    Assert.assertTrue("application start time is not greater than 0",
-        application.getStartTime() > 0);
-    Assert.assertTrue("application start time is before currentTime", 
-        application.getStartTime() <= System.currentTimeMillis());
-    Assert.assertEquals("application user is not correct",
-        user, application.getUser());
-    Assert.assertEquals("application id is not correct",
-        applicationId, application.getApplicationId());
-    Assert.assertEquals("application progress is not correct",
-        (float)0.0, application.getProgress(), (float)0.0);
-    Assert.assertEquals("application queue is not correct",
-        queue, application.getQueue());
-    Assert.assertEquals("application name is not correct",
-        name, application.getName());
-    Assert.assertEquals("application finish time is not 0 and should be",
-        0, application.getFinishTime());
-    Assert.assertNull("application tracking url is not correct",
-        application.getTrackingUrl());
+    assertTrue(application.getStartTime() > 0,
+        "application start time is not greater than 0");
+    assertTrue(application.getStartTime() <= System.currentTimeMillis(),
+        "application start time is before currentTime");
+    assertEquals(user, application.getUser(), "application user is not correct");
+    assertEquals(applicationId, application.getApplicationId(),
+        "application id is not correct");
+    assertEquals((float) 0.0, application.getProgress(),
+        (float) 0.0, "application progress is not correct");
+    assertEquals(queue, application.getQueue(),
+        "application queue is not correct");
+    assertEquals(name, application.getName(), "application name is not correct");
+    assertEquals(0, application.getFinishTime(),
+        "application finish time is not 0 and should be");
+    assertNull(application.getTrackingUrl(),
+        "application tracking url is not correct");
     StringBuilder diag = application.getDiagnostics();
-    Assert.assertEquals("application diagnostics is not correct",
-        0, diag.length());
+    assertEquals(0, diag.length(), "application diagnostics is not correct");
   }
 
   // test to make sure times are set when app finishes
   private void assertStartTimeSet(RMApp application) {
-    Assert.assertTrue("application start time is before test case start time",
-        application.getStartTime() >= testCaseStartTime);
-    Assert.assertTrue("application start time is before currentTime",
-        application.getStartTime() <= System.currentTimeMillis());
+    assertTrue(application.getStartTime() >= testCaseStartTime,
+        "application start time is before test case start time");
+    assertTrue(application.getStartTime() <= System.currentTimeMillis(),
+        "application start time is before currentTime");
   }
 
   private static void assertAppState(RMAppState state, RMApp application) {
-    Assert.assertEquals("application state should have been " + state, 
-        state, application.getState());
+    assertEquals(state, application.getState(),
+        "application state should have been " + state);
   }
 
   private static void assertFinalAppStatus(FinalApplicationStatus status, RMApp application) {
-    Assert.assertEquals("Final application status should have been " + status, 
-        status, application.getFinalApplicationStatus());
+    assertEquals(status, application.getFinalApplicationStatus(),
+        "Final application status should have been " + status);
   }
   
   // test to make sure times are set when app finishes
   private void assertTimesAtFinish(RMApp application) {
     assertStartTimeSet(application);
-    Assert.assertTrue("application finish time is not >= start time",
-        (application.getFinishTime() >= application.getStartTime()));
+    assertTrue((application.getFinishTime() >= application.getStartTime()),
+        "application finish time is not >= start time");
   }
 
   private void assertAppFinalStateSaved(RMApp application){
@@ -503,8 +500,8 @@ public class TestRMAppTransitions {
     assertAppState(RMAppState.KILLED, application);
     assertFinalAppStatus(FinalApplicationStatus.KILLED, application);
     StringBuilder diag = application.getDiagnostics();
-    Assert.assertEquals("application diagnostics is not correct",
-        "Application killed by user.", diag.toString());
+    assertEquals("Application killed by user.", diag.toString(),
+        "application diagnostics is not correct");
   }
 
   private void assertFailed(RMApp application, String regex) {
@@ -512,8 +509,8 @@ public class TestRMAppTransitions {
     assertAppState(RMAppState.FAILED, application);
     assertFinalAppStatus(FinalApplicationStatus.FAILED, application);
     StringBuilder diag = application.getDiagnostics();
-    Assert.assertTrue("application diagnostics is not correct",
-        diag.toString().matches(regex));
+    assertTrue(diag.toString().matches(regex),
+        "application diagnostics is not correct");
   }
 
   private void sendAppUpdateSavedEvent(RMApp application) {
@@ -629,7 +626,7 @@ public class TestRMAppTransitions {
     // unmanaged AMs don't use the FINISHING state
     assert submissionContext == null || !submissionContext.getUnmanagedAM();
     RMApp application = testCreateAppFinalSaving(submissionContext);
-    Assert.assertNotNull("app shouldn't be null", application);
+    assertNotNull(application, "app shouldn't be null");
     // FINAL_SAVING => FINISHING event RMAppEventType.APP_UPDATED
     RMAppEvent appUpdated =
         new RMAppEvent(application.getApplicationId(), RMAppEventType.APP_UPDATE_SAVED);
@@ -671,13 +668,15 @@ public class TestRMAppTransitions {
     assertTimesAtFinish(application);
     // finished without a proper unregister implies failed
     assertFinalAppStatus(FinalApplicationStatus.FAILED, application);
-    Assert.assertTrue("Finished app missing diagnostics",
-        application.getDiagnostics().indexOf(diagnostics) != -1);
+    assertTrue(application.getDiagnostics().indexOf(diagnostics) != -1,
+        "Finished app missing diagnostics");
     return application;
   }
 
-  @Test
-  public void testUnmanagedApp() throws Exception {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testUnmanagedApp(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppTransitions(pIsSecurityEnabled);
     ApplicationSubmissionContext subContext = new ApplicationSubmissionContextPBImpl();
     subContext.setUnmanagedAM(true);
 
@@ -685,8 +684,8 @@ public class TestRMAppTransitions {
     LOG.info("--- START: testUnmanagedAppSuccessPath ---");
     final String diagMsg = "some diagnostics";
     RMApp application = testCreateAppFinished(subContext, diagMsg);
-    Assert.assertTrue("Finished app missing diagnostics",
-        application.getDiagnostics().indexOf(diagMsg) != -1);
+    assertTrue(application.getDiagnostics().indexOf(diagMsg) != -1,
+        "Finished app missing diagnostics");
 
     // reset the counter of Mockito.verify
     reset(writer);
@@ -700,26 +699,31 @@ public class TestRMAppTransitions {
     application.handle(event);
     rmDispatcher.await();
     RMAppAttempt appAttempt = application.getCurrentAppAttempt();
-    Assert.assertEquals(1, appAttempt.getAppAttemptId().getAttemptId());
+    assertEquals(1, appAttempt.getAppAttemptId().getAttemptId());
     sendAppUpdateSavedEvent(application);
     assertFailed(application,
         ".*Unmanaged application.*Failing the application.*");
     assertAppFinalStateSaved(application);
     verifyRMAppFieldsForFinalTransitions(application);
   }
-  
-  @Test
-  public void testAppSuccessPath() throws Exception {
+
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAppSuccessPath(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppTransitions(pIsSecurityEnabled);
     LOG.info("--- START: testAppSuccessPath ---");
     final String diagMsg = "some diagnostics";
     RMApp application = testCreateAppFinished(null, diagMsg);
-    Assert.assertTrue("Finished application missing diagnostics",
-        application.getDiagnostics().indexOf(diagMsg) != -1);
+    assertTrue(application.getDiagnostics().indexOf(diagMsg) != -1,
+        "Finished application missing diagnostics");
     verifyRMAppFieldsForFinalTransitions(application);
   }
 
-  @Test (timeout = 30000)
-  public void testAppRecoverPath() throws IOException {
+  @Timeout(value = 30)
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAppRecoverPath(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppTransitions(pIsSecurityEnabled);
     LOG.info("--- START: testAppRecoverPath ---");
     ApplicationSubmissionContext sub =
         Records.newRecord(ApplicationSubmissionContext.class);
@@ -727,8 +731,11 @@ public class TestRMAppTransitions {
     testCreateAppSubmittedRecovery(sub);
   }
 
-  @Test (timeout = 30000)
-  public void testAppNewKill() throws IOException {
+  @Timeout(value = 30)
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAppNewKill(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppTransitions(pIsSecurityEnabled);
     LOG.info("--- START: testAppNewKill ---");
 
     UserGroupInformation fooUser = UserGroupInformation.createUserForTesting(
@@ -749,8 +756,10 @@ public class TestRMAppTransitions {
     verifyRMAppFieldsForFinalTransitions(application);
   }
 
-  @Test
-  public void testAppNewReject() throws IOException {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAppNewReject(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppTransitions(pIsSecurityEnabled);
     LOG.info("--- START: testAppNewReject ---");
 
     RMApp application = createNewTestApp(null);
@@ -767,8 +776,11 @@ public class TestRMAppTransitions {
     verifyRMAppFieldsForFinalTransitions(application);
   }
 
-  @Test (timeout = 30000)
-  public void testAppNewRejectAddToStore() throws IOException {
+  @Timeout(value = 30)
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAppNewRejectAddToStore(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppTransitions(pIsSecurityEnabled);
     LOG.info("--- START: testAppNewRejectAddToStore ---");
 
     RMApp application = createNewTestApp(null);
@@ -786,8 +798,11 @@ public class TestRMAppTransitions {
     rmContext.getStateStore().removeApplication(application);
   }
 
-  @Test (timeout = 30000)
-  public void testAppNewSavingKill() throws IOException {
+  @Timeout(value = 30)
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAppNewSavingKill(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppTransitions(pIsSecurityEnabled);
     LOG.info("--- START: testAppNewSavingKill ---");
 
     RMApp application = testCreateAppNewSaving(null);
@@ -808,8 +823,11 @@ public class TestRMAppTransitions {
     verifyRMAppFieldsForFinalTransitions(application);
   }
 
-  @Test (timeout = 30000)
-  public void testAppNewSavingReject() throws IOException {
+  @Timeout(value = 30)
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAppNewSavingReject(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppTransitions(pIsSecurityEnabled);
     LOG.info("--- START: testAppNewSavingReject ---");
 
     RMApp application = testCreateAppNewSaving(null);
@@ -826,8 +844,11 @@ public class TestRMAppTransitions {
     verifyRMAppFieldsForFinalTransitions(application);
   }
 
-  @Test (timeout = 30000)
-  public void testAppNewSavingSaveReject() throws IOException {
+  @Timeout(value = 30)
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAppNewSavingSaveReject(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppTransitions(pIsSecurityEnabled);
     LOG.info("--- START: testAppNewSavingSaveReject ---");
     RMApp application = testCreateAppNewSaving(null);
     // NEW_SAVING => FAILED event RMAppEventType.APP_SAVE_FAILED
@@ -843,8 +864,11 @@ public class TestRMAppTransitions {
     assertTimesAtFinish(application);
   }
 
-  @Test (timeout = 30000)
-  public void testAppSubmittedRejected() throws IOException {
+  @Timeout(value = 30)
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAppSubmittedRejected(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppTransitions(pIsSecurityEnabled);
     LOG.info("--- START: testAppSubmittedRejected ---");
 
     RMApp application = testCreateAppSubmittedNoRecovery(null);
@@ -861,8 +885,11 @@ public class TestRMAppTransitions {
     verifyRMAppFieldsForFinalTransitions(application);
   }
 
-  @Test
-  public void testAppSubmittedKill() throws IOException, InterruptedException {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAppSubmittedKill(boolean pIsSecurityEnabled)
+      throws Exception {
+    initTestRMAppTransitions(pIsSecurityEnabled);
     LOG.info("--- START: testAppSubmittedKill---");
     RMApp application = testCreateAppSubmittedNoRecovery(null);
 
@@ -884,13 +911,15 @@ public class TestRMAppTransitions {
     verifyRMAppFieldsForFinalTransitions(application);
   }
 
-  @Test
-  public void testAppAcceptedFailed() throws IOException {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAppAcceptedFailed(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppTransitions(pIsSecurityEnabled);
     LOG.info("--- START: testAppAcceptedFailed ---");
 
     RMApp application = testCreateAppAccepted(null);
     // ACCEPTED => ACCEPTED event RMAppEventType.RMAppEventType.ATTEMPT_FAILED
-    Assert.assertTrue(maxAppAttempts > 1);
+    assertTrue(maxAppAttempts > 1);
     for (int i=1; i < maxAppAttempts; i++) {
       RMAppEvent event = 
           new RMAppFailedAttemptEvent(application.getApplicationId(), 
@@ -919,8 +948,11 @@ public class TestRMAppTransitions {
     verifyApplicationFinished(RMAppState.FAILED);
   }
 
-  @Test
-  public void testAppAcceptedKill() throws IOException, InterruptedException {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAppAcceptedKill(boolean pIsSecurityEnabled)
+      throws Exception {
+    initTestRMAppTransitions(pIsSecurityEnabled);
     LOG.info("--- START: testAppAcceptedKill ---");
     RMApp application = testCreateAppAccepted(null);
     // ACCEPTED => KILLED event RMAppEventType.KILL
@@ -948,8 +980,10 @@ public class TestRMAppTransitions {
     verifyRMAppFieldsForFinalTransitions(application);
   }
 
-  @Test
-  public void testAppAcceptedAccepted() throws IOException {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAppAcceptedAccepted(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppTransitions(pIsSecurityEnabled);
     LOG.info("--- START: testAppAcceptedAccepted ---");
 
     RMApp application = testCreateAppAccepted(null);
@@ -963,8 +997,10 @@ public class TestRMAppTransitions {
     assertAppStateLaunchTimeSaved(1234L);
   }
 
-  @Test
-  public void testAcquiredReleased() throws IOException {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAcquiredReleased(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppTransitions(pIsSecurityEnabled);
     RMApp application = testCreateAppSubmittedNoRecovery(null);
     NodeId nodeId = NodeId.newInstance("host", 1234);
     application.handle(
@@ -974,9 +1010,11 @@ public class TestRMAppTransitions {
     assertEquals(0, logAggregationReportsForApp.size());
   }
 
-  @Test
-  public void testAppAcceptedAttemptKilled() throws IOException,
-      InterruptedException {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAppAcceptedAttemptKilled(boolean pIsSecurityEnabled)
+      throws Exception {
+    initTestRMAppTransitions(pIsSecurityEnabled);
     LOG.info("--- START: testAppAcceptedAttemptKilled ---");
     RMApp application = testCreateAppAccepted(null);
 
@@ -997,8 +1035,10 @@ public class TestRMAppTransitions {
     verifyAppRemovedSchedulerEvent(application, RMAppState.KILLED);
   }
 
-  @Test
-  public void testAppRunningKill() throws IOException {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAppRunningKill(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppTransitions(pIsSecurityEnabled);
     LOG.info("--- START: testAppRunningKill ---");
 
     RMApp application = testCreateAppRunning(null);
@@ -1022,17 +1062,19 @@ public class TestRMAppTransitions {
     verifyRMAppFieldsForFinalTransitions(application);
   }
 
-  @Test
-  public void testAppRunningFailed() throws IOException {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAppRunningFailed(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppTransitions(pIsSecurityEnabled);
     LOG.info("--- START: testAppRunningFailed ---");
 
     RMApp application = testCreateAppRunning(null);
     RMAppAttempt appAttempt = application.getCurrentAppAttempt();
     int expectedAttemptId = 1;
-    Assert.assertEquals(expectedAttemptId, 
+    assertEquals(expectedAttemptId,
         appAttempt.getAppAttemptId().getAttemptId());
     // RUNNING => FAILED/RESTARTING event RMAppEventType.ATTEMPT_FAILED
-    Assert.assertTrue(maxAppAttempts > 1);
+    assertTrue(maxAppAttempts > 1);
     for (int i=1; i<maxAppAttempts; i++) {
       RMAppEvent event = 
           new RMAppFailedAttemptEvent(application.getApplicationId(), 
@@ -1041,7 +1083,7 @@ public class TestRMAppTransitions {
       rmDispatcher.await();
       assertAppState(RMAppState.ACCEPTED, application);
       appAttempt = application.getCurrentAppAttempt();
-      Assert.assertEquals(++expectedAttemptId, 
+      assertEquals(++expectedAttemptId,
           appAttempt.getAppAttemptId().getAttemptId());
       event = 
           new RMAppEvent(application.getApplicationId(), 
@@ -1080,8 +1122,10 @@ public class TestRMAppTransitions {
     verifyRMAppFieldsForFinalTransitions(application);
   }
 
-  @Test
-  public void testAppAtFinishingIgnoreKill() throws Exception {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAppAtFinishingIgnoreKill(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppTransitions(pIsSecurityEnabled);
     LOG.info("--- START: testAppAtFinishingIgnoreKill ---");
 
     RMApp application = testCreateAppFinishing(null);
@@ -1098,8 +1142,10 @@ public class TestRMAppTransitions {
   // App_Saved event, we stay on FINAL_SAVING on Attempt_Finished event
   // and then directly jump from FINAL_SAVING to FINISHED state on App_Saved
   // event
-  @Test
-  public void testAppFinalSavingToFinished() throws IOException {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAppFinalSavingToFinished(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppTransitions(pIsSecurityEnabled);
     LOG.info("--- START: testAppFinalSavingToFinished ---");
 
     RMApp application = testCreateAppFinalSaving(null);
@@ -1117,13 +1163,15 @@ public class TestRMAppTransitions {
     assertTimesAtFinish(application);
     // finished without a proper unregister implies failed
     assertFinalAppStatus(FinalApplicationStatus.FAILED, application);
-    Assert.assertTrue("Finished app missing diagnostics", application
-      .getDiagnostics().indexOf(diagMsg) != -1);
+    assertTrue(application.getDiagnostics().indexOf(diagMsg) != -1,
+        "Finished app missing diagnostics");
     verifyRMAppFieldsForFinalTransitions(application);
   }
 
-  @Test
-  public void testAppFinishedFinished() throws Exception {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAppFinishedFinished(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppTransitions(pIsSecurityEnabled);
     LOG.info("--- START: testAppFinishedFinished ---");
 
     RMApp application = testCreateAppFinished(null, "");
@@ -1135,17 +1183,20 @@ public class TestRMAppTransitions {
     rmDispatcher.await();
     assertTimesAtFinish(application);
     assertAppState(RMAppState.FINISHED, application);
-    Assert.assertEquals(0, application.getRanNodes().size());
+    assertEquals(0, application.getRanNodes().size());
     StringBuilder diag = application.getDiagnostics();
-    Assert.assertEquals("application diagnostics is not correct",
-        "", diag.toString());
+    assertEquals("", diag.toString(),
+        "application diagnostics is not correct");
     verifyApplicationFinished(RMAppState.FINISHED);
     verifyAppRemovedSchedulerEvent(application, RMAppState.FINISHED);
     verifyRMAppFieldsForFinalTransitions(application);
   }
 
-  @Test (timeout = 30000)
-  public void testAppFailedFailed() throws IOException {
+  @Timeout(value = 30)
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAppFailedFailed(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppTransitions(pIsSecurityEnabled);
     LOG.info("--- START: testAppFailedFailed ---");
 
     RMApp application = testCreateAppNewSaving(null);
@@ -1174,8 +1225,11 @@ public class TestRMAppTransitions {
     verifyRMAppFieldsForFinalTransitions(application);
   }
 
-  @Test (timeout = 30000)
-  public void testAppKilledKilled() throws IOException {
+  @Timeout(value = 30)
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAppKilledKilled(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppTransitions(pIsSecurityEnabled);
     LOG.info("--- START: testAppKilledKilled ---");
 
     RMApp application = testCreateAppRunning(null);
@@ -1228,9 +1282,12 @@ public class TestRMAppTransitions {
     assertAppState(RMAppState.KILLED, application);
     verifyRMAppFieldsForFinalTransitions(application);
   }
-  
-  @Test (timeout = 30000)
-  public void testAppStartAfterKilled() {
+
+  @Timeout(value = 30)
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAppStartAfterKilled(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppTransitions(pIsSecurityEnabled);
     LOG.info("--- START: testAppStartAfterKilled ---");
 
     ApplicationId applicationId = MockApps.newAppID(appId++);
@@ -1240,7 +1297,7 @@ public class TestRMAppTransitions {
           @Override
           protected void onInvalidStateTransition(RMAppEventType rmAppEventType,
                   RMAppState state) {
-            Assert.fail("RMAppImpl: can't handle " + rmAppEventType
+            fail("RMAppImpl: can't handle " + rmAppEventType
                                  + " at state " + state);
           }
       };
@@ -1264,8 +1321,11 @@ public class TestRMAppTransitions {
     assertAppState(RMAppState.KILLED, application);
   }
 
-  @Test(timeout = 30000)
-  public void testAppsRecoveringStates() throws Exception {
+  @Timeout(value = 30)
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAppsRecoveringStates(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppTransitions(pIsSecurityEnabled);
     RMState state = new RMState();
     Map<ApplicationId, ApplicationStateData> applicationState =
         state.getApplicationState();
@@ -1292,20 +1352,20 @@ public class TestRMAppTransitions {
             Collections.singletonList(BuilderUtils.newResourceRequest(
                 RMAppAttemptImpl.AM_CONTAINER_PRIORITY, ResourceRequest.ANY,
                 submissionContext.getResource(), 1)));
-    Assert.assertEquals(RMAppState.NEW, application.getState());
+    assertEquals(RMAppState.NEW, application.getState());
 
     RMAppEvent recoverEvent =
         new RMAppRecoverEvent(application.getApplicationId(), rmState);
     // Trigger RECOVER event.
     application.handle(recoverEvent);
     // Application final status looked from recoveredFinalStatus
-    Assert.assertTrue("Application is not in recoveredFinalStatus.",
-        RMAppImpl.isAppInFinalState(application));
+    assertTrue(RMAppImpl.isAppInFinalState(application),
+        "Application is not in recoveredFinalStatus.");
 
     rmDispatcher.await();
     RMAppState finalState = appState.getState();
-    Assert.assertEquals("Application is not in finalState.", finalState,
-        application.getState());
+    assertEquals(finalState, application.getState(),
+        "Application is not in finalState.");
     verifyRMAppFieldsForFinalTransitions(application);
   }
   
@@ -1319,27 +1379,28 @@ public class TestRMAppTransitions {
             null, app.getLaunchTime(), app.getFinishTime(), null);
     applicationState.put(app.getApplicationId(), appState);
   }
-  
-  @Test
-  public void testGetAppReport() throws IOException {
+
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testGetAppReport(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppTransitions(pIsSecurityEnabled);
     RMApp app = createNewTestApp(null);
     assertAppState(RMAppState.NEW, app);
     ApplicationReport report = app.createAndGetApplicationReport(null, true);
-    Assert.assertNotNull(report.getApplicationResourceUsageReport());
+    assertNotNull(report.getApplicationResourceUsageReport());
     assertThat(report.getApplicationResourceUsageReport()).
         isEqualTo(RMServerUtils.DUMMY_APPLICATION_RESOURCE_USAGE_REPORT);
     report = app.createAndGetApplicationReport("clientuser", true);
-    Assert.assertNotNull(report.getApplicationResourceUsageReport());
-    Assert.assertTrue("bad proxy url for app",
-        report.getTrackingUrl().endsWith("/proxy/" + app.getApplicationId()
-            + "/"));
+    assertNotNull(report.getApplicationResourceUsageReport());
+    assertTrue(report.getTrackingUrl().endsWith("/proxy/" + app.getApplicationId()
+        + "/"), "bad proxy url for app");
   }
 
   private void verifyAppBeforeFinishEvent(RMApp app) {
     assertEquals(0L, ((RMAppImpl) app).getLogAggregationStartTime());
     //RMAppEventType.APP_UPDATE_SAVED sets the finish time
-    assertTrue("App manager events should not be received!",
-            appManagerDispatcher.events.isEmpty());
+    assertTrue(appManagerDispatcher.events.isEmpty(),
+        "App manager events should not be received!");
   }
 
   private void verifyAppAfterFinishEvent(RMApp app) {
@@ -1363,22 +1424,22 @@ public class TestRMAppTransitions {
     ArgumentCaptor<RMAppState> finalState =
         ArgumentCaptor.forClass(RMAppState.class);
     verify(writer).applicationFinished(any(RMApp.class), finalState.capture());
-    Assert.assertEquals(state, finalState.getValue());
+    assertEquals(state, finalState.getValue());
     finalState = ArgumentCaptor.forClass(RMAppState.class);
     verify(publisher).appFinished(any(RMApp.class), finalState.capture(),
         anyLong());
-    Assert.assertEquals(state, finalState.getValue());
+    assertEquals(state, finalState.getValue());
   }
   
   private void verifyAppRemovedSchedulerEvent(RMApp app,
       RMAppState finalState) {
     SchedulerEvent lastEvent = schedulerDispatcher.lastSchedulerEvent;
-    Assert.assertEquals(SchedulerEventType.APP_REMOVED, lastEvent.getType());
+    assertEquals(SchedulerEventType.APP_REMOVED, lastEvent.getType());
     if (lastEvent instanceof AppRemovedSchedulerEvent) {
       AppRemovedSchedulerEvent appRemovedEvent =
           (AppRemovedSchedulerEvent) lastEvent;
-      Assert.assertEquals(finalState, appRemovedEvent.getFinalState());
-      Assert.assertEquals(app.getApplicationId().getId(),
+      assertEquals(finalState, appRemovedEvent.getFinalState());
+      assertEquals(app.getApplicationId().getId(),
           appRemovedEvent.getApplicationID().getId());
     }
   }

+ 8 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestAMLivelinessMonitor.java

@@ -26,14 +26,17 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
 import org.apache.hadoop.yarn.util.ControlledClock;
 
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.mockito.Mockito.mock;
 
 public class TestAMLivelinessMonitor {
 
-  @Test(timeout = 10000)
+  @Test
+  @Timeout(value = 10)
   public void testResetTimer() throws Exception {
     YarnConfiguration conf = new YarnConfiguration();
     UserGroupInformation.setConfiguration(conf);
@@ -58,7 +61,7 @@ public class TestAMLivelinessMonitor {
         dispatcher, clock) {
       @Override
       protected void expire(ApplicationAttemptId id) {
-        Assert.assertEquals(id, attemptId);
+        assertEquals(id, attemptId);
         expired[0] = true;
       }
     };
@@ -75,7 +78,7 @@ public class TestAMLivelinessMonitor {
       Thread.sleep(100);
     }
     // expired[0] would be set to true without resetTimer
-    Assert.assertFalse(expired[0]);
+    assertFalse(expired[0]);
     rm.stop();
   }
 }

+ 13 - 18
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptImplDiagnostics.java

@@ -27,22 +27,18 @@ import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.util.BoundedAppender;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
+import org.junit.jupiter.api.Test;
 
-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.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 /**
- * Testing {@link RMAppAttemptImpl#diagnostics} scenarios.
+ * Testing {@link RMAppAttemptImpl#getDiagnostics} scenarios.
  */
 public class TestRMAppAttemptImplDiagnostics {
 
-  @Rule
-  public ExpectedException expectedException = ExpectedException.none();
-
   @Test
   public void whenCreatedWithDefaultConfigurationSuccess() {
     final Configuration configuration = new Configuration();
@@ -54,12 +50,11 @@ public class TestRMAppAttemptImplDiagnostics {
 
   @Test
   public void whenCreatedWithWrongConfigurationError() {
-    final Configuration configuration = new Configuration();
-    configuration.setInt(YarnConfiguration.APP_ATTEMPT_DIAGNOSTICS_LIMIT_KC, 0);
-
-    expectedException.expect(YarnRuntimeException.class);
-
-    createRMAppAttemptImpl(configuration);
+    assertThrows(YarnRuntimeException.class, () -> {
+      final Configuration configuration = new Configuration();
+      configuration.setInt(YarnConfiguration.APP_ATTEMPT_DIAGNOSTICS_LIMIT_KC, 0);
+      createRMAppAttemptImpl(configuration);
+    });
   }
 
   @Test
@@ -72,8 +67,8 @@ public class TestRMAppAttemptImplDiagnostics {
     final String withinLimit = RandomStringUtils.random(1024);
     appAttempt.appendDiagnostics(withinLimit);
 
-    assertEquals("messages within limit should be preserved", withinLimit,
-        appAttempt.getDiagnostics());
+    assertEquals(withinLimit,
+        appAttempt.getDiagnostics(), "messages within limit should be preserved");
   }
 
   @Test
@@ -90,8 +85,8 @@ public class TestRMAppAttemptImplDiagnostics {
         BoundedAppender.TRUNCATED_MESSAGES_TEMPLATE, 1024,
         1025, beyondLimit.substring(1));
 
-    assertEquals("messages beyond limit should be truncated", truncated,
-        appAttempt.getDiagnostics());
+    assertEquals(truncated,
+        appAttempt.getDiagnostics(), "messages beyond limit should be truncated");
   }
 
   private RMAppAttemptImpl createRMAppAttemptImpl(

+ 297 - 180
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java

@@ -18,12 +18,13 @@
 package org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt;
 
 import static org.apache.hadoop.yarn.util.StringHelper.pjoin;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assume.assumeTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assumptions.assumeTrue;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyLong;
 import static org.mockito.ArgumentMatchers.argThat;
@@ -43,6 +44,8 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -113,19 +116,14 @@ import org.apache.hadoop.yarn.server.security.MasterKeyData;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Timeout;
 import org.mockito.ArgumentCaptor;
 import org.mockito.ArgumentMatcher;
 import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
-@RunWith(value = Parameterized.class)
 public class TestRMAppAttemptTransitions {
 
   private static final Logger LOG =
@@ -223,7 +221,6 @@ public class TestRMAppAttemptTransitions {
   private ApplicationSubmissionContext submissionContext = null;
   private boolean unmanagedAM;
 
-  @Parameterized.Parameters
   public static Collection<Object[]> getTestParameters() {
     return Arrays.asList(new Object[][] {
         { Boolean.FALSE },
@@ -231,12 +228,13 @@ public class TestRMAppAttemptTransitions {
     });
   }
 
-  public TestRMAppAttemptTransitions(Boolean isSecurityEnabled) {
-    this.isSecurityEnabled = isSecurityEnabled;
+  private void initTestRMAppAttemptTransitions(boolean pIsSecurityEnabled)
+      throws Exception {
+    this.isSecurityEnabled = pIsSecurityEnabled;
+    setUp();
   }
 
   @SuppressWarnings("deprecation")
-  @Before
   public void setUp() throws Exception {
     AuthenticationMethod authMethod = AuthenticationMethod.SIMPLE;
     if (isSecurityEnabled) {
@@ -331,8 +329,8 @@ public class TestRMAppAttemptTransitions {
         protected void onInvalidTranstion(
                 RMAppAttemptEventType rmAppAttemptEventType,
                 RMAppAttemptState state) {
-            Assert.assertTrue("RMAppAttemptImpl can't handle "
-                + rmAppAttemptEventType + " at state " + state, false);
+            assertTrue(false, "RMAppAttemptImpl can't handle "
+                + rmAppAttemptEventType + " at state " + state);
         }
     };
 
@@ -343,7 +341,7 @@ public class TestRMAppAttemptTransitions {
     testAppAttemptNewState();
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     ((AsyncDispatcher)this.spyRMContext.getDispatcher()).stop();
   }
@@ -351,7 +349,7 @@ public class TestRMAppAttemptTransitions {
   private String getProxyUrl(RMAppAttempt appAttempt) {
     String url = rmContext.getAppProxyUrl(conf,
         appAttempt.getAppAttemptId().getApplicationId());
-    Assert.assertNotEquals("N/A", url);
+    assertNotEquals("N/A", url);
     return url;
   }
 
@@ -588,7 +586,7 @@ public class TestRMAppAttemptTransitions {
     verifyAttemptFinalStateSaved();
     assertEquals(finishedContainerCount, applicationAttempt
         .getJustFinishedContainers().size());
-    Assert.assertEquals(0, getFinishedContainersSentToAM(applicationAttempt)
+    assertEquals(0, getFinishedContainersSentToAM(applicationAttempt)
         .size());
     assertEquals(container, applicationAttempt.getMasterContainer());
     assertEquals(finalStatus, applicationAttempt.getFinalApplicationStatus());
@@ -752,8 +750,10 @@ public class TestRMAppAttemptTransitions {
           RMAppAttemptEventType.ATTEMPT_UPDATE_SAVED));
   }
 
-  @Test
-  public void testUsageReport() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testUsageReport(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     // scheduler has info on running apps
     ApplicationAttemptId attemptId = applicationAttempt.getAppAttemptId();
     ApplicationResourceUsageReport appResUsgRpt =
@@ -773,8 +773,8 @@ public class TestRMAppAttemptTransitions {
     // expect usage stats to come from the scheduler report
     ApplicationResourceUsageReport report = 
         applicationAttempt.getApplicationResourceUsageReport();
-    Assert.assertEquals(123456L, report.getMemorySeconds());
-    Assert.assertEquals(55544L, report.getVcoreSeconds());
+    assertEquals(123456L, report.getMemorySeconds());
+    assertEquals(55544L, report.getVcoreSeconds());
 
     // finish app attempt and remove it from scheduler 
     when(appResUsgRpt.getMemorySeconds()).thenReturn(223456L);
@@ -789,12 +789,15 @@ public class TestRMAppAttemptTransitions {
     when(scheduler.getSchedulerAppInfo(eq(attemptId))).thenReturn(null);
 
     report = applicationAttempt.getApplicationResourceUsageReport();
-    Assert.assertEquals(223456, report.getMemorySeconds());
-    Assert.assertEquals(75544, report.getVcoreSeconds());
+    assertEquals(223456, report.getMemorySeconds());
+    assertEquals(75544, report.getVcoreSeconds());
   }
 
-  @Test
-  public void testUnmanagedAMUnexpectedRegistration() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testUnmanagedAMUnexpectedRegistration(boolean pIsSecurityEnabled)
+      throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     unmanagedAM = true;
     when(submissionContext.getUnmanagedAM()).thenReturn(true);
 
@@ -812,8 +815,11 @@ public class TestRMAppAttemptTransitions {
         "Unmanaged AM must register after AM attempt reaches LAUNCHED state.");
   }
 
-  @Test
-  public void testUnmanagedAMContainersCleanup() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testUnmanagedAMContainersCleanup(boolean pIsSecurityEnabled)
+      throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     unmanagedAM = true;
     when(submissionContext.getUnmanagedAM()).thenReturn(true);
     when(submissionContext.getKeepContainersAcrossApplicationAttempts())
@@ -829,8 +835,10 @@ public class TestRMAppAttemptTransitions {
     assertFalse(transferStateFromPreviousAttempt);
   }
 
-  @Test
-  public void testNewToKilled() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testNewToKilled(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     applicationAttempt.handle(
         new RMAppAttemptEvent(
             applicationAttempt.getAppAttemptId(), 
@@ -839,10 +847,12 @@ public class TestRMAppAttemptTransitions {
         applicationAttempt.createApplicationAttemptState());
     testAppAttemptKilledState(null, EMPTY_DIAGNOSTICS);
     verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
-  } 
-  
-  @Test
-  public void testNewToRecovered() {
+  }
+
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testNewToRecovered(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     applicationAttempt.handle(
         new RMAppAttemptEvent(
             applicationAttempt.getAppAttemptId(), 
@@ -850,8 +860,10 @@ public class TestRMAppAttemptTransitions {
     testAppAttemptRecoveredState();
   }
 
-  @Test
-  public void testSubmittedToKilled() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testSubmittedToKilled(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     submitApplicationAttempt();
     applicationAttempt.handle(
         new RMAppAttemptEvent(
@@ -862,8 +874,10 @@ public class TestRMAppAttemptTransitions {
     testAppAttemptKilledState(null, EMPTY_DIAGNOSTICS);
   }
 
-  @Test
-  public void testScheduledToKilled() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testScheduledToKilled(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     scheduleApplicationAttempt();
     applicationAttempt.handle(        
         new RMAppAttemptEvent(
@@ -874,8 +888,10 @@ public class TestRMAppAttemptTransitions {
     testAppAttemptKilledState(null, EMPTY_DIAGNOSTICS);
   }
 
-  @Test
-  public void testAMCrashAtScheduled() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAMCrashAtScheduled(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     // This is to test sending CONTAINER_FINISHED event at SCHEDULED state.
     // Verify the state transition is correct.
     scheduleApplicationAttempt();
@@ -901,8 +917,10 @@ public class TestRMAppAttemptTransitions {
     verifyApplicationAttemptFinished(RMAppAttemptState.FAILED);
   }
 
-  @Test
-  public void testAllocatedToKilled() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAllocatedToKilled(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     Container amContainer = allocateApplicationAttempt();
     applicationAttempt.handle(
         new RMAppAttemptEvent(
@@ -913,8 +931,10 @@ public class TestRMAppAttemptTransitions {
     testAppAttemptKilledState(amContainer, EMPTY_DIAGNOSTICS);
   }
 
-  @Test
-  public void testAllocatedToFailed() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAllocatedToFailed(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     Container amContainer = allocateApplicationAttempt();
     String diagnostics = "Launch Failed";
     applicationAttempt.handle(
@@ -925,16 +945,22 @@ public class TestRMAppAttemptTransitions {
     testAppAttemptFailedState(amContainer, diagnostics);
   }
 
-  @Test(timeout = 10000)
-  public void testAllocatedToRunning() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  @Timeout(value = 10)
+  public void testAllocatedToRunning(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     Container amContainer = allocateApplicationAttempt();
     // Register attempt event arrives before launched attempt event
     runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
     launchApplicationAttempt(amContainer, RMAppAttemptState.RUNNING);
   }
 
-  @Test(timeout = 10000)
-  public void testCreateAppAttemptReport() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  @Timeout(value = 10)
+  public void testCreateAppAttemptReport(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     RMAppAttemptState[] attemptStates = RMAppAttemptState.values();
     applicationAttempt.handle(new RMAppAttemptEvent(
         applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.KILL));
@@ -946,8 +972,11 @@ public class TestRMAppAttemptTransitions {
     }
   }
 
-  @Test(timeout = 10000)
-  public void testLaunchedAtFinalSaving() {
+  @Timeout(value = 10)
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testLaunchedAtFinalSaving(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     Container amContainer = allocateApplicationAttempt();
 
     // ALLOCATED->FINAL_SAVING
@@ -978,8 +1007,11 @@ public class TestRMAppAttemptTransitions {
         applicationAttempt.getAppAttemptState());
   }
 
-  @Test(timeout = 10000)
-  public void testAttemptAddedAtFinalSaving() {
+  @Timeout(value = 10)
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAttemptAddedAtFinalSaving(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     submitApplicationAttempt();
 
     // SUBMITTED->FINAL_SAVING
@@ -995,8 +1027,11 @@ public class TestRMAppAttemptTransitions {
                    applicationAttempt.getAppAttemptState());
   }
 
-  @Test(timeout = 10000)
-  public void testAttemptRegisteredAtFailed() {
+  @Timeout(value = 10)
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAttemptRegisteredAtFailed(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     Container amContainer = allocateApplicationAttempt();
     launchApplicationAttempt(amContainer);
 
@@ -1021,8 +1056,10 @@ public class TestRMAppAttemptTransitions {
         applicationAttempt.getAppAttemptState());
   }
 
-  @Test
-  public void testAttemptLaunchFailedAtFailed() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAttemptLaunchFailedAtFailed(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     Container amContainer = allocateApplicationAttempt();
     launchApplicationAttempt(amContainer);
     //send CONTAINER_FINISHED event
@@ -1045,8 +1082,10 @@ public class TestRMAppAttemptTransitions {
         applicationAttempt.getAppAttemptState());
   }
 
-  @Test
-  public void testAMCrashAtAllocated() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAMCrashAtAllocated(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     Container amContainer = allocateApplicationAttempt();
     String containerDiagMsg = "some error";
     int exitCode = 123;
@@ -1068,9 +1107,11 @@ public class TestRMAppAttemptTransitions {
     verifyAMCrashAtAllocatedDiagnosticInfo(applicationAttempt.getDiagnostics(),
       exitCode, shouldCheckURL);
   }
-  
-  @Test
-  public void testRunningToFailed() {
+
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testRunningToFailed(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     Container amContainer = allocateApplicationAttempt();
     launchApplicationAttempt(amContainer);
     runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
@@ -1112,8 +1153,10 @@ public class TestRMAppAttemptTransitions {
     verifyApplicationAttemptFinished(RMAppAttemptState.FAILED);
   }
 
-  @Test
-  public void testRunningToKilled() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testRunningToKilled(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     Container amContainer = allocateApplicationAttempt();
     launchApplicationAttempt(amContainer);
     runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
@@ -1152,8 +1195,11 @@ public class TestRMAppAttemptTransitions {
     verifyApplicationAttemptFinished(RMAppAttemptState.KILLED);
   }
 
-  @Test(timeout=10000)
-  public void testLaunchedExpire() {
+  @Timeout(value = 10)
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testLaunchedExpire(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     Container amContainer = allocateApplicationAttempt();
     launchApplicationAttempt(amContainer);
     applicationAttempt.handle(new RMAppAttemptEvent(
@@ -1163,8 +1209,8 @@ public class TestRMAppAttemptTransitions {
     sendAttemptUpdateSavedEvent(applicationAttempt);
     assertEquals(RMAppAttemptState.FAILED,
         applicationAttempt.getAppAttemptState());
-    assertTrue("expire diagnostics missing",
-        applicationAttempt.getDiagnostics().contains("timed out"));
+    assertTrue(applicationAttempt.getDiagnostics().contains("timed out"),
+        "expire diagnostics missing");
     String rmAppPageUrl = pjoin(RM_WEBAPP_ADDR, "cluster", "app",
         applicationAttempt.getAppAttemptId().getApplicationId());
     assertEquals(rmAppPageUrl, applicationAttempt.getOriginalTrackingUrl());
@@ -1174,8 +1220,11 @@ public class TestRMAppAttemptTransitions {
   }
 
   @SuppressWarnings("unchecked")
-  @Test(timeout=10000)
-  public void testLaunchedFailWhileAHSEnabled() {
+  @Timeout(value = 10)
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testLaunchedFailWhileAHSEnabled(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     Configuration myConf = new Configuration(conf);
     myConf.setBoolean(YarnConfiguration.APPLICATION_HISTORY_ENABLED, true);
     ApplicationId applicationId = MockApps.newAppID(appId);
@@ -1246,8 +1295,11 @@ public class TestRMAppAttemptTransitions {
     assertEquals(rmAppPageUrl, myApplicationAttempt.getTrackingUrl());
   }
 
-  @Test(timeout=20000)
-  public void testRunningExpire() {
+  @Timeout(value = 20)
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testRunningExpire(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     Container amContainer = allocateApplicationAttempt();
     launchApplicationAttempt(amContainer);
     runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
@@ -1258,8 +1310,8 @@ public class TestRMAppAttemptTransitions {
     sendAttemptUpdateSavedEvent(applicationAttempt);
     assertEquals(RMAppAttemptState.FAILED,
         applicationAttempt.getAppAttemptState());
-    assertTrue("expire diagnostics missing",
-        applicationAttempt.getDiagnostics().contains("timed out"));
+    assertTrue(applicationAttempt.getDiagnostics().contains("timed out"),
+        "expire diagnostics missing");
     String rmAppPageUrl = pjoin(RM_WEBAPP_ADDR, "cluster", "app",
         applicationAttempt.getAppAttemptId().getApplicationId());
     assertEquals(rmAppPageUrl, applicationAttempt.getOriginalTrackingUrl());
@@ -1269,8 +1321,10 @@ public class TestRMAppAttemptTransitions {
     verifyApplicationAttemptFinished(RMAppAttemptState.FAILED);
   }
 
-  @Test 
-  public void testUnregisterToKilledFinishing() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testUnregisterToKilledFinishing(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     Container amContainer = allocateApplicationAttempt();
     launchApplicationAttempt(amContainer);
     runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
@@ -1279,33 +1333,45 @@ public class TestRMAppAttemptTransitions {
         "Killed by user");
   }
 
-  @Test
-  public void testTrackingUrlUnmanagedAM() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testTrackingUrlUnmanagedAM(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     testUnmanagedAMSuccess("oldTrackingUrl");
   }
 
-  @Test
-  public void testEmptyTrackingUrlUnmanagedAM() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testEmptyTrackingUrlUnmanagedAM(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     testUnmanagedAMSuccess("");
   }
 
-  @Test
-  public void testNullTrackingUrlUnmanagedAM() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testNullTrackingUrlUnmanagedAM(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     testUnmanagedAMSuccess(null);
   }
 
-  @Test
-  public void testManagedAMWithTrackingUrl() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testManagedAMWithTrackingUrl(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     testTrackingUrlManagedAM("theTrackingUrl");
   }
 
-  @Test
-  public void testManagedAMWithEmptyTrackingUrl() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testManagedAMWithEmptyTrackingUrl(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     testTrackingUrlManagedAM("");
   }
 
-  @Test
-  public void testManagedAMWithNullTrackingUrl() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testManagedAMWithNullTrackingUrl(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     testTrackingUrlManagedAM(null);
   }
 
@@ -1317,8 +1383,10 @@ public class TestRMAppAttemptTransitions {
         FinalApplicationStatus.SUCCEEDED, url, "Successful");
   }
 
-  @Test
-  public void testUnregisterToSuccessfulFinishing() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testUnregisterToSuccessfulFinishing(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     Container amContainer = allocateApplicationAttempt();
     launchApplicationAttempt(amContainer);
     runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
@@ -1326,8 +1394,10 @@ public class TestRMAppAttemptTransitions {
         FinalApplicationStatus.SUCCEEDED, "mytrackingurl", "Successful");
   }
 
-  @Test
-  public void testFinishingKill() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testFinishingKill(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     Container amContainer = allocateApplicationAttempt();
     launchApplicationAttempt(amContainer);
     runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
@@ -1344,8 +1414,10 @@ public class TestRMAppAttemptTransitions {
         diagnostics);
   }
 
-  @Test
-  public void testFinishingExpire() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testFinishingExpire(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     Container amContainer = allocateApplicationAttempt();
     launchApplicationAttempt(amContainer);
     runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
@@ -1362,8 +1434,10 @@ public class TestRMAppAttemptTransitions {
         diagnostics, 0, false);
   }
 
-  @Test
-  public void testFinishingToFinishing() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testFinishingToFinishing(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     Container amContainer = allocateApplicationAttempt();
     launchApplicationAttempt(amContainer);
     runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
@@ -1386,8 +1460,10 @@ public class TestRMAppAttemptTransitions {
         diagnostics);
   }
 
-  @Test
-  public void testSuccessfulFinishingToFinished() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testSuccessfulFinishingToFinished(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     Container amContainer = allocateApplicationAttempt();
     launchApplicationAttempt(amContainer);
     runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
@@ -1411,9 +1487,11 @@ public class TestRMAppAttemptTransitions {
   // Attempt_Saved event, we stay on FINAL_SAVING on Container_Finished event
   // and then directly jump from FINAL_SAVING to FINISHED state on Attempt_Saved
   // event
-  @Test
-  public void
-      testFinalSavingToFinishedWithContainerFinished() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testFinalSavingToFinishedWithContainerFinished(boolean pIsSecurityEnabled)
+      throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     Container amContainer = allocateApplicationAttempt();
     launchApplicationAttempt(amContainer);
     runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
@@ -1444,8 +1522,11 @@ public class TestRMAppAttemptTransitions {
   // While attempt is at FINAL_SAVING, Expire event may come before
   // Attempt_Saved event, we stay on FINAL_SAVING on Expire event and then
   // directly jump from FINAL_SAVING to FINISHED state on Attempt_Saved event.
-  @Test
-  public void testFinalSavingToFinishedWithExpire() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testFinalSavingToFinishedWithExpire(boolean pIsSecurityEnabled)
+      throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     Container amContainer = allocateApplicationAttempt();
     launchApplicationAttempt(amContainer);
     runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
@@ -1470,8 +1551,10 @@ public class TestRMAppAttemptTransitions {
       diagnostics, 0, false);
   }
 
-  @Test
-  public void testFinishedContainer() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testFinishedContainer(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     Container amContainer = allocateApplicationAttempt();
     launchApplicationAttempt(amContainer);
     runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
@@ -1497,37 +1580,37 @@ public class TestRMAppAttemptTransitions {
         ArgumentCaptor.forClass(RMNodeFinishedContainersPulledByAMEvent.class);
 
     // Verify justFinishedContainers
-    Assert.assertEquals(1, applicationAttempt.getJustFinishedContainers()
+    assertEquals(1, applicationAttempt.getJustFinishedContainers()
         .size());
-    Assert.assertEquals(container1.getId(), applicationAttempt
+    assertEquals(container1.getId(), applicationAttempt
         .getJustFinishedContainers().get(0).getContainerId());
-    Assert.assertEquals(0, getFinishedContainersSentToAM(applicationAttempt)
+    assertEquals(0, getFinishedContainersSentToAM(applicationAttempt)
         .size());
 
     // Verify finishedContainersSentToAM gets container after pull
     List<ContainerStatus> containerStatuses = applicationAttempt
         .pullJustFinishedContainers();
-    Assert.assertEquals(1, containerStatuses.size());
+    assertEquals(1, containerStatuses.size());
     Mockito.verify(rmnodeEventHandler, never()).handle(Mockito
         .any(RMNodeEvent.class));
-    Assert.assertTrue(applicationAttempt.getJustFinishedContainers().isEmpty());
-    Assert.assertEquals(1, getFinishedContainersSentToAM(applicationAttempt)
+    assertTrue(applicationAttempt.getJustFinishedContainers().isEmpty());
+    assertEquals(1, getFinishedContainersSentToAM(applicationAttempt)
         .size());
 
     // Verify container is acked to NM via the RMNodeEvent after second pull
     containerStatuses = applicationAttempt.pullJustFinishedContainers();
-    Assert.assertEquals(0, containerStatuses.size());
+    assertEquals(0, containerStatuses.size());
     Mockito.verify(rmnodeEventHandler).handle(captor.capture());
-    Assert.assertEquals(container1.getId(), captor.getValue().getContainers()
+    assertEquals(container1.getId(), captor.getValue().getContainers()
         .get(0));
-    Assert.assertTrue(applicationAttempt.getJustFinishedContainers().isEmpty());
-    Assert.assertEquals(0, getFinishedContainersSentToAM(applicationAttempt)
+    assertTrue(applicationAttempt.getJustFinishedContainers().isEmpty());
+    assertEquals(0, getFinishedContainersSentToAM(applicationAttempt)
         .size());
 
     // verify if no containers to acknowledge to NM then event should not be
     // triggered. Number of times event invoked is 1 i.e on second pull
     containerStatuses = applicationAttempt.pullJustFinishedContainers();
-    Assert.assertEquals(0, containerStatuses.size());
+    assertEquals(0, containerStatuses.size());
     Mockito.verify(rmnodeEventHandler, times(1))
         .handle(Mockito.any(RMNodeEvent.class));
   }
@@ -1536,8 +1619,11 @@ public class TestRMAppAttemptTransitions {
    * Check a completed container that is not yet pulled by AM heartbeat,
    * is ACKed to NM for cleanup when the AM container exits.
    */
-  @Test
-  public void testFinishedContainerNotBeingPulledByAMHeartbeat() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testFinishedContainerNotBeingPulledByAMHeartbeat(
+      boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     Container amContainer = allocateApplicationAttempt();
     launchApplicationAttempt(amContainer);
     runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
@@ -1561,11 +1647,11 @@ public class TestRMAppAttemptTransitions {
     // Verify justFinishedContainers
     ArgumentCaptor<RMNodeFinishedContainersPulledByAMEvent> captor =
         ArgumentCaptor.forClass(RMNodeFinishedContainersPulledByAMEvent.class);
-    Assert.assertEquals(1, applicationAttempt.getJustFinishedContainers()
+    assertEquals(1, applicationAttempt.getJustFinishedContainers()
         .size());
-    Assert.assertEquals(container1.getId(), applicationAttempt
+    assertEquals(container1.getId(), applicationAttempt
         .getJustFinishedContainers().get(0).getContainerId());
-    Assert.assertTrue(
+    assertTrue(
         getFinishedContainersSentToAM(applicationAttempt).isEmpty());
 
     // finish AM container to emulate AM exit event
@@ -1580,15 +1666,15 @@ public class TestRMAppAttemptTransitions {
     List<RMNodeFinishedContainersPulledByAMEvent> containerPulledEvents =
         captor.getAllValues();
     // Verify AM container is acked to NM via the RMNodeEvent immediately
-    Assert.assertEquals(amContainer.getId(),
+    assertEquals(amContainer.getId(),
         containerPulledEvents.get(0).getContainers().get(0));
     // Verify the non-AM container is acked to NM via the RMNodeEvent
-    Assert.assertEquals(container1.getId(),
+    assertEquals(container1.getId(),
         containerPulledEvents.get(1).getContainers().get(0));
-    Assert.assertTrue("No container shall be added to justFinishedContainers" +
-            " as soon as AM container exits",
-        applicationAttempt.getJustFinishedContainers().isEmpty());
-    Assert.assertTrue(
+    assertTrue(applicationAttempt.getJustFinishedContainers().isEmpty(),
+        "No container shall be added to justFinishedContainers" +
+        " as soon as AM container exits");
+    assertTrue(
         getFinishedContainersSentToAM(applicationAttempt).isEmpty());
   }
 
@@ -1596,8 +1682,10 @@ public class TestRMAppAttemptTransitions {
    * Check a completed container is ACKed to NM for cleanup after the AM
    * container has exited.
    */
-  @Test
-  public void testFinishedContainerAfterAMExit() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testFinishedContainerAfterAMExit(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     Container amContainer = allocateApplicationAttempt();
     launchApplicationAttempt(amContainer);
     runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
@@ -1617,7 +1705,7 @@ public class TestRMAppAttemptTransitions {
     ArgumentCaptor<RMNodeFinishedContainersPulledByAMEvent> captor =
         ArgumentCaptor.forClass(RMNodeFinishedContainersPulledByAMEvent.class);
     Mockito.verify(rmnodeEventHandler).handle(captor.capture());
-    Assert.assertEquals(amContainer.getId(),
+    assertEquals(amContainer.getId(),
         captor.getValue().getContainers().get(0));
 
     // Complete a non-AM container
@@ -1636,13 +1724,12 @@ public class TestRMAppAttemptTransitions {
     captor = ArgumentCaptor.forClass(
         RMNodeFinishedContainersPulledByAMEvent.class);
     Mockito.verify(rmnodeEventHandler, times(2)).handle(captor.capture());
-    Assert.assertEquals(container1.getId(),
+    assertEquals(container1.getId(),
         captor.getAllValues().get(1).getContainers().get(0));
-    Assert.assertTrue("No container shall be added to justFinishedContainers" +
-            " after AM container exited",
-        applicationAttempt.getJustFinishedContainers().isEmpty());
-    Assert.assertTrue(
-        getFinishedContainersSentToAM(applicationAttempt).isEmpty());
+    assertTrue(applicationAttempt.getJustFinishedContainers().isEmpty(),
+        "No container shall be added to justFinishedContainers" +
+        " after AM container exited");
+    assertTrue(getFinishedContainersSentToAM(applicationAttempt).isEmpty());
   }
 
   private static List<ContainerStatus> getFinishedContainersSentToAM(
@@ -1658,22 +1745,24 @@ public class TestRMAppAttemptTransitions {
   // this is to test user can get client tokens only after the client token
   // master key is saved in the state store and also registered in
   // ClientTokenSecretManager
-  @Test
-  public void testGetClientToken() throws Exception {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testGetClientToken(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     assumeTrue(isSecurityEnabled);
     Container amContainer = allocateApplicationAttempt();
 
     // before attempt is launched, can not get ClientToken
     Token<ClientToAMTokenIdentifier> token =
         applicationAttempt.createClientToken(null);
-    Assert.assertNull(token);
+    assertNull(token);
 
     launchApplicationAttempt(amContainer);
     // after attempt is launched , can get ClientToken
     token = applicationAttempt.createClientToken(null);
-    Assert.assertNull(token);
+    assertNull(token);
     token = applicationAttempt.createClientToken("clientuser");
-    Assert.assertNotNull(token);
+    assertNotNull(token);
 
     applicationAttempt.handle(new RMAppAttemptEvent(applicationAttempt
       .getAppAttemptId(), RMAppAttemptEventType.KILL));
@@ -1682,24 +1771,26 @@ public class TestRMAppAttemptTransitions {
     sendAttemptUpdateSavedEvent(applicationAttempt);
     // after attempt is killed, can not get Client Token
     token = applicationAttempt.createClientToken(null);
-    Assert.assertNull(token);
+    assertNull(token);
     token = applicationAttempt.createClientToken("clientuser");
-    Assert.assertNull(token);
+    assertNull(token);
   }
 
   // this is to test master key is saved in the secret manager only after
   // attempt is launched and in secure-mode
-  @Test
-  public void testApplicationAttemptMasterKey() throws Exception {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testApplicationAttemptMasterKey(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     Container amContainer = allocateApplicationAttempt();
     ApplicationAttemptId appid = applicationAttempt.getAppAttemptId();
     boolean isMasterKeyExisted = clientToAMTokenManager.hasMasterKey(appid);
 
     if (isSecurityEnabled) {
-      Assert.assertTrue(isMasterKeyExisted);
-      Assert.assertNotNull(clientToAMTokenManager.getMasterKey(appid));
+      assertTrue(isMasterKeyExisted);
+      assertNotNull(clientToAMTokenManager.getMasterKey(appid));
     } else {
-      Assert.assertFalse(isMasterKeyExisted);
+      assertFalse(isMasterKeyExisted);
     }
     launchApplicationAttempt(amContainer);
     applicationAttempt.handle(new RMAppAttemptEvent(applicationAttempt
@@ -1709,11 +1800,13 @@ public class TestRMAppAttemptTransitions {
     sendAttemptUpdateSavedEvent(applicationAttempt);
     // after attempt is killed, can not get MasterKey
     isMasterKeyExisted = clientToAMTokenManager.hasMasterKey(appid);
-    Assert.assertFalse(isMasterKeyExisted);
+    assertFalse(isMasterKeyExisted);
   }
 
-  @Test
-  public void testFailedToFailed() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testFailedToFailed(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     // create a failed attempt.
     when(submissionContext.getKeepContainersAcrossApplicationAttempts())
       .thenReturn(true);
@@ -1757,8 +1850,10 @@ public class TestRMAppAttemptTransitions {
     assertTrue(found);
   }
 
-  @Test
-  public void testContainerRemovedBeforeAllocate() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testContainerRemovedBeforeAllocate(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     scheduleApplicationAttempt();
 
     // Mock the allocation of AM container
@@ -1787,8 +1882,10 @@ public class TestRMAppAttemptTransitions {
   }
 
   @SuppressWarnings("deprecation")
-  @Test
-  public void testContainersCleanupForLastAttempt() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testContainersCleanupForLastAttempt(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     // create a failed attempt.
     applicationAttempt =
         new RMAppAttemptImpl(applicationAttempt.getAppAttemptId(), spyRMContext,
@@ -1819,8 +1916,11 @@ public class TestRMAppAttemptTransitions {
   }
   
   @SuppressWarnings("unchecked")
-  @Test
-  public void testScheduleTransitionReplaceAMContainerRequestWithDefaults() {
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testScheduleTransitionReplaceAMContainerRequestWithDefaults(
+      boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     YarnScheduler mockScheduler = mock(YarnScheduler.class);
     when(mockScheduler.allocate(any(ApplicationAttemptId.class),
         any(List.class), any(List.class), any(List.class), any(List.class), any(List.class),
@@ -1862,8 +1962,11 @@ public class TestRMAppAttemptTransitions {
         (RMAppAttemptImpl) applicationAttempt, null);
   }
 
-  @Test(timeout = 30000)
-  public void testNewToFailed() {
+  @Timeout(value = 30)
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testNewToFailed(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     applicationAttempt.handle(new RMAppAttemptEvent(applicationAttempt
         .getAppAttemptId(), RMAppAttemptEventType.FAIL, FAILED_DIAGNOSTICS));
     assertEquals(YarnApplicationAttemptState.NEW,
@@ -1872,8 +1975,11 @@ public class TestRMAppAttemptTransitions {
     verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
   }
 
-  @Test(timeout = 30000)
-  public void testSubmittedToFailed() {
+  @Timeout(value = 30)
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testSubmittedToFailed(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     submitApplicationAttempt();
     applicationAttempt.handle(new RMAppAttemptEvent(applicationAttempt
         .getAppAttemptId(), RMAppAttemptEventType.FAIL, FAILED_DIAGNOSTICS));
@@ -1882,8 +1988,11 @@ public class TestRMAppAttemptTransitions {
     testAppAttemptFailedState(null, FAILED_DIAGNOSTICS);
   }
 
-  @Test(timeout = 30000)
-  public void testScheduledToFailed() {
+  @Timeout(value = 30)
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testScheduledToFailed(boolean pIsSecurityEnabled) throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     scheduleApplicationAttempt();
     applicationAttempt.handle(new RMAppAttemptEvent(applicationAttempt
         .getAppAttemptId(), RMAppAttemptEventType.FAIL, FAILED_DIAGNOSTICS));
@@ -1892,8 +2001,12 @@ public class TestRMAppAttemptTransitions {
     testAppAttemptFailedState(null, FAILED_DIAGNOSTICS);
   }
 
-  @Test(timeout = 30000)
-  public void testAllocatedToFailedUserTriggeredFailEvent() {
+  @Timeout(value = 30)
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testAllocatedToFailedUserTriggeredFailEvent(boolean pIsSecurityEnabled)
+      throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     Container amContainer = allocateApplicationAttempt();
     assertEquals(YarnApplicationAttemptState.ALLOCATED,
         applicationAttempt.createApplicationAttemptState());
@@ -1902,8 +2015,12 @@ public class TestRMAppAttemptTransitions {
     testAppAttemptFailedState(amContainer, FAILED_DIAGNOSTICS);
   }
 
-  @Test(timeout = 30000)
-  public void testRunningToFailedUserTriggeredFailEvent() {
+  @Timeout(value = 30)
+  @ParameterizedTest
+  @MethodSource("getTestParameters")
+  public void testRunningToFailedUserTriggeredFailEvent(boolean pIsSecurityEnabled)
+      throws Exception {
+    initTestRMAppAttemptTransitions(pIsSecurityEnabled);
     Container amContainer = allocateApplicationAttempt();
     launchApplicationAttempt(amContainer);
     runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
@@ -1936,15 +2053,15 @@ public class TestRMAppAttemptTransitions {
 
   private void verifyAMCrashAtAllocatedDiagnosticInfo(String diagnostics,
         int exitCode, boolean shouldCheckURL) {
-    assertTrue("Diagnostic information does not point the logs to the users",
-      diagnostics.contains("logs"));
-    assertTrue("Diagnostic information does not contain application attempt id",
-      diagnostics.contains(applicationAttempt.getAppAttemptId().toString()));
-    assertTrue("Diagnostic information does not contain application exit code",
-      diagnostics.contains("exitCode: " + exitCode));
+    assertTrue(diagnostics.contains("logs"),
+        "Diagnostic information does not point the logs to the users");
+    assertTrue(diagnostics.contains(applicationAttempt.getAppAttemptId().toString()),
+        "Diagnostic information does not contain application attempt id");
+    assertTrue(diagnostics.contains("exitCode: " + exitCode),
+        "Diagnostic information does not contain application exit code");
     if (shouldCheckURL) {
-      assertTrue("Diagnostic information does not contain application proxy URL",
-          diagnostics.contains(applicationAttempt.getTrackingUrl()));
+      assertTrue(diagnostics.contains(applicationAttempt.getTrackingUrl()),
+          "Diagnostic information does not contain application proxy URL");
     }
   }
 
@@ -1981,12 +2098,12 @@ public class TestRMAppAttemptTransitions {
         ArgumentCaptor.forClass(RMAppAttemptState.class);
     verify(writer).applicationAttemptFinished(
         any(RMAppAttempt.class), finalState.capture());
-    Assert.assertEquals(state, finalState.getValue());
+    assertEquals(state, finalState.getValue());
     finalState =
         ArgumentCaptor.forClass(RMAppAttemptState.class);
     verify(publisher).appAttemptFinished(any(RMAppAttempt.class), finalState.capture(),
         any(RMApp.class), anyLong());
-    Assert.assertEquals(state, finalState.getValue());
+    assertEquals(state, finalState.getValue());
   }
 
 }

+ 31 - 25
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java

@@ -18,16 +18,20 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.rmcontainer;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.anyLong;
+import static org.mockito.Mockito.anyLong;
+import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.reset;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.reset;
 
 import java.util.ArrayList;
 import java.util.concurrent.ConcurrentHashMap;
@@ -74,11 +78,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.Alloca
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.mockito.ArgumentCaptor;
 import org.mockito.ArgumentMatchers;
-import org.mockito.Mockito;
 
 @SuppressWarnings({ "unchecked", "rawtypes" })
 public class TestRMContainerImpl {
@@ -110,7 +113,7 @@ public class TestRMContainerImpl {
         spy(new ConcurrentHashMap<ApplicationId, RMApp>());
     RMApp rmApp = mock(RMApp.class);
     when(rmApp.getRMAppAttempt(any())).thenReturn(null);
-    Mockito.doReturn(rmApp)
+    doReturn(rmApp)
         .when(rmApps).get(ArgumentMatchers.<ApplicationId>any());
 
     RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class);
@@ -298,7 +301,7 @@ public class TestRMContainerImpl {
 
     // Verify whether list of ResourceRequest is present in RMContainer
     // while moving to ALLOCATED state
-    Assert.assertNotNull(
+    assertNotNull(
         scheduler.getRMContainer(containerId2).getContainerRequest());
 
     // Allocate container
@@ -308,11 +311,12 @@ public class TestRMContainerImpl {
 
     // After RMContainer moving to ACQUIRED state, list of ResourceRequest will
     // be empty
-    Assert.assertNull(
+    assertNull(
         scheduler.getRMContainer(containerId2).getContainerRequest());
   }
 
-  @Test (timeout = 180000)
+  @Test
+  @Timeout(value = 180)
   public void testStoreAllContainerMetrics() throws Exception {
     Configuration conf = new Configuration();
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
@@ -352,7 +356,8 @@ public class TestRMContainerImpl {
     verify(publisher, times(2)).containerFinished(any(RMContainer.class), anyLong());
   }
 
-  @Test (timeout = 180000)
+  @Test
+  @Timeout(value = 180)
   public void testStoreOnlyAMContainerMetrics() throws Exception {
     Configuration conf = new Configuration();
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
@@ -418,7 +423,7 @@ public class TestRMContainerImpl {
         spy(new ConcurrentHashMap<ApplicationId, RMApp>());
     RMApp rmApp = mock(RMApp.class);
     when(rmApp.getRMAppAttempt(any())).thenReturn(null);
-    Mockito.doReturn(rmApp).when(rmApps).get(
+    doReturn(rmApp).when(rmApps).get(
         ArgumentMatchers.<ApplicationId>any());
 
     RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class);
@@ -446,7 +451,7 @@ public class TestRMContainerImpl {
         SchedulerRequestKey.extractFrom(container), appAttemptId,
         nodeId, "user", rmContext);
 
-    Assert.assertEquals(0,
+    assertEquals(0,
         tagsManager.getNodeCardinalityByOp(nodeId,
             AllocationTags.createSingleAppAllocationTags(
                 TestUtils.getMockApplicationId(1), null),
@@ -456,7 +461,7 @@ public class TestRMContainerImpl {
         RMContainerEventType.START));
     schedulerNode.allocateContainer(rmContainer);
 
-    Assert.assertEquals(1,
+    assertEquals(1,
         tagsManager.getNodeCardinalityByOp(nodeId,
             AllocationTags.createSingleAppAllocationTags(appId, null),
             Long::max));
@@ -466,7 +471,7 @@ public class TestRMContainerImpl {
         RMContainerEventType.KILL));
     schedulerNode.releaseContainer(container.getId(), true);
 
-    Assert.assertEquals(0,
+    assertEquals(0,
         tagsManager.getNodeCardinalityByOp(nodeId,
             AllocationTags.createSingleAppAllocationTags(appId, null),
             Long::max));
@@ -476,7 +481,7 @@ public class TestRMContainerImpl {
         SchedulerRequestKey.extractFrom(container), appAttemptId,
         nodeId, "user", rmContext);
 
-    Assert.assertEquals(0,
+    assertEquals(0,
         tagsManager.getNodeCardinalityByOp(nodeId,
             AllocationTags.createSingleAppAllocationTags(appId, null),
             Long::max));
@@ -486,7 +491,7 @@ public class TestRMContainerImpl {
         RMContainerEventType.START));
     schedulerNode.allocateContainer(rmContainer);
 
-    Assert.assertEquals(1,
+    assertEquals(1,
         tagsManager.getNodeCardinalityByOp(nodeId,
             AllocationTags.createSingleAppAllocationTags(appId, null),
             Long::max));
@@ -499,7 +504,7 @@ public class TestRMContainerImpl {
         RMContainerEventType.FINISHED));
     schedulerNode.releaseContainer(container.getId(), true);
 
-    Assert.assertEquals(0,
+    assertEquals(0,
         tagsManager.getNodeCardinalityByOp(nodeId,
             AllocationTags.createSingleAppAllocationTags(appId, null),
             Long::max));
@@ -510,7 +515,7 @@ public class TestRMContainerImpl {
         nodeId, "user", rmContext);
     rmContainer.setAllocationTags(ImmutableSet.of("mapper"));
 
-    Assert.assertEquals(0,
+    assertEquals(0,
         tagsManager.getNodeCardinalityByOp(nodeId,
             AllocationTags.createSingleAppAllocationTags(appId, null),
             Long::max));
@@ -519,7 +524,7 @@ public class TestRMContainerImpl {
         RMContainerEventType.START));
     schedulerNode.allocateContainer(rmContainer);
 
-    Assert.assertEquals(1,
+    assertEquals(1,
         tagsManager.getNodeCardinalityByOp(nodeId,
             AllocationTags.createSingleAppAllocationTags(appId, null),
             Long::max));
@@ -535,7 +540,7 @@ public class TestRMContainerImpl {
         RMContainerEventType.FINISHED));
     schedulerNode.releaseContainer(container.getId(), true);
 
-    Assert.assertEquals(0,
+    assertEquals(0,
         tagsManager.getNodeCardinalityByOp(nodeId,
             AllocationTags.createSingleAppAllocationTags(appId, null),
             Long::max));
@@ -546,7 +551,7 @@ public class TestRMContainerImpl {
         "user", rmContext);
     rmContainer.setAllocationTags(ImmutableSet.of("mapper"));
 
-    Assert.assertEquals(0,
+    assertEquals(0,
         tagsManager.getNodeCardinalityByOp(nodeId,
             AllocationTags.createSingleAppAllocationTags(appId, null),
             Long::max));
@@ -559,13 +564,14 @@ public class TestRMContainerImpl {
     rmContainer
         .handle(new RMContainerRecoverEvent(containerId, containerStatus));
 
-    Assert.assertEquals(1,
+    assertEquals(1,
         tagsManager.getNodeCardinalityByOp(nodeId,
             AllocationTags.createSingleAppAllocationTags(appId, null),
             Long::max));
   }
 
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testContainerAcquiredAtKilled() {
     DrainDispatcher drainDispatcher = new DrainDispatcher();
     EventHandler<RMAppAttemptEvent> appAttemptEventHandler = mock(
@@ -616,7 +622,7 @@ public class TestRMContainerImpl {
         @Override
         protected void onInvalidStateTransition(
             RMContainerEventType rmContainerEventType, RMContainerState state) {
-            Assert.fail("RMContainerImpl: can't handle " + rmContainerEventType
+            fail("RMContainerImpl: can't handle " + rmContainerEventType
                 + " at state " + state);
         }
     };