瀏覽代碼

YARN-9858. Optimize RMContext getExclusiveEnforcedPartitions. Contributed by Jonathan Hung.

bibinchundatt 5 年之前
父節點
當前提交
5cd6eb2a18

+ 16 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -23,7 +23,9 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -3630,6 +3632,20 @@ public class YarnConfiguration extends Configuration {
   public static final String EXCLUSIVE_ENFORCED_PARTITIONS = NODE_LABELS_PREFIX
       + EXCLUSIVE_ENFORCED_PARTITIONS_SUFFIX;
 
+  @Private
+  public static Set<String> getExclusiveEnforcedPartitions(
+      Configuration conf) {
+    Set<String> exclusiveEnforcedPartitions = new HashSet<>();
+    String[] configuredPartitions = conf.getStrings(
+        EXCLUSIVE_ENFORCED_PARTITIONS);
+    if (configuredPartitions != null) {
+      for (String partition : configuredPartitions) {
+        exclusiveEnforcedPartitions.add(partition);
+      }
+    }
+    return exclusiveEnforcedPartitions;
+  }
+
   public static final String MAX_CLUSTER_LEVEL_APPLICATION_PRIORITY =
       YARN_PREFIX + "cluster.max-application-priority";
 

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

@@ -118,6 +118,7 @@ final class DefaultAMSProcessor implements ApplicationMasterServiceProcessor {
   private ResourceProfilesManager resourceProfilesManager;
   private boolean timelineServiceV2Enabled;
   private boolean nodelabelsEnabled;
+  private Set<String> exclusiveEnforcedPartitions;
 
   @Override
   public void init(ApplicationMasterServiceContext amsContext,
@@ -128,6 +129,8 @@ final class DefaultAMSProcessor implements ApplicationMasterServiceProcessor {
         timelineServiceV2Enabled(rmContext.getYarnConfiguration());
     this.nodelabelsEnabled = YarnConfiguration
         .areNodeLabelsEnabled(rmContext.getYarnConfiguration());
+    this.exclusiveEnforcedPartitions = YarnConfiguration
+        .getExclusiveEnforcedPartitions(rmContext.getYarnConfiguration());
   }
 
   @Override
@@ -238,8 +241,7 @@ final class DefaultAMSProcessor implements ApplicationMasterServiceProcessor {
       }
       if (ResourceRequest.ANY.equals(req.getResourceName())) {
         SchedulerUtils.enforcePartitionExclusivity(req,
-            getRmContext().getExclusiveEnforcedPartitions(),
-            asc.getNodeLabelExpression());
+            exclusiveEnforcedPartitions, asc.getNodeLabelExpression());
       }
     }
 

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

@@ -125,7 +125,8 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
         timelineServiceV2Enabled(conf);
     this.nodeLabelsEnabled = YarnConfiguration
         .areNodeLabelsEnabled(rmContext.getYarnConfiguration());
-    this.exclusiveEnforcedPartitions = context.getExclusiveEnforcedPartitions();
+    this.exclusiveEnforcedPartitions = YarnConfiguration
+        .getExclusiveEnforcedPartitions(rmContext.getYarnConfiguration());
   }
 
   /**

+ 0 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java

@@ -19,7 +19,6 @@
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 import java.nio.ByteBuffer;
-import java.util.Set;
 import java.util.concurrent.ConcurrentMap;
 
 import org.apache.hadoop.conf.Configuration;
@@ -189,7 +188,4 @@ public interface RMContext extends ApplicationMasterServiceContext {
 
   void setMultiNodeSortingManager(
       MultiNodeSortingManager<SchedulerNode> multiNodeSortingManager);
-
-  Set<String> getExclusiveEnforcedPartitions();
-
 }

+ 0 - 19
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java

@@ -21,8 +21,6 @@ package org.apache.hadoop.yarn.server.resourcemanager;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.nio.ByteBuffer;
-import java.util.HashSet;
-import java.util.Set;
 import java.util.concurrent.ConcurrentMap;
 
 import org.apache.commons.logging.Log;
@@ -35,7 +33,6 @@ import org.apache.hadoop.yarn.LocalConfigurationProvider;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.conf.ConfigurationProvider;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
 import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
@@ -646,20 +643,4 @@ public class RMContextImpl implements RMContext {
   public NodeAttributesManager getNodeAttributesManager() {
     return activeServiceContext.getNodeAttributesManager();
   }
-
-  public Set<String> getExclusiveEnforcedPartitions() {
-    Set<String> exclusiveEnforcedPartitions = new HashSet<>();
-    Configuration conf = getYarnConfiguration();
-    if (conf == null) {
-      return new HashSet<>();
-    }
-    String[] configuredPartitions = conf.getStrings(
-        YarnConfiguration.EXCLUSIVE_ENFORCED_PARTITIONS);
-    if (configuredPartitions != null) {
-      for (String partition : configuredPartitions) {
-        exclusiveEnforcedPartitions.add(partition);
-      }
-    }
-    return exclusiveEnforcedPartitions;
-  }
 }