Prechádzať zdrojové kódy

YARN-8202. DefaultAMSProcessor should properly check units of requested custom resource types against minimum/maximum allocation (snemeth via rkanter)

(cherry picked from commit c8b53c43644b4ad22d5385c22cad8ed573c0b1ba)
Robert Kanter 7 rokov pred
rodič
commit
0506c762b2

+ 57 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/ContainerRequestCreator.java

@@ -0,0 +1,57 @@
+/*
+ * 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.mapreduce.v2.app.rm;
+
+import org.apache.hadoop.mapreduce.v2.api.records.JobId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
+import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
+import org.apache.hadoop.net.NetworkTopology;
+import org.apache.hadoop.yarn.api.records.Resource;
+
+final class ContainerRequestCreator {
+
+  private ContainerRequestCreator() {}
+
+  static ContainerRequestEvent createRequest(JobId jobId, int taskAttemptId,
+          Resource resource, String[] hosts) {
+    return createRequest(jobId, taskAttemptId, resource, hosts,
+            false, false);
+  }
+
+  static ContainerRequestEvent createRequest(JobId jobId, int taskAttemptId,
+          Resource resource, String[] hosts, boolean earlierFailedAttempt,
+          boolean reduce) {
+    final TaskId taskId;
+    if (reduce) {
+      taskId = MRBuilderUtils.newTaskId(jobId, 0, TaskType.REDUCE);
+    } else {
+      taskId = MRBuilderUtils.newTaskId(jobId, 0, TaskType.MAP);
+    }
+    TaskAttemptId attemptId = MRBuilderUtils.newTaskAttemptId(taskId,
+            taskAttemptId);
+
+    if (earlierFailedAttempt) {
+      return ContainerRequestEvent
+              .createContainerRequestEventForFailedContainer(attemptId,
+                      resource);
+    }
+    return new ContainerRequestEvent(attemptId, resource, hosts,
+            new String[]{NetworkTopology.DEFAULT_RACK});
+  }
+}

Rozdielové dáta súboru neboli zobrazené, pretože súbor je príliš veľký
+ 205 - 196
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java


+ 34 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/UnitsConversionUtil.java

@@ -175,16 +175,8 @@ public class UnitsConversionUtil {
    */
   public static int compare(String unitA, long valueA, String unitB,
       long valueB) {
-    if (unitA == null || unitB == null || !KNOWN_UNITS.contains(unitA)
-        || !KNOWN_UNITS.contains(unitB)) {
-      throw new IllegalArgumentException("Units cannot be null");
-    }
-    if (!KNOWN_UNITS.contains(unitA)) {
-      throw new IllegalArgumentException("Unknown unit '" + unitA + "'");
-    }
-    if (!KNOWN_UNITS.contains(unitB)) {
-      throw new IllegalArgumentException("Unknown unit '" + unitB + "'");
-    }
+    checkUnitArgument(unitA);
+    checkUnitArgument(unitB);
     if (unitA.equals(unitB)) {
       return Long.compare(valueA, valueB);
     }
@@ -218,4 +210,36 @@ public class UnitsConversionUtil {
       return tmpA.compareTo(tmpB);
     }
   }
+
+  private static void checkUnitArgument(String unit) {
+    if (unit == null) {
+      throw new IllegalArgumentException("Unit cannot be null");
+    } else if (!KNOWN_UNITS.contains(unit)) {
+      throw new IllegalArgumentException("Unknown unit '" + unit + "'");
+    }
+  }
+
+  /**
+   * Compare a unit to another unit.
+   * <br>
+   * Examples:<br>
+   * 1. 'm' (milli) is smaller than 'k' (kilo), so compareUnits("m", "k")
+   * will return -1.<br>
+   * 2. 'M' (MEGA) is greater than 'k' (kilo), so compareUnits("M", "k") will
+   * return 1.
+   *
+   * @param unitA first unit
+   * @param unitB second unit
+   * @return +1, 0 or -1 depending on whether the relationship between units
+   * is smaller than,
+   * equal to or lesser than.
+   */
+  public static int compareUnits(String unitA, String unitB) {
+    checkUnitArgument(unitA);
+    checkUnitArgument(unitB);
+    int unitAPos = SORTED_UNITS.indexOf(unitA);
+    int unitBPos = SORTED_UNITS.indexOf(unitB);
+
+    return Integer.compare(unitAPos, unitBPos);
+  }
 }

+ 93 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/resourcetypes/ResourceTypesTestHelper.java

@@ -0,0 +1,93 @@
+/*
+ * 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.resourcetypes;
+
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Contains helper methods to create Resource and ResourceInformation objects.
+ * ResourceInformation can be created from a resource name
+ * and a resource descriptor as well that comprises amount and unit.
+ */
+public final class ResourceTypesTestHelper {
+
+  private static final Pattern RESOURCE_VALUE_AND_UNIT_PATTERN =
+      Pattern.compile("(\\d+)([A-za-z]*)");
+
+  private ResourceTypesTestHelper() {}
+
+  private static final RecordFactory RECORD_FACTORY = RecordFactoryProvider
+          .getRecordFactory(null);
+
+  private static final class ResourceValueAndUnit {
+    private final Long value;
+    private final String unit;
+
+    private ResourceValueAndUnit(Long value, String unit) {
+      this.value = value;
+      this.unit = unit;
+    }
+  }
+
+  public static Resource newResource(long memory, int vCores, Map<String,
+          String> customResources) {
+    Resource resource = RECORD_FACTORY.newRecordInstance(Resource.class);
+    resource.setMemorySize(memory);
+    resource.setVirtualCores(vCores);
+
+    for (Map.Entry<String, String> customResource :
+            customResources.entrySet()) {
+      String resourceName = customResource.getKey();
+      ResourceInformation resourceInformation =
+              createResourceInformation(resourceName,
+                      customResource.getValue());
+      resource.setResourceInformation(resourceName, resourceInformation);
+    }
+    return resource;
+  }
+
+  public static ResourceInformation createResourceInformation(String
+          resourceName, String descriptor) {
+    ResourceValueAndUnit resourceValueAndUnit =
+            getResourceValueAndUnit(descriptor);
+    return ResourceInformation
+            .newInstance(resourceName, resourceValueAndUnit.unit,
+                    resourceValueAndUnit.value);
+  }
+
+  private static ResourceValueAndUnit getResourceValueAndUnit(String val) {
+    Matcher matcher = RESOURCE_VALUE_AND_UNIT_PATTERN.matcher(val);
+    if (!matcher.find()) {
+      throw new RuntimeException("Invalid pattern of resource descriptor: " +
+              val);
+    } else if (matcher.groupCount() != 2) {
+      throw new RuntimeException("Capturing group count in string " +
+              val + " is not 2!");
+    }
+    long value = Long.parseLong(matcher.group(1));
+
+    return new ResourceValueAndUnit(value, matcher.group(2));
+  }
+
+}

+ 6 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java

@@ -183,7 +183,7 @@ public class BuilderUtils {
   public static NodeId newNodeId(String host, int port) {
     return NodeId.newInstance(host, port);
   }
-  
+
   public static NodeReport newNodeReport(NodeId nodeId, NodeState nodeState,
       String httpAddress, String rackName, Resource used, Resource capability,
       int numContainers, String healthReport, long lastHealthReportTime) {
@@ -421,7 +421,7 @@ public class BuilderUtils {
     report.setPriority(priority);
     return report;
   }
-  
+
   public static ApplicationSubmissionContext newApplicationSubmissionContext(
       ApplicationId applicationId, String applicationName, String queue,
       Priority priority, ContainerLaunchContext amContainer,
@@ -476,6 +476,10 @@ public class BuilderUtils {
     return resource;
   }
 
+  public static Resource newEmptyResource() {
+    return recordFactory.newRecordInstance(Resource.class);
+  }
+
   public static URL newURL(String scheme, String host, int port, String file) {
     URL url = recordFactory.newRecordInstance(URL.class);
     url.setScheme(scheme);

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

@@ -49,6 +49,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
+import org.apache.hadoop.yarn.util.UnitsConversionUtil;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -283,24 +284,10 @@ public class SchedulerUtils {
   private static void validateResourceRequest(ResourceRequest resReq,
       Resource maximumResource, QueueInfo queueInfo, RMContext rmContext)
       throws InvalidResourceRequestException {
-    Resource requestedResource = resReq.getCapability();
-    for (int i = 0; i < ResourceUtils.getNumberOfKnownResourceTypes(); i++) {
-      ResourceInformation reqRI = requestedResource.getResourceInformation(i);
-      ResourceInformation maxRI = maximumResource.getResourceInformation(i);
-      if (reqRI.getValue() < 0 || reqRI.getValue() > maxRI.getValue()) {
-        throw new InvalidResourceRequestException(
-            "Invalid resource request, requested resource type=[" + reqRI
-                .getName()
-                + "] < 0 or greater than maximum allowed allocation. Requested "
-                + "resource=" + requestedResource
-                + ", maximum allowed allocation=" + maximumResource
-                + ", please note that maximum allowed allocation is calculated "
-                + "by scheduler based on maximum resource of registered "
-                + "NodeManagers, which might be less than configured "
-                + "maximum allocation=" + ResourceUtils
-                .getResourceTypesMaximumAllocation());
-      }
-    }
+    final Resource requestedResource = resReq.getCapability();
+    checkResourceRequestAgainstAvailableResource(requestedResource,
+        maximumResource);
+
     String labelExp = resReq.getNodeLabelExpression();
     // we don't allow specify label expression other than resourceName=ANY now
     if (!ResourceRequest.ANY.equals(resReq.getResourceName())
@@ -338,6 +325,78 @@ public class SchedulerUtils {
     }
   }
 
+  @Private
+  @VisibleForTesting
+  static void checkResourceRequestAgainstAvailableResource(Resource reqResource,
+      Resource availableResource) throws InvalidResourceRequestException {
+    for (int i = 0; i < ResourceUtils.getNumberOfKnownResourceTypes(); i++) {
+      final ResourceInformation requestedRI =
+          reqResource.getResourceInformation(i);
+      final String reqResourceName = requestedRI.getName();
+
+      if (requestedRI.getValue() < 0) {
+        throwInvalidResourceException(reqResource, availableResource,
+            reqResourceName);
+      }
+
+      final ResourceInformation availableRI =
+          availableResource.getResourceInformation(reqResourceName);
+
+      long requestedResourceValue = requestedRI.getValue();
+      long availableResourceValue = availableRI.getValue();
+      int unitsRelation = UnitsConversionUtil
+          .compareUnits(requestedRI.getUnits(), availableRI.getUnits());
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Requested resource information: " + requestedRI);
+        LOG.debug("Available resource information: " + availableRI);
+        LOG.debug("Relation of units: " + unitsRelation);
+      }
+
+      // requested resource unit is less than available resource unit
+      // e.g. requestedUnit: "m", availableUnit: "K")
+      if (unitsRelation < 0) {
+        availableResourceValue =
+            UnitsConversionUtil.convert(availableRI.getUnits(),
+                requestedRI.getUnits(), availableRI.getValue());
+
+        // requested resource unit is greater than available resource unit
+        // e.g. requestedUnit: "G", availableUnit: "M")
+      } else if (unitsRelation > 0) {
+        requestedResourceValue =
+            UnitsConversionUtil.convert(requestedRI.getUnits(),
+                availableRI.getUnits(), requestedRI.getValue());
+      }
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Requested resource value after conversion: " +
+                requestedResourceValue);
+        LOG.info("Available resource value after conversion: " +
+                availableResourceValue);
+      }
+
+      if (requestedResourceValue > availableResourceValue) {
+        throwInvalidResourceException(reqResource, availableResource,
+            reqResourceName);
+      }
+    }
+  }
+
+  private static void throwInvalidResourceException(Resource reqResource,
+      Resource availableResource, String reqResourceName)
+      throws InvalidResourceRequestException {
+    throw new InvalidResourceRequestException(
+        "Invalid resource request, requested resource type=[" + reqResourceName
+            + "] < 0 or greater than maximum allowed allocation. Requested "
+            + "resource=" + reqResource + ", maximum allowed allocation="
+            + availableResource
+            + ", please note that maximum allowed allocation is calculated "
+            + "by scheduler based on maximum resource of registered "
+            + "NodeManagers, which might be less than configured "
+            + "maximum allocation="
+            + ResourceUtils.getResourceTypesMaximumAllocation());
+  }
+
   private static void checkQueueLabelInLabelManager(String labelExpression,
       RMContext rmContext) throws InvalidLabelResourceRequestException {
     // check node label manager contains this label

+ 141 - 44
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterService.java

@@ -22,9 +22,13 @@ import static java.lang.Thread.sleep;
 import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB;
 import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES;
 
+
+import static org.junit.Assert.fail;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.List;
@@ -61,6 +65,7 @@ import org.apache.hadoop.yarn.exceptions.InvalidContainerReleaseException;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
+import org.apache.hadoop.yarn.resourcetypes.ResourceTypesTestHelper;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.TestResourceProfiles;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
@@ -75,6 +80,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair
+        .FairSchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
@@ -365,7 +373,7 @@ public class TestApplicationMasterService {
       am2.addContainerToBeReleased(cId);
       try {
         am2.schedule();
-        Assert.fail("Exception was expected!!");
+        fail("Exception was expected!!");
       } catch (InvalidContainerReleaseException e) {
         StringBuilder sb = new StringBuilder("Cannot release container : ");
         sb.append(cId.toString());
@@ -460,7 +468,7 @@ public class TestApplicationMasterService {
               FinalApplicationStatus.FAILED, "", "");
       try {
         am1.unregisterAppAttempt(req, false);
-        Assert.fail("ApplicationMasterNotRegisteredException should be thrown");
+        fail("ApplicationMasterNotRegisteredException should be thrown");
       } catch (ApplicationMasterNotRegisteredException e) {
         Assert.assertNotNull(e);
         Assert.assertNotNull(e.getMessage());
@@ -468,7 +476,7 @@ public class TestApplicationMasterService {
             "Application Master is trying to unregister before registering for:"
         ));
       } catch (Exception e) {
-        Assert.fail("ApplicationMasterNotRegisteredException should be thrown");
+        fail("ApplicationMasterNotRegisteredException should be thrown");
       }
 
       am1.registerAppAttempt();
@@ -627,9 +635,7 @@ public class TestApplicationMasterService {
       Assert.assertEquals("UPDATE_OUTSTANDING_ERROR",
           response.getUpdateErrors().get(0).getReason());
     } finally {
-      if (rm != null) {
-        rm.close();
-      }
+      rm.close();
     }
   }
 
@@ -709,34 +715,48 @@ public class TestApplicationMasterService {
 
     ResourceUtils.initializeResourcesFromResourceInformationMap(riMap);
 
-    CapacitySchedulerConfiguration csconf =
-        new CapacitySchedulerConfiguration();
-    csconf.setResourceComparator(DominantResourceCalculator.class);
+    final YarnConfiguration yarnConf;
+    if (schedulerCls.getCanonicalName()
+        .equals(CapacityScheduler.class.getCanonicalName())) {
+      CapacitySchedulerConfiguration csConf =
+          new CapacitySchedulerConfiguration();
+      csConf.setResourceComparator(DominantResourceCalculator.class);
+      yarnConf = new YarnConfiguration(csConf);
+    } else if (schedulerCls.getCanonicalName()
+        .equals(FairScheduler.class.getCanonicalName())) {
+      FairSchedulerConfiguration fsConf = new FairSchedulerConfiguration();
+      yarnConf = new YarnConfiguration(fsConf);
+    } else {
+      throw new IllegalStateException(
+          "Scheduler class is of wrong type: " + schedulerCls);
+    }
 
-    YarnConfiguration conf = new YarnConfiguration(csconf);
     // Don't reset resource types since we have already configured resource
     // types
-    conf.setBoolean(TestResourceProfiles.TEST_CONF_RESET_RESOURCE_TYPES, false);
-    conf.setClass(YarnConfiguration.RM_SCHEDULER, schedulerCls,
+    yarnConf.setBoolean(TestResourceProfiles.TEST_CONF_RESET_RESOURCE_TYPES,
+        false);
+    yarnConf.setClass(YarnConfiguration.RM_SCHEDULER, schedulerCls,
         ResourceScheduler.class);
-    conf.setBoolean(YarnConfiguration.RM_RESOURCE_PROFILES_ENABLED, false);
+    yarnConf.setBoolean(YarnConfiguration.RM_RESOURCE_PROFILES_ENABLED, false);
 
-    MockRM rm = new MockRM(conf);
+    MockRM rm = new MockRM(yarnConf);
     rm.start();
 
     MockNM nm1 = rm.registerNode("199.99.99.1:1234", TestUtils
         .createResource(DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
             DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES, null));
 
-    RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "default");
+    RMApp app1 = rm.submitApp(GB, "app", "user", null, "default");
     MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm1);
 
     // Now request resource, memory > allowed
     boolean exception = false;
     try {
-      am1.allocate(Arrays.asList(ResourceRequest.newBuilder().capability(
-          Resource.newInstance(9 * GB, 1)).numContainers(1).resourceName("*")
-          .build()), null);
+      am1.allocate(Collections.singletonList(ResourceRequest.newBuilder()
+              .capability(Resource.newInstance(9 * GB, 1))
+              .numContainers(1)
+              .resourceName("*")
+              .build()), null);
     } catch (InvalidResourceRequestException e) {
       exception = true;
     }
@@ -744,10 +764,12 @@ public class TestApplicationMasterService {
 
     exception = false;
     try {
-      // Now request resource, vcore > allowed
-      am1.allocate(Arrays.asList(ResourceRequest.newBuilder().capability(
-          Resource.newInstance(8 * GB, 18)).numContainers(1).resourceName("*")
-          .build()), null);
+      // Now request resource, vcores > allowed
+      am1.allocate(Collections.singletonList(ResourceRequest.newBuilder()
+              .capability(Resource.newInstance(8 * GB, 18))
+              .numContainers(1)
+              .resourceName("*")
+              .build()), null);
     } catch (InvalidResourceRequestException e) {
       exception = true;
     }
@@ -756,6 +778,73 @@ public class TestApplicationMasterService {
     rm.close();
   }
 
+  @Test
+  public void testValidateRequestCapacityAgainstMinMaxAllocationWithDifferentUnits()
+      throws Exception {
+
+    // Initialize resource map for 2 types.
+    Map<String, ResourceInformation> riMap = new HashMap<>();
+
+    // Initialize mandatory resources
+    ResourceInformation memory =
+        ResourceInformation.newInstance(ResourceInformation.MEMORY_MB.getName(),
+            ResourceInformation.MEMORY_MB.getUnits(),
+            YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
+            YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB);
+    ResourceInformation vcores =
+        ResourceInformation.newInstance(ResourceInformation.VCORES.getName(),
+            ResourceInformation.VCORES.getUnits(),
+            YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES,
+            DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
+    ResourceInformation res1 =
+        ResourceInformation.newInstance("res_1", "G", 0, 4);
+    riMap.put(ResourceInformation.MEMORY_URI, memory);
+    riMap.put(ResourceInformation.VCORES_URI, vcores);
+    riMap.put("res_1", res1);
+
+    ResourceUtils.initializeResourcesFromResourceInformationMap(riMap);
+
+    FairSchedulerConfiguration fsConf =
+            new FairSchedulerConfiguration();
+
+    YarnConfiguration yarnConf = new YarnConfiguration(fsConf);
+    // Don't reset resource types since we have already configured resource
+    // types
+    yarnConf.setBoolean(TestResourceProfiles.TEST_CONF_RESET_RESOURCE_TYPES,
+        false);
+    yarnConf.setClass(YarnConfiguration.RM_SCHEDULER, FairScheduler.class,
+        ResourceScheduler.class);
+    yarnConf.setBoolean(YarnConfiguration.RM_RESOURCE_PROFILES_ENABLED, false);
+
+    MockRM rm = new MockRM(yarnConf);
+    rm.start();
+
+    MockNM nm1 = rm.registerNode("199.99.99.1:1234",
+        ResourceTypesTestHelper.newResource(
+            DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
+            DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
+            ImmutableMap.<String, String> builder()
+                .put("res_1", "5G").build()));
+
+    RMApp app1 = rm.submitApp(GB, "app", "user", null, "default");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm1);
+
+    // Now request res_1, 500M < 5G so it should be allowed
+    try {
+      am1.allocate(Collections.singletonList(ResourceRequest.newBuilder()
+          .capability(ResourceTypesTestHelper.newResource(4 * GB, 1,
+              ImmutableMap.<String, String> builder()
+                  .put("res_1", "500M")
+                      .build()))
+          .numContainers(1).resourceName("*").build()), null);
+    } catch (InvalidResourceRequestException e) {
+      fail(
+          "Allocate request should be accepted but exception was thrown: " + e);
+    }
+
+    rm.close();
+  }
+
   @Test(timeout = 300000)
   public void testValidateRequestCapacityAgainstMinMaxAllocationFor3rdResourceTypes()
       throws Exception {
@@ -774,11 +863,11 @@ public class TestApplicationMasterService {
         ResourceInformation.VCORES.getUnits(),
         YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES,
         DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
-    ResourceInformation res_1 = ResourceInformation.newInstance("res_1",
+    ResourceInformation res1 = ResourceInformation.newInstance("res_1",
         ResourceInformation.VCORES.getUnits(), 0, 4);
     riMap.put(ResourceInformation.MEMORY_URI, memory);
     riMap.put(ResourceInformation.VCORES_URI, vcores);
-    riMap.put("res_1", res_1);
+    riMap.put("res_1", res1);
 
     ResourceUtils.initializeResourcesFromResourceInformationMap(riMap);
 
@@ -786,15 +875,16 @@ public class TestApplicationMasterService {
         new CapacitySchedulerConfiguration();
     csconf.setResourceComparator(DominantResourceCalculator.class);
 
-    YarnConfiguration conf = new YarnConfiguration(csconf);
+    YarnConfiguration yarnConf = new YarnConfiguration(csconf);
     // Don't reset resource types since we have already configured resource
     // types
-    conf.setBoolean(TestResourceProfiles.TEST_CONF_RESET_RESOURCE_TYPES, false);
-    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+    yarnConf.setBoolean(TestResourceProfiles.TEST_CONF_RESET_RESOURCE_TYPES,
+        false);
+    yarnConf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
         ResourceScheduler.class);
-    conf.setBoolean(YarnConfiguration.RM_RESOURCE_PROFILES_ENABLED, false);
+    yarnConf.setBoolean(YarnConfiguration.RM_RESOURCE_PROFILES_ENABLED, false);
 
-    MockRM rm = new MockRM(conf);
+    MockRM rm = new MockRM(yarnConf);
     rm.start();
 
     CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
@@ -805,18 +895,21 @@ public class TestApplicationMasterService {
             DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
             ImmutableMap.of("res_1", 4)));
 
-    RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "default");
+    RMApp app1 = rm.submitApp(GB, "app", "user", null, "default");
     MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm1);
 
-    Assert.assertEquals(Resource.newInstance(1 * GB, 1),
+    Assert.assertEquals(Resource.newInstance(GB, 1),
         leafQueue.getUsedResources());
 
     // Now request resource, memory > allowed
     boolean exception = false;
     try {
-      am1.allocate(Arrays.asList(ResourceRequest.newBuilder().capability(
-          TestUtils.createResource(9 * GB, 1, ImmutableMap.of("res_1", 1)))
-          .numContainers(1).resourceName("*").build()), null);
+      am1.allocate(Collections.singletonList(ResourceRequest.newBuilder()
+              .capability(TestUtils.createResource(9 * GB, 1,
+                      ImmutableMap.of("res_1", 1)))
+              .numContainers(1)
+              .resourceName("*")
+              .build()), null);
     } catch (InvalidResourceRequestException e) {
       exception = true;
     }
@@ -824,11 +917,13 @@ public class TestApplicationMasterService {
 
     exception = false;
     try {
-      // Now request resource, vcore > allowed
-      am1.allocate(Arrays.asList(ResourceRequest.newBuilder().capability(
-          TestUtils.createResource(8 * GB, 18, ImmutableMap.of("res_1", 1)))
-          .numContainers(1).resourceName("*")
-          .build()), null);
+      // Now request resource, vcores > allowed
+      am1.allocate(Collections.singletonList(ResourceRequest.newBuilder()
+          .capability(
+              TestUtils.createResource(8 * GB, 18, ImmutableMap.of("res_1", 1)))
+              .numContainers(1)
+              .resourceName("*")
+              .build()), null);
     } catch (InvalidResourceRequestException e) {
       exception = true;
     }
@@ -837,10 +932,12 @@ public class TestApplicationMasterService {
     exception = false;
     try {
       // Now request resource, res_1 > allowed
-      am1.allocate(Arrays.asList(ResourceRequest.newBuilder().capability(
-          TestUtils.createResource(8 * GB, 1, ImmutableMap.of("res_1", 100)))
-          .numContainers(1).resourceName("*")
-          .build()), null);
+      am1.allocate(Collections.singletonList(ResourceRequest.newBuilder()
+              .capability(TestUtils.createResource(8 * GB, 1,
+                      ImmutableMap.of("res_1", 100)))
+              .numContainers(1)
+              .resourceName("*")
+              .build()), null);
     } catch (InvalidResourceRequestException e) {
       exception = true;
     }
@@ -856,7 +953,7 @@ public class TestApplicationMasterService {
       rmContainer.handle(
           new RMContainerEvent(containerId, RMContainerEventType.LAUNCHED));
     } else {
-      Assert.fail("Cannot find RMContainer");
+      fail("Cannot find RMContainer");
     }
   }
 }

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

@@ -26,7 +26,9 @@ import static org.mockito.Matchers.anyBoolean;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
+import java.io.ByteArrayInputStream;
 import java.io.IOException;
+import java.io.InputStream;
 import java.net.InetSocketAddress;
 import java.security.PrivilegedAction;
 import java.util.Arrays;
@@ -35,6 +37,7 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 
+import com.google.common.collect.ImmutableMap;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -42,6 +45,7 @@ import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.yarn.LocalConfigurationProvider;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
@@ -63,8 +67,10 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.InvalidLabelResourceRequestException;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceBlacklistRequestException;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
+import org.apache.hadoop.yarn.resourcetypes.ResourceTypesTestHelper;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MockRMWithAMS;
@@ -83,20 +89,79 @@ import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Sets;
+import org.junit.rules.ExpectedException;
 
 public class TestSchedulerUtils {
 
   private static final Log LOG = LogFactory.getLog(TestSchedulerUtils.class);
-  
+  private static Resource configuredMaxAllocation;
+
+  private static class CustomResourceTypesConfigurationProvider
+          extends LocalConfigurationProvider {
+
+    @Override
+    public InputStream getConfigurationInputStream(Configuration bootstrapConf,
+            String name) throws YarnException, IOException {
+      if (YarnConfiguration.RESOURCE_TYPES_CONFIGURATION_FILE.equals(name)) {
+        return new ByteArrayInputStream(
+                ("<configuration>\n" +
+                        " <property>\n" +
+                        "   <name>yarn.resource-types</name>\n" +
+                        "   <value>custom-resource-1," +
+                        "custom-resource-2,custom-resource-3</value>\n" +
+                        " </property>\n" +
+                        " <property>\n" +
+                        "   <name>yarn.resource-types" +
+                        ".custom-resource-1.units</name>\n" +
+                        "   <value>G</value>\n" +
+                        " </property>\n" +
+                        " <property>\n" +
+                        "   <name>yarn.resource-types" +
+                        ".custom-resource-2.units</name>\n" +
+                        "   <value>G</value>\n" +
+                        " </property>\n" +
+                        "</configuration>\n").getBytes());
+      } else {
+        return super.getConfigurationInputStream(bootstrapConf, name);
+      }
+    }
+  }
   private RMContext rmContext = getMockRMContext();
+
   private static YarnConfiguration conf = new YarnConfiguration();
 
+  @Rule
+  public ExpectedException exception = ExpectedException.none();
+
+  private void initResourceTypes() {
+    Configuration yarnConf = new Configuration();
+    yarnConf.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS,
+        CustomResourceTypesConfigurationProvider.class.getName());
+    ResourceUtils.resetResourceTypes(yarnConf);
+  }
+
+  @Before
+  public void setUp() {
+    initResourceTypes();
+    //this needs to be initialized after initResourceTypes is called
+    configuredMaxAllocation = Resource.newInstance(8192, 4,
+            ImmutableMap.<String,
+                    Long>builder()
+                    .put("custom-resource-1", Long.MAX_VALUE)
+                    .put("custom-resource-2", Long.MAX_VALUE)
+                    .put("custom-resource-3", Long.MAX_VALUE)
+                    .build());
+  }
+
   @Test (timeout = 30000)
   public void testNormalizeRequest() {
     ResourceCalculator resourceCalculator = new DefaultResourceCalculator();
@@ -150,16 +215,18 @@ public class TestSchedulerUtils {
     // multiple of minMemory > maxMemory, then reduce to maxMemory
     SchedulerUtils.normalizeRequest(ask, resourceCalculator, minResource,
         maxResource);
-    assertEquals(maxResource.getMemorySize(), ask.getCapability().getMemorySize());
+    assertEquals(maxResource.getMemorySize(),
+        ask.getCapability().getMemorySize());
 
     // ask is more than max
     maxResource = Resources.createResource(maxMemory, 0);
     ask.setCapability(Resources.createResource(maxMemory + 100));
     SchedulerUtils.normalizeRequest(ask, resourceCalculator, minResource,
         maxResource);
-    assertEquals(maxResource.getMemorySize(), ask.getCapability().getMemorySize());
+    assertEquals(maxResource.getMemorySize(),
+        ask.getCapability().getMemorySize());
   }
-  
+
   @Test (timeout = 30000)
   public void testNormalizeRequestWithDominantResourceCalculator() {
     ResourceCalculator resourceCalculator = new DominantResourceCalculator();
@@ -201,10 +268,11 @@ public class TestSchedulerUtils {
     Set<String> queueAccessibleNodeLabels = Sets.newHashSet();
     QueueInfo queueInfo = mock(QueueInfo.class);
     when(queueInfo.getQueueName()).thenReturn("queue");
-    when(queueInfo.getAccessibleNodeLabels()).thenReturn(queueAccessibleNodeLabels);
+    when(queueInfo.getAccessibleNodeLabels())
+        .thenReturn(queueAccessibleNodeLabels);
     when(scheduler.getQueueInfo(any(String.class), anyBoolean(), anyBoolean()))
         .thenReturn(queueInfo);
-    
+
     Resource maxResource = Resources.createResource(
         YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
         YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
@@ -363,7 +431,7 @@ public class TestSchedulerUtils {
       rmContext.getNodeLabelManager().removeFromClusterNodeLabels(
           Arrays.asList("x"));
     }
-    Assert.assertTrue("InvalidLabelResourceRequestException excpeted",
+    Assert.assertTrue("InvalidLabelResourceRequestException expected",
         invalidlabelexception);
     // queue is "*", always succeeded
     try {
@@ -610,11 +678,9 @@ public class TestSchedulerUtils {
 
     // more than max vcores
     try {
-      Resource resource =
-          Resources
-              .createResource(
-                  YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
-                  YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES + 1);
+      Resource resource = Resources.createResource(
+          YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
+          YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES + 1);
       ResourceRequest resReq =
           BuilderUtils.newResourceRequest(mock(Priority.class),
               ResourceRequest.ANY, resource, 1);
@@ -648,10 +714,10 @@ public class TestSchedulerUtils {
     waitForLaunchedState(attempt);
 
     // Create a client to the RM.
-    final Configuration conf = rm.getConfig();
-    final YarnRPC rpc = YarnRPC.create(conf);
+    final Configuration yarnConf = rm.getConfig();
+    final YarnRPC rpc = YarnRPC.create(yarnConf);
 
-    UserGroupInformation currentUser = 
+    UserGroupInformation currentUser =
         UserGroupInformation.createRemoteUser(applicationAttemptId.toString());
     Credentials credentials = containerManager.getContainerCredentials();
     final InetSocketAddress rmBindAddress =
@@ -665,7 +731,7 @@ public class TestSchedulerUtils {
           @Override
           public ApplicationMasterProtocol run() {
             return (ApplicationMasterProtocol) rpc.getProxy(
-              ApplicationMasterProtocol.class, rmBindAddress, conf);
+              ApplicationMasterProtocol.class, rmBindAddress, yarnConf);
           }
         });
 
@@ -775,6 +841,127 @@ public class TestSchedulerUtils {
     }
   }
 
+  @Test
+  public void testCustomResourceRequestedUnitIsSmallerThanAvailableUnit()
+      throws InvalidResourceRequestException {
+    Resource requestedResource =
+        ResourceTypesTestHelper.newResource(1, 1,
+                ImmutableMap.of("custom-resource-1", "11"));
+
+    Resource availableResource =
+        ResourceTypesTestHelper.newResource(1, 1,
+                ImmutableMap.of("custom-resource-1", "0G"));
+
+    exception.expect(InvalidResourceRequestException.class);
+    exception.expectMessage(InvalidResourceRequestExceptionMessageGenerator
+        .create().withRequestedResourceType("custom-resource-1")
+        .withRequestedResource(requestedResource)
+        .withAvailableAllocation(availableResource)
+        .withMaxAllocation(configuredMaxAllocation).build());
+
+    SchedulerUtils.checkResourceRequestAgainstAvailableResource(
+        requestedResource, availableResource);
+  }
+
+  @Test
+  public void testCustomResourceRequestedUnitIsSmallerThanAvailableUnit2() {
+    Resource requestedResource =
+        ResourceTypesTestHelper.newResource(1, 1,
+                ImmutableMap.of("custom-resource-1", "11"));
+
+    Resource availableResource =
+        ResourceTypesTestHelper.newResource(1, 1,
+                ImmutableMap.of("custom-resource-1", "1G"));
+
+    try {
+      SchedulerUtils.checkResourceRequestAgainstAvailableResource(
+          requestedResource, availableResource);
+    } catch (InvalidResourceRequestException e) {
+      fail(String.format(
+          "Resource request should be accepted. Requested: %s, available: %s",
+          requestedResource, availableResource));
+    }
+  }
+
+  @Test
+  public void testCustomResourceRequestedUnitIsGreaterThanAvailableUnit()
+      throws InvalidResourceRequestException {
+    Resource requestedResource =
+        ResourceTypesTestHelper.newResource(1, 1,
+                ImmutableMap.of("custom-resource-1", "1M"));
+
+    Resource availableResource = ResourceTypesTestHelper.newResource(1, 1,
+        ImmutableMap.<String, String> builder().put("custom-resource-1", "120k")
+            .build());
+
+    exception.expect(InvalidResourceRequestException.class);
+    exception.expectMessage(InvalidResourceRequestExceptionMessageGenerator
+        .create().withRequestedResourceType("custom-resource-1")
+        .withRequestedResource(requestedResource)
+        .withAvailableAllocation(availableResource)
+        .withMaxAllocation(configuredMaxAllocation).build());
+    SchedulerUtils.checkResourceRequestAgainstAvailableResource(
+        requestedResource, availableResource);
+  }
+
+  @Test
+  public void testCustomResourceRequestedUnitIsGreaterThanAvailableUnit2() {
+    Resource requestedResource = ResourceTypesTestHelper.newResource(1, 1,
+        ImmutableMap.<String, String> builder().put("custom-resource-1", "11M")
+            .build());
+
+    Resource availableResource =
+        ResourceTypesTestHelper.newResource(1, 1,
+                ImmutableMap.of("custom-resource-1", "1G"));
+
+    try {
+      SchedulerUtils.checkResourceRequestAgainstAvailableResource(
+          requestedResource, availableResource);
+    } catch (InvalidResourceRequestException e) {
+      fail(String.format(
+          "Resource request should be accepted. Requested: %s, available: %s",
+          requestedResource, availableResource));
+    }
+  }
+
+  @Test
+  public void testCustomResourceRequestedUnitIsSameAsAvailableUnit() {
+    Resource requestedResource = ResourceTypesTestHelper.newResource(1, 1,
+        ImmutableMap.of("custom-resource-1", "11M"));
+
+    Resource availableResource = ResourceTypesTestHelper.newResource(1, 1,
+        ImmutableMap.of("custom-resource-1", "100M"));
+
+    try {
+      SchedulerUtils.checkResourceRequestAgainstAvailableResource(
+          requestedResource, availableResource);
+    } catch (InvalidResourceRequestException e) {
+      fail(String.format(
+          "Resource request should be accepted. Requested: %s, available: %s",
+          requestedResource, availableResource));
+    }
+  }
+
+  @Test
+  public void testCustomResourceRequestedUnitIsSameAsAvailableUnit2()
+      throws InvalidResourceRequestException {
+    Resource requestedResource = ResourceTypesTestHelper.newResource(1, 1,
+        ImmutableMap.of("custom-resource-1", "110M"));
+
+    Resource availableResource = ResourceTypesTestHelper.newResource(1, 1,
+        ImmutableMap.of("custom-resource-1", "100M"));
+
+    exception.expect(InvalidResourceRequestException.class);
+    exception.expectMessage(InvalidResourceRequestExceptionMessageGenerator
+        .create().withRequestedResourceType("custom-resource-1")
+        .withRequestedResource(requestedResource)
+        .withAvailableAllocation(availableResource)
+        .withMaxAllocation(configuredMaxAllocation).build());
+
+    SchedulerUtils.checkResourceRequestAgainstAvailableResource(
+        requestedResource, availableResource);
+  }
+
   public static void waitSchedulerApplicationAttemptStopped(
       AbstractYarnScheduler ys,
       ApplicationAttemptId attemptId) throws InterruptedException {
@@ -801,8 +988,7 @@ public class TestSchedulerUtils {
   public static SchedulerApplication<SchedulerApplicationAttempt>
       verifyAppAddedAndRemovedFromScheduler(
           Map<ApplicationId, SchedulerApplication<SchedulerApplicationAttempt>> applications,
-          EventHandler<SchedulerEvent> handler, String queueName)
-          throws Exception {
+          EventHandler<SchedulerEvent> handler, String queueName) {
 
     ApplicationId appId =
         ApplicationId.newInstance(System.currentTimeMillis(), 1);
@@ -832,4 +1018,60 @@ public class TestSchedulerUtils {
     when(rmContext.getNodeLabelManager()).thenReturn(nlm);
     return rmContext;
   }
+
+  private static class InvalidResourceRequestExceptionMessageGenerator {
+
+    private StringBuilder sb;
+    private Resource requestedResource;
+    private Resource availableAllocation;
+    private Resource configuredMaxAllowedAllocation;
+    private String resourceType;
+
+    InvalidResourceRequestExceptionMessageGenerator(StringBuilder sb) {
+      this.sb = sb;
+    }
+
+    public static InvalidResourceRequestExceptionMessageGenerator create() {
+      return new InvalidResourceRequestExceptionMessageGenerator(
+          new StringBuilder());
+    }
+
+    InvalidResourceRequestExceptionMessageGenerator withRequestedResource(
+            Resource r) {
+      this.requestedResource = r;
+      return this;
+    }
+
+    InvalidResourceRequestExceptionMessageGenerator withRequestedResourceType(
+            String rt) {
+      this.resourceType = rt;
+      return this;
+    }
+
+    InvalidResourceRequestExceptionMessageGenerator withAvailableAllocation(
+            Resource r) {
+      this.availableAllocation = r;
+      return this;
+    }
+
+    InvalidResourceRequestExceptionMessageGenerator withMaxAllocation(
+        Resource r) {
+      this.configuredMaxAllowedAllocation = r;
+      return this;
+    }
+
+    public String build() {
+      return sb
+          .append("Invalid resource request, requested resource type=[")
+          .append(resourceType).append("]")
+          .append(" < 0 or greater than maximum allowed allocation. ")
+          .append("Requested resource=").append(requestedResource).append(", ")
+          .append("maximum allowed allocation=").append(availableAllocation)
+          .append(", please note that maximum allowed allocation is calculated "
+              + "by scheduler based on maximum resource of " +
+                  "registered NodeManagers, which might be less than " +
+                  "configured maximum allocation=")
+          .append(configuredMaxAllowedAllocation).toString();
+    }
+  }
 }

Niektoré súbory nie sú zobrazené, pretože je v týchto rozdielových dátach zmenené mnoho súborov