瀏覽代碼

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

* YARN-11262. Upgrade JUnit from 4 to 5 in hadoop-yarn-server-resourcemanager Part6.

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 月之前
父節點
當前提交
3b8942fd76
共有 42 個文件被更改,包括 1874 次插入1780 次删除
  1. 52 38
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterServiceTestBase.java
  2. 24 21
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
  3. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
  4. 15 14
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/QueueACLsTestBase.java
  5. 33 25
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ReservationACLsTestBase.java
  6. 34 31
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java
  7. 144 139
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java
  8. 16 15
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManagerWithFairScheduler.java
  9. 31 22
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java
  10. 36 31
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
  11. 23 18
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceCapacity.java
  12. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceFair.java
  13. 14 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceInterceptor.java
  14. 14 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestCapacitySchedulerMetrics.java
  15. 258 243
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
  16. 10 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java
  17. 14 13
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClusterMetrics.java
  18. 52 47
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestContainerResourceUsage.java
  19. 16 14
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestDecommissioningNodesWatcher.java
  20. 8 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestLeaderElectorService.java
  21. 13 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestMoveApplication.java
  22. 25 21
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestNodeBlacklistingOnAMFailures.java
  23. 125 111
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestOpportunisticContainerAllocatorAMService.java
  24. 108 102
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
  25. 9 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAuditLogger.java
  26. 6 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.java
  27. 90 90
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java
  28. 10 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHAForNodeLabels.java
  29. 12 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHAMetrics.java
  30. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHATimelineCollectors.java
  31. 275 303
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
  32. 11 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMProxyUsersConf.java
  33. 56 55
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMServerUtils.java
  34. 23 21
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMStoreCommands.java
  35. 9 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMTimelineService.java
  36. 34 30
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java
  37. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManagerMXBean.java
  38. 207 217
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
  39. 5 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestSignalContainer.java
  40. 16 15
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestTokenClientRMService.java
  41. 20 20
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithCSRFFilter.java
  42. 12 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithXFSFilter.java

+ 52 - 38
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterServiceTestBase.java

@@ -52,9 +52,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoSchedule
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceUtils;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -66,7 +66,10 @@ import java.util.Map;
 import static java.lang.Thread.sleep;
 import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB;
 import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES;
-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.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 /**
  * Base class for Application Master test classes.
@@ -122,14 +125,15 @@ public abstract class ApplicationMasterServiceTestBase {
         .build()), null);
   }
 
-  @Before
+  @BeforeEach
   public void setup() {
     conf = new YarnConfiguration();
     conf.setClass(YarnConfiguration.RM_SCHEDULER, FifoScheduler.class,
         ResourceScheduler.class);
   }
 
-  @Test(timeout = 3000000)
+  @Test
+  @Timeout(value = 3000)
   public void testRMIdentifierOnContainerAllocation() throws Exception {
     MockRM rm = new MockRM(conf);
     rm.start();
@@ -163,12 +167,13 @@ public abstract class ApplicationMasterServiceTestBase {
     ContainerTokenIdentifier tokenId =
         BuilderUtils.newContainerTokenIdentifier(allocatedContainer
             .getContainerToken());
-    Assert.assertEquals(MockRM.getClusterTimeStamp(),
+    assertEquals(MockRM.getClusterTimeStamp(),
             tokenId.getRMIdentifier());
     rm.stop();
   }
 
-  @Test(timeout = 3000000)
+  @Test
+  @Timeout(value = 3000)
   public void testAllocateResponseIdOverflow() throws Exception {
     MockRM rm = new MockRM(conf);
 
@@ -188,21 +193,22 @@ public abstract class ApplicationMasterServiceTestBase {
       am1.registerAppAttempt();
 
       // Set the last responseId to be Integer.MAX_VALUE
-      Assert.assertTrue(am1.setApplicationLastResponseId(Integer.MAX_VALUE));
+      assertTrue(am1.setApplicationLastResponseId(Integer.MAX_VALUE));
 
       // Both allocate should succeed
       am1.schedule(); // send allocate with responseId = Integer.MAX_VALUE
-      Assert.assertEquals(0, am1.getResponseId());
+      assertEquals(0, am1.getResponseId());
 
       am1.schedule(); // send allocate with responseId = 0
-      Assert.assertEquals(1, am1.getResponseId());
+      assertEquals(1, am1.getResponseId());
 
     } finally {
       rm.stop();
     }
   }
 
-  @Test(timeout=600000)
+  @Test
+  @Timeout(value = 600)
   public void testInvalidContainerReleaseRequest() throws Exception {
     MockRM rm = new MockRM(conf);
 
@@ -232,7 +238,7 @@ public abstract class ApplicationMasterServiceTestBase {
         alloc1Response = am1.schedule();
       }
 
-      Assert.assertTrue(alloc1Response.getAllocatedContainers().size() > 0);
+      assertTrue(alloc1Response.getAllocatedContainers().size() > 0);
 
       RMApp app2 = MockRMAppSubmitter.submitWithMemory(1024, rm);
 
@@ -252,14 +258,15 @@ public abstract class ApplicationMasterServiceTestBase {
         sb.append(cId.toString());
         sb.append(" not belonging to this application attempt : ");
         sb.append(attempt2.getAppAttemptId().toString());
-        Assert.assertTrue(e.getMessage().contains(sb.toString()));
+        assertTrue(e.getMessage().contains(sb.toString()));
       }
     } finally {
       rm.stop();
     }
   }
 
-  @Test(timeout=1200000)
+  @Test
+  @Timeout(value = 1200)
   public void testProgressFilter() throws Exception{
     MockRM rm = new MockRM(conf);
     rm.start();
@@ -324,7 +331,8 @@ public abstract class ApplicationMasterServiceTestBase {
     }
   }
 
-  @Test(timeout=1200000)
+  @Test
+  @Timeout(value = 1200)
   public void testFinishApplicationMasterBeforeRegistering() throws Exception {
     MockRM rm = new MockRM(conf);
 
@@ -342,9 +350,9 @@ public abstract class ApplicationMasterServiceTestBase {
         am1.unregisterAppAttempt(req, false);
         fail("ApplicationMasterNotRegisteredException should be thrown");
       } catch (ApplicationMasterNotRegisteredException e) {
-        Assert.assertNotNull(e);
-        Assert.assertNotNull(e.getMessage());
-        Assert.assertTrue(e.getMessage().contains(
+        assertNotNull(e);
+        assertNotNull(e.getMessage());
+        assertTrue(e.getMessage().contains(
             "Application Master is trying to unregister before registering for:"
         ));
       } catch (Exception e) {
@@ -361,7 +369,8 @@ public abstract class ApplicationMasterServiceTestBase {
     }
   }
 
-  @Test(timeout = 1200000)
+  @Test
+  @Timeout(value = 1200)
   public void testRepeatedFinishApplicationMaster() throws Exception {
 
     CountingDispatcher dispatcher = new CountingDispatcher();
@@ -387,8 +396,8 @@ public abstract class ApplicationMasterServiceTestBase {
         am1.unregisterAppAttempt(req, false);
       }
       rm.drainEvents();
-      Assert.assertEquals("Expecting only one event", 1,
-          dispatcher.getEventCount());
+      assertEquals(1, dispatcher.getEventCount(),
+          "Expecting only one event");
     } finally {
       rm.stop();
     }
@@ -412,7 +421,8 @@ public abstract class ApplicationMasterServiceTestBase {
     }
   }
 
-  @Test(timeout = 3000000)
+  @Test
+  @Timeout(value = 3000)
   public void testResourceTypes() throws Exception {
     HashMap<YarnConfiguration,
         EnumSet<YarnServiceProtos.SchedulerResourceTypes>> driver =
@@ -464,12 +474,13 @@ public abstract class ApplicationMasterServiceTestBase {
       EnumSet<YarnServiceProtos.SchedulerResourceTypes> types =
               resp.getSchedulerResourceTypes();
       LOG.info("types = " + types.toString());
-      Assert.assertEquals(expectedValue, types);
+      assertEquals(expectedValue, types);
       rm.stop();
     }
   }
 
-  @Test(timeout=1200000)
+  @Test
+  @Timeout(value = 1200)
   public void  testAllocateAfterUnregister() throws Exception {
     MockRM rm = new MockRM(conf);
     rm.start();
@@ -495,10 +506,11 @@ public abstract class ApplicationMasterServiceTestBase {
     nm1.nodeHeartbeat(true);
     rm.drainEvents();
     alloc1Response = am1.schedule();
-    Assert.assertEquals(0, alloc1Response.getAllocatedContainers().size());
+    assertEquals(0, alloc1Response.getAllocatedContainers().size());
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testUpdateTrackingUrl() throws Exception {
     conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
         ResourceScheduler.class);
@@ -514,7 +526,7 @@ public abstract class ApplicationMasterServiceTestBase {
     RMAppAttempt attempt1 = app1.getCurrentAppAttempt();
     MockAM am1 = rm.sendAMLaunched(attempt1.getAppAttemptId());
     am1.registerAppAttempt();
-    Assert.assertEquals("N/A", rm.getRMContext().getRMApps().get(
+    assertEquals("N/A", rm.getRMContext().getRMApps().get(
         app1.getApplicationId()).getOriginalTrackingUrl());
 
     AllocateRequestPBImpl allocateRequest = new AllocateRequestPBImpl();
@@ -526,17 +538,18 @@ public abstract class ApplicationMasterServiceTestBase {
     // wait until RMAppAttemptEventType.STATUS_UPDATE is handled
     rm.drainEvents();
 
-    Assert.assertEquals(newTrackingUrl, rm.getRMContext().getRMApps().get(
+    assertEquals(newTrackingUrl, rm.getRMContext().getRMApps().get(
         app1.getApplicationId()).getOriginalTrackingUrl());
 
     // Send it again
     am1.allocate(allocateRequest);
-    Assert.assertEquals(newTrackingUrl, rm.getRMContext().getRMApps().get(
+    assertEquals(newTrackingUrl, rm.getRMContext().getRMApps().get(
         app1.getApplicationId()).getOriginalTrackingUrl());
     rm.stop();
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testValidateRequestCapacityAgainstMinMaxAllocation()
       throws Exception {
     Map<String, ResourceInformation> riMap =
@@ -582,7 +595,7 @@ public abstract class ApplicationMasterServiceTestBase {
     } catch (InvalidResourceRequestException e) {
       exception = true;
     }
-    Assert.assertTrue(exception);
+    assertTrue(exception);
 
     exception = false;
     try {
@@ -595,12 +608,13 @@ public abstract class ApplicationMasterServiceTestBase {
     } catch (InvalidResourceRequestException e) {
       exception = true;
     }
-    Assert.assertTrue(exception);
+    assertTrue(exception);
 
     rm.close();
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testRequestCapacityMinMaxAllocationForResourceTypes()
       throws Exception {
     Map<String, ResourceInformation> riMap = initializeMandatoryResources();
@@ -637,25 +651,25 @@ public abstract class ApplicationMasterServiceTestBase {
     RMApp app1 = MockRMAppSubmitter.submit(rm, data);
     MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm1);
 
-    Assert.assertEquals(Resource.newInstance(GB, 1),
+    assertEquals(Resource.newInstance(GB, 1),
         getResourceUsageForQueue(rm, getDefaultQueueName()));
 
     // Request memory > allowed
     try {
       requestResources(am1, 9 * GB, 1, ImmutableMap.of());
-      Assert.fail("Should throw InvalidResourceRequestException");
+      fail("Should throw InvalidResourceRequestException");
     } catch (InvalidResourceRequestException ignored) {}
 
     try {
       // Request vcores > allowed
       requestResources(am1, GB, 18, ImmutableMap.of());
-      Assert.fail("Should throw InvalidResourceRequestException");
+      fail("Should throw InvalidResourceRequestException");
     } catch (InvalidResourceRequestException ignored) {}
 
     try {
       // Request custom resource 'res_1' > allowed
       requestResources(am1, GB, 2, ImmutableMap.of(CUSTOM_RES, 100));
-      Assert.fail("Should throw InvalidResourceRequestException");
+      fail("Should throw InvalidResourceRequestException");
     } catch (InvalidResourceRequestException ignored) {}
 
     rm.close();

+ 24 - 21
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java

@@ -19,6 +19,9 @@
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 import static org.apache.hadoop.yarn.server.resourcemanager.MockNM.createMockNodeStatus;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import org.apache.hadoop.classification.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
@@ -100,7 +103,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSe
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.YarnVersionInfo;
 import org.apache.hadoop.yarn.util.resource.ResourceUtils;
-import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.event.Level;
@@ -239,7 +241,7 @@ public class MockRM extends ResourceManager {
       throws InterruptedException {
     drainEventsImplicitly();
     RMApp app = getRMContext().getRMApps().get(appId);
-    Assert.assertNotNull("app shouldn't be null", app);
+    assertNotNull(app, "app shouldn't be null");
     final int timeoutMsecs = 80 * SECOND;
     int timeWaiting = 0;
     while (!finalStates.contains(app.getState())) {
@@ -254,8 +256,8 @@ public class MockRM extends ResourceManager {
     }
 
     LOG.info("App State is : " + app.getState());
-    Assert.assertTrue("App State is not correct (timeout).",
-        finalStates.contains(app.getState()));
+    assertTrue(finalStates.contains(app.getState()),
+        "App State is not correct (timeout).");
   }
 
   /**
@@ -270,7 +272,7 @@ public class MockRM extends ResourceManager {
       throws InterruptedException {
     drainEventsImplicitly();
     RMApp app = getRMContext().getRMApps().get(appId);
-    Assert.assertNotNull("app shouldn't be null", app);
+    assertNotNull(app, "app shouldn't be null");
     final int timeoutMsecs = 80 * SECOND;
     int timeWaiting = 0;
     while (!finalState.equals(app.getState())) {
@@ -285,8 +287,8 @@ public class MockRM extends ResourceManager {
     }
 
     LOG.info("App State is : " + app.getState());
-    Assert.assertEquals("App State is not correct (timeout).", finalState,
-      app.getState());
+    assertEquals(finalState, app.getState(),
+        "App State is not correct (timeout).");
   }
 
   /**
@@ -316,7 +318,7 @@ public class MockRM extends ResourceManager {
       throws InterruptedException {
     drainEventsImplicitly();
     RMApp app = getRMContext().getRMApps().get(attemptId.getApplicationId());
-    Assert.assertNotNull("app shouldn't be null", app);
+    assertNotNull(app, "app shouldn't be null");
     RMAppAttempt attempt = app.getRMAppAttempt(attemptId);
     MockRM.waitForState(attempt, finalState, timeoutMsecs);
   }
@@ -359,8 +361,8 @@ public class MockRM extends ResourceManager {
     }
 
     LOG.info("Attempt State is : " + attempt.getAppAttemptState());
-    Assert.assertEquals("Attempt state is not correct (timeout).", finalState,
-        attempt.getState());
+    assertEquals(finalState, attempt.getState(),
+        "Attempt state is not correct (timeout).");
   }
 
   public void waitForContainerToComplete(RMAppAttempt attempt,
@@ -384,7 +386,7 @@ public class MockRM extends ResourceManager {
   public MockAM waitForNewAMToLaunchAndRegister(ApplicationId appId, int attemptSize,
       MockNM nm) throws Exception {
     RMApp app = getRMContext().getRMApps().get(appId);
-    Assert.assertNotNull(app);
+    assertNotNull(app);
     int timeWaiting = 0;
     while (app.getAppAttempts().size() != attemptSize) {
       if (timeWaiting >= TIMEOUT_MS_FOR_ATTEMPT) {
@@ -607,7 +609,7 @@ public class MockRM extends ResourceManager {
       Thread.sleep(WAIT_MS_PER_LOOP);
       timeWaiting += WAIT_MS_PER_LOOP;
     }
-    Assert.assertNotNull("node shouldn't be null (timedout)", node);
+    assertNotNull(node, "node shouldn't be null (timedout)");
     while (!finalState.equals(node.getState())) {
       if (timeWaiting >= TIMEOUT_MS_FOR_CONTAINER_AND_NODE) {
         break;
@@ -620,22 +622,22 @@ public class MockRM extends ResourceManager {
     }
 
     LOG.info("Node " + nodeId + " State is : " + node.getState());
-    Assert.assertEquals("Node state is not correct (timedout)", finalState,
-        node.getState());
+    assertEquals(finalState, node.getState(),
+        "Node state is not correct (timedout)");
   }
 
   public void sendNodeGracefulDecommission(
       MockNM nm, int timeout) throws Exception {
     RMNodeImpl node = (RMNodeImpl)
         getRMContext().getRMNodes().get(nm.getNodeId());
-    Assert.assertNotNull("node shouldn't be null", node);
+    assertNotNull(node, "node shouldn't be null");
     node.handle(new RMNodeDecommissioningEvent(nm.getNodeId(), timeout));
   }
 
   public void sendNodeEvent(MockNM nm, RMNodeEventType event) throws Exception {
     RMNodeImpl node = (RMNodeImpl)
         getRMContext().getRMNodes().get(nm.getNodeId());
-    Assert.assertNotNull("node shouldn't be null", node);
+    assertNotNull(node, "node shouldn't be null");
     node.handle(new RMNodeEvent(nm.getNodeId(), event));
   }
 
@@ -883,9 +885,10 @@ public class MockRM extends ResourceManager {
       }
       tick++;
     }
-    Assert.assertNotNull("Timed out waiting for SchedulerApplicationAttempt=" +
-      attemptId + " to be added.", ((AbstractYarnScheduler)
-        rm.getResourceScheduler()).getApplicationAttempt(attemptId));
+    assertNotNull(((AbstractYarnScheduler)
+        rm.getResourceScheduler()).getApplicationAttempt(attemptId),
+        "Timed out waiting for SchedulerApplicationAttempt=" +
+        attemptId + " to be added.");
   }
 
   public static MockAM launchAMWhenAsyncSchedulingEnabled(RMApp app, MockRM rm)
@@ -1033,8 +1036,8 @@ public class MockRM extends ResourceManager {
       Thread.sleep(WAIT_MS_PER_LOOP);
       timeWaiting += WAIT_MS_PER_LOOP;
     }
-    Assert.assertTrue("app is not removed from scheduler (timeout).",
-        !apps.containsKey(appId));
+    assertTrue(!apps.containsKey(appId),
+        "app is not removed from scheduler (timeout).");
     LOG.info("app is removed from scheduler, " + appId);
   }
 

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -38,7 +40,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.ResourceLocalizationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.ResourceLocalizationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RestartContainerResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RollbackResponse;
-import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -225,10 +226,10 @@ public class NodeManager implements ContainerManagementProtocol {
 
   synchronized public void checkResourceUsage() {
     LOG.info("Checking resource usage for " + containerManagerAddress);
-    Assert.assertEquals(available.getMemorySize(),
+    assertEquals(available.getMemorySize(),
         resourceManager.getResourceScheduler().getNodeReport(
             this.nodeId).getAvailableResource().getMemorySize());
-    Assert.assertEquals(used.getMemorySize(),
+    assertEquals(used.getMemorySize(),
         resourceManager.getResourceScheduler().getNodeReport(
             this.nodeId).getUsedResource().getMemorySize());
   }

+ 15 - 14
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/QueueACLsTestBase.java

@@ -18,13 +18,16 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.QueueACL;
-import org.junit.Assert;
 
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
@@ -211,14 +214,12 @@ public abstract class QueueACLsTestBase extends ACLsTestBase {
     UserGroupInformation user = UserGroupInformation.getCurrentUser();
 
     String failureMsg = "Wrong %s access to %s queue";
-    Assert.assertEquals(
-        String.format(failureMsg, QueueACL.ADMINISTER_QUEUE, queueName),
-        access, resourceManager.getResourceScheduler()
-        .checkAccess(user, QueueACL.ADMINISTER_QUEUE, queueName));
-    Assert.assertEquals(
-        String.format(failureMsg, QueueACL.SUBMIT_APPLICATIONS, queueName),
-        access, resourceManager.getResourceScheduler()
-        .checkAccess(user, QueueACL.SUBMIT_APPLICATIONS, queueName));
+    assertEquals(access, resourceManager.getResourceScheduler()
+        .checkAccess(user, QueueACL.ADMINISTER_QUEUE, queueName),
+        String.format(failureMsg, QueueACL.ADMINISTER_QUEUE, queueName));
+    assertEquals(access, resourceManager.getResourceScheduler()
+        .checkAccess(user, QueueACL.SUBMIT_APPLICATIONS, queueName),
+        String.format(failureMsg, QueueACL.SUBMIT_APPLICATIONS, queueName));
   }
 
   private void verifyGetClientAMToken(String submitter, String queueAdmin,
@@ -236,7 +237,7 @@ public abstract class QueueACLsTestBase extends ACLsTestBase {
     GetApplicationReportResponse adMinUserGetReport =
         adMinUserClient.getApplicationReport(appReportRequest);
 
-    Assert.assertEquals(submitterGetReport.getApplicationReport()
+    assertEquals(submitterGetReport.getApplicationReport()
       .getClientToAMToken(), adMinUserGetReport.getApplicationReport()
       .getClientToAMToken());
   }
@@ -255,12 +256,12 @@ public abstract class QueueACLsTestBase extends ACLsTestBase {
     // Kill app as the killer
     try {
       killerClient.forceKillApplication(finishAppRequest);
-      Assert.fail("App killing by the enemy should fail!!");
+      fail("App killing by the enemy should fail!!");
     } catch (YarnException e) {
       LOG.info("Got exception while killing app as the enemy", e);
-      Assert.assertTrue(e.getMessage().contains(
-        "User " + killer + " cannot perform operation MODIFY_APP on "
-            + applicationId));
+      assertTrue(e.getMessage().contains(
+          "User " + killer + " cannot perform operation MODIFY_APP on "
+          + applicationId));
     }
 
     getRMClientForUser(submitter).forceKillApplication(finishAppRequest);

+ 33 - 25
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ReservationACLsTestBase.java

@@ -18,6 +18,9 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+
 import java.io.File;
 import java.io.IOException;
 import java.util.Arrays;
@@ -58,13 +61,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair
     .allocationfile.AllocationFileWriter;
 import org.apache.hadoop.yarn.util.resource.Resources;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 
-@RunWith(Parameterized.class)
 public class ReservationACLsTestBase extends ACLsTestBase {
 
   private final int defaultDuration = 600000;
@@ -77,28 +77,36 @@ public class ReservationACLsTestBase extends ACLsTestBase {
   private Configuration configuration;
   private boolean useFullQueuePath;
 
-  public ReservationACLsTestBase(Configuration conf, boolean useFullPath) {
+  @Override
+  public void setup() throws InterruptedException, IOException {
+    super.setup();
+  }
+
+  public void initReservationACLsTestBase(Configuration conf, boolean useFullPath)
+      throws IOException, InterruptedException {
     configuration = conf;
     useFullQueuePath = useFullPath;
+    setup();
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
     if (resourceManager != null) {
       resourceManager.stop();
     }
   }
 
-  @Parameterized.Parameters
   public static Collection<Object[]> data() throws IOException {
-    return Arrays.asList(new Object[][] {
-            { createCapacitySchedulerConfiguration(), false },
-            { createFairSchedulerConfiguration(), true }
+    return Arrays.asList(new Object[][]{
+        {createCapacitySchedulerConfiguration(), true},
+        {createFairSchedulerConfiguration(), true}
     });
   }
 
-  @Test
-  public void testApplicationACLs() throws Exception {
+  @ParameterizedTest
+  @MethodSource("data")
+  public void testApplicationACLs(Configuration conf, boolean useFullPath) throws Exception {
+    initReservationACLsTestBase(conf, useFullPath);
     registerNode("test:1234", 8192, 8);
     String queueA = !useFullQueuePath? QUEUEA : CapacitySchedulerConfiguration
             .ROOT + "." + QUEUEA;
@@ -258,7 +266,7 @@ public class ReservationACLsTestBase extends ACLsTestBase {
     try {
       ReservationId reservationId = createReservation(submitter);
       submitReservation(submitter, queueName, reservationId);
-      Assert.fail("Submit reservation by the enemy should fail!");
+      fail("Submit reservation by the enemy should fail!");
     } catch (YarnException e) {
       handleAdministerException(e, submitter, queueName, ReservationACL
               .SUBMIT_RESERVATIONS.name());
@@ -286,7 +294,7 @@ public class ReservationACLsTestBase extends ACLsTestBase {
 
     try {
       listReservation(lister, queueName);
-      Assert.fail("List reservation by the enemy should fail!");
+      fail("List reservation by the enemy should fail!");
     } catch (YarnException e) {
       handleAdministerException(e, lister, queueName, ReservationACL
               .LIST_RESERVATIONS.name());
@@ -316,7 +324,7 @@ public class ReservationACLsTestBase extends ACLsTestBase {
     submitReservation(originalSubmitter, queueName, reservationId);
     try {
       listReservationById(lister, reservationId, queueName);
-      Assert.fail("List reservation by the enemy should fail!");
+      fail("List reservation by the enemy should fail!");
     } catch (YarnException e) {
       handleAdministerException(e, lister, queueName, ReservationACL
               .LIST_RESERVATIONS.name());
@@ -341,7 +349,7 @@ public class ReservationACLsTestBase extends ACLsTestBase {
 
     try {
       deleteReservation(killer, reservationId);
-      Assert.fail("Reservation deletion by the enemy should fail!");
+      fail("Reservation deletion by the enemy should fail!");
     } catch (YarnException e) {
       handleAdministerException(e, killer, queueName, ReservationACL
               .ADMINISTER_RESERVATIONS.name());
@@ -378,7 +386,7 @@ public class ReservationACLsTestBase extends ACLsTestBase {
     ApplicationClientProtocol unauthorizedClient = getRMClientForUser(updater);
     try {
       unauthorizedClient.updateReservation(updateRequest);
-      Assert.fail("Reservation updating by the enemy should fail.");
+      fail("Reservation updating by the enemy should fail.");
     } catch (YarnException e) {
       handleAdministerException(e, updater, queueName, ReservationACL
               .ADMINISTER_RESERVATIONS.name());
@@ -454,9 +462,9 @@ public class ReservationACLsTestBase extends ACLsTestBase {
   private void handleAdministerException(Exception e, String user, String
           queue, String operation) {
     LOG.info("Got exception while killing app as the enemy", e);
-    Assert.assertTrue(e.getMessage().contains("User " + user
-            + " cannot perform operation " + operation + " on queue "
-            + queue));
+    assertTrue(e.getMessage().contains("User " + user
+        + " cannot perform operation " + operation + " on queue "
+        + queue));
   }
 
   private void registerNode(String host, int memory, int vCores) throws
@@ -477,13 +485,13 @@ public class ReservationACLsTestBase extends ACLsTestBase {
         Thread.sleep(100);
       } while (attempts-- > 0);
       if (attempts <= 0) {
-        Assert.fail("Exhausted attempts in checking if node capacity was "
-                + "added to the plan");
+        fail("Exhausted attempts in checking if node capacity was "
+            + "added to the plan");
       }
 
     } catch (Exception e) {
       e.printStackTrace();
-      Assert.fail(e.getMessage());
+      fail(e.getMessage());
     }
   }
 

+ 34 - 31
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java

@@ -18,6 +18,11 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+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 java.io.IOException;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
@@ -27,6 +32,8 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
 
+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;
@@ -75,28 +82,22 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.util.Records;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-@RunWith(Parameterized.class)
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+
 public class TestAMAuthorization {
 
   private static final Logger LOG =
       LoggerFactory.getLogger(TestAMAuthorization.class);
 
-  private final Configuration conf;
+  private Configuration conf;
   private MockRM rm;
 
   // Note : Any test case in ResourceManager package that creates a proxy has
   // to be run with enabling hadoop.security.token.service.use_ip. And reset
   // to false at the end of test class. See YARN-5208
-  @BeforeClass
+  @BeforeAll
   public static void setUp() {
     Configuration conf = new Configuration();
     conf.setBoolean(
@@ -104,7 +105,7 @@ public class TestAMAuthorization {
     SecurityUtil.setConfiguration(conf);
   }
 
-  @AfterClass
+  @AfterAll
   public static void resetConf() {
     Configuration conf = new Configuration();
     conf.setBoolean(
@@ -112,7 +113,6 @@ public class TestAMAuthorization {
     SecurityUtil.setConfiguration(conf);
   }
 
-  @Parameters
   public static Collection<Object[]> configs() {
     Configuration conf = new Configuration();
     Configuration confWithSecurity = new Configuration();
@@ -122,12 +122,12 @@ public class TestAMAuthorization {
     return Arrays.asList(new Object[][] {{ conf }, { confWithSecurity} });
   }
 
-  public TestAMAuthorization(Configuration conf) {
-    this.conf = conf;
+  public void initTestAMAuthorization(Configuration pConf) {
+    this.conf = pConf;
     UserGroupInformation.setConfiguration(conf);
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
     if (rm != null) {
       rm.stop();
@@ -258,8 +258,10 @@ public class TestAMAuthorization {
     }
   }
 
-  @Test
-  public void testAuthorizedAccess() throws Exception {
+  @ParameterizedTest
+  @MethodSource("configs")
+  public void testAuthorizedAccess(Configuration pConf) throws Exception {
+    initTestAMAuthorization(pConf);
     MyContainerManager containerManager = new MyContainerManager();
     rm =
         new MockRMWithAMS(conf, containerManager);
@@ -285,7 +287,7 @@ public class TestAMAuthorization {
       LOG.info("Waiting for AM Launch to happen..");
       Thread.sleep(1000);
     }
-    Assert.assertNotNull(containerManager.containerTokens);
+    assertNotNull(containerManager.containerTokens);
 
     RMAppAttempt attempt = app.getCurrentAppAttempt();
     ApplicationAttemptId applicationAttemptId = attempt.getAppAttemptId();
@@ -317,17 +319,18 @@ public class TestAMAuthorization {
         .newRecord(RegisterApplicationMasterRequest.class);
     RegisterApplicationMasterResponse response =
         client.registerApplicationMaster(request);
-    Assert.assertNotNull(response.getClientToAMTokenMasterKey());
+    assertNotNull(response.getClientToAMTokenMasterKey());
     if (UserGroupInformation.isSecurityEnabled()) {
-      Assert
-        .assertTrue(response.getClientToAMTokenMasterKey().array().length > 0);
+      assertTrue(response.getClientToAMTokenMasterKey().array().length > 0);
     }
-    Assert.assertEquals("Register response has bad ACLs", "*",
-        response.getApplicationACLs().get(ApplicationAccessType.VIEW_APP));
+    assertEquals("*", response.getApplicationACLs().get(ApplicationAccessType.VIEW_APP),
+        "Register response has bad ACLs");
   }
 
-  @Test
-  public void testUnauthorizedAccess() throws Exception {
+  @ParameterizedTest
+  @MethodSource("configs")
+  public void testUnauthorizedAccess(Configuration pConf) throws Exception {
+    initTestAMAuthorization(pConf);
     MyContainerManager containerManager = new MyContainerManager();
     rm = new MockRMWithAMS(conf, containerManager);
     rm.start();
@@ -343,7 +346,7 @@ public class TestAMAuthorization {
       LOG.info("Waiting for AM Launch to happen..");
       Thread.sleep(1000);
     }
-    Assert.assertNotNull(containerManager.containerTokens);
+    assertNotNull(containerManager.containerTokens);
 
     RMAppAttempt attempt = app.getCurrentAppAttempt();
     ApplicationAttemptId applicationAttemptId = attempt.getAppAttemptId();
@@ -373,7 +376,7 @@ public class TestAMAuthorization {
         .newRecord(RegisterApplicationMasterRequest.class);
     try {
       client.registerApplicationMaster(request);
-      Assert.fail("Should fail with authorization error");
+      fail("Should fail with authorization error");
     } catch (Exception e) {
       if (isCause(AccessControlException.class, e)) {
         // Because there are no tokens, the request should be rejected as the
@@ -385,7 +388,7 @@ public class TestAMAuthorization {
           expectedMessage =
               "SIMPLE authentication is not enabled.  Available:[TOKEN]";
         }
-        Assert.assertTrue(e.getCause().getMessage().contains(expectedMessage)); 
+        assertTrue(e.getCause().getMessage().contains(expectedMessage));
       } else {
         throw e;
       }
@@ -441,7 +444,7 @@ public class TestAMAuthorization {
           + "Current state is " + attempt.getAppAttemptState());
       Thread.sleep(1000);
     }
-    Assert.assertEquals(attempt.getAppAttemptState(),
+    assertEquals(attempt.getAppAttemptState(),
         RMAppAttemptState.LAUNCHED);
   }
 }

+ 144 - 139
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java

@@ -84,16 +84,16 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 
 import org.apache.hadoop.thirdparty.com.google.common.collect.Maps;
 
+import org.junit.jupiter.api.extension.BeforeEachCallback;
+import org.junit.jupiter.api.extension.ExtensionContext;
+import org.junit.jupiter.api.extension.RegisterExtension;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestWatcher;
-import org.junit.runner.Description;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 import org.mockito.ArgumentCaptor;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
@@ -101,6 +101,7 @@ import org.mockito.stubbing.Answer;
 import java.io.IOException;
 import java.lang.annotation.Retention;
 import java.lang.annotation.RetentionPolicy;
+import java.lang.reflect.Method;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -118,8 +119,11 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.A
 import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.PREFIX;
 import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePrefixes.getAutoCreatedQueueTemplateConfPrefix;
 import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePrefixes.getQueuePrefix;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyString;
 import static org.mockito.ArgumentMatchers.eq;
@@ -138,8 +142,9 @@ import static org.mockito.Mockito.when;
  */
 
 public class TestAppManager extends AppManagerTestBase{
-  @Rule
-  public UseCapacitySchedulerRule shouldUseCs = new UseCapacitySchedulerRule();
+
+  @RegisterExtension
+  private UseCapacitySchedulerRule shouldUseCs = new UseCapacitySchedulerRule();
 
   private static final Logger LOG =
       LoggerFactory.getLogger(TestAppManager.class);
@@ -248,7 +253,7 @@ public class TestAppManager extends AppManagerTestBase{
   private QueueInfo mockDefaultQueueInfo;
 
   @SuppressWarnings("deprecation")
-  @Before
+  @BeforeEach
   public void setUp() throws IOException {
     long now = System.currentTimeMillis();
 
@@ -510,7 +515,7 @@ public class TestAppManager extends AppManagerTestBase{
         ((CapacityScheduler) newMockRM.getResourceScheduler());
     cs.getCapacitySchedulerQueueManager().createQueue(user2QueuePath);
     AutoCreatedLeafQueue autoCreatedLeafQueue = (AutoCreatedLeafQueue) cs.getQueue("user2");
-    Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedLeafQueue);
+    assertNotNull(autoCreatedLeafQueue, "Auto Creation of Queue failed");
     ManagedParentQueue parentQueue = (ManagedParentQueue) cs.getQueue("parent");
     assertEquals(parentQueue, autoCreatedLeafQueue.getParent());
     // reinitialize to load the ACLs for the queue
@@ -599,12 +604,12 @@ public class TestAppManager extends AppManagerTestBase{
     cs.getCapacitySchedulerQueueManager().createQueue(new QueuePath("root.parent.user2.user3"));
 
     ParentQueue autoCreatedParentQueue = (ParentQueue) cs.getQueue("user2");
-    Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedParentQueue);
+    assertNotNull(autoCreatedParentQueue, "Auto Creation of Queue failed");
     ParentQueue parentQueue = (ParentQueue) cs.getQueue("parent");
     assertEquals(parentQueue, autoCreatedParentQueue.getParent());
 
     LeafQueue autoCreatedLeafQueue = (LeafQueue) cs.getQueue("user3");
-    Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedLeafQueue);
+    assertNotNull(autoCreatedLeafQueue, "Auto Creation of Queue failed");
     assertEquals(autoCreatedParentQueue, autoCreatedLeafQueue.getParent());
 
     // reinitialize to load the ACLs for the queue
@@ -722,12 +727,12 @@ public class TestAppManager extends AppManagerTestBase{
     cs.getCapacitySchedulerQueueManager().createQueue(new QueuePath("root.parent.user2.user3"));
 
     ParentQueue autoCreatedParentQueue = (ParentQueue) cs.getQueue("user2");
-    Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedParentQueue);
+    assertNotNull(autoCreatedParentQueue, "Auto Creation of Queue failed");
     ParentQueue parentQueue = (ParentQueue) cs.getQueue("parent");
     assertEquals(parentQueue, autoCreatedParentQueue.getParent());
 
     LeafQueue autoCreatedLeafQueue = (LeafQueue) cs.getQueue("user3");
-    Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedLeafQueue);
+    assertNotNull(autoCreatedLeafQueue, "Auto Creation of Queue failed");
     assertEquals(autoCreatedParentQueue, autoCreatedLeafQueue.getParent());
 
     // reinitialize to load the ACLs for the queue
@@ -822,7 +827,7 @@ public class TestAppManager extends AppManagerTestBase{
 
     ParentQueue autoCreatedParentQueue = (ParentQueue) cs.getQueue("parent");
     LeafQueue autoCreatedLeafQueue = (LeafQueue) cs.getQueue("user2");
-    Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedLeafQueue);
+    assertNotNull(autoCreatedLeafQueue, "Auto Creation of Queue failed");
     assertEquals(autoCreatedParentQueue, autoCreatedLeafQueue.getParent());
 
     // reinitialize to load the ACLs for the queue
@@ -916,12 +921,12 @@ public class TestAppManager extends AppManagerTestBase{
     cs.getCapacitySchedulerQueueManager().createQueue(new QueuePath("root.user2.user3"));
 
     ParentQueue autoCreatedParentQueue = (ParentQueue) cs.getQueue("user2");
-    Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedParentQueue);
+    assertNotNull(autoCreatedParentQueue, "Auto Creation of Queue failed");
     ParentQueue parentQueue = (ParentQueue) cs.getQueue("root");
     assertEquals(parentQueue, autoCreatedParentQueue.getParent());
 
     LeafQueue autoCreatedLeafQueue = (LeafQueue) cs.getQueue("user3");
-    Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedLeafQueue);
+    assertNotNull(autoCreatedLeafQueue, "Auto Creation of Queue failed");
     assertEquals(autoCreatedParentQueue, autoCreatedLeafQueue.getParent());
 
     // reinitialize to load the ACLs for the queue
@@ -994,16 +999,16 @@ public class TestAppManager extends AppManagerTestBase{
     cs.getCapacitySchedulerQueueManager().createQueue(new QueuePath("root.user2.user3.queue"));
 
     ParentQueue autoCreatedParentQueue = (ParentQueue) cs.getQueue("user2");
-    Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedParentQueue);
+    assertNotNull(autoCreatedParentQueue, "Auto Creation of Queue failed");
     ParentQueue parentQueue = (ParentQueue) cs.getQueue("root");
     assertEquals(parentQueue, autoCreatedParentQueue.getParent());
 
     ParentQueue autoCreatedParentQueue2 = (ParentQueue) cs.getQueue("user3");
-    Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedParentQueue2);
+    assertNotNull(autoCreatedParentQueue2, "Auto Creation of Queue failed");
     assertEquals(autoCreatedParentQueue, autoCreatedParentQueue2.getParent());
 
     LeafQueue autoCreatedLeafQueue = (LeafQueue) cs.getQueue("queue");
-    Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedLeafQueue);
+    assertNotNull(autoCreatedLeafQueue, "Auto Creation of Queue failed");
     assertEquals(autoCreatedParentQueue, autoCreatedParentQueue2.getParent());
 
     // reinitialize to load the ACLs for the queue
@@ -1046,7 +1051,7 @@ public class TestAppManager extends AppManagerTestBase{
                                                  String user) {
     try {
       appManager.submitApplication(submission, user);
-      Assert.fail(
+      fail(
           String.format("should fail since %s does not have permission to submit to queue", user));
     } catch (YarnException e) {
       assertTrue(e.getCause() instanceof AccessControlException);
@@ -1060,10 +1065,9 @@ public class TestAppManager extends AppManagerTestBase{
                                           String expectedQueue) throws YarnException {
     appManager.submitApplication(submission, user);
     RMApp app = rmContext.getRMApps().get(submission.getApplicationId());
-    Assert.assertNotNull("app should not be null", app);
-    Assert.assertEquals(String.format("the queue should be placed on '%s' queue", expectedQueue),
-        expectedQueue,
-        app.getQueue());
+    assertNotNull(app, "app should not be null");
+    assertEquals(expectedQueue, app.getQueue(),
+        String.format("the queue should be placed on '%s' queue", expectedQueue));
   }
 
   private static ApplicationSubmissionContext createAppSubmissionContext(ApplicationId id) {
@@ -1078,7 +1082,7 @@ public class TestAppManager extends AppManagerTestBase{
     return appSubmission;
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
     setAppEventType(RMAppEventType.KILL);
     ((Service)rmContext.getDispatcher()).stop();
@@ -1096,18 +1100,18 @@ public class TestAppManager extends AppManagerTestBase{
     conf.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, 10);
     TestRMAppManager appMonitor = new TestRMAppManager(rmContext,conf);
 
-    Assert.assertEquals("Number of apps incorrect before checkAppTimeLimit",
-        10, rmContext.getRMApps().size());
+    assertEquals(10, rmContext.getRMApps().size(),
+        "Number of apps incorrect before checkAppTimeLimit");
 
     // add them to completed apps list
     addToCompletedApps(appMonitor, rmContext);
 
     // shouldn't  have to many apps
     appMonitor.checkAppNumCompletedLimit();
-    Assert.assertEquals("Number of apps incorrect after # completed check", 10,
-        rmContext.getRMApps().size());
-    Assert.assertEquals("Number of completed apps incorrect after check", 10,
-        appMonitor.getCompletedAppsListSize());
+    assertEquals(10, rmContext.getRMApps().size(),
+        "Number of apps incorrect after # completed check");
+    assertEquals(10, appMonitor.getCompletedAppsListSize(),
+        "Number of completed apps incorrect after check");
     verify(rmContext.getStateStore(), never()).removeApplication(
       isA(RMApp.class));
   }
@@ -1141,7 +1145,7 @@ public class TestAppManager extends AppManagerTestBase{
       if (e instanceof YarnException) {
         assertTrue(e.getCause() instanceof AccessControlException);
       } else {
-        Assert.fail("Yarn exception is expected : " + e.getMessage());
+        fail("Yarn exception is expected : " + e.getMessage());
       }
     } finally {
       mockRM.close();
@@ -1158,18 +1162,18 @@ public class TestAppManager extends AppManagerTestBase{
     conf.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, 3);
     TestRMAppManager appMonitor = new TestRMAppManager(rmContext, conf);
 
-    Assert.assertEquals("Number of apps incorrect before", 10, rmContext
-        .getRMApps().size());
+    assertEquals(10, rmContext
+        .getRMApps().size(), "Number of apps incorrect before");
 
     // add them to completed apps list
     addToCompletedApps(appMonitor, rmContext);
 
     // shouldn't  have to many apps
     appMonitor.checkAppNumCompletedLimit();
-    Assert.assertEquals("Number of apps incorrect after # completed check", 3,
-        rmContext.getRMApps().size());
-    Assert.assertEquals("Number of completed apps incorrect after check", 3,
-        appMonitor.getCompletedAppsListSize());
+    assertEquals(3, rmContext.getRMApps().size(),
+        "Number of apps incorrect after # completed check");
+    assertEquals(3, appMonitor.getCompletedAppsListSize(),
+        "Number of completed apps incorrect after check");
     verify(rmContext.getStateStore(), times(7)).removeApplication(
       isA(RMApp.class));
   }
@@ -1188,7 +1192,7 @@ public class TestAppManager extends AppManagerTestBase{
 
     // clear out applications map
     rmContext.getRMApps().clear();
-    Assert.assertEquals("map isn't empty", 0, rmContext.getRMApps().size());
+    assertEquals(0, rmContext.getRMApps().size(), "map isn't empty");
 
     // 6 applications are in final state, 4 are not in final state.
     // / set with various finished states
@@ -1215,18 +1219,18 @@ public class TestAppManager extends AppManagerTestBase{
     app = new MockRMApp(9, now - 20000, RMAppState.FAILED);
     rmContext.getRMApps().put(app.getApplicationId(), app);
 
-    Assert.assertEquals("Number of apps incorrect before", 10, rmContext
-        .getRMApps().size());
+    assertEquals(10, rmContext
+        .getRMApps().size(), "Number of apps incorrect before");
 
     // add them to completed apps list
     addToCompletedApps(appMonitor, rmContext);
 
     // shouldn't  have to many apps
     appMonitor.checkAppNumCompletedLimit();
-    Assert.assertEquals("Number of apps incorrect after # completed check", 6,
-        rmContext.getRMApps().size());
-    Assert.assertEquals("Number of completed apps incorrect after check", 2,
-        appMonitor.getCompletedAppsListSize());
+    assertEquals(6, rmContext.getRMApps().size(),
+        "Number of apps incorrect after # completed check");
+    assertEquals(2, appMonitor.getCompletedAppsListSize(),
+        "Number of completed apps incorrect after check");
     // 6 applications in final state, 4 of them are removed
     verify(rmContext.getStateStore(), times(4)).removeApplication(
       isA(RMApp.class));
@@ -1239,13 +1243,13 @@ public class TestAppManager extends AppManagerTestBase{
     RMContext rmContext = mockRMContext(10, now - 20000);
     TestRMAppManager appMonitor = new TestRMAppManager(rmContext, new Configuration());
 
-    Assert.assertEquals("Number of apps incorrect before", 10, rmContext
-        .getRMApps().size());
+    assertEquals(10, rmContext
+        .getRMApps().size(), "Number of apps incorrect before");
 
     appMonitor.finishApplication(null);
 
-    Assert.assertEquals("Number of completed apps incorrect after check", 0,
-        appMonitor.getCompletedAppsListSize());
+    assertEquals(0, appMonitor.getCompletedAppsListSize(),
+        "Number of completed apps incorrect after check");
   }
 
   @Test
@@ -1257,19 +1261,19 @@ public class TestAppManager extends AppManagerTestBase{
     conf.setInt(YarnConfiguration.RM_STATE_STORE_MAX_COMPLETED_APPLICATIONS, 0);
     conf.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, 0);
     TestRMAppManager appMonitor = new TestRMAppManager(rmContext, conf);
-    Assert.assertEquals("Number of apps incorrect before", 10, rmContext
-        .getRMApps().size());
+    assertEquals(10, rmContext
+        .getRMApps().size(), "Number of apps incorrect before");
 
     addToCompletedApps(appMonitor, rmContext);
-    Assert.assertEquals("Number of completed apps incorrect", 10,
-        appMonitor.getCompletedAppsListSize());
+    assertEquals(10, appMonitor.getCompletedAppsListSize(),
+        "Number of completed apps incorrect");
 
     appMonitor.checkAppNumCompletedLimit();
 
-    Assert.assertEquals("Number of apps incorrect after # completed check", 0,
-        rmContext.getRMApps().size());
-    Assert.assertEquals("Number of completed apps incorrect after check", 0,
-        appMonitor.getCompletedAppsListSize());
+    assertEquals(0, rmContext.getRMApps().size(),
+        "Number of apps incorrect after # completed check");
+    assertEquals(0, appMonitor.getCompletedAppsListSize(),
+        "Number of completed apps incorrect after check");
     verify(rmContext.getStateStore(), times(10)).removeApplication(
       isA(RMApp.class));
   }
@@ -1288,19 +1292,19 @@ public class TestAppManager extends AppManagerTestBase{
     TestRMAppManager appMonitor = new TestRMAppManager(rmContext, conf);
 
     addToCompletedApps(appMonitor, rmContext);
-    Assert.assertEquals("Number of completed apps incorrect", allApps,
-        appMonitor.getCompletedAppsListSize());
+    assertEquals(allApps, appMonitor.getCompletedAppsListSize(),
+        "Number of completed apps incorrect");
     appMonitor.checkAppNumCompletedLimit();
 
-    Assert.assertEquals("Number of apps incorrect after # completed check",
-      maxAppsInMemory, rmContext.getRMApps().size());
-    Assert.assertEquals("Number of completed apps incorrect after check",
-      maxAppsInMemory, appMonitor.getCompletedAppsListSize());
+    assertEquals(maxAppsInMemory, rmContext.getRMApps().size(),
+        "Number of apps incorrect after # completed check");
+    assertEquals(maxAppsInMemory, appMonitor.getCompletedAppsListSize(),
+        "Number of completed apps incorrect after check");
 
     int numRemoveAppsFromStateStore = 10 - maxAppsInStateStore;
     verify(rmContext.getStateStore(), times(numRemoveAppsFromStateStore))
       .removeApplication(isA(RMApp.class));
-    Assert.assertEquals(maxAppsInStateStore,
+    assertEquals(maxAppsInStateStore,
       appMonitor.getNumberOfCompletedAppsInStateStore());
   }
 
@@ -1317,18 +1321,18 @@ public class TestAppManager extends AppManagerTestBase{
     TestRMAppManager appMonitor = new TestRMAppManager(rmContext, conf);
 
     addToCompletedApps(appMonitor, rmContext);
-    Assert.assertEquals("Number of completed apps incorrect", allApps,
-        appMonitor.getCompletedAppsListSize());
+    assertEquals(allApps, appMonitor.getCompletedAppsListSize(),
+        "Number of completed apps incorrect");
     appMonitor.checkAppNumCompletedLimit();
 
     int numRemoveApps = allApps - maxAppsInMemory;
-    Assert.assertEquals("Number of apps incorrect after # completed check",
-      maxAppsInMemory, rmContext.getRMApps().size());
-    Assert.assertEquals("Number of completed apps incorrect after check",
-      maxAppsInMemory, appMonitor.getCompletedAppsListSize());
+    assertEquals(maxAppsInMemory, rmContext.getRMApps().size(),
+        "Number of apps incorrect after # completed check");
+    assertEquals(maxAppsInMemory, appMonitor.getCompletedAppsListSize(),
+        "Number of completed apps incorrect after check");
     verify(rmContext.getStateStore(), times(numRemoveApps)).removeApplication(
       isA(RMApp.class));
-    Assert.assertEquals(maxAppsInMemory,
+    assertEquals(maxAppsInMemory,
       appMonitor.getNumberOfCompletedAppsInStateStore());
   }
 
@@ -1340,7 +1344,7 @@ public class TestAppManager extends AppManagerTestBase{
     rmContext.getDispatcher().register(RMAppManagerEventType.class, testAppManagerDispatcher);
     ((Service)rmContext.getDispatcher()).init(conf);
     ((Service)rmContext.getDispatcher()).start();
-    Assert.assertEquals("app event type is wrong before", RMAppEventType.KILL, appEventType);
+    assertEquals(RMAppEventType.KILL, appEventType, "app event type is wrong before");
   }
 
   @SuppressWarnings("deprecation")
@@ -1360,8 +1364,8 @@ public class TestAppManager extends AppManagerTestBase{
     asContext.setAMContainerResourceRequests(cloneResourceRequests(reqs));
     // getAMContainerResourceRequest uses the first entry of
     // getAMContainerResourceRequests
-    Assert.assertEquals(reqs.get(0), asContext.getAMContainerResourceRequest());
-    Assert.assertEquals(reqs, asContext.getAMContainerResourceRequests());
+    assertEquals(reqs.get(0), asContext.getAMContainerResourceRequest());
+    assertEquals(reqs, asContext.getAMContainerResourceRequests());
     RMApp app = testRMAppSubmit();
     for (ResourceRequest req : reqs) {
       req.setNodeLabelExpression(RMNodeLabelsManager.NO_LABEL);
@@ -1369,7 +1373,7 @@ public class TestAppManager extends AppManagerTestBase{
 
     // setAMContainerResourceRequests has priority over
     // setAMContainerResourceRequest and setResource
-    Assert.assertEquals(reqs, app.getAMResourceRequests());
+    assertEquals(reqs, app.getAMResourceRequests());
   }
 
   @SuppressWarnings("deprecation")
@@ -1384,12 +1388,12 @@ public class TestAppManager extends AppManagerTestBase{
     asContext.setAMContainerResourceRequest(ResourceRequest.clone(req));
     // getAMContainerResourceRequests uses a singleton list of
     // getAMContainerResourceRequest
-    Assert.assertEquals(req, asContext.getAMContainerResourceRequest());
-    Assert.assertEquals(req, asContext.getAMContainerResourceRequests().get(0));
-    Assert.assertEquals(1, asContext.getAMContainerResourceRequests().size());
+    assertEquals(req, asContext.getAMContainerResourceRequest());
+    assertEquals(req, asContext.getAMContainerResourceRequests().get(0));
+    assertEquals(1, asContext.getAMContainerResourceRequests().size());
     RMApp app = testRMAppSubmit();
     // setAMContainerResourceRequest has priority over setResource
-    Assert.assertEquals(Collections.singletonList(req),
+    assertEquals(Collections.singletonList(req),
         app.getAMResourceRequests());
   }
 
@@ -1416,7 +1420,7 @@ public class TestAppManager extends AppManagerTestBase{
 
     RMApp app = rmContext.getRMApps().get(appId);
     waitUntilEventProcessed();
-    Assert.assertEquals(defaultAMNodeLabel,
+    assertEquals(defaultAMNodeLabel,
         app.getAMResourceRequests().get(0).getNodeLabelExpression());
   }
 
@@ -1427,7 +1431,7 @@ public class TestAppManager extends AppManagerTestBase{
     RMApp app = testRMAppSubmit();
 
     // setResource
-    Assert.assertEquals(Collections.singletonList(
+    assertEquals(Collections.singletonList(
         ResourceRequest.newInstance(RMAppAttemptImpl.AM_CONTAINER_PRIORITY,
         ResourceRequest.ANY, Resources.createResource(1024), 1, true,
             "")),
@@ -1440,9 +1444,9 @@ public class TestAppManager extends AppManagerTestBase{
     asContext.setAMContainerResourceRequests(null);
     try {
       testRMAppSubmit();
-      Assert.fail("Should have failed due to no ResourceRequest");
+      fail("Should have failed due to no ResourceRequest");
     } catch (InvalidResourceRequestException e) {
-      Assert.assertEquals(
+      assertEquals(
           "Invalid resource request, no resources requested",
           e.getMessage());
     }
@@ -1476,7 +1480,7 @@ public class TestAppManager extends AppManagerTestBase{
       req.setNumContainers(1);
       req.setPriority(Priority.newInstance(0));
     }
-    Assert.assertEquals(reqs, app.getAMResourceRequests());
+    assertEquals(reqs, app.getAMResourceRequests());
   }
 
   @Test
@@ -1491,12 +1495,12 @@ public class TestAppManager extends AppManagerTestBase{
     asContext.setAMContainerResourceRequests(cloneResourceRequests(reqs));
     // getAMContainerResourceRequest uses the first entry of
     // getAMContainerResourceRequests
-    Assert.assertEquals(reqs, asContext.getAMContainerResourceRequests());
+    assertEquals(reqs, asContext.getAMContainerResourceRequests());
     try {
       testRMAppSubmit();
-      Assert.fail("Should have failed due to missing ANY ResourceRequest");
+      fail("Should have failed due to missing ANY ResourceRequest");
     } catch (InvalidResourceRequestException e) {
-      Assert.assertEquals(
+      assertEquals(
           "Invalid resource request, no resource request specified with *",
           e.getMessage());
     }
@@ -1514,12 +1518,12 @@ public class TestAppManager extends AppManagerTestBase{
     asContext.setAMContainerResourceRequests(cloneResourceRequests(reqs));
     // getAMContainerResourceRequest uses the first entry of
     // getAMContainerResourceRequests
-    Assert.assertEquals(reqs, asContext.getAMContainerResourceRequests());
+    assertEquals(reqs, asContext.getAMContainerResourceRequests());
     try {
       testRMAppSubmit();
-      Assert.fail("Should have failed due to too many ANY ResourceRequests");
+      fail("Should have failed due to too many ANY ResourceRequests");
     } catch (InvalidResourceRequestException e) {
-      Assert.assertEquals(
+      assertEquals(
           "Invalid resource request, only one resource request with * is " +
               "allowed", e.getMessage());
     }
@@ -1532,17 +1536,17 @@ public class TestAppManager extends AppManagerTestBase{
 
   private RMApp waitUntilEventProcessed() throws InterruptedException {
     RMApp app = rmContext.getRMApps().get(appId);
-    Assert.assertNotNull("app is null", app);
-    Assert.assertEquals("app id doesn't match", appId, app.getApplicationId());
-    Assert.assertEquals("app state doesn't match", RMAppState.NEW, app.getState());
+    assertNotNull(app, "app is null");
+    assertEquals(appId, app.getApplicationId(), "app id doesn't match");
+    assertEquals(RMAppState.NEW, app.getState(), "app state doesn't match");
     // wait for event to be processed
     int timeoutSecs = 0;
     while ((getAppEventType() == RMAppEventType.KILL) &&
         timeoutSecs++ < 20) {
       Thread.sleep(1000);
     }
-    Assert.assertEquals("app event type sent is wrong", RMAppEventType.START,
-        getAppEventType());
+    assertEquals(RMAppEventType.START, getAppEventType(),
+        "app event type sent is wrong");
     return app;
   }
 
@@ -1559,25 +1563,25 @@ public class TestAppManager extends AppManagerTestBase{
     asContext.getAMContainerSpec().setTokens(securityTokens);
     try {
       appMonitor.submitApplication(asContext, "test");
-      Assert.fail("Application submission should fail because" +
+      fail("Application submission should fail because" +
           " Tokens are invalid.");
     } catch (YarnException e) {
       // Exception is expected
-      assertTrue("The thrown exception is not" +
-          " java.io.EOFException",
-          e.getMessage().contains("java.io.EOFException"));
+      assertTrue(e.getMessage().contains("java.io.EOFException"),
+          "The thrown exception is not java.io.EOFException");
     }
     int timeoutSecs = 0;
     while ((getAppEventType() == RMAppEventType.KILL) &&
         timeoutSecs++ < 20) {
       Thread.sleep(1000);
     }
-    Assert.assertEquals("app event type sent is wrong",
-        RMAppEventType.APP_REJECTED, getAppEventType());
+    assertEquals(RMAppEventType.APP_REJECTED, getAppEventType(),
+        "app event type sent is wrong");
     asContext.getAMContainerSpec().setTokens(null);
   }
 
-  @Test (timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testRMAppSubmitMaxAppAttempts() throws Exception {
     int[] globalMaxAppAttempts = new int[] { 10, 1 };
     int[] rmAmMaxAttempts = new int[] { 8, 1 };
@@ -1607,8 +1611,8 @@ public class TestAppManager extends AppManagerTestBase{
         }
         appMonitor.submitApplication(asContext, "test");
         RMApp app = rmContext.getRMApps().get(appID);
-        Assert.assertEquals("max application attempts doesn't match",
-            expectedNums[i][j], app.getMaxAppAttempts());
+        assertEquals(expectedNums[i][j], app.getMaxAppAttempts(),
+            "max application attempts doesn't match");
 
         // wait for event to be processed
         int timeoutSecs = 0;
@@ -1621,34 +1625,34 @@ public class TestAppManager extends AppManagerTestBase{
     }
   }
 
-  @Test (timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testRMAppSubmitDuplicateApplicationId() throws Exception {
     ApplicationId appId = MockApps.newAppID(0);
     asContext.setApplicationId(appId);
     RMApp appOrig = rmContext.getRMApps().get(appId);
-    assertTrue("app name matches "
-        + "but shouldn't", "testApp1" != appOrig.getName());
+    assertTrue("testApp1" != appOrig.getName(), "app name matches "
+        + "but shouldn't");
 
     // our testApp1 should be rejected and original app with same id should be left in place
     try {
       appMonitor.submitApplication(asContext, "test");
-      Assert.fail("Exception is expected when applicationId is duplicate.");
+      fail("Exception is expected when applicationId is duplicate.");
     } catch (YarnException e) {
-      assertTrue("The thrown exception is not the expectd one.",
-          e.getMessage().contains("Cannot add a duplicate!"));
+      assertTrue(e.getMessage().contains("Cannot add a duplicate!"),
+          "The thrown exception is not the expectd one.");
     }
 
     // make sure original app didn't get removed
     RMApp app = rmContext.getRMApps().get(appId);
-    Assert.assertNotNull("app is null", app);
-    Assert.assertEquals("app id doesn't match",
-        appId, app.getApplicationId());
-    Assert.assertEquals("app state doesn't match",
-        RMAppState.FINISHED, app.getState());
+    assertNotNull(app, "app is null");
+    assertEquals(appId, app.getApplicationId(), "app id doesn't match");
+    assertEquals(RMAppState.FINISHED, app.getState(), "app state doesn't match");
   }
 
   @SuppressWarnings("deprecation")
-  @Test (timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testRMAppSubmitInvalidResourceRequest() throws Exception {
     asContext.setResource(Resources.createResource(
         YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB + 1));
@@ -1656,19 +1660,20 @@ public class TestAppManager extends AppManagerTestBase{
     // submit an app
     try {
       appMonitor.submitApplication(asContext, "test");
-      Assert.fail("Application submission should fail because resource" +
+      fail("Application submission should fail because resource" +
           " request is invalid.");
     } catch (YarnException e) {
       // Exception is expected
       // TODO Change this to assert the expected exception type - post YARN-142
       // sub-task related to specialized exceptions.
-      assertTrue("The thrown exception is not" +
-          " InvalidResourceRequestException",
-          e.getMessage().contains("Invalid resource request"));
+      assertTrue(e.getMessage().contains("Invalid resource request"),
+          "The thrown exception is not" +
+          " InvalidResourceRequestException");
     }
   }
 
-  @Test (timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testEscapeApplicationSummary() {
     RMApp app = mock(RMAppImpl.class);
     ApplicationSubmissionContext asc = mock(ApplicationSubmissionContext.class);
@@ -1709,8 +1714,8 @@ public class TestAppManager extends AppManagerTestBase{
         new RMAppManager.ApplicationSummary().createAppSummary(app);
     String msg = summary.toString();
     LOG.info("summary: " + msg);
-    Assert.assertFalse(msg.contains("\n"));
-    Assert.assertFalse(msg.contains("\r"));
+    assertFalse(msg.contains("\n"));
+    assertFalse(msg.contains("\r"));
 
     String escaped = "\\n\\n\\r\\r";
     assertTrue(msg.contains("Multiline" + escaped +"AppName"));
@@ -1759,16 +1764,16 @@ public class TestAppManager extends AppManagerTestBase{
     event = new RMAppEvent(appId, RMAppEventType.APP_NEW_SAVED);
     rmContext.getRMApps().get(appId).handle(event);
 
-    Assert.assertNotNull("app is null", app);
-    Assert.assertEquals("newQueue", asContext.getQueue());
+    assertNotNull(app, "app is null");
+    assertEquals("newQueue", asContext.getQueue());
 
     // wait for event to be processed
     int timeoutSecs = 0;
     while ((getAppEventType() == RMAppEventType.KILL) && timeoutSecs++ < 20) {
       Thread.sleep(1000);
     }
-    Assert.assertEquals("app event type sent is wrong", RMAppEventType.START,
-        getAppEventType());
+    assertEquals(RMAppEventType.START, getAppEventType(),
+        "app event type sent is wrong");
   }
 
   private static ResourceScheduler mockResourceScheduler() {
@@ -2012,7 +2017,7 @@ public class TestAppManager extends AppManagerTestBase{
             .thenReturn(null);
     String userNameForPlacement = appMonitor
             .getUserNameForPlacement(user, asContext, placementMgr);
-    Assert.assertEquals(expectedUser, userNameForPlacement);
+    assertEquals(expectedUser, userNameForPlacement);
   }
 
   @Test
@@ -2056,8 +2061,8 @@ public class TestAppManager extends AppManagerTestBase{
     verify(scheduler).checkAccess(any(UserGroupInformation.class),
         any(QueueACL.class), queueNameCaptor.capture());
 
-    assertEquals("Expected access check for queue",
-        expectedQueue, queueNameCaptor.getValue());
+    assertEquals(expectedQueue, queueNameCaptor.getValue(),
+        "Expected access check for queue");
   }
 
   private void enableApplicationTagPlacement(boolean userHasAccessToQueue,
@@ -2090,7 +2095,7 @@ public class TestAppManager extends AppManagerTestBase{
             .thenReturn(appContext);
     String userNameForPlacement = appMonitor
             .getUserNameForPlacement(submittingUser, asContext, placementMgr);
-    Assert.assertEquals(expectedUser, userNameForPlacement);
+    assertEquals(expectedUser, userNameForPlacement);
   }
 
   private void setApplicationTags(String... tags) {
@@ -2099,13 +2104,13 @@ public class TestAppManager extends AppManagerTestBase{
     asContext.setApplicationTags(applicationTags);
   }
 
-  private class UseCapacitySchedulerRule extends TestWatcher {
+  private class UseCapacitySchedulerRule implements BeforeEachCallback {
     private boolean useCapacityScheduler;
 
     @Override
-    protected void starting(Description d) {
-      useCapacityScheduler =
-          d.getAnnotation(UseMockCapacityScheduler.class) != null;
+    public void beforeEach(ExtensionContext context) throws Exception {
+      Method testMethod = context.getRequiredTestMethod();
+      useCapacityScheduler = testMethod.getAnnotation(UseMockCapacityScheduler.class) != null;
     }
 
     public boolean useCapacityScheduler() {

+ 16 - 15
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManagerWithFairScheduler.java

@@ -18,7 +18,9 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
-import static junit.framework.TestCase.assertTrue;
+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.apache.hadoop.yarn.exceptions.InvalidResourceRequestException.InvalidResourceType.GREATER_THEN_MAX_ALLOCATION;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
@@ -55,10 +57,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSec
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.util.Records;
 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;
 
 /**
  * Testing RMAppManager application submission with fair scheduler.
@@ -74,7 +75,7 @@ public class TestAppManagerWithFairScheduler extends AppManagerTestBase {
   private static String allocFileName =
       GenericTestUtils.getTestDir(TEST_FOLDER).getAbsolutePath();
 
-  @Before
+  @BeforeEach
   public void setup() throws IOException {
     // Basic config with one queue (override in test if needed)
     AllocationFileWriter.create()
@@ -98,7 +99,7 @@ public class TestAppManagerWithFairScheduler extends AppManagerTestBase {
         masterService, new ApplicationACLsManager(conf), conf);
   }
 
-  @After
+  @AfterEach
   public void teardown(){
     File allocFile = GenericTestUtils.getTestDir(TEST_FOLDER);
     allocFile.delete();
@@ -131,9 +132,9 @@ public class TestAppManagerWithFairScheduler extends AppManagerTestBase {
         .thenReturn(new ApplicationPlacementContext("limited"));
     try {
       rmAppManager.submitApplication(asContext, "test");
-      Assert.fail("Test should fail on too high allocation!");
+      fail("Test should fail on too high allocation!");
     } catch (InvalidResourceRequestException e) {
-      Assert.assertEquals(GREATER_THEN_MAX_ALLOCATION,
+      assertEquals(GREATER_THEN_MAX_ALLOCATION,
           e.getInvalidResourceType());
     }
 
@@ -176,10 +177,10 @@ public class TestAppManagerWithFairScheduler extends AppManagerTestBase {
         .thenReturn(new ApplicationPlacementContext("noaccess"));
     try {
       rmAppManager.submitApplication(asContext, "test");
-      Assert.fail("Test should have failed with access denied");
+      fail("Test should have failed with access denied");
     } catch (YarnException e) {
-      assertTrue("Access exception not found",
-          e.getCause() instanceof AccessControlException);
+      assertTrue(e.getCause() instanceof AccessControlException,
+          "Access exception not found");
     }
     // Submit to submit access queue
     when(placementMgr.placeApplication(any(), any(), any(Boolean.class)))
@@ -249,10 +250,10 @@ public class TestAppManagerWithFairScheduler extends AppManagerTestBase {
         .thenReturn(new ApplicationPlacementContext("root.noaccess.child"));
     try {
       rmAppManager.submitApplication(asContext, "test");
-      Assert.fail("Test should have failed with access denied");
+      fail("Test should have failed with access denied");
     } catch (YarnException e) {
-      assertTrue("Access exception not found",
-          e.getCause() instanceof AccessControlException);
+      assertTrue(e.getCause() instanceof AccessControlException,
+          "Access exception not found");
     }
     // Submit to submitonly parent with non existent child queue
     when(placementMgr.placeApplication(any(), any(), any(Boolean.class)))

+ 31 - 22
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java

@@ -18,6 +18,10 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -51,9 +55,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 public class TestApplicationCleanup {
 
@@ -62,14 +66,14 @@ public class TestApplicationCleanup {
   
   private YarnConfiguration conf;
   
-  @Before
+  @BeforeEach
   public void setup() throws UnknownHostException {
     GenericTestUtils.setRootLogLevel(Level.DEBUG);
     conf = new YarnConfiguration();
     UserGroupInformation.setConfiguration(conf);
     conf.set(YarnConfiguration.RECOVERY_ENABLED, "true");
     conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
-    Assert.assertTrue(YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS > 1);
+    assertTrue(YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS > 1);
   }
 
   @SuppressWarnings("resource")
@@ -110,7 +114,7 @@ public class TestApplicationCleanup {
       contReceived += conts.size();
       nm1.nodeHeartbeat(true);
     }
-    Assert.assertEquals(request, contReceived);
+    assertEquals(request, contReceived);
     
     am.unregisterAppAttempt();
     NodeHeartbeatResponse resp = nm1.nodeHeartbeat(attempt.getAppAttemptId(), 1,
@@ -142,10 +146,10 @@ public class TestApplicationCleanup {
       numCleanedApps = appsToCleanup.size();
     }
     
-    Assert.assertEquals(1, appsToCleanup.size());
-    Assert.assertEquals(app.getApplicationId(), appsToCleanup.get(0));
-    Assert.assertEquals(1, numCleanedApps);
-    Assert.assertEquals(2, numCleanedContainers);
+    assertEquals(1, appsToCleanup.size());
+    assertEquals(app.getApplicationId(), appsToCleanup.get(0));
+    assertEquals(1, numCleanedApps);
+    assertEquals(2, numCleanedContainers);
 
     rm.stop();
   }
@@ -190,7 +194,7 @@ public class TestApplicationCleanup {
       contReceived += conts.size();
       nm1.nodeHeartbeat(true);
     }
-    Assert.assertEquals(request, contReceived);
+    assertEquals(request, contReceived);
 
     // Release a container.
     ArrayList<ContainerId> release = new ArrayList<ContainerId>();
@@ -252,7 +256,7 @@ public class TestApplicationCleanup {
     } else {
       LOG.info("Got cleanup for " + contsToClean.get(0));
     }
-    Assert.assertEquals(1, cleanedConts);
+    assertEquals(1, cleanedConts);
   }
 
   private void waitForAppCleanupMessageRecved(MockNM nm, ApplicationId appId)
@@ -283,7 +287,8 @@ public class TestApplicationCleanup {
   }
   
   @SuppressWarnings("resource")
-  @Test (timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testAppCleanupWhenRMRestartedAfterAppFinished() throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
     // start RM
@@ -318,7 +323,8 @@ public class TestApplicationCleanup {
   }
   
   @SuppressWarnings("resource")
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testAppCleanupWhenRMRestartedBeforeAppFinished() throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
 
@@ -375,7 +381,8 @@ public class TestApplicationCleanup {
     rm2.stop();
   }
 
-  @Test (timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testContainerCleanupWhenRMRestartedAppNotRegistered() throws
       Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
@@ -411,7 +418,8 @@ public class TestApplicationCleanup {
     rm2.stop();
   }
 
-  @Test (timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testAppCleanupWhenNMReconnects() throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
 
@@ -449,7 +457,8 @@ public class TestApplicationCleanup {
   // 6. Verify for Memory Used, it should be 1024
   // 7. Send AM heatbeat to RM. Allocated response should contain completed
   // container.
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testProcessingNMContainerStatusesOnNMRestart() throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
 
@@ -472,7 +481,7 @@ public class TestApplicationCleanup {
         am0.allocateAndWaitForContainers(noOfContainers, containerMemory, nm1);
 
     // 3. Verify for number of container allocated by RM
-    Assert.assertEquals(noOfContainers, allocateContainers.size());
+    assertEquals(noOfContainers, allocateContainers.size());
     Container container = allocateContainers.get(0);
 
     nm1.nodeHeartbeat(am0.getApplicationAttemptId(), 1, ContainerState.RUNNING);
@@ -485,7 +494,7 @@ public class TestApplicationCleanup {
     // requested memory. 1024 + 2048=3072
     ResourceScheduler rs = rm1.getRMContext().getScheduler();
     long allocatedMB = rs.getRootQueueMetrics().getAllocatedMB();
-    Assert.assertEquals(amMemory + containerMemory, allocatedMB);
+    assertEquals(amMemory + containerMemory, allocatedMB);
 
     // 5. Re-register NM by sending completed container status
     List<NMContainerStatus> nMContainerStatusForApp =
@@ -496,7 +505,7 @@ public class TestApplicationCleanup {
     waitForClusterMemory(nm1, rs, amMemory);
 
     // 6. Verify for Memory Used, it should be 1024
-    Assert.assertEquals(amMemory, rs.getRootQueueMetrics().getAllocatedMB());
+    assertEquals(amMemory, rs.getRootQueueMetrics().getAllocatedMB());
 
     // 7. Send AM heatbeat to RM. Allocated response should contain completed
     // container
@@ -506,7 +515,7 @@ public class TestApplicationCleanup {
     AllocateResponse allocate = am0.allocate(req);
     List<ContainerStatus> completedContainersStatuses =
         allocate.getCompletedContainersStatuses();
-    Assert.assertEquals(noOfContainers, completedContainersStatuses.size());
+    assertEquals(noOfContainers, completedContainersStatuses.size());
 
     // Application clean up should happen Cluster memory used is 0
     nm1.nodeHeartbeat(am0.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
@@ -523,7 +532,7 @@ public class TestApplicationCleanup {
 
       Thread.sleep(100);
       if (counter++ == 50) {
-        Assert.fail("Wait for cluster memory is timed out.Expected="
+        fail("Wait for cluster memory is timed out.Expected="
             + clusterMemory + " Actual="
             + rs.getRootQueueMetrics().getAllocatedMB());
       }

+ 36 - 31
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java

@@ -89,12 +89,16 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.ProxyCAManager;
 import org.apache.hadoop.yarn.server.security.AMSecretKeys;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.server.webproxy.ProxyCA;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import java.util.function.Supplier;
 
-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.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
@@ -244,20 +248,20 @@ public class TestApplicationMasterLauncher {
     } catch (TimeoutException e) {
       fail("timed out while waiting for AM Launch to happen.");
     }
-    Assert.assertTrue(containerManager.launched);
+    assertTrue(containerManager.launched);
 
     RMAppAttempt attempt = app.getCurrentAppAttempt();
     ApplicationAttemptId appAttemptId = attempt.getAppAttemptId();
-    Assert.assertEquals(appAttemptId.toString(),
+    assertEquals(appAttemptId.toString(),
         containerManager.attemptIdAtContainerManager);
-    Assert.assertEquals(app.getSubmitTime(),
+    assertEquals(app.getSubmitTime(),
         containerManager.submitTimeAtContainerManager);
-    Assert.assertEquals(app.getRMAppAttempt(appAttemptId)
+    assertEquals(app.getRMAppAttempt(appAttemptId)
         .getMasterContainer().getId()
         .toString(), containerManager.containerIdAtContainerManager);
-    Assert.assertEquals(nm1.getNodeId().toString(),
+    assertEquals(nm1.getNodeId().toString(),
         containerManager.nmHostAtContainerManager);
-    Assert.assertEquals(YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS,
+    assertEquals(YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS,
         containerManager.maxAppAttempts);
 
     MockAM am = new MockAM(rm.getRMContext(), rm
@@ -278,7 +282,7 @@ public class TestApplicationMasterLauncher {
     } catch (TimeoutException e) {
       fail("timed out while waiting for AM cleanup to happen.");
     }
-    Assert.assertTrue(containerManager.cleanedup);
+    assertTrue(containerManager.cleanedup);
 
     rm.waitForState(am.getApplicationAttemptId(), RMAppAttemptState.FINISHED);
     rm.stop();
@@ -312,9 +316,9 @@ public class TestApplicationMasterLauncher {
             attempt, AMLauncherEventType.LAUNCH, rm.getConfig()) {
         @Override
         public void onAMLaunchFailed(ContainerId containerId, Exception e) {
-          Assert.assertFalse("NullPointerException happens "
-                 + " while launching " + containerId,
-                   e instanceof NullPointerException);
+          assertFalse(e instanceof NullPointerException,
+              "NullPointerException happens "
+              + " while launching " + containerId);
         }
         @Override
         protected ContainerManagementProtocol getContainerMgrProxy(
@@ -378,7 +382,8 @@ public class TestApplicationMasterLauncher {
 
 
   @SuppressWarnings("unused")
-  @Test(timeout = 100000)
+  @Test
+  @Timeout(value = 100)
   public void testallocateBeforeAMRegistration() throws Exception {
     boolean thrown = false;
     GenericTestUtils.setRootLogLevel(Level.DEBUG);
@@ -396,7 +401,7 @@ public class TestApplicationMasterLauncher {
     AllocateResponse ar = null;
     try {
       ar = am.allocate("h1", 1000, request, new ArrayList<ContainerId>());
-      Assert.fail();
+      fail();
     } catch (ApplicationMasterNotRegisteredException e) {
     }
 
@@ -407,16 +412,16 @@ public class TestApplicationMasterLauncher {
     try {
       amrs = am.allocate(new ArrayList<ResourceRequest>(),
           new ArrayList<ContainerId>());
-      Assert.fail();
+      fail();
     } catch (ApplicationMasterNotRegisteredException e) {
     }
 
     am.registerAppAttempt();
     try {
       am.registerAppAttempt(false);
-      Assert.fail();
+      fail();
     } catch (Exception e) {
-      Assert.assertEquals(AMRMClientUtils.APP_ALREADY_REGISTERED_MESSAGE
+      assertEquals(AMRMClientUtils.APP_ALREADY_REGISTERED_MESSAGE
           + attempt.getAppAttemptId().getApplicationId(), e.getMessage());
     }
 
@@ -430,7 +435,7 @@ public class TestApplicationMasterLauncher {
     try {
       amrs = am.allocate(new ArrayList<ResourceRequest>(),
         new ArrayList<ContainerId>());
-      Assert.fail();
+      fail();
     } catch (ApplicationAttemptNotFoundException e) {
     }
   }
@@ -477,7 +482,7 @@ public class TestApplicationMasterLauncher {
       fail("timed out while waiting for AM Launch to happen.");
     }
 
-    Assert.assertEquals(
+    assertEquals(
         app.getCurrentAppAttempt().getMasterContainer().getNodeId().getHost(),
         host);
 
@@ -531,7 +536,7 @@ public class TestApplicationMasterLauncher {
     try {
       launcher.setupTokens(amContainer, containerId);
     } catch (java.io.EOFException e) {
-      Assert.fail("EOFException should not happen.");
+      fail("EOFException should not happen.");
     }
 
     // verify token
@@ -539,30 +544,30 @@ public class TestApplicationMasterLauncher {
     dibb.reset(amContainer.getTokens());
     Credentials credentials = new Credentials();
     credentials.readTokenStorageStream(dibb);
-    Assert.assertEquals(1, credentials.numberOfTokens());
+    assertEquals(1, credentials.numberOfTokens());
     org.apache.hadoop.security.token.Token<? extends TokenIdentifier> token =
         credentials.getAllTokens().iterator().next();
-    Assert.assertEquals(tokenIdentifier.getKind(), token.getKind());
-    Assert.assertArrayEquals(tokenIdentifier.getBytes(), token.getIdentifier());
-    Assert.assertArrayEquals("password".getBytes(), token.getPassword());
+    assertEquals(tokenIdentifier.getKind(), token.getKind());
+    assertArrayEquals(tokenIdentifier.getBytes(), token.getIdentifier());
+    assertArrayEquals("password".getBytes(), token.getPassword());
 
     // verify keystore and truststore
     if (https) {
-      Assert.assertEquals(4, credentials.numberOfSecretKeys());
-      Assert.assertArrayEquals("keystore".getBytes(),
+      assertEquals(4, credentials.numberOfSecretKeys());
+      assertArrayEquals("keystore".getBytes(),
           credentials.getSecretKey(
               AMSecretKeys.YARN_APPLICATION_AM_KEYSTORE));
-      Assert.assertArrayEquals("kPassword".getBytes(),
+      assertArrayEquals("kPassword".getBytes(),
           credentials.getSecretKey(
               AMSecretKeys.YARN_APPLICATION_AM_KEYSTORE_PASSWORD));
-      Assert.assertArrayEquals("truststore".getBytes(),
+      assertArrayEquals("truststore".getBytes(),
           credentials.getSecretKey(
               AMSecretKeys.YARN_APPLICATION_AM_TRUSTSTORE));
-      Assert.assertArrayEquals("tPassword".getBytes(),
+      assertArrayEquals("tPassword".getBytes(),
           credentials.getSecretKey(
               AMSecretKeys.YARN_APPLICATION_AM_TRUSTSTORE_PASSWORD));
     } else {
-      Assert.assertEquals(0, credentials.numberOfSecretKeys());
+      assertEquals(0, credentials.numberOfSecretKeys());
     }
   }
 

+ 23 - 18
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceCapacity.java

@@ -47,8 +47,8 @@ import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -57,7 +57,8 @@ import java.util.List;
 import java.util.Set;
 
 import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils.toSet;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
 
 /**
  * Unit tests for {@link ApplicationMasterService}
@@ -103,7 +104,8 @@ public class TestApplicationMasterServiceCapacity extends
     }
   }
 
-  @Test(timeout=60000)
+  @Test
+  @Timeout(value = 60)
   public void testInvalidIncreaseDecreaseRequest() throws Exception {
     conf = new YarnConfiguration();
     conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
@@ -142,8 +144,8 @@ public class TestApplicationMasterServiceCapacity extends
                   ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
                   ContainerUpdateType.INCREASE_RESOURCE,
                   Resources.createResource(-1), null)));
-      Assert.assertEquals(1, response.getUpdateErrors().size());
-      Assert.assertEquals("RESOURCE_OUTSIDE_ALLOWED_RANGE",
+      assertEquals(1, response.getUpdateErrors().size());
+      assertEquals("RESOURCE_OUTSIDE_ALLOWED_RANGE",
           response.getUpdateErrors().get(0).getReason());
 
       // Target resource is more than maxAllocation, should fail
@@ -154,8 +156,8 @@ public class TestApplicationMasterServiceCapacity extends
               Resources.add(
                   registerResponse.getMaximumResourceCapability(),
                   Resources.createResource(1)), null)));
-      Assert.assertEquals(1, response.getUpdateErrors().size());
-      Assert.assertEquals("RESOURCE_OUTSIDE_ALLOWED_RANGE",
+      assertEquals(1, response.getUpdateErrors().size());
+      assertEquals("RESOURCE_OUTSIDE_ALLOWED_RANGE",
           response.getUpdateErrors().get(0).getReason());
 
       // Contains multiple increase/decrease requests for same containerId
@@ -168,13 +170,14 @@ public class TestApplicationMasterServiceCapacity extends
               ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
               ContainerUpdateType.DECREASE_RESOURCE,
               Resources.createResource(1024, 1), null)));
-      Assert.assertEquals(1, response.getUpdateErrors().size());
-      Assert.assertEquals("UPDATE_OUTSTANDING_ERROR",
+      assertEquals(1, response.getUpdateErrors().size());
+      assertEquals("UPDATE_OUTSTANDING_ERROR",
           response.getUpdateErrors().get(0).getReason());
     }
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testPriorityInAllocatedResponse() throws Exception {
     conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
         ResourceScheduler.class);
@@ -206,7 +209,7 @@ public class TestApplicationMasterServiceCapacity extends
     allocateRequest.setAskList(ask);
 
     AllocateResponse response1 = am1.allocate(allocateRequest);
-    Assert.assertEquals(appPriority1, response1.getApplicationPriority());
+    assertEquals(appPriority1, response1.getApplicationPriority());
 
     // Change the priority of App1 to 8
     Priority appPriority2 = Priority.newInstance(8);
@@ -216,11 +219,12 @@ public class TestApplicationMasterServiceCapacity extends
         appPriority2);
 
     AllocateResponse response2 = am1.allocate(allocateRequest);
-    Assert.assertEquals(appPriority2, response2.getApplicationPriority());
+    assertEquals(appPriority2, response2.getApplicationPriority());
     rm.stop();
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testGetNMNumInAllocatedResponseWithOutNodeLabel() throws Exception {
     conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, ResourceScheduler.class);
     MockRM rm = new MockRM(conf);
@@ -252,7 +256,7 @@ public class TestApplicationMasterServiceCapacity extends
     allocateRequest.setAskList(ask);
 
     AllocateResponse response1 = am1.allocate(allocateRequest);
-    Assert.assertEquals(3, response1.getNumClusterNodes());
+    assertEquals(3, response1.getNumClusterNodes());
 
     rm.stop();
   }
@@ -285,7 +289,8 @@ public class TestApplicationMasterServiceCapacity extends
     return conf;
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testGetNMNumInAllocatedResponseWithNodeLabel() throws Exception {
     conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, ResourceScheduler.class);
     conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
@@ -368,10 +373,10 @@ public class TestApplicationMasterServiceCapacity extends
     }
 
     //has 3 nodes with node label "x"
-    Assert.assertEquals(3, response1.getNumClusterNodes());
+    assertEquals(3, response1.getNumClusterNodes());
 
     //has 1 node with node label "y"
-    Assert.assertEquals(1, response2.getNumClusterNodes());
+    assertEquals(1, response2.getNumClusterNodes());
 
     rm.stop();
   }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceFair.java

@@ -30,14 +30,14 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSLeafQueue;
 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.util.resource.ResourceUtils;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.util.Collections;
 import java.util.Map;
 
 import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB;
 import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.fail;
 
 /**
  * Unit tests for {@link ApplicationMasterService} with {@link FairScheduler}.

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

@@ -16,6 +16,8 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.yarn.ams.ApplicationMasterServiceContext;
@@ -37,9 +39,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler
     .ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import java.io.IOException;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -152,14 +154,15 @@ public class TestApplicationMasterServiceInterceptor {
   private static YarnConfiguration conf;
   private static final int GB = 1024;
 
-  @Before
+  @BeforeEach
   public void setup() {
     conf = new YarnConfiguration();
     conf.setClass(YarnConfiguration.RM_SCHEDULER, FifoScheduler.class,
         ResourceScheduler.class);
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testApplicationMasterInterceptor() throws Exception {
     conf.set(YarnConfiguration.RM_APPLICATION_MASTER_SERVICE_PROCESSORS,
         TestInterceptor1.class.getName() + ","
@@ -201,17 +204,17 @@ public class TestApplicationMasterServiceInterceptor {
             .getContainerToken());
     am1.unregisterAppAttempt();
 
-    Assert.assertEquals(1, beforeRegCount.get());
-    Assert.assertEquals(1, afterRegCount.get());
+    assertEquals(1, beforeRegCount.get());
+    assertEquals(1, afterRegCount.get());
 
     // The allocate calls should be incremented twice
-    Assert.assertEquals(allocCount * 2, beforeAllocCount.get());
-    Assert.assertEquals(allocCount * 2, afterAllocCount.get());
+    assertEquals(allocCount * 2, beforeAllocCount.get());
+    assertEquals(allocCount * 2, afterAllocCount.get());
 
     // Finish should only be called once, since the FirstInterceptor
     // does not forward the call.
-    Assert.assertEquals(1, beforeFinishCount.get());
-    Assert.assertEquals(1, afterFinishCount.get());
+    assertEquals(1, beforeFinishCount.get());
+    assertEquals(1, afterFinishCount.get());
     rm.stop();
   }
 }

+ 14 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestCapacitySchedulerMetrics.java

@@ -18,6 +18,11 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+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 org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
@@ -27,9 +32,8 @@ 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.server.resourcemanager.scheduler.capacity.CapacitySchedulerMetrics;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
 
 import java.util.ArrayList;
 import java.util.concurrent.TimeoutException;
@@ -67,7 +71,7 @@ public class TestCapacitySchedulerMetrics {
     nm2.nodeHeartbeat(true);
 
     CapacitySchedulerMetrics csMetrics = CapacitySchedulerMetrics.getMetrics();
-    Assert.assertNotNull(csMetrics);
+    assertNotNull(csMetrics);
     try {
       GenericTestUtils.waitFor(()
           -> csMetrics.getNumOfNodeUpdate() == 2, 100, 3000);
@@ -75,11 +79,11 @@ public class TestCapacitySchedulerMetrics {
           .waitFor(() -> csMetrics.getNumOfSchedulerNodeHBInterval() == 2,
               100, 3000);
     } catch(TimeoutException e) {
-      Assert.fail("CS metrics not updated on node-update events.");
+      fail("CS metrics not updated on node-update events.");
     }
 
-    Assert.assertEquals(0, csMetrics.getNumOfAllocates());
-    Assert.assertEquals(0, csMetrics.getNumOfCommitSuccess());
+    assertEquals(0, csMetrics.getNumOfAllocates());
+    assertEquals(0, csMetrics.getNumOfCommitSuccess());
 
     RMApp rmApp = MockRMAppSubmitter.submit(rm,
         MockRMAppSubmissionData.Builder.createWithMemory(1024, rm)
@@ -108,16 +112,16 @@ public class TestCapacitySchedulerMetrics {
           .waitFor(() -> csMetrics.getNumOfSchedulerNodeHBInterval() == 4,
               100, 3000);
       // For async mode, the number of alloc might be bigger than 1
-      Assert.assertTrue(csMetrics.getNumOfAllocates() > 0);
+      assertTrue(csMetrics.getNumOfAllocates() > 0);
       // But there will be only 2 successful commit (1 AM + 1 task)
       GenericTestUtils.waitFor(()
           -> csMetrics.getNumOfCommitSuccess() == 2, 100, 3000);
     } catch(TimeoutException e) {
-      Assert.fail("CS metrics not updated on node-update events.");
+      fail("CS metrics not updated on node-update events.");
     }
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
     if (rm != null) {
       rm.stop();

File diff suppressed because it is too large
+ 258 - 243
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java


+ 10 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java

@@ -17,8 +17,8 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyString;
 import static org.mockito.ArgumentMatchers.eq;
@@ -41,8 +41,7 @@ import org.apache.hadoop.thirdparty.protobuf.InvalidProtocolBufferException;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.RMDelegationTokenIdentifierData;
-import org.junit.AfterClass;
-import org.junit.Assert;
+import org.junit.jupiter.api.AfterAll;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -78,9 +77,9 @@ import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.resource.Resources;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 
 
 public class TestClientRMTokens {
@@ -91,7 +90,7 @@ public class TestClientRMTokens {
   // Note : Any test case in ResourceManager package that creates a proxy has
   // to be run with enabling hadoop.security.token.service.use_ip. And reset
   // to false at the end of test class. See YARN-5208
-  @BeforeClass
+  @BeforeAll
   public static void setUp() {
     Configuration conf = new Configuration();
     conf.setBoolean(
@@ -99,7 +98,7 @@ public class TestClientRMTokens {
     SecurityUtil.setConfiguration(conf);
   }
 
-  @AfterClass
+  @AfterAll
   public static void tearDown() {
     Configuration conf = new Configuration();
     conf.setBoolean(
@@ -107,7 +106,7 @@ public class TestClientRMTokens {
     SecurityUtil.setConfiguration(conf);
   }
 
-  @Before
+  @BeforeEach
   public void resetSecretManager() {
     RMDelegationTokenIdentifier.Renewer.setSecretManager(null, null);
   }
@@ -149,7 +148,7 @@ public class TestClientRMTokens {
       // Create a user for the renewr and fake the authentication-method
       UserGroupInformation loggedInUser = UserGroupInformation
           .createRemoteUser("testrenewer@APACHE.ORG");
-      Assert.assertEquals("testrenewer", loggedInUser.getShortUserName());
+      assertEquals("testrenewer", loggedInUser.getShortUserName());
       // Default realm is APACHE.ORG
       loggedInUser.setAuthenticationMethod(AuthenticationMethod.KERBEROS);
 

+ 14 - 13
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClusterMetrics.java

@@ -18,13 +18,14 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
 import org.apache.hadoop.metrics2.MetricsSystem;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Assert;
-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.function.Supplier;
 
@@ -40,24 +41,24 @@ public class TestClusterMetrics {
   @Test
   public void testAmMetrics() throws Exception {
     assert(metrics != null);
-    Assert.assertTrue(!metrics.aMLaunchDelay.changed());
-    Assert.assertTrue(!metrics.aMRegisterDelay.changed());
-    Assert.assertTrue(!metrics.getAMContainerAllocationDelay().changed());
+    assertTrue(!metrics.aMLaunchDelay.changed());
+    assertTrue(!metrics.aMRegisterDelay.changed());
+    assertTrue(!metrics.getAMContainerAllocationDelay().changed());
     metrics.addAMLaunchDelay(1);
     metrics.addAMRegisterDelay(1);
     metrics.addAMContainerAllocationDelay(1);
-    Assert.assertTrue(metrics.aMLaunchDelay.changed());
-    Assert.assertTrue(metrics.aMRegisterDelay.changed());
-    Assert.assertTrue(metrics.getAMContainerAllocationDelay().changed());
+    assertTrue(metrics.aMLaunchDelay.changed());
+    assertTrue(metrics.aMRegisterDelay.changed());
+    assertTrue(metrics.getAMContainerAllocationDelay().changed());
   }
 
-  @Before
+  @BeforeEach
   public void setup() {
     DefaultMetricsSystem.initialize("ResourceManager");
     metrics = ClusterMetrics.getMetrics();
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
     ClusterMetrics.destroy();
 
@@ -69,7 +70,7 @@ public class TestClusterMetrics {
 
   @Test
   public void testClusterMetrics() throws Exception {
-    Assert.assertTrue(!metrics.containerAssignedPerSecond.changed());
+    assertTrue(!metrics.containerAssignedPerSecond.changed());
     metrics.incrNumContainerAssigned();
     metrics.incrNumContainerAssigned();
     GenericTestUtils.waitFor(new Supplier<Boolean>() {

+ 52 - 47
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestContainerResourceUsage.java

@@ -18,6 +18,10 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+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.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -47,16 +51,16 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AggregateAppR
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.slf4j.event.Level;
-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;
 
 public class TestContainerResourceUsage {
 
   private YarnConfiguration conf;
 
-  @Before
+  @BeforeEach
   public void setup() throws UnknownHostException {
     GenericTestUtils.setRootLogLevel(Level.DEBUG);
     conf = new YarnConfiguration();
@@ -65,11 +69,12 @@ public class TestContainerResourceUsage {
         YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
   }
 
-  @Test (timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testUsageWithOneAttemptAndOneContainer() throws Exception {
     MockRM rm = new MockRM(conf);
     rm.start();
@@ -81,14 +86,12 @@ public class TestContainerResourceUsage {
     RMApp app0 = MockRMAppSubmitter.submitWithMemory(200, rm);
 
     RMAppMetrics rmAppMetrics = app0.getRMAppMetrics();
-    Assert.assertTrue(
+    assertTrue(rmAppMetrics.getMemorySeconds() == 0,
         "Before app submittion, memory seconds should have been 0 but was "
-                          + rmAppMetrics.getMemorySeconds(),
-        rmAppMetrics.getMemorySeconds() == 0);
-    Assert.assertTrue(
+        + rmAppMetrics.getMemorySeconds());
+    assertTrue(rmAppMetrics.getVcoreSeconds() == 0,
         "Before app submission, vcore seconds should have been 0 but was "
-                          + rmAppMetrics.getVcoreSeconds(),
-        rmAppMetrics.getVcoreSeconds() == 0);
+        + rmAppMetrics.getVcoreSeconds());
 
     RMAppAttempt attempt0 = app0.getCurrentAppAttempt();
 
@@ -109,29 +112,28 @@ public class TestContainerResourceUsage {
     }
 
     rmAppMetrics = app0.getRMAppMetrics();
-    Assert.assertTrue(
+    assertTrue(rmAppMetrics.getMemorySeconds() > 0,
         "While app is running, memory seconds should be >0 but is "
-            + rmAppMetrics.getMemorySeconds(),
-        rmAppMetrics.getMemorySeconds() > 0);
-    Assert.assertTrue(
+        + rmAppMetrics.getMemorySeconds());
+    assertTrue(rmAppMetrics.getVcoreSeconds() > 0,
         "While app is running, vcore seconds should be >0 but is "
-            + rmAppMetrics.getVcoreSeconds(),
-        rmAppMetrics.getVcoreSeconds() > 0);
+        + rmAppMetrics.getVcoreSeconds());
 
     MockRM.finishAMAndVerifyAppState(app0, rm, nm, am0);
 
     AggregateAppResourceUsage ru = calculateContainerResourceMetrics(rmContainer);
     rmAppMetrics = app0.getRMAppMetrics();
 
-    Assert.assertEquals("Unexpected MemorySeconds value",
-        ru.getMemorySeconds(), rmAppMetrics.getMemorySeconds());
-    Assert.assertEquals("Unexpected VcoreSeconds value",
-        ru.getVcoreSeconds(), rmAppMetrics.getVcoreSeconds());
+    assertEquals(ru.getMemorySeconds(), rmAppMetrics.getMemorySeconds(),
+        "Unexpected MemorySeconds value");
+    assertEquals(ru.getVcoreSeconds(), rmAppMetrics.getVcoreSeconds(),
+        "Unexpected VcoreSeconds value");
 
     rm.stop();
   }
 
-  @Test (timeout = 120000)
+  @Test
+  @Timeout(value = 120)
   public void testUsageWithMultipleContainersAndRMRestart() throws Exception {
     // Set max attempts to 1 so that when the first attempt fails, the app
     // won't try to start a new one.
@@ -223,12 +225,12 @@ public class TestContainerResourceUsage {
     }
 
     RMAppMetrics metricsBefore = app0.getRMAppMetrics();
-    Assert.assertEquals("Unexpected MemorySeconds value",
-        memorySeconds, metricsBefore.getMemorySeconds());
-    Assert.assertEquals("Unexpected VcoreSeconds value",
-        vcoreSeconds, metricsBefore.getVcoreSeconds());
-    Assert.assertEquals("Unexpected totalAllocatedContainers value",
-        NUM_CONTAINERS + 1, metricsBefore.getTotalAllocatedContainers());
+    assertEquals(memorySeconds, metricsBefore.getMemorySeconds(),
+        "Unexpected MemorySeconds value");
+    assertEquals(vcoreSeconds, metricsBefore.getVcoreSeconds(),
+        "Unexpected VcoreSeconds value");
+    assertEquals(NUM_CONTAINERS + 1, metricsBefore.getTotalAllocatedContainers(),
+        "Unexpected totalAllocatedContainers value");
 
     // create new RM to represent RM restart. Load up the state store.
     MockRM rm1 = new MockRM(conf, memStore);
@@ -238,13 +240,14 @@ public class TestContainerResourceUsage {
 
     // Compare container resource usage metrics from before and after restart.
     RMAppMetrics metricsAfter = app0After.getRMAppMetrics();
-    Assert.assertEquals("Vcore seconds were not the same after RM Restart",
-        metricsBefore.getVcoreSeconds(), metricsAfter.getVcoreSeconds());
-    Assert.assertEquals("Memory seconds were not the same after RM Restart",
-        metricsBefore.getMemorySeconds(), metricsAfter.getMemorySeconds());
-    Assert.assertEquals("TotalAllocatedContainers was not the same after " +
-        "RM Restart", metricsBefore.getTotalAllocatedContainers(),
-        metricsAfter.getTotalAllocatedContainers());
+    assertEquals(metricsBefore.getVcoreSeconds(), metricsAfter.getVcoreSeconds(),
+        "Vcore seconds were not the same after RM Restart");
+    assertEquals(metricsBefore.getMemorySeconds(), metricsAfter.getMemorySeconds(),
+        "Memory seconds were not the same after RM Restart");
+    assertEquals(metricsBefore.getTotalAllocatedContainers(),
+        metricsAfter.getTotalAllocatedContainers(),
+        "TotalAllocatedContainers was not the same after " +
+        "RM Restart");
 
     rm0.stop();
     rm0.close();
@@ -252,12 +255,14 @@ public class TestContainerResourceUsage {
     rm1.close();
   }
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testUsageAfterAMRestartWithMultipleContainers() throws Exception {
     amRestartTests(false);
   }
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testUsageAfterAMRestartKeepContainers() throws Exception {
     amRestartTests(true);
   }
@@ -339,9 +344,9 @@ public class TestContainerResourceUsage {
           vcoreSeconds += ru.getVcoreSeconds();
         } else {
           // The remaining container should be RUNNING.
-          Assert.assertTrue("After first attempt failed, remaining container "
-                        + "should still be running. ",
-                        c.getContainerState().equals(ContainerState.RUNNING));
+          assertTrue(c.getContainerState().equals(ContainerState.RUNNING),
+              "After first attempt failed, remaining container "
+              + "should still be running. ");
         }
       }
     } else {
@@ -360,7 +365,7 @@ public class TestContainerResourceUsage {
 
     // assert this is a new AM.
     RMAppAttempt attempt2 = app.getCurrentAppAttempt();
-    Assert.assertFalse(attempt2.getAppAttemptId()
+    assertFalse(attempt2.getAppAttemptId()
                                .equals(am0.getApplicationAttemptId()));
 
     rm.waitForState(attempt2.getAppAttemptId(), RMAppAttemptState.SCHEDULED);
@@ -408,10 +413,10 @@ public class TestContainerResourceUsage {
     
     RMAppMetrics rmAppMetrics = app.getRMAppMetrics();
 
-    Assert.assertEquals("Unexpected MemorySeconds value",
-        memorySeconds, rmAppMetrics.getMemorySeconds());
-    Assert.assertEquals("Unexpected VcoreSeconds value",
-        vcoreSeconds, rmAppMetrics.getVcoreSeconds());
+    assertEquals(memorySeconds, rmAppMetrics.getMemorySeconds(),
+        "Unexpected MemorySeconds value");
+    assertEquals(vcoreSeconds, rmAppMetrics.getVcoreSeconds(),
+        "Unexpected VcoreSeconds value");
 
     rm.stop();
     return;

+ 16 - 14
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestDecommissioningNodesWatcher.java

@@ -18,6 +18,9 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
@@ -37,9 +40,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStatusEvent;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
 
 /**
  * This class tests DecommissioningNodesWatcher.
@@ -84,25 +86,25 @@ public class TestDecommissioningNodesWatcher {
     nodeStatus = createNodeStatus(id1, app, 2);
     node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus));
     watcher.update(node1, nodeStatus);
-    Assert.assertFalse(watcher.checkReadyToBeDecommissioned(id1));
+    assertFalse(watcher.checkReadyToBeDecommissioned(id1));
 
     nodeStatus = createNodeStatus(id1, app, 1);
     node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus));
     watcher.update(node1, nodeStatus);
-    Assert.assertEquals(DecommissioningNodeStatus.WAIT_CONTAINER,
+    assertEquals(DecommissioningNodeStatus.WAIT_CONTAINER,
         watcher.checkDecommissioningStatus(id1));
 
     nodeStatus = createNodeStatus(id1, app, 0);
     watcher.update(node1, nodeStatus);
     node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus));
-    Assert.assertEquals(DecommissioningNodeStatus.WAIT_APP,
+    assertEquals(DecommissioningNodeStatus.WAIT_APP,
         watcher.checkDecommissioningStatus(id1));
 
     // Set app to be FINISHED and verified DecommissioningNodeStatus is READY.
     MockRM.finishAMAndVerifyAppState(app, rm, nm1, am);
     rm.waitForState(app.getApplicationId(), RMAppState.FINISHED);
     watcher.update(node1, nodeStatus);
-    Assert.assertEquals(DecommissioningNodeStatus.READY,
+    assertEquals(DecommissioningNodeStatus.READY,
         watcher.checkDecommissioningStatus(id1));
   }
 
@@ -132,13 +134,13 @@ public class TestDecommissioningNodesWatcher {
     NodeStatus nodeStatus = createNodeStatus(id1, app, 3);
     node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus));
 
-    Assert.assertEquals(1, node1.getRunningApps().size());
+    assertEquals(1, node1.getRunningApps().size());
 
     // update node with 0 running containers
     nodeStatus = createNodeStatus(id1, app, 0);
     node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus));
 
-    Assert.assertEquals(1, node1.getRunningApps().size());
+    assertEquals(1, node1.getRunningApps().size());
 
     // Setup nm1 as DECOMMISSIONING for DecommissioningNodesWatcher. Right now
     // there is no container running on the node.
@@ -149,20 +151,20 @@ public class TestDecommissioningNodesWatcher {
     // we should still get WAIT_APP as container for a running app previously
     // ran on this node.
     watcher.update(node1, nodeStatus);
-    Assert.assertFalse(watcher.checkReadyToBeDecommissioned(id1));
-    Assert.assertEquals(DecommissioningNodeStatus.WAIT_APP,
+    assertFalse(watcher.checkReadyToBeDecommissioned(id1));
+    assertEquals(DecommissioningNodeStatus.WAIT_APP,
         watcher.checkDecommissioningStatus(id1));
 
     // Set app to be FINISHED and verified DecommissioningNodeStatus is READY.
     MockRM.finishAMAndVerifyAppState(app, rm, nm1, am);
     rm.waitForState(app.getApplicationId(), RMAppState.FINISHED);
-    Assert.assertEquals(0, node1.getRunningApps().size());
+    assertEquals(0, node1.getRunningApps().size());
     watcher.update(node1, nodeStatus);
-    Assert.assertEquals(DecommissioningNodeStatus.READY,
+    assertEquals(DecommissioningNodeStatus.READY,
         watcher.checkDecommissioningStatus(id1));
   }
 
-  @After
+  @AfterEach
   public void tearDown() {
     if (rm != null) {
       rm.stop();

+ 8 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestLeaderElectorService.java

@@ -33,9 +33,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
 import org.slf4j.event.Level;
 import org.apache.zookeeper.ZooKeeper;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import java.io.IOException;
 import java.util.Collection;
@@ -54,7 +55,7 @@ public class TestLeaderElectorService {
   MockRM rm1;
   MockRM rm2;
   TestingCluster zkCluster;
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     GenericTestUtils.setRootLogLevel(Level.INFO);
     conf = new Configuration();
@@ -73,7 +74,7 @@ public class TestLeaderElectorService {
     zkCluster.start();
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     if (rm1 != null) {
       rm1.stop();
@@ -87,7 +88,8 @@ public class TestLeaderElectorService {
   // 2. rm2 standby
   // 3. stop rm1
   // 4. rm2 become active
-  @Test (timeout = 20000)
+  @Test
+  @Timeout(value = 20)
   public void testRMShutDownCauseFailover() throws Exception {
     rm1 = startRM("rm1", HAServiceState.ACTIVE);
     rm2 = startRM("rm2", HAServiceState.STANDBY);

+ 13 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestMoveApplication.java

@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.security.AccessControlException;
 import java.security.PrivilegedExceptionAction;
@@ -36,16 +36,17 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 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.scheduler.fifo.FifoScheduler;
-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 TestMoveApplication {
   private ResourceManager resourceManager = null;
   private static boolean failMove;
   private Configuration conf;
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     conf = new YarnConfiguration();
     conf.setClass(YarnConfiguration.RM_SCHEDULER, FifoSchedulerWithMove.class,
@@ -59,7 +60,7 @@ public class TestMoveApplication {
     failMove = false;
   }
   
-  @After
+  @AfterEach
   public void tearDown() {
     resourceManager.stop();
   }
@@ -91,7 +92,8 @@ public class TestMoveApplication {
     }
   }
 
-  @Test (timeout = 10000)
+  @Test
+  @Timeout(value = 10)
   public void testMoveTooLate() throws Exception {
     // Submit application
     Application application = new Application("user1", resourceManager);
@@ -119,8 +121,9 @@ public class TestMoveApplication {
     }
   }
   
-  @Test (timeout = 10000)
-      public
+  @Test
+ @Timeout(value = 10)
+     public
       void testMoveSuccessful() throws Exception {
     MockRM rm1 = new MockRM(conf);
     rm1.start();

+ 25 - 21
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestNodeBlacklistingOnAMFailures.java

@@ -18,6 +18,9 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+
 import java.util.ArrayList;
 import java.util.List;
 
@@ -48,9 +51,9 @@ 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.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 /**
  * Validate system behavior when the am-scheduling logic 'blacklists' a node for
@@ -87,13 +90,14 @@ public class TestNodeBlacklistingOnAMFailures {
     return MockRMAppSubmitter.submit(rm, data);
   }
 
-  @Before
+  @BeforeEach
   public void setup() {
     QueueMetrics.clearQueueMetrics();
     DefaultMetricsSystem.setMiniClusterMode(true);
   }
 
-  @Test(timeout = 100000)
+  @Test
+  @Timeout(value = 100)
   public void testNodeBlacklistingOnAMFailure() throws Exception {
 
     YarnConfiguration conf = new YarnConfiguration();
@@ -160,10 +164,9 @@ public class TestNodeBlacklistingOnAMFailures {
       currentNode.nodeHeartbeat(true);
       rm.drainEvents();
 
-      Assert.assertEquals(
+      assertEquals(RMAppAttemptState.SCHEDULED, attempt.getAppAttemptState(),
           "AppAttemptState should still be SCHEDULED if currentNode is "
-              + "blacklisted correctly", RMAppAttemptState.SCHEDULED,
-          attempt.getAppAttemptState());
+          + "blacklisted correctly");
     }
 
     // Now try the other node
@@ -181,22 +184,21 @@ public class TestNodeBlacklistingOnAMFailures {
     nodeWhereAMRan = rmContainer.getAllocatedNode();
 
     // The other node should now receive the assignment
-    Assert.assertEquals(
-        "After blacklisting, AM should have run on the other node",
-        otherNode.getNodeId(), nodeWhereAMRan);
+    assertEquals(otherNode.getNodeId(), nodeWhereAMRan,
+        "After blacklisting, AM should have run on the other node");
 
     am2.registerAppAttempt();
     rm.waitForState(app.getApplicationId(), RMAppState.RUNNING);
 
     List<Container> allocatedContainers =
         TestAMRestart.allocateContainers(currentNode, am2, 1);
-    Assert.assertEquals(
+    assertEquals(currentNode.getNodeId(), allocatedContainers.get(0).getNodeId(),
         "Even though AM is blacklisted from the node, application can "
-            + "still allocate non-AM containers there",
-        currentNode.getNodeId(), allocatedContainers.get(0).getNodeId());
+        + "still allocate non-AM containers there");
   }
 
-  @Test(timeout = 100000)
+  @Test
+  @Timeout(value = 100)
   public void testNodeBlacklistingOnAMFailureStrictNodeLocality()
       throws Exception {
     YarnConfiguration conf = new YarnConfiguration();
@@ -251,7 +253,7 @@ public class TestNodeBlacklistingOnAMFailures {
         ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
     RMContainer rmContainer = scheduler.getRMContainer(amContainerId);
     NodeId nodeWhereAMRan = rmContainer.getAllocatedNode();
-    Assert.assertEquals(nm2.getNodeId(), nodeWhereAMRan);
+    assertEquals(nm2.getNodeId(), nodeWhereAMRan);
 
     // Set the exist status to INVALID so that we can verify that the system
     // automatically blacklisting the node
@@ -277,13 +279,14 @@ public class TestNodeBlacklistingOnAMFailures {
     // The second AM should be on the same node because the strict locality
     // made the eligible nodes only 1, so the blacklisting threshold kicked in
     System.out.println("AM ran on " + nodeWhereAMRan);
-    Assert.assertEquals(nm2.getNodeId(), nodeWhereAMRan);
+    assertEquals(nm2.getNodeId(), nodeWhereAMRan);
 
     am2.registerAppAttempt();
     rm.waitForState(app.getApplicationId(), RMAppState.RUNNING);
   }
 
-  @Test(timeout = 100000)
+  @Test
+  @Timeout(value = 100)
   public void testNodeBlacklistingOnAMFailureRelaxedNodeLocality()
       throws Exception {
     YarnConfiguration conf = new YarnConfiguration();
@@ -338,7 +341,7 @@ public class TestNodeBlacklistingOnAMFailures {
         ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
     RMContainer rmContainer = scheduler.getRMContainer(amContainerId);
     NodeId nodeWhereAMRan = rmContainer.getAllocatedNode();
-    Assert.assertEquals(nm2.getNodeId(), nodeWhereAMRan);
+    assertEquals(nm2.getNodeId(), nodeWhereAMRan);
 
     // Set the exist status to INVALID so that we can verify that the system
     // automatically blacklisting the node
@@ -368,13 +371,14 @@ public class TestNodeBlacklistingOnAMFailures {
     // The second AM should be on a different node because the relaxed locality
     // made the app schedulable on other nodes and nm2 is blacklisted
     System.out.println("AM ran on " + nodeWhereAMRan);
-    Assert.assertNotEquals(nm2.getNodeId(), nodeWhereAMRan);
+    assertNotEquals(nm2.getNodeId(), nodeWhereAMRan);
 
     am2.registerAppAttempt();
     rm.waitForState(app.getApplicationId(), RMAppState.RUNNING);
   }
 
-  @Test(timeout = 100000)
+  @Test
+  @Timeout(value = 100)
   public void testNoBlacklistingForNonSystemErrors() throws Exception {
 
     YarnConfiguration conf = new YarnConfiguration();

+ 125 - 111
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestOpportunisticContainerAllocatorAMService.java

@@ -18,6 +18,11 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ipc.ProtobufRpcEngine2;
 import org.apache.hadoop.ipc.RPC;
@@ -98,10 +103,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSe
 import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerContext;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 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;
+import org.junit.jupiter.api.Timeout;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
@@ -124,7 +129,7 @@ public class TestOpportunisticContainerAllocatorAMService {
   private OpportunisticContainersStatus oppContainersStatus =
       getOpportunisticStatus();
 
-  @Before
+  @BeforeEach
   public void createAndStartRM() {
     CapacitySchedulerConfiguration csConf =
         new CapacitySchedulerConfiguration();
@@ -168,7 +173,7 @@ public class TestOpportunisticContainerAllocatorAMService {
     rm.start();
   }
 
-  @After
+  @AfterEach
   public void stopRM() {
     if (rm != null) {
       rm.stop();
@@ -177,7 +182,9 @@ public class TestOpportunisticContainerAllocatorAMService {
     OpportunisticSchedulerMetrics.resetMetrics();
   }
 
-  @Test(timeout = 600000)
+  @Test
+  @Timeout(value = 600)
+  @SuppressWarnings("checkstyle:MethodLength")
   public void testContainerPromoteAndDemoteBeforeContainerStart() throws Exception {
     HashMap<NodeId, MockNM> nodes = new HashMap<>();
     MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
@@ -236,7 +243,7 @@ public class TestOpportunisticContainerAllocatorAMService {
         null);
     List<Container> allocatedContainers = allocateResponse
         .getAllocatedContainers();
-    Assert.assertEquals(2, allocatedContainers.size());
+    assertEquals(2, allocatedContainers.size());
     Container container = allocatedContainers.get(0);
     MockNM allocNode = nodes.get(container.getNodeId());
     MockNM sameHostDiffNode = null;
@@ -258,7 +265,7 @@ public class TestOpportunisticContainerAllocatorAMService {
     sameHostDiffNode.nodeHeartbeat(oppContainersStatus, true);
     rm.drainEvents();
     allocateResponse =  am1.allocate(new ArrayList<>(), new ArrayList<>());
-    Assert.assertEquals(0, allocateResponse.getUpdatedContainers().size());
+    assertEquals(0, allocateResponse.getUpdatedContainers().size());
 
     // Wait for scheduler to process all events
     dispatcher.waitForEventThreadToWait();
@@ -271,11 +278,11 @@ public class TestOpportunisticContainerAllocatorAMService {
         Arrays.asList(UpdateContainerRequest.newInstance(0,
             container.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
             null, ExecutionType.GUARANTEED)));
-    Assert.assertEquals(0, allocateResponse.getUpdatedContainers().size());
-    Assert.assertEquals(1, allocateResponse.getUpdateErrors().size());
-    Assert.assertEquals("UPDATE_OUTSTANDING_ERROR",
+    assertEquals(0, allocateResponse.getUpdatedContainers().size());
+    assertEquals(1, allocateResponse.getUpdateErrors().size());
+    assertEquals("UPDATE_OUTSTANDING_ERROR",
         allocateResponse.getUpdateErrors().get(0).getReason());
-    Assert.assertEquals(container.getId(),
+    assertEquals(container.getId(),
         allocateResponse.getUpdateErrors().get(0)
             .getUpdateContainerRequest().getContainerId());
 
@@ -286,14 +293,14 @@ public class TestOpportunisticContainerAllocatorAMService {
             container.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
             null, ExecutionType.GUARANTEED)));
 
-    Assert.assertEquals(0, allocateResponse.getUpdatedContainers().size());
-    Assert.assertEquals(1, allocateResponse.getUpdateErrors().size());
-    Assert.assertEquals("INCORRECT_CONTAINER_VERSION_ERROR",
+    assertEquals(0, allocateResponse.getUpdatedContainers().size());
+    assertEquals(1, allocateResponse.getUpdateErrors().size());
+    assertEquals("INCORRECT_CONTAINER_VERSION_ERROR",
         allocateResponse.getUpdateErrors().get(0).getReason());
-    Assert.assertEquals(0,
+    assertEquals(0,
         allocateResponse.getUpdateErrors().get(0)
             .getCurrentContainerVersion());
-    Assert.assertEquals(container.getId(),
+    assertEquals(container.getId(),
         allocateResponse.getUpdateErrors().get(0)
             .getUpdateContainerRequest().getContainerId());
 
@@ -301,12 +308,12 @@ public class TestOpportunisticContainerAllocatorAMService {
     allocNode.nodeHeartbeat(oppContainersStatus, true);
     rm.drainEvents();
     allocateResponse =  am1.allocate(new ArrayList<>(), new ArrayList<>());
-    Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size());
+    assertEquals(1, allocateResponse.getUpdatedContainers().size());
     Container uc =
         allocateResponse.getUpdatedContainers().get(0).getContainer();
-    Assert.assertEquals(ExecutionType.GUARANTEED, uc.getExecutionType());
-    Assert.assertEquals(uc.getId(), container.getId());
-    Assert.assertEquals(uc.getVersion(), container.getVersion() + 1);
+    assertEquals(ExecutionType.GUARANTEED, uc.getExecutionType());
+    assertEquals(uc.getId(), container.getId());
+    assertEquals(uc.getVersion(), container.getVersion() + 1);
 
     // Verify Metrics After OPP allocation :
     // Allocated cores+mem should have increased, available should decrease
@@ -322,7 +329,7 @@ public class TestOpportunisticContainerAllocatorAMService {
     RMContainer rmContainer = ((CapacityScheduler) scheduler)
         .getApplicationAttempt(
         uc.getId().getApplicationAttemptId()).getRMContainer(uc.getId());
-    Assert.assertEquals(RMContainerState.ACQUIRED, rmContainer.getState());
+    assertEquals(RMContainerState.ACQUIRED, rmContainer.getState());
 
     // Now demote the container back..
     allocateResponse = am1.sendContainerUpdateRequest(
@@ -330,11 +337,11 @@ public class TestOpportunisticContainerAllocatorAMService {
             uc.getId(), ContainerUpdateType.DEMOTE_EXECUTION_TYPE,
             null, ExecutionType.OPPORTUNISTIC)));
     // This should happen in the same heartbeat..
-    Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size());
+    assertEquals(1, allocateResponse.getUpdatedContainers().size());
     uc = allocateResponse.getUpdatedContainers().get(0).getContainer();
-    Assert.assertEquals(ExecutionType.OPPORTUNISTIC, uc.getExecutionType());
-    Assert.assertEquals(uc.getId(), container.getId());
-    Assert.assertEquals(uc.getVersion(), container.getVersion() + 2);
+    assertEquals(ExecutionType.OPPORTUNISTIC, uc.getExecutionType());
+    assertEquals(uc.getId(), container.getId());
+    assertEquals(uc.getVersion(), container.getVersion() + 2);
 
     // Wait for scheduler to finish processing events
     dispatcher.waitForEventThreadToWait();
@@ -344,7 +351,8 @@ public class TestOpportunisticContainerAllocatorAMService {
     verifyMetrics(metrics, 15360, 15, 1024, 1, 1);
   }
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testContainerPromoteAfterContainerStart() throws Exception {
     HashMap<NodeId, MockNM> nodes = new HashMap<>();
     MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
@@ -393,7 +401,7 @@ public class TestOpportunisticContainerAllocatorAMService {
         null);
     List<Container> allocatedContainers = allocateResponse
         .getAllocatedContainers();
-    Assert.assertEquals(2, allocatedContainers.size());
+    assertEquals(2, allocatedContainers.size());
     Container container = allocatedContainers.get(0);
     MockNM allocNode = nodes.get(container.getNodeId());
 
@@ -409,7 +417,7 @@ public class TestOpportunisticContainerAllocatorAMService {
         .getApplicationAttempt(
             container.getId().getApplicationAttemptId()).getRMContainer(
             container.getId());
-    Assert.assertEquals(RMContainerState.RUNNING, rmContainer.getState());
+    assertEquals(RMContainerState.RUNNING, rmContainer.getState());
 
     // Verify Metrics After OPP allocation (Nothing should change)
     verifyMetrics(metrics, 7168, 7, 1024, 1, 1);
@@ -427,11 +435,11 @@ public class TestOpportunisticContainerAllocatorAMService {
         Arrays.asList(UpdateContainerRequest.newInstance(0,
             container.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
             null, ExecutionType.GUARANTEED)));
-    Assert.assertEquals(0, allocateResponse.getUpdatedContainers().size());
-    Assert.assertEquals(1, allocateResponse.getUpdateErrors().size());
-    Assert.assertEquals("UPDATE_OUTSTANDING_ERROR",
+    assertEquals(0, allocateResponse.getUpdatedContainers().size());
+    assertEquals(1, allocateResponse.getUpdateErrors().size());
+    assertEquals("UPDATE_OUTSTANDING_ERROR",
         allocateResponse.getUpdateErrors().get(0).getReason());
-    Assert.assertEquals(container.getId(),
+    assertEquals(container.getId(),
         allocateResponse.getUpdateErrors().get(0)
             .getUpdateContainerRequest().getContainerId());
 
@@ -443,25 +451,26 @@ public class TestOpportunisticContainerAllocatorAMService {
     rm.drainEvents();
 
     allocateResponse =  am1.allocate(new ArrayList<>(), new ArrayList<>());
-    Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size());
+    assertEquals(1, allocateResponse.getUpdatedContainers().size());
     Container uc =
         allocateResponse.getUpdatedContainers().get(0).getContainer();
-    Assert.assertEquals(ExecutionType.GUARANTEED, uc.getExecutionType());
-    Assert.assertEquals(uc.getId(), container.getId());
-    Assert.assertEquals(uc.getVersion(), container.getVersion() + 1);
+    assertEquals(ExecutionType.GUARANTEED, uc.getExecutionType());
+    assertEquals(uc.getId(), container.getId());
+    assertEquals(uc.getVersion(), container.getVersion() + 1);
 
     // Verify that the Container is still in RUNNING state wrt RM..
     rmContainer = ((CapacityScheduler) scheduler)
         .getApplicationAttempt(
             uc.getId().getApplicationAttemptId()).getRMContainer(uc.getId());
-    Assert.assertEquals(RMContainerState.RUNNING, rmContainer.getState());
+    assertEquals(RMContainerState.RUNNING, rmContainer.getState());
 
     // Verify Metrics After OPP allocation :
     // Allocated cores+mem should have increased, available should decrease
     verifyMetrics(metrics, 6144, 6, 2048, 2, 2);
   }
 
-  @Test(timeout = 600000)
+  @Test
+  @Timeout(value = 600)
   public void testContainerPromoteAfterContainerComplete() throws Exception {
     HashMap<NodeId, MockNM> nodes = new HashMap<>();
     MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
@@ -511,7 +520,7 @@ public class TestOpportunisticContainerAllocatorAMService {
         null);
     List<Container> allocatedContainers = allocateResponse
         .getAllocatedContainers();
-    Assert.assertEquals(2, allocatedContainers.size());
+    assertEquals(2, allocatedContainers.size());
     Container container = allocatedContainers.get(0);
     MockNM allocNode = nodes.get(container.getNodeId());
 
@@ -527,7 +536,7 @@ public class TestOpportunisticContainerAllocatorAMService {
         .getApplicationAttempt(
             container.getId().getApplicationAttemptId()).getRMContainer(
             container.getId());
-    Assert.assertEquals(RMContainerState.RUNNING, rmContainer.getState());
+    assertEquals(RMContainerState.RUNNING, rmContainer.getState());
 
     // Container Completed in the NM
     allocNode.nodeHeartbeat(Arrays.asList(
@@ -541,7 +550,7 @@ public class TestOpportunisticContainerAllocatorAMService {
         .getApplicationAttempt(
             container.getId().getApplicationAttemptId()).getRMContainer(
             container.getId());
-    Assert.assertNull(rmContainer);
+    assertNull(rmContainer);
 
     // Verify Metrics After OPP allocation (Nothing should change)
     verifyMetrics(metrics, 7168, 7, 1024, 1, 1);
@@ -553,16 +562,16 @@ public class TestOpportunisticContainerAllocatorAMService {
             container.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
             null, ExecutionType.GUARANTEED)));
 
-    Assert.assertEquals(1,
+    assertEquals(1,
         allocateResponse.getCompletedContainersStatuses().size());
-    Assert.assertEquals(container.getId(),
+    assertEquals(container.getId(),
         allocateResponse.getCompletedContainersStatuses().get(0)
             .getContainerId());
-    Assert.assertEquals(0, allocateResponse.getUpdatedContainers().size());
-    Assert.assertEquals(1, allocateResponse.getUpdateErrors().size());
-    Assert.assertEquals("INVALID_CONTAINER_ID",
+    assertEquals(0, allocateResponse.getUpdatedContainers().size());
+    assertEquals(1, allocateResponse.getUpdateErrors().size());
+    assertEquals("INVALID_CONTAINER_ID",
         allocateResponse.getUpdateErrors().get(0).getReason());
-    Assert.assertEquals(container.getId(),
+    assertEquals(container.getId(),
         allocateResponse.getUpdateErrors().get(0)
             .getUpdateContainerRequest().getContainerId());
 
@@ -570,7 +579,8 @@ public class TestOpportunisticContainerAllocatorAMService {
     verifyMetrics(metrics, 7168, 7, 1024, 1, 1);
   }
 
-  @Test(timeout = 600000)
+  @Test
+  @Timeout(value = 600)
   public void testContainerAutoUpdateContainer() throws Exception {
     rm.stop();
     createAndStartRMWithAutoUpdateContainer();
@@ -608,7 +618,7 @@ public class TestOpportunisticContainerAllocatorAMService {
         allocateResponse.getAllocatedContainers();
     allocatedContainers.addAll(
         am1.allocate(null, null).getAllocatedContainers());
-    Assert.assertEquals(2, allocatedContainers.size());
+    assertEquals(2, allocatedContainers.size());
     Container container = allocatedContainers.get(0);
     // Start Container in NM
     nm1.nodeHeartbeat(Arrays.asList(ContainerStatus
@@ -620,7 +630,7 @@ public class TestOpportunisticContainerAllocatorAMService {
     RMContainer rmContainer = ((CapacityScheduler) scheduler)
         .getApplicationAttempt(container.getId().getApplicationAttemptId())
         .getRMContainer(container.getId());
-    Assert.assertEquals(RMContainerState.RUNNING, rmContainer.getState());
+    assertEquals(RMContainerState.RUNNING, rmContainer.getState());
 
     // Send Promotion req... this should result in update error
     // Since the container doesn't exist anymore..
@@ -636,17 +646,17 @@ public class TestOpportunisticContainerAllocatorAMService {
     // Get the update response on next allocate
     allocateResponse = am1.allocate(new ArrayList<>(), new ArrayList<>());
     // Check the update response from YARNRM
-    Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size());
+    assertEquals(1, allocateResponse.getUpdatedContainers().size());
     UpdatedContainer uc = allocateResponse.getUpdatedContainers().get(0);
-    Assert.assertEquals(container.getId(), uc.getContainer().getId());
-    Assert.assertEquals(ExecutionType.GUARANTEED,
+    assertEquals(container.getId(), uc.getContainer().getId());
+    assertEquals(ExecutionType.GUARANTEED,
         uc.getContainer().getExecutionType());
     // Check that the container is updated in NM through NM heartbeat response
     NodeHeartbeatResponse response = nm1.nodeHeartbeat(true);
-    Assert.assertEquals(1, response.getContainersToUpdate().size());
+    assertEquals(1, response.getContainersToUpdate().size());
     Container containersFromNM = response.getContainersToUpdate().get(0);
-    Assert.assertEquals(container.getId(), containersFromNM.getId());
-    Assert.assertEquals(ExecutionType.GUARANTEED,
+    assertEquals(container.getId(), containersFromNM.getId());
+    assertEquals(ExecutionType.GUARANTEED,
         containersFromNM.getExecutionType());
 
     //Increase resources
@@ -662,10 +672,10 @@ public class TestOpportunisticContainerAllocatorAMService {
     if (allocateResponse.getUpdatedContainers().size() == 0) {
       allocateResponse = am1.allocate(new ArrayList<>(), new ArrayList<>());
     }
-    Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size());
+    assertEquals(1, allocateResponse.getUpdatedContainers().size());
     uc = allocateResponse.getUpdatedContainers().get(0);
-    Assert.assertEquals(container.getId(), uc.getContainer().getId());
-    Assert.assertEquals(Resource.newInstance(2 * GB, 1),
+    assertEquals(container.getId(), uc.getContainer().getId());
+    assertEquals(Resource.newInstance(2 * GB, 1),
         uc.getContainer().getResource());
     rm.drainEvents();
 
@@ -674,8 +684,8 @@ public class TestOpportunisticContainerAllocatorAMService {
     if (response.getContainersToUpdate().size() == 0) {
       response = nm1.nodeHeartbeat(true);
     }
-    Assert.assertEquals(1, response.getContainersToUpdate().size());
-    Assert.assertEquals(Resource.newInstance(2 * GB, 1),
+    assertEquals(1, response.getContainersToUpdate().size());
+    assertEquals(Resource.newInstance(2 * GB, 1),
         response.getContainersToUpdate().get(0).getResource());
 
     //Decrease resources
@@ -683,14 +693,14 @@ public class TestOpportunisticContainerAllocatorAMService {
         UpdateContainerRequest.newInstance(2, container.getId(),
             ContainerUpdateType.DECREASE_RESOURCE,
             Resources.createResource(1 * GB, 1), null)));
-    Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size());
+    assertEquals(1, allocateResponse.getUpdatedContainers().size());
     rm.drainEvents();
 
     // Check that the container resources are decreased
     // in NM through NM heartbeat response
     response = nm1.nodeHeartbeat(true);
-    Assert.assertEquals(1, response.getContainersToUpdate().size());
-    Assert.assertEquals(Resource.newInstance(1 * GB, 1),
+    assertEquals(1, response.getContainersToUpdate().size());
+    assertEquals(Resource.newInstance(1 * GB, 1),
         response.getContainersToUpdate().get(0).getResource());
 
     nm1.nodeHeartbeat(oppContainersStatus, true);
@@ -709,30 +719,31 @@ public class TestOpportunisticContainerAllocatorAMService {
       allocateResponse = am1.allocate(new ArrayList<>(), new ArrayList<>());
     }
     // Check the update response from YARNRM
-    Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size());
+    assertEquals(1, allocateResponse.getUpdatedContainers().size());
     uc = allocateResponse.getUpdatedContainers().get(0);
-    Assert.assertEquals(ExecutionType.OPPORTUNISTIC,
+    assertEquals(ExecutionType.OPPORTUNISTIC,
         uc.getContainer().getExecutionType());
     // Check that the container is updated in NM through NM heartbeat response
     if (response.getContainersToUpdate().size() == 0) {
       response = nm1.nodeHeartbeat(oppContainersStatus, true);
     }
-    Assert.assertEquals(1, response.getContainersToUpdate().size());
-    Assert.assertEquals(ExecutionType.OPPORTUNISTIC,
+    assertEquals(1, response.getContainersToUpdate().size());
+    assertEquals(ExecutionType.OPPORTUNISTIC,
         response.getContainersToUpdate().get(0).getExecutionType());
   }
 
   private void verifyMetrics(QueueMetrics metrics, long availableMB,
       int availableVirtualCores, long allocatedMB,
       int allocatedVirtualCores, int allocatedContainers) {
-    Assert.assertEquals(availableMB, metrics.getAvailableMB());
-    Assert.assertEquals(availableVirtualCores, metrics.getAvailableVirtualCores());
-    Assert.assertEquals(allocatedMB, metrics.getAllocatedMB());
-    Assert.assertEquals(allocatedVirtualCores, metrics.getAllocatedVirtualCores());
-    Assert.assertEquals(allocatedContainers, metrics.getAllocatedContainers());
+    assertEquals(availableMB, metrics.getAvailableMB());
+    assertEquals(availableVirtualCores, metrics.getAvailableVirtualCores());
+    assertEquals(allocatedMB, metrics.getAllocatedMB());
+    assertEquals(allocatedVirtualCores, metrics.getAllocatedVirtualCores());
+    assertEquals(allocatedContainers, metrics.getAllocatedContainers());
   }
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testOpportunisticSchedulerMetrics() throws Exception {
     HashMap<NodeId, MockNM> nodes = new HashMap<>();
     MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
@@ -783,12 +794,12 @@ public class TestOpportunisticContainerAllocatorAMService {
 
     List<Container> allocatedContainers = allocateResponse
         .getAllocatedContainers();
-    Assert.assertEquals(2, allocatedContainers.size());
+    assertEquals(2, allocatedContainers.size());
 
-    Assert.assertEquals(allocContainers + 2, metrics.getAllocatedContainers());
-    Assert.assertEquals(aggrAllocatedContainers + 2,
+    assertEquals(allocContainers + 2, metrics.getAllocatedContainers());
+    assertEquals(aggrAllocatedContainers + 2,
         metrics.getAggregatedAllocatedContainers());
-    Assert.assertEquals(aggrOffSwitchContainers + 2,
+    assertEquals(aggrOffSwitchContainers + 2,
         metrics.getAggregatedOffSwitchContainers());
 
     Container container = allocatedContainers.get(0);
@@ -806,7 +817,7 @@ public class TestOpportunisticContainerAllocatorAMService {
         .getApplicationAttempt(
             container.getId().getApplicationAttemptId()).getRMContainer(
             container.getId());
-    Assert.assertEquals(RMContainerState.RUNNING, rmContainer.getState());
+    assertEquals(RMContainerState.RUNNING, rmContainer.getState());
 
     // Container Completed in the NM
     allocNode.nodeHeartbeat(Arrays.asList(
@@ -820,10 +831,10 @@ public class TestOpportunisticContainerAllocatorAMService {
         .getApplicationAttempt(
             container.getId().getApplicationAttemptId()).getRMContainer(
             container.getId());
-    Assert.assertNull(rmContainer);
+    assertNull(rmContainer);
 
-    Assert.assertEquals(allocContainers + 1, metrics.getAllocatedContainers());
-    Assert.assertEquals(aggrReleasedContainers + 1,
+    assertEquals(allocContainers + 1, metrics.getAllocatedContainers());
+    assertEquals(aggrReleasedContainers + 1,
         metrics.getAggregatedReleasedContainers());
   }
 
@@ -872,7 +883,7 @@ public class TestOpportunisticContainerAllocatorAMService {
             ExecutionType.OPPORTUNISTIC, -1);
 
     // Make sure that numbers start with 0
-    Assert.assertEquals(0, metrics.getAllocatedContainers());
+    assertEquals(0, metrics.getAllocatedContainers());
 
     // Recover one OContainer only
     rm.registerNode("h2:1234", 4096, 1,
@@ -880,7 +891,7 @@ public class TestOpportunisticContainerAllocatorAMService {
             appAttemptId.getApplicationId()),
         Collections.singletonList(recoverOContainerReport1));
 
-    Assert.assertEquals(1, metrics.getAllocatedContainers());
+    assertEquals(1, metrics.getAllocatedContainers());
 
     // Recover two OContainers at once
     final ContainerId recoverOContainerId3 = ContainerId.newContainerId(
@@ -909,7 +920,7 @@ public class TestOpportunisticContainerAllocatorAMService {
             appAttemptId.getApplicationId()),
         Arrays.asList(recoverOContainerReport2, recoverOContainerReport3));
 
-    Assert.assertEquals(3, metrics.getAllocatedContainers());
+    assertEquals(3, metrics.getAllocatedContainers());
 
     // Make sure that the recovered GContainer
     // does not increment OContainer count
@@ -929,7 +940,7 @@ public class TestOpportunisticContainerAllocatorAMService {
             appAttemptId.getApplicationId()),
         Collections.singletonList(recoverGContainerReport));
 
-    Assert.assertEquals(3, metrics.getAllocatedContainers());
+    assertEquals(3, metrics.getAllocatedContainers());
 
     final ContainerId completedOContainerId = ContainerId.newContainerId(
         appAttemptId, 6);
@@ -948,10 +959,11 @@ public class TestOpportunisticContainerAllocatorAMService {
             appAttemptId.getApplicationId()),
         Collections.singletonList(completedOContainerReport));
 
-    Assert.assertEquals(3, metrics.getAllocatedContainers());
+    assertEquals(3, metrics.getAllocatedContainers());
   }
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testAMCrashDuringAllocate() throws Exception {
     MockNM nm = new MockNM("h:1234", 4096, rm.getResourceTrackerService());
     nm.registerNode();
@@ -986,7 +998,8 @@ public class TestOpportunisticContainerAllocatorAMService {
         "*", Resources.createResource(1 * GB), 2)), null);
   }
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testNodeRemovalDuringAllocate() throws Exception {
     MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
     MockNM nm2 = new MockNM("h2:1234", 4096, rm.getResourceTrackerService());
@@ -1032,7 +1045,7 @@ public class TestOpportunisticContainerAllocatorAMService {
       }
       Thread.sleep(50);
     }
-    Assert.assertEquals(2, ctxt.getNodeMap().size());
+    assertEquals(2, ctxt.getNodeMap().size());
     // Remove node from scheduler but not from AM Service.
     scheduler.handle(new NodeRemovedSchedulerEvent(rmNode1));
     // After removal of node 1, only 1 node will be applicable for scheduling.
@@ -1043,17 +1056,18 @@ public class TestOpportunisticContainerAllocatorAMService {
                 "*", Resources.createResource(1 * GB), 2)),
             null);
       } catch (Exception e) {
-        Assert.fail("Allocate request should be handled on node removal");
+        fail("Allocate request should be handled on node removal");
       }
       if (ctxt.getNodeMap().size() == 1) {
         break;
       }
       Thread.sleep(50);
     }
-    Assert.assertEquals(1, ctxt.getNodeMap().size());
+    assertEquals(1, ctxt.getNodeMap().size());
   }
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testAppAttemptRemovalAfterNodeRemoval() throws Exception {
     MockNM nm = new MockNM("h:1234", 4096, rm.getResourceTrackerService());
 
@@ -1189,7 +1203,7 @@ public class TestOpportunisticContainerAllocatorAMService {
                 ((RegisterApplicationMasterRequestPBImpl)factory
                     .newRecordInstance(
                         RegisterApplicationMasterRequest.class)).getProto()));
-    Assert.assertEquals("dummyQueue", regResp.getQueue());
+    assertEquals("dummyQueue", regResp.getQueue());
     FinishApplicationMasterResponse finishResp =
         new FinishApplicationMasterResponsePBImpl(
             ampProxy.finishApplicationMaster(null,
@@ -1197,7 +1211,7 @@ public class TestOpportunisticContainerAllocatorAMService {
                     .newRecordInstance(
                         FinishApplicationMasterRequest.class)).getProto()
             ));
-    Assert.assertEquals(false, finishResp.getIsUnregistered());
+    assertEquals(false, finishResp.getIsUnregistered());
     AllocateResponse allocResp =
         new AllocateResponsePBImpl(
             ampProxy.allocate(null,
@@ -1205,10 +1219,10 @@ public class TestOpportunisticContainerAllocatorAMService {
                     .newRecordInstance(AllocateRequest.class)).getProto())
         );
     List<Container> allocatedContainers = allocResp.getAllocatedContainers();
-    Assert.assertEquals(1, allocatedContainers.size());
-    Assert.assertEquals(ExecutionType.OPPORTUNISTIC,
+    assertEquals(1, allocatedContainers.size());
+    assertEquals(ExecutionType.OPPORTUNISTIC,
         allocatedContainers.get(0).getExecutionType());
-    Assert.assertEquals(12345, allocResp.getNumClusterNodes());
+    assertEquals(12345, allocResp.getNumClusterNodes());
 
 
     // Verify that the DistrubutedSchedulingService can handle the
@@ -1225,12 +1239,12 @@ public class TestOpportunisticContainerAllocatorAMService {
                 ((RegisterApplicationMasterRequestPBImpl)factory
                     .newRecordInstance(RegisterApplicationMasterRequest.class))
                     .getProto()));
-    Assert.assertEquals(54321l, dsRegResp.getContainerIdStart());
-    Assert.assertEquals(4,
+    assertEquals(54321L, dsRegResp.getContainerIdStart());
+    assertEquals(4,
         dsRegResp.getMaxContainerResource().getVirtualCores());
-    Assert.assertEquals(1024,
+    assertEquals(1024,
         dsRegResp.getMinContainerResource().getMemorySize());
-    Assert.assertEquals(2,
+    assertEquals(2,
         dsRegResp.getIncrContainerResource().getVirtualCores());
 
     DistributedSchedulingAllocateRequestPBImpl distAllReq =
@@ -1242,9 +1256,9 @@ public class TestOpportunisticContainerAllocatorAMService {
         new DistributedSchedulingAllocateResponsePBImpl(
             dsProxy.allocateForDistributedScheduling(null,
                 distAllReq.getProto()));
-    Assert.assertEquals(
+    assertEquals(
         "h1", dsAllocResp.getNodesForScheduling().get(0).getNodeId().getHost());
-    Assert.assertEquals(
+    assertEquals(
         "l1", dsAllocResp.getNodesForScheduling().get(1).getNodePartition());
 
     FinishApplicationMasterResponse dsfinishResp =
@@ -1253,7 +1267,7 @@ public class TestOpportunisticContainerAllocatorAMService {
                 ((FinishApplicationMasterRequestPBImpl) factory
                     .newRecordInstance(FinishApplicationMasterRequest.class))
                     .getProto()));
-    Assert.assertEquals(
+    assertEquals(
         false, dsfinishResp.getIsUnregistered());
   }
 
@@ -1315,11 +1329,11 @@ public class TestOpportunisticContainerAllocatorAMService {
         List<ResourceRequest> askList =
             request.getAllocateRequest().getAskList();
         List<Container> allocatedContainers = request.getAllocatedContainers();
-        Assert.assertEquals(1, allocatedContainers.size());
-        Assert.assertEquals(ExecutionType.OPPORTUNISTIC,
+        assertEquals(1, allocatedContainers.size());
+        assertEquals(ExecutionType.OPPORTUNISTIC,
             allocatedContainers.get(0).getExecutionType());
-        Assert.assertEquals(1, askList.size());
-        Assert.assertTrue(askList.get(0)
+        assertEquals(1, askList.size());
+        assertTrue(askList.get(0)
             .getExecutionTypeRequest().getEnforceExecutionType());
         DistributedSchedulingAllocateResponse resp = factory
             .newRecordInstance(DistributedSchedulingAllocateResponse.class);

+ 108 - 102
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java

@@ -19,8 +19,11 @@
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.assertEquals;
-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.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.DataOutputStream;
 import java.io.File;
@@ -91,10 +94,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import static org.apache.hadoop.yarn.conf.YarnConfiguration.RM_PROXY_USER_PREFIX;
 import static org.apache.hadoop.yarn.server.resourcemanager.resource.DynamicResourceConfiguration.NODES;
 import static org.apache.hadoop.yarn.server.resourcemanager.resource.DynamicResourceConfiguration.PREFIX;
-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 org.mockito.Mockito;
 
 import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList;
@@ -105,8 +108,6 @@ import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.AddToClusterNodeLabelsRequestProto;
 
-import static org.junit.Assert.assertTrue;
-
 public class TestRMAdminService {
 
   private Configuration configuration;
@@ -122,7 +123,7 @@ public class TestRMAdminService {
         YarnConfiguration.DR_CONFIGURATION_FILE);
   }
 
-  @Before
+  @BeforeEach
   public void setup() throws IOException {
     QueueMetrics.clearQueueMetrics();
     DefaultMetricsSystem.setMiniClusterMode(true);
@@ -147,7 +148,7 @@ public class TestRMAdminService {
     MockUnixGroupsMapping.resetGroups();
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws IOException {
     if (rm != null) {
       rm.stop();
@@ -169,7 +170,7 @@ public class TestRMAdminService {
 
     try {
       rm.adminService.refreshQueues(RefreshQueuesRequest.newInstance());
-      Assert.assertEquals(maxAppsBefore, cs.getConfiguration()
+      assertEquals(maxAppsBefore, cs.getConfiguration()
           .getMaximumSystemApplications());
     } catch (Exception ex) {
       fail("Using localConfigurationProvider. Should not get any exception.");
@@ -239,8 +240,8 @@ public class TestRMAdminService {
     rm.adminService.refreshQueues(RefreshQueuesRequest.newInstance());
 
     int maxAppsAfter = cs.getConfiguration().getMaximumSystemApplications();
-    Assert.assertEquals(maxAppsAfter, 5000);
-    Assert.assertTrue(maxAppsAfter != maxAppsBefore);
+    assertEquals(maxAppsAfter, 5000);
+    assertTrue(maxAppsAfter != maxAppsBefore);
   }
 
   @Test
@@ -313,7 +314,7 @@ public class TestRMAdminService {
     NodeId nid = NodeId.fromString("h1:1234");
     RMNode ni = rm.getRMContext().getRMNodes().get(nid);
     Resource resource = ni.getTotalCapability();
-    Assert.assertEquals("<memory:5120, vCores:5>", resource.toString());
+    assertEquals("<memory:5120, vCores:5>", resource.toString());
 
     DynamicResourceConfiguration drConf =
         new DynamicResourceConfiguration();
@@ -328,7 +329,7 @@ public class TestRMAdminService {
 
     RMNode niAfter = rm.getRMContext().getRMNodes().get(nid);
     Resource resourceAfter = niAfter.getTotalCapability();
-    Assert.assertEquals("<memory:4096, vCores:4>", resourceAfter.toString());
+    assertEquals("<memory:4096, vCores:4>", resourceAfter.toString());
   }
 
   @Test
@@ -353,7 +354,7 @@ public class TestRMAdminService {
     NodeId nid = NodeId.fromString("h1:1234");
     RMNode ni = rm.getRMContext().getRMNodes().get(nid);
     Resource resource = ni.getTotalCapability();
-    Assert.assertEquals("<memory:2048, vCores:2>", resource.toString());
+    assertEquals("<memory:2048, vCores:2>", resource.toString());
 
     DynamicResourceConfiguration drConf =
         new DynamicResourceConfiguration();
@@ -375,10 +376,10 @@ public class TestRMAdminService {
 
     RMNode niAfter = rm.getRMContext().getRMNodes().get(nid);
     Resource resourceAfter = niAfter.getTotalCapability();
-    Assert.assertEquals("<memory:4096, vCores:4>", resourceAfter.toString());
+    assertEquals("<memory:4096, vCores:4>", resourceAfter.toString());
 
-    Assert.assertEquals(4096, nm.getMemory());
-    Assert.assertEquals(4, nm.getvCores());
+    assertEquals(4096, nm.getMemory());
+    assertEquals(4, nm.getvCores());
   }
 
   @Test
@@ -403,7 +404,7 @@ public class TestRMAdminService {
     NodeId nid = NodeId.fromString("h1:1234");
     RMNode ni = rm.getRMContext().getRMNodes().get(nid);
     Resource resource = ni.getTotalCapability();
-    Assert.assertEquals("<memory:2048, vCores:2>", resource.toString());
+    assertEquals("<memory:2048, vCores:2>", resource.toString());
 
     DynamicResourceConfiguration drConf =
         new DynamicResourceConfiguration();
@@ -424,10 +425,10 @@ public class TestRMAdminService {
 
     RMNode niAfter = rm.getRMContext().getRMNodes().get(nid);
     Resource resourceAfter = niAfter.getTotalCapability();
-    Assert.assertEquals("<memory:4096, vCores:4>", resourceAfter.toString());
+    assertEquals("<memory:4096, vCores:4>", resourceAfter.toString());
 
-    Assert.assertEquals(4096, nm.getMemory());
-    Assert.assertEquals(4, nm.getvCores());
+    assertEquals(4096, nm.getMemory());
+    assertEquals(4, nm.getvCores());
   }
 
   @Test
@@ -451,7 +452,7 @@ public class TestRMAdminService {
     NodeId nid = NodeId.fromString("h1:1234");
     RMNode ni = rm.getRMContext().getRMNodes().get(nid);
     Resource resource = ni.getTotalCapability();
-    Assert.assertEquals("<memory:5120, vCores:5>", resource.toString());
+    assertEquals("<memory:5120, vCores:5>", resource.toString());
 
     DynamicResourceConfiguration drConf =
         new DynamicResourceConfiguration();
@@ -473,7 +474,7 @@ public class TestRMAdminService {
 
     RMNode niAfter = rm.getRMContext().getRMNodes().get(nid);
     Resource resourceAfter = niAfter.getTotalCapability();
-    Assert.assertEquals("<memory:4096, vCores:4>", resourceAfter.toString());
+    assertEquals("<memory:4096, vCores:4>", resourceAfter.toString());
 
     // Replace original dr file with an empty dr file, and validate node
     // registration with new resources will take effective now.
@@ -496,7 +497,7 @@ public class TestRMAdminService {
     resourceAfter = niAfter.getTotalCapability();
     // new resource in registration should take effective as we empty
     // dynamic resource file already.
-    Assert.assertEquals("<memory:8192, vCores:8>", resourceAfter.toString());
+    assertEquals("<memory:8192, vCores:8>", resourceAfter.toString());
   }
 
   @Test
@@ -541,8 +542,8 @@ public class TestRMAdminService {
     String aclStringAfter =
         rm.adminService.getAccessControlList().getAclString().trim();
 
-    Assert.assertTrue(!aclStringAfter.equals(aclStringBefore));
-    Assert.assertEquals(aclStringAfter, "world:anyone:rwcda," +
+    assertTrue(!aclStringAfter.equals(aclStringBefore));
+    assertEquals(aclStringAfter, "world:anyone:rwcda," +
         UserGroupInformation.getCurrentUser().getShortUserName());
   }
 
@@ -643,7 +644,7 @@ public class TestRMAdminService {
       AccessControlList accessList =
           manager.getProtocolsAcls(protocolClass);
       if (protocolClass == protocol) {
-        Assert.assertEquals(accessList.getAclString(),
+        assertEquals(accessList.getAclString(),
             aclString);
       } else {
         assertThat(accessList.getAclString()).isEqualTo("*");
@@ -691,14 +692,14 @@ public class TestRMAdminService {
 
     rm.adminService.refreshSuperUserGroupsConfiguration(
         RefreshSuperUserGroupsConfigurationRequest.newInstance());
-    Assert.assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyGroups()
+    assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyGroups()
         .get("hadoop.proxyuser.test.groups").size() == 1);
-    Assert.assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyGroups()
+    assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyGroups()
         .get("hadoop.proxyuser.test.groups").contains("test_groups"));
 
-    Assert.assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyHosts()
+    assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyHosts()
         .get("hadoop.proxyuser.test.hosts").size() == 1);
-    Assert.assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyHosts()
+    assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyHosts()
         .get("hadoop.proxyuser.test.hosts").contains("test_hosts"));
 
     Configuration yarnConf = new Configuration(false);
@@ -709,14 +710,14 @@ public class TestRMAdminService {
     // RM specific configs will overwrite the common ones
     rm.adminService.refreshSuperUserGroupsConfiguration(
         RefreshSuperUserGroupsConfigurationRequest.newInstance());
-    Assert.assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyGroups()
+    assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyGroups()
         .get("hadoop.proxyuser.test.groups").size() == 1);
-    Assert.assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyGroups()
+    assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyGroups()
         .get("hadoop.proxyuser.test.groups").contains("test_groups_1"));
 
-    Assert.assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyHosts()
+    assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyHosts()
         .get("hadoop.proxyuser.test.hosts").size() == 1);
-    Assert.assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyHosts()
+    assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyHosts()
         .get("hadoop.proxyuser.test.hosts").contains("test_hosts_1"));
   }
 
@@ -772,11 +773,11 @@ public class TestRMAdminService {
     List<String> groupBefore =
         new ArrayList<String>(Groups.getUserToGroupsMappingService(
             configuration).getGroups(user));
-    Assert.assertTrue(groupBefore.contains("test_group_A")
+    assertTrue(groupBefore.contains("test_group_A")
         && groupBefore.contains("test_group_B")
         && groupBefore.contains("test_group_C") && groupBefore.size() == 3);
-    Assert.assertTrue(groupWithInit.size() != groupBefore.size());
-    Assert.assertFalse(groupWithInit.contains("test_group_A")
+    assertTrue(groupWithInit.size() != groupBefore.size());
+    assertFalse(groupWithInit.contains("test_group_A")
         || groupWithInit.contains("test_group_B")
         || groupWithInit.contains("test_group_C"));
 
@@ -790,7 +791,7 @@ public class TestRMAdminService {
         Groups.getUserToGroupsMappingService(configuration).getGroups(user);
 
     // should get the updated groups
-    Assert.assertTrue(groupAfter.contains("test_group_D")
+    assertTrue(groupAfter.contains("test_group_D")
         && groupAfter.contains("test_group_E")
         && groupAfter.contains("test_group_F") && groupAfter.size() == 3);
 
@@ -832,7 +833,7 @@ public class TestRMAdminService {
       excludeHostsFile.delete();
     }
     if (!excludeHostsFile.createNewFile()) {
-      Assert.fail("Can not create " + "excludeHosts");
+      fail("Can not create " + "excludeHosts");
     }
     PrintWriter fileWriter = new PrintWriter(excludeHostsFile);
     fileWriter.write("0.0.0.0:123");
@@ -849,8 +850,8 @@ public class TestRMAdminService {
         .newInstance(DecommissionType.NORMAL));
     Set<String> excludeHosts =
         rm.getNodesListManager().getHostsReader().getExcludedHosts();
-    Assert.assertTrue(excludeHosts.size() == 1);
-    Assert.assertTrue(excludeHosts.contains("0.0.0.0:123"));
+    assertTrue(excludeHosts.size() == 1);
+    assertTrue(excludeHosts.contains("0.0.0.0:123"));
   }
 
   @Test
@@ -873,17 +874,17 @@ public class TestRMAdminService {
       rm1 = new MockRM(conf1);
       rm1.init(conf1);
       rm1.start();
-      Assert.assertTrue(rm1.getRMContext().getHAServiceState()
+      assertTrue(rm1.getRMContext().getHAServiceState()
           == HAServiceState.STANDBY);
 
       rm2 = new MockRM(conf2);
       rm2.init(conf1);
       rm2.start();
-      Assert.assertTrue(rm2.getRMContext().getHAServiceState()
+      assertTrue(rm2.getRMContext().getHAServiceState()
           == HAServiceState.STANDBY);
 
       rm1.adminService.transitionToActive(requestInfo);
-      Assert.assertTrue(rm1.getRMContext().getHAServiceState()
+      assertTrue(rm1.getRMContext().getHAServiceState()
           == HAServiceState.ACTIVE);
 
       CapacitySchedulerConfiguration csConf =
@@ -897,28 +898,28 @@ public class TestRMAdminService {
       int maxApps =
           ((CapacityScheduler) rm1.getRMContext().getScheduler())
               .getConfiguration().getMaximumSystemApplications();
-      Assert.assertEquals(maxApps, 5000);
+      assertEquals(maxApps, 5000);
 
       // Before failover happens, the maxApps is
       // still the default value on the standby rm : rm2
       int maxAppsBeforeFailOver =
           ((CapacityScheduler) rm2.getRMContext().getScheduler())
               .getConfiguration().getMaximumSystemApplications();
-      Assert.assertEquals(maxAppsBeforeFailOver, 10000);
+      assertEquals(maxAppsBeforeFailOver, 10000);
 
       // Do the failover
       rm1.adminService.transitionToStandby(requestInfo);
       rm2.adminService.transitionToActive(requestInfo);
-      Assert.assertTrue(rm1.getRMContext().getHAServiceState()
+      assertTrue(rm1.getRMContext().getHAServiceState()
           == HAServiceState.STANDBY);
-      Assert.assertTrue(rm2.getRMContext().getHAServiceState()
+      assertTrue(rm2.getRMContext().getHAServiceState()
           == HAServiceState.ACTIVE);
 
       int maxAppsAfter =
           ((CapacityScheduler) rm2.getRMContext().getScheduler())
               .getConfiguration().getMaximumSystemApplications();
 
-      Assert.assertEquals(maxAppsAfter, 5000);
+      assertEquals(maxAppsAfter, 5000);
     } finally {
       if (rm1 != null) {
         rm1.stop();
@@ -1034,7 +1035,7 @@ public class TestRMAdminService {
       excludeHostsFile.delete();
     }
     if (!excludeHostsFile.createNewFile()) {
-      Assert.fail("Can not create " + "excludeHosts");
+      fail("Can not create " + "excludeHosts");
     }
     PrintWriter fileWriter = new PrintWriter(excludeHostsFile);
     fileWriter.write("0.0.0.0:123");
@@ -1085,21 +1086,21 @@ public class TestRMAdminService {
       Set<String> excludeHosts =
           resourceManager.getRMContext().getNodesListManager()
               .getHostsReader().getExcludedHosts();
-      Assert.assertTrue(excludeHosts.size() == 1);
-      Assert.assertTrue(excludeHosts.contains("0.0.0.0:123"));
+      assertTrue(excludeHosts.size() == 1);
+      assertTrue(excludeHosts.contains("0.0.0.0:123"));
 
       // validate values for admin-acls
       String aclStringAfter =
           resourceManager.adminService.getAccessControlList()
               .getAclString().trim();
-      Assert.assertEquals(aclStringAfter, "world:anyone:rwcda," +
+      assertEquals(aclStringAfter, "world:anyone:rwcda," +
           UserGroupInformation.getCurrentUser().getShortUserName());
 
       // validate values for queue configuration
       CapacityScheduler cs =
           (CapacityScheduler) resourceManager.getRMContext().getScheduler();
       int maxAppsAfter = cs.getConfiguration().getMaximumSystemApplications();
-      Assert.assertEquals(maxAppsAfter, 5000);
+      assertEquals(maxAppsAfter, 5000);
 
       // verify service Acls for AdminService
       ServiceAuthorizationManager adminServiceServiceManager =
@@ -1135,14 +1136,14 @@ public class TestRMAdminService {
 
       // verify ProxyUsers and ProxyHosts
       ProxyUsers.refreshSuperUserGroupsConfiguration(configuration);
-      Assert.assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyGroups()
+      assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyGroups()
           .get("hadoop.proxyuser.test.groups").size() == 1);
-      Assert.assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyGroups()
+      assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyGroups()
           .get("hadoop.proxyuser.test.groups").contains("test_groups"));
 
-      Assert.assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyHosts()
+      assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyHosts()
           .get("hadoop.proxyuser.test.hosts").size() == 1);
-      Assert.assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyHosts()
+      assertTrue(ProxyUsers.getDefaultImpersonationProvider().getProxyHosts()
           .get("hadoop.proxyuser.test.hosts").contains("test_hosts"));
 
       // verify UserToGroupsMappings
@@ -1150,7 +1151,7 @@ public class TestRMAdminService {
       List<String> groupAfter =
           Groups.getUserToGroupsMappingService(configuration).getGroups(
               UserGroupInformation.getCurrentUser().getUserName());
-      Assert.assertTrue(groupAfter.contains("test_group_D")
+      assertTrue(groupAfter.contains("test_group_D")
           && groupAfter.contains("test_group_E")
           && groupAfter.contains("test_group_F") && groupAfter.size() == 3);
     } finally {
@@ -1219,25 +1220,27 @@ public class TestRMAdminService {
     rm.close();
   }
 
-  @Test(expected = YarnException.class)
+  @Test
   public void testModifyLabelsOnNodesWithCentralizedConfigurationDisabled()
       throws IOException, YarnException {
-    // create RM and set it's ACTIVE, and set distributed node label
-    // configuration to true
-    MockRM rm = new MockRM();
-    rm.adminService.isCentralizedNodeLabelConfiguration = false;
-
-    ((RMContextImpl) rm.getRMContext())
-        .setHAServiceState(HAServiceState.ACTIVE);
-    RMNodeLabelsManager labelMgr = rm.rmContext.getNodeLabelManager();
-
-    // by default, distributed configuration for node label is disabled, this
-    // should pass
-    labelMgr.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet.of("x", "y"));
-    rm.adminService.replaceLabelsOnNode(ReplaceLabelsOnNodeRequest
-        .newInstance(ImmutableMap.of(NodeId.newInstance("host", 0),
-            (Set<String>) ImmutableSet.of("x"))));
-    rm.close();
+    assertThrows(YarnException.class, () -> {
+      // create RM and set it's ACTIVE, and set distributed node label
+      // configuration to true
+      MockRM mockRM = new MockRM();
+      mockRM.adminService.isCentralizedNodeLabelConfiguration = false;
+
+      ((RMContextImpl) mockRM.getRMContext())
+      .setHAServiceState(HAServiceState.ACTIVE);
+      RMNodeLabelsManager labelMgr = mockRM.rmContext.getNodeLabelManager();
+
+      // by default, distributed configuration for node label is disabled, this
+      // should pass
+      labelMgr.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet.of("x", "y"));
+      mockRM.adminService.replaceLabelsOnNode(ReplaceLabelsOnNodeRequest
+          .newInstance(ImmutableMap.of(NodeId.newInstance("host", 0),
+          (Set<String>) ImmutableSet.of("x"))));
+      mockRM.close();
+    });
   }
 
   @Test
@@ -1348,7 +1351,8 @@ public class TestRMAdminService {
     rm.close();
   }
 
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testAdminRefreshClusterMaxPriority() throws Exception,
       YarnException {
     configuration.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS,
@@ -1364,7 +1368,7 @@ public class TestRMAdminService {
     rm.start();
 
     CapacityScheduler cs = (CapacityScheduler) rm.getRMContext().getScheduler();
-    Assert.assertEquals(5, cs.getMaxClusterLevelAppPriority().getPriority());
+    assertEquals(5, cs.getMaxClusterLevelAppPriority().getPriority());
 
     yarnConf = new YarnConfiguration();
     yarnConf
@@ -1375,7 +1379,7 @@ public class TestRMAdminService {
       rm.adminService
           .refreshClusterMaxPriority(RefreshClusterMaxPriorityRequest
               .newInstance());
-      Assert.assertEquals(10, cs.getMaxClusterLevelAppPriority().getPriority());
+      assertEquals(10, cs.getMaxClusterLevelAppPriority().getPriority());
     } catch (Exception ex) {
       fail("Could not refresh cluster max priority.");
     }
@@ -1390,7 +1394,7 @@ public class TestRMAdminService {
         confFile.delete();
       }
       if (!confFile.createNewFile()) {
-        Assert.fail("Can not create " + confXMLName);
+        fail("Can not create " + confXMLName);
       }
       output = new DataOutputStream(
           new FileOutputStream(confFile));
@@ -1495,7 +1499,7 @@ public class TestRMAdminService {
     try {
       resourceManager.init(configuration);
       resourceManager.start();
-      Assert.assertTrue(resourceManager.getRMContext().getHAServiceState()
+      assertTrue(resourceManager.getRMContext().getHAServiceState()
           == HAServiceState.STANDBY);
       resourceManager.adminService.transitionToActive(requestInfo);
     } finally {
@@ -1537,12 +1541,13 @@ public class TestRMAdminService {
       rm1.init(conf);
       fail("The RM allowed an invalid configuration");
     } catch (YarnRuntimeException e) {
-      assertTrue("The RM initialization threw an unexpected exception",
-          e.getMessage().startsWith(HAUtil.BAD_CONFIG_MESSAGE_PREFIX));
+      assertTrue(e.getMessage().startsWith(HAUtil.BAD_CONFIG_MESSAGE_PREFIX),
+          "The RM initialization threw an unexpected exception");
     }
   }
 
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testAdminAddToClusterNodeLabelsWithDeprecatedAPIs()
       throws Exception, YarnException {
     configuration.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS,
@@ -1586,11 +1591,13 @@ public class TestRMAdminService {
         .getClusterNodeLabels(GetClusterNodeLabelsRequest.newInstance());
     NodeLabel labelX = NodeLabel.newInstance("a");
     NodeLabel labelY = NodeLabel.newInstance("b");
-    Assert.assertTrue(
-        response.getNodeLabelList().containsAll(Arrays.asList(labelX, labelY)));
+    assertTrue(response.getNodeLabelList().containsAll(
+        Arrays.asList(labelX, labelY)));
   }
 
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
+  @SuppressWarnings("checkstyle:MethodLength")
   public void testMapAttributesToNodes() throws Exception, YarnException {
     // 1. Need to test for the Invalid Node
     // 1.1. Need to test for active nodes
@@ -1655,9 +1662,9 @@ public class TestRMAdminService {
         .removeNodeAttributes(Mockito.anyMap());
 
     // Assert node to attributes mappings are empty.
-    Assert.assertTrue("Attributes of host4 should be empty",
-        rm.getRMContext().getNodeAttributesManager()
-            .getAttributesForNode("host4").isEmpty());
+    assertTrue(rm.getRMContext().getNodeAttributesManager()
+        .getAttributesForNode("host4").isEmpty(),
+        "Attributes of host4 should be empty");
     // remove non existing attributes.
     request = NodesToAttributesMappingRequest
         .newInstance(AttributeMappingOperationType.REMOVE, ImmutableList
@@ -1669,7 +1676,7 @@ public class TestRMAdminService {
       rm.adminService.mapAttributesToNodes(request);
       fail("Should have failed for non exists attribute");
     } catch (Exception ex) {
-      assertTrue("Exception expected if attributes does not exist", true);
+      assertTrue(true, "Exception expected if attributes does not exist");
     }
 
     request =
@@ -1684,9 +1691,8 @@ public class TestRMAdminService {
       rm.adminService.mapAttributesToNodes(request);
       fail("host5 is not a valid node, It should have failed");
     } catch (YarnException ex) {
-      Assert.assertEquals("Exception Message is not as desired",
-          " Following nodes does not exist : [host5]",
-          ex.getCause().getMessage());
+      assertEquals(" Following nodes does not exist : [host5]",
+          ex.getCause().getMessage(), "Exception Message is not as desired");
     }
 
     request =
@@ -1707,9 +1713,8 @@ public class TestRMAdminService {
       rm.adminService.mapAttributesToNodes(request);
       fail("host with the port should fail as only hostnames are validated");
     } catch (YarnException ex) {
-      Assert.assertEquals("Exception Message is not as desired",
-          " Following nodes does not exist : [host4:8889, host2:8889]",
-          ex.getCause().getMessage());
+      assertEquals(" Following nodes does not exist : [host4:8889, host2:8889]",
+          ex.getCause().getMessage(), "Exception Message is not as desired");
     }
 
     request =
@@ -1743,9 +1748,10 @@ public class TestRMAdminService {
       rm.adminService.mapAttributesToNodes(request);
       fail("This operation should fail as prefix should be \"nm.yarn.io\".");
     } catch (YarnException ex) {
-      Assert.assertEquals("Exception Message is not as desired",
+      assertEquals(
           "Invalid Attribute Mapping for the node host5. Prefix should be "
-              + "rm.yarn.io", ex.getCause().getMessage());
+          + "rm.yarn.io", ex.getCause().getMessage(),
+          "Exception Message is not as desired");
     }
 
     rm.close();

+ 9 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAuditLogger.java

@@ -17,7 +17,9 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager;
 
-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.fail;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -46,9 +48,8 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.Keys;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 
 /**
@@ -69,7 +70,7 @@ public class TestRMAuditLogger {
   private static final byte[] CALLER_SIGNATURE = "signature".getBytes();
   private static final String PARTITION = "label1";
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     when(APPID.toString()).thenReturn("app_1");
     when(ATTEMPTID.toString()).thenReturn("app_attempt_1");
@@ -242,7 +243,7 @@ public class TestRMAuditLogger {
     } catch (UnknownHostException uhe) {
       // should not happen as long as IP address format
       // stays the same
-      Assert.fail("Check ip address being constructed");
+      fail("Check ip address being constructed");
     }
     testSuccessLogFormatHelperWithIP(checkIP, appId, attemptId, containerId,
         addr);
@@ -404,8 +405,8 @@ public class TestRMAuditLogger {
         throws ServiceException {
       // Ensure clientId is received
       byte[] clientId = Server.getClientId();
-      Assert.assertNotNull(clientId);
-      Assert.assertEquals(ClientId.BYTE_LENGTH, clientId.length);
+      assertNotNull(clientId);
+      assertEquals(ClientId.BYTE_LENGTH, clientId.length);
       // test with ip set
       testSuccessLogFormat(true);
       testFailureLogFormat(true);

+ 6 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMDispatcher.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.Mockito.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
@@ -33,13 +34,14 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerPreemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 public class TestRMDispatcher {
 
   @SuppressWarnings("unchecked")
-  @Test(timeout=10000)
+  @Test
+  @Timeout(value = 10)
   public void testSchedulerEventDispatcherForPreemptionEvents() {
     AsyncDispatcher rmDispatcher = new AsyncDispatcher();
     CapacityScheduler sched = spy(new CapacityScheduler());
@@ -73,7 +75,7 @@ public class TestRMDispatcher {
       verify(sched).markContainerForPreemption(appAttemptId, container);
       verify(sched).markContainerForKillable(container);
     } catch (InterruptedException e) {
-      Assert.fail();
+      fail();
     } finally {
       schedulerDispatcher.stop();
       rmDispatcher.stop();

+ 90 - 90
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java

@@ -27,10 +27,11 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
@@ -71,9 +72,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptS
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import org.glassfish.jersey.jettison.internal.entity.JettisonObjectProvider;
 
@@ -96,7 +97,7 @@ public class TestRMHA {
   private static final String RM3_ADDRESS = "2.2.2.2:2";
   private static final String RM3_NODE_ID = "rm3";
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     configuration = new Configuration();
     UserGroupInformation.setConfiguration(configuration);
@@ -128,21 +129,20 @@ public class TestRMHA {
   }
 
   private void checkStandbyRMFunctionality() throws IOException {
-    assertEquals(STATE_ERR, HAServiceState.STANDBY,
-        rm.adminService.getServiceStatus().getState());
-    assertFalse("Active RM services are started",
-        rm.areActiveServicesRunning());
-    assertTrue("RM is not ready to become active",
-        rm.adminService.getServiceStatus().isReadyToBecomeActive());
+    assertEquals(HAServiceState.STANDBY,
+        rm.adminService.getServiceStatus().getState(), STATE_ERR);
+    assertFalse(rm.areActiveServicesRunning(),
+        "Active RM services are started");
+    assertTrue(rm.adminService.getServiceStatus().isReadyToBecomeActive(),
+        "RM is not ready to become active");
   }
 
   private void checkActiveRMFunctionality() throws Exception {
-    assertEquals(STATE_ERR, HAServiceState.ACTIVE,
-        rm.adminService.getServiceStatus().getState());
-    assertTrue("Active RM services aren't started",
-        rm.areActiveServicesRunning());
-    assertTrue("RM is not ready to become active",
-        rm.adminService.getServiceStatus().isReadyToBecomeActive());
+    assertEquals(HAServiceState.ACTIVE,
+        rm.adminService.getServiceStatus().getState(), STATE_ERR);
+    assertTrue(rm.areActiveServicesRunning(), "Active RM services aren't started");
+    assertTrue(rm.adminService.getServiceStatus().isReadyToBecomeActive(),
+        "RM is not ready to become active");
 
     try {
       rm.getNewAppId();
@@ -180,7 +180,7 @@ public class TestRMHA {
         response.getMediaType().toString());
     JSONObject json = response.readEntity(JSONObject.class);
 
-    assertEquals("incorrect number of elements", 1, json.length());
+    assertEquals(1, json.length(), "incorrect number of elements");
     JSONObject appJson = json.getJSONObject("app");
     assertEquals("ACCEPTED", appJson.getString("state"));
     // Other stuff is verified in the regular web-services related tests
@@ -197,7 +197,8 @@ public class TestRMHA {
    * 6. Stop the RM: All services should stop and RM should not be ready to
    * become Active
    */
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testFailoverAndTransitions() throws Exception {
     configuration.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
     Configuration conf = new YarnConfiguration(configuration);
@@ -207,10 +208,10 @@ public class TestRMHA {
     StateChangeRequestInfo requestInfo = new StateChangeRequestInfo(
         HAServiceProtocol.RequestSource.REQUEST_BY_USER);
 
-    assertEquals(STATE_ERR, HAServiceState.INITIALIZING,
-        rm.adminService.getServiceStatus().getState());
-    assertFalse("RM is ready to become active before being started",
-        rm.adminService.getServiceStatus().isReadyToBecomeActive());
+    assertEquals(HAServiceState.INITIALIZING,
+        rm.adminService.getServiceStatus().getState(), STATE_ERR);
+    assertFalse(rm.adminService.getServiceStatus().isReadyToBecomeActive(),
+        "RM is ready to become active before being started");
     checkMonitorHealth();
 
     rm.start();
@@ -251,12 +252,11 @@ public class TestRMHA {
     // 6. Stop the RM. All services should stop and RM should not be ready to
     // become active
     rm.stop();
-    assertEquals(STATE_ERR, HAServiceState.STOPPING,
-        rm.adminService.getServiceStatus().getState());
-    assertFalse("RM is ready to become active even after it is stopped",
-        rm.adminService.getServiceStatus().isReadyToBecomeActive());
-    assertFalse("Active RM services are started",
-        rm.areActiveServicesRunning());
+    assertEquals(HAServiceState.STOPPING,
+        rm.adminService.getServiceStatus().getState(), STATE_ERR);
+    assertFalse(rm.adminService.getServiceStatus().isReadyToBecomeActive(),
+        "RM is ready to become active even after it is stopped");
+    assertFalse(rm.areActiveServicesRunning(), "Active RM services are started");
     checkMonitorHealth();
   }
 
@@ -341,10 +341,10 @@ public class TestRMHA {
     StateChangeRequestInfo requestInfo = new StateChangeRequestInfo(
         HAServiceProtocol.RequestSource.REQUEST_BY_USER);
 
-    assertEquals(STATE_ERR, HAServiceState.INITIALIZING,
-        rm.adminService.getServiceStatus().getState());
-    assertFalse("RM is ready to become active before being started",
-        rm.adminService.getServiceStatus().isReadyToBecomeActive());
+    assertEquals(HAServiceState.INITIALIZING,
+        rm.adminService.getServiceStatus().getState(), STATE_ERR);
+    assertFalse(rm.adminService.getServiceStatus().isReadyToBecomeActive(),
+        "RM is ready to become active before being started");
     rm.start();
 
     //call transitions to standby and active a couple of times
@@ -359,11 +359,11 @@ public class TestRMHA {
     assertTrue(!dispatcher.isStopped());
 
     rm.adminService.transitionToActive(requestInfo);
-    assertEquals(errorMessageForEventHandler, expectedEventHandlerCount,
+    assertEquals(expectedEventHandlerCount,
         ((MyCountingDispatcher) rm.getRMContext().getDispatcher())
-            .getEventHandlerCount());
-    assertEquals(errorMessageForService, expectedServiceCount,
-        rm.getServices().size());
+        .getEventHandlerCount(), errorMessageForEventHandler);
+    assertEquals(expectedServiceCount,
+        rm.getServices().size(), errorMessageForService);
 
 
     // Keep the dispatcher reference before transitioning to standby
@@ -371,11 +371,11 @@ public class TestRMHA {
 
 
     rm.adminService.transitionToStandby(requestInfo);
-    assertEquals(errorMessageForEventHandler, expectedEventHandlerCount,
+    assertEquals(expectedEventHandlerCount,
         ((MyCountingDispatcher) rm.getRMContext().getDispatcher())
-            .getEventHandlerCount());
-    assertEquals(errorMessageForService, expectedServiceCount,
-        rm.getServices().size());
+        .getEventHandlerCount(), errorMessageForEventHandler);
+    assertEquals(expectedServiceCount,
+        rm.getServices().size(), errorMessageForService);
 
     assertTrue(dispatcher.isStopped());
 
@@ -408,7 +408,7 @@ public class TestRMHA {
       rm.init(conf);
       fail("Should get an exception here.");
     } catch (Exception ex) {
-      Assert.assertTrue(ex.getMessage().contains(
+      assertTrue(ex.getMessage().contains(
           "Invalid configuration! Can not find valid RM_HA_ID."));
     }
   }
@@ -421,7 +421,8 @@ public class TestRMHA {
     innerTestHAWithRMHostName(true);
   }
 
-  @Test(timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testFailoverWhenTransitionToActiveThrowException()
       throws Exception {
     configuration.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
@@ -447,10 +448,10 @@ public class TestRMHA {
         new StateChangeRequestInfo(
             HAServiceProtocol.RequestSource.REQUEST_BY_USER);
 
-    assertEquals(STATE_ERR, HAServiceState.INITIALIZING, rm.adminService
-        .getServiceStatus().getState());
-    assertFalse("RM is ready to become active before being started",
-        rm.adminService.getServiceStatus().isReadyToBecomeActive());
+    assertEquals(HAServiceState.INITIALIZING, rm.adminService
+        .getServiceStatus().getState(), STATE_ERR);
+    assertFalse(rm.adminService.getServiceStatus().isReadyToBecomeActive(),
+        "RM is ready to become active before being started");
     checkMonitorHealth();
 
     rm.start();
@@ -460,7 +461,7 @@ public class TestRMHA {
     // 2. Try Transition to active, throw exception
     try {
       rm.adminService.transitionToActive(requestInfo);
-      Assert.fail("Transitioned to Active should throw exception.");
+      fail("Transitioned to Active should throw exception.");
     } catch (Exception e) {
       assertTrue("Error when transitioning to Active mode".contains(e
           .getMessage()));
@@ -500,10 +501,10 @@ public class TestRMHA {
         new StateChangeRequestInfo(
             HAServiceProtocol.RequestSource.REQUEST_BY_USER);
 
-    assertEquals(STATE_ERR, HAServiceState.INITIALIZING, rm.adminService
-        .getServiceStatus().getState());
-    assertFalse("RM is ready to become active before being started",
-        rm.adminService.getServiceStatus().isReadyToBecomeActive());
+    assertEquals(HAServiceState.INITIALIZING, rm.adminService
+        .getServiceStatus().getState(), STATE_ERR);
+    assertFalse(rm.adminService.getServiceStatus().isReadyToBecomeActive(),
+        "RM is ready to become active before being started");
     checkMonitorHealth();
 
     rm.start();
@@ -562,7 +563,7 @@ public class TestRMHA {
     verifyClusterMetrics(1, 1, 1, 1, 2048, 1);
     assertEquals(1, rm.getRMContext().getRMNodes().size());
     assertEquals(1, rm.getRMContext().getRMApps().size());
-    Assert.assertNotNull("Node not registered", nm);
+    assertNotNull(nm, "Node not registered");
 
     rm.adminService.transitionToStandby(requestInfo);
     checkMonitorHealth();
@@ -597,7 +598,7 @@ public class TestRMHA {
     // 4. Try Transition to active, throw exception
     try {
       rm.adminService.transitionToActive(requestInfo);
-      Assert.fail("Transitioned to Active should throw exception.");
+      fail("Transitioned to Active should throw exception.");
     } catch (Exception e) {
       assertTrue("Error when transitioning to Active mode".contains(e
           .getMessage()));
@@ -608,7 +609,8 @@ public class TestRMHA {
     assertEquals(0, rm.getRMContext().getRMApps().size());
   }
 
-  @Test(timeout = 9000000)
+  @Test
+  @Timeout(value = 9000)
   public void testTransitionedToActiveRefreshFail() throws Exception {
     configuration.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
     rm = new MockRM(configuration) {
@@ -648,21 +650,21 @@ public class TestRMHA {
         ((FailFastDispatcher) rm.rmContext.getDispatcher());
     // Verify transition to transitionToStandby
     rm.adminService.transitionToStandby(requestInfo);
-    assertEquals("Fatal Event should be 0", 0, dispatcher.getEventCount());
-    assertEquals("HA state should be in standBy State", HAServiceState.STANDBY,
-        rm.getRMContext().getHAServiceState());
+    assertEquals(0, dispatcher.getEventCount(), "Fatal Event should be 0");
+    assertEquals(HAServiceState.STANDBY,
+        rm.getRMContext().getHAServiceState(), "HA state should be in standBy State");
     try {
       // Verify refreshAll call failure and check fail Event is dispatched
       rm.adminService.transitionToActive(requestInfo);
-      Assert.fail("Transition to Active should have failed for refreshAll()");
+      fail("Transition to Active should have failed for refreshAll()");
     } catch (Exception e) {
-      assertTrue("Service fail Exception expected",
-          e instanceof ServiceFailedException);
+      assertTrue(e instanceof ServiceFailedException,
+          "Service fail Exception expected");
     }
     // Since refreshAll failed we are expecting fatal event to be send
     // Then fatal event is send RM will shutdown
     dispatcher.await();
-    assertEquals("Fatal Event to be received", 1, dispatcher.getEventCount());
+    assertEquals(1, dispatcher.getEventCount(), "Fatal Event to be received");
     // Check of refreshAll success HA can be active
     rm.adminService.transitionToActive(requestInfo);
     assertEquals(HAServiceState.ACTIVE, rm.getRMContext().getHAServiceState());
@@ -706,7 +708,7 @@ public class TestRMHA {
       }
     }, 100, 3000);
     rm.stop();
-    Assert.assertEquals(1, appMaster.getLeastLoadedNodes().size());
+    assertEquals(1, appMaster.getLeastLoadedNodes().size());
 
   }
 
@@ -744,9 +746,8 @@ public class TestRMHA {
     checkActiveRMFunctionality();
 
     // 5. Check ResourceProfilesManager
-    Assert.assertNotNull(
-        "ResourceProfilesManager should not be null!",
-        rm.getRMContext().getResourceProfilesManager());
+    assertNotNull(rm.getRMContext().getResourceProfilesManager(),
+        "ResourceProfilesManager should not be null!");
   }
 
   @Test
@@ -769,12 +770,12 @@ public class TestRMHA {
       if (confFile.exists()) {
         hasRenamed = confFile.renameTo(backupConfFile);
         if (!hasRenamed) {
-          Assert.fail("Can not rename " + confFile.getAbsolutePath() + " to "
+          fail("Can not rename " + confFile.getAbsolutePath() + " to "
               + backupConfFile.getAbsolutePath());
         }
       }
       if (!confFile.createNewFile()) {
-        Assert.fail(
+        fail(
             "Can not create " + YarnConfiguration.YARN_SITE_CONFIGURATION_FILE);
       }
       output = new DataOutputStream(Files.newOutputStream(confFile.toPath()));
@@ -867,20 +868,20 @@ public class TestRMHA {
       rm = new MockRM(conf);
       rm.init(conf);
       for (String confKey : YarnConfiguration.getServiceAddressConfKeys(conf)) {
-        assertEquals("RPC address not set for " + confKey,
-            RM1_ADDRESS, conf.get(HAUtil.addSuffix(confKey, RM1_NODE_ID)));
-        assertEquals("RPC address not set for " + confKey,
-            RM2_ADDRESS, conf.get(HAUtil.addSuffix(confKey, RM2_NODE_ID)));
-        assertEquals("RPC address not set for " + confKey,
-            RM3_ADDRESS, conf.get(HAUtil.addSuffix(confKey, RM3_NODE_ID)));
+        assertEquals(RM1_ADDRESS, conf.get(HAUtil.addSuffix(confKey, RM1_NODE_ID)),
+            "RPC address not set for " + confKey);
+        assertEquals(RM2_ADDRESS, conf.get(HAUtil.addSuffix(confKey, RM2_NODE_ID)),
+            "RPC address not set for " + confKey);
+        assertEquals(RM3_ADDRESS, conf.get(HAUtil.addSuffix(confKey, RM3_NODE_ID)),
+            "RPC address not set for " + confKey);
         if (includeBindHost) {
-          assertEquals("Web address misconfigured WITH bind-host",
-                       rm.webAppAddress.substring(0, 7), "9.9.9.9");
+          assertEquals(rm.webAppAddress.substring(0, 7), "9.9.9.9",
+              "Web address misconfigured WITH bind-host");
         } else {
           //YarnConfiguration tries to figure out which rm host it's on by binding to it,
           //which doesn't happen for any of these fake addresses, so we end up with 0.0.0.0
-          assertEquals("Web address misconfigured WITHOUT bind-host",
-                       rm.webAppAddress.substring(0, 7), "0.0.0.0");
+          assertEquals(rm.webAppAddress.substring(0, 7), "0.0.0.0",
+              "Web address misconfigured WITHOUT bind-host");
         }
       }
     } catch (YarnRuntimeException e) {
@@ -900,13 +901,12 @@ public class TestRMHA {
       Configuration conf = new YarnConfiguration(configuration);
       rm = new MockRM(conf);
       rm.init(conf);
-      assertEquals("RPC address not set for " + YarnConfiguration.RM_ADDRESS,
-          "1.1.1.1:8032",
-          conf.get(HAUtil.addSuffix(YarnConfiguration.RM_ADDRESS, RM1_NODE_ID)));
-      assertEquals("RPC address not set for " + YarnConfiguration.RM_ADDRESS,
-          "0.0.0.0:8032",
-          conf.get(HAUtil.addSuffix(YarnConfiguration.RM_ADDRESS, RM2_NODE_ID)));
-
+      assertEquals("1.1.1.1:8032",
+          conf.get(HAUtil.addSuffix(YarnConfiguration.RM_ADDRESS, RM1_NODE_ID)),
+          "RPC address not set for " + YarnConfiguration.RM_ADDRESS);
+      assertEquals("0.0.0.0:8032",
+          conf.get(HAUtil.addSuffix(YarnConfiguration.RM_ADDRESS, RM2_NODE_ID)),
+          "RPC address not set for " + YarnConfiguration.RM_ADDRESS);
     } catch (YarnRuntimeException e) {
       fail("Should not throw any exceptions.");
     }
@@ -941,11 +941,11 @@ public class TestRMHA {
         Thread.sleep(1000);
       }
     }
-    assertTrue(message, isAllMetricAssertionDone);
+    assertTrue(isAllMetricAssertionDone, message);
   }
 
   private void assertMetric(String metricName, long expected, long actual) {
-    assertEquals("Incorrect value for metric " + metricName, expected, actual);
+    assertEquals(expected, actual, "Incorrect value for metric " + metricName);
   }
 
   @SuppressWarnings("rawtypes")

+ 10 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHAForNodeLabels.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
 import java.io.File;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -29,9 +31,8 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet;
 
@@ -39,7 +40,7 @@ public class TestRMHAForNodeLabels extends RMHATestBase {
   public static final Logger LOG = LoggerFactory
       .getLogger(TestRMHAForNodeLabels.class);
 
-  @Before
+  @BeforeEach
   @Override
   public void setup() throws Exception {
     super.setup();
@@ -81,13 +82,12 @@ public class TestRMHAForNodeLabels extends RMHATestBase {
     explicitFailover();
 
     // Check labels in rm2
-    Assert
-        .assertTrue(rm2.getRMContext().getNodeLabelManager()
-            .getClusterNodeLabelNames()
-            .containsAll(ImmutableSet.of("a", "b", "c")));
-    Assert.assertTrue(rm2.getRMContext().getNodeLabelManager()
+    assertTrue(rm2.getRMContext().getNodeLabelManager()
+        .getClusterNodeLabelNames()
+        .containsAll(ImmutableSet.of("a", "b", "c")));
+    assertTrue(rm2.getRMContext().getNodeLabelManager()
         .getNodeLabels().get(NodeId.newInstance("host1", 0)).contains("a"));
-    Assert.assertTrue(rm2.getRMContext().getNodeLabelManager()
+    assertTrue(rm2.getRMContext().getNodeLabelManager()
         .getNodeLabels().get(NodeId.newInstance("host2", 0)).contains("b"));
   }
 }

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

@@ -23,15 +23,16 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.conf.HAUtil;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
 import java.lang.management.ManagementFactory;
 
-import static junit.framework.TestCase.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
 
 /**
  * Metrics related RM HA testing. Metrics are mostly static singletons. To
@@ -47,7 +48,7 @@ public class TestRMHAMetrics {
   private static final String RM2_ADDRESS = "0.0.0.0:0";
   private static final String RM2_NODE_ID = "rm2";
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     configuration = new Configuration();
     UserGroupInformation.setConfiguration(configuration);
@@ -65,7 +66,8 @@ public class TestRMHAMetrics {
     DefaultMetricsSystem.shutdown();
   }
 
-  @Test(timeout = 300000)
+  @Test
+  @Timeout(value = 300)
   public void testMetricsAfterTransitionToStandby() throws Exception {
     configuration.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
     Configuration conf = new YarnConfiguration(configuration);
@@ -76,20 +78,19 @@ public class TestRMHAMetrics {
     ObjectName mxbeanName =
         new ObjectName("Hadoop:service=ResourceManager,name=RMInfo");
 
-    Assert.assertEquals("initializing",
+    assertEquals("initializing",
         (String) mbs.getAttribute(mxbeanName, "State"));
 
     rm.start();
-    Assert.assertEquals("standby",
+    assertEquals("standby",
         (String) mbs.getAttribute(mxbeanName, "State"));
 
     rm.transitionToActive();
-    Assert
-        .assertEquals("active",
+    assertEquals("active",
             (String) mbs.getAttribute(mxbeanName, "State"));
 
     rm.transitionToStandby(true);
-    Assert.assertEquals("standby",
+    assertEquals("standby",
         (String) mbs.getAttribute(mxbeanName, "State"));
 
     assertNotNull(DefaultMetricsSystem.instance().getSource("JvmMetrics"));

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

@@ -25,8 +25,8 @@ import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 import java.util.function.Supplier;
 
@@ -34,7 +34,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 /**
  * Test if the new active RM could recover collector status on a state
@@ -42,7 +42,7 @@ import static org.junit.Assert.assertEquals;
  */
 public class TestRMHATimelineCollectors extends RMHATestBase {
 
-  @Before
+  @BeforeEach
   @Override
   public void setup() throws Exception {
     super.setup();

File diff suppressed because it is too large
+ 275 - 303
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java


+ 11 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMProxyUsersConf.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+import static org.junit.jupiter.api.Assertions.fail;
+
 import java.util.Arrays;
 import java.util.Collection;
 
@@ -27,12 +29,9 @@ import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import static org.apache.hadoop.yarn.conf.YarnConfiguration.RM_PROXY_USER_PREFIX;
-import org.junit.Assert;
-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;
 
-@RunWith(Parameterized.class)
 public class TestRMProxyUsersConf {
 
   private static final UserGroupInformation FOO_USER =
@@ -41,14 +40,13 @@ public class TestRMProxyUsersConf {
       UserGroupInformation.createUserForTesting("bar", new String[] { "bar_group" });
   private final String ipAddress = "127.0.0.1";
 
-  @Parameterized.Parameters
   public static Collection<Object[]> headers() {
-    return Arrays.asList(new Object[][] { { 0 }, { 1 }, { 2 } });
+    return Arrays.asList(new Object[][]{{0}, {1}, {2}});
   }
 
   private Configuration conf;
 
-  public TestRMProxyUsersConf(int round) {
+  public void initTestRMProxyUsersConf(int round) {
     conf = new YarnConfiguration();
     switch (round) {
     case 0:
@@ -78,8 +76,10 @@ public class TestRMProxyUsersConf {
     }
   }
 
-  @Test
-  public void testProxyUserConfiguration() throws Exception {
+  @ParameterizedTest
+  @MethodSource("headers")
+  public void testProxyUserConfiguration(int round) throws Exception {
+    initTestRMProxyUsersConf(round);
     MockRM rm = null;
     try {
       rm = new MockRM(conf);
@@ -94,7 +94,7 @@ public class TestRMProxyUsersConf {
             ipAddress);
       } catch (AuthorizationException e) {
         // Exception is not expected
-        Assert.fail();
+        fail();
       }
     } finally {
       if (rm != null) {

+ 56 - 55
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMServerUtils.java

@@ -19,6 +19,8 @@
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.apache.hadoop.yarn.api.records.ContainerUpdateType.INCREASE_RESOURCE;
 import static org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils.RESOURCE_OUTSIDE_ALLOWED_RANGE;
 
@@ -53,8 +55,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 import org.mockito.Mockito;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -112,22 +113,22 @@ public class TestRMServerUtils {
     ContainerUpdates containerUpdates =
         RMServerUtils.validateAndSplitUpdateResourceRequests(rmContext,
             allocateRequest, maxAllocation, updateErrors);
-    Assert.assertEquals(1, updateErrors.size());
-    Assert.assertEquals(resource + 1, updateErrors.get(0)
+    assertEquals(1, updateErrors.size());
+    assertEquals(resource + 1, updateErrors.get(0)
         .getUpdateContainerRequest().getCapability().getMemorySize());
-    Assert.assertEquals(resource + 1, updateErrors.get(0)
+    assertEquals(resource + 1, updateErrors.get(0)
         .getUpdateContainerRequest().getCapability().getVirtualCores());
-    Assert.assertEquals(RESOURCE_OUTSIDE_ALLOWED_RANGE,
+    assertEquals(RESOURCE_OUTSIDE_ALLOWED_RANGE,
         updateErrors.get(0).getReason());
 
-    Assert.assertEquals(1, containerUpdates.getIncreaseRequests().size());
+    assertEquals(1, containerUpdates.getIncreaseRequests().size());
     UpdateContainerRequest increaseRequest =
         containerUpdates.getIncreaseRequests().get(0);
-    Assert.assertEquals(capabilityOk.getVirtualCores(),
+    assertEquals(capabilityOk.getVirtualCores(),
         increaseRequest.getCapability().getVirtualCores());
-    Assert.assertEquals(capabilityOk.getMemorySize(),
+    assertEquals(capabilityOk.getMemorySize(),
         increaseRequest.getCapability().getMemorySize());
-    Assert.assertEquals(containerIdOk, increaseRequest.getContainerId());
+    assertEquals(containerIdOk, increaseRequest.getContainerId());
   }
 
   @Test
@@ -142,22 +143,22 @@ public class TestRMServerUtils {
     when(rmContext.getInactiveRMNodes()).thenReturn(inactiveList);
     List<RMNode> result = RMServerUtils.queryRMNodes(rmContext,
         EnumSet.of(NodeState.SHUTDOWN));
-    Assert.assertTrue(result.size() != 0);
+    assertTrue(result.size() != 0);
     assertThat(result.get(0)).isEqualTo(rmNode1);
     when(rmNode1.getState()).thenReturn(NodeState.DECOMMISSIONED);
     result = RMServerUtils.queryRMNodes(rmContext,
         EnumSet.of(NodeState.DECOMMISSIONED));
-    Assert.assertTrue(result.size() != 0);
+    assertTrue(result.size() != 0);
     assertThat(result.get(0)).isEqualTo(rmNode1);
     when(rmNode1.getState()).thenReturn(NodeState.LOST);
     result = RMServerUtils.queryRMNodes(rmContext,
         EnumSet.of(NodeState.LOST));
-    Assert.assertTrue(result.size() != 0);
+    assertTrue(result.size() != 0);
     assertThat(result.get(0)).isEqualTo(rmNode1);
     when(rmNode1.getState()).thenReturn(NodeState.REBOOTED);
     result = RMServerUtils.queryRMNodes(rmContext,
         EnumSet.of(NodeState.REBOOTED));
-    Assert.assertTrue(result.size() != 0);
+    assertTrue(result.size() != 0);
     assertThat(result.get(0)).isEqualTo(rmNode1);
   }
 
@@ -187,46 +188,46 @@ public class TestRMServerUtils {
         true, null);
     List<ResourceRequest> reqs = new ArrayList<>();
     reqs.add(anyReq);
-    Assert.assertEquals(100,
+    assertEquals(100,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
 
     ResourceRequest rackReq = createResourceRequest("/rack1", true, null);
     reqs.add(rackReq);
-    Assert.assertEquals(30,
+    assertEquals(30,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
     anyReq.setRelaxLocality(false);
-    Assert.assertEquals(30,
+    assertEquals(30,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
     rackReq.setRelaxLocality(false);
-    Assert.assertEquals(100,
+    assertEquals(100,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
 
     ResourceRequest node1Req = createResourceRequest("node1", false, null);
     reqs.add(node1Req);
-    Assert.assertEquals(100,
+    assertEquals(100,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
     node1Req.setRelaxLocality(true);
-    Assert.assertEquals(1,
+    assertEquals(1,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
     rackReq.setRelaxLocality(true);
-    Assert.assertEquals(31,
+    assertEquals(31,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
 
     ResourceRequest node2Req = createResourceRequest("node2", false, null);
     reqs.add(node2Req);
-    Assert.assertEquals(31,
+    assertEquals(31,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
     node2Req.setRelaxLocality(true);
-    Assert.assertEquals(31,
+    assertEquals(31,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
     rackReq.setRelaxLocality(false);
-    Assert.assertEquals(2,
+    assertEquals(2,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
     node1Req.setRelaxLocality(false);
-    Assert.assertEquals(1,
+    assertEquals(1,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
     node2Req.setRelaxLocality(false);
-    Assert.assertEquals(100,
+    assertEquals(100,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
   }
 
@@ -261,10 +262,10 @@ public class TestRMServerUtils {
         true, null);
     List<ResourceRequest> reqs = new ArrayList<>();
     reqs.add(anyReq);
-    Assert.assertEquals(80,
+    assertEquals(80,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
     anyReq.setNodeLabelExpression("label1");
-    Assert.assertEquals(10,
+    assertEquals(10,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
   }
 
@@ -320,46 +321,46 @@ public class TestRMServerUtils {
         true, null);
     List<ResourceRequest> reqs = new ArrayList<>();
     reqs.add(anyReq);
-    Assert.assertEquals(80,
+    assertEquals(80,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
 
     ResourceRequest rackReq = createResourceRequest("/rack1", true, null);
     reqs.add(rackReq);
-    Assert.assertEquals(20,
+    assertEquals(20,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
     anyReq.setRelaxLocality(false);
-    Assert.assertEquals(20,
+    assertEquals(20,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
     rackReq.setRelaxLocality(false);
-    Assert.assertEquals(80,
+    assertEquals(80,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
 
     ResourceRequest node1Req = createResourceRequest("node1", false, null);
     reqs.add(node1Req);
-    Assert.assertEquals(80,
+    assertEquals(80,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
     node1Req.setRelaxLocality(true);
-    Assert.assertEquals(0,
+    assertEquals(0,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
     rackReq.setRelaxLocality(true);
-    Assert.assertEquals(20,
+    assertEquals(20,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
 
     ResourceRequest node2Req = createResourceRequest("node2", false, null);
     reqs.add(node2Req);
-    Assert.assertEquals(20,
+    assertEquals(20,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
     node2Req.setRelaxLocality(true);
-    Assert.assertEquals(20,
+    assertEquals(20,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
     rackReq.setRelaxLocality(false);
-    Assert.assertEquals(1,
+    assertEquals(1,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
     node1Req.setRelaxLocality(false);
-    Assert.assertEquals(1,
+    assertEquals(1,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
     node2Req.setRelaxLocality(false);
-    Assert.assertEquals(80,
+    assertEquals(80,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
 
     anyReq.setNodeLabelExpression("label1");
@@ -369,65 +370,65 @@ public class TestRMServerUtils {
     anyReq.setRelaxLocality(true);
     reqs = new ArrayList<>();
     reqs.add(anyReq);
-    Assert.assertEquals(15,
+    assertEquals(15,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
 
     rackReq.setRelaxLocality(true);
     reqs.add(rackReq);
-    Assert.assertEquals(10,
+    assertEquals(10,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
     anyReq.setRelaxLocality(false);
-    Assert.assertEquals(10,
+    assertEquals(10,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
     rackReq.setRelaxLocality(false);
-    Assert.assertEquals(15,
+    assertEquals(15,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
 
     node1Req.setRelaxLocality(false);
     reqs.add(node1Req);
-    Assert.assertEquals(15,
+    assertEquals(15,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
     node1Req.setRelaxLocality(true);
-    Assert.assertEquals(1,
+    assertEquals(1,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
     rackReq.setRelaxLocality(true);
-    Assert.assertEquals(11,
+    assertEquals(11,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
 
     node2Req.setRelaxLocality(false);
     reqs.add(node2Req);
-    Assert.assertEquals(11,
+    assertEquals(11,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
     node2Req.setRelaxLocality(true);
-    Assert.assertEquals(11,
+    assertEquals(11,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
     rackReq.setRelaxLocality(false);
-    Assert.assertEquals(1,
+    assertEquals(1,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
     node1Req.setRelaxLocality(false);
-    Assert.assertEquals(0,
+    assertEquals(0,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
     node2Req.setRelaxLocality(false);
-    Assert.assertEquals(15,
+    assertEquals(15,
         RMServerUtils.getApplicableNodeCountForAM(rmContext, conf, reqs));
   }
   @Test
   public void testConvertRmAppAttemptStateToYarnApplicationAttemptState() {
-    Assert.assertEquals(
+    assertEquals(
         YarnApplicationAttemptState.FAILED,
         RMServerUtils.convertRmAppAttemptStateToYarnApplicationAttemptState(
             RMAppAttemptState.FINAL_SAVING,
             RMAppAttemptState.FAILED
         )
     );
-    Assert.assertEquals(
+    assertEquals(
         YarnApplicationAttemptState.SCHEDULED,
         RMServerUtils.convertRmAppAttemptStateToYarnApplicationAttemptState(
             RMAppAttemptState.FINAL_SAVING,
             RMAppAttemptState.SCHEDULED
         )
     );
-    Assert.assertEquals(
+    assertEquals(
         YarnApplicationAttemptState.NEW,
         RMServerUtils.convertRmAppAttemptStateToYarnApplicationAttemptState(
             RMAppAttemptState.NEW,

+ 23 - 21
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMStoreCommands.java

@@ -18,11 +18,11 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
-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.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 org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.test.TestingServer;
@@ -34,7 +34,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.TestZKRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 public class TestRMStoreCommands {
 
@@ -52,9 +52,10 @@ public class TestRMStoreCommands {
       rm.start();
       rm.getRMContext().getRMAdminService().transitionToActive(req);
       String zkStateRoot = ZKRMStateStore.ROOT_ZNODE_NAME;
-      assertEquals("RM State store parent path should have a child node " +
-          zkStateRoot, zkStateRoot, curatorFramework.getChildren().forPath(
-              YarnConfiguration.DEFAULT_ZK_RM_STATE_STORE_PARENT_PATH).get(0));
+      assertEquals(zkStateRoot, curatorFramework.getChildren().forPath(
+          YarnConfiguration.DEFAULT_ZK_RM_STATE_STORE_PARENT_PATH).get(0),
+          "RM State store parent path should have a child node " +
+          zkStateRoot);
       rm.close();
       try {
         ResourceManager.deleteRMStateStore(conf);
@@ -62,9 +63,9 @@ public class TestRMStoreCommands {
         fail("Exception should not be thrown during format rm state store" +
             " operation.");
       }
-      assertTrue("After store format parent path should have no child nodes",
-          curatorFramework.getChildren().forPath(
-          YarnConfiguration.DEFAULT_ZK_RM_STATE_STORE_PARENT_PATH).isEmpty());
+      assertTrue(curatorFramework.getChildren().forPath(
+          YarnConfiguration.DEFAULT_ZK_RM_STATE_STORE_PARENT_PATH).isEmpty(),
+          "After store format parent path should have no child nodes");
     }
   }
 
@@ -86,8 +87,8 @@ public class TestRMStoreCommands {
           YarnConfiguration.RM_SCHEDCONF_STORE_ZK_PARENT_PATH,
           YarnConfiguration.DEFAULT_RM_SCHEDCONF_STORE_ZK_PARENT_PATH)
           + "/CONF_STORE";
-      assertNotNull("Failed to initialize ZKConfigurationStore",
-          curatorFramework.checkExists().forPath(confStorePath));
+      assertNotNull(curatorFramework.checkExists().forPath(confStorePath),
+          "Failed to initialize ZKConfigurationStore");
 
       rm.close();
       try {
@@ -96,8 +97,8 @@ public class TestRMStoreCommands {
         fail("Exception should not be thrown during format rm conf store" +
             " operation.");
       }
-      assertNull("Failed to format ZKConfigurationStore",
-          curatorFramework.checkExists().forPath(confStorePath));
+      assertNull(curatorFramework.checkExists().forPath(confStorePath),
+          "Failed to format ZKConfigurationStore");
     }
   }
 
@@ -126,8 +127,8 @@ public class TestRMStoreCommands {
         if (path.equals(ZKRMStateStore.RM_APP_ROOT_HIERARCHIES)) {
           continue;
         }
-        assertEquals("Application node for " + appId + " should exist",
-            appId, path);
+        assertEquals(appId, path,
+            "Application node for " + appId + " should exist");
       }
       try {
         ResourceManager.removeApplication(conf, appId);
@@ -135,11 +136,12 @@ public class TestRMStoreCommands {
         fail("Exception should not be thrown while removing app from " +
             "rm state store.");
       }
-      assertTrue("After remove app from store there should be no child nodes" +
-          " for application in app root path",
+      assertTrue(
           curatorFramework.getChildren().forPath(appRootPath).size() == 1 &&
           curatorFramework.getChildren().forPath(appRootPath).get(0).equals(
-              ZKRMStateStore.RM_APP_ROOT_HIERARCHIES));
+          ZKRMStateStore.RM_APP_ROOT_HIERARCHIES),
+          "After remove app from store there should be no child nodes" +
+          " for application in app root path");
     }
   }
 }

+ 9 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMTimelineService.java

@@ -18,6 +18,9 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
@@ -28,8 +31,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.metrics.TimelineServiceV2Pu
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
 import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 /**
  * Tests that the RM creates timeline services (v1/v2) as specified by the
@@ -41,7 +43,7 @@ public class TestRMTimelineService {
   private void setup(boolean v1Enabled, boolean v2Enabled,
                      boolean systemMetricEnabled) {
     Configuration conf = new YarnConfiguration(new Configuration(false));
-    Assert.assertFalse(YarnConfiguration.timelineServiceEnabled(conf));
+    assertFalse(YarnConfiguration.timelineServiceEnabled(conf));
 
     conf.setBoolean(YarnConfiguration.SYSTEM_METRICS_PUBLISHER_ENABLED,
         systemMetricEnabled);
@@ -87,11 +89,11 @@ public class TestRMTimelineService {
     }
 
     if(systemMetricEnabled) {
-      Assert.assertEquals(v1Enabled, v1PublisherServiceFound);
-      Assert.assertEquals(v2Enabled, v2PublisherServiceFound);
+      assertEquals(v1Enabled, v1PublisherServiceFound);
+      assertEquals(v2Enabled, v2PublisherServiceFound);
     } else {
-      Assert.assertEquals(false, v1PublisherServiceFound);
-      Assert.assertEquals(false, v2PublisherServiceFound);
+      assertEquals(false, v1PublisherServiceFound);
+      assertEquals(false, v2PublisherServiceFound);
     }
   }
 

+ 34 - 30
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java

@@ -19,8 +19,11 @@
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 import static org.apache.hadoop.yarn.server.resourcemanager.MockNM.createMockNodeStatus;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.fail;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.IOException;
 import java.util.Collection;
@@ -51,12 +54,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateS
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.converter.FSConfigConverterTestCommons;
 import org.apache.hadoop.yarn.server.security.http.RMAuthenticationFilterInitializer;
 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.ExpectedException;
+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;
 
@@ -66,11 +67,9 @@ public class TestResourceManager {
 
   private ResourceManager resourceManager = null;
 
-  @Rule
-  public ExpectedException thrown = ExpectedException.none();
   private FSConfigConverterTestCommons converterTestCommons;
 
-  @Before
+  @BeforeEach
   public void setUp() throws Exception {
     YarnConfiguration conf = new YarnConfiguration();
     UserGroupInformation.setConfiguration(conf);
@@ -84,7 +83,7 @@ public class TestResourceManager {
     converterTestCommons.setUp();
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws Exception {
     resourceManager.stop();
     converterTestCommons.tearDown();
@@ -245,7 +244,8 @@ public class TestResourceManager {
     }
   }
 
-  @Test (timeout = 30000)
+  @Test
+  @Timeout(value = 30)
   public void testResourceManagerInitConfigValidation() throws Exception {
     Configuration conf = new YarnConfiguration();
     conf.setInt(YarnConfiguration.GLOBAL_RM_AM_MAX_ATTEMPTS, -1);
@@ -287,7 +287,8 @@ public class TestResourceManager {
     }
   }
 
-  @Test(timeout = 50000)
+  @Test
+  @Timeout(value = 50)
   public void testFilterOverrides() throws Exception {
     String filterInitializerConfKey = "hadoop.http.filter.initializers";
     String[] filterInitializers =
@@ -326,11 +327,11 @@ public class TestResourceManager {
         // just want to test filter settings
         String tmp = resourceManager.getConfig().get(filterInitializerConfKey);
         if (filterInitializer.contains(this.getClass().getName())) {
-          Assert.assertEquals(RMAuthenticationFilterInitializer.class.getName()
+          assertEquals(RMAuthenticationFilterInitializer.class.getName()
               + "," + this.getClass().getName(), tmp);
         } else {
-          Assert.assertEquals(
-            RMAuthenticationFilterInitializer.class.getName(), tmp);
+          assertEquals(
+              RMAuthenticationFilterInitializer.class.getName(), tmp);
         }
         resourceManager.stop();
       }
@@ -352,10 +353,10 @@ public class TestResourceManager {
         // just want to test filter settings
         String tmp = resourceManager.getConfig().get(filterInitializerConfKey);
         if (filterInitializer.equals(StaticUserWebFilter.class.getName())) {
-          Assert.assertEquals(RMAuthenticationFilterInitializer.class.getName()
+          assertEquals(RMAuthenticationFilterInitializer.class.getName()
               + "," + StaticUserWebFilter.class.getName(), tmp);
         } else {
-          Assert.assertEquals(
+          assertEquals(
             RMAuthenticationFilterInitializer.class.getName(), tmp);
         }
         resourceManager.stop();
@@ -370,18 +371,21 @@ public class TestResourceManager {
    */
   @Test
   public void testUserProvidedUGIConf() throws Exception {
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Invalid attribute value for "
+
+    IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, () -> {
+      Configuration dummyConf = new YarnConfiguration();
+      dummyConf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
+          "DUMMYAUTH");
+      ResourceManager dummyResourceManager = new ResourceManager();
+      try {
+        dummyResourceManager.init(dummyConf);
+      } finally {
+        dummyResourceManager.stop();
+      }
+    });
+
+    assertThat(exception.getMessage()).contains("Invalid attribute value for "
         + CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION
         + " of DUMMYAUTH");
-    Configuration dummyConf = new YarnConfiguration();
-    dummyConf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
-        "DUMMYAUTH");
-    ResourceManager dummyResourceManager = new ResourceManager();
-    try {
-      dummyResourceManager.init(dummyConf);
-    } finally {
-      dummyResourceManager.stop();
-    }
   }
 }

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManagerMXBean.java

@@ -17,13 +17,14 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
@@ -50,7 +51,7 @@ public class TestResourceManagerMXBean {
       // Get attribute "SecurityEnabled"
       boolean securityEnabled = (boolean) mbs.getAttribute(mxbeanName,
               "SecurityEnabled");
-      Assert.assertEquals(resourceManager.isSecurityEnabled(), securityEnabled);
+      assertEquals(resourceManager.isSecurityEnabled(), securityEnabled);
     }
   }
 }

File diff suppressed because it is too large
+ 207 - 217
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java


+ 5 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestSignalContainer.java

@@ -18,12 +18,13 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
 import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
-import org.junit.Assert;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -38,7 +39,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 public class TestSignalContainer {
 
@@ -85,7 +86,7 @@ public class TestSignalContainer {
         nm1.nodeHeartbeat(true);
       }
     }
-    Assert.assertEquals(request, conts.size());
+    assertEquals(request, conts.size());
 
     for(Container container : conts) {
       rm.signalToContainer(container.getId(),
@@ -107,7 +108,7 @@ public class TestSignalContainer {
     }
 
     // Verify NM receives the expected number of signal container requests.
-    Assert.assertEquals(request, signaledConts);
+    assertEquals(request, signaledConts);
 
     am.unregisterAppAttempt();
     nm1.nodeHeartbeat(attempt.getAppAttemptId(), 1, ContainerState.COMPLETE);

+ 16 - 15
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestTokenClientRMService.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+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;
 
@@ -39,10 +41,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.NullRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.Records;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 
 public class TestTokenClientRMService {
 
@@ -69,7 +70,7 @@ public class TestTokenClientRMService {
   private static final UserGroupInformation otherKerb = UserGroupInformation
       .createRemoteUser(otherPrincipal, AuthMethod.KERBEROS);
 
-  @BeforeClass
+  @BeforeAll
   public static void setupSecretManager() throws IOException {
     ResourceManager rm = mock(ResourceManager.class);
     RMContext rmContext = mock(RMContext.class);
@@ -88,7 +89,7 @@ public class TestTokenClientRMService {
        .setAuthenticationMethod(AuthenticationMethod.KERBEROS);
   }
 
-  @AfterClass
+  @AfterAll
   public static void teardownSecretManager() {
     if (dtsm != null) {
       dtsm.stopThreads();
@@ -128,9 +129,9 @@ public class TestTokenClientRMService {
             checkTokenRenewal(owner, other);
             return null;
           } catch (YarnException ex) {
-            Assert.assertTrue(ex.getMessage().contains(
+            assertTrue(ex.getMessage().contains(
                 owner.getUserName() + " tries to renew a token"));
-            Assert.assertTrue(ex.getMessage().contains(
+            assertTrue(ex.getMessage().contains(
                 "with non-matching renewer " + other.getUserName()));
             throw ex;
           }
@@ -139,7 +140,7 @@ public class TestTokenClientRMService {
     } catch (Exception e) {
       return;
     }
-    Assert.fail("renew should have failed");
+    fail("renew should have failed");
   }
 
   @Test
@@ -219,12 +220,12 @@ public class TestTokenClientRMService {
             public Void run() throws Exception {
               try {
                 checkTokenCancellation(rmService, tokOwner, tokRenewer);
-                Assert.fail("We should not reach here; token owner = "
+                fail("We should not reach here; token owner = "
                     + tokOwner.getUserName() + ", renewer = "
                     + tokRenewer.getUserName());
                 return null;
               } catch (YarnException e) {
-                Assert.assertTrue(e.getMessage().contains(
+                assertTrue(e.getMessage().contains(
                     testerKerb.getUserName()
                         + " is not authorized to cancel the token"));
                 return null;
@@ -232,7 +233,7 @@ public class TestTokenClientRMService {
             }
           });
         } catch (Exception e) {
-          Assert.fail("Unexpected exception; " + e.getMessage());
+          fail("Unexpected exception; " + e.getMessage());
         }
       }
     }
@@ -249,12 +250,12 @@ public class TestTokenClientRMService {
             public Void run() throws Exception {
               try {
                 checkTokenCancellation(tokOwner, tokRenewer);
-                Assert.fail("We should not reach here; token owner = "
+                fail("We should not reach here; token owner = "
                     + tokOwner.getUserName() + ", renewer = "
                     + tokRenewer.getUserName());
                 return null;
               } catch (YarnException ex) {
-                Assert.assertTrue(ex.getMessage().contains(
+                assertTrue(ex.getMessage().contains(
                     tester.getUserName()
                         + " is not authorized to cancel the token"));
                 return null;
@@ -262,7 +263,7 @@ public class TestTokenClientRMService {
             }
           });
         } catch (Exception e) {
-          Assert.fail("Unexpected exception; " + e.getMessage());
+          fail("Unexpected exception; " + e.getMessage());
         }
       }
     }

+ 20 - 20
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithCSRFFilter.java

@@ -36,8 +36,8 @@ import org.glassfish.jersey.internal.inject.AbstractBinder;
 import org.glassfish.jersey.jettison.JettisonFeature;
 import org.glassfish.jersey.server.ResourceConfig;
 import org.glassfish.jersey.test.TestProperties;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.w3c.dom.Document;
 import org.w3c.dom.Element;
 import org.w3c.dom.NodeList;
@@ -55,8 +55,8 @@ import java.io.StringReader;
 import java.util.HashMap;
 import java.util.Map;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -108,7 +108,7 @@ public class TestRMWithCSRFFilter extends JerseyTestBase {
     }
   }
 
-  @Before
+  @BeforeEach
   @Override
   public void setUp() throws Exception {
     super.setUp();
@@ -137,8 +137,8 @@ public class TestRMWithCSRFFilter extends JerseyTestBase {
         .header(RestCsrfPreventionFilter.HEADER_USER_AGENT,"Mozilla/5.0")
         .header("X-XSRF-HEADER", "")
         .get(Response.class);
-    assertTrue("Should have been accepted", response.getStatus() ==
-        Response.Status.OK.getStatusCode());
+    assertTrue(response.getStatus() ==
+        Response.Status.OK.getStatusCode(), "Should have been accepted");
     assertEquals(MediaType.APPLICATION_XML_TYPE + ";" + JettyUtils.UTF_8,
         response.getMediaType().toString());
     String xml = response.readEntity(String.class);
@@ -152,8 +152,8 @@ public class TestRMWithCSRFFilter extends JerseyTestBase {
         .path("info").request("application/xml")
         .header(RestCsrfPreventionFilter.HEADER_USER_AGENT,"Mozilla/5.0")
         .head();
-    assertTrue("Should have been allowed", response.getStatus() ==
-        Response.Status.OK.getStatusCode());
+    assertTrue(response.getStatus() ==
+        Response.Status.OK.getStatusCode(), "Should have been allowed");
   }
 
   @Test
@@ -162,8 +162,8 @@ public class TestRMWithCSRFFilter extends JerseyTestBase {
     Response response = r.path("ws").path("v1").path("cluster")
         .path("info").request("application/xml")
         .get(Response.class);
-    assertTrue("Should have been accepted", response.getStatus() ==
-        Response.Status.OK.getStatusCode());
+    assertTrue(response.getStatus() ==
+        Response.Status.OK.getStatusCode(), "Should have been accepted");
     assertEquals(MediaType.APPLICATION_XML_TYPE + ";" + JettyUtils.UTF_8,
         response.getMediaType().toString());
     String xml = response.readEntity(String.class);
@@ -177,7 +177,7 @@ public class TestRMWithCSRFFilter extends JerseyTestBase {
     is.setCharacterStream(new StringReader(xml));
     Document dom = db.parse(is);
     NodeList nodes = dom.getElementsByTagName("clusterInfo");
-    assertEquals("incorrect number of elements", 1, nodes.getLength());
+    assertEquals(1, nodes.getLength(), "incorrect number of elements");
 
     for (int i = 0; i < nodes.getLength(); i++) {
       Element element = (Element) nodes.item(i);
@@ -209,14 +209,14 @@ public class TestRMWithCSRFFilter extends JerseyTestBase {
                                    String resourceManagerBuildVersion,
                                    String resourceManagerVersion) {
 
-    assertEquals("clusterId doesn't match: ",
-                 ResourceManager.getClusterTimeStamp(), clusterid);
-    assertEquals("startedOn doesn't match: ",
-                 ResourceManager.getClusterTimeStamp(), startedon);
-    assertTrue("stated doesn't match: " + state,
-               state.matches(STATE.INITED.toString()));
-    assertTrue("HA state doesn't match: " + haState,
-               haState.matches("INITIALIZING"));
+    assertEquals(ResourceManager.getClusterTimeStamp(), clusterid,
+        "clusterId doesn't match: ");
+    assertEquals(ResourceManager.getClusterTimeStamp(), startedon,
+        "startedOn doesn't match: ");
+    assertTrue(state.matches(STATE.INITED.toString()),
+        "stated doesn't match: " + state);
+    assertTrue(haState.matches("INITIALIZING"),
+        "HA state doesn't match: " + haState);
 
     WebServicesTestUtils.checkStringMatch("hadoopVersionBuiltOn",
                                           VersionInfo.getDate(), hadoopVersionBuiltOn);

+ 12 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithXFSFilter.java

@@ -18,15 +18,18 @@
 
 package org.apache.hadoop.yarn.webapp;
 
+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 org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
 
 import java.io.IOException;
 import java.net.HttpURLConnection;
@@ -62,7 +65,7 @@ public class TestRMWithXFSFilter {
     URL url = new URL("http://localhost:8088/ws/v1/cluster/info");
     HttpURLConnection conn = (HttpURLConnection) url.openConnection();
     String xfoHeader = conn.getHeaderField("X-FRAME-OPTIONS");
-    Assert.assertTrue(xfoHeader.endsWith(HttpServer2.XFrameOption
+    assertTrue(xfoHeader.endsWith(HttpServer2.XFrameOption
         .SAMEORIGIN.toString()));
   }
 
@@ -74,7 +77,7 @@ public class TestRMWithXFSFilter {
     URL url = new URL("http://localhost:8088/ws/v1/cluster/info");
     HttpURLConnection conn = (HttpURLConnection) url.openConnection();
     String xfoHeader = conn.getHeaderField("X-FRAME-OPTIONS");
-    Assert.assertTrue(xfoHeader.endsWith(HttpServer2.XFrameOption
+    assertTrue(xfoHeader.endsWith(HttpServer2.XFrameOption
         .SAMEORIGIN.toString()));
   }
 
@@ -86,7 +89,7 @@ public class TestRMWithXFSFilter {
     URL url = new URL("http://localhost:8088/logs");
     HttpURLConnection conn = (HttpURLConnection) url.openConnection();
     String xfoHeader = conn.getHeaderField("X-FRAME-OPTIONS");
-    Assert.assertTrue(xfoHeader.endsWith(HttpServer2.XFrameOption
+    assertTrue(xfoHeader.endsWith(HttpServer2.XFrameOption
         .SAMEORIGIN.toString()));
   }
 
@@ -97,17 +100,17 @@ public class TestRMWithXFSFilter {
     URL url = new URL("http://localhost:8088/ws/v1/cluster/info");
     HttpURLConnection conn = (HttpURLConnection) url.openConnection();
     String xfoHeader = conn.getHeaderField("X-FRAME-OPTIONS");
-    Assert.assertNull("Unexpected X-FRAME-OPTION in header", xfoHeader);
+    assertNull(xfoHeader, "Unexpected X-FRAME-OPTION in header");
   }
 
   @Test
   public void testXFrameOptionsIllegalOption() {
-    IllegalArgumentException e = Assert.assertThrows(
+    IllegalArgumentException e = assertThrows(
         IllegalArgumentException.class,
         () -> createMockRm(true, "otherValue"));
   }
 
-  @After
+  @AfterEach
   public void tearDown() throws IOException {
     rm.close();
   }

Some files were not shown because too many files changed in this diff