Bladeren bron

YARN-10929. Do not use a separate config in legacy CS AQC. Contributed by Benjamin Teke

Szilard Nemeth 3 jaren geleden
bovenliggende
commit
364d38f00d
13 gewijzigde bestanden met toevoegingen van 84 en 140 verwijderingen
  1. 17 27
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
  2. 29 40
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractLeafQueue.java
  3. 3 14
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractManagedParentQueue.java
  4. 5 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AutoCreatedLeafQueue.java
  5. 7 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueuePreemptionSettings.java
  6. 4 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueContext.java
  7. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
  8. 7 21
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ManagedParentQueue.java
  9. 5 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
  10. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PlanQueue.java
  11. 2 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueAllocationSettings.java
  12. 2 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ReservationQueue.java
  13. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerPerf.java

+ 17 - 27
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java

@@ -173,10 +173,9 @@ public abstract class AbstractCSQueue implements CSQueue {
     return new QueuePath(parent.getQueuePath(), queueName);
   }
 
-  protected void setupConfigurableCapacities(
-      CapacitySchedulerConfiguration configuration) {
+  protected void setupConfigurableCapacities() {
     CSQueueUtils.loadCapacitiesByLabelsFromConf(queuePath, queueCapacities,
-        configuration, this.queueNodeLabelsSettings.getConfiguredNodeLabels());
+        queueContext.getConfiguration(), this.queueNodeLabelsSettings.getConfiguredNodeLabels());
   }
 
   @Override
@@ -329,14 +328,14 @@ public abstract class AbstractCSQueue implements CSQueue {
     return this.queueNodeLabelsSettings.getDefaultLabelExpression();
   }
 
-  protected void setupQueueConfigs(Resource clusterResource,
-      CapacitySchedulerConfiguration configuration) throws
+  protected void setupQueueConfigs(Resource clusterResource) throws
       IOException {
 
     writeLock.lock();
     try {
+      CapacitySchedulerConfiguration configuration = queueContext.getConfiguration();
       if (isDynamicQueue() || this instanceof AbstractAutoCreatedLeafQueue) {
-        setDynamicQueueProperties(configuration);
+        setDynamicQueueProperties();
       }
 
       // Collect and set the Node label configuration
@@ -344,7 +343,7 @@ public abstract class AbstractCSQueue implements CSQueue {
           getQueuePath(), queueContext.getQueueManager().getConfiguredNodeLabelsForAllQueues());
 
       // Initialize the queue capacities
-      setupConfigurableCapacities(configuration);
+      setupConfigurableCapacities();
       updateAbsoluteCapacities();
       updateCapacityConfigType();
 
@@ -354,26 +353,23 @@ public abstract class AbstractCSQueue implements CSQueue {
 
       // Setup queue's maximumAllocation respecting the global
       // and the queue settings
-      // TODO remove the getConfiguration() param after the AQC configuration duplication
-      //  removal is resolved
-      this.queueAllocationSettings.setupMaximumAllocation(configuration,
-          queueContext.getConfiguration(), getQueuePath(),
+      this.queueAllocationSettings.setupMaximumAllocation(configuration, getQueuePath(),
           parent);
 
       // Initialize the queue state based on previous state, configured state
       // and its parent state
-      initializeQueueState(configuration);
+      initializeQueueState();
 
       authorizer = YarnAuthorizationProvider.getInstance(configuration);
 
       this.acls = configuration.getAcls(getQueuePath());
 
-      this.userWeights = getUserWeightsFromHierarchy(configuration);
+      this.userWeights = getUserWeightsFromHierarchy();
 
       this.reservationsContinueLooking =
           configuration.getReservationContinueLook();
 
-      this.configuredCapacityVectors = queueContext.getConfiguration()
+      this.configuredCapacityVectors = configuration
           .parseConfiguredResourceVector(queuePath.getFullPath(),
               this.queueNodeLabelsSettings.getConfiguredNodeLabels());
 
@@ -382,10 +378,7 @@ public abstract class AbstractCSQueue implements CSQueue {
           this, labelManager, null);
 
       // Store preemption settings
-      // TODO remove the getConfiguration() param after the AQC configuration duplication
-      //  removal is resolved
-      this.preemptionSettings = new CSQueuePreemptionSettings(this, configuration,
-          queueContext.getConfiguration());
+      this.preemptionSettings = new CSQueuePreemptionSettings(this, configuration);
       this.priority = configuration.getQueuePriority(
           getQueuePath());
 
@@ -403,14 +396,12 @@ public abstract class AbstractCSQueue implements CSQueue {
 
   /**
    * Set properties specific to dynamic queues.
-   * @param configuration configuration on which the properties are set
    */
-  protected void setDynamicQueueProperties(
-      CapacitySchedulerConfiguration configuration) {
+  protected void setDynamicQueueProperties() {
     // Set properties from parent template
     if (parent instanceof ParentQueue) {
       ((ParentQueue) parent).getAutoCreatedQueueTemplate()
-          .setTemplateEntriesForChild(configuration, getQueuePath());
+          .setTemplateEntriesForChild(queueContext.getConfiguration(), getQueuePath());
 
       String parentTemplate = String.format("%s.%s", parent.getQueuePath(),
           AutoCreatedQueueTemplate.AUTO_QUEUE_TEMPLATE_PREFIX);
@@ -428,8 +419,7 @@ public abstract class AbstractCSQueue implements CSQueue {
     }
   }
 
-  private UserWeights getUserWeightsFromHierarchy(
-      CapacitySchedulerConfiguration configuration) {
+  private UserWeights getUserWeightsFromHierarchy() {
     UserWeights unionInheritedWeights = UserWeights.createEmpty();
     CSQueue parentQ = parent;
     if (parentQ != null) {
@@ -439,7 +429,7 @@ public abstract class AbstractCSQueue implements CSQueue {
     // Insert this queue's userWeights, overriding parent's userWeights if
     // there is an overlap.
     unionInheritedWeights.addFrom(
-        configuration.getAllUserWeightsForQueue(getQueuePath()));
+        queueContext.getConfiguration().getAllUserWeightsForQueue(getQueuePath()));
     return unionInheritedWeights;
   }
 
@@ -572,9 +562,9 @@ public abstract class AbstractCSQueue implements CSQueue {
     return configuredCapacityVectors.get(label);
   }
 
-  private void initializeQueueState(CapacitySchedulerConfiguration configuration) {
+  private void initializeQueueState() {
     QueueState previousState = getState();
-    QueueState configuredState = configuration
+    QueueState configuredState = queueContext.getConfiguration()
         .getConfiguredState(getQueuePath());
     QueueState parentState = (parent == null) ? null : parent.getState();
 

+ 29 - 40
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractLeafQueue.java

@@ -166,15 +166,12 @@ public class AbstractLeafQueue extends AbstractCSQueue {
   }
 
   @SuppressWarnings("checkstyle:nowhitespaceafter")
-  protected void setupQueueConfigs(Resource clusterResource,
-      CapacitySchedulerConfiguration conf) throws
+  protected void setupQueueConfigs(Resource clusterResource) throws
       IOException {
     writeLock.lock();
     try {
-      // TODO conf parameter can be a modified configuration with template entries and missing
-      //  some global configs. This config duplication needs to be removed.
-      CapacitySchedulerConfiguration originalConfiguration = queueContext.getConfiguration();
-      super.setupQueueConfigs(clusterResource, conf);
+      CapacitySchedulerConfiguration configuration = queueContext.getConfiguration();
+      super.setupQueueConfigs(clusterResource);
 
       this.lastClusterResource = clusterResource;
 
@@ -189,26 +186,26 @@ public class AbstractLeafQueue extends AbstractCSQueue {
       setQueueResourceLimitsInfo(clusterResource);
 
       setOrderingPolicy(
-          conf.<FiCaSchedulerApp>getAppOrderingPolicy(getQueuePath()));
+          configuration.<FiCaSchedulerApp>getAppOrderingPolicy(getQueuePath()));
 
-      usersManager.setUserLimit(conf.getUserLimit(getQueuePath()));
-      usersManager.setUserLimitFactor(conf.getUserLimitFactor(getQueuePath()));
+      usersManager.setUserLimit(configuration.getUserLimit(getQueuePath()));
+      usersManager.setUserLimitFactor(configuration.getUserLimitFactor(getQueuePath()));
 
       maxAMResourcePerQueuePercent =
-          conf.getMaximumApplicationMasterResourcePerQueuePercent(
+          configuration.getMaximumApplicationMasterResourcePerQueuePercent(
               getQueuePath());
 
-      maxApplications = conf.getMaximumApplicationsPerQueue(getQueuePath());
+      maxApplications = configuration.getMaximumApplicationsPerQueue(getQueuePath());
       if (maxApplications < 0) {
         int maxGlobalPerQueueApps =
-            conf.getGlobalMaximumApplicationsPerQueue();
+            configuration.getGlobalMaximumApplicationsPerQueue();
         if (maxGlobalPerQueueApps > 0) {
           maxApplications = maxGlobalPerQueueApps;
         }
       }
 
-      priorityAcls = conf.getPriorityAcls(getQueuePath(),
-          originalConfiguration.getClusterLevelApplicationMaxPriority());
+      priorityAcls = configuration.getPriorityAcls(getQueuePath(),
+          configuration.getClusterLevelApplicationMaxPriority());
 
       Set<String> accessibleNodeLabels = this.queueNodeLabelsSettings.getAccessibleNodeLabels();
       if (!SchedulerUtils.checkQueueLabelExpression(accessibleNodeLabels,
@@ -224,10 +221,10 @@ public class AbstractLeafQueue extends AbstractCSQueue {
                         .join(getAccessibleNodeLabels().iterator(), ',')));
       }
 
-      nodeLocalityDelay = originalConfiguration.getNodeLocalityDelay();
-      rackLocalityAdditionalDelay = originalConfiguration
+      nodeLocalityDelay = configuration.getNodeLocalityDelay();
+      rackLocalityAdditionalDelay = configuration
           .getRackLocalityAdditionalDelay();
-      rackLocalityFullReset = originalConfiguration
+      rackLocalityFullReset = configuration
           .getRackLocalityFullReset();
 
       // re-init this since max allocation could have changed
@@ -250,10 +247,10 @@ public class AbstractLeafQueue extends AbstractCSQueue {
       }
 
       defaultAppPriorityPerQueue = Priority.newInstance(
-          conf.getDefaultApplicationPriorityConfPerQueue(getQueuePath()));
+          configuration.getDefaultApplicationPriorityConfPerQueue(getQueuePath()));
 
       // Validate leaf queue's user's weights.
-      float queueUserLimit = Math.min(100.0f, conf.getUserLimit(getQueuePath()));
+      float queueUserLimit = Math.min(100.0f, configuration.getUserLimit(getQueuePath()));
       getUserWeights().validateForLeafQueue(queueUserLimit, getQueuePath());
       usersManager.updateUserWeights();
 
@@ -529,9 +526,8 @@ public class AbstractLeafQueue extends AbstractCSQueue {
     }
   }
 
-  protected void reinitialize(
-      CSQueue newlyParsedQueue, Resource clusterResource,
-      CapacitySchedulerConfiguration configuration) throws
+  @Override
+  public void reinitialize(CSQueue newlyParsedQueue, Resource clusterResource) throws
       IOException {
 
     writeLock.lock();
@@ -565,20 +561,12 @@ public class AbstractLeafQueue extends AbstractCSQueue {
             + newMax);
       }
 
-      setupQueueConfigs(clusterResource, configuration);
+      setupQueueConfigs(clusterResource);
     } finally {
       writeLock.unlock();
     }
   }
 
-  @Override
-  public void reinitialize(
-      CSQueue newlyParsedQueue, Resource clusterResource)
-      throws IOException {
-    reinitialize(newlyParsedQueue, clusterResource,
-        queueContext.getConfiguration());
-  }
-
   @Override
   public void submitApplicationAttempt(FiCaSchedulerApp application,
       String userName) {
@@ -1700,13 +1688,13 @@ public class AbstractLeafQueue extends AbstractCSQueue {
   }
 
   @Override
-  protected void setDynamicQueueProperties(CapacitySchedulerConfiguration configuration) {
+  protected void setDynamicQueueProperties() {
     // set to -1, to disable it
-    configuration.setUserLimitFactor(getQueuePath(), -1);
+    queueContext.getConfiguration().setUserLimitFactor(getQueuePath(), -1);
     // Set Max AM percentage to a higher value
-    configuration.setMaximumApplicationMasterResourcePerQueuePercent(
+    queueContext.getConfiguration().setMaximumApplicationMasterResourcePerQueuePercent(
         getQueuePath(), 1f);
-    super.setDynamicQueueProperties(configuration);
+    super.setDynamicQueueProperties();
   }
 
   private void updateSchedulerHealthForCompletedContainer(
@@ -1948,7 +1936,7 @@ public class AbstractLeafQueue extends AbstractCSQueue {
       super.updateEffectiveResources(clusterResource);
 
       // Update maximum applications for the queue and for users
-      updateMaximumApplications(queueContext.getConfiguration());
+      updateMaximumApplications();
 
       updateCurrentResourceLimits(currentResourceLimits, clusterResource);
 
@@ -2342,11 +2330,12 @@ public class AbstractLeafQueue extends AbstractCSQueue {
     }
   }
 
-  void updateMaximumApplications(CapacitySchedulerConfiguration conf) {
-    int maxAppsForQueue = conf.getMaximumApplicationsPerQueue(getQueuePath());
+  void updateMaximumApplications() {
+    CapacitySchedulerConfiguration configuration = queueContext.getConfiguration();
+    int maxAppsForQueue = configuration.getMaximumApplicationsPerQueue(getQueuePath());
 
-    int maxDefaultPerQueueApps = conf.getGlobalMaximumApplicationsPerQueue();
-    int maxSystemApps = conf.getMaximumSystemApplications();
+    int maxDefaultPerQueueApps = configuration.getGlobalMaximumApplicationsPerQueue();
+    int maxSystemApps = configuration.getMaximumSystemApplications();
     int baseMaxApplications = maxDefaultPerQueueApps > 0 ?
         Math.min(maxDefaultPerQueueApps, maxSystemApps)
         : maxSystemApps;

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

@@ -19,7 +19,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
 
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common
@@ -55,7 +54,7 @@ public abstract class AbstractManagedParentQueue extends ParentQueue {
     writeLock.lock();
     try {
       // Set new configs
-      setupQueueConfigs(clusterResource, queueContext.getConfiguration());
+      setupQueueConfigs(clusterResource);
 
     } finally {
       writeLock.unlock();
@@ -175,22 +174,12 @@ public abstract class AbstractManagedParentQueue extends ParentQueue {
     CapacitySchedulerConfiguration leafQueueConfigs = new
         CapacitySchedulerConfiguration(new Configuration(false), false);
 
-    Map<String, String> rtProps = queueContext
-        .getConfiguration().getConfigurationProperties()
-        .getPropertiesWithPrefix(YarnConfiguration.RESOURCE_TYPES + ".", true);
-    for (Map.Entry<String, String> entry : rtProps.entrySet()) {
-      leafQueueConfigs.set(entry.getKey(), entry.getValue());
-    }
-
     Map<String, String> templateConfigs = queueContext
         .getConfiguration().getConfigurationProperties()
         .getPropertiesWithPrefix(configPrefix, true);
 
-    for (final Iterator<Map.Entry<String, String>> iterator =
-         templateConfigs.entrySet().iterator(); iterator.hasNext(); ) {
-      Map.Entry<String, String> confKeyValuePair = iterator.next();
-      leafQueueConfigs.set(confKeyValuePair.getKey(),
-          confKeyValuePair.getValue());
+    for (Map.Entry<String, String> confKeyValuePair : templateConfigs.entrySet()) {
+      leafQueueConfigs.set(confKeyValuePair.getKey(), confKeyValuePair.getValue());
     }
 
     return leafQueueConfigs;

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AutoCreatedLeafQueue.java

@@ -42,7 +42,8 @@ public class AutoCreatedLeafQueue extends AbstractAutoCreatedLeafQueue {
   public AutoCreatedLeafQueue(CapacitySchedulerQueueContext queueContext, String queueName,
       ManagedParentQueue parent) throws IOException {
     super(queueContext, queueName, parent, null);
-    super.setupQueueConfigs(queueContext.getClusterResource(), parent.getLeafQueueConfigs(queueName));
+    parent.setLeafQueueConfigs(queueName);
+    super.setupQueueConfigs(queueContext.getClusterResource());
 
     updateCapacitiesToZero();
   }
@@ -56,8 +57,8 @@ public class AutoCreatedLeafQueue extends AbstractAutoCreatedLeafQueue {
 
       ManagedParentQueue managedParentQueue = (ManagedParentQueue) parent;
 
-      super.reinitialize(newlyParsedQueue, clusterResource, managedParentQueue
-          .getLeafQueueConfigs(newlyParsedQueue.getQueueShortName()));
+      managedParentQueue.setLeafQueueConfigs(newlyParsedQueue.getQueueShortName());
+      super.reinitialize(newlyParsedQueue, clusterResource);
 
       //Reset capacities to 0 since reinitialize above
       // queueCapacities to initialize to configured capacity which might
@@ -122,8 +123,7 @@ public class AutoCreatedLeafQueue extends AbstractAutoCreatedLeafQueue {
   }
 
   @Override
-  protected void setDynamicQueueProperties(
-      CapacitySchedulerConfiguration configuration) {
+  protected void setDynamicQueueProperties() {
     String parentTemplate = String.format("%s.%s", getParent().getQueuePath(),
         CapacitySchedulerConfiguration
             .AUTO_CREATED_LEAF_QUEUE_TEMPLATE_PREFIX);

+ 7 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueuePreemptionSettings.java

@@ -26,13 +26,10 @@ public class CSQueuePreemptionSettings {
 
   public CSQueuePreemptionSettings(
       CSQueue queue,
-      CapacitySchedulerConfiguration configuration,
-      CapacitySchedulerConfiguration originalSchedulerConfiguration) {
-    this.preemptionDisabled = isQueueHierarchyPreemptionDisabled(queue, configuration,
-        originalSchedulerConfiguration);
+      CapacitySchedulerConfiguration configuration) {
+    this.preemptionDisabled = isQueueHierarchyPreemptionDisabled(queue, configuration);
     this.intraQueuePreemptionDisabledInHierarchy =
-        isIntraQueueHierarchyPreemptionDisabled(queue, configuration,
-            originalSchedulerConfiguration);
+        isIntraQueueHierarchyPreemptionDisabled(queue, configuration);
   }
 
   /**
@@ -46,10 +43,9 @@ public class CSQueuePreemptionSettings {
    * @return true if queue has cross-queue preemption disabled, false otherwise
    */
   private boolean isQueueHierarchyPreemptionDisabled(CSQueue q,
-      CapacitySchedulerConfiguration configuration,
-      CapacitySchedulerConfiguration originalSchedulerConfiguration) {
+      CapacitySchedulerConfiguration configuration) {
     boolean systemWidePreemption =
-        originalSchedulerConfiguration
+        configuration
             .getBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS,
                 YarnConfiguration.DEFAULT_RM_SCHEDULER_ENABLE_MONITORS);
     CSQueue parentQ = q.getParent();
@@ -85,10 +81,9 @@ public class CSQueuePreemptionSettings {
    * @return true if queue has intra-queue preemption disabled, false otherwise
    */
   private boolean isIntraQueueHierarchyPreemptionDisabled(CSQueue q,
-      CapacitySchedulerConfiguration configuration,
-      CapacitySchedulerConfiguration originalSchedulerConfiguration) {
+      CapacitySchedulerConfiguration configuration) {
     boolean systemWideIntraQueuePreemption =
-        originalSchedulerConfiguration.getBoolean(
+        configuration.getBoolean(
             CapacitySchedulerConfiguration.INTRAQUEUE_PREEMPTION_ENABLED,
             CapacitySchedulerConfiguration
                 .DEFAULT_INTRAQUEUE_PREEMPTION_ENABLED);

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

@@ -96,6 +96,10 @@ public class CapacitySchedulerQueueContext {
     return configuration;
   }
 
+  public void setConfigurationEntry(String name, String value) {
+    this.configuration.set(name, value);
+  }
+
   public Resource getMinimumAllocation() {
     return minimumAllocation;
   }

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

@@ -41,6 +41,6 @@ public class LeafQueue extends AbstractLeafQueue {
       IOException {
     super(queueContext, queueName, parent, old, isDynamic);
 
-    setupQueueConfigs(queueContext.getClusterResource(), queueContext.getConfiguration());
+    setupQueueConfigs(queueContext.getClusterResource());
   }
 }

+ 7 - 21
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ManagedParentQueue.java

@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
@@ -33,7 +32,6 @@ import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -454,25 +452,13 @@ public class ManagedParentQueue extends AbstractManagedParentQueue {
     }
   }
 
-  public CapacitySchedulerConfiguration getLeafQueueConfigs(
-      String leafQueueName) {
-    return getLeafQueueConfigs(getLeafQueueTemplate().getLeafQueueConfigs(),
-        leafQueueName);
-  }
-
-  public CapacitySchedulerConfiguration getLeafQueueConfigs(
-      CapacitySchedulerConfiguration templateConfig, String leafQueueName) {
-    CapacitySchedulerConfiguration leafQueueConfigTemplate = new
-        CapacitySchedulerConfiguration(new Configuration(false), false);
-    for (final Iterator<Map.Entry<String, String>> iterator =
-         templateConfig.iterator(); iterator.hasNext();) {
-      Map.Entry<String, String> confKeyValuePair = iterator.next();
-      final String name = confKeyValuePair.getKey().replaceFirst(
-          CapacitySchedulerConfiguration
-              .AUTO_CREATED_LEAF_QUEUE_TEMPLATE_PREFIX,
-          leafQueueName);
-      leafQueueConfigTemplate.set(name, confKeyValuePair.getValue());
+  public void setLeafQueueConfigs(String leafQueueName) {
+    CapacitySchedulerConfiguration templateConfig = leafQueueTemplate.getLeafQueueConfigs();
+    for (Map.Entry<String, String> confKeyValuePair : templateConfig) {
+      final String name = confKeyValuePair.getKey()
+          .replaceFirst(CapacitySchedulerConfiguration.AUTO_CREATED_LEAF_QUEUE_TEMPLATE_PREFIX,
+              leafQueueName);
+      queueContext.setConfigurationEntry(name, confKeyValuePair.getValue());
     }
-    return leafQueueConfigTemplate;
   }
 }

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java

@@ -134,7 +134,7 @@ public class ParentQueue extends AbstractCSQueue {
         queueContext.getConfiguration()
             .getAllowZeroCapacitySum(getQueuePath());
 
-    setupQueueConfigs(queueContext.getClusterResource(), queueContext.getConfiguration());
+    setupQueueConfigs(queueContext.getClusterResource());
   }
 
   // returns what is configured queue ordering policy
@@ -144,14 +144,14 @@ public class ParentQueue extends AbstractCSQueue {
         queueOrderingPolicy.getConfigName();
   }
 
-  protected void setupQueueConfigs(Resource clusterResource,
-      CapacitySchedulerConfiguration configuration)
+  protected void setupQueueConfigs(Resource clusterResource)
       throws IOException {
     writeLock.lock();
     try {
+      CapacitySchedulerConfiguration configuration = queueContext.getConfiguration();
       autoCreatedQueueTemplate = new AutoCreatedQueueTemplate(
           configuration, this.queuePath);
-      super.setupQueueConfigs(clusterResource, configuration);
+      super.setupQueueConfigs(clusterResource);
       StringBuilder aclsString = new StringBuilder();
       for (Map.Entry<AccessType, AccessControlList> e : getACLs().entrySet()) {
         aclsString.append(e.getKey()).append(":")
@@ -633,7 +633,7 @@ public class ParentQueue extends AbstractCSQueue {
       ParentQueue newlyParsedParentQueue = (ParentQueue) newlyParsedQueue;
 
       // Set new configs
-      setupQueueConfigs(clusterResource, queueContext.getConfiguration());
+      setupQueueConfigs(clusterResource);
 
       // Re-configure existing child queues and add new ones
       // The CS has already checked to ensure all existing child queues are present!

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

@@ -104,7 +104,7 @@ public class PlanQueue extends AbstractManagedParentQueue {
       }
 
       // Set new configs
-      setupQueueConfigs(clusterResource, queueContext.getConfiguration());
+      setupQueueConfigs(clusterResource);
 
       updateQuotas(newlyParsedParentQueue.userLimit,
           newlyParsedParentQueue.userLimitFactor,

+ 2 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueAllocationSettings.java

@@ -36,17 +36,10 @@ public class QueueAllocationSettings {
     this.minimumAllocation = minimumAllocation;
   }
 
-  void setupMaximumAllocation(CapacitySchedulerConfiguration configuration,
-      CapacitySchedulerConfiguration originalSchedulerConfiguration, String queuePath,
+  void setupMaximumAllocation(CapacitySchedulerConfiguration configuration, String queuePath,
       CSQueue parent) {
-    /* YARN-10869: When using AutoCreatedLeafQueues, the passed configuration
-     * object is a cloned one containing only the template configs
-     * (see ManagedParentQueue#getLeafQueueConfigs). To ensure that the actual
-     * cluster maximum allocation is fetched the original config object should
-     * be used.
-     */
     Resource clusterMax = ResourceUtils
-        .fetchMaximumAllocationFromConfig(originalSchedulerConfiguration);
+        .fetchMaximumAllocationFromConfig(configuration);
     Resource queueMax = configuration.getQueueMaximumAllocation(queuePath);
 
     maximumAllocation = Resources.clone(

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

@@ -39,8 +39,7 @@ public class ReservationQueue extends AbstractAutoCreatedLeafQueue {
   public ReservationQueue(CapacitySchedulerQueueContext queueContext, String queueName,
       PlanQueue parent) throws IOException {
     super(queueContext, queueName, parent, null);
-    super.setupQueueConfigs(queueContext.getClusterResource(),
-        queueContext.getConfiguration());
+    super.setupQueueConfigs(queueContext.getClusterResource());
 
     // the following parameters are common to all reservation in the plan
     updateQuotas(parent.getUserLimitForReservation(),
@@ -84,8 +83,7 @@ public class ReservationQueue extends AbstractAutoCreatedLeafQueue {
   }
 
   @Override
-  protected void setupConfigurableCapacities(CapacitySchedulerConfiguration
-      configuration) {
+  protected void setupConfigurableCapacities() {
     super.updateAbsoluteCapacities();
   }
 }

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

@@ -177,7 +177,7 @@ public class TestCapacitySchedulerPerf {
       LeafQueue qb = (LeafQueue)cs.getQueue(queueName);
       // For now make user limit large so we can activate all applications
       qb.setUserLimitFactor((float)100.0);
-      qb.setupConfigurableCapacities(cs.getConfiguration());
+      qb.setupConfigurableCapacities();
       lqs[i] = qb;
     }