瀏覽代碼

YARN-9508. YarnConfiguration areNodeLabel enabled is costly in allocation flow. Contributed by Bilwa S T.

bibinchundatt 6 年之前
父節點
當前提交
570fa2da20

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

@@ -117,6 +117,7 @@ final class DefaultAMSProcessor implements ApplicationMasterServiceProcessor {
   private RMContext rmContext;
   private ResourceProfilesManager resourceProfilesManager;
   private boolean timelineServiceV2Enabled;
+  private boolean nodelabelsEnabled;
 
   @Override
   public void init(ApplicationMasterServiceContext amsContext,
@@ -125,6 +126,8 @@ final class DefaultAMSProcessor implements ApplicationMasterServiceProcessor {
     this.resourceProfilesManager = rmContext.getResourceProfilesManager();
     this.timelineServiceV2Enabled = YarnConfiguration.
         timelineServiceV2Enabled(rmContext.getYarnConfiguration());
+    this.nodelabelsEnabled = YarnConfiguration
+        .areNodeLabelsEnabled(rmContext.getYarnConfiguration());
   }
 
   @Override
@@ -242,7 +245,7 @@ final class DefaultAMSProcessor implements ApplicationMasterServiceProcessor {
     try {
       RMServerUtils.normalizeAndValidateRequests(ask,
           maximumCapacity, app.getQueue(),
-          getScheduler(), getRmContext());
+          getScheduler(), getRmContext(), nodelabelsEnabled);
     } catch (InvalidResourceRequestException e) {
       RMAppAttempt rmAppAttempt = app.getRMAppAttempt(appAttemptId);
       handleInvalidResourceException(e, rmAppAttempt);

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

@@ -99,6 +99,7 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
   private Configuration conf;
   private YarnAuthorizationProvider authorizer;
   private boolean timelineServiceV2Enabled;
+  private boolean nodeLabelsEnabled;
 
   public RMAppManager(RMContext context,
       YarnScheduler scheduler, ApplicationMasterService masterService,
@@ -121,6 +122,8 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
     this.authorizer = YarnAuthorizationProvider.getInstance(conf);
     this.timelineServiceV2Enabled = YarnConfiguration.
         timelineServiceV2Enabled(conf);
+    this.nodeLabelsEnabled = YarnConfiguration
+        .areNodeLabelsEnabled(rmContext.getYarnConfiguration());
   }
 
   /**
@@ -602,7 +605,7 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
         Resource maxAllocation = scheduler.getMaximumResourceCapability(queue);
         for (ResourceRequest amReq : amReqs) {
           SchedulerUtils.normalizeAndValidateRequest(amReq, maxAllocation,
-              queue, scheduler, isRecovery, rmContext, null);
+              queue, isRecovery, rmContext, null, nodeLabelsEnabled);
 
           amReq.setCapability(scheduler.getNormalizedResource(
               amReq.getCapability(), maxAllocation));

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java

@@ -245,7 +245,8 @@ public class RMServerUtils {
    */
   public static void normalizeAndValidateRequests(List<ResourceRequest> ask,
       Resource maximumAllocation, String queueName, YarnScheduler scheduler,
-      RMContext rmContext) throws InvalidResourceRequestException {
+      RMContext rmContext, boolean nodeLabelsEnabled)
+          throws InvalidResourceRequestException {
     // Get queue from scheduler
     QueueInfo queueInfo = null;
     try {
@@ -257,7 +258,7 @@ public class RMServerUtils {
 
     for (ResourceRequest resReq : ask) {
       SchedulerUtils.normalizeAndValidateRequest(resReq, maximumAllocation,
-          queueName, scheduler, rmContext, queueInfo);
+          queueName, rmContext, queueInfo, nodeLabelsEnabled);
     }
   }
 

+ 11 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java

@@ -42,7 +42,6 @@ import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.InvalidLabelResourceRequestException;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException
@@ -259,12 +258,12 @@ public class SchedulerUtils {
   }
 
   public static void normalizeAndValidateRequest(ResourceRequest resReq,
-      Resource maximumAllocation, String queueName, YarnScheduler scheduler,
-      boolean isRecovery, RMContext rmContext, QueueInfo queueInfo)
-      throws InvalidResourceRequestException {
+      Resource maximumAllocation, String queueName, boolean isRecovery,
+      RMContext rmContext, QueueInfo queueInfo, boolean nodeLabelsEnabled)
+          throws InvalidResourceRequestException {
     Configuration conf = rmContext.getYarnConfiguration();
     // If Node label is not enabled throw exception
-    if (null != conf && !YarnConfiguration.areNodeLabelsEnabled(conf)) {
+    if (null != conf && !nodeLabelsEnabled) {
       String labelExp = resReq.getNodeLabelExpression();
       if (!(RMNodeLabelsManager.NO_LABEL.equals(labelExp)
           || null == labelExp)) {
@@ -280,7 +279,8 @@ public class SchedulerUtils {
     }
     if (null == queueInfo) {
       try {
-        queueInfo = scheduler.getQueueInfo(queueName, false, false);
+        queueInfo = rmContext.getScheduler().getQueueInfo(queueName, false,
+            false);
       } catch (IOException e) {
         //Queue may not exist since it could be auto-created in case of
         // dynamic queues
@@ -294,11 +294,11 @@ public class SchedulerUtils {
   }
 
   public static void normalizeAndValidateRequest(ResourceRequest resReq,
-      Resource maximumAllocation, String queueName, YarnScheduler scheduler,
-      RMContext rmContext, QueueInfo queueInfo)
-      throws InvalidResourceRequestException {
-    normalizeAndValidateRequest(resReq, maximumAllocation, queueName, scheduler,
-        false, rmContext, queueInfo);
+      Resource maximumAllocation, String queueName, RMContext rmContext,
+      QueueInfo queueInfo, boolean nodeLabelsEnabled)
+          throws InvalidResourceRequestException {
+    normalizeAndValidateRequest(resReq, maximumAllocation, queueName, false,
+        rmContext, queueInfo, nodeLabelsEnabled);
   }
 
   /**

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java

@@ -204,6 +204,7 @@ public class TestAppManager extends AppManagerTestBase{
     metricsPublisher = mock(SystemMetricsPublisher.class);
     context.setSystemMetricsPublisher(metricsPublisher);
     context.setRMApplicationHistoryWriter(writer);
+    ((RMContextImpl)context).setYarnConfiguration(new YarnConfiguration());
     return context;
   }
 

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java

@@ -1020,7 +1020,8 @@ public class TestSchedulerUtils {
       Resource maxAllocation)
       throws InvalidResourceRequestException {
     SchedulerUtils.normalizeAndValidateRequest(resReq, maxAllocation, queueName,
-        scheduler, rmContext, null);
+        rmContext, null, YarnConfiguration
+            .areNodeLabelsEnabled(rmContext.getYarnConfiguration()));
   }
 
   private static class InvalidResourceRequestExceptionMessageGenerator {