Browse Source

YARN-11708: Setting maximum-application-lifetime using AQCv2 templates doesn't apply on the first submitted app (#7041)

Susheel Gupta 7 months ago
parent
commit
1b5a2a7f65

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java

@@ -1269,7 +1269,7 @@ public class RMAppImpl implements RMApp, Recoverable {
       long applicationLifetime =
           app.getApplicationLifetime(ApplicationTimeoutType.LIFETIME);
       applicationLifetime = app.scheduler
-          .checkAndGetApplicationLifetime(app.queue, applicationLifetime);
+          .checkAndGetApplicationLifetime(app.queue, applicationLifetime, app);
       if (applicationLifetime > 0) {
         // calculate next timeout value
         Long newTimeout =

+ 3 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java

@@ -82,6 +82,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitorManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
@@ -1679,7 +1680,8 @@ public abstract class AbstractYarnScheduler
   }
 
   @Override
-  public long checkAndGetApplicationLifetime(String queueName, long lifetime) {
+  public long checkAndGetApplicationLifetime(String queueName, long lifetime,
+                                             RMAppImpl app) {
     // Lifetime is the application lifetime by default.
     return lifetime;
   }

+ 4 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java

@@ -44,6 +44,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
@@ -423,11 +424,13 @@ public interface YarnScheduler extends EventHandler<SchedulerEvent> {
    * Queue lifetime.
    * @param queueName Name of the Queue
    * @param lifetime configured application lifetime
+   * @param app details of app
    * @return valid lifetime as per queue
    */
   @Public
   @Evolving
-  long checkAndGetApplicationLifetime(String queueName, long lifetime);
+  long checkAndGetApplicationLifetime(String queueName, long lifetime,
+                                      RMAppImpl app);
 
   /**
    * Get maximum lifetime for a queue.

+ 35 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.placement.ApplicationPlacem
 import org.apache.hadoop.yarn.server.resourcemanager.placement.CSMappingPlacementRule;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementFactory;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementRule;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.Marker;
@@ -3374,14 +3375,45 @@ public class CapacityScheduler extends
 
   @Override
   public long checkAndGetApplicationLifetime(String queueName,
-      long lifetimeRequestedByApp) {
-    readLock.lock();
+                                     long lifetimeRequestedByApp, RMAppImpl app) {
+    CSQueue queue;
+
+    writeLock.lock();
     try {
-      CSQueue queue = getQueue(queueName);
+      queue = getQueue(queueName);
+
+      // This handles the case where the first submitted app in aqc queue
+      // does not exist, addressing the issue related to YARN-11708.
+      if (queue == null) {
+        queue = getOrCreateQueueFromPlacementContext(app.getApplicationId(),
+            app.getUser(), app.getQueue(), app.getApplicationPlacementContext(), false);
+      }
+
+      if (queue == null) {
+        String message = "Application " + app.getApplicationId()
+              + " submitted by user " + app.getUser();
+        if (isAmbiguous(queueName)) {
+          message = message + " to ambiguous queue: " + queueName
+              + " please use full queue path instead.";
+        } else {
+          message = message + "Application " + app.getApplicationId() +
+              " submitted by user " + app.getUser() + " to unknown queue: " + queueName;
+        }
+        this.rmContext.getDispatcher().getEventHandler().handle(
+            new RMAppEvent(app.getApplicationId(), RMAppEventType.APP_REJECTED,
+                message));
+        return lifetimeRequestedByApp;
+      }
+
       if (!(queue instanceof AbstractLeafQueue)) {
         return lifetimeRequestedByApp;
       }
+    } finally {
+      writeLock.unlock();
+    }
 
+    readLock.lock();
+    try {
       long defaultApplicationLifetime =
           queue.getDefaultApplicationLifetime();
       long maximumApplicationLifetime =

+ 3 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java

@@ -59,6 +59,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationCons
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
@@ -2042,7 +2043,8 @@ public class FairScheduler extends
   }
 
   @Override
-  public long checkAndGetApplicationLifetime(String queueName, long lifetime) {
+  public long checkAndGetApplicationLifetime(String queueName, long lifetime,
+                                             RMAppImpl app) {
     // Lifetime is the application lifetime by default.
     return lifetime;
   }

+ 19 - 19
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java

@@ -2482,12 +2482,12 @@ public class TestCapacityScheduler {
     // positive integer value
     CapacityScheduler cs = setUpCSQueue(maxLifetime, defaultLifetime);
     Assert.assertEquals(maxLifetime,
-        cs.checkAndGetApplicationLifetime("default", 100));
-    Assert.assertEquals(9, cs.checkAndGetApplicationLifetime("default", 9));
+        cs.checkAndGetApplicationLifetime("default", 100, null));
+    Assert.assertEquals(9, cs.checkAndGetApplicationLifetime("default", 9, null));
     Assert.assertEquals(defaultLifetime,
-        cs.checkAndGetApplicationLifetime("default", -1));
+        cs.checkAndGetApplicationLifetime("default", -1, null));
     Assert.assertEquals(defaultLifetime,
-        cs.checkAndGetApplicationLifetime("default", 0));
+        cs.checkAndGetApplicationLifetime("default", 0, null));
     Assert.assertEquals(maxLifetime,
         cs.getMaximumApplicationLifetime("default"));
 
@@ -2495,11 +2495,11 @@ public class TestCapacityScheduler {
     defaultLifetime = -1;
     // test for default values
     cs = setUpCSQueue(maxLifetime, defaultLifetime);
-    Assert.assertEquals(100, cs.checkAndGetApplicationLifetime("default", 100));
+    Assert.assertEquals(100, cs.checkAndGetApplicationLifetime("default", 100, null));
     Assert.assertEquals(defaultLifetime,
-        cs.checkAndGetApplicationLifetime("default", -1));
+        cs.checkAndGetApplicationLifetime("default", -1, null));
     Assert.assertEquals(defaultLifetime,
-        cs.checkAndGetApplicationLifetime("default", 0));
+        cs.checkAndGetApplicationLifetime("default", 0, null));
     Assert.assertEquals(maxLifetime,
         cs.getMaximumApplicationLifetime("default"));
 
@@ -2507,32 +2507,32 @@ public class TestCapacityScheduler {
     defaultLifetime = 10;
     cs = setUpCSQueue(maxLifetime, defaultLifetime);
     Assert.assertEquals(maxLifetime,
-        cs.checkAndGetApplicationLifetime("default", 100));
+        cs.checkAndGetApplicationLifetime("default", 100, null));
     Assert.assertEquals(defaultLifetime,
-        cs.checkAndGetApplicationLifetime("default", -1));
+        cs.checkAndGetApplicationLifetime("default", -1, null));
     Assert.assertEquals(defaultLifetime,
-        cs.checkAndGetApplicationLifetime("default", 0));
+        cs.checkAndGetApplicationLifetime("default", 0, null));
     Assert.assertEquals(maxLifetime,
         cs.getMaximumApplicationLifetime("default"));
 
     maxLifetime = 0;
     defaultLifetime = 0;
     cs = setUpCSQueue(maxLifetime, defaultLifetime);
-    Assert.assertEquals(100, cs.checkAndGetApplicationLifetime("default", 100));
+    Assert.assertEquals(100, cs.checkAndGetApplicationLifetime("default", 100, null));
     Assert.assertEquals(defaultLifetime,
-        cs.checkAndGetApplicationLifetime("default", -1));
+        cs.checkAndGetApplicationLifetime("default", -1, null));
     Assert.assertEquals(defaultLifetime,
-        cs.checkAndGetApplicationLifetime("default", 0));
+        cs.checkAndGetApplicationLifetime("default", 0, null));
 
     maxLifetime = 10;
     defaultLifetime = -1;
     cs = setUpCSQueue(maxLifetime, defaultLifetime);
     Assert.assertEquals(maxLifetime,
-        cs.checkAndGetApplicationLifetime("default", 100));
+        cs.checkAndGetApplicationLifetime("default", 100, null));
     Assert.assertEquals(maxLifetime,
-        cs.checkAndGetApplicationLifetime("default", -1));
+        cs.checkAndGetApplicationLifetime("default", -1, null));
     Assert.assertEquals(maxLifetime,
-        cs.checkAndGetApplicationLifetime("default", 0));
+        cs.checkAndGetApplicationLifetime("default", 0, null));
 
     maxLifetime = 5;
     defaultLifetime = 10;
@@ -2549,11 +2549,11 @@ public class TestCapacityScheduler {
     defaultLifetime = 10;
     cs = setUpCSQueue(maxLifetime, defaultLifetime);
     Assert.assertEquals(100,
-        cs.checkAndGetApplicationLifetime("default", 100));
+        cs.checkAndGetApplicationLifetime("default", 100, null));
     Assert.assertEquals(defaultLifetime,
-        cs.checkAndGetApplicationLifetime("default", -1));
+        cs.checkAndGetApplicationLifetime("default", -1, null));
     Assert.assertEquals(defaultLifetime,
-        cs.checkAndGetApplicationLifetime("default", 0));
+        cs.checkAndGetApplicationLifetime("default", 0, null));
   }
 
   private CapacityScheduler setUpCSQueue(long maxLifetime,

+ 54 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAutoQueueCreation.java

@@ -90,7 +90,6 @@ import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager
     .NO_LABEL;
 import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueueUtils.EPSILON;
 
-import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.ROOT;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
@@ -618,6 +617,60 @@ public class TestCapacitySchedulerAutoQueueCreation
     }
   }
 
+  @Test
+  public void testAutoQueueCreationWithWeightModeAndMaxAppLifetimeFirstSubmittedApp()
+      throws Exception {
+    if (mockRM != null) {
+      mockRM.stop();
+    }
+
+    long maxRootLifetime = 20L;
+    long defaultRootLifetime = 10L;
+
+    QueuePath testQueue = new QueuePath("root.test");
+
+    CapacitySchedulerConfiguration conf = setupSchedulerConfiguration();
+    conf.setQueues(ROOT, new String[] {"test"});
+    conf.setAutoQueueCreationV2Enabled(testQueue, true);
+    conf.setCapacity(DEFAULT, "1w");
+    conf.setCapacity(testQueue, "2w");
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+
+    conf.setMaximumLifetimePerQueue(ROOT, maxRootLifetime);
+    conf.setDefaultLifetimePerQueue(ROOT, defaultRootLifetime);
+
+    MockRM newMockRM = new MockRM(conf);
+    newMockRM.start();
+    ((CapacityScheduler) newMockRM.getResourceScheduler()).start();
+
+    CapacityScheduler newCS =
+        (CapacityScheduler) newMockRM.getResourceScheduler();
+
+    Priority appPriority = Priority.newInstance(0);
+    MockRMAppSubmissionData app = MockRMAppSubmissionData.Builder.createWithMemory(1024, newMockRM)
+              .withAppPriority(appPriority)
+              .withQueue("root.test.user")
+              .build();
+    RMApp app1 = MockRMAppSubmitter.submit(newMockRM, app);
+
+    Assert.assertEquals(newCS.getMaximumApplicationLifetime("root.test.user"), 20L);
+
+    try {
+      newMockRM.waitForState(app1.getApplicationId(), RMAppState.KILLED);
+      long totalTimeRun = app1.getFinishTime() - app1.getSubmitTime();
+
+      Assert.assertEquals(RMAppState.KILLED, app1.getState());
+      Assert.assertTrue("Application killed before default lifetime value",
+          totalTimeRun > (defaultRootLifetime * 1000));
+      Assert.assertTrue(
+          "Application killed after max lifetime value " + totalTimeRun,
+          totalTimeRun < (maxRootLifetime * 1000));
+    } finally {
+      ((CapacityScheduler) newMockRM.getResourceScheduler()).stop();
+      newMockRM.stop();
+    }
+  }
 
   /**
    * This test case checks if a mapping rule can put an application to an auto