Browse Source

YARN-6689. PlacementRule should be configurable. (Jonathan Hung via xgong)

Xuan 8 năm trước cách đây
mục cha
commit
edc0aa95f1

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

@@ -205,6 +205,12 @@ public class YarnConfiguration extends Configuration {
   public static final boolean DEFAULT_RM_SCHEDULER_USE_PORT_FOR_NODE_NAME = 
       false;
 
+  /** Configured scheduler queue placement rules. */
+  public static final String QUEUE_PLACEMENT_RULES = YARN_PREFIX
+      + "scheduler.queue-placement-rules";
+  /** UserGroupMappingPlacementRule configuration string. */
+  public static final String USER_GROUP_PLACEMENT_RULE = "user-group";
+
   /** Enable Resource Manager webapp ui actions */
   public static final String RM_WEBAPP_UI_ACTIONS_ENABLED =
     RM_PREFIX + "webapp.ui-actions.enabled";

+ 10 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml

@@ -3012,4 +3012,14 @@
     <value>64</value>
   </property>
 
+  <property>
+    <description>
+      Comma-separated list of PlacementRules to determine how applications
+      submitted by certain users get mapped to certain queues. Default is
+      user-group, which corresponds to UserGroupMappingPlacementRule.
+    </description>
+    <name>yarn.scheduler.queue-placement-rules</name>
+    <value>user-group</value>
+  </property>
+
 </configuration>

+ 45 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/PlacementFactory.java

@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.placement;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * Factory class for creating instances of {@link PlacementRule}.
+ */
+public final class PlacementFactory {
+
+  private static final Log LOG = LogFactory.getLog(PlacementFactory.class);
+
+  private PlacementFactory() {
+    // Unused.
+  }
+
+  public static PlacementRule getPlacementRule(String ruleStr,
+      Configuration conf)
+      throws ClassNotFoundException {
+    Class<? extends PlacementRule> ruleClass = Class.forName(ruleStr)
+        .asSubclass(PlacementRule.class);
+    LOG.info("Using PlacementRule implementation - " + ruleClass);
+    return ReflectionUtils.newInstance(ruleClass, conf);
+  }
+}

+ 30 - 7
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

@@ -69,6 +69,7 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+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.placement.UserGroupMappingPlacementRule;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.UserGroupMappingPlacementRule.QueueMapping;
@@ -563,15 +564,37 @@ public class CapacityScheduler extends
   }
 
   private void updatePlacementRules() throws IOException {
+    // Initialize placement rules
+    Collection<String> placementRuleStrs = conf.getStringCollection(
+        YarnConfiguration.QUEUE_PLACEMENT_RULES);
     List<PlacementRule> placementRules = new ArrayList<>();
-
-    // Initialize UserGroupMappingPlacementRule
-    // TODO, need make this defineable by configuration.
-    UserGroupMappingPlacementRule ugRule = getUserGroupMappingPlacementRule();
-    if (null != ugRule) {
-      placementRules.add(ugRule);
+    if (placementRuleStrs.isEmpty()) {
+      PlacementRule ugRule = getUserGroupMappingPlacementRule();
+      if (null != ugRule) {
+        placementRules.add(ugRule);
+      }
+    } else {
+      for (String placementRuleStr : placementRuleStrs) {
+        switch (placementRuleStr) {
+        case YarnConfiguration.USER_GROUP_PLACEMENT_RULE:
+          PlacementRule ugRule = getUserGroupMappingPlacementRule();
+          if (null != ugRule) {
+            placementRules.add(ugRule);
+          }
+          break;
+        default:
+          try {
+            PlacementRule rule = PlacementFactory.getPlacementRule(
+                placementRuleStr, conf);
+            if (null != rule) {
+              placementRules.add(rule);
+            }
+          } catch (ClassNotFoundException cnfe) {
+            throw new IOException(cnfe);
+          }
+        }
+      }
     }
-
     rmContext.getQueuePlacementManager().updateRules(placementRules);
   }